Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
18 changes: 18 additions & 0 deletions docs/layouts/shortcodes/generated/core_configuration.html
Original file line number Diff line number Diff line change
Expand Up @@ -146,6 +146,24 @@
<td>String</td>
<td>Specifies the commit user prefix.</td>
</tr>
<tr>
<td><h5>compaction.lookup-delay.l0-file-trigger</h5></td>
<td style="word-wrap: break-word;">5</td>
<td>Integer</td>
<td>The L0 file trigger for a delayed lookup compaction. A delayed lookup compaction will only be performed when L0 files reach this config value.</td>
</tr>
<tr>
<td><h5>compaction.lookup-delay.partition-threshold</h5></td>
<td style="word-wrap: break-word;">(none)</td>
<td>Duration</td>
<td>The threshold of partitioned pk table to perform delayed lookup compaction. Delayed lookup compaction can be configured with a less frequent compaction strategy to sacrifice timeliness for overall resource usage saving. This option is only valid for a partitioned pk table when CoreOptions#needLookup() is true.</td>
</tr>
<tr>
<td><h5>compaction.lookup-delay.stop-trigger</h5></td>
<td style="word-wrap: break-word;">10</td>
<td>Integer</td>
<td>The stop trigger for a delayed lookup compaction. For every stop trigger, a forced lookup compaction will be performed to flush L0 files to higher level.</td>
</tr>
<tr>
<td><h5>compaction.max-size-amplification-percent</h5></td>
<td style="word-wrap: break-word;">200</td>
Expand Down
40 changes: 40 additions & 0 deletions paimon-common/src/main/java/org/apache/paimon/CoreOptions.java
Original file line number Diff line number Diff line change
Expand Up @@ -627,6 +627,33 @@ public class CoreOptions implements Serializable {
text("Default value of Bucketed Append Table is '5'."))
.build());

public static final ConfigOption<Duration> LOOKUP_DELAY_PARTITION_THRESHOLD =
key("compaction.lookup-delay.partition-threshold")
.durationType()
.noDefaultValue()
.withDescription(
"The threshold of partitioned pk table to perform delayed lookup compaction. "
+ "Delayed lookup compaction can be configured with a less frequent "
+ "compaction strategy to sacrifice timeliness for overall resource usage saving. "
+ "This option is only valid for a partitioned pk table when "
+ "CoreOptions#needLookup() is true.");

public static final ConfigOption<Integer> LOOKUP_DELAY_L0_FILE_TRIGGER =
key("compaction.lookup-delay.l0-file-trigger")
.intType()
.defaultValue(5)
.withDescription(
"The L0 file trigger for a delayed lookup compaction. A delayed lookup compaction "
+ "will only be performed when L0 files reach this config value.");

public static final ConfigOption<Integer> LOOKUP_DELAY_STOP_TRIGGER =
key("compaction.lookup-delay.stop-trigger")
.intType()
.defaultValue(LOOKUP_DELAY_L0_FILE_TRIGGER.defaultValue() * 2)
.withDescription(
"The stop trigger for a delayed lookup compaction. For every stop trigger, "
+ "a forced lookup compaction will be performed to flush L0 files to higher level.");

public static final ConfigOption<ChangelogProducer> CHANGELOG_PRODUCER =
key("changelog-producer")
.enumType(ChangelogProducer.class)
Expand Down Expand Up @@ -2043,6 +2070,19 @@ public Duration optimizedCompactionInterval() {
return options.get(COMPACTION_OPTIMIZATION_INTERVAL);
}

@Nullable
public Duration lookupDelayPartitionThreshold() {
return options.get(LOOKUP_DELAY_PARTITION_THRESHOLD);
}

public int lookupDelayL0FileTrigger() {
return options.get(LOOKUP_DELAY_L0_FILE_TRIGGER);
}

public int lookupDelayStopTrigger() {
return options.get(LOOKUP_DELAY_STOP_TRIGGER);
}

public int numSortedRunStopTrigger() {
Integer stopTrigger = options.get(NUM_SORTED_RUNS_STOP_TRIGGER);
if (stopTrigger == null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -215,6 +215,11 @@ public boolean isCompacting() {
return compactManager.isCompacting();
}

@Override
public boolean hasDelayedCompact() {
return false;
}

@VisibleForTesting
void flush(boolean waitForLatestCompaction, boolean forcedFullCompaction) throws Exception {
List<DataFileMeta> flushedFiles = sinkWriter.flush();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -186,6 +186,11 @@ public Optional<CompactResult> getCompactionResult(boolean blocking)
return result;
}

@Override
public boolean hasDelayedCompact() {
return false;
}

@VisibleForTesting
Optional<List<DataFileMeta>> pickCompactBefore() {
if (toCompact.isEmpty()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,4 +54,9 @@ Optional<CompactResult> getCompactionResult(boolean blocking)

/** Check if a compaction is in progress, or if a compaction result remains to be fetched. */
boolean isCompacting();

/**
* Check if the {@code CompactManager} has delayed compacts need to be triggered in the future.
*/
boolean hasDelayedCompact();
}
Original file line number Diff line number Diff line change
Expand Up @@ -75,6 +75,11 @@ public boolean isCompacting() {
return false;
}

@Override
public boolean hasDelayedCompact() {
return false;
}

@Override
public void close() throws IOException {}
}
Original file line number Diff line number Diff line change
Expand Up @@ -142,7 +142,7 @@ private long newSequenceNumber() {
}

@VisibleForTesting
CompactManager compactManager() {
public CompactManager compactManager() {
return compactManager;
}

Expand Down Expand Up @@ -283,6 +283,11 @@ public boolean isCompacting() {
return compactManager.isCompacting();
}

@Override
public boolean hasDelayedCompact() {
return compactManager.hasDelayedCompact();
}

@Override
public void sync() throws Exception {
trySyncLatestCompaction(true);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,145 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.paimon.mergetree.compact;

import org.apache.paimon.annotation.VisibleForTesting;
import org.apache.paimon.data.InternalRow;
import org.apache.paimon.deletionvectors.DeletionVectorsMaintainer;
import org.apache.paimon.mergetree.Levels;
import org.apache.paimon.operation.metrics.CompactionMetrics;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import javax.annotation.Nullable;

import java.time.Duration;
import java.time.LocalDateTime;
import java.time.format.DateTimeFormatter;
import java.util.Comparator;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.atomic.AtomicInteger;

/** Compact manager to perform the delayed lookup compaction. */
public class DelayedLookupCompactManager extends MergeTreeCompactManager {

private static final Logger LOG = LoggerFactory.getLogger(DelayedLookupCompactManager.class);
private final CompactStrategy strategy;
private final LocalDateTime currentPartitionDate;
private final Duration delayPartitionThreshold;
private final int l0FileTriggerThreshold;
private final int stopTriggerThreshold;

private final AtomicInteger triggerCount = new AtomicInteger(0);

public DelayedLookupCompactManager(
ExecutorService executor,
Levels levels,
CompactStrategy strategy,
Comparator<InternalRow> keyComparator,
long compactionFileSize,
int numSortedRunStopTrigger,
CompactRewriter rewriter,
@Nullable CompactionMetrics.Reporter metricsReporter,
@Nullable DeletionVectorsMaintainer dvMaintainer,
boolean lazyGenDeletionFile,
LocalDateTime currentPartitionDate,
Duration delayPartitionThreshold,
int l0FileTriggerThreshold,
int stopTriggerThreshold) {
super(
executor,
levels,
strategy,
keyComparator,
compactionFileSize,
numSortedRunStopTrigger,
rewriter,
metricsReporter,
dvMaintainer,
lazyGenDeletionFile);
this.strategy = strategy;
this.currentPartitionDate = currentPartitionDate;
this.delayPartitionThreshold = delayPartitionThreshold;
this.l0FileTriggerThreshold = l0FileTriggerThreshold;
this.stopTriggerThreshold = stopTriggerThreshold;
}

@Override
public void triggerCompaction(boolean fullCompaction) {
if (isDelayedCompactPartition()) {
if (shouldTriggerDelayedLookupCompact()) {
triggerCount.set(0);
super.triggerCompaction(fullCompaction);
} else {
LOG.info(
"Skip to trigger this lookup compaction because the compaction trigger count {} has not reached "
+ "the stop trigger threshold {} or l0 file size {} is less than l0 file trigger threshold {}.",
triggerCount.get(),
stopTriggerThreshold,
level0Files(),
l0FileTriggerThreshold);
triggerCount.incrementAndGet();
}
} else {
super.triggerCompaction(fullCompaction);
}
}

@Override
public boolean hasDelayedCompact() {
return strategy instanceof ForceUpLevel0Compaction && level0Files() > 0;
}

@VisibleForTesting
public boolean shouldTriggerDelayedLookupCompact() {
if (level0Files() >= l0FileTriggerThreshold) {
LOG.info("Trigger delayed lookup compact for L0 file count: {}", level0Files());
return true;
} else if (triggerCount.get() >= stopTriggerThreshold) {
LOG.info("Trigger delayed lookup compact for stopTrigger count: {}", triggerCount);
return true;
} else {
return false;
}
}

@VisibleForTesting
public boolean isDelayedCompactPartition() {
if (delayPartitionThreshold == null) {
return false;
}
LocalDateTime delayedCompactPartition = LocalDateTime.now().minus(delayPartitionThreshold);
// For delayedCompactPartitionThreshold=20250120, any data insert into partitions<=20250120
// should be delayed compact.
boolean result = !currentPartitionDate.isAfter(delayedCompactPartition);
if (LOG.isDebugEnabled()) {
LOG.debug(
"Current partition Date: {}, delayed compact partition threshold: {}, delayed compact partition result: {}.",
currentPartitionDate.format(DateTimeFormatter.BASIC_ISO_DATE),
delayedCompactPartition.format(DateTimeFormatter.BASIC_ISO_DATE),
result);
}
return result;
}

private int level0Files() {
return levels().level0().size();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -240,6 +240,11 @@ public Optional<CompactResult> getCompactionResult(boolean blocking)
return result;
}

@Override
public boolean hasDelayedCompact() {
return false;
}

private void reportLevel0FileCount() {
if (metricsReporter != null) {
metricsReporter.reportLevel0FileCount(levels.level0().size());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -290,9 +290,12 @@ Function<WriterContainer<T>, Boolean> createConflictAwareWriterCleanChecker(
//
// Condition 2: No compaction is in progress. That is, no more changelog will be
// produced.
//
// Condition 3: The writer has no delayed compaction.
return writerContainer ->
writerContainer.lastModifiedCommitIdentifier < latestCommittedIdentifier
&& !writerContainer.writer.isCompacting();
&& !writerContainer.writer.isCompacting()
&& !writerContainer.writer.hasDelayedCompact();
}

protected static <T>
Expand Down Expand Up @@ -397,7 +400,7 @@ public Map<BinaryRow, List<Integer>> getActiveBuckets() {
return result;
}

protected WriterContainer<T> getWriterWrapper(BinaryRow partition, int bucket) {
public WriterContainer<T> getWriterWrapper(BinaryRow partition, int bucket) {
Map<Integer, WriterContainer<T>> buckets = writers.get(partition);
if (buckets == null) {
buckets = new HashMap<>();
Expand Down Expand Up @@ -552,7 +555,7 @@ protected WriterContainer(
}

@VisibleForTesting
Map<BinaryRow, Map<Integer, WriterContainer<T>>> writers() {
public Map<BinaryRow, Map<Integer, WriterContainer<T>>> writers() {
return writers;
}
}
Loading
Loading