From a33bba3b99d5210784ad7a4dae44834c069cc5ef Mon Sep 17 00:00:00 2001 From: luoyuxia Date: Tue, 16 Dec 2025 09:50:48 +0800 Subject: [PATCH 1/5] [lake] support lake tiering service force to commit due to timeout --- .../apache/fluss/config/ConfigOptions.java | 17 ++ .../flink/tiering/LakeTieringJobBuilder.java | 13 + .../event/TieringReachMaxDurationEvent.java | 58 ++++ .../flink/tiering/source/TieringSource.java | 56 +++- .../source/TieringSourceFetcherManager.java | 87 ++++++ .../tiering/source/TieringSourceReader.java | 43 ++- .../tiering/source/TieringSplitReader.java | 171 ++++++++++-- .../enumerator/TieringSourceEnumerator.java | 96 ++++++- .../tiering/source/split/TieringLogSplit.java | 27 +- .../source/split/TieringSnapshotSplit.java | 28 +- .../tiering/source/split/TieringSplit.java | 14 +- .../source/split/TieringSplitGenerator.java | 11 +- .../source/split/TieringSplitSerializer.java | 9 +- .../fluss/flink/tiering/TieringITCase.java | 258 +++++++++++++++++ .../source/TieringSourceReaderTest.java | 203 ++++++++++++++ .../TieringSourceEnumeratorTest.java | 262 ++++++++++++------ .../split/TieringSplitSerializerTest.java | 52 +++- .../lakehouse/TestingPaimonStoragePlugin.java | 171 +++++++++++- fluss-test-coverage/pom.xml | 1 + 19 files changed, 1428 insertions(+), 149 deletions(-) create mode 100644 fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/event/TieringReachMaxDurationEvent.java create mode 100644 fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSourceFetcherManager.java create mode 100644 fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/TieringITCase.java create mode 100644 fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/TieringSourceReaderTest.java diff --git a/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java b/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java index 7e04c3a1cc..12aa7c32aa 100644 --- a/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java +++ b/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java @@ -1851,6 +1851,23 @@ public class ConfigOptions { + ConfigOptions.TABLE_DATALAKE_AUTO_EXPIRE_SNAPSHOT + " is false."); + public static final ConfigOption LAKE_TIERING_TABLE_DURATION_MAX = + key("lake.tiering.table.duration.max") + .durationType() + .defaultValue(Duration.ofMinutes(30)) + .withDescription( + "The maximum duration for tiering a single table. If tiering a table exceeds this duration, " + + "it will be force completed: the tiering will be finalized and committed to the data lake " + + "(e.g., Paimon) immediately, even if they haven't reached their desired stopping offsets."); + + public static final ConfigOption LAKE_TIERING_TABLE_DURATION_DETECT_INTERVAL = + key("lake.tiering.table.duration.detect-interval") + .durationType() + .defaultValue(Duration.ofSeconds(30)) + .withDescription( + "The interval to check if a table tiering operation has reached the maximum duration. " + + "The enumerator will periodically check tiering tables and force complete those that exceed the maximum duration."); + // ------------------------------------------------------------------------ // ConfigOptions for fluss kafka // ------------------------------------------------------------------------ diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/LakeTieringJobBuilder.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/LakeTieringJobBuilder.java index 74cf91e0e0..70774c8a60 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/LakeTieringJobBuilder.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/LakeTieringJobBuilder.java @@ -34,6 +34,8 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; +import static org.apache.fluss.config.ConfigOptions.LAKE_TIERING_TABLE_DURATION_DETECT_INTERVAL; +import static org.apache.fluss.config.ConfigOptions.LAKE_TIERING_TABLE_DURATION_MAX; import static org.apache.fluss.flink.tiering.source.TieringSource.TIERING_SOURCE_TRANSFORMATION_UID; import static org.apache.fluss.flink.tiering.source.TieringSourceOptions.POLL_TIERING_TABLE_INTERVAL; import static org.apache.fluss.utils.Preconditions.checkNotNull; @@ -89,6 +91,17 @@ public JobClient build() throws Exception { tieringSourceBuilder.withPollTieringTableIntervalMs( flussConfig.get(POLL_TIERING_TABLE_INTERVAL).toMillis()); } + + if (lakeTieringConfig.get(LAKE_TIERING_TABLE_DURATION_MAX) != null) { + tieringSourceBuilder.withTieringTableDurationMax( + lakeTieringConfig.get(LAKE_TIERING_TABLE_DURATION_MAX).toMillis()); + } + + if (lakeTieringConfig.get(LAKE_TIERING_TABLE_DURATION_DETECT_INTERVAL) != null) { + tieringSourceBuilder.withTieringTableDurationDetectInterval( + lakeTieringConfig.get(LAKE_TIERING_TABLE_DURATION_DETECT_INTERVAL).toMillis()); + } + TieringSource tieringSource = tieringSourceBuilder.build(); DataStreamSource source = env.fromSource( diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/event/TieringReachMaxDurationEvent.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/event/TieringReachMaxDurationEvent.java new file mode 100644 index 0000000000..dd3f32e262 --- /dev/null +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/event/TieringReachMaxDurationEvent.java @@ -0,0 +1,58 @@ +/* + * 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.fluss.flink.tiering.event; + +import org.apache.flink.api.connector.source.SourceEvent; + +import java.util.Objects; + +/** + * SourceEvent used to notify TieringSourceReader that a table has reached the maximum tiering + * duration and should be force completed. + */ +public class TieringReachMaxDurationEvent implements SourceEvent { + + private static final long serialVersionUID = 1L; + + private final long tableId; + + public TieringReachMaxDurationEvent(long tableId) { + this.tableId = tableId; + } + + public long getTableId() { + return tableId; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof TieringReachMaxDurationEvent)) { + return false; + } + TieringReachMaxDurationEvent that = (TieringReachMaxDurationEvent) o; + return tableId == that.tableId; + } + + @Override + public int hashCode() { + return Objects.hashCode(tableId); + } +} diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSource.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSource.java index b2d8c28dc9..5d637c9071 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSource.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSource.java @@ -36,12 +36,16 @@ import org.apache.flink.api.connector.source.SourceReaderContext; import org.apache.flink.api.connector.source.SplitEnumerator; import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.streaming.api.graph.StreamGraphHasherV2; import java.nio.charset.StandardCharsets; +import static org.apache.fluss.config.ConfigOptions.LAKE_TIERING_TABLE_DURATION_DETECT_INTERVAL; +import static org.apache.fluss.config.ConfigOptions.LAKE_TIERING_TABLE_DURATION_MAX; import static org.apache.fluss.flink.tiering.source.TieringSourceOptions.POLL_TIERING_TABLE_INTERVAL; /** @@ -61,14 +65,20 @@ public class TieringSource private final Configuration flussConf; private final LakeTieringFactory lakeTieringFactory; private final long pollTieringTableIntervalMs; + private final long tieringTableDurationMaxMs; + private final long tieringTableDurationDetectIntervalMs; public TieringSource( Configuration flussConf, LakeTieringFactory lakeTieringFactory, - long pollTieringTableIntervalMs) { + long pollTieringTableIntervalMs, + long tieringTableDurationMaxMs, + long tieringTableDurationDetectIntervalMs) { this.flussConf = flussConf; this.lakeTieringFactory = lakeTieringFactory; this.pollTieringTableIntervalMs = pollTieringTableIntervalMs; + this.tieringTableDurationMaxMs = tieringTableDurationMaxMs; + this.tieringTableDurationDetectIntervalMs = tieringTableDurationDetectIntervalMs; } @Override @@ -78,19 +88,26 @@ public Boundedness getBoundedness() { @Override public SplitEnumerator createEnumerator( - SplitEnumeratorContext splitEnumeratorContext) throws Exception { + SplitEnumeratorContext splitEnumeratorContext) { return new TieringSourceEnumerator( - flussConf, splitEnumeratorContext, pollTieringTableIntervalMs); + flussConf, + splitEnumeratorContext, + pollTieringTableIntervalMs, + tieringTableDurationMaxMs, + tieringTableDurationDetectIntervalMs); } @Override public SplitEnumerator restoreEnumerator( SplitEnumeratorContext splitEnumeratorContext, - TieringSourceEnumeratorState tieringSourceEnumeratorState) - throws Exception { + TieringSourceEnumeratorState tieringSourceEnumeratorState) { // stateless operator return new TieringSourceEnumerator( - flussConf, splitEnumeratorContext, pollTieringTableIntervalMs); + flussConf, + splitEnumeratorContext, + pollTieringTableIntervalMs, + tieringTableDurationMaxMs, + tieringTableDurationDetectIntervalMs); } @Override @@ -107,8 +124,11 @@ public SimpleVersionedSerializer getSplitSerializer() { @Override public SourceReader, TieringSplit> createReader( SourceReaderContext sourceReaderContext) { + FutureCompletingBlockingQueue>> + elementsQueue = new FutureCompletingBlockingQueue<>(); Connection connection = ConnectionFactory.createConnection(flussConf); - return new TieringSourceReader<>(sourceReaderContext, connection, lakeTieringFactory); + return new TieringSourceReader<>( + elementsQueue, sourceReaderContext, connection, lakeTieringFactory); } /** This follows the operator uid hash generation logic of flink {@link StreamGraphHasherV2}. */ @@ -126,6 +146,10 @@ public static class Builder { private final LakeTieringFactory lakeTieringFactory; private long pollTieringTableIntervalMs = POLL_TIERING_TABLE_INTERVAL.defaultValue().toMillis(); + private long tieringTableDurationMaxMs = + LAKE_TIERING_TABLE_DURATION_MAX.defaultValue().toMillis(); + private long tieringTableDurationDetectIntervalMs = + LAKE_TIERING_TABLE_DURATION_DETECT_INTERVAL.defaultValue().toMillis(); public Builder( Configuration flussConf, LakeTieringFactory lakeTieringFactory) { @@ -138,8 +162,24 @@ public Builder withPollTieringTableIntervalMs(long pollTieringTable return this; } + public Builder withTieringTableDurationMax(long tieringTableDurationMaxMs) { + this.tieringTableDurationMaxMs = tieringTableDurationMaxMs; + return this; + } + + public Builder withTieringTableDurationDetectInterval( + long tieringTableDurationDetectIntervalMs) { + this.tieringTableDurationDetectIntervalMs = tieringTableDurationDetectIntervalMs; + return this; + } + public TieringSource build() { - return new TieringSource<>(flussConf, lakeTieringFactory, pollTieringTableIntervalMs); + return new TieringSource<>( + flussConf, + lakeTieringFactory, + pollTieringTableIntervalMs, + tieringTableDurationMaxMs, + tieringTableDurationDetectIntervalMs); } } } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSourceFetcherManager.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSourceFetcherManager.java new file mode 100644 index 0000000000..68db87671f --- /dev/null +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSourceFetcherManager.java @@ -0,0 +1,87 @@ +/* + * 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.fluss.flink.tiering.source; + +import org.apache.fluss.flink.adapter.SingleThreadFetcherManagerAdapter; +import org.apache.fluss.flink.tiering.source.split.TieringSplit; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher; +import org.apache.flink.connector.base.source.reader.fetcher.SplitFetcherTask; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; + +import java.util.Collection; +import java.util.function.Consumer; +import java.util.function.Supplier; + +/** + * The SplitFetcherManager for tiering source. This class is needed to help notify a table reaches + * to deadline of tiering to {@link TieringSplitReader}. + */ +public class TieringSourceFetcherManager + extends SingleThreadFetcherManagerAdapter< + TableBucketWriteResult, TieringSplit> { + + public TieringSourceFetcherManager( + FutureCompletingBlockingQueue>> + elementsQueue, + Supplier, TieringSplit>> + splitReaderSupplier, + Configuration configuration, + Consumer> splitFinishedHook) { + super(elementsQueue, splitReaderSupplier, configuration, splitFinishedHook); + } + + public void markTableReachTieringMaxDuration(long tableId) { + if (!fetchers.isEmpty()) { + // The fetcher thread is still running. This should be the majority of the cases. + fetchers.values() + .forEach( + splitFetcher -> + enqueueMarkTableReachTieringMaxDurationTask( + splitFetcher, tableId)); + } else { + SplitFetcher, TieringSplit> splitFetcher = + createSplitFetcher(); + enqueueMarkTableReachTieringMaxDurationTask(splitFetcher, tableId); + startFetcher(splitFetcher); + } + } + + private void enqueueMarkTableReachTieringMaxDurationTask( + SplitFetcher, TieringSplit> splitFetcher, + long reachTieringDeadlineTable) { + splitFetcher.enqueueTask( + new SplitFetcherTask() { + @Override + public boolean run() { + ((TieringSplitReader) splitFetcher.getSplitReader()) + .handleTableReachTieringMaxDuration(reachTieringDeadlineTable); + return true; + } + + @Override + public void wakeUp() { + // do nothing + } + }); + } +} diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSourceReader.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSourceReader.java index a6b14b320d..63e73bab97 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSourceReader.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSourceReader.java @@ -18,23 +18,31 @@ package org.apache.fluss.flink.tiering.source; import org.apache.fluss.annotation.Internal; +import org.apache.fluss.annotation.VisibleForTesting; import org.apache.fluss.client.Connection; +import org.apache.fluss.flink.adapter.SingleThreadMultiplexSourceReaderBaseAdapter; +import org.apache.fluss.flink.tiering.event.TieringReachMaxDurationEvent; import org.apache.fluss.flink.tiering.source.split.TieringSplit; import org.apache.fluss.flink.tiering.source.state.TieringSplitState; import org.apache.fluss.lake.writer.LakeTieringFactory; +import org.apache.flink.api.connector.source.SourceEvent; import org.apache.flink.api.connector.source.SourceReader; import org.apache.flink.api.connector.source.SourceReaderContext; -import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import java.time.Duration; import java.util.Collections; import java.util.List; import java.util.Map; +import static org.apache.fluss.flink.tiering.source.TieringSplitReader.DEFAULT_POLL_TIMEOUT; + /** A {@link SourceReader} that read records from Fluss and write to lake. */ @Internal public final class TieringSourceReader - extends SingleThreadMultiplexSourceReaderBase< + extends SingleThreadMultiplexSourceReaderBaseAdapter< TableBucketWriteResult, TableBucketWriteResult, TieringSplit, @@ -43,11 +51,29 @@ public final class TieringSourceReader private final Connection connection; public TieringSourceReader( + FutureCompletingBlockingQueue>> + elementsQueue, SourceReaderContext context, Connection connection, LakeTieringFactory lakeTieringFactory) { + this(elementsQueue, context, connection, lakeTieringFactory, DEFAULT_POLL_TIMEOUT); + } + + @VisibleForTesting + TieringSourceReader( + FutureCompletingBlockingQueue>> + elementsQueue, + SourceReaderContext context, + Connection connection, + LakeTieringFactory lakeTieringFactory, + Duration pollTimeout) { super( - () -> new TieringSplitReader<>(connection, lakeTieringFactory), + elementsQueue, + new TieringSourceFetcherManager<>( + elementsQueue, + () -> new TieringSplitReader<>(connection, lakeTieringFactory, pollTimeout), + context.getConfiguration(), + (ignore) -> {}), new TableBucketWriteResultEmitter<>(), context.getConfiguration(), context); @@ -89,6 +115,17 @@ protected TieringSplit toSplitType(String splitId, TieringSplitState splitState) return splitState.toSourceSplit(); } + @Override + public void handleSourceEvents(SourceEvent sourceEvent) { + if (sourceEvent instanceof TieringReachMaxDurationEvent) { + TieringReachMaxDurationEvent reachMaxDurationEvent = + (TieringReachMaxDurationEvent) sourceEvent; + long tableId = reachMaxDurationEvent.getTableId(); + ((TieringSourceFetcherManager) splitFetcherManager) + .markTableReachTieringMaxDuration(tableId); + } + } + @Override public void close() throws Exception { super.close(); diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java index b9fe79e3d3..2a797add33 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java @@ -57,6 +57,7 @@ import static org.apache.fluss.utils.Preconditions.checkArgument; import static org.apache.fluss.utils.Preconditions.checkNotNull; +import static org.apache.fluss.utils.Preconditions.checkState; /** The {@link SplitReader} implementation which will read Fluss and write to lake. */ public class TieringSplitReader @@ -64,18 +65,25 @@ public class TieringSplitReader private static final Logger LOG = LoggerFactory.getLogger(TieringSplitReader.class); - private static final Duration POLL_TIMEOUT = Duration.ofMillis(10000L); + public static final Duration DEFAULT_POLL_TIMEOUT = Duration.ofMillis(10_000L); // unknown bucket timestamp for empty split or snapshot split private static final long UNKNOWN_BUCKET_TIMESTAMP = -1; + // unknown bucket offset for empty split or snapshot split + private static final long UNKNOWN_BUCKET_OFFSET = -1; + private final LakeTieringFactory lakeTieringFactory; + private final Duration pollTimeout; + // the id for the pending tables to be tiered private final Queue pendingTieringTables; // the table_id to the pending splits private final Map> pendingTieringSplits; + private final Set reachTieringMaxDurationTables; + private final Map> lakeWriters; private final Connection connection; @@ -92,38 +100,54 @@ public class TieringSplitReader // map from table bucket to split id private final Map currentTableSplitsByBucket; private final Map currentTableStoppingOffsets; - private final Set currentTableEmptyLogSplits; + + private final Map currentTableTieredOffsetAndTimestamp; + + private final Set currentEmptySplits; public TieringSplitReader( Connection connection, LakeTieringFactory lakeTieringFactory) { + this(connection, lakeTieringFactory, DEFAULT_POLL_TIMEOUT); + } + + protected TieringSplitReader( + Connection connection, + LakeTieringFactory lakeTieringFactory, + Duration pollTimeout) { this.lakeTieringFactory = lakeTieringFactory; // owned by TieringSourceReader this.connection = connection; this.pendingTieringTables = new ArrayDeque<>(); this.pendingTieringSplits = new HashMap<>(); this.currentTableStoppingOffsets = new HashMap<>(); - this.currentTableEmptyLogSplits = new HashSet<>(); + this.currentTableTieredOffsetAndTimestamp = new HashMap<>(); + this.currentEmptySplits = new HashSet<>(); this.currentTableSplitsByBucket = new HashMap<>(); this.lakeWriters = new HashMap<>(); this.currentPendingSnapshotSplits = new ArrayDeque<>(); + this.reachTieringMaxDurationTables = new HashSet<>(); + this.pollTimeout = pollTimeout; } @Override public RecordsWithSplitIds> fetch() throws IOException { // check empty splits - if (!currentTableEmptyLogSplits.isEmpty()) { - LOG.info("Empty split(s) {} finished.", currentTableEmptyLogSplits); - TableBucketWriteResultWithSplitIds records = forEmptySplits(currentTableEmptyLogSplits); - currentTableEmptyLogSplits.forEach( + if (!currentEmptySplits.isEmpty()) { + LOG.info("Empty split(s) {} finished.", currentEmptySplits); + TableBucketWriteResultWithSplitIds records = forEmptySplits(currentEmptySplits); + currentEmptySplits.forEach( split -> currentTableSplitsByBucket.remove(split.getTableBucket())); mayFinishCurrentTable(); - currentTableEmptyLogSplits.clear(); + currentEmptySplits.clear(); return records; } checkSplitOrStartNext(); // may read snapshot firstly if (currentSnapshotSplitReader != null) { + // for snapshot split, we don't force to complete it + // since we rely on the log offset for the snapshot to + // do next tiering, if force to complete, we can't get the log offset CloseableIterator recordIterator = currentSnapshotSplitReader.readBatch(); if (recordIterator == null) { LOG.info("Split {} is finished", currentSnapshotSplit.splitId()); @@ -134,7 +158,11 @@ public RecordsWithSplitIds> fetch() throws I } } else { if (currentLogScanner != null) { - ScanRecords scanRecords = currentLogScanner.poll(POLL_TIMEOUT); + // force to complete records + if (reachTieringMaxDurationTables.contains(currentTableId)) { + return forceCompleteTieringLogRecords(); + } + ScanRecords scanRecords = currentLogScanner.poll(pollTimeout); return forLogRecords(scanRecords); } else { return emptyTableBucketWriteResultWithSplitIds(); @@ -152,6 +180,15 @@ public void handleSplitsChanges(SplitsChange splitsChange) { } for (TieringSplit split : splitsChange.splits()) { LOG.info("add split {}", split.splitId()); + if (split.isForceIgnore()) { + // if the split is forced to ignore, + // mark it as empty + LOG.info( + "ignore split {} since the split is set to force to ignore", + split.splitId()); + currentEmptySplits.add(split); + continue; + } long tableId = split.getTableBucket().getTableId(); // the split belongs to the current table if (currentTableId != null && currentTableId == tableId) { @@ -248,6 +285,58 @@ private void mayCreateLogScanner() { } } + private RecordsWithSplitIds> + forceCompleteTieringLogRecords() throws IOException { + Map> writeResults = new HashMap<>(); + Map finishedSplitIds = new HashMap<>(); + + // force finish all splits + Iterator> currentTieringSplitsIterator = + currentTableSplitsByBucket.entrySet().iterator(); + while (currentTieringSplitsIterator.hasNext()) { + Map.Entry entry = currentTieringSplitsIterator.next(); + TableBucket bucket = entry.getKey(); + TieringSplit split = entry.getValue(); + if (split != null && split.isTieringLogSplit()) { + // get the current offset, timestamp that tiered so far + LogOffsetAndTimestamp logOffsetAndTimestamp = + currentTableTieredOffsetAndTimestamp.get(bucket); + long logEndOffset = + logOffsetAndTimestamp == null + ? UNKNOWN_BUCKET_OFFSET + // logEndOffset is equal to offset tiered + 1 + : logOffsetAndTimestamp.logOffset + 1; + long timestamp = + logOffsetAndTimestamp == null + ? UNKNOWN_BUCKET_TIMESTAMP + : logOffsetAndTimestamp.timestamp; + TableBucketWriteResult bucketWriteResult = + completeLakeWriter( + bucket, split.getPartitionName(), logEndOffset, timestamp); + + if (logEndOffset == UNKNOWN_BUCKET_OFFSET) { + // when the log end offset is unknown, the write result must be + // null, otherwise, we should throw exception directly to avoid data + // inconsistent + checkState( + bucketWriteResult.writeResult() == null, + "bucketWriteResult must be null when log end offset is unknown when tiering " + + split); + } + + writeResults.put(bucket, bucketWriteResult); + finishedSplitIds.put(bucket, split.splitId()); + LOG.info( + "Split {} is forced to be finished due to tiering reach max duration.", + split.splitId()); + currentTieringSplitsIterator.remove(); + } + } + reachTieringMaxDurationTables.remove(this.currentTableId); + mayFinishCurrentTable(); + return new TableBucketWriteResultWithSplitIds(writeResults, finishedSplitIds); + } + private RecordsWithSplitIds> forLogRecords( ScanRecords scanRecords) throws IOException { Map> writeResults = new HashMap<>(); @@ -272,6 +361,9 @@ private RecordsWithSplitIds> forLogRecords( } } ScanRecord lastRecord = bucketScanRecords.get(bucketScanRecords.size() - 1); + currentTableTieredOffsetAndTimestamp.put( + bucket, + new LogOffsetAndTimestamp(lastRecord.logOffset(), lastRecord.timestamp())); // has arrived into the end of the split, if (lastRecord.logOffset() >= stoppingOffset - 1) { currentTableStoppingOffsets.remove(bucket); @@ -293,7 +385,11 @@ private RecordsWithSplitIds> forLogRecords( lastRecord.timestamp())); // put split of the bucket finishedSplitIds.put(bucket, currentSplitId); - LOG.info("Split {} has been finished.", currentSplitId); + LOG.info( + "Finish tier bucket {} for table {}, split: {}.", + bucket, + currentTablePath, + currentSplitId); } } @@ -327,8 +423,11 @@ private TableBucketWriteResult completeLakeWriter( long maxTimestamp) throws IOException { LakeWriter lakeWriter = lakeWriters.remove(bucket); - WriteResult writeResult = lakeWriter.complete(); - lakeWriter.close(); + WriteResult writeResult = null; + if (lakeWriter != null) { + writeResult = lakeWriter.complete(); + lakeWriter.close(); + } return toTableBucketWriteResult( currentTablePath, bucket, @@ -339,22 +438,22 @@ private TableBucketWriteResult completeLakeWriter( checkNotNull(currentTableNumberOfSplits)); } - private TableBucketWriteResultWithSplitIds forEmptySplits(Set emptySplits) { + private TableBucketWriteResultWithSplitIds forEmptySplits(Set emptySplits) { Map> writeResults = new HashMap<>(); Map finishedSplitIds = new HashMap<>(); - for (TieringLogSplit logSplit : emptySplits) { - TableBucket tableBucket = logSplit.getTableBucket(); - finishedSplitIds.put(tableBucket, logSplit.splitId()); + for (TieringSplit tieringSplit : emptySplits) { + TableBucket tableBucket = tieringSplit.getTableBucket(); + finishedSplitIds.put(tableBucket, tieringSplit.splitId()); writeResults.put( tableBucket, toTableBucketWriteResult( - logSplit.getTablePath(), + tieringSplit.getTablePath(), tableBucket, - logSplit.getPartitionName(), + tieringSplit.getPartitionName(), null, - logSplit.getStoppingOffset(), + UNKNOWN_BUCKET_OFFSET, UNKNOWN_BUCKET_TIMESTAMP, - logSplit.getNumberOfSplits())); + tieringSplit.getNumberOfSplits())); } return new TableBucketWriteResultWithSplitIds(writeResults, finishedSplitIds); } @@ -362,7 +461,6 @@ private TableBucketWriteResultWithSplitIds forEmptySplits(Set e private void mayFinishCurrentTable() throws IOException { // no any pending splits for the table, just finish the table if (currentTableSplitsByBucket.isEmpty()) { - LOG.info("Finish tier table {} of table id {}.", currentTablePath, currentTableId); finishCurrentTable(); } } @@ -377,6 +475,11 @@ private TableBucketWriteResultWithSplitIds finishCurrentSnapshotSplit() throws I currentSnapshotSplit.getPartitionName(), logEndOffset, UNKNOWN_BUCKET_TIMESTAMP); + LOG.info( + "Finish tier bucket {} for table {}, split: {}.", + tableBucket, + currentTablePath, + splitId); closeCurrentSnapshotSplit(); mayFinishCurrentTable(); return new TableBucketWriteResultWithSplitIds( @@ -437,10 +540,21 @@ private void finishCurrentTable() throws IOException { currentTableNumberOfSplits = null; currentPendingSnapshotSplits.clear(); currentTableStoppingOffsets.clear(); - currentTableEmptyLogSplits.clear(); + currentTableTieredOffsetAndTimestamp.clear(); currentTableSplitsByBucket.clear(); } + /** + * Handle a table reach max tiering duration. This will mark the current table as reaching max + * duration, and it will be force completed in the next fetch cycle. + */ + public void handleTableReachTieringMaxDuration(long tableId) { + if ((currentTableId != null && currentTableId.equals(tableId)) + || pendingTieringSplits.containsKey(tableId)) { + reachTieringMaxDurationTables.add(tableId); + } + } + @Override public void wakeUp() { if (currentLogScanner != null) { @@ -466,7 +580,7 @@ private void subscribeLog(TieringLogSplit logSplit) { long stoppingOffset = logSplit.getStoppingOffset(); long startingOffset = logSplit.getStartingOffset(); if (startingOffset >= stoppingOffset || stoppingOffset <= 0) { - currentTableEmptyLogSplits.add(logSplit); + currentEmptySplits.add(logSplit); return; } else { currentTableStoppingOffsets.put(tableBucket, stoppingOffset); @@ -559,4 +673,15 @@ public Set finishedSplits() { return new HashSet<>(bucketSplits.values()); } } + + private static final class LogOffsetAndTimestamp { + + private final long logOffset; + private final long timestamp; + + public LogOffsetAndTimestamp(long logOffset, long timestamp) { + this.logOffset = logOffset; + this.timestamp = timestamp; + } + } } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumerator.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumerator.java index c1c26390a3..5907a30a6e 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumerator.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumerator.java @@ -26,6 +26,7 @@ import org.apache.fluss.flink.metrics.FlinkMetricRegistry; import org.apache.fluss.flink.tiering.event.FailedTieringEvent; import org.apache.fluss.flink.tiering.event.FinishedTieringEvent; +import org.apache.fluss.flink.tiering.event.TieringReachMaxDurationEvent; import org.apache.fluss.flink.tiering.source.split.TieringSplit; import org.apache.fluss.flink.tiering.source.split.TieringSplitGenerator; import org.apache.fluss.flink.tiering.source.state.TieringSourceEnumeratorState; @@ -39,6 +40,8 @@ import org.apache.fluss.rpc.messages.PbLakeTieringTableInfo; import org.apache.fluss.rpc.metrics.ClientMetricGroup; import org.apache.fluss.utils.MapUtils; +import org.apache.fluss.utils.clock.Clock; +import org.apache.fluss.utils.clock.SystemClock; import org.apache.flink.api.connector.source.ReaderInfo; import org.apache.flink.api.connector.source.SourceEvent; @@ -54,7 +57,9 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -92,12 +97,18 @@ public class TieringSourceEnumerator private final SplitEnumeratorContext context; private final SplitEnumeratorMetricGroup enumeratorMetricGroup; private final long pollTieringTableIntervalMs; + private final long tieringTableDurationMaxMs; + private final long tieringTableDurationDetectIntervalMs; private final List pendingSplits; private final Set readersAwaitingSplit; + + private final Map tieringTablesDeadline; private final Map tieringTableEpochs; private final Map failedTableEpochs; private final Map finishedTableEpochs; + private final Clock clock; + // lazily instantiated private RpcClient rpcClient; private CoordinatorGateway coordinatorGateway; @@ -113,16 +124,38 @@ public class TieringSourceEnumerator public TieringSourceEnumerator( Configuration flussConf, SplitEnumeratorContext context, - long pollTieringTableIntervalMs) { + long pollTieringTableIntervalMs, + long tieringTableDurationMaxMs, + long tieringTableDurationDetectIntervalMs) { + this( + flussConf, + context, + pollTieringTableIntervalMs, + tieringTableDurationMaxMs, + tieringTableDurationDetectIntervalMs, + SystemClock.getInstance()); + } + + public TieringSourceEnumerator( + Configuration flussConf, + SplitEnumeratorContext context, + long pollTieringTableIntervalMs, + long tieringTableDurationMaxMs, + long tieringTableDurationDetectIntervalMs, + Clock clock) { this.flussConf = flussConf; this.context = context; this.enumeratorMetricGroup = context.metricGroup(); this.pollTieringTableIntervalMs = pollTieringTableIntervalMs; + this.tieringTableDurationMaxMs = tieringTableDurationMaxMs; + this.tieringTableDurationDetectIntervalMs = tieringTableDurationDetectIntervalMs; this.pendingSplits = new ArrayList<>(); this.readersAwaitingSplit = new TreeSet<>(); this.tieringTableEpochs = MapUtils.newConcurrentHashMap(); this.finishedTableEpochs = MapUtils.newConcurrentHashMap(); this.failedTableEpochs = MapUtils.newConcurrentHashMap(); + this.tieringTablesDeadline = MapUtils.newConcurrentHashMap(); + this.clock = clock; } @Override @@ -158,6 +191,12 @@ public void start() { this::generateAndAssignSplits, 0, pollTieringTableIntervalMs); + + this.context.callAsync( + this::checkTableReachMaxTieringDuration, + this::handleReachMaxTieringDurationTables, + 0, + tieringTableDurationDetectIntervalMs); } @Override @@ -243,6 +282,7 @@ public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { } else { finishedTableEpochs.put(finishedTableId, tieringEpoch); } + tieringTablesDeadline.remove(finishedTableId); } if (sourceEvent instanceof FailedTieringEvent) { @@ -261,6 +301,7 @@ public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { } else { failedTableEpochs.put(failedTableId, tieringEpoch); } + tieringTablesDeadline.remove(failedTableId); } if (!finishedTableEpochs.isEmpty() || !failedTableEpochs.isEmpty()) { @@ -286,6 +327,53 @@ private void handleSourceReaderFailOver() { } } + private Set checkTableReachMaxTieringDuration() { + Set tieringReachMaxDurationTables = new HashSet<>(); + long currentTime = clock.milliseconds(); + for (Map.Entry tieringTableDeadline : tieringTablesDeadline.entrySet()) { + long tableId = tieringTableDeadline.getKey(); + long deadline = tieringTableDeadline.getValue(); + if (deadline < currentTime) { + tieringReachMaxDurationTables.add(tableId); + } + } + return tieringReachMaxDurationTables; + } + + private void handleReachMaxTieringDurationTables( + Set tieringReachMaxDurationTables, Throwable throwable) { + if (throwable != null) { + LOG.error("Fail to check tiering timeout tables.", throwable); + return; + } + + for (Long reachMaxDurationTable : tieringReachMaxDurationTables) { + for (TieringSplit tieringSplit : pendingSplits) { + if (tieringSplit.getTableBucket().getTableId() == reachMaxDurationTable) { + // force ignore this tiering split since the tiering for this table is timeout, + // we have to force to set to ignore the tiering split so that the + // tiering source reader can ignore them directly + tieringSplit.forceIgnore(); + } else { + // we can break directly, if found any one split's table id is not equal to the + // timeout + // table, the following split must be not equal to the table id + break; + } + } + + LOG.debug("Found the table {} reach max tiering duration.", reachMaxDurationTable); + + // broadcast the tiering reach max duration event to all readers, + // we broadcast all for simplicity + Set readers = new HashSet<>(context.registeredReaders().keySet()); + for (int reader : readers) { + context.sendEventToSourceReader( + reader, new TieringReachMaxDurationEvent(reachMaxDurationTable)); + } + } + } + private void generateAndAssignSplits( @Nullable Tuple3 tieringTable, Throwable throwable) { if (throwable != null) { @@ -303,6 +391,7 @@ private void assignSplits() { return; } /* This method may be called from both addSplitsBack and handleSplitRequest, make it thread safe. */ + // todo: do we need to add lock? synchronized (readersAwaitingSplit) { if (!readersAwaitingSplit.isEmpty()) { final Integer[] readers = readersAwaitingSplit.toArray(new Integer[0]); @@ -378,6 +467,9 @@ private void generateTieringSplits(Tuple3 tieringTable) List tieringSplits = populateNumberOfTieringSplits( splitGenerator.generateTableSplits(tieringTable.f2)); + // shuffle tiering split to avoid splits tiering skew + // after introduce tiering max duration + Collections.shuffle(tieringSplits); LOG.info( "Generate Tiering {} splits for table {} with cost {}ms.", tieringSplits.size(), @@ -391,6 +483,8 @@ private void generateTieringSplits(Tuple3 tieringTable) } else { tieringTableEpochs.put(tieringTable.f0, tieringTable.f1); pendingSplits.addAll(tieringSplits); + tieringTablesDeadline.put( + tieringTable.f0, clock.milliseconds() + tieringTableDurationMaxMs); } } catch (Exception e) { LOG.warn("Fail to generate Tiering splits for table {}.", tieringTable.f2, e); diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringLogSplit.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringLogSplit.java index 3751e531f8..ac8c783af2 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringLogSplit.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringLogSplit.java @@ -54,7 +54,25 @@ public TieringLogSplit( long startingOffset, long stoppingOffset, int numberOfSplits) { - super(tablePath, tableBucket, partitionName, numberOfSplits); + this( + tablePath, + tableBucket, + partitionName, + startingOffset, + stoppingOffset, + false, + numberOfSplits); + } + + public TieringLogSplit( + TablePath tablePath, + TableBucket tableBucket, + @Nullable String partitionName, + long startingOffset, + long stoppingOffset, + boolean forceIgnore, + int numberOfSplits) { + super(tablePath, tableBucket, partitionName, forceIgnore, numberOfSplits); this.startingOffset = startingOffset; this.stoppingOffset = stoppingOffset; } @@ -82,12 +100,14 @@ public String toString() { + ", partitionName='" + partitionName + '\'' + + ", numberOfSplits=" + + numberOfSplits + + ", forceIgnore=" + + forceIgnore + ", startingOffset=" + startingOffset + ", stoppingOffset=" + stoppingOffset - + ", numberOfSplits=" - + numberOfSplits + '}'; } @@ -99,6 +119,7 @@ public TieringLogSplit copy(int numberOfSplits) { partitionName, startingOffset, stoppingOffset, + forceIgnore, numberOfSplits); } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSnapshotSplit.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSnapshotSplit.java index a0a095d578..0044659d2b 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSnapshotSplit.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSnapshotSplit.java @@ -43,10 +43,16 @@ public TieringSnapshotSplit( TableBucket tableBucket, @Nullable String partitionName, long snapshotId, - long logOffsetOfSnapshot) { - super(tablePath, tableBucket, partitionName, UNKNOWN_NUMBER_OF_SPLITS); - this.snapshotId = snapshotId; - this.logOffsetOfSnapshot = logOffsetOfSnapshot; + long logOffsetOfSnapshot, + int numberOfSplits) { + this( + tablePath, + tableBucket, + partitionName, + snapshotId, + logOffsetOfSnapshot, + numberOfSplits, + false); } public TieringSnapshotSplit( @@ -55,8 +61,9 @@ public TieringSnapshotSplit( @Nullable String partitionName, long snapshotId, long logOffsetOfSnapshot, - int numberOfSplits) { - super(tablePath, tableBucket, partitionName, numberOfSplits); + int numberOfSplits, + boolean forceIgnore) { + super(tablePath, tableBucket, partitionName, forceIgnore, numberOfSplits); this.snapshotId = snapshotId; this.logOffsetOfSnapshot = logOffsetOfSnapshot; } @@ -84,12 +91,14 @@ public String toString() { + ", partitionName='" + partitionName + '\'' + + ", numberOfSplits=" + + numberOfSplits + + ", forceIgnore=" + + forceIgnore + ", snapshotId=" + snapshotId + ", logOffsetOfSnapshot=" + logOffsetOfSnapshot - + ", numberOfSplits=" - + numberOfSplits + '}'; } @@ -101,7 +110,8 @@ public TieringSnapshotSplit copy(int numberOfSplits) { partitionName, snapshotId, logOffsetOfSnapshot, - numberOfSplits); + numberOfSplits, + forceIgnore); } @Override diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSplit.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSplit.java index 9da98b8387..cbd3482a8c 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSplit.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSplit.java @@ -38,6 +38,7 @@ public abstract class TieringSplit implements SourceSplit { protected final TableBucket tableBucket; @Nullable protected final String partitionName; + protected boolean forceIgnore; // the total number of splits in one round of tiering protected final int numberOfSplits; @@ -45,6 +46,7 @@ public TieringSplit( TablePath tablePath, TableBucket tableBucket, @Nullable String partitionName, + boolean forceIgnore, int numberOfSplits) { this.tablePath = tablePath; this.tableBucket = tableBucket; @@ -54,6 +56,7 @@ public TieringSplit( throw new IllegalArgumentException( "Partition name and partition id must be both null or both not null."); } + this.forceIgnore = forceIgnore; this.numberOfSplits = numberOfSplits; } @@ -72,6 +75,14 @@ public final boolean isTieringLogSplit() { return getClass() == TieringLogSplit.class; } + public void forceIgnore() { + this.forceIgnore = true; + } + + public boolean isForceIgnore() { + return forceIgnore; + } + /** Casts this split into a {@link TieringLogSplit}. */ public TieringLogSplit asTieringLogSplit() { return (TieringLogSplit) this; @@ -128,11 +139,12 @@ public boolean equals(Object object) { return Objects.equals(tablePath, that.tablePath) && Objects.equals(tableBucket, that.tableBucket) && Objects.equals(partitionName, that.partitionName) + && forceIgnore == that.forceIgnore && numberOfSplits == that.numberOfSplits; } @Override public int hashCode() { - return Objects.hash(tablePath, tableBucket, partitionName, numberOfSplits); + return Objects.hash(tablePath, tableBucket, partitionName, forceIgnore, numberOfSplits); } } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSplitGenerator.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSplitGenerator.java index 373118a972..0fd814c14d 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSplitGenerator.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSplitGenerator.java @@ -57,7 +57,6 @@ public TieringSplitGenerator(Admin flussAdmin) { } public List generateTableSplits(TablePath tablePath) throws Exception { - final TableInfo tableInfo = flussAdmin.getTableInfo(tablePath).get(); final BucketOffsetsRetriever bucketOffsetsRetriever = new BucketOffsetsRetrieverImpl(flussAdmin, tablePath); @@ -287,7 +286,7 @@ private Optional generateSplitForPrimaryKeyTableBucket( latestBucketOffset, 0)); } else { - LOG.info( + LOG.debug( "The lastCommittedBucketOffset {} is equals or bigger than latestBucketOffset {}, skip generating split for bucket {}", lastCommittedBucketOffset, latestBucketOffset, @@ -320,8 +319,7 @@ private Optional generateSplitForLogTableBucket( tableBucket, partitionName, EARLIEST_OFFSET, - latestBucketOffset, - 0)); + latestBucketOffset)); } else { // the bucket has been tiered, scan remain fluss log if (lastCommittedBucketOffset < latestBucketOffset) { @@ -331,11 +329,10 @@ private Optional generateSplitForLogTableBucket( tableBucket, partitionName, lastCommittedBucketOffset, - latestBucketOffset, - 0)); + latestBucketOffset)); } } - LOG.info( + LOG.debug( "The lastCommittedBucketOffset {} is equals or bigger than latestBucketOffset {}, skip generating split for bucket {}", lastCommittedBucketOffset, latestBucketOffset, diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSplitSerializer.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSplitSerializer.java index 1c2997a540..85b3429afe 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSplitSerializer.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSplitSerializer.java @@ -74,6 +74,9 @@ public byte[] serialize(TieringSplit split) throws IOException { out.writeBoolean(false); } + // write force ignore + out.writeBoolean(split.isForceIgnore()); + // write number of splits out.writeInt(split.getNumberOfSplits()); if (split.isTieringSnapshotSplit()) { @@ -126,6 +129,8 @@ public TieringSplit deserialize(int version, byte[] serialized) throws IOExcepti } TableBucket tableBucket = new TableBucket(tableId, partitionId, bucketId); + boolean forceIgnore = in.readBoolean(); + // deserialize number of splits int numberOfSplits = in.readInt(); @@ -140,7 +145,8 @@ public TieringSplit deserialize(int version, byte[] serialized) throws IOExcepti partitionName, snapshotId, logOffsetOfSnapshot, - numberOfSplits); + numberOfSplits, + forceIgnore); } else { // deserialize starting offset long startingOffset = in.readLong(); @@ -152,6 +158,7 @@ public TieringSplit deserialize(int version, byte[] serialized) throws IOExcepti partitionName, startingOffset, stoppingOffset, + forceIgnore, numberOfSplits); } } diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/TieringITCase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/TieringITCase.java new file mode 100644 index 0000000000..ce8c20c41e --- /dev/null +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/TieringITCase.java @@ -0,0 +1,258 @@ +/* + * 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.fluss.flink.tiering; + +import org.apache.fluss.client.Connection; +import org.apache.fluss.client.ConnectionFactory; +import org.apache.fluss.client.admin.Admin; +import org.apache.fluss.client.metadata.LakeSnapshot; +import org.apache.fluss.client.table.Table; +import org.apache.fluss.client.table.writer.AppendWriter; +import org.apache.fluss.client.table.writer.TableWriter; +import org.apache.fluss.client.table.writer.UpsertWriter; +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.exception.FlussRuntimeException; +import org.apache.fluss.exception.LakeTableSnapshotNotExistException; +import org.apache.fluss.metadata.DataLakeFormat; +import org.apache.fluss.metadata.Schema; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TableDescriptor; +import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.row.BinaryString; +import org.apache.fluss.row.GenericRow; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.server.testutils.FlussClusterExtension; +import org.apache.fluss.types.DataTypes; +import org.apache.fluss.utils.ExceptionUtils; + +import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +import java.nio.file.Files; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static org.apache.fluss.config.ConfigOptions.LAKE_TIERING_TABLE_DURATION_DETECT_INTERVAL; +import static org.apache.fluss.config.ConfigOptions.LAKE_TIERING_TABLE_DURATION_MAX; +import static org.apache.fluss.testutils.common.CommonTestUtils.waitValue; +import static org.assertj.core.api.Assertions.assertThat; + +/** The IT case for tiering. */ +class TieringITCase { + + @RegisterExtension + public static final FlussClusterExtension FLUSS_CLUSTER_EXTENSION = + FlussClusterExtension.builder() + .setClusterConf(initConfig()) + .setNumOfTabletServers(3) + .build(); + + protected static String warehousePath; + protected static Connection conn; + protected static Admin admin; + protected static StreamExecutionEnvironment execEnv; + + protected static Configuration initConfig() { + Configuration conf = new Configuration(); + conf.set(ConfigOptions.KV_SNAPSHOT_INTERVAL, Duration.ofSeconds(1)) + // not to clean snapshots for test purpose + .set(ConfigOptions.KV_MAX_RETAINED_SNAPSHOTS, Integer.MAX_VALUE); + conf.setString("datalake.format", "paimon"); + conf.setString("datalake.paimon.metastore", "filesystem"); + try { + warehousePath = + Files.createTempDirectory("fluss-testing-datalake-tiered") + .resolve("warehouse") + .toString(); + } catch (Exception e) { + throw new FlussRuntimeException("Failed to create warehouse path"); + } + conf.setString("datalake.paimon.warehouse", warehousePath); + return conf; + } + + @BeforeAll + static void beforeAll() { + conn = ConnectionFactory.createConnection(FLUSS_CLUSTER_EXTENSION.getClientConfig()); + admin = conn.getAdmin(); + execEnv = + StreamExecutionEnvironment.getExecutionEnvironment() + .setParallelism(1) + .setRuntimeMode(RuntimeExecutionMode.STREAMING); + } + + @Test + void testTieringReachMaxDuration() throws Exception { + TablePath logTablePath = TablePath.of("fluss", "logtable"); + createTable(logTablePath, false); + TablePath pkTablePath = TablePath.of("fluss", "pktable"); + long pkTableId = createTable(pkTablePath, true); + + // write some records to log table + List rows = new ArrayList<>(); + int recordCount = 6; + for (int i = 0; i < recordCount; i++) { + rows.add(GenericRow.of(i, BinaryString.fromString("v" + i))); + } + writeRows(logTablePath, rows, true); + + rows = new ArrayList<>(); + // write 6 records to primary key table, each bucket should only contain few record + for (int i = 0; i < recordCount; i++) { + rows.add(GenericRow.of(i, BinaryString.fromString("v" + i))); + } + writeRows(pkTablePath, rows, false); + + waitUntilSnapshot(pkTableId, 3, 0); + + JobClient jobClient = buildTieringJob(execEnv); + + try { + // verify the tiered records is less than the table total record to + // make sure tiering is forced to complete when reach max duration + LakeSnapshot logTableLakeSnapshot = waitLakeSnapshot(logTablePath); + long tieredRecords = countTieredRecords(logTableLakeSnapshot); + assertThat(tieredRecords).isLessThan(recordCount); + + // verify the tiered records is less than the table total record to + // make sure tiering is forced to complete when reach max duration + LakeSnapshot pkTableLakeSnapshot = waitLakeSnapshot(pkTablePath); + tieredRecords = countTieredRecords(pkTableLakeSnapshot); + assertThat(tieredRecords).isLessThan(recordCount); + } finally { + jobClient.cancel(); + } + } + + @AfterAll + static void afterAll() throws Exception { + if (admin != null) { + admin.close(); + admin = null; + } + if (conn != null) { + conn.close(); + conn = null; + } + } + + private long countTieredRecords(LakeSnapshot lakeSnapshot) throws Exception { + return lakeSnapshot.getTableBucketsOffset().values().stream() + .mapToLong(Long::longValue) + .sum(); + } + + private LakeSnapshot waitLakeSnapshot(TablePath tablePath) { + return waitValue( + () -> { + try { + return Optional.of(admin.getLatestLakeSnapshot(tablePath).get()); + } catch (Exception e) { + if (ExceptionUtils.stripExecutionException(e) + instanceof LakeTableSnapshotNotExistException) { + return Optional.empty(); + } + throw e; + } + }, + Duration.ofSeconds(30), + "Fail to wait for one round of tiering finish for table " + tablePath); + } + + private long createTable(TablePath tablePath, boolean isPrimaryKeyTable) throws Exception { + Schema.Builder schemaBuilder = + Schema.newBuilder().column("a", DataTypes.INT()).column("b", DataTypes.STRING()); + if (isPrimaryKeyTable) { + schemaBuilder.primaryKey("a"); + } + TableDescriptor.Builder tableDescriptorBuilder = + TableDescriptor.builder() + .schema(schemaBuilder.build()) + .distributedBy(3, "a") + .property(ConfigOptions.TABLE_DATALAKE_ENABLED.key(), "true") + .property(ConfigOptions.TABLE_DATALAKE_FRESHNESS, Duration.ofMillis(500)); + + // see TestingPaimonStoragePlugin#TestingPaimonWriter, we set write-pause + // to 1s to make it easy to mock tiering reach max duration + Map customProperties = Collections.singletonMap("write-pause", "1s"); + tableDescriptorBuilder.customProperties(customProperties); + return createTable(tablePath, tableDescriptorBuilder.build()); + } + + protected long createTable(TablePath tablePath, TableDescriptor tableDescriptor) + throws Exception { + admin.createTable(tablePath, tableDescriptor, true).get(); + return admin.getTableInfo(tablePath).get().getTableId(); + } + + private void writeRows(TablePath tablePath, List rows, boolean append) + throws Exception { + try (Table table = conn.getTable(tablePath)) { + TableWriter tableWriter; + if (append) { + tableWriter = table.newAppend().createWriter(); + } else { + tableWriter = table.newUpsert().createWriter(); + } + for (InternalRow row : rows) { + if (tableWriter instanceof AppendWriter) { + ((AppendWriter) tableWriter).append(row); + } else { + ((UpsertWriter) tableWriter).upsert(row); + } + } + tableWriter.flush(); + } + } + + private JobClient buildTieringJob(StreamExecutionEnvironment execEnv) throws Exception { + Configuration lakeTieringConfig = new Configuration(); + lakeTieringConfig.set(LAKE_TIERING_TABLE_DURATION_MAX, Duration.ofSeconds(1)); + lakeTieringConfig.set(LAKE_TIERING_TABLE_DURATION_DETECT_INTERVAL, Duration.ofMillis(100)); + + Configuration flussConfig = new Configuration(); + flussConfig.setString( + ConfigOptions.BOOTSTRAP_SERVERS.key(), + FLUSS_CLUSTER_EXTENSION.getBootstrapServers()); + return LakeTieringJobBuilder.newBuilder( + execEnv, + flussConfig, + new Configuration(), + lakeTieringConfig, + DataLakeFormat.PAIMON.toString()) + .build(); + } + + protected void waitUntilSnapshot(long tableId, int bucketNum, long snapshotId) { + for (int i = 0; i < bucketNum; i++) { + TableBucket tableBucket = new TableBucket(tableId, i); + FLUSS_CLUSTER_EXTENSION.waitUntilSnapshotFinished(tableBucket, snapshotId); + } + } +} diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/TieringSourceReaderTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/TieringSourceReaderTest.java new file mode 100644 index 0000000000..8868b29832 --- /dev/null +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/TieringSourceReaderTest.java @@ -0,0 +1,203 @@ +/* + * 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.fluss.flink.tiering.source; + +import org.apache.fluss.client.Connection; +import org.apache.fluss.client.ConnectionFactory; +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.flink.tiering.TestingLakeTieringFactory; +import org.apache.fluss.flink.tiering.TestingWriteResult; +import org.apache.fluss.flink.tiering.event.TieringReachMaxDurationEvent; +import org.apache.fluss.flink.tiering.source.split.TieringLogSplit; +import org.apache.fluss.flink.utils.FlinkTestBase; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TablePath; + +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.flink.connector.testutils.source.reader.TestingReaderContext; +import org.apache.flink.connector.testutils.source.reader.TestingReaderOutput; +import org.junit.jupiter.api.Test; + +import java.lang.reflect.Field; +import java.time.Duration; +import java.util.Arrays; +import java.util.Collections; + +import static org.apache.fluss.client.table.scanner.log.LogScanner.EARLIEST_OFFSET; +import static org.apache.fluss.testutils.DataTestUtils.row; +import static org.apache.fluss.testutils.common.CommonTestUtils.retry; +import static org.apache.fluss.testutils.common.CommonTestUtils.waitUntil; +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link TieringSourceReader}. */ +class TieringSourceReaderTest extends FlinkTestBase { + + @Test + void testHandleTieringReachMaxDurationEvent() throws Exception { + TablePath tablePath = TablePath.of("fluss", "test_tiering_reach_max_duration"); + long tableId = createTable(tablePath, DEFAULT_LOG_TABLE_DESCRIPTOR); + Configuration conf = new Configuration(FLUSS_CLUSTER_EXTENSION.getClientConfig()); + conf.set( + ConfigOptions.CLIENT_WRITER_BUCKET_NO_KEY_ASSIGNER, + ConfigOptions.NoKeyAssigner.ROUND_ROBIN); + try (Connection connection = ConnectionFactory.createConnection(conf)) { + FutureCompletingBlockingQueue< + RecordsWithSplitIds>> + elementsQueue = new FutureCompletingBlockingQueue<>(16); + TestingReaderContext readerContext = new TestingReaderContext(); + try (TieringSourceReader reader = + new TieringSourceReader<>( + elementsQueue, + readerContext, + connection, + new TestingLakeTieringFactory(), + Duration.ofMillis(500))) { + + reader.start(); + + // no data, add a split for the table, + // should be force be complete after reach max duration + TieringLogSplit split = + new TieringLogSplit( + tablePath, new TableBucket(tableId, 0), null, EARLIEST_OFFSET, 100); + reader.addSplits(Collections.singletonList(split)); + + // send TieringReachMaxDurationEvent + TieringReachMaxDurationEvent event = new TieringReachMaxDurationEvent(tableId); + reader.handleSourceEvents(event); + + retry( + Duration.ofMinutes(1), + () -> { + TestingReaderOutput> output = + new TestingReaderOutput<>(); + // should force to finish, the write result is null + reader.pollNext(output); + assertThat(output.getEmittedRecords()).hasSize(1); + TableBucketWriteResult result = + output.getEmittedRecords().get(0); + assertThat(result.writeResult()).isNull(); + }); + + // write some data + writeRows( + connection, + tablePath, + Arrays.asList(row(0, "v0"), row(1, "v1"), row(2, "v2")), + true); + split = + new TieringLogSplit( + tablePath, + new TableBucket(tableId, 2), + null, + EARLIEST_OFFSET, + // use 100L as end offset, so that + // tiering won't be finished if no tiering reach max duration logic + 100L); + + reader.addSplits(Collections.singletonList(split)); + + // wait to run one round of tiering to do some tiering + FutureCompletingBlockingQueue< + RecordsWithSplitIds>> + blockingQueue = getElementsQueue(reader); + // wait blockingQueue is not empty to make sure we have one fetch + // in tiering source reader + waitUntil( + () -> !blockingQueue.isEmpty(), + Duration.ofSeconds(30), + "Fail to wait element queue is not empty."); + + // send TieringReachMaxDurationEvent + event = new TieringReachMaxDurationEvent(tableId); + reader.handleSourceEvents(event); + + // make sure tiering will be finished, still maintain the result + // of previous tiering + retry( + Duration.ofMinutes(1), + () -> { + TestingReaderOutput> + output1 = new TestingReaderOutput<>(); + + // should force to finish, the write result isn't null + reader.pollNext(output1); + assertThat(output1.getEmittedRecords()).hasSize(1); + TableBucketWriteResult result = + output1.getEmittedRecords().get(0); + TestingWriteResult testingWriteResult = result.writeResult(); + assertThat(testingWriteResult).isNotNull(); + assertThat(result.logEndOffset()).isEqualTo(1); + }); + + // test add split with force ignore + split = + new TieringLogSplit( + tablePath, + new TableBucket(tableId, 1), + null, + EARLIEST_OFFSET, + 100L); + split.forceIgnore(); + reader.addSplits(Collections.singletonList(split)); + // should skip tiering for this split + retry( + Duration.ofMinutes(1), + () -> { + TestingReaderOutput> + output1 = new TestingReaderOutput<>(); + // should force to finish, and the result is null + reader.pollNext(output1); + assertThat(output1.getEmittedRecords()).hasSize(1); + TableBucketWriteResult result = + output1.getEmittedRecords().get(0); + assertThat(result.writeResult()).isNull(); + }); + } + } + } + + /** + * Get the elementsQueue from TieringSourceReader using reflection. + * + * @param reader the TieringSourceReader instance + * @return the elementsQueue field value + */ + @SuppressWarnings("unchecked") + private FutureCompletingBlockingQueue< + RecordsWithSplitIds>> + getElementsQueue(TieringSourceReader reader) throws Exception { + Class clazz = reader.getClass(); + while (clazz != null) { + try { + Field elementsQueueField = clazz.getDeclaredField("elementsQueue"); + elementsQueueField.setAccessible(true); + return (FutureCompletingBlockingQueue< + RecordsWithSplitIds>>) + elementsQueueField.get(reader); + } catch (NoSuchFieldException e) { + // Try parent class + clazz = clazz.getSuperclass(); + } + } + throw new RuntimeException("No elementsQueue field found"); + } +} diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumeratorTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumeratorTest.java index 15f62f3d1b..8781569cf4 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumeratorTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumeratorTest.java @@ -20,6 +20,8 @@ import org.apache.fluss.config.Configuration; import org.apache.fluss.flink.tiering.event.FailedTieringEvent; import org.apache.fluss.flink.tiering.event.FinishedTieringEvent; +import org.apache.fluss.flink.tiering.event.TieringFailOverEvent; +import org.apache.fluss.flink.tiering.event.TieringReachMaxDurationEvent; import org.apache.fluss.flink.tiering.source.TieringTestBase; import org.apache.fluss.flink.tiering.source.split.TieringLogSplit; import org.apache.fluss.flink.tiering.source.split.TieringSnapshotSplit; @@ -30,7 +32,10 @@ import org.apache.fluss.rpc.messages.CommitLakeTableSnapshotRequest; import org.apache.fluss.rpc.messages.PbLakeTableOffsetForBucket; import org.apache.fluss.rpc.messages.PbLakeTableSnapshotInfo; +import org.apache.fluss.utils.clock.ManualClock; +import org.apache.flink.api.connector.source.ReaderInfo; +import org.apache.flink.api.connector.source.SourceEvent; import org.apache.flink.api.connector.source.SplitsAssignment; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; @@ -38,9 +43,9 @@ import javax.annotation.Nullable; +import java.time.Duration; import java.util.ArrayList; import java.util.Collections; -import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -76,7 +81,7 @@ void testPrimaryKeyTableWithNoSnapshotSplits() throws Throwable { try (FlussMockSplitEnumeratorContext context = new FlussMockSplitEnumeratorContext<>(numSubtasks)) { TieringSourceEnumerator enumerator = - new TieringSourceEnumerator(flussConf, context, 500); + createDefaultTieringSourceEnumerator(flussConf, context); enumerator.start(); assertThat(context.getSplitsAssignmentSequence()).isEmpty(); @@ -119,25 +124,23 @@ void testPrimaryKeyTableWithNoSnapshotSplits() throws Throwable { } waitUntilTieringTableSplitAssignmentReady(context, DEFAULT_BUCKET_NUM, 500L); - Map> expectedLogAssignment = new HashMap<>(); + List expectedLogAssignment = new ArrayList<>(); for (int tableBucket = 0; tableBucket < DEFAULT_BUCKET_NUM; tableBucket++) { - expectedLogAssignment.put( - tableBucket, - Collections.singletonList( - new TieringLogSplit( - tablePath, - new TableBucket(tableId, tableBucket), - null, - bucketOffsetOfInitialWrite.get(tableBucket), - bucketOffsetOfInitialWrite.get(tableBucket) - + bucketOffsetOfSecondWrite.get(tableBucket), - expectNumberOfSplits))); - } - Map> actualLogAssignment = new HashMap<>(); - for (SplitsAssignment a : context.getSplitsAssignmentSequence()) { - actualLogAssignment.putAll(a.assignment()); + expectedLogAssignment.add( + new TieringLogSplit( + tablePath, + new TableBucket(tableId, tableBucket), + null, + bucketOffsetOfInitialWrite.get(tableBucket), + bucketOffsetOfInitialWrite.get(tableBucket) + + bucketOffsetOfSecondWrite.get(tableBucket), + expectNumberOfSplits)); } - assertThat(actualLogAssignment).isEqualTo(expectedLogAssignment); + List actualLogAssignment = new ArrayList<>(); + context.getSplitsAssignmentSequence() + .forEach(a -> a.assignment().values().forEach(actualLogAssignment::addAll)); + assertThat(actualLogAssignment) + .containsExactlyInAnyOrderElementsOf(expectedLogAssignment); } } @@ -157,7 +160,7 @@ void testPrimaryKeyTableWithSnapshotSplits() throws Throwable { try (FlussMockSplitEnumeratorContext context = new FlussMockSplitEnumeratorContext<>(numSubtasks)) { TieringSourceEnumerator enumerator = - new TieringSourceEnumerator(flussConf, context, 500); + createDefaultTieringSourceEnumerator(flussConf, context); enumerator.start(); assertThat(context.getSplitsAssignmentSequence()).isEmpty(); @@ -166,24 +169,22 @@ void testPrimaryKeyTableWithSnapshotSplits() throws Throwable { registerReaderAndHandleSplitRequests(context, enumerator, numSubtasks, 0); waitUntilTieringTableSplitAssignmentReady(context, DEFAULT_BUCKET_NUM, 3000L); - Map> expectedSnapshotAssignment = new HashMap<>(); + List expectedSnapshotAssignment = new ArrayList<>(); for (int tableBucket = 0; tableBucket < DEFAULT_BUCKET_NUM; tableBucket++) { - expectedSnapshotAssignment.put( - tableBucket, - Collections.singletonList( - new TieringSnapshotSplit( - tablePath, - new TableBucket(tableId, tableBucket), - null, - snapshotId, - bucketOffsetOfInitialWrite.get(tableBucket), - expectNumberOfSplits))); - } - Map> actualSnapshotAssignment = new HashMap<>(); - for (SplitsAssignment a : context.getSplitsAssignmentSequence()) { - actualSnapshotAssignment.putAll(a.assignment()); + expectedSnapshotAssignment.add( + new TieringSnapshotSplit( + tablePath, + new TableBucket(tableId, tableBucket), + null, + snapshotId, + bucketOffsetOfInitialWrite.get(tableBucket), + expectNumberOfSplits)); } - assertThat(actualSnapshotAssignment).isEqualTo(expectedSnapshotAssignment); + List actualAssignment = new ArrayList<>(); + context.getSplitsAssignmentSequence() + .forEach(a -> a.assignment().values().forEach(actualAssignment::addAll)); + assertThat(actualAssignment) + .containsExactlyInAnyOrderElementsOf(expectedSnapshotAssignment); // mock finished tiered this round, check second round context.getSplitsAssignmentSequence().clear(); @@ -212,25 +213,23 @@ void testPrimaryKeyTableWithSnapshotSplits() throws Throwable { // three log splits will be ready soon waitUntilTieringTableSplitAssignmentReady(context, DEFAULT_BUCKET_NUM, 500L); - Map> expectedLogAssignment = new HashMap<>(); + List expectedLogAssignment = new ArrayList<>(); for (int tableBucket = 0; tableBucket < DEFAULT_BUCKET_NUM; tableBucket++) { - expectedLogAssignment.put( - tableBucket, - Collections.singletonList( - new TieringLogSplit( - tablePath, - new TableBucket(tableId, tableBucket), - null, - bucketOffsetOfInitialWrite.get(tableBucket), - bucketOffsetOfInitialWrite.get(tableBucket) - + bucketOffsetOfSecondWrite.get(tableBucket), - expectNumberOfSplits))); - } - Map> actualLogAssignment = new HashMap<>(); - for (SplitsAssignment a : context.getSplitsAssignmentSequence()) { - actualLogAssignment.putAll(a.assignment()); + expectedLogAssignment.add( + new TieringLogSplit( + tablePath, + new TableBucket(tableId, tableBucket), + null, + bucketOffsetOfInitialWrite.get(tableBucket), + bucketOffsetOfInitialWrite.get(tableBucket) + + bucketOffsetOfSecondWrite.get(tableBucket), + expectNumberOfSplits)); } - assertThat(actualLogAssignment).isEqualTo(expectedLogAssignment); + List actualLogAssignment = new ArrayList<>(); + context.getSplitsAssignmentSequence() + .forEach(a -> a.assignment().values().forEach(actualLogAssignment::addAll)); + assertThat(actualLogAssignment) + .containsExactlyInAnyOrderElementsOf(expectedLogAssignment); } } @@ -244,7 +243,7 @@ void testLogTableSplits() throws Throwable { try (FlussMockSplitEnumeratorContext context = new FlussMockSplitEnumeratorContext<>(numSubtasks)) { TieringSourceEnumerator enumerator = - new TieringSourceEnumerator(flussConf, context, 500); + createDefaultTieringSourceEnumerator(flussConf, context); enumerator.start(); assertThat(context.getSplitsAssignmentSequence()).isEmpty(); @@ -273,7 +272,7 @@ void testLogTableSplits() throws Throwable { context.getSplitsAssignmentSequence() .forEach(a -> a.assignment().values().forEach(actualAssignment::addAll)); - assertThat(actualAssignment).isEqualTo(expectedAssignment); + assertThat(actualAssignment).containsExactlyInAnyOrderElementsOf(expectedAssignment); // mock finished tiered this round, check second round context.getSplitsAssignmentSequence().clear(); @@ -303,25 +302,23 @@ void testLogTableSplits() throws Throwable { waitUntilTieringTableSplitAssignmentReady(context, DEFAULT_BUCKET_NUM, 500L); - Map> expectedLogAssignment = new HashMap<>(); + List expectedLogAssignment = new ArrayList<>(); for (int tableBucket = 0; tableBucket < DEFAULT_BUCKET_NUM; tableBucket++) { - expectedLogAssignment.put( - tableBucket, - Collections.singletonList( - new TieringLogSplit( - tablePath, - new TableBucket(tableId, tableBucket), - null, - bucketOffsetOfInitialWrite.get(tableBucket), - bucketOffsetOfInitialWrite.get(tableBucket) - + bucketOffsetOfSecondWrite.get(tableBucket), - expectNumberOfSplits))); - } - Map> actualLogAssignment = new HashMap<>(); - for (SplitsAssignment a : context.getSplitsAssignmentSequence()) { - actualLogAssignment.putAll(a.assignment()); + expectedLogAssignment.add( + new TieringLogSplit( + tablePath, + new TableBucket(tableId, tableBucket), + null, + bucketOffsetOfInitialWrite.get(tableBucket), + bucketOffsetOfInitialWrite.get(tableBucket) + + bucketOffsetOfSecondWrite.get(tableBucket), + expectNumberOfSplits)); } - assertThat(actualLogAssignment).isEqualTo(expectedLogAssignment); + List actualLogAssignment = new ArrayList<>(); + context.getSplitsAssignmentSequence() + .forEach(a -> a.assignment().values().forEach(actualLogAssignment::addAll)); + assertThat(actualLogAssignment) + .containsExactlyInAnyOrderElementsOf(expectedLogAssignment); } } @@ -340,7 +337,7 @@ void testPartitionedPrimaryKeyTable() throws Throwable { try (FlussMockSplitEnumeratorContext context = new FlussMockSplitEnumeratorContext<>(numSubtasks)) { TieringSourceEnumerator enumerator = - new TieringSourceEnumerator(flussConf, context, 500); + createDefaultTieringSourceEnumerator(flussConf, context); enumerator.start(); assertThat(context.getSplitsAssignmentSequence()).isEmpty(); @@ -421,8 +418,8 @@ void testPartitionedPrimaryKeyTable() throws Throwable { context.getSplitsAssignmentSequence()) { splitsAssignment.assignment().values().forEach(actualLogAssignment::addAll); } - assertThat(sortSplits(actualLogAssignment)) - .isEqualTo(sortSplits(expectedLogAssignment)); + assertThat(actualLogAssignment) + .containsExactlyInAnyOrderElementsOf(expectedLogAssignment); } } @@ -441,7 +438,7 @@ void testPartitionedLogTableSplits() throws Throwable { try (FlussMockSplitEnumeratorContext context = new FlussMockSplitEnumeratorContext<>(numSubtasks)) { TieringSourceEnumerator enumerator = - new TieringSourceEnumerator(flussConf, context, 500); + createDefaultTieringSourceEnumerator(flussConf, context); enumerator.start(); assertThat(context.getSplitsAssignmentSequence()).isEmpty(); @@ -478,7 +475,7 @@ void testPartitionedLogTableSplits() throws Throwable { context.getSplitsAssignmentSequence()) { splitsAssignment.assignment().values().forEach(actualAssignment::addAll); } - assertThat(sortSplits(actualAssignment)).isEqualTo(sortSplits(expectedAssignment)); + assertThat(actualAssignment).containsExactlyInAnyOrderElementsOf(expectedAssignment); // mock finished tiered this round, check second round context.getSplitsAssignmentSequence().clear(); @@ -547,8 +544,8 @@ void testPartitionedLogTableSplits() throws Throwable { context.getSplitsAssignmentSequence()) { splitsAssignment.assignment().values().forEach(actualLogAssignment::addAll); } - assertThat(sortSplits(actualLogAssignment)) - .isEqualTo(sortSplits(expectedLogAssignment)); + assertThat(actualLogAssignment) + .containsExactlyInAnyOrderElementsOf(expectedLogAssignment); } } @@ -588,7 +585,7 @@ void testHandleFailedTieringTableEvent() throws Throwable { context.getSplitsAssignmentSequence() .forEach(a -> a.assignment().values().forEach(actualAssignment::addAll)); - assertThat(actualAssignment).isEqualTo(expectedAssignment); + assertThat(actualAssignment).containsExactlyInAnyOrderElementsOf(expectedAssignment); // mock tiering fail by send tiering fail event context.getSplitsAssignmentSequence().clear(); @@ -602,7 +599,7 @@ void testHandleFailedTieringTableEvent() throws Throwable { List actualAssignment1 = new ArrayList<>(); context.getSplitsAssignmentSequence() .forEach(a -> a.assignment().values().forEach(actualAssignment1::addAll)); - assertThat(actualAssignment1).isEqualTo(expectedAssignment); + assertThat(actualAssignment1).containsExactlyInAnyOrderElementsOf(expectedAssignment); } } @@ -619,7 +616,7 @@ void testHandleReaderFailOver() throws Throwable { try (FlussMockSplitEnumeratorContext context = new FlussMockSplitEnumeratorContext<>(3)) { TieringSourceEnumerator enumerator = - new TieringSourceEnumerator(flussConf, context, 500); + createDefaultTieringSourceEnumerator(flussConf, context); enumerator.start(); assertThat(context.getSplitsAssignmentSequence()).isEmpty(); @@ -705,12 +702,6 @@ private void waitUntilTieringTableSplitAssignmentReady( } } - private static List sortSplits(List splits) { - return splits.stream() - .sorted(Comparator.comparing(Object::toString)) - .collect(Collectors.toList()); - } - private void verifyTieringSplitAssignment( FlussMockSplitEnumeratorContext context, int expectedSplitSize, @@ -729,4 +720,99 @@ private void verifyTieringSplitAssignment( assertThat(allTieringSplits) .allMatch(tieringSplit -> tieringSplit.getTablePath().equals(expectedTablePath)); } + + private TieringSourceEnumerator createDefaultTieringSourceEnumerator( + Configuration flussConf, MockSplitEnumeratorContext context) { + return new TieringSourceEnumerator( + flussConf, + context, + 500, + Duration.ofMinutes(10).toMillis(), + Duration.ofSeconds(10).toMillis()); + } + + private TieringSourceEnumerator createTieringSourceEnumeratorWithManualClock( + Configuration flussConf, + MockSplitEnumeratorContext context, + ManualClock clock, + long tieringTableDurationMaxMs, + long tieringTableDurationDetectIntervalMs) { + return new TieringSourceEnumerator( + flussConf, + context, + 500, + tieringTableDurationMaxMs, + tieringTableDurationDetectIntervalMs, + clock); + } + + @Test + void testTableReachMaxTieringDuration() throws Throwable { + TablePath tablePath = TablePath.of(DEFAULT_DB, "tiering-max-duration-test-log-table"); + long tableId = createTable(tablePath, DEFAULT_LOG_TABLE_DESCRIPTOR); + int numSubtasks = 2; + + long tieringTableDurationMaxMs = Duration.ofMinutes(10).toMillis(); + long tieringTableDurationDetectIntervalMs = Duration.ofMillis(100).toMillis(); + + ManualClock manualClock = new ManualClock(); + + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(numSubtasks); + TieringSourceEnumerator enumerator = + createTieringSourceEnumeratorWithManualClock( + flussConf, + context, + manualClock, + tieringTableDurationMaxMs, + tieringTableDurationDetectIntervalMs); ) { + enumerator.start(); + + // Register all readers + for (int subtaskId = 0; subtaskId < numSubtasks; subtaskId++) { + registerReader(context, enumerator, subtaskId, "localhost-" + subtaskId); + } + + appendRow(tablePath, DEFAULT_LOG_TABLE_DESCRIPTOR, 0, 10); + + for (int subTask = 0; subTask < numSubtasks; subTask++) { + enumerator.handleSplitRequest(subTask, "localhost-" + subTask); + } + + // Wait for initial assignment + waitUntilTieringTableSplitAssignmentReady(context, 2, 200L); + + // Advance time to mock exceed max duration + manualClock.advanceTime(Duration.ofMillis(tieringTableDurationMaxMs + 60_000)); + + // Run periodic callable to trigger max duration check + // Index 0 is for requestTieringTableSplitsViaHeartBeat + // Index 1 is for checkTableReachMaxTieringDuration + context.runPeriodicCallable(1); + + // Verify that TieringReachMaxDurationEvent was sent to all readers + // Use reflection to access events sent to readers + Map> eventsToReaders = context.getSentSourceEvent(); + assertThat(eventsToReaders).hasSize(numSubtasks); + for (Map.Entry> entry : eventsToReaders.entrySet()) { + assertThat(entry.getValue()) + .containsExactly(new TieringReachMaxDurationEvent(tableId)); + } + + // clear split assignment + context.getSplitsAssignmentSequence().clear(); + + // request a split again + enumerator.handleSplitRequest(0, "localhost-0"); + + // the split should be marked as forceIgnore + waitUntilTieringTableSplitAssignmentReady(context, 1, 100L); + + List assignedSplits = new ArrayList<>(); + context.getSplitsAssignmentSequence() + .forEach(a -> a.assignment().values().forEach(assignedSplits::addAll)); + assertThat(assignedSplits).hasSize(1); + assertThat(assignedSplits.get(0).isForceIgnore()).isTrue(); + } + } } diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/split/TieringSplitSerializerTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/split/TieringSplitSerializerTest.java index 33356dec40..6c4774870a 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/split/TieringSplitSerializerTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/split/TieringSplitSerializerTest.java @@ -20,6 +20,7 @@ import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TablePath; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; @@ -68,8 +69,8 @@ void testTieringSnapshotSplitStringExpression(Boolean isPartitionedTable) throws String expectedSplitString = isPartitionedTable - ? "TieringSnapshotSplit{tablePath=test_db.test_partitioned_table, tableBucket=TableBucket{tableId=1, partitionId=100, bucket=2}, partitionName='1024', snapshotId=0, logOffsetOfSnapshot=200, numberOfSplits=30}" - : "TieringSnapshotSplit{tablePath=test_db.test_table, tableBucket=TableBucket{tableId=1, bucket=2}, partitionName='null', snapshotId=0, logOffsetOfSnapshot=200, numberOfSplits=30}"; + ? "TieringSnapshotSplit{tablePath=test_db.test_partitioned_table, tableBucket=TableBucket{tableId=1, partitionId=100, bucket=2}, partitionName='1024', numberOfSplits=30, forceIgnore=false, snapshotId=0, logOffsetOfSnapshot=200}" + : "TieringSnapshotSplit{tablePath=test_db.test_table, tableBucket=TableBucket{tableId=1, bucket=2}, partitionName='null', numberOfSplits=30, forceIgnore=false, snapshotId=0, logOffsetOfSnapshot=200}"; assertThat(new TieringSnapshotSplit(path, bucket, partitionName, 0L, 200L, 30).toString()) .isEqualTo(expectedSplitString); } @@ -102,9 +103,52 @@ void testTieringLogSplitStringExpression(Boolean isPartitionedTable) throws Exce String expectedSplitString = isPartitionedTable - ? "TieringLogSplit{tablePath=test_db.test_partitioned_table, tableBucket=TableBucket{tableId=1, partitionId=100, bucket=2}, partitionName='1024', startingOffset=100, stoppingOffset=200, numberOfSplits=2}" - : "TieringLogSplit{tablePath=test_db.test_table, tableBucket=TableBucket{tableId=1, bucket=2}, partitionName='null', startingOffset=100, stoppingOffset=200, numberOfSplits=2}"; + ? "TieringLogSplit{tablePath=test_db.test_partitioned_table, tableBucket=TableBucket{tableId=1, partitionId=100, bucket=2}, partitionName='1024', numberOfSplits=2, forceIgnore=false, startingOffset=100, stoppingOffset=200}" + : "TieringLogSplit{tablePath=test_db.test_table, tableBucket=TableBucket{tableId=1, bucket=2}, partitionName='null', numberOfSplits=2, forceIgnore=false, startingOffset=100, stoppingOffset=200}"; assertThat(new TieringLogSplit(path, bucket, partitionName, 100, 200, 2).toString()) .isEqualTo(expectedSplitString); } + + @Test + void testForceIgnoreSerde() throws Exception { + // Test TieringSnapshotSplit with forceIgnore set at creation + TieringSnapshotSplit snapshotSplitWithForceIgnore = + new TieringSnapshotSplit(tablePath, tableBucket, null, 0L, 200L, 10, true); + byte[] serialized = serializer.serialize(snapshotSplitWithForceIgnore); + TieringSnapshotSplit deserializedSnapshotSplit = + (TieringSnapshotSplit) serializer.deserialize(serializer.getVersion(), serialized); + assertThat(deserializedSnapshotSplit).isEqualTo(snapshotSplitWithForceIgnore); + + // Test TieringLogSplit with forceIgnore set at creation + TieringLogSplit logSplitWithForceIgnore = + new TieringLogSplit(tablePath, tableBucket, null, 100, 200, true, 40); + serialized = serializer.serialize(logSplitWithForceIgnore); + TieringLogSplit deserializedLogSplit = + (TieringLogSplit) serializer.deserialize(serializer.getVersion(), serialized); + assertThat(deserializedLogSplit).isEqualTo(logSplitWithForceIgnore); + + // Test TieringSnapshotSplit with forceIgnore set after creation + TieringSnapshotSplit snapshotSplit = + new TieringSnapshotSplit(tablePath, tableBucket, null, 0L, 200L, 10, false); + assertThat(snapshotSplit.isForceIgnore()).isFalse(); + snapshotSplit.forceIgnore(); + assertThat(snapshotSplit.isForceIgnore()).isTrue(); + + serialized = serializer.serialize(snapshotSplit); + deserializedSnapshotSplit = + (TieringSnapshotSplit) serializer.deserialize(serializer.getVersion(), serialized); + assertThat(deserializedSnapshotSplit).isEqualTo(snapshotSplit); + + // Test TieringLogSplit with forceIgnore set after creation + TieringLogSplit logSplit = + new TieringLogSplit(tablePath, tableBucket, null, 100, 200, false, 40); + assertThat(logSplit.isForceIgnore()).isFalse(); + logSplit.forceIgnore(); + assertThat(logSplit.isForceIgnore()).isTrue(); + + serialized = serializer.serialize(logSplit); + deserializedLogSplit = + (TieringLogSplit) serializer.deserialize(serializer.getVersion(), serialized); + assertThat(deserializedLogSplit).isEqualTo(logSplit); + } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/lakehouse/TestingPaimonStoragePlugin.java b/fluss-server/src/test/java/org/apache/fluss/server/lakehouse/TestingPaimonStoragePlugin.java index aa030d0256..d21c7174ce 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/lakehouse/TestingPaimonStoragePlugin.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/lakehouse/TestingPaimonStoragePlugin.java @@ -17,23 +17,42 @@ package org.apache.fluss.server.lakehouse; +import org.apache.fluss.config.ConfigOption; import org.apache.fluss.config.Configuration; import org.apache.fluss.exception.TableAlreadyExistException; import org.apache.fluss.exception.TableNotExistException; +import org.apache.fluss.lake.committer.CommittedLakeSnapshot; +import org.apache.fluss.lake.committer.CommitterInitContext; +import org.apache.fluss.lake.committer.LakeCommitter; import org.apache.fluss.lake.lakestorage.LakeCatalog; import org.apache.fluss.lake.lakestorage.LakeStorage; import org.apache.fluss.lake.lakestorage.LakeStoragePlugin; +import org.apache.fluss.lake.serializer.SimpleVersionedSerializer; import org.apache.fluss.lake.source.LakeSource; import org.apache.fluss.lake.writer.LakeTieringFactory; +import org.apache.fluss.lake.writer.LakeWriter; +import org.apache.fluss.lake.writer.WriterInitContext; import org.apache.fluss.metadata.DataLakeFormat; import org.apache.fluss.metadata.TableChange; import org.apache.fluss.metadata.TableDescriptor; +import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.record.LogRecord; +import javax.annotation.Nullable; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.time.Duration; import java.util.HashMap; import java.util.List; import java.util.Map; +import static org.apache.fluss.config.ConfigBuilder.key; + /** A plugin of paimon just for testing purpose. */ public class TestingPaimonStoragePlugin implements LakeStoragePlugin { @@ -54,7 +73,7 @@ public static class TestingPaimonLakeStorage implements LakeStorage { @Override public LakeTieringFactory createLakeTieringFactory() { - throw new UnsupportedOperationException("createLakeTieringFactory is not supported."); + return new TestingPaimonTieringFactory(); } @Override @@ -96,4 +115,154 @@ public TableDescriptor getTable(TablePath tablePath) { return tableByPath.get(tablePath); } } + + private static class TestingPaimonTieringFactory + implements LakeTieringFactory { + + @Override + public LakeWriter createLakeWriter( + WriterInitContext writerInitContext) { + return new TestingPaimonWriter(writerInitContext.tableInfo()); + } + + @Override + public SimpleVersionedSerializer getWriteResultSerializer() { + return new TestingPaimonWriteResultSerializer(); + } + + @Override + public LakeCommitter createLakeCommitter( + CommitterInitContext committerInitContext) throws IOException { + return new TestingPaimonCommitter(); + } + + @Override + public SimpleVersionedSerializer getCommittableSerializer() { + return new SimpleVersionedSerializer() { + @Override + public int getVersion() { + return 0; + } + + @Override + public byte[] serialize(TestPaimonCommittable obj) throws IOException { + return new byte[0]; + } + + @Override + public TestPaimonCommittable deserialize(int version, byte[] serialized) + throws IOException { + return new TestPaimonCommittable(); + } + }; + } + } + + private static class TestingPaimonWriter implements LakeWriter { + + static ConfigOption writePauseOption = + key("write-pause").durationType().noDefaultValue(); + + private int writtenRecords = 0; + private final Duration writePause; + + private TestingPaimonWriter(TableInfo tableInfo) { + this.writePause = tableInfo.getCustomProperties().get(writePauseOption); + } + + @Override + public void write(LogRecord record) throws IOException { + try { + if (writePause != null) { + Thread.sleep(writePause.toMillis()); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException("Interrupted while pausing before write", e); + } + writtenRecords += 1; + } + + @Override + public TestingPaimonWriteResult complete() throws IOException { + return new TestingPaimonWriteResult(writtenRecords); + } + + @Override + public void close() throws IOException { + // do nothing + } + } + + private static class TestingPaimonWriteResult { + private final int writtenRecords; + + public TestingPaimonWriteResult(int writtenRecords) { + this.writtenRecords = writtenRecords; + } + } + + private static class TestingPaimonWriteResultSerializer + implements SimpleVersionedSerializer { + + @Override + public int getVersion() { + return 1; + } + + @Override + public byte[] serialize(TestingPaimonWriteResult result) throws IOException { + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + ObjectOutputStream oos = new ObjectOutputStream(baos)) { + oos.writeInt(result.writtenRecords); + return baos.toByteArray(); + } + } + + @Override + public TestingPaimonWriteResult deserialize(int version, byte[] serialized) + throws IOException { + try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized); + ObjectInputStream ois = new ObjectInputStream(bais)) { + return new TestingPaimonWriteResult(ois.readInt()); + } + } + } + + private static class TestPaimonCommittable {} + + private static class TestingPaimonCommitter + implements LakeCommitter { + + @Override + public TestPaimonCommittable toCommittable( + List testingPaimonWriteResults) throws IOException { + return new TestPaimonCommittable(); + } + + @Override + public long commit( + TestPaimonCommittable committable, Map snapshotProperties) + throws IOException { + // do nothing, and always return 1 as committed snapshot + return 1; + } + + @Override + public void abort(TestPaimonCommittable committable) throws IOException { + // do nothing + } + + @Nullable + @Override + public CommittedLakeSnapshot getMissingLakeSnapshot( + @Nullable Long latestLakeSnapshotIdOfFluss) throws IOException { + return null; + } + + @Override + public void close() throws Exception { + // do nothing + } + } } diff --git a/fluss-test-coverage/pom.xml b/fluss-test-coverage/pom.xml index 16e942860e..de785f7163 100644 --- a/fluss-test-coverage/pom.xml +++ b/fluss-test-coverage/pom.xml @@ -448,6 +448,7 @@ org.apache.fluss.flink.tiering.source.TieringWriterInitContext org.apache.fluss.flink.tiering.source.TieringSourceReader + org.apache.fluss.flink.tiering.source.TieringSourceFetcherManager org.apache.fluss.flink.tiering.source.TableBucketWriteResultEmitter From f2bd2262a844fafcf5edccce45e24c26422ff767 Mon Sep 17 00:00:00 2001 From: luoyuxia Date: Thu, 15 Jan 2026 15:43:26 +0800 Subject: [PATCH 2/5] address comments --- .../tiering/source/TieringSplitReader.java | 5 +- .../enumerator/TieringSourceEnumerator.java | 41 +++-- .../tiering/source/split/TieringLogSplit.java | 18 +-- .../source/split/TieringSnapshotSplit.java | 10 +- .../tiering/source/split/TieringSplit.java | 37 +++-- .../source/split/TieringSplitSerializer.java | 24 +-- .../flink/tiering/FlinkTieringTestBase.java | 76 ++++++++-- .../flink/tiering/TieringFailoverITCase.java | 82 ++-------- .../fluss/flink/tiering/TieringITCase.java | 142 +++--------------- .../source/TieringSourceReaderTest.java | 4 +- .../TieringSourceEnumeratorTest.java | 19 +-- .../split/TieringSplitSerializerTest.java | 46 +++--- .../TestingValuesLakeTieringFactory.java | 2 +- .../tiering/TestingValuesLakeWriter.java | 21 ++- 14 files changed, 222 insertions(+), 305 deletions(-) diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java index 2a797add33..66e903a0da 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java @@ -180,11 +180,11 @@ public void handleSplitsChanges(SplitsChange splitsChange) { } for (TieringSplit split : splitsChange.splits()) { LOG.info("add split {}", split.splitId()); - if (split.isForceIgnore()) { + if (split.shouldSkipCurrentRound()) { // if the split is forced to ignore, // mark it as empty LOG.info( - "ignore split {} since the split is set to force to ignore", + "ignore split {} since the split is set to skip the current round of tiering.", split.splitId()); currentEmptySplits.add(split); continue; @@ -551,6 +551,7 @@ private void finishCurrentTable() throws IOException { public void handleTableReachTieringMaxDuration(long tableId) { if ((currentTableId != null && currentTableId.equals(tableId)) || pendingTieringSplits.containsKey(tableId)) { + LOG.info("Table {} reach tiering max duration, will force to complete.", tableId); reachTieringMaxDurationTables.add(tableId); } } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumerator.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumerator.java index 5907a30a6e..9bc157999b 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumerator.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumerator.java @@ -149,8 +149,8 @@ public TieringSourceEnumerator( this.pollTieringTableIntervalMs = pollTieringTableIntervalMs; this.tieringTableDurationMaxMs = tieringTableDurationMaxMs; this.tieringTableDurationDetectIntervalMs = tieringTableDurationDetectIntervalMs; - this.pendingSplits = new ArrayList<>(); - this.readersAwaitingSplit = new TreeSet<>(); + this.pendingSplits = Collections.synchronizedList(new ArrayList<>()); + this.readersAwaitingSplit = Collections.synchronizedSet(new TreeSet<>()); this.tieringTableEpochs = MapUtils.newConcurrentHashMap(); this.finishedTableEpochs = MapUtils.newConcurrentHashMap(); this.failedTableEpochs = MapUtils.newConcurrentHashMap(); @@ -350,10 +350,9 @@ private void handleReachMaxTieringDurationTables( for (Long reachMaxDurationTable : tieringReachMaxDurationTables) { for (TieringSplit tieringSplit : pendingSplits) { if (tieringSplit.getTableBucket().getTableId() == reachMaxDurationTable) { - // force ignore this tiering split since the tiering for this table is timeout, - // we have to force to set to ignore the tiering split so that the - // tiering source reader can ignore them directly - tieringSplit.forceIgnore(); + // mark this tiering split to skip the current round since the tiering for + // this table has timed out, so the tiering source reader can skip them directly + tieringSplit.skipCurrentRound(); } else { // we can break directly, if found any one split's table id is not equal to the // timeout @@ -362,7 +361,7 @@ private void handleReachMaxTieringDurationTables( } } - LOG.debug("Found the table {} reach max tiering duration.", reachMaxDurationTable); + LOG.info("Found the table {} reach max tiering duration.", reachMaxDurationTable); // broadcast the tiering reach max duration event to all readers, // we broadcast all for simplicity @@ -386,25 +385,21 @@ private void generateAndAssignSplits( } private void assignSplits() { - // we don't assign splits during failovering + // we don't assign splits during failover if (isFailOvering) { return; } - /* This method may be called from both addSplitsBack and handleSplitRequest, make it thread safe. */ - // todo: do we need to add lock? - synchronized (readersAwaitingSplit) { - if (!readersAwaitingSplit.isEmpty()) { - final Integer[] readers = readersAwaitingSplit.toArray(new Integer[0]); - for (Integer nextAwaitingReader : readers) { - if (!context.registeredReaders().containsKey(nextAwaitingReader)) { - readersAwaitingSplit.remove(nextAwaitingReader); - continue; - } - if (!pendingSplits.isEmpty()) { - TieringSplit tieringSplit = pendingSplits.remove(0); - context.assignSplit(tieringSplit, nextAwaitingReader); - readersAwaitingSplit.remove(nextAwaitingReader); - } + if (!readersAwaitingSplit.isEmpty()) { + final Integer[] readers = readersAwaitingSplit.toArray(new Integer[0]); + for (Integer nextAwaitingReader : readers) { + if (!context.registeredReaders().containsKey(nextAwaitingReader)) { + readersAwaitingSplit.remove(nextAwaitingReader); + continue; + } + if (!pendingSplits.isEmpty()) { + TieringSplit tieringSplit = pendingSplits.remove(0); + context.assignSplit(tieringSplit, nextAwaitingReader); + readersAwaitingSplit.remove(nextAwaitingReader); } } } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringLogSplit.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringLogSplit.java index ac8c783af2..afd8d75f8f 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringLogSplit.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringLogSplit.java @@ -60,8 +60,8 @@ public TieringLogSplit( partitionName, startingOffset, stoppingOffset, - false, - numberOfSplits); + numberOfSplits, + false); } public TieringLogSplit( @@ -70,9 +70,9 @@ public TieringLogSplit( @Nullable String partitionName, long startingOffset, long stoppingOffset, - boolean forceIgnore, - int numberOfSplits) { - super(tablePath, tableBucket, partitionName, forceIgnore, numberOfSplits); + int numberOfSplits, + boolean skipCurrentRound) { + super(tablePath, tableBucket, partitionName, numberOfSplits, skipCurrentRound); this.startingOffset = startingOffset; this.stoppingOffset = stoppingOffset; } @@ -102,8 +102,8 @@ public String toString() { + '\'' + ", numberOfSplits=" + numberOfSplits - + ", forceIgnore=" - + forceIgnore + + ", skipCurrentRound=" + + skipCurrentRound + ", startingOffset=" + startingOffset + ", stoppingOffset=" @@ -119,8 +119,8 @@ public TieringLogSplit copy(int numberOfSplits) { partitionName, startingOffset, stoppingOffset, - forceIgnore, - numberOfSplits); + numberOfSplits, + skipCurrentRound); } @Override diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSnapshotSplit.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSnapshotSplit.java index 0044659d2b..e6b37a925f 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSnapshotSplit.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSnapshotSplit.java @@ -62,8 +62,8 @@ public TieringSnapshotSplit( long snapshotId, long logOffsetOfSnapshot, int numberOfSplits, - boolean forceIgnore) { - super(tablePath, tableBucket, partitionName, forceIgnore, numberOfSplits); + boolean skipCurrentRound) { + super(tablePath, tableBucket, partitionName, numberOfSplits, skipCurrentRound); this.snapshotId = snapshotId; this.logOffsetOfSnapshot = logOffsetOfSnapshot; } @@ -93,8 +93,8 @@ public String toString() { + '\'' + ", numberOfSplits=" + numberOfSplits - + ", forceIgnore=" - + forceIgnore + + ", skipCurrentRound=" + + skipCurrentRound + ", snapshotId=" + snapshotId + ", logOffsetOfSnapshot=" @@ -111,7 +111,7 @@ public TieringSnapshotSplit copy(int numberOfSplits) { snapshotId, logOffsetOfSnapshot, numberOfSplits, - forceIgnore); + skipCurrentRound); } @Override diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSplit.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSplit.java index cbd3482a8c..d6827a62cf 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSplit.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSplit.java @@ -38,16 +38,21 @@ public abstract class TieringSplit implements SourceSplit { protected final TableBucket tableBucket; @Nullable protected final String partitionName; - protected boolean forceIgnore; // the total number of splits in one round of tiering protected final int numberOfSplits; + /** + * Indicates whether to skip tiering data for this split in the current round of tiering. When + * set to true, the split will not be processed and tiering for the split will be skipped. + */ + protected boolean skipCurrentRound; + public TieringSplit( TablePath tablePath, TableBucket tableBucket, @Nullable String partitionName, - boolean forceIgnore, - int numberOfSplits) { + int numberOfSplits, + boolean skipCurrentRound) { this.tablePath = tablePath; this.tableBucket = tableBucket; this.partitionName = partitionName; @@ -56,8 +61,8 @@ public TieringSplit( throw new IllegalArgumentException( "Partition name and partition id must be both null or both not null."); } - this.forceIgnore = forceIgnore; this.numberOfSplits = numberOfSplits; + this.skipCurrentRound = skipCurrentRound; } /** Checks whether this split is a primary key table split to tier. */ @@ -75,12 +80,21 @@ public final boolean isTieringLogSplit() { return getClass() == TieringLogSplit.class; } - public void forceIgnore() { - this.forceIgnore = true; + /** + * Marks this split to skip reading data in the current round. Once called, the split will not + * be processed and data reading will be skipped. + */ + public void skipCurrentRound() { + this.skipCurrentRound = true; } - public boolean isForceIgnore() { - return forceIgnore; + /** + * Returns whether this split should skip tiering data in the current round of tiering. + * + * @return true if the split should skip tiering data, false otherwise + */ + public boolean shouldSkipCurrentRound() { + return skipCurrentRound; } /** Casts this split into a {@link TieringLogSplit}. */ @@ -139,12 +153,13 @@ public boolean equals(Object object) { return Objects.equals(tablePath, that.tablePath) && Objects.equals(tableBucket, that.tableBucket) && Objects.equals(partitionName, that.partitionName) - && forceIgnore == that.forceIgnore - && numberOfSplits == that.numberOfSplits; + && numberOfSplits == that.numberOfSplits + && skipCurrentRound == that.skipCurrentRound; } @Override public int hashCode() { - return Objects.hash(tablePath, tableBucket, partitionName, forceIgnore, numberOfSplits); + return Objects.hash( + tablePath, tableBucket, partitionName, numberOfSplits, skipCurrentRound); } } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSplitSerializer.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSplitSerializer.java index 85b3429afe..62c4f78868 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSplitSerializer.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSplitSerializer.java @@ -17,6 +17,8 @@ package org.apache.fluss.flink.tiering.source.split; +import org.apache.fluss.flink.tiering.source.TieringSource; +import org.apache.fluss.flink.tiering.source.enumerator.TieringSourceEnumerator; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TablePath; @@ -26,7 +28,13 @@ import java.io.IOException; -/** A serializer for the {@link TieringSplit}. */ +/** + * A serializer for the {@link TieringSplit}. + * + *

This serializer is only used to serialize and deserialize splits sent from {@link + * TieringSourceEnumerator} to {@link TieringSource} for network transmission. Therefore, it does + * not need to consider compatibility. + */ public class TieringSplitSerializer implements SimpleVersionedSerializer { public static final TieringSplitSerializer INSTANCE = new TieringSplitSerializer(); @@ -74,11 +82,10 @@ public byte[] serialize(TieringSplit split) throws IOException { out.writeBoolean(false); } - // write force ignore - out.writeBoolean(split.isForceIgnore()); - // write number of splits out.writeInt(split.getNumberOfSplits()); + // write skipCurrentRound + out.writeBoolean(split.shouldSkipCurrentRound()); if (split.isTieringSnapshotSplit()) { // Snapshot split TieringSnapshotSplit tieringSnapshotSplit = split.asTieringSnapshotSplit(); @@ -129,10 +136,9 @@ public TieringSplit deserialize(int version, byte[] serialized) throws IOExcepti } TableBucket tableBucket = new TableBucket(tableId, partitionId, bucketId); - boolean forceIgnore = in.readBoolean(); - // deserialize number of splits int numberOfSplits = in.readInt(); + boolean skipCurrentRound = in.readBoolean(); if (splitKind == TIERING_SNAPSHOT_SPLIT_FLAG) { // deserialize snapshot id @@ -146,7 +152,7 @@ public TieringSplit deserialize(int version, byte[] serialized) throws IOExcepti snapshotId, logOffsetOfSnapshot, numberOfSplits, - forceIgnore); + skipCurrentRound); } else { // deserialize starting offset long startingOffset = in.readLong(); @@ -158,8 +164,8 @@ public TieringSplit deserialize(int version, byte[] serialized) throws IOExcepti partitionName, startingOffset, stoppingOffset, - forceIgnore, - numberOfSplits); + numberOfSplits, + skipCurrentRound); } } } diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/FlinkTieringTestBase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/FlinkTieringTestBase.java index 130a308126..7d0518f83f 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/FlinkTieringTestBase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/FlinkTieringTestBase.java @@ -22,6 +22,7 @@ import org.apache.fluss.client.ConnectionFactory; import org.apache.fluss.client.admin.Admin; import org.apache.fluss.client.table.Table; +import org.apache.fluss.client.table.writer.AppendWriter; import org.apache.fluss.client.table.writer.TableWriter; import org.apache.fluss.client.table.writer.UpsertWriter; import org.apache.fluss.config.ConfigOptions; @@ -37,6 +38,7 @@ import org.apache.fluss.server.testutils.FlussClusterExtension; import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.core.execution.JobClient; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; @@ -44,8 +46,11 @@ import org.junit.jupiter.api.extension.RegisterExtension; import java.time.Duration; +import java.util.Collections; import java.util.List; +import java.util.Map; +import static org.apache.fluss.flink.tiering.source.TieringSourceOptions.POLL_TIERING_TABLE_INTERVAL; import static org.apache.fluss.testutils.common.CommonTestUtils.retry; import static org.assertj.core.api.Assertions.assertThat; @@ -69,7 +74,7 @@ private static Configuration initConfig() { Configuration conf = new Configuration(); conf.set(ConfigOptions.KV_MAX_RETAINED_SNAPSHOTS, Integer.MAX_VALUE); - // Configure the tiering sink to be Lance + // Configure the tiering sink to be Lance for testing purpose conf.set(ConfigOptions.DATALAKE_FORMAT, DataLakeFormat.LANCE); return conf; } @@ -91,29 +96,38 @@ static void afterAll() throws Exception { conn.close(); conn = null; } + + System.out.println("after all"); } @BeforeEach - public void beforeEach() { + void beforeEach() { execEnv = StreamExecutionEnvironment.getExecutionEnvironment(); execEnv.setRuntimeMode(RuntimeExecutionMode.STREAMING); execEnv.setParallelism(2); + execEnv.enableCheckpointing(500); + } + + protected long createTable(TablePath tablePath, Schema schema) throws Exception { + return createTable(tablePath, 1, Collections.emptyList(), schema, Collections.emptyMap()); } - protected long createPkTable( - TablePath tablePath, int bucketNum, boolean enableAutoCompaction, Schema schema) + protected long createTable( + TablePath tablePath, + int bucketNum, + List bucketKeys, + Schema schema, + Map customProperties) throws Exception { - TableDescriptor.Builder pkTableBuilder = + TableDescriptor.Builder tableBuilder = TableDescriptor.builder() .schema(schema) - .distributedBy(bucketNum) + .distributedBy(bucketNum, bucketKeys) .property(ConfigOptions.TABLE_DATALAKE_ENABLED.key(), "true") - .property(ConfigOptions.TABLE_DATALAKE_FRESHNESS, Duration.ofMillis(500)); + .property(ConfigOptions.TABLE_DATALAKE_FRESHNESS, Duration.ofMillis(500)) + .customProperties(customProperties); - if (enableAutoCompaction) { - pkTableBuilder.property(ConfigOptions.TABLE_DATALAKE_AUTO_COMPACTION.key(), "true"); - } - return createTable(tablePath, pkTableBuilder.build()); + return createTable(tablePath, tableBuilder.build()); } protected long createTable(TablePath tablePath, TableDescriptor tableDescriptor) @@ -138,18 +152,52 @@ protected Replica getLeaderReplica(TableBucket tableBucket) { return FLUSS_CLUSTER_EXTENSION.waitAndGetLeaderReplica(tableBucket); } - protected void writeRows(TablePath tablePath, List rows) throws Exception { + protected void writeRows(TablePath tablePath, List rows, boolean append) + throws Exception { try (Table table = conn.getTable(tablePath)) { TableWriter tableWriter; - tableWriter = table.newUpsert().createWriter(); + if (append) { + tableWriter = table.newAppend().createWriter(); + } else { + tableWriter = table.newUpsert().createWriter(); + } for (InternalRow row : rows) { - ((UpsertWriter) tableWriter).upsert(row); + if (tableWriter instanceof AppendWriter) { + ((AppendWriter) tableWriter).append(row); + } else { + ((UpsertWriter) tableWriter).upsert(row); + } } tableWriter.flush(); } } + protected void waitUntilSnapshot(long tableId, int bucketNum, long snapshotId) { + for (int i = 0; i < bucketNum; i++) { + TableBucket tableBucket = new TableBucket(tableId, i); + FLUSS_CLUSTER_EXTENSION.waitUntilSnapshotFinished(tableBucket, snapshotId); + } + } + public List getValuesRecords(TablePath tablePath) { return TestingValuesLake.getResults(tablePath.toString()); } + + protected JobClient buildTieringJob(StreamExecutionEnvironment execEnv) throws Exception { + return buildTieringJob(execEnv, new Configuration()); + } + + protected JobClient buildTieringJob( + StreamExecutionEnvironment execEnv, Configuration lakeTieringConfig) throws Exception { + Configuration flussConfig = new Configuration(clientConf); + flussConfig.set(POLL_TIERING_TABLE_INTERVAL, Duration.ofMillis(500L)); + + return LakeTieringJobBuilder.newBuilder( + execEnv, + flussConfig, + new Configuration(), + lakeTieringConfig, + DataLakeFormat.LANCE.toString()) + .build(); + } } diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/TieringFailoverITCase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/TieringFailoverITCase.java index 8470312252..07b2ddbd08 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/TieringFailoverITCase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/TieringFailoverITCase.java @@ -18,38 +18,23 @@ package org.apache.fluss.flink.tiering; -import org.apache.fluss.config.Configuration; -import org.apache.fluss.flink.tiering.committer.CommittableMessageTypeInfo; -import org.apache.fluss.flink.tiering.committer.TieringCommitOperatorFactory; -import org.apache.fluss.flink.tiering.source.TableBucketWriteResultTypeInfo; -import org.apache.fluss.flink.tiering.source.TieringSource; import org.apache.fluss.lake.values.TestingValuesLake; -import org.apache.fluss.lake.values.tiering.TestingValuesLakeTieringFactory; -import org.apache.fluss.lake.writer.LakeTieringFactory; import org.apache.fluss.metadata.Schema; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.row.InternalRow; import org.apache.fluss.types.DataTypes; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.configuration.PipelineOptions; import org.apache.flink.core.execution.JobClient; -import org.apache.flink.streaming.api.datastream.DataStreamSource; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; +import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; -import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.Iterator; import java.util.List; -import static org.apache.fluss.flink.tiering.source.TieringSource.TIERING_SOURCE_TRANSFORMATION_UID; -import static org.apache.fluss.flink.tiering.source.TieringSourceOptions.POLL_TIERING_TABLE_INTERVAL; import static org.apache.fluss.testutils.DataTestUtils.row; import static org.assertj.core.api.Assertions.assertThat; @@ -57,8 +42,6 @@ class TieringFailoverITCase extends FlinkTieringTestBase { protected static final String DEFAULT_DB = "fluss"; - private static StreamExecutionEnvironment execEnv; - private static final Schema schema = Schema.newBuilder() .column("f_int", DataTypes.INT()) @@ -69,21 +52,23 @@ class TieringFailoverITCase extends FlinkTieringTestBase { @BeforeAll protected static void beforeAll() { FlinkTieringTestBase.beforeAll(); - execEnv = StreamExecutionEnvironment.getExecutionEnvironment(); - execEnv.setParallelism(2); - execEnv.enableCheckpointing(1000); + } + + @AfterAll + protected static void afterAll() throws Exception { + FlinkTieringTestBase.afterAll(); } @Test void testTiering() throws Exception { // create a pk table, write some records and wait until snapshot finished TablePath t1 = TablePath.of(DEFAULT_DB, "pkTable"); - long t1Id = createPkTable(t1, 1, false, schema); + long t1Id = createTable(t1, schema); TableBucket t1Bucket = new TableBucket(t1Id, 0); // write records List rows = Arrays.asList(row(1, "i1"), row(2, "i2"), row(3, "i3")); List expectedRows = new ArrayList<>(rows); - writeRows(t1, rows); + writeRows(t1, rows, false); FLUSS_CLUSTER_EXTENSION.triggerAndWaitSnapshot(t1); // fail the first write to the pk table @@ -102,7 +87,7 @@ void testTiering() throws Exception { rows = Arrays.asList(row(1, "i11"), row(2, "i22"), row(3, "i33")); expectedRows.addAll(rows); // write records - writeRows(t1, rows); + writeRows(t1, rows, false); // check the status of replica of t1 after synced // not check start offset since we won't @@ -115,54 +100,7 @@ void testTiering() throws Exception { } } - @SuppressWarnings({"rawtypes", "unchecked"}) - protected JobClient buildTieringJob(StreamExecutionEnvironment execEnv) throws Exception { - Configuration flussConfig = new Configuration(clientConf); - flussConfig.set(POLL_TIERING_TABLE_INTERVAL, Duration.ofMillis(500L)); - - LakeTieringFactory lakeTieringFactory = new TestingValuesLakeTieringFactory(); - - // build tiering source - TieringSource.Builder tieringSourceBuilder = - new TieringSource.Builder<>(flussConfig, lakeTieringFactory); - if (flussConfig.get(POLL_TIERING_TABLE_INTERVAL) != null) { - tieringSourceBuilder.withPollTieringTableIntervalMs( - flussConfig.get(POLL_TIERING_TABLE_INTERVAL).toMillis()); - } - TieringSource tieringSource = tieringSourceBuilder.build(); - DataStreamSource source = - execEnv.fromSource( - tieringSource, - WatermarkStrategy.noWatermarks(), - "TieringSource", - TableBucketWriteResultTypeInfo.of( - () -> lakeTieringFactory.getWriteResultSerializer())); - - source.getTransformation().setUid(TIERING_SOURCE_TRANSFORMATION_UID); - - source.transform( - "TieringCommitter", - CommittableMessageTypeInfo.of( - () -> lakeTieringFactory.getCommittableSerializer()), - new TieringCommitOperatorFactory( - flussConfig, - Configuration.fromMap(Collections.emptyMap()), - lakeTieringFactory)) - .setParallelism(1) - .setMaxParallelism(1) - .sinkTo(new DiscardingSink()) - .name("end") - .setParallelism(1); - String jobName = - execEnv.getConfiguration() - .getOptional(PipelineOptions.NAME) - .orElse("Fluss Lake Tiering FailOver IT Test."); - - return execEnv.executeAsync(jobName); - } - - private void checkDataInValuesTable(TablePath tablePath, List expectedRows) - throws Exception { + private void checkDataInValuesTable(TablePath tablePath, List expectedRows) { Iterator actualIterator = getValuesRecords(tablePath).iterator(); Iterator iterator = expectedRows.iterator(); while (iterator.hasNext() && actualIterator.hasNext()) { diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/TieringITCase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/TieringITCase.java index ce8c20c41e..9b9939682f 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/TieringITCase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/TieringITCase.java @@ -18,27 +18,14 @@ package org.apache.fluss.flink.tiering; -import org.apache.fluss.client.Connection; -import org.apache.fluss.client.ConnectionFactory; -import org.apache.fluss.client.admin.Admin; import org.apache.fluss.client.metadata.LakeSnapshot; -import org.apache.fluss.client.table.Table; -import org.apache.fluss.client.table.writer.AppendWriter; -import org.apache.fluss.client.table.writer.TableWriter; -import org.apache.fluss.client.table.writer.UpsertWriter; -import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; -import org.apache.fluss.exception.FlussRuntimeException; import org.apache.fluss.exception.LakeTableSnapshotNotExistException; -import org.apache.fluss.metadata.DataLakeFormat; import org.apache.fluss.metadata.Schema; -import org.apache.fluss.metadata.TableBucket; -import org.apache.fluss.metadata.TableDescriptor; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.row.BinaryString; import org.apache.fluss.row.GenericRow; import org.apache.fluss.row.InternalRow; -import org.apache.fluss.server.testutils.FlussClusterExtension; import org.apache.fluss.types.DataTypes; import org.apache.fluss.utils.ExceptionUtils; @@ -47,10 +34,9 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.RegisterExtension; -import java.nio.file.Files; import java.time.Duration; import java.util.ArrayList; import java.util.Collections; @@ -64,43 +50,20 @@ import static org.assertj.core.api.Assertions.assertThat; /** The IT case for tiering. */ -class TieringITCase { +class TieringITCase extends FlinkTieringTestBase { - @RegisterExtension - public static final FlussClusterExtension FLUSS_CLUSTER_EXTENSION = - FlussClusterExtension.builder() - .setClusterConf(initConfig()) - .setNumOfTabletServers(3) - .build(); - - protected static String warehousePath; - protected static Connection conn; - protected static Admin admin; - protected static StreamExecutionEnvironment execEnv; + @BeforeAll + protected static void beforeAll() { + FlinkTieringTestBase.beforeAll(); + } - protected static Configuration initConfig() { - Configuration conf = new Configuration(); - conf.set(ConfigOptions.KV_SNAPSHOT_INTERVAL, Duration.ofSeconds(1)) - // not to clean snapshots for test purpose - .set(ConfigOptions.KV_MAX_RETAINED_SNAPSHOTS, Integer.MAX_VALUE); - conf.setString("datalake.format", "paimon"); - conf.setString("datalake.paimon.metastore", "filesystem"); - try { - warehousePath = - Files.createTempDirectory("fluss-testing-datalake-tiered") - .resolve("warehouse") - .toString(); - } catch (Exception e) { - throw new FlussRuntimeException("Failed to create warehouse path"); - } - conf.setString("datalake.paimon.warehouse", warehousePath); - return conf; + @AfterAll + protected static void afterAll() throws Exception { + FlinkTieringTestBase.afterAll(); } - @BeforeAll - static void beforeAll() { - conn = ConnectionFactory.createConnection(FLUSS_CLUSTER_EXTENSION.getClientConfig()); - admin = conn.getAdmin(); + @BeforeEach + void beforeEach() { execEnv = StreamExecutionEnvironment.getExecutionEnvironment() .setParallelism(1) @@ -131,7 +94,11 @@ void testTieringReachMaxDuration() throws Exception { waitUntilSnapshot(pkTableId, 3, 0); - JobClient jobClient = buildTieringJob(execEnv); + // set tiering duration to a small value for testing purpose + Configuration lakeTieringConfig = new Configuration(); + lakeTieringConfig.set(LAKE_TIERING_TABLE_DURATION_MAX, Duration.ofSeconds(1)); + lakeTieringConfig.set(LAKE_TIERING_TABLE_DURATION_DETECT_INTERVAL, Duration.ofMillis(100)); + JobClient jobClient = buildTieringJob(execEnv, lakeTieringConfig); try { // verify the tiered records is less than the table total record to @@ -150,18 +117,6 @@ void testTieringReachMaxDuration() throws Exception { } } - @AfterAll - static void afterAll() throws Exception { - if (admin != null) { - admin.close(); - admin = null; - } - if (conn != null) { - conn.close(); - conn = null; - } - } - private long countTieredRecords(LakeSnapshot lakeSnapshot) throws Exception { return lakeSnapshot.getTableBucketsOffset().values().stream() .mapToLong(Long::longValue) @@ -191,68 +146,15 @@ private long createTable(TablePath tablePath, boolean isPrimaryKeyTable) throws if (isPrimaryKeyTable) { schemaBuilder.primaryKey("a"); } - TableDescriptor.Builder tableDescriptorBuilder = - TableDescriptor.builder() - .schema(schemaBuilder.build()) - .distributedBy(3, "a") - .property(ConfigOptions.TABLE_DATALAKE_ENABLED.key(), "true") - .property(ConfigOptions.TABLE_DATALAKE_FRESHNESS, Duration.ofMillis(500)); // see TestingPaimonStoragePlugin#TestingPaimonWriter, we set write-pause // to 1s to make it easy to mock tiering reach max duration Map customProperties = Collections.singletonMap("write-pause", "1s"); - tableDescriptorBuilder.customProperties(customProperties); - return createTable(tablePath, tableDescriptorBuilder.build()); - } - - protected long createTable(TablePath tablePath, TableDescriptor tableDescriptor) - throws Exception { - admin.createTable(tablePath, tableDescriptor, true).get(); - return admin.getTableInfo(tablePath).get().getTableId(); - } - - private void writeRows(TablePath tablePath, List rows, boolean append) - throws Exception { - try (Table table = conn.getTable(tablePath)) { - TableWriter tableWriter; - if (append) { - tableWriter = table.newAppend().createWriter(); - } else { - tableWriter = table.newUpsert().createWriter(); - } - for (InternalRow row : rows) { - if (tableWriter instanceof AppendWriter) { - ((AppendWriter) tableWriter).append(row); - } else { - ((UpsertWriter) tableWriter).upsert(row); - } - } - tableWriter.flush(); - } - } - - private JobClient buildTieringJob(StreamExecutionEnvironment execEnv) throws Exception { - Configuration lakeTieringConfig = new Configuration(); - lakeTieringConfig.set(LAKE_TIERING_TABLE_DURATION_MAX, Duration.ofSeconds(1)); - lakeTieringConfig.set(LAKE_TIERING_TABLE_DURATION_DETECT_INTERVAL, Duration.ofMillis(100)); - - Configuration flussConfig = new Configuration(); - flussConfig.setString( - ConfigOptions.BOOTSTRAP_SERVERS.key(), - FLUSS_CLUSTER_EXTENSION.getBootstrapServers()); - return LakeTieringJobBuilder.newBuilder( - execEnv, - flussConfig, - new Configuration(), - lakeTieringConfig, - DataLakeFormat.PAIMON.toString()) - .build(); - } - - protected void waitUntilSnapshot(long tableId, int bucketNum, long snapshotId) { - for (int i = 0; i < bucketNum; i++) { - TableBucket tableBucket = new TableBucket(tableId, i); - FLUSS_CLUSTER_EXTENSION.waitUntilSnapshotFinished(tableBucket, snapshotId); - } + return createTable( + tablePath, + 3, + Collections.singletonList("a"), + schemaBuilder.build(), + customProperties); } } diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/TieringSourceReaderTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/TieringSourceReaderTest.java index 8868b29832..9e9de2c792 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/TieringSourceReaderTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/TieringSourceReaderTest.java @@ -148,7 +148,7 @@ void testHandleTieringReachMaxDurationEvent() throws Exception { assertThat(result.logEndOffset()).isEqualTo(1); }); - // test add split with force ignore + // test add split with skipCurrentRound split = new TieringLogSplit( tablePath, @@ -156,7 +156,7 @@ void testHandleTieringReachMaxDurationEvent() throws Exception { null, EARLIEST_OFFSET, 100L); - split.forceIgnore(); + split.skipCurrentRound(); reader.addSplits(Collections.singletonList(split)); // should skip tiering for this split retry( diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumeratorTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumeratorTest.java index 8781569cf4..9c3e97335e 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumeratorTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumeratorTest.java @@ -20,7 +20,6 @@ import org.apache.fluss.config.Configuration; import org.apache.fluss.flink.tiering.event.FailedTieringEvent; import org.apache.fluss.flink.tiering.event.FinishedTieringEvent; -import org.apache.fluss.flink.tiering.event.TieringFailOverEvent; import org.apache.fluss.flink.tiering.event.TieringReachMaxDurationEvent; import org.apache.fluss.flink.tiering.source.TieringTestBase; import org.apache.fluss.flink.tiering.source.split.TieringLogSplit; @@ -34,9 +33,9 @@ import org.apache.fluss.rpc.messages.PbLakeTableSnapshotInfo; import org.apache.fluss.utils.clock.ManualClock; -import org.apache.flink.api.connector.source.ReaderInfo; import org.apache.flink.api.connector.source.SourceEvent; import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.api.connector.source.mocks.MockSplitEnumeratorContext; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -276,10 +275,8 @@ void testLogTableSplits() throws Throwable { // mock finished tiered this round, check second round context.getSplitsAssignmentSequence().clear(); - final Map bucketOffsetOfEarliest = new HashMap<>(); final Map bucketOffsetOfInitialWrite = new HashMap<>(); for (int tableBucket = 0; tableBucket < DEFAULT_BUCKET_NUM; tableBucket++) { - bucketOffsetOfEarliest.put(tableBucket, EARLIEST_OFFSET); bucketOffsetOfInitialWrite.put( tableBucket, bucketOffsetOfFirstWrite.getOrDefault(tableBucket, 0L)); } @@ -483,10 +480,8 @@ void testPartitionedLogTableSplits() throws Throwable { long snapshot = 1; for (Map.Entry partitionNameById : partitionNameByIds.entrySet()) { long partitionId = partitionNameById.getValue(); - Map partitionInitialBucketOffsets = new HashMap<>(); Map partitionBucketOffsetOfInitialWrite = new HashMap<>(); for (int tableBucket = 0; tableBucket < DEFAULT_BUCKET_NUM; tableBucket++) { - partitionInitialBucketOffsets.put(tableBucket, EARLIEST_OFFSET); partitionBucketOffsetOfInitialWrite.put( tableBucket, bucketOffsetOfFirstWrite @@ -561,7 +556,7 @@ void testHandleFailedTieringTableEvent() throws Throwable { try (FlussMockSplitEnumeratorContext context = new FlussMockSplitEnumeratorContext<>(numSubtasks)) { TieringSourceEnumerator enumerator = - new TieringSourceEnumerator(flussConf, context, 500); + createDefaultTieringSourceEnumerator(flussConf, context); enumerator.start(); assertThat(context.getSplitsAssignmentSequence()).isEmpty(); @@ -757,8 +752,8 @@ void testTableReachMaxTieringDuration() throws Throwable { ManualClock manualClock = new ManualClock(); - try (MockSplitEnumeratorContext context = - new MockSplitEnumeratorContext<>(numSubtasks); + try (FlussMockSplitEnumeratorContext context = + new FlussMockSplitEnumeratorContext<>(numSubtasks); TieringSourceEnumerator enumerator = createTieringSourceEnumeratorWithManualClock( flussConf, @@ -770,7 +765,7 @@ void testTableReachMaxTieringDuration() throws Throwable { // Register all readers for (int subtaskId = 0; subtaskId < numSubtasks; subtaskId++) { - registerReader(context, enumerator, subtaskId, "localhost-" + subtaskId); + context.registerSourceReader(subtaskId, subtaskId, "localhost-" + subtaskId); } appendRow(tablePath, DEFAULT_LOG_TABLE_DESCRIPTOR, 0, 10); @@ -805,14 +800,14 @@ void testTableReachMaxTieringDuration() throws Throwable { // request a split again enumerator.handleSplitRequest(0, "localhost-0"); - // the split should be marked as forceIgnore + // the split should be marked as skipCurrentRound waitUntilTieringTableSplitAssignmentReady(context, 1, 100L); List assignedSplits = new ArrayList<>(); context.getSplitsAssignmentSequence() .forEach(a -> a.assignment().values().forEach(assignedSplits::addAll)); assertThat(assignedSplits).hasSize(1); - assertThat(assignedSplits.get(0).isForceIgnore()).isTrue(); + assertThat(assignedSplits.get(0).shouldSkipCurrentRound()).isTrue(); } } } diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/split/TieringSplitSerializerTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/split/TieringSplitSerializerTest.java index 6c4774870a..45cbd76a80 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/split/TieringSplitSerializerTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/split/TieringSplitSerializerTest.java @@ -69,8 +69,8 @@ void testTieringSnapshotSplitStringExpression(Boolean isPartitionedTable) throws String expectedSplitString = isPartitionedTable - ? "TieringSnapshotSplit{tablePath=test_db.test_partitioned_table, tableBucket=TableBucket{tableId=1, partitionId=100, bucket=2}, partitionName='1024', numberOfSplits=30, forceIgnore=false, snapshotId=0, logOffsetOfSnapshot=200}" - : "TieringSnapshotSplit{tablePath=test_db.test_table, tableBucket=TableBucket{tableId=1, bucket=2}, partitionName='null', numberOfSplits=30, forceIgnore=false, snapshotId=0, logOffsetOfSnapshot=200}"; + ? "TieringSnapshotSplit{tablePath=test_db.test_partitioned_table, tableBucket=TableBucket{tableId=1, partitionId=100, bucket=2}, partitionName='1024', numberOfSplits=30, skipCurrentRound=false, snapshotId=0, logOffsetOfSnapshot=200}" + : "TieringSnapshotSplit{tablePath=test_db.test_table, tableBucket=TableBucket{tableId=1, bucket=2}, partitionName='null', numberOfSplits=30, skipCurrentRound=false, snapshotId=0, logOffsetOfSnapshot=200}"; assertThat(new TieringSnapshotSplit(path, bucket, partitionName, 0L, 200L, 30).toString()) .isEqualTo(expectedSplitString); } @@ -103,48 +103,48 @@ void testTieringLogSplitStringExpression(Boolean isPartitionedTable) throws Exce String expectedSplitString = isPartitionedTable - ? "TieringLogSplit{tablePath=test_db.test_partitioned_table, tableBucket=TableBucket{tableId=1, partitionId=100, bucket=2}, partitionName='1024', numberOfSplits=2, forceIgnore=false, startingOffset=100, stoppingOffset=200}" - : "TieringLogSplit{tablePath=test_db.test_table, tableBucket=TableBucket{tableId=1, bucket=2}, partitionName='null', numberOfSplits=2, forceIgnore=false, startingOffset=100, stoppingOffset=200}"; + ? "TieringLogSplit{tablePath=test_db.test_partitioned_table, tableBucket=TableBucket{tableId=1, partitionId=100, bucket=2}, partitionName='1024', numberOfSplits=2, skipCurrentRound=false, startingOffset=100, stoppingOffset=200}" + : "TieringLogSplit{tablePath=test_db.test_table, tableBucket=TableBucket{tableId=1, bucket=2}, partitionName='null', numberOfSplits=2, skipCurrentRound=false, startingOffset=100, stoppingOffset=200}"; assertThat(new TieringLogSplit(path, bucket, partitionName, 100, 200, 2).toString()) .isEqualTo(expectedSplitString); } @Test - void testForceIgnoreSerde() throws Exception { - // Test TieringSnapshotSplit with forceIgnore set at creation - TieringSnapshotSplit snapshotSplitWithForceIgnore = + void testSkipCurrentRoundSerde() throws Exception { + // Test TieringSnapshotSplit with skipCurrentRound set at creation + TieringSnapshotSplit snapshotSplitWithSkipCurrentRound = new TieringSnapshotSplit(tablePath, tableBucket, null, 0L, 200L, 10, true); - byte[] serialized = serializer.serialize(snapshotSplitWithForceIgnore); + byte[] serialized = serializer.serialize(snapshotSplitWithSkipCurrentRound); TieringSnapshotSplit deserializedSnapshotSplit = (TieringSnapshotSplit) serializer.deserialize(serializer.getVersion(), serialized); - assertThat(deserializedSnapshotSplit).isEqualTo(snapshotSplitWithForceIgnore); + assertThat(deserializedSnapshotSplit).isEqualTo(snapshotSplitWithSkipCurrentRound); - // Test TieringLogSplit with forceIgnore set at creation - TieringLogSplit logSplitWithForceIgnore = - new TieringLogSplit(tablePath, tableBucket, null, 100, 200, true, 40); - serialized = serializer.serialize(logSplitWithForceIgnore); + // Test TieringLogSplit with skipCurrentRound set at creation + TieringLogSplit logSplitWithSkipCurrentRound = + new TieringLogSplit(tablePath, tableBucket, null, 100, 200, 40, true); + serialized = serializer.serialize(logSplitWithSkipCurrentRound); TieringLogSplit deserializedLogSplit = (TieringLogSplit) serializer.deserialize(serializer.getVersion(), serialized); - assertThat(deserializedLogSplit).isEqualTo(logSplitWithForceIgnore); + assertThat(deserializedLogSplit).isEqualTo(logSplitWithSkipCurrentRound); - // Test TieringSnapshotSplit with forceIgnore set after creation + // Test TieringSnapshotSplit with skipCurrentRound set after creation TieringSnapshotSplit snapshotSplit = new TieringSnapshotSplit(tablePath, tableBucket, null, 0L, 200L, 10, false); - assertThat(snapshotSplit.isForceIgnore()).isFalse(); - snapshotSplit.forceIgnore(); - assertThat(snapshotSplit.isForceIgnore()).isTrue(); + assertThat(snapshotSplit.shouldSkipCurrentRound()).isFalse(); + snapshotSplit.skipCurrentRound(); + assertThat(snapshotSplit.shouldSkipCurrentRound()).isTrue(); serialized = serializer.serialize(snapshotSplit); deserializedSnapshotSplit = (TieringSnapshotSplit) serializer.deserialize(serializer.getVersion(), serialized); assertThat(deserializedSnapshotSplit).isEqualTo(snapshotSplit); - // Test TieringLogSplit with forceIgnore set after creation + // Test TieringLogSplit with skipCurrentRound set after creation TieringLogSplit logSplit = - new TieringLogSplit(tablePath, tableBucket, null, 100, 200, false, 40); - assertThat(logSplit.isForceIgnore()).isFalse(); - logSplit.forceIgnore(); - assertThat(logSplit.isForceIgnore()).isTrue(); + new TieringLogSplit(tablePath, tableBucket, null, 100, 200, 40, false); + assertThat(logSplit.shouldSkipCurrentRound()).isFalse(); + logSplit.skipCurrentRound(); + assertThat(logSplit.shouldSkipCurrentRound()).isTrue(); serialized = serializer.serialize(logSplit); deserializedLogSplit = diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/lake/values/tiering/TestingValuesLakeTieringFactory.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/lake/values/tiering/TestingValuesLakeTieringFactory.java index 3a9c785788..e966fb2d4f 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/lake/values/tiering/TestingValuesLakeTieringFactory.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/lake/values/tiering/TestingValuesLakeTieringFactory.java @@ -35,7 +35,7 @@ public class TestingValuesLakeTieringFactory @Override public LakeWriter createLakeWriter( WriterInitContext writerInitContext) throws IOException { - return new TestingValuesLakeWriter(writerInitContext.tablePath().toString()); + return new TestingValuesLakeWriter(writerInitContext.tableInfo()); } @Override diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/lake/values/tiering/TestingValuesLakeWriter.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/lake/values/tiering/TestingValuesLakeWriter.java index 480ebe3077..2764ba0f7c 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/lake/values/tiering/TestingValuesLakeWriter.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/lake/values/tiering/TestingValuesLakeWriter.java @@ -18,29 +18,46 @@ package org.apache.fluss.lake.values.tiering; +import org.apache.fluss.config.ConfigOption; import org.apache.fluss.lake.serializer.SimpleVersionedSerializer; import org.apache.fluss.lake.values.TestingValuesLake; import org.apache.fluss.lake.writer.LakeWriter; +import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.record.LogRecord; import org.apache.fluss.utils.InstantiationUtils; import java.io.IOException; import java.io.Serializable; +import java.time.Duration; import java.util.UUID; +import static org.apache.fluss.config.ConfigBuilder.key; + /** Implementation of {@link LakeWriter} for values lake. */ public class TestingValuesLakeWriter implements LakeWriter { private final String tableId; private final String writerId; + static ConfigOption writePauseOption = + key("write-pause").durationType().noDefaultValue(); + private final Duration writePause; - public TestingValuesLakeWriter(String tableId) { - this.tableId = tableId; + public TestingValuesLakeWriter(TableInfo tableInfo) { + this.tableId = tableInfo.getTablePath().toString(); this.writerId = UUID.randomUUID().toString(); + this.writePause = tableInfo.getCustomProperties().get(writePauseOption); } @Override public void write(LogRecord record) throws IOException { + try { + if (writePause != null) { + Thread.sleep(writePause.toMillis()); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException("Interrupted while pausing before write", e); + } TestingValuesLake.writeRecord(tableId, writerId, record); } From d2a54952743b3b7d596be60e3bb2ef47fc1b63bc Mon Sep 17 00:00:00 2001 From: luoyuxia Date: Wed, 21 Jan 2026 15:26:46 +0800 Subject: [PATCH 3/5] address comment & remove unnecessary option --- .../apache/fluss/config/ConfigOptions.java | 12 +-- .../flink/tiering/LakeTieringJobBuilder.java | 14 +-- .../flink/tiering/source/TieringSource.java | 33 ++----- .../tiering/source/TieringSplitReader.java | 2 +- .../enumerator/TieringSourceEnumerator.java | 93 ++++++------------- .../flink/tiering/FlinkTieringTestBase.java | 2 - .../fluss/flink/tiering/TieringITCase.java | 9 +- .../TieringSourceEnumeratorTest.java | 65 ++++--------- 8 files changed, 65 insertions(+), 165 deletions(-) diff --git a/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java b/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java index 12aa7c32aa..f62eb2236d 100644 --- a/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java +++ b/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java @@ -1851,8 +1851,8 @@ public class ConfigOptions { + ConfigOptions.TABLE_DATALAKE_AUTO_EXPIRE_SNAPSHOT + " is false."); - public static final ConfigOption LAKE_TIERING_TABLE_DURATION_MAX = - key("lake.tiering.table.duration.max") + public static final ConfigOption LAKE_TIERING_TABLE_MAX_DURATION = + key("lake.tiering.table.max-duration") .durationType() .defaultValue(Duration.ofMinutes(30)) .withDescription( @@ -1860,14 +1860,6 @@ public class ConfigOptions { + "it will be force completed: the tiering will be finalized and committed to the data lake " + "(e.g., Paimon) immediately, even if they haven't reached their desired stopping offsets."); - public static final ConfigOption LAKE_TIERING_TABLE_DURATION_DETECT_INTERVAL = - key("lake.tiering.table.duration.detect-interval") - .durationType() - .defaultValue(Duration.ofSeconds(30)) - .withDescription( - "The interval to check if a table tiering operation has reached the maximum duration. " - + "The enumerator will periodically check tiering tables and force complete those that exceed the maximum duration."); - // ------------------------------------------------------------------------ // ConfigOptions for fluss kafka // ------------------------------------------------------------------------ diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/LakeTieringJobBuilder.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/LakeTieringJobBuilder.java index 70774c8a60..f027bf8601 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/LakeTieringJobBuilder.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/LakeTieringJobBuilder.java @@ -34,8 +34,7 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; -import static org.apache.fluss.config.ConfigOptions.LAKE_TIERING_TABLE_DURATION_DETECT_INTERVAL; -import static org.apache.fluss.config.ConfigOptions.LAKE_TIERING_TABLE_DURATION_MAX; +import static org.apache.fluss.config.ConfigOptions.LAKE_TIERING_TABLE_MAX_DURATION; import static org.apache.fluss.flink.tiering.source.TieringSource.TIERING_SOURCE_TRANSFORMATION_UID; import static org.apache.fluss.flink.tiering.source.TieringSourceOptions.POLL_TIERING_TABLE_INTERVAL; import static org.apache.fluss.utils.Preconditions.checkNotNull; @@ -92,14 +91,9 @@ public JobClient build() throws Exception { flussConfig.get(POLL_TIERING_TABLE_INTERVAL).toMillis()); } - if (lakeTieringConfig.get(LAKE_TIERING_TABLE_DURATION_MAX) != null) { - tieringSourceBuilder.withTieringTableDurationMax( - lakeTieringConfig.get(LAKE_TIERING_TABLE_DURATION_MAX).toMillis()); - } - - if (lakeTieringConfig.get(LAKE_TIERING_TABLE_DURATION_DETECT_INTERVAL) != null) { - tieringSourceBuilder.withTieringTableDurationDetectInterval( - lakeTieringConfig.get(LAKE_TIERING_TABLE_DURATION_DETECT_INTERVAL).toMillis()); + if (lakeTieringConfig.get(LAKE_TIERING_TABLE_MAX_DURATION) != null) { + tieringSourceBuilder.withTieringTableMaxDurationMs( + lakeTieringConfig.get(LAKE_TIERING_TABLE_MAX_DURATION).toMillis()); } TieringSource tieringSource = tieringSourceBuilder.build(); diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSource.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSource.java index 5d637c9071..b2d8df8dfb 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSource.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSource.java @@ -44,8 +44,7 @@ import java.nio.charset.StandardCharsets; -import static org.apache.fluss.config.ConfigOptions.LAKE_TIERING_TABLE_DURATION_DETECT_INTERVAL; -import static org.apache.fluss.config.ConfigOptions.LAKE_TIERING_TABLE_DURATION_MAX; +import static org.apache.fluss.config.ConfigOptions.LAKE_TIERING_TABLE_MAX_DURATION; import static org.apache.fluss.flink.tiering.source.TieringSourceOptions.POLL_TIERING_TABLE_INTERVAL; /** @@ -66,19 +65,16 @@ public class TieringSource private final LakeTieringFactory lakeTieringFactory; private final long pollTieringTableIntervalMs; private final long tieringTableDurationMaxMs; - private final long tieringTableDurationDetectIntervalMs; public TieringSource( Configuration flussConf, LakeTieringFactory lakeTieringFactory, long pollTieringTableIntervalMs, - long tieringTableDurationMaxMs, - long tieringTableDurationDetectIntervalMs) { + long tieringTableDurationMaxMs) { this.flussConf = flussConf; this.lakeTieringFactory = lakeTieringFactory; this.pollTieringTableIntervalMs = pollTieringTableIntervalMs; this.tieringTableDurationMaxMs = tieringTableDurationMaxMs; - this.tieringTableDurationDetectIntervalMs = tieringTableDurationDetectIntervalMs; } @Override @@ -93,8 +89,7 @@ public SplitEnumerator createEnumera flussConf, splitEnumeratorContext, pollTieringTableIntervalMs, - tieringTableDurationMaxMs, - tieringTableDurationDetectIntervalMs); + tieringTableDurationMaxMs); } @Override @@ -106,8 +101,7 @@ public SplitEnumerator restoreEnumer flussConf, splitEnumeratorContext, pollTieringTableIntervalMs, - tieringTableDurationMaxMs, - tieringTableDurationDetectIntervalMs); + tieringTableDurationMaxMs); } @Override @@ -146,10 +140,8 @@ public static class Builder { private final LakeTieringFactory lakeTieringFactory; private long pollTieringTableIntervalMs = POLL_TIERING_TABLE_INTERVAL.defaultValue().toMillis(); - private long tieringTableDurationMaxMs = - LAKE_TIERING_TABLE_DURATION_MAX.defaultValue().toMillis(); - private long tieringTableDurationDetectIntervalMs = - LAKE_TIERING_TABLE_DURATION_DETECT_INTERVAL.defaultValue().toMillis(); + private long tieringTableMaxDurationMs = + LAKE_TIERING_TABLE_MAX_DURATION.defaultValue().toMillis(); public Builder( Configuration flussConf, LakeTieringFactory lakeTieringFactory) { @@ -162,14 +154,8 @@ public Builder withPollTieringTableIntervalMs(long pollTieringTable return this; } - public Builder withTieringTableDurationMax(long tieringTableDurationMaxMs) { - this.tieringTableDurationMaxMs = tieringTableDurationMaxMs; - return this; - } - - public Builder withTieringTableDurationDetectInterval( - long tieringTableDurationDetectIntervalMs) { - this.tieringTableDurationDetectIntervalMs = tieringTableDurationDetectIntervalMs; + public Builder withTieringTableMaxDurationMs(long tieringTableDurationMaxMs) { + this.tieringTableMaxDurationMs = tieringTableDurationMaxMs; return this; } @@ -178,8 +164,7 @@ public TieringSource build() { flussConf, lakeTieringFactory, pollTieringTableIntervalMs, - tieringTableDurationMaxMs, - tieringTableDurationDetectIntervalMs); + tieringTableMaxDurationMs); } } } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java index 66e903a0da..3ee566c421 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java @@ -533,7 +533,7 @@ private void finishCurrentTable() throws IOException { } catch (Exception e) { throw new IOException("Fail to finish current table.", e); } - + reachTieringMaxDurationTables.remove(currentTableId); // before switch to a new table, mark all as empty or null currentTableId = null; currentTablePath = null; diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumerator.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumerator.java index 9bc157999b..d2102db1d0 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumerator.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumerator.java @@ -40,8 +40,6 @@ import org.apache.fluss.rpc.messages.PbLakeTieringTableInfo; import org.apache.fluss.rpc.metrics.ClientMetricGroup; import org.apache.fluss.utils.MapUtils; -import org.apache.fluss.utils.clock.Clock; -import org.apache.fluss.utils.clock.SystemClock; import org.apache.flink.api.connector.source.ReaderInfo; import org.apache.flink.api.connector.source.SourceEvent; @@ -65,6 +63,8 @@ import java.util.Set; import java.util.TreeSet; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -95,20 +95,17 @@ public class TieringSourceEnumerator private final Configuration flussConf; private final SplitEnumeratorContext context; + private final ScheduledExecutorService timerService; private final SplitEnumeratorMetricGroup enumeratorMetricGroup; private final long pollTieringTableIntervalMs; - private final long tieringTableDurationMaxMs; - private final long tieringTableDurationDetectIntervalMs; + private final long tieringTableMaxDurationMs; private final List pendingSplits; private final Set readersAwaitingSplit; - private final Map tieringTablesDeadline; private final Map tieringTableEpochs; private final Map failedTableEpochs; private final Map finishedTableEpochs; - private final Clock clock; - // lazily instantiated private RpcClient rpcClient; private CoordinatorGateway coordinatorGateway; @@ -125,37 +122,20 @@ public TieringSourceEnumerator( Configuration flussConf, SplitEnumeratorContext context, long pollTieringTableIntervalMs, - long tieringTableDurationMaxMs, - long tieringTableDurationDetectIntervalMs) { - this( - flussConf, - context, - pollTieringTableIntervalMs, - tieringTableDurationMaxMs, - tieringTableDurationDetectIntervalMs, - SystemClock.getInstance()); - } - - public TieringSourceEnumerator( - Configuration flussConf, - SplitEnumeratorContext context, - long pollTieringTableIntervalMs, - long tieringTableDurationMaxMs, - long tieringTableDurationDetectIntervalMs, - Clock clock) { + long tieringTableMaxDurationMs) { this.flussConf = flussConf; this.context = context; + this.timerService = + Executors.newSingleThreadScheduledExecutor( + r -> new Thread(r, "Tiering-Timer-Thread")); this.enumeratorMetricGroup = context.metricGroup(); this.pollTieringTableIntervalMs = pollTieringTableIntervalMs; - this.tieringTableDurationMaxMs = tieringTableDurationMaxMs; - this.tieringTableDurationDetectIntervalMs = tieringTableDurationDetectIntervalMs; + this.tieringTableMaxDurationMs = tieringTableMaxDurationMs; this.pendingSplits = Collections.synchronizedList(new ArrayList<>()); this.readersAwaitingSplit = Collections.synchronizedSet(new TreeSet<>()); this.tieringTableEpochs = MapUtils.newConcurrentHashMap(); this.finishedTableEpochs = MapUtils.newConcurrentHashMap(); this.failedTableEpochs = MapUtils.newConcurrentHashMap(); - this.tieringTablesDeadline = MapUtils.newConcurrentHashMap(); - this.clock = clock; } @Override @@ -191,12 +171,6 @@ public void start() { this::generateAndAssignSplits, 0, pollTieringTableIntervalMs); - - this.context.callAsync( - this::checkTableReachMaxTieringDuration, - this::handleReachMaxTieringDurationTables, - 0, - tieringTableDurationDetectIntervalMs); } @Override @@ -282,7 +256,6 @@ public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { } else { finishedTableEpochs.put(finishedTableId, tieringEpoch); } - tieringTablesDeadline.remove(finishedTableId); } if (sourceEvent instanceof FailedTieringEvent) { @@ -301,7 +274,6 @@ public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { } else { failedTableEpochs.put(failedTableId, tieringEpoch); } - tieringTablesDeadline.remove(failedTableId); } if (!finishedTableEpochs.isEmpty() || !failedTableEpochs.isEmpty()) { @@ -327,29 +299,17 @@ private void handleSourceReaderFailOver() { } } - private Set checkTableReachMaxTieringDuration() { - Set tieringReachMaxDurationTables = new HashSet<>(); - long currentTime = clock.milliseconds(); - for (Map.Entry tieringTableDeadline : tieringTablesDeadline.entrySet()) { - long tableId = tieringTableDeadline.getKey(); - long deadline = tieringTableDeadline.getValue(); - if (deadline < currentTime) { - tieringReachMaxDurationTables.add(tableId); - } - } - return tieringReachMaxDurationTables; - } - - private void handleReachMaxTieringDurationTables( - Set tieringReachMaxDurationTables, Throwable throwable) { - if (throwable != null) { - LOG.error("Fail to check tiering timeout tables.", throwable); - return; - } + @VisibleForTesting + protected void handleTableTieringReachMaxDuration(long tableId, long tieringEpoch) { + Long currentEpoch = tieringTableEpochs.get(tableId); + if (currentEpoch != null && currentEpoch.equals(tieringEpoch)) { + LOG.info( + "Table {} reached max duration ({}ms). Force completing.", + tableId, + tieringTableMaxDurationMs); - for (Long reachMaxDurationTable : tieringReachMaxDurationTables) { for (TieringSplit tieringSplit : pendingSplits) { - if (tieringSplit.getTableBucket().getTableId() == reachMaxDurationTable) { + if (tieringSplit.getTableBucket().getTableId() == tableId) { // mark this tiering split to skip the current round since the tiering for // this table has timed out, so the tiering source reader can skip them directly tieringSplit.skipCurrentRound(); @@ -361,14 +321,11 @@ private void handleReachMaxTieringDurationTables( } } - LOG.info("Found the table {} reach max tiering duration.", reachMaxDurationTable); - // broadcast the tiering reach max duration event to all readers, // we broadcast all for simplicity Set readers = new HashSet<>(context.registeredReaders().keySet()); for (int reader : readers) { - context.sendEventToSourceReader( - reader, new TieringReachMaxDurationEvent(reachMaxDurationTable)); + context.sendEventToSourceReader(reader, new TieringReachMaxDurationEvent(tableId)); } } } @@ -478,8 +435,15 @@ private void generateTieringSplits(Tuple3 tieringTable) } else { tieringTableEpochs.put(tieringTable.f0, tieringTable.f1); pendingSplits.addAll(tieringSplits); - tieringTablesDeadline.put( - tieringTable.f0, clock.milliseconds() + tieringTableDurationMaxMs); + + timerService.schedule( + () -> + context.runInCoordinatorThread( + () -> + handleTableTieringReachMaxDuration( + tieringTable.f0, tieringTable.f1)), + tieringTableMaxDurationMs, + TimeUnit.MILLISECONDS); } } catch (Exception e) { LOG.warn("Fail to generate Tiering splits for table {}.", tieringTable.f2, e); @@ -503,6 +467,7 @@ public TieringSourceEnumeratorState snapshotState(long checkpointId) throws Exce @Override public void close() throws IOException { closed = true; + timerService.shutdownNow(); if (rpcClient != null) { failedTableEpochs.putAll(tieringTableEpochs); tieringTableEpochs.clear(); diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/FlinkTieringTestBase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/FlinkTieringTestBase.java index 7d0518f83f..0971231ff0 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/FlinkTieringTestBase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/FlinkTieringTestBase.java @@ -96,8 +96,6 @@ static void afterAll() throws Exception { conn.close(); conn = null; } - - System.out.println("after all"); } @BeforeEach diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/TieringITCase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/TieringITCase.java index 9b9939682f..e542b9316d 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/TieringITCase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/TieringITCase.java @@ -44,8 +44,7 @@ import java.util.Map; import java.util.Optional; -import static org.apache.fluss.config.ConfigOptions.LAKE_TIERING_TABLE_DURATION_DETECT_INTERVAL; -import static org.apache.fluss.config.ConfigOptions.LAKE_TIERING_TABLE_DURATION_MAX; +import static org.apache.fluss.config.ConfigOptions.LAKE_TIERING_TABLE_MAX_DURATION; import static org.apache.fluss.testutils.common.CommonTestUtils.waitValue; import static org.assertj.core.api.Assertions.assertThat; @@ -63,6 +62,7 @@ protected static void afterAll() throws Exception { } @BeforeEach + @Override void beforeEach() { execEnv = StreamExecutionEnvironment.getExecutionEnvironment() @@ -96,8 +96,7 @@ void testTieringReachMaxDuration() throws Exception { // set tiering duration to a small value for testing purpose Configuration lakeTieringConfig = new Configuration(); - lakeTieringConfig.set(LAKE_TIERING_TABLE_DURATION_MAX, Duration.ofSeconds(1)); - lakeTieringConfig.set(LAKE_TIERING_TABLE_DURATION_DETECT_INTERVAL, Duration.ofMillis(100)); + lakeTieringConfig.set(LAKE_TIERING_TABLE_MAX_DURATION, Duration.ofSeconds(1)); JobClient jobClient = buildTieringJob(execEnv, lakeTieringConfig); try { @@ -117,7 +116,7 @@ void testTieringReachMaxDuration() throws Exception { } } - private long countTieredRecords(LakeSnapshot lakeSnapshot) throws Exception { + private long countTieredRecords(LakeSnapshot lakeSnapshot) { return lakeSnapshot.getTableBucketsOffset().values().stream() .mapToLong(Long::longValue) .sum(); diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumeratorTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumeratorTest.java index 9c3e97335e..5984218109 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumeratorTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumeratorTest.java @@ -31,7 +31,6 @@ import org.apache.fluss.rpc.messages.CommitLakeTableSnapshotRequest; import org.apache.fluss.rpc.messages.PbLakeTableOffsetForBucket; import org.apache.fluss.rpc.messages.PbLakeTableSnapshotInfo; -import org.apache.fluss.utils.clock.ManualClock; import org.apache.flink.api.connector.source.SourceEvent; import org.apache.flink.api.connector.source.SplitsAssignment; @@ -79,8 +78,7 @@ void testPrimaryKeyTableWithNoSnapshotSplits() throws Throwable { // test get snapshot split & log split and the assignment try (FlussMockSplitEnumeratorContext context = new FlussMockSplitEnumeratorContext<>(numSubtasks)) { - TieringSourceEnumerator enumerator = - createDefaultTieringSourceEnumerator(flussConf, context); + TieringSourceEnumerator enumerator = createTieringSourceEnumerator(flussConf, context); enumerator.start(); assertThat(context.getSplitsAssignmentSequence()).isEmpty(); @@ -158,8 +156,7 @@ void testPrimaryKeyTableWithSnapshotSplits() throws Throwable { // test get snapshot split assignment try (FlussMockSplitEnumeratorContext context = new FlussMockSplitEnumeratorContext<>(numSubtasks)) { - TieringSourceEnumerator enumerator = - createDefaultTieringSourceEnumerator(flussConf, context); + TieringSourceEnumerator enumerator = createTieringSourceEnumerator(flussConf, context); enumerator.start(); assertThat(context.getSplitsAssignmentSequence()).isEmpty(); @@ -241,8 +238,7 @@ void testLogTableSplits() throws Throwable { // test get log split and the assignment try (FlussMockSplitEnumeratorContext context = new FlussMockSplitEnumeratorContext<>(numSubtasks)) { - TieringSourceEnumerator enumerator = - createDefaultTieringSourceEnumerator(flussConf, context); + TieringSourceEnumerator enumerator = createTieringSourceEnumerator(flussConf, context); enumerator.start(); assertThat(context.getSplitsAssignmentSequence()).isEmpty(); @@ -333,8 +329,7 @@ void testPartitionedPrimaryKeyTable() throws Throwable { // test get snapshot split assignment try (FlussMockSplitEnumeratorContext context = new FlussMockSplitEnumeratorContext<>(numSubtasks)) { - TieringSourceEnumerator enumerator = - createDefaultTieringSourceEnumerator(flussConf, context); + TieringSourceEnumerator enumerator = createTieringSourceEnumerator(flussConf, context); enumerator.start(); assertThat(context.getSplitsAssignmentSequence()).isEmpty(); @@ -434,8 +429,7 @@ void testPartitionedLogTableSplits() throws Throwable { // test get log split assignment try (FlussMockSplitEnumeratorContext context = new FlussMockSplitEnumeratorContext<>(numSubtasks)) { - TieringSourceEnumerator enumerator = - createDefaultTieringSourceEnumerator(flussConf, context); + TieringSourceEnumerator enumerator = createTieringSourceEnumerator(flussConf, context); enumerator.start(); assertThat(context.getSplitsAssignmentSequence()).isEmpty(); @@ -555,8 +549,7 @@ void testHandleFailedTieringTableEvent() throws Throwable { // test get log split and the assignment try (FlussMockSplitEnumeratorContext context = new FlussMockSplitEnumeratorContext<>(numSubtasks)) { - TieringSourceEnumerator enumerator = - createDefaultTieringSourceEnumerator(flussConf, context); + TieringSourceEnumerator enumerator = createTieringSourceEnumerator(flussConf, context); enumerator.start(); assertThat(context.getSplitsAssignmentSequence()).isEmpty(); @@ -610,8 +603,7 @@ void testHandleReaderFailOver() throws Throwable { try (FlussMockSplitEnumeratorContext context = new FlussMockSplitEnumeratorContext<>(3)) { - TieringSourceEnumerator enumerator = - createDefaultTieringSourceEnumerator(flussConf, context); + TieringSourceEnumerator enumerator = createTieringSourceEnumerator(flussConf, context); enumerator.start(); assertThat(context.getSplitsAssignmentSequence()).isEmpty(); @@ -716,29 +708,16 @@ private void verifyTieringSplitAssignment( .allMatch(tieringSplit -> tieringSplit.getTablePath().equals(expectedTablePath)); } - private TieringSourceEnumerator createDefaultTieringSourceEnumerator( + private TieringSourceEnumerator createTieringSourceEnumerator( Configuration flussConf, MockSplitEnumeratorContext context) { - return new TieringSourceEnumerator( - flussConf, - context, - 500, - Duration.ofMinutes(10).toMillis(), - Duration.ofSeconds(10).toMillis()); + return createTieringSourceEnumerator(flussConf, context, Duration.ofSeconds(10).toMillis()); } - private TieringSourceEnumerator createTieringSourceEnumeratorWithManualClock( + private TieringSourceEnumerator createTieringSourceEnumerator( Configuration flussConf, MockSplitEnumeratorContext context, - ManualClock clock, - long tieringTableDurationMaxMs, - long tieringTableDurationDetectIntervalMs) { - return new TieringSourceEnumerator( - flussConf, - context, - 500, - tieringTableDurationMaxMs, - tieringTableDurationDetectIntervalMs, - clock); + long tieringTableDurationMaxMs) { + return new TieringSourceEnumerator(flussConf, context, 500, tieringTableDurationMaxMs); } @Test @@ -748,19 +727,12 @@ void testTableReachMaxTieringDuration() throws Throwable { int numSubtasks = 2; long tieringTableDurationMaxMs = Duration.ofMinutes(10).toMillis(); - long tieringTableDurationDetectIntervalMs = Duration.ofMillis(100).toMillis(); - - ManualClock manualClock = new ManualClock(); try (FlussMockSplitEnumeratorContext context = new FlussMockSplitEnumeratorContext<>(numSubtasks); TieringSourceEnumerator enumerator = - createTieringSourceEnumeratorWithManualClock( - flussConf, - context, - manualClock, - tieringTableDurationMaxMs, - tieringTableDurationDetectIntervalMs); ) { + createTieringSourceEnumerator( + flussConf, context, tieringTableDurationMaxMs); ) { enumerator.start(); // Register all readers @@ -777,13 +749,8 @@ void testTableReachMaxTieringDuration() throws Throwable { // Wait for initial assignment waitUntilTieringTableSplitAssignmentReady(context, 2, 200L); - // Advance time to mock exceed max duration - manualClock.advanceTime(Duration.ofMillis(tieringTableDurationMaxMs + 60_000)); - - // Run periodic callable to trigger max duration check - // Index 0 is for requestTieringTableSplitsViaHeartBeat - // Index 1 is for checkTableReachMaxTieringDuration - context.runPeriodicCallable(1); + // call handleTableTieringReachMaxDuration to mock tiering reach max duration + enumerator.handleTableTieringReachMaxDuration(tableId, 1); // Verify that TieringReachMaxDurationEvent was sent to all readers // Use reflection to access events sent to readers From 38222c2d77e9dd6f9b260e4191c019e33fb556c5 Mon Sep 17 00:00:00 2001 From: luoyuxia Date: Wed, 21 Jan 2026 19:48:00 +0800 Subject: [PATCH 4/5] minor fix --- .../apache/fluss/config/ConfigOptions.java | 9 ---- .../flink/tiering/LakeTieringJobBuilder.java | 19 +++---- .../flink/tiering/source/TieringSource.java | 52 ++++++------------- .../source/TieringSourceFetcherManager.java | 7 ++- .../tiering/source/TieringSplitReader.java | 12 ++--- .../enumerator/TieringSourceEnumerator.java | 39 +++++++------- .../source/split/TieringSplitGenerator.java | 4 +- .../TieringSourceEnumeratorTest.java | 23 ++------ 8 files changed, 56 insertions(+), 109 deletions(-) diff --git a/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java b/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java index f62eb2236d..7e04c3a1cc 100644 --- a/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java +++ b/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java @@ -1851,15 +1851,6 @@ public class ConfigOptions { + ConfigOptions.TABLE_DATALAKE_AUTO_EXPIRE_SNAPSHOT + " is false."); - public static final ConfigOption LAKE_TIERING_TABLE_MAX_DURATION = - key("lake.tiering.table.max-duration") - .durationType() - .defaultValue(Duration.ofMinutes(30)) - .withDescription( - "The maximum duration for tiering a single table. If tiering a table exceeds this duration, " - + "it will be force completed: the tiering will be finalized and committed to the data lake " - + "(e.g., Paimon) immediately, even if they haven't reached their desired stopping offsets."); - // ------------------------------------------------------------------------ // ConfigOptions for fluss kafka // ------------------------------------------------------------------------ diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/LakeTieringJobBuilder.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/LakeTieringJobBuilder.java index f027bf8601..4c6fab4fa9 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/LakeTieringJobBuilder.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/LakeTieringJobBuilder.java @@ -17,6 +17,12 @@ package org.apache.fluss.flink.tiering; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.configuration.PipelineOptions; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; import org.apache.fluss.config.Configuration; import org.apache.fluss.flink.tiering.committer.CommittableMessageTypeInfo; import org.apache.fluss.flink.tiering.committer.TieringCommitOperatorFactory; @@ -27,14 +33,6 @@ import org.apache.fluss.lake.lakestorage.LakeStoragePluginSetUp; import org.apache.fluss.lake.writer.LakeTieringFactory; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.configuration.PipelineOptions; -import org.apache.flink.core.execution.JobClient; -import org.apache.flink.streaming.api.datastream.DataStreamSource; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; - -import static org.apache.fluss.config.ConfigOptions.LAKE_TIERING_TABLE_MAX_DURATION; import static org.apache.fluss.flink.tiering.source.TieringSource.TIERING_SOURCE_TRANSFORMATION_UID; import static org.apache.fluss.flink.tiering.source.TieringSourceOptions.POLL_TIERING_TABLE_INTERVAL; import static org.apache.fluss.utils.Preconditions.checkNotNull; @@ -91,11 +89,6 @@ public JobClient build() throws Exception { flussConfig.get(POLL_TIERING_TABLE_INTERVAL).toMillis()); } - if (lakeTieringConfig.get(LAKE_TIERING_TABLE_MAX_DURATION) != null) { - tieringSourceBuilder.withTieringTableMaxDurationMs( - lakeTieringConfig.get(LAKE_TIERING_TABLE_MAX_DURATION).toMillis()); - } - TieringSource tieringSource = tieringSourceBuilder.build(); DataStreamSource source = env.fromSource( diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSource.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSource.java index b2d8df8dfb..072547fc0d 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSource.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSource.java @@ -17,6 +17,17 @@ package org.apache.fluss.flink.tiering.source; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.streaming.api.graph.StreamGraphHasherV2; import org.apache.fluss.client.Connection; import org.apache.fluss.client.ConnectionFactory; import org.apache.fluss.config.Configuration; @@ -30,21 +41,8 @@ import org.apache.fluss.shaded.guava32.com.google.common.hash.Hasher; import org.apache.fluss.shaded.guava32.com.google.common.hash.Hashing; -import org.apache.flink.api.connector.source.Boundedness; -import org.apache.flink.api.connector.source.Source; -import org.apache.flink.api.connector.source.SourceReader; -import org.apache.flink.api.connector.source.SourceReaderContext; -import org.apache.flink.api.connector.source.SplitEnumerator; -import org.apache.flink.api.connector.source.SplitEnumeratorContext; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; -import org.apache.flink.core.io.SimpleVersionedSerializer; -import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.streaming.api.graph.StreamGraphHasherV2; - import java.nio.charset.StandardCharsets; -import static org.apache.fluss.config.ConfigOptions.LAKE_TIERING_TABLE_MAX_DURATION; import static org.apache.fluss.flink.tiering.source.TieringSourceOptions.POLL_TIERING_TABLE_INTERVAL; /** @@ -64,17 +62,14 @@ public class TieringSource private final Configuration flussConf; private final LakeTieringFactory lakeTieringFactory; private final long pollTieringTableIntervalMs; - private final long tieringTableDurationMaxMs; public TieringSource( Configuration flussConf, LakeTieringFactory lakeTieringFactory, - long pollTieringTableIntervalMs, - long tieringTableDurationMaxMs) { + long pollTieringTableIntervalMs) { this.flussConf = flussConf; this.lakeTieringFactory = lakeTieringFactory; this.pollTieringTableIntervalMs = pollTieringTableIntervalMs; - this.tieringTableDurationMaxMs = tieringTableDurationMaxMs; } @Override @@ -86,10 +81,7 @@ public Boundedness getBoundedness() { public SplitEnumerator createEnumerator( SplitEnumeratorContext splitEnumeratorContext) { return new TieringSourceEnumerator( - flussConf, - splitEnumeratorContext, - pollTieringTableIntervalMs, - tieringTableDurationMaxMs); + flussConf, splitEnumeratorContext, pollTieringTableIntervalMs); } @Override @@ -98,10 +90,7 @@ public SplitEnumerator restoreEnumer TieringSourceEnumeratorState tieringSourceEnumeratorState) { // stateless operator return new TieringSourceEnumerator( - flussConf, - splitEnumeratorContext, - pollTieringTableIntervalMs, - tieringTableDurationMaxMs); + flussConf, splitEnumeratorContext, pollTieringTableIntervalMs); } @Override @@ -140,8 +129,6 @@ public static class Builder { private final LakeTieringFactory lakeTieringFactory; private long pollTieringTableIntervalMs = POLL_TIERING_TABLE_INTERVAL.defaultValue().toMillis(); - private long tieringTableMaxDurationMs = - LAKE_TIERING_TABLE_MAX_DURATION.defaultValue().toMillis(); public Builder( Configuration flussConf, LakeTieringFactory lakeTieringFactory) { @@ -154,17 +141,8 @@ public Builder withPollTieringTableIntervalMs(long pollTieringTable return this; } - public Builder withTieringTableMaxDurationMs(long tieringTableDurationMaxMs) { - this.tieringTableMaxDurationMs = tieringTableDurationMaxMs; - return this; - } - public TieringSource build() { - return new TieringSource<>( - flussConf, - lakeTieringFactory, - pollTieringTableIntervalMs, - tieringTableMaxDurationMs); + return new TieringSource<>(flussConf, lakeTieringFactory, pollTieringTableIntervalMs); } } } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSourceFetcherManager.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSourceFetcherManager.java index 68db87671f..ded342e8e4 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSourceFetcherManager.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSourceFetcherManager.java @@ -18,15 +18,14 @@ package org.apache.fluss.flink.tiering.source; -import org.apache.fluss.flink.adapter.SingleThreadFetcherManagerAdapter; -import org.apache.fluss.flink.tiering.source.split.TieringSplit; - import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; import org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher; import org.apache.flink.connector.base.source.reader.fetcher.SplitFetcherTask; import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.fluss.flink.adapter.SingleThreadFetcherManagerAdapter; +import org.apache.fluss.flink.tiering.source.split.TieringSplit; import java.util.Collection; import java.util.function.Consumer; @@ -34,7 +33,7 @@ /** * The SplitFetcherManager for tiering source. This class is needed to help notify a table reaches - * to deadline of tiering to {@link TieringSplitReader}. + * the max duration of tiering to {@link TieringSplitReader}. */ public class TieringSourceFetcherManager extends SingleThreadFetcherManagerAdapter< diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java index 3ee566c421..4215ed162c 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java @@ -17,6 +17,11 @@ package org.apache.fluss.flink.tiering.source; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; +import org.apache.fluss.annotation.VisibleForTesting; import org.apache.fluss.client.Connection; import org.apache.fluss.client.table.Table; import org.apache.fluss.client.table.scanner.ScanRecord; @@ -33,16 +38,10 @@ import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.utils.CloseableIterator; - -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; -import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; -import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.annotation.Nullable; - import java.io.IOException; import java.time.Duration; import java.util.ArrayDeque; @@ -110,6 +109,7 @@ public TieringSplitReader( this(connection, lakeTieringFactory, DEFAULT_POLL_TIMEOUT); } + @VisibleForTesting protected TieringSplitReader( Connection connection, LakeTieringFactory lakeTieringFactory, diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumerator.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumerator.java index d2102db1d0..2f75d8e65d 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumerator.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumerator.java @@ -17,6 +17,13 @@ package org.apache.fluss.flink.tiering.source.enumerator; +import org.apache.flink.api.connector.source.ReaderInfo; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.metrics.groups.SplitEnumeratorMetricGroup; +import org.apache.flink.util.FlinkRuntimeException; import org.apache.fluss.annotation.VisibleForTesting; import org.apache.fluss.client.Connection; import org.apache.fluss.client.ConnectionFactory; @@ -30,6 +37,7 @@ import org.apache.fluss.flink.tiering.source.split.TieringSplit; import org.apache.fluss.flink.tiering.source.split.TieringSplitGenerator; import org.apache.fluss.flink.tiering.source.state.TieringSourceEnumeratorState; +import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.rpc.GatewayClientProxy; import org.apache.fluss.rpc.RpcClient; @@ -40,19 +48,10 @@ import org.apache.fluss.rpc.messages.PbLakeTieringTableInfo; import org.apache.fluss.rpc.metrics.ClientMetricGroup; import org.apache.fluss.utils.MapUtils; - -import org.apache.flink.api.connector.source.ReaderInfo; -import org.apache.flink.api.connector.source.SourceEvent; -import org.apache.flink.api.connector.source.SplitEnumerator; -import org.apache.flink.api.connector.source.SplitEnumeratorContext; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.metrics.groups.SplitEnumeratorMetricGroup; -import org.apache.flink.util.FlinkRuntimeException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.annotation.Nullable; - import java.io.IOException; import java.util.ArrayList; import java.util.Collections; @@ -98,13 +97,13 @@ public class TieringSourceEnumerator private final ScheduledExecutorService timerService; private final SplitEnumeratorMetricGroup enumeratorMetricGroup; private final long pollTieringTableIntervalMs; - private final long tieringTableMaxDurationMs; private final List pendingSplits; private final Set readersAwaitingSplit; private final Map tieringTableEpochs; private final Map failedTableEpochs; private final Map finishedTableEpochs; + private final Set tieringReachMaxDurationsTables; // lazily instantiated private RpcClient rpcClient; @@ -121,8 +120,7 @@ public class TieringSourceEnumerator public TieringSourceEnumerator( Configuration flussConf, SplitEnumeratorContext context, - long pollTieringTableIntervalMs, - long tieringTableMaxDurationMs) { + long pollTieringTableIntervalMs) { this.flussConf = flussConf; this.context = context; this.timerService = @@ -130,12 +128,12 @@ public TieringSourceEnumerator( r -> new Thread(r, "Tiering-Timer-Thread")); this.enumeratorMetricGroup = context.metricGroup(); this.pollTieringTableIntervalMs = pollTieringTableIntervalMs; - this.tieringTableMaxDurationMs = tieringTableMaxDurationMs; this.pendingSplits = Collections.synchronizedList(new ArrayList<>()); this.readersAwaitingSplit = Collections.synchronizedSet(new TreeSet<>()); this.tieringTableEpochs = MapUtils.newConcurrentHashMap(); this.finishedTableEpochs = MapUtils.newConcurrentHashMap(); this.failedTableEpochs = MapUtils.newConcurrentHashMap(); + this.tieringReachMaxDurationsTables = Collections.synchronizedSet(new TreeSet<>()); } @Override @@ -303,10 +301,8 @@ private void handleSourceReaderFailOver() { protected void handleTableTieringReachMaxDuration(long tableId, long tieringEpoch) { Long currentEpoch = tieringTableEpochs.get(tableId); if (currentEpoch != null && currentEpoch.equals(tieringEpoch)) { - LOG.info( - "Table {} reached max duration ({}ms). Force completing.", - tableId, - tieringTableMaxDurationMs); + LOG.info("Table {} reached max duration. Force completing.", tableId); + tieringReachMaxDurationsTables.add(tableId); for (TieringSplit tieringSplit : pendingSplits) { if (tieringSplit.getTableBucket().getTableId() == tableId) { @@ -416,9 +412,10 @@ private void generateTieringSplits(Tuple3 tieringTable) long start = System.currentTimeMillis(); LOG.info("Generate Tiering splits for table {}.", tieringTable.f2); try { + TablePath tablePath = tieringTable.f2; + final TableInfo tableInfo = flussAdmin.getTableInfo(tablePath).get(); List tieringSplits = - populateNumberOfTieringSplits( - splitGenerator.generateTableSplits(tieringTable.f2)); + populateNumberOfTieringSplits(splitGenerator.generateTableSplits(tableInfo)); // shuffle tiering split to avoid splits tiering skew // after introduce tiering max duration Collections.shuffle(tieringSplits); @@ -442,7 +439,9 @@ private void generateTieringSplits(Tuple3 tieringTable) () -> handleTableTieringReachMaxDuration( tieringTable.f0, tieringTable.f1)), - tieringTableMaxDurationMs, + + // for simplicity, we use the freshness as + tableInfo.getTableConfig().getDataLakeFreshness().toMillis(), TimeUnit.MILLISECONDS); } } catch (Exception e) { diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSplitGenerator.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSplitGenerator.java index 0fd814c14d..1bb94a7a92 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSplitGenerator.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/split/TieringSplitGenerator.java @@ -56,8 +56,8 @@ public TieringSplitGenerator(Admin flussAdmin) { this.flussAdmin = flussAdmin; } - public List generateTableSplits(TablePath tablePath) throws Exception { - final TableInfo tableInfo = flussAdmin.getTableInfo(tablePath).get(); + public List generateTableSplits(TableInfo tableInfo) throws Exception { + TablePath tablePath = tableInfo.getTablePath(); final BucketOffsetsRetriever bucketOffsetsRetriever = new BucketOffsetsRetrieverImpl(flussAdmin, tablePath); diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumeratorTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumeratorTest.java index 5984218109..9c2f493879 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumeratorTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumeratorTest.java @@ -17,6 +17,9 @@ package org.apache.fluss.flink.tiering.source.enumerator; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.api.connector.source.mocks.MockSplitEnumeratorContext; import org.apache.fluss.config.Configuration; import org.apache.fluss.flink.tiering.event.FailedTieringEvent; import org.apache.fluss.flink.tiering.event.FinishedTieringEvent; @@ -31,17 +34,11 @@ import org.apache.fluss.rpc.messages.CommitLakeTableSnapshotRequest; import org.apache.fluss.rpc.messages.PbLakeTableOffsetForBucket; import org.apache.fluss.rpc.messages.PbLakeTableSnapshotInfo; - -import org.apache.flink.api.connector.source.SourceEvent; -import org.apache.flink.api.connector.source.SplitsAssignment; -import org.apache.flink.api.connector.source.mocks.MockSplitEnumeratorContext; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import javax.annotation.Nullable; - -import java.time.Duration; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -710,14 +707,7 @@ private void verifyTieringSplitAssignment( private TieringSourceEnumerator createTieringSourceEnumerator( Configuration flussConf, MockSplitEnumeratorContext context) { - return createTieringSourceEnumerator(flussConf, context, Duration.ofSeconds(10).toMillis()); - } - - private TieringSourceEnumerator createTieringSourceEnumerator( - Configuration flussConf, - MockSplitEnumeratorContext context, - long tieringTableDurationMaxMs) { - return new TieringSourceEnumerator(flussConf, context, 500, tieringTableDurationMaxMs); + return new TieringSourceEnumerator(flussConf, context, 500); } @Test @@ -726,13 +716,10 @@ void testTableReachMaxTieringDuration() throws Throwable { long tableId = createTable(tablePath, DEFAULT_LOG_TABLE_DESCRIPTOR); int numSubtasks = 2; - long tieringTableDurationMaxMs = Duration.ofMinutes(10).toMillis(); - try (FlussMockSplitEnumeratorContext context = new FlussMockSplitEnumeratorContext<>(numSubtasks); TieringSourceEnumerator enumerator = - createTieringSourceEnumerator( - flussConf, context, tieringTableDurationMaxMs); ) { + createTieringSourceEnumerator(flussConf, context); ) { enumerator.start(); // Register all readers From 3c0655c8c72b907f165ec6b90f583f9328c4321a Mon Sep 17 00:00:00 2001 From: luoyuxia Date: Wed, 21 Jan 2026 21:14:39 +0800 Subject: [PATCH 5/5] use freshness as max tiering duration and mark forced finished table as pending again --- .../flink/tiering/LakeTieringJobBuilder.java | 13 +- .../flink/tiering/source/TieringSource.java | 23 ++-- .../source/TieringSourceFetcherManager.java | 5 +- .../tiering/source/TieringSplitReader.java | 10 +- .../enumerator/TieringSourceEnumerator.java | 114 +++++++++++++----- .../flink/tiering/FlinkTieringTestBase.java | 7 -- .../fluss/flink/tiering/TieringITCase.java | 10 +- .../TieringSourceEnumeratorTest.java | 112 ++++++++++++++--- fluss-rpc/src/main/proto/FlussApi.proto | 6 + .../coordinator/CoordinatorService.java | 12 +- .../coordinator/LakeTableTieringManager.java | 54 ++++++--- .../LakeTableTieringManagerTest.java | 31 ++++- 12 files changed, 290 insertions(+), 107 deletions(-) diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/LakeTieringJobBuilder.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/LakeTieringJobBuilder.java index 4c6fab4fa9..ab02d5b0e0 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/LakeTieringJobBuilder.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/LakeTieringJobBuilder.java @@ -17,12 +17,6 @@ package org.apache.fluss.flink.tiering; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.configuration.PipelineOptions; -import org.apache.flink.core.execution.JobClient; -import org.apache.flink.streaming.api.datastream.DataStreamSource; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; import org.apache.fluss.config.Configuration; import org.apache.fluss.flink.tiering.committer.CommittableMessageTypeInfo; import org.apache.fluss.flink.tiering.committer.TieringCommitOperatorFactory; @@ -33,6 +27,13 @@ import org.apache.fluss.lake.lakestorage.LakeStoragePluginSetUp; import org.apache.fluss.lake.writer.LakeTieringFactory; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.configuration.PipelineOptions; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; + import static org.apache.fluss.flink.tiering.source.TieringSource.TIERING_SOURCE_TRANSFORMATION_UID; import static org.apache.fluss.flink.tiering.source.TieringSourceOptions.POLL_TIERING_TABLE_INTERVAL; import static org.apache.fluss.utils.Preconditions.checkNotNull; diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSource.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSource.java index 072547fc0d..04448c1939 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSource.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSource.java @@ -17,17 +17,6 @@ package org.apache.fluss.flink.tiering.source; -import org.apache.flink.api.connector.source.Boundedness; -import org.apache.flink.api.connector.source.Source; -import org.apache.flink.api.connector.source.SourceReader; -import org.apache.flink.api.connector.source.SourceReaderContext; -import org.apache.flink.api.connector.source.SplitEnumerator; -import org.apache.flink.api.connector.source.SplitEnumeratorContext; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; -import org.apache.flink.core.io.SimpleVersionedSerializer; -import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.streaming.api.graph.StreamGraphHasherV2; import org.apache.fluss.client.Connection; import org.apache.fluss.client.ConnectionFactory; import org.apache.fluss.config.Configuration; @@ -41,6 +30,18 @@ import org.apache.fluss.shaded.guava32.com.google.common.hash.Hasher; import org.apache.fluss.shaded.guava32.com.google.common.hash.Hashing; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.streaming.api.graph.StreamGraphHasherV2; + import java.nio.charset.StandardCharsets; import static org.apache.fluss.flink.tiering.source.TieringSourceOptions.POLL_TIERING_TABLE_INTERVAL; diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSourceFetcherManager.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSourceFetcherManager.java index ded342e8e4..4f67e12195 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSourceFetcherManager.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSourceFetcherManager.java @@ -18,14 +18,15 @@ package org.apache.fluss.flink.tiering.source; +import org.apache.fluss.flink.adapter.SingleThreadFetcherManagerAdapter; +import org.apache.fluss.flink.tiering.source.split.TieringSplit; + import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; import org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher; import org.apache.flink.connector.base.source.reader.fetcher.SplitFetcherTask; import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; -import org.apache.fluss.flink.adapter.SingleThreadFetcherManagerAdapter; -import org.apache.fluss.flink.tiering.source.split.TieringSplit; import java.util.Collection; import java.util.function.Consumer; diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java index 4215ed162c..87daf64da1 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java @@ -17,10 +17,6 @@ package org.apache.fluss.flink.tiering.source; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; -import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; -import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; import org.apache.fluss.annotation.VisibleForTesting; import org.apache.fluss.client.Connection; import org.apache.fluss.client.table.Table; @@ -38,10 +34,16 @@ import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.utils.CloseableIterator; + +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.annotation.Nullable; + import java.io.IOException; import java.time.Duration; import java.util.ArrayDeque; diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumerator.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumerator.java index 2f75d8e65d..7fd36c656a 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumerator.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumerator.java @@ -17,13 +17,6 @@ package org.apache.fluss.flink.tiering.source.enumerator; -import org.apache.flink.api.connector.source.ReaderInfo; -import org.apache.flink.api.connector.source.SourceEvent; -import org.apache.flink.api.connector.source.SplitEnumerator; -import org.apache.flink.api.connector.source.SplitEnumeratorContext; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.metrics.groups.SplitEnumeratorMetricGroup; -import org.apache.flink.util.FlinkRuntimeException; import org.apache.fluss.annotation.VisibleForTesting; import org.apache.fluss.client.Connection; import org.apache.fluss.client.ConnectionFactory; @@ -48,10 +41,19 @@ import org.apache.fluss.rpc.messages.PbLakeTieringTableInfo; import org.apache.fluss.rpc.metrics.ClientMetricGroup; import org.apache.fluss.utils.MapUtils; + +import org.apache.flink.api.connector.source.ReaderInfo; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.metrics.groups.SplitEnumeratorMetricGroup; +import org.apache.flink.util.FlinkRuntimeException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.annotation.Nullable; + import java.io.IOException; import java.util.ArrayList; import java.util.Collections; @@ -102,7 +104,7 @@ public class TieringSourceEnumerator private final Map tieringTableEpochs; private final Map failedTableEpochs; - private final Map finishedTableEpochs; + private final Map finishedTables; private final Set tieringReachMaxDurationsTables; // lazily instantiated @@ -131,7 +133,7 @@ public TieringSourceEnumerator( this.pendingSplits = Collections.synchronizedList(new ArrayList<>()); this.readersAwaitingSplit = Collections.synchronizedSet(new TreeSet<>()); this.tieringTableEpochs = MapUtils.newConcurrentHashMap(); - this.finishedTableEpochs = MapUtils.newConcurrentHashMap(); + this.finishedTables = MapUtils.newConcurrentHashMap(); this.failedTableEpochs = MapUtils.newConcurrentHashMap(); this.tieringReachMaxDurationsTables = Collections.synchronizedSet(new TreeSet<>()); } @@ -179,8 +181,24 @@ public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname } LOG.info("TieringSourceReader {} requests split.", subtaskId); readersAwaitingSplit.add(subtaskId); - this.context.callAsync( - this::requestTieringTableSplitsViaHeartBeat, this::generateAndAssignSplits); + + // If pending splits exist, assign them directly to the requesting reader + if (!pendingSplits.isEmpty()) { + assignSplits(); + } else { + // Note: Ideally, only one table should be tiering at a time. + // Here we block to request a tiering table synchronously to avoid multiple threads + // requesting tiering tables concurrently, which would cause the enumerator to contain + // multiple tiering tables simultaneously. This is not optimal for tiering performance. + Tuple3 tieringTable = null; + Throwable throwable = null; + try { + tieringTable = this.requestTieringTableSplitsViaHeartBeat(); + } catch (Throwable t) { + throwable = t; + } + this.generateAndAssignSplits(tieringTable, throwable); + } } @Override @@ -252,7 +270,9 @@ public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { "The finished table {} is not in tiering table, won't report it to Fluss to mark as finished.", finishedTableId); } else { - finishedTableEpochs.put(finishedTableId, tieringEpoch); + boolean isForceFinished = tieringReachMaxDurationsTables.remove(finishedTableId); + finishedTables.put( + finishedTableId, TieringFinishInfo.from(tieringEpoch, isForceFinished)); } } @@ -274,7 +294,7 @@ public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { } } - if (!finishedTableEpochs.isEmpty() || !failedTableEpochs.isEmpty()) { + if (!finishedTables.isEmpty() || !failedTableEpochs.isEmpty()) { // call one round of heartbeat to notify table has been finished or failed this.context.callAsync( this::requestTieringTableSplitsViaHeartBeat, this::generateAndAssignSplits); @@ -288,6 +308,7 @@ private void handleSourceReaderFailOver() { // we need to make all as failed failedTableEpochs.putAll(new HashMap<>(tieringTableEpochs)); tieringTableEpochs.clear(); + tieringReachMaxDurationsTables.clear(); // also clean all pending splits since we mark all as failed pendingSplits.clear(); if (!failedTableEpochs.isEmpty()) { @@ -298,10 +319,11 @@ private void handleSourceReaderFailOver() { } @VisibleForTesting - protected void handleTableTieringReachMaxDuration(long tableId, long tieringEpoch) { + protected void handleTableTieringReachMaxDuration( + TablePath tablePath, long tableId, long tieringEpoch) { Long currentEpoch = tieringTableEpochs.get(tableId); if (currentEpoch != null && currentEpoch.equals(tieringEpoch)) { - LOG.info("Table {} reached max duration. Force completing.", tableId); + LOG.info("Table {}-{} reached max duration. Force completing.", tablePath, tableId); tieringReachMaxDurationsTables.add(tableId); for (TieringSplit tieringSplit : pendingSplits) { @@ -309,11 +331,6 @@ protected void handleTableTieringReachMaxDuration(long tableId, long tieringEpoc // mark this tiering split to skip the current round since the tiering for // this table has timed out, so the tiering source reader can skip them directly tieringSplit.skipCurrentRound(); - } else { - // we can break directly, if found any one split's table id is not equal to the - // timeout - // table, the following split must be not equal to the table id - break; } } @@ -362,13 +379,13 @@ private void assignSplits() { if (closed) { return null; } - Map currentFinishedTableEpochs = new HashMap<>(this.finishedTableEpochs); + Map currentFinishedTables = new HashMap<>(this.finishedTables); Map currentFailedTableEpochs = new HashMap<>(this.failedTableEpochs); LakeTieringHeartbeatRequest tieringHeartbeatRequest = tieringTableHeartBeat( basicHeartBeat(), this.tieringTableEpochs, - currentFinishedTableEpochs, + currentFinishedTables, currentFailedTableEpochs, this.flussCoordinatorEpoch); @@ -397,9 +414,9 @@ private void assignSplits() { waitHeartbeatResponse(coordinatorGateway.lakeTieringHeartbeat(tieringHeartbeatRequest)); } - // if come to here, we can remove currentFinishedTableEpochs/failedTableEpochs to avoid send + // if come to here, we can remove currentFinishedTables/failedTableEpochs to avoid send // in next round - currentFinishedTableEpochs.forEach(finishedTableEpochs::remove); + currentFinishedTables.forEach(finishedTables::remove); currentFailedTableEpochs.forEach(failedTableEpochs::remove); return lakeTieringInfo; } @@ -428,7 +445,7 @@ private void generateTieringSplits(Tuple3 tieringTable) LOG.info( "Generate Tiering splits for table {} is empty, no need to tier data.", tieringTable.f2.getTableName()); - finishedTableEpochs.put(tieringTable.f0, tieringTable.f1); + finishedTables.put(tieringTable.f0, TieringFinishInfo.from(tieringTable.f1)); } else { tieringTableEpochs.put(tieringTable.f0, tieringTable.f1); pendingSplits.addAll(tieringSplits); @@ -438,7 +455,9 @@ private void generateTieringSplits(Tuple3 tieringTable) context.runInCoordinatorThread( () -> handleTableTieringReachMaxDuration( - tieringTable.f0, tieringTable.f1)), + tablePath, + tieringTable.f0, + tieringTable.f1)), // for simplicity, we use the freshness as tableInfo.getTableConfig().getDataLakeFreshness().toMillis(), @@ -537,16 +556,28 @@ static LakeTieringHeartbeatRequest heartBeatWithRequestNewTieringTable( static LakeTieringHeartbeatRequest tieringTableHeartBeat( LakeTieringHeartbeatRequest heartbeatRequest, Map tieringTableEpochs, - Map finishedTableEpochs, + Map finishedTables, Map failedTableEpochs, int coordinatorEpoch) { if (!tieringTableEpochs.isEmpty()) { heartbeatRequest.addAllTieringTables( toPbHeartbeatReqForTable(tieringTableEpochs, coordinatorEpoch)); } - if (!finishedTableEpochs.isEmpty()) { + if (!finishedTables.isEmpty()) { + Map finishTieringEpochs = new HashMap<>(); + Set forceFinishedTables = new HashSet<>(); + finishedTables.forEach( + (tableId, tieringFinishInfo) -> { + finishTieringEpochs.put(tableId, tieringFinishInfo.tieringEpoch); + if (tieringFinishInfo.isForceFinished) { + forceFinishedTables.add(tableId); + } + }); heartbeatRequest.addAllFinishedTables( - toPbHeartbeatReqForTable(finishedTableEpochs, coordinatorEpoch)); + toPbHeartbeatReqForTable(finishTieringEpochs, coordinatorEpoch)); + for (long forceFinishedTableId : forceFinishedTables) { + heartbeatRequest.addForceFinishedTable(forceFinishedTableId); + } } // add failed tiering table to heart beat request return failedTableHeartBeat(heartbeatRequest, failedTableEpochs, coordinatorEpoch); @@ -590,4 +621,29 @@ static LakeTieringHeartbeatResponse waitHeartbeatResponse( } } } + + private static class TieringFinishInfo { + /** The epoch of the tiering operation for this table. */ + long tieringEpoch; + + /** + * Whether this table was force finished due to reaching the maximum tiering duration. When + * a table's tiering operation exceeds the max duration (data lake freshness), it will be + * force finished to prevent it from blocking other tables' tiering operations. + */ + boolean isForceFinished; + + public static TieringFinishInfo from(long tieringEpoch) { + return new TieringFinishInfo(tieringEpoch, false); + } + + public static TieringFinishInfo from(long tieringEpoch, boolean isForceFinished) { + return new TieringFinishInfo(tieringEpoch, isForceFinished); + } + + private TieringFinishInfo(long tieringEpoch, boolean isForceFinished) { + this.tieringEpoch = tieringEpoch; + this.isForceFinished = isForceFinished; + } + } } diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/FlinkTieringTestBase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/FlinkTieringTestBase.java index 0971231ff0..7e3ee48a78 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/FlinkTieringTestBase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/FlinkTieringTestBase.java @@ -170,13 +170,6 @@ protected void writeRows(TablePath tablePath, List rows, boolean ap } } - protected void waitUntilSnapshot(long tableId, int bucketNum, long snapshotId) { - for (int i = 0; i < bucketNum; i++) { - TableBucket tableBucket = new TableBucket(tableId, i); - FLUSS_CLUSTER_EXTENSION.waitUntilSnapshotFinished(tableBucket, snapshotId); - } - } - public List getValuesRecords(TablePath tablePath) { return TestingValuesLake.getResults(tablePath.toString()); } diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/TieringITCase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/TieringITCase.java index e542b9316d..fe81b5a842 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/TieringITCase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/TieringITCase.java @@ -44,7 +44,6 @@ import java.util.Map; import java.util.Optional; -import static org.apache.fluss.config.ConfigOptions.LAKE_TIERING_TABLE_MAX_DURATION; import static org.apache.fluss.testutils.common.CommonTestUtils.waitValue; import static org.assertj.core.api.Assertions.assertThat; @@ -75,7 +74,7 @@ void testTieringReachMaxDuration() throws Exception { TablePath logTablePath = TablePath.of("fluss", "logtable"); createTable(logTablePath, false); TablePath pkTablePath = TablePath.of("fluss", "pktable"); - long pkTableId = createTable(pkTablePath, true); + createTable(pkTablePath, true); // write some records to log table List rows = new ArrayList<>(); @@ -92,11 +91,10 @@ void testTieringReachMaxDuration() throws Exception { } writeRows(pkTablePath, rows, false); - waitUntilSnapshot(pkTableId, 3, 0); + FLUSS_CLUSTER_EXTENSION.triggerAndWaitSnapshot(pkTablePath); // set tiering duration to a small value for testing purpose Configuration lakeTieringConfig = new Configuration(); - lakeTieringConfig.set(LAKE_TIERING_TABLE_MAX_DURATION, Duration.ofSeconds(1)); JobClient jobClient = buildTieringJob(execEnv, lakeTieringConfig); try { @@ -139,7 +137,7 @@ private LakeSnapshot waitLakeSnapshot(TablePath tablePath) { "Fail to wait for one round of tiering finish for table " + tablePath); } - private long createTable(TablePath tablePath, boolean isPrimaryKeyTable) throws Exception { + private void createTable(TablePath tablePath, boolean isPrimaryKeyTable) throws Exception { Schema.Builder schemaBuilder = Schema.newBuilder().column("a", DataTypes.INT()).column("b", DataTypes.STRING()); if (isPrimaryKeyTable) { @@ -149,7 +147,7 @@ private long createTable(TablePath tablePath, boolean isPrimaryKeyTable) throws // see TestingPaimonStoragePlugin#TestingPaimonWriter, we set write-pause // to 1s to make it easy to mock tiering reach max duration Map customProperties = Collections.singletonMap("write-pause", "1s"); - return createTable( + createTable( tablePath, 3, Collections.singletonList("a"), diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumeratorTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumeratorTest.java index 9c2f493879..7ea150049e 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumeratorTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumeratorTest.java @@ -17,9 +17,7 @@ package org.apache.fluss.flink.tiering.source.enumerator; -import org.apache.flink.api.connector.source.SourceEvent; -import org.apache.flink.api.connector.source.SplitsAssignment; -import org.apache.flink.api.connector.source.mocks.MockSplitEnumeratorContext; +import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; import org.apache.fluss.flink.tiering.event.FailedTieringEvent; import org.apache.fluss.flink.tiering.event.FinishedTieringEvent; @@ -30,15 +28,22 @@ import org.apache.fluss.flink.tiering.source.split.TieringSplit; import org.apache.fluss.flink.tiering.source.split.TieringSplitGenerator; import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TableChange; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.rpc.messages.CommitLakeTableSnapshotRequest; import org.apache.fluss.rpc.messages.PbLakeTableOffsetForBucket; import org.apache.fluss.rpc.messages.PbLakeTableSnapshotInfo; + +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.api.connector.source.mocks.MockSplitEnumeratorContext; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import javax.annotation.Nullable; + +import java.time.Duration; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -48,6 +53,7 @@ import static org.apache.fluss.client.table.scanner.log.LogScanner.EARLIEST_OFFSET; import static org.apache.fluss.config.ConfigOptions.TABLE_AUTO_PARTITION_NUM_PRECREATE; +import static org.apache.fluss.testutils.common.CommonTestUtils.retry; import static org.assertj.core.api.Assertions.assertThat; /** Unit tests for {@link TieringSourceEnumerator} and {@link TieringSplitGenerator}. */ @@ -627,11 +633,16 @@ void testHandleReaderFailOver() throws Throwable { // readers failover again: Enumerator marks tablePath2 as failed and clears its splits // register readers and process split requests for attempt 2 - registerReaderAndHandleSplitRequests(context, enumerator, 3, 2); + registerReaderAndHandleSplitRequests(context, enumerator, 2, 2); // now, should get no tiering split, since all pending split for tablePath1 and - // tablePath2 is clear + // tablePath2 is clear and there still one sub-task is not registered verifyTieringSplitAssignment(context, 0, tablePath2); + + // register reader 2 again, should get tiering split for table1 since the failover is + // finished, and reader2 request tiering split + registerSingleReaderAndHandleSplitRequests(context, enumerator, 2, 2); + verifyTieringSplitAssignment(context, 3, tablePath1); } } @@ -665,12 +676,21 @@ private void registerReaderAndHandleSplitRequests( int numSubtasks, int attemptNumber) { for (int subtaskId = 0; subtaskId < numSubtasks; subtaskId++) { - context.registerSourceReader(subtaskId, attemptNumber, "localhost-" + subtaskId); - enumerator.addReader(subtaskId); - enumerator.handleSplitRequest(subtaskId, "localhost-" + subtaskId); + registerSingleReaderAndHandleSplitRequests( + context, enumerator, subtaskId, attemptNumber); } } + private void registerSingleReaderAndHandleSplitRequests( + FlussMockSplitEnumeratorContext context, + TieringSourceEnumerator enumerator, + int subtaskId, + int attemptNumber) { + context.registerSourceReader(subtaskId, attemptNumber, "localhost-" + subtaskId); + enumerator.addReader(subtaskId); + enumerator.handleSplitRequest(subtaskId, "localhost-" + subtaskId); + } + private void waitUntilTieringTableSplitAssignmentReady( FlussMockSplitEnumeratorContext context, int expectedSplitsNum, @@ -736,17 +756,20 @@ void testTableReachMaxTieringDuration() throws Throwable { // Wait for initial assignment waitUntilTieringTableSplitAssignmentReady(context, 2, 200L); - // call handleTableTieringReachMaxDuration to mock tiering reach max duration - enumerator.handleTableTieringReachMaxDuration(tableId, 1); - - // Verify that TieringReachMaxDurationEvent was sent to all readers - // Use reflection to access events sent to readers - Map> eventsToReaders = context.getSentSourceEvent(); - assertThat(eventsToReaders).hasSize(numSubtasks); - for (Map.Entry> entry : eventsToReaders.entrySet()) { - assertThat(entry.getValue()) - .containsExactly(new TieringReachMaxDurationEvent(tableId)); - } + retry( + Duration.ofSeconds(30), + () -> { + // Verify that TieringReachMaxDurationEvent was sent to all readers + // Use reflection to access events sent to readers + Map> eventsToReaders = + context.getSentSourceEvent(); + assertThat(eventsToReaders).hasSize(numSubtasks); + for (Map.Entry> entry : + eventsToReaders.entrySet()) { + assertThat(entry.getValue()) + .containsExactly(new TieringReachMaxDurationEvent(tableId)); + } + }); // clear split assignment context.getSplitsAssignmentSequence().clear(); @@ -762,6 +785,57 @@ void testTableReachMaxTieringDuration() throws Throwable { .forEach(a -> a.assignment().values().forEach(assignedSplits::addAll)); assertThat(assignedSplits).hasSize(1); assertThat(assignedSplits.get(0).shouldSkipCurrentRound()).isTrue(); + + // alter table freshness to 10 min to make sure we won't assign in + // normal finish + conn.getAdmin() + .alterTable( + tablePath, + Collections.singletonList( + TableChange.set( + ConfigOptions.TABLE_DATALAKE_FRESHNESS.key(), "10min")), + false) + .get(); + + // Mock tiering finished + // This simulates the reader finishing tiering after reaching max duration + enumerator.handleSourceEvent(0, new FinishedTieringEvent(tableId)); + + // Clear split assignment to prepare for next round + context.getSplitsAssignmentSequence().clear(); + + // Run periodic callable to trigger heartbeat, which will report force finish to + // coordinator + // The coordinator will move the table from Tiered to Pending state + if (!context.getPeriodicCallables().isEmpty()) { + context.runPeriodicCallable(0); + } + + // Request table again - since the table was force finished, it should be immediately + // available in Pending state + for (int subTask = 0; subTask < numSubtasks; subTask++) { + enumerator.handleSplitRequest(subTask, "localhost-" + subTask); + } + + // Run periodic callable again to request table from coordinator + if (!context.getPeriodicCallables().isEmpty()) { + context.runPeriodicCallable(0); + } + + // The table should be immediately requested again (epoch should be 2) + // Wait for the table to be assigned again + waitUntilTieringTableSplitAssignmentReady(context, 2, 500L); + + List reassignedSplits = new ArrayList<>(); + context.getSplitsAssignmentSequence() + .forEach(a -> a.assignment().values().forEach(reassignedSplits::addAll)); + assertThat(reassignedSplits).hasSize(2); + // Verify that the table is requested again with a new epoch (epoch 2) + assertThat(reassignedSplits) + .allMatch( + split -> + split.getTableBucket().getTableId() == tableId + && !split.shouldSkipCurrentRound()); } } } diff --git a/fluss-rpc/src/main/proto/FlussApi.proto b/fluss-rpc/src/main/proto/FlussApi.proto index db9d614354..a9cba3f476 100644 --- a/fluss-rpc/src/main/proto/FlussApi.proto +++ b/fluss-rpc/src/main/proto/FlussApi.proto @@ -540,6 +540,12 @@ message LakeTieringHeartbeatRequest { repeated PbHeartbeatReqForTable failed_tables = 3; // whether to request a table optional bool request_table = 4; + // Table IDs that were force finished due to reaching the maximum tiering duration. + // When a table's tiering operation exceeds the max duration (data lake freshness), + // it will be force finished to prevent it from blocking other tables' tiering operations. + // These table IDs must also appear in finished_tables. The coordinator uses this field to + // distinguish between normally finished tables and force finished tables. + repeated int64 force_finished_tables = 5; } message LakeTieringHeartbeatResponse { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorService.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorService.java index 94d56dd338..05bc28e18a 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorService.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorService.java @@ -150,9 +150,11 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.function.Supplier; @@ -743,6 +745,12 @@ public CompletableFuture lakeTieringHeartbeat( } } + // process force finished tables + Set forceFinishedTableId = new HashSet<>(); + for (long forceFinishTableId : request.getForceFinishedTables()) { + forceFinishedTableId.add(forceFinishTableId); + } + // process finished tables for (PbHeartbeatReqForTable finishTable : request.getFinishedTablesList()) { PbHeartbeatRespForTable pbHeartbeatRespForTable = @@ -750,7 +758,9 @@ public CompletableFuture lakeTieringHeartbeat( try { validateHeartbeatRequest(finishTable, currentCoordinatorEpoch); lakeTableTieringManager.finishTableTiering( - finishTable.getTableId(), finishTable.getTieringEpoch()); + finishTable.getTableId(), + finishTable.getTieringEpoch(), + forceFinishedTableId.contains(finishTable.getTableId())); } catch (Throwable e) { pbHeartbeatRespForTable.setError(ApiError.fromThrowable(e).toErrorResponse()); } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/LakeTableTieringManager.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/LakeTableTieringManager.java index c25c779691..a5267e0b7d 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/LakeTableTieringManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/LakeTableTieringManager.java @@ -83,19 +83,22 @@ * ▼ ▼ * ┌──────────┐ (lake freshness > tiering interval) * │Scheduled ├──────┐ - * └──────────┘ ▼ - * ▲ ┌───────┐ (assign to tier service) ┌───────┐ - * | |Pending├──────────────────────────►|Tiering├─┐ - * | └───────┘ └───┬───┘ │ - * | ▲ ┌─────────────────┘ │ - * | | | (timeout or failure) | (finished) - * | | ▼ ▼ - * | | (retry) ┌─────────┐ ┌────────┐ - * | └────────────│ Failed │ │ Tiered │ - * | └─────────┘ └────┬───┘ - * | | - * └──────────────────────────────────────────────────────┘ - * (ready for next round of tiering) + * └─────▲────┘ ▼ + * │ ┌───────┐ (assign to tier service) ┌───────┐ + * │ |Pending├──────────────────────────►|Tiering├─┐ + * │ └───▲───┘ └───┬───┘ │ + * │ │ ┌─────────────────┘ │ + * │ │ | (timeout or failure) | (finished) + * │ │ ▼ ▼ + * │ │ (retry) ┌─────────┐ ┌────────┐ + * │ │ │ Failed │ │ Tiered │ + * │ │ └─────────┘ └───┬────┘ + * │ │ │ + * │ │ (force finished) │ (ready for next round) + * │ └───────────────────────────────────────-┘ + * │ │ + * └───────────────────────────────────────────────────┘ + * (ready for next round of tiering) * } */ public class LakeTableTieringManager implements AutoCloseable { @@ -330,15 +333,20 @@ public LakeTieringTableInfo requestTable() { }); } - public void finishTableTiering(long tableId, long tieredEpoch) { + public void finishTableTiering(long tableId, long tieredEpoch, boolean isForceFinished) { inLock( lock, () -> { validateTieringServiceRequest(tableId, tieredEpoch); // to tiered state firstly doHandleStateChange(tableId, TieringState.Tiered); - // then to scheduled state to enable other tiering service can pick it - doHandleStateChange(tableId, TieringState.Scheduled); + if (isForceFinished) { + // add to pending again since it's forced to finish + doHandleStateChange(tableId, TieringState.Pending); + } else { + // then to scheduled state to enable other tiering service can pick it + doHandleStateChange(tableId, TieringState.Scheduled); + } }); } @@ -420,7 +428,17 @@ private void validateTieringServiceRequest(long tableId, long tieringEpoch) { *

Tiering -> Failed * *

-- When the tiering service timeout to report heartbeat or report failure for the table, - * do: transmit to Tiered state + * do: transmit to Failed state + * + *

Tiered -> Pending + * + *

-- When the tiering is force finished due to exceeding the specified tiering duration, do: + * transmit to Pending state to enable immediate re-tiering + * + *

Tiered -> Scheduled + * + *

-- When the tiering is normally finished, do: transmit to Scheduled state to wait for the + * next round of tiering */ private void doHandleStateChange(long tableId, TieringState targetState) { TieringState currentState = tieringStates.get(tableId); @@ -570,7 +588,7 @@ public Set validPreviousStates() { Pending { @Override public Set validPreviousStates() { - return EnumSet.of(Scheduled, Failed); + return EnumSet.of(Scheduled, Failed, Tiered); } }, // When one tiering service is tiering the table, the state will be Tiering diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/LakeTableTieringManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/LakeTableTieringManagerTest.java index fb0b75b18a..80b589dc06 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/LakeTableTieringManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/LakeTableTieringManagerTest.java @@ -131,7 +131,7 @@ void testRemoveLakeTable() { assertThatThrownBy(() -> tableTieringManager.reportTieringFail(tableId1, 1)) .isInstanceOf(TableNotExistException.class) .hasMessage("The table %d doesn't exist.", tableId1); - assertThatThrownBy(() -> tableTieringManager.finishTableTiering(tableId1, 1)) + assertThatThrownBy(() -> tableTieringManager.finishTableTiering(tableId1, 1, false)) .isInstanceOf(TableNotExistException.class) .hasMessage("The table %d doesn't exist.", tableId1); } @@ -152,7 +152,7 @@ void testFinishTableTieringReTriggerSchedule() { assertThat(tableTieringManager.requestTable()).isNull(); // mock lake tiering finish one-round tiering - tableTieringManager.finishTableTiering(tableId1, tieredEpoch); + tableTieringManager.finishTableTiering(tableId1, tieredEpoch, false); // not advance time, request table should return null assertThat(tableTieringManager.requestTable()).isNull(); @@ -211,12 +211,12 @@ void testTieringServiceTimeOutReTriggerPending() { .hasMessage( "The tiering epoch %d is not match current epoch %d in coordinator for table %d.", 1, 2, tableId1); - assertThatThrownBy(() -> tableTieringManager.finishTableTiering(tableId1, 1)) + assertThatThrownBy(() -> tableTieringManager.finishTableTiering(tableId1, 1, false)) .isInstanceOf(FencedTieringEpochException.class) .hasMessage( "The tiering epoch %d is not match current epoch %d in coordinator for table %d.", 1, 2, tableId1); - assertThatThrownBy(() -> tableTieringManager.finishTableTiering(tableId1, 3)) + assertThatThrownBy(() -> tableTieringManager.finishTableTiering(tableId1, 3, false)) .isInstanceOf(FencedTieringEpochException.class) .hasMessage( "The tiering epoch %d is not match current epoch %d in coordinator for table %d.", @@ -238,6 +238,29 @@ void testTieringFail() { assertRequestTable(tableId1, tablePath1, 2); } + @Test + void testForceFinishTableTieringImmediatelyRePending() { + long tableId1 = 1L; + TablePath tablePath1 = TablePath.of("db", "table1"); + TableInfo tableInfo1 = createTableInfo(tableId1, tablePath1, Duration.ofSeconds(10)); + tableTieringManager.addNewLakeTable(tableInfo1); + manualClock.advanceTime(Duration.ofSeconds(10)); + // check requested table + assertRequestTable(tableId1, tablePath1, 1); + + // request table should return null since the table is tiering + assertThat(tableTieringManager.requestTable()).isNull(); + + // mock lake tiering force finish (e.g., due to exceeding tiering duration) + tableTieringManager.finishTableTiering(tableId1, 1, true); + // should immediately be re-pending and can be requested again without waiting + assertRequestTable(tableId1, tablePath1, 2); + + // verify it can be requested again immediately after force finish + // request table should return null since the table is tiering + assertThat(tableTieringManager.requestTable()).isNull(); + } + private TableInfo createTableInfo(long tableId, TablePath tablePath, Duration freshness) { TableDescriptor tableDescriptor = TableDescriptor.builder()