Skip to content

Commit

Permalink
1
Browse files Browse the repository at this point in the history
  • Loading branch information
Zouxxyy committed Jan 19, 2024
1 parent 97d1314 commit a99234b
Show file tree
Hide file tree
Showing 5 changed files with 13 additions and 68 deletions.
11 changes: 7 additions & 4 deletions paimon-common/src/main/java/org/apache/paimon/CoreOptions.java
Original file line number Diff line number Diff line change
Expand Up @@ -275,7 +275,8 @@ public class CoreOptions implements Serializable {
.noDefaultValue()
.withDescription(
"If the maximum number of sort readers exceeds this value, a spill will be attempted. "
+ "This prevents too many readers from consuming too much memory and causing OOM.");
+ "This prevents too many readers from consuming too much memory and causing OOM,"
+ "the default value is 'num-sorted-run.compaction-trigger' + 1.");

public static final ConfigOption<MemorySize> SORT_SPILL_BUFFER_SIZE =
key("sort-spill-buffer-size")
Expand Down Expand Up @@ -383,7 +384,8 @@ public class CoreOptions implements Serializable {
.intType()
.noDefaultValue()
.withDescription(
"Total level number, for example, there are 3 levels, including 0,1,2 levels.");
"Total level number, for example, there are 3 levels, including 0,1,2 levels,"
+ " the default value is 'num-sorted-run.compaction-trigger' + 1.");

public static final ConfigOption<Boolean> COMMIT_FORCE_COMPACT =
key("commit.force-compact")
Expand Down Expand Up @@ -1200,8 +1202,9 @@ public SortEngine sortEngine() {
public int sortSpillThreshold() {
Integer maxSortedRunNum = options.get(SORT_SPILL_THRESHOLD);
if (maxSortedRunNum == null) {
int stopNum = numSortedRunStopTrigger();
maxSortedRunNum = Math.max(stopNum, stopNum + 1);
maxSortedRunNum = numSortedRunCompactionTrigger() + 1;
} else {
maxSortedRunNum = Math.max(numSortedRunStopTrigger(), maxSortedRunNum);
}
checkArgument(maxSortedRunNum > 1, "The sort spill threshold cannot be smaller than 2.");
return maxSortedRunNum;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,8 +41,8 @@ public interface CompactStrategy {
/** Pick a compaction unit consisting of all existing files. */
static Optional<CompactUnit> pickFullCompaction(int numLevels, List<LevelSortedRun> runs) {
int maxLevel = numLevels - 1;
if (runs.size() == 1 && runs.get(0).level() == maxLevel) {
// only 1 sorted run on the max level, nothing to compact
if (runs.isEmpty() || (runs.size() == 1 && runs.get(0).level() == maxLevel)) {
// no sorted run or only 1 sorted run on the max level, no need to compact
return Optional.empty();
} else {
return Optional.of(CompactUnit.fromLevelRuns(maxLevel, runs));
Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -43,8 +43,8 @@
import org.apache.paimon.mergetree.compact.CompactRewriter;
import org.apache.paimon.mergetree.compact.CompactStrategy;
import org.apache.paimon.mergetree.compact.FirstRowMergeTreeCompactRewriter;
import org.apache.paimon.mergetree.compact.ForceUpLevel0Compaction;
import org.apache.paimon.mergetree.compact.FullChangelogMergeTreeCompactRewriter;
import org.apache.paimon.mergetree.compact.LookupCompaction;
import org.apache.paimon.mergetree.compact.LookupMergeTreeCompactRewriter;
import org.apache.paimon.mergetree.compact.MergeFunctionFactory;
import org.apache.paimon.mergetree.compact.MergeTreeCompactManager;
Expand Down Expand Up @@ -160,7 +160,7 @@ protected MergeTreeWriter createWriter(
options.optimizedCompactionInterval());
CompactStrategy compactStrategy =
options.changelogProducer() == ChangelogProducer.LOOKUP
? new LookupCompaction(universalCompaction)
? new ForceUpLevel0Compaction(universalCompaction)
: universalCompaction;
CompactManager compactManager =
createCompactManager(partition, bucket, compactStrategy, compactExecutor, levels);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -266,7 +266,8 @@ public void testSizeRatioThreshold() {

@Test
public void testLookup() {
LookupCompaction compaction = new LookupCompaction(new UniversalCompaction(25, 1, 3));
ForceUpLevel0Compaction compaction =
new ForceUpLevel0Compaction(new UniversalCompaction(25, 1, 3));

// level 0 to max level
Optional<CompactUnit> pick = compaction.pick(3, level0(1, 2, 2, 2));
Expand Down

0 comments on commit a99234b

Please sign in to comment.