Skip to content

Commit 75c855e

Browse files
committed
[core] Rename CompactFutureManager.isCompaction to compactNotCompleted
1 parent d866158 commit 75c855e

File tree

13 files changed

+21
-17
lines changed

13 files changed

+21
-17
lines changed

paimon-common/src/main/java/org/apache/paimon/CoreOptions.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -2539,7 +2539,7 @@ public boolean prepareCommitWaitCompaction() {
25392539
return options.get(LOOKUP_WAIT);
25402540
}
25412541

2542-
public boolean statefulLookup() {
2542+
public boolean laziedLookup() {
25432543
return needLookup()
25442544
&& (!options.get(LOOKUP_WAIT) || LookupCompactMode.GENTLE.equals(lookupCompact()));
25452545
}

paimon-core/src/main/java/org/apache/paimon/append/AppendOnlyWriter.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -210,9 +210,9 @@ public CommitIncrement prepareCommit(boolean waitCompaction) throws Exception {
210210
}
211211

212212
@Override
213-
public boolean isCompacting() {
213+
public boolean compactNotCompleted() {
214214
compactManager.triggerCompaction(false);
215-
return compactManager.isCompacting();
215+
return compactManager.compactNotCompleted();
216216
}
217217

218218
@VisibleForTesting

paimon-core/src/main/java/org/apache/paimon/compact/CompactFutureManager.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -40,7 +40,7 @@ public void cancelCompaction() {
4040
}
4141

4242
@Override
43-
public boolean isCompacting() {
43+
public boolean compactNotCompleted() {
4444
return taskFuture != null;
4545
}
4646

paimon-core/src/main/java/org/apache/paimon/compact/CompactManager.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -56,5 +56,5 @@ Optional<CompactResult> getCompactionResult(boolean blocking)
5656
* Check if a compaction is in progress, or if a compaction result remains to be fetched, or if
5757
* a compaction should be triggered later.
5858
*/
59-
boolean isCompacting();
59+
boolean compactNotCompleted();
6060
}

paimon-core/src/main/java/org/apache/paimon/compact/NoopCompactManager.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -71,7 +71,7 @@ public Optional<CompactResult> getCompactionResult(boolean blocking)
7171
public void cancelCompaction() {}
7272

7373
@Override
74-
public boolean isCompacting() {
74+
public boolean compactNotCompleted() {
7575
return false;
7676
}
7777

paimon-core/src/main/java/org/apache/paimon/mergetree/MergeTreeWriter.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -278,9 +278,9 @@ public CommitIncrement prepareCommit(boolean waitCompaction) throws Exception {
278278
}
279279

280280
@Override
281-
public boolean isCompacting() {
281+
public boolean compactNotCompleted() {
282282
compactManager.triggerCompaction(false);
283-
return compactManager.isCompacting();
283+
return compactManager.compactNotCompleted();
284284
}
285285

286286
@Override

paimon-core/src/main/java/org/apache/paimon/mergetree/compact/MergeTreeCompactManager.java

Lines changed: 5 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -244,8 +244,11 @@ public Optional<CompactResult> getCompactionResult(boolean blocking)
244244
}
245245

246246
@Override
247-
public boolean isCompacting() {
248-
return super.isCompacting() || (needLookup && !levels().level0().isEmpty());
247+
public boolean compactNotCompleted() {
248+
// If it is a lookup compaction, we should ensure that all level 0 files are consumed, so
249+
// here we need to make the outside think that we still need to do unfinished compact
250+
// working
251+
return super.compactNotCompleted() || (needLookup && !levels().level0().isEmpty());
249252
}
250253

251254
private void reportMetrics() {

paimon-core/src/main/java/org/apache/paimon/mergetree/compact/UniversalCompaction.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -78,7 +78,8 @@ public UniversalCompaction(
7878
this.opCompactionInterval =
7979
opCompactionInterval == null ? null : opCompactionInterval.toMillis();
8080
this.maxLookupCompactInterval = maxLookupCompactInterval;
81-
this.lookupCompactTriggerCount = new AtomicInteger(0);
81+
this.lookupCompactTriggerCount =
82+
maxLookupCompactInterval == null ? null : new AtomicInteger(0);
8283
}
8384

8485
@Override

paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreWrite.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -294,7 +294,7 @@ Function<WriterContainer<T>, Boolean> createConflictAwareWriterCleanChecker(
294294
// Condition 3: The writer has no postponed compaction like gentle lookup compaction.
295295
return writerContainer ->
296296
writerContainer.lastModifiedCommitIdentifier < latestCommittedIdentifier
297-
&& !writerContainer.writer.isCompacting();
297+
&& !writerContainer.writer.compactNotCompleted();
298298
}
299299

300300
protected static <T>

paimon-core/src/main/java/org/apache/paimon/postpone/PostponeBucketWriter.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -84,7 +84,7 @@ public CommitIncrement prepareCommit(boolean waitCompaction) throws Exception {
8484
}
8585

8686
@Override
87-
public boolean isCompacting() {
87+
public boolean compactNotCompleted() {
8888
return false;
8989
}
9090

0 commit comments

Comments
 (0)