diff --git a/fluss-client/src/main/java/org/apache/fluss/client/admin/Admin.java b/fluss-client/src/main/java/org/apache/fluss/client/admin/Admin.java index b5ae7b06b9..2a5b39ba9a 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/admin/Admin.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/admin/Admin.java @@ -409,21 +409,67 @@ CompletableFuture getKvSnapshotMetadata( TableBucket bucket, long snapshotId); /** - * Get table lake snapshot info of the given table asynchronously. + * Retrieves the absolute latest lake snapshot metadata for a table asynchronously. * - *

It'll get the latest snapshot for all the buckets of the table. + *

This returns the most recent snapshot regardless of its visibility or compaction status. + * It includes the latest tiered offsets for all buckets. * - *

The following exceptions can be anticipated when calling {@code get()} on returned future. + *

Exceptions expected when calling {@code get()} on the returned future: * *

* - * @param tablePath the table path of the table. + * @param tablePath The path of the target table. + * @return A future returning the latest tiered snapshot. */ CompletableFuture getLatestLakeSnapshot(TablePath tablePath); + /** + * Retrieves a specific historical lake snapshot by its ID asynchronously. + * + *

It provides the tiered bucket offsets as they existed at the moment the specified snapshot + * was committed. + * + *

Exceptions expected when calling {@code get()} on the returned future: + * + *

+ * + * @param tablePath The path of the target table. + * @param snapshotId The unique identifier of the snapshot. + * @return A future returning the specific lake snapshot. + */ + CompletableFuture getLakeSnapshot(TablePath tablePath, long snapshotId); + + /** + * Retrieves the latest readable lake snapshot and its corresponding readable log offsets. + * + *

For Paimon DV tables, the tiered log offset may not be readable because the corresponding + * data might be in the L0 layer. Using tiered offset directly can lead to data loss. This + * method returns a readable snapshot (where L0 data has been compacted) and its corresponding + * readable offsets, which represent safe log offsets that can be read without data loss. + * + *

For union read operations, use this method instead of {@link + * #getLatestLakeSnapshot(TablePath)} to ensure data safety and avoid data loss. + * + *

Exceptions expected when calling {@code get()} on the returned future: + * + *

    + *
  • {@link TableNotExistException}: If the table path is invalid. + *
  • {@link LakeTableSnapshotNotExistException}: If no readable snapshot exists yet. + *
+ * + * @param tablePath The path of the target table. + * @return A future returning a {@link LakeSnapshot} containing the readable snapshot ID and + * readable log offsets for each bucket. + */ + CompletableFuture getReadableLakeSnapshot(TablePath tablePath); + /** * List offset for the specified buckets. This operation enables to find the beginning offset, * end offset as well as the offset matching a timestamp in buckets. diff --git a/fluss-client/src/main/java/org/apache/fluss/client/admin/FlussAdmin.java b/fluss-client/src/main/java/org/apache/fluss/client/admin/FlussAdmin.java index ac34caed87..cd4e221ef5 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/admin/FlussAdmin.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/admin/FlussAdmin.java @@ -60,8 +60,8 @@ import org.apache.fluss.rpc.messages.DropTableRequest; import org.apache.fluss.rpc.messages.GetDatabaseInfoRequest; import org.apache.fluss.rpc.messages.GetKvSnapshotMetadataRequest; +import org.apache.fluss.rpc.messages.GetLakeSnapshotRequest; import org.apache.fluss.rpc.messages.GetLatestKvSnapshotsRequest; -import org.apache.fluss.rpc.messages.GetLatestLakeSnapshotRequest; import org.apache.fluss.rpc.messages.GetTableInfoRequest; import org.apache.fluss.rpc.messages.GetTableSchemaRequest; import org.apache.fluss.rpc.messages.ListAclsRequest; @@ -383,13 +383,38 @@ public CompletableFuture getKvSnapshotMetadata( @Override public CompletableFuture getLatestLakeSnapshot(TablePath tablePath) { - GetLatestLakeSnapshotRequest request = new GetLatestLakeSnapshotRequest(); + GetLakeSnapshotRequest request = new GetLakeSnapshotRequest(); request.setTablePath() .setDatabaseName(tablePath.getDatabaseName()) .setTableName(tablePath.getTableName()); return readOnlyGateway - .getLatestLakeSnapshot(request) + .getLakeSnapshot(request) + .thenApply(ClientRpcMessageUtils::toLakeTableSnapshotInfo); + } + + @Override + public CompletableFuture getLakeSnapshot(TablePath tablePath, long snapshotId) { + GetLakeSnapshotRequest request = new GetLakeSnapshotRequest(); + request.setTablePath() + .setDatabaseName(tablePath.getDatabaseName()) + .setTableName(tablePath.getTableName()); + request.setSnapshotId(snapshotId); + + return readOnlyGateway + .getLakeSnapshot(request) + .thenApply(ClientRpcMessageUtils::toLakeTableSnapshotInfo); + } + + @Override + public CompletableFuture getReadableLakeSnapshot(TablePath tablePath) { + GetLakeSnapshotRequest request = new GetLakeSnapshotRequest(); + request.setTablePath() + .setDatabaseName(tablePath.getDatabaseName()) + .setTableName(tablePath.getTableName()); + request.setReadable(true); + return readOnlyGateway + .getLakeSnapshot(request) .thenApply(ClientRpcMessageUtils::toLakeTableSnapshotInfo); } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java b/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java index 3e349cca3a..50c9e8c07d 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java @@ -42,8 +42,8 @@ import org.apache.fluss.rpc.messages.DropPartitionRequest; import org.apache.fluss.rpc.messages.GetFileSystemSecurityTokenResponse; import org.apache.fluss.rpc.messages.GetKvSnapshotMetadataResponse; +import org.apache.fluss.rpc.messages.GetLakeSnapshotResponse; import org.apache.fluss.rpc.messages.GetLatestKvSnapshotsResponse; -import org.apache.fluss.rpc.messages.GetLatestLakeSnapshotResponse; import org.apache.fluss.rpc.messages.ListOffsetsRequest; import org.apache.fluss.rpc.messages.ListPartitionInfosResponse; import org.apache.fluss.rpc.messages.LookupRequest; @@ -210,7 +210,7 @@ public static KvSnapshotMetadata toKvSnapshotMetadata(GetKvSnapshotMetadataRespo toFsPathAndFileName(response.getSnapshotFilesList()), response.getLogOffset()); } - public static LakeSnapshot toLakeTableSnapshotInfo(GetLatestLakeSnapshotResponse response) { + public static LakeSnapshot toLakeTableSnapshotInfo(GetLakeSnapshotResponse response) { long tableId = response.getTableId(); long snapshotId = response.getSnapshotId(); Map tableBucketsOffset = diff --git a/fluss-common/src/main/java/org/apache/fluss/lake/committer/CommitterInitContext.java b/fluss-common/src/main/java/org/apache/fluss/lake/committer/CommitterInitContext.java index dab6b14852..497a7f9a7b 100644 --- a/fluss-common/src/main/java/org/apache/fluss/lake/committer/CommitterInitContext.java +++ b/fluss-common/src/main/java/org/apache/fluss/lake/committer/CommitterInitContext.java @@ -51,4 +51,11 @@ public interface CommitterInitContext { * @return the lake tiering config */ Configuration lakeTieringConfig(); + + /** + * Returns the fluss config. + * + * @return the fluss config + */ + Configuration flussConfig(); } diff --git a/fluss-common/src/main/java/org/apache/fluss/lake/committer/LakeCommitResult.java b/fluss-common/src/main/java/org/apache/fluss/lake/committer/LakeCommitResult.java new file mode 100644 index 0000000000..6195f88d7c --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/lake/committer/LakeCommitResult.java @@ -0,0 +1,162 @@ +/* + * 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.lake.committer; + +import org.apache.fluss.annotation.PublicEvolving; +import org.apache.fluss.metadata.TableBucket; + +import javax.annotation.Nullable; + +import java.util.Map; +import java.util.Objects; + +/** + * The result of a lake commit operation, containing the committed snapshot ID and the readable + * snapshot information. + * + *

For most implementations, the readable snapshot is the same as the committed snapshot, and the + * log end offsets are the same as the tiered offsets from TieringCommitOperator. + * + *

For Paimon DV tables, the readable snapshot will be different from the committed snapshot, and + * the log end offsets will be different as well (based on compaction status). + * + * @since 0.9 + */ +@PublicEvolving +public class LakeCommitResult { + + // -1 to enforce to keep all previous snapshots + public static final Long KEEP_ALL = -1L; + + // The snapshot ID that was just committed + private final long committedSnapshotId; + + private final boolean committedIsReadable; + + // The earliest snapshot ID to keep, null means not to keep any previous snapshot + @Nullable private final Long earliestSnapshotIDToKeep; + + @Nullable private final ReadableSnapshot readableSnapshot; + + private LakeCommitResult( + long committedSnapshotId, + boolean committedIsReadable, + @Nullable ReadableSnapshot readableSnapshot, + @Nullable Long earliestSnapshotIDToKeep) { + this.committedSnapshotId = committedSnapshotId; + this.committedIsReadable = committedIsReadable; + this.readableSnapshot = readableSnapshot; + this.earliestSnapshotIDToKeep = earliestSnapshotIDToKeep; + } + + public static LakeCommitResult committedIsReadable(long committedSnapshotId) { + return new LakeCommitResult(committedSnapshotId, true, null, null); + } + + public static LakeCommitResult unknownReadableSnapshot(long committedSnapshotId) { + return new LakeCommitResult(committedSnapshotId, false, null, KEEP_ALL); + } + + public static LakeCommitResult withReadableSnapshot( + long committedSnapshotId, + // the readable snapshot id + long readableSnapshotId, + // the tiered log end offset for readable snapshot + Map tieredLogEndOffsets, + // the readable log end offset for readable snapshot + Map readableLogEndOffsets, + @Nullable Long earliestSnapshotIDToKeep) { + return new LakeCommitResult( + committedSnapshotId, + false, + new ReadableSnapshot( + readableSnapshotId, tieredLogEndOffsets, readableLogEndOffsets), + earliestSnapshotIDToKeep); + } + + public long getCommittedSnapshotId() { + return committedSnapshotId; + } + + public boolean committedIsReadable() { + return committedIsReadable; + } + + @Nullable + public ReadableSnapshot getReadableSnapshot() { + return readableSnapshot; + } + + /** + * Gets the earliest snapshot ID to keep. + * + * @return the earliest snapshot ID to keep + */ + @Nullable + public Long getEarliestSnapshotIDToKeep() { + return earliestSnapshotIDToKeep; + } + + /** todo. */ + public static class ReadableSnapshot { + private final long readableSnapshotId; + private final Map tieredLogEndOffsets; + private final Map readableLogEndOffsets; + + public ReadableSnapshot( + Long readableSnapshotId, + Map tieredLogEndOffsets, + Map readableLogEndOffsets) { + this.readableSnapshotId = readableSnapshotId; + this.tieredLogEndOffsets = tieredLogEndOffsets; + this.readableLogEndOffsets = readableLogEndOffsets; + } + + public Long getReadableSnapshotId() { + return readableSnapshotId; + } + + public Map getTieredLogEndOffsets() { + return tieredLogEndOffsets; + } + + public Map getReadableLogEndOffsets() { + return readableLogEndOffsets; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ReadableSnapshot that = (ReadableSnapshot) o; + return readableSnapshotId == that.readableSnapshotId + && Objects.equals(tieredLogEndOffsets, that.tieredLogEndOffsets) + && Objects.equals(readableLogEndOffsets, that.readableLogEndOffsets); + } + + @Override + public int hashCode() { + return Objects.hash(readableSnapshotId, tieredLogEndOffsets, readableLogEndOffsets); + } + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/lake/committer/LakeCommitter.java b/fluss-common/src/main/java/org/apache/fluss/lake/committer/LakeCommitter.java index 607dc78667..33e89afc9a 100644 --- a/fluss-common/src/main/java/org/apache/fluss/lake/committer/LakeCommitter.java +++ b/fluss-common/src/main/java/org/apache/fluss/lake/committer/LakeCommitter.java @@ -56,10 +56,13 @@ public interface LakeCommitter extends AutoCloseable * * @param committable the committable object * @param snapshotProperties the properties that lake supported to store in snapshot - * @return the committed snapshot ID + * @return the commit result, which always includes the latest committed snapshot ID and may + * optionally contain distinct readable snapshot information if the physical tiered offsets + * do not yet represent a consistent readable state (e.g., in Paimon DV tables where the + * tiered log records may still in level0 which is not readable). * @throws IOException if an I/O error occurs */ - long commit(CommittableT committable, Map snapshotProperties) + LakeCommitResult commit(CommittableT committable, Map snapshotProperties) throws IOException; /** diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/lake/LakeSplitGenerator.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/lake/LakeSplitGenerator.java index cd0926294c..85c9e5b048 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/lake/LakeSplitGenerator.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/lake/LakeSplitGenerator.java @@ -87,7 +87,7 @@ public LakeSplitGenerator( public List generateHybridLakeFlussSplits() throws Exception { LakeSnapshot lakeSnapshotInfo; try { - lakeSnapshotInfo = flussAdmin.getLatestLakeSnapshot(tableInfo.getTablePath()).get(); + lakeSnapshotInfo = flussAdmin.getReadableLakeSnapshot(tableInfo.getTablePath()).get(); } catch (Exception exception) { if (ExceptionUtils.stripExecutionException(exception) instanceof LakeTableSnapshotNotExistException) { @@ -119,7 +119,7 @@ public List generateHybridLakeFlussSplits() throws Exception { lakeSplits, isLogTable, tableBucketsOffset, partitionNameById); } else { Map> nonPartitionLakeSplits = - lakeSplits.values().iterator().next(); + lakeSplits.isEmpty() ? null : lakeSplits.values().iterator().next(); // non-partitioned table return generateNoPartitionedTableSplit( nonPartitionLakeSplits, isLogTable, tableBucketsOffset); @@ -307,7 +307,7 @@ private SourceSplitBase generateSplitForPrimaryKeyTableBucket( } private List generateNoPartitionedTableSplit( - Map> lakeSplits, + @Nullable Map> lakeSplits, boolean isLogTable, Map tableBucketSnapshotLogOffset) { // iterate all bucket diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/FlussTableLakeSnapshotCommitter.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/FlussTableLakeSnapshotCommitter.java index fe37c91e5b..b18690cb36 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/FlussTableLakeSnapshotCommitter.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/FlussTableLakeSnapshotCommitter.java @@ -21,6 +21,7 @@ import org.apache.fluss.client.metadata.MetadataUpdater; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; +import org.apache.fluss.lake.committer.LakeCommitResult; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.metrics.registry.MetricRegistry; @@ -41,7 +42,10 @@ import org.apache.fluss.rpc.protocol.ApiError; import org.apache.fluss.utils.ExceptionUtils; +import javax.annotation.Nullable; + import java.io.IOException; +import java.util.Collections; import java.util.List; import java.util.Map; @@ -87,7 +91,8 @@ public void open() { metadataUpdater::getCoordinatorServer, rpcClient, CoordinatorGateway.class); } - String prepareLakeSnapshot( + @VisibleForTesting + public String prepareLakeSnapshot( long tableId, TablePath tablePath, Map logEndOffsets) throws IOException { PbPrepareLakeTableRespForTable prepareResp; @@ -121,21 +126,92 @@ String prepareLakeSnapshot( } } - void commit( + public void commit( long tableId, - long lakeSnapshotId, - String lakeBucketOffsetsPath, + TablePath tablePath, + LakeCommitResult lakeCommitResult, + String lakeBucketTieredOffsetsPath, Map logEndOffsets, Map logMaxTieredTimestamps) throws IOException { + Long earliestSnapshotIDToKeep = lakeCommitResult.getEarliestSnapshotIDToKeep(); + if (lakeCommitResult.committedIsReadable()) { + commit( + tableId, + lakeCommitResult.getCommittedSnapshotId(), + lakeBucketTieredOffsetsPath, + lakeBucketTieredOffsetsPath, + logEndOffsets, + logMaxTieredTimestamps, + earliestSnapshotIDToKeep); + } else { + LakeCommitResult.ReadableSnapshot readableSnapshot = + lakeCommitResult.getReadableSnapshot(); + if (readableSnapshot == null) { + // readable snapshot is unknown + commit( + tableId, + lakeCommitResult.getCommittedSnapshotId(), + lakeBucketTieredOffsetsPath, + null, + logEndOffsets, + logMaxTieredTimestamps, + earliestSnapshotIDToKeep); + } else { + // readable snapshot is known, we will first commit a snapshot with readable bucket + // offset + // prepare a readable bucket offset file for the readable snapshot + String readableSnapshotReadableOffsetsPath = + prepareLakeSnapshot( + tableId, tablePath, readableSnapshot.getReadableLogEndOffsets()); + + // prepare a tiered bucket offset file for the readable snapshot + String readableSnapshotTieredOffsetsPath = + prepareLakeSnapshot( + tableId, tablePath, readableSnapshot.getTieredLogEndOffsets()); + // commit the readable snapshot + commit( + tableId, + readableSnapshot.getReadableSnapshotId(), + readableSnapshotTieredOffsetsPath, + readableSnapshotReadableOffsetsPath, + Collections.emptyMap(), + Collections.emptyMap(), + earliestSnapshotIDToKeep); + + // commit the tiered snapshot (with earliestSnapshotIDToKeep) + commit( + tableId, + lakeCommitResult.getCommittedSnapshotId(), + lakeBucketTieredOffsetsPath, + // no readable snapshot offset path + null, + logEndOffsets, + logMaxTieredTimestamps, + earliestSnapshotIDToKeep); + } + } + } + + void commit( + long tableId, + long snapshotId, + String lakeBucketTieredOffsetsPath, + @Nullable String readableLakeBucketTieredOffsetsPath, + Map logEndOffsets, + Map logMaxTieredTimestamps, + @Nullable Long earliestSnapshotIDToKeep) + throws IOException { try { CommitLakeTableSnapshotRequest request = toCommitLakeTableSnapshotRequest( tableId, - lakeSnapshotId, - lakeBucketOffsetsPath, + snapshotId, + lakeBucketTieredOffsetsPath, + readableLakeBucketTieredOffsetsPath, logEndOffsets, - logMaxTieredTimestamps); + logMaxTieredTimestamps, + earliestSnapshotIDToKeep); List commitLakeTableSnapshotRespForTables = coordinatorGateway.commitLakeTableSnapshot(request).get().getTableRespsList(); checkState(commitLakeTableSnapshotRespForTables.size() == 1); @@ -148,7 +224,7 @@ void commit( throw new IOException( String.format( "Fail to commit table lake snapshot id %d of table %d to Fluss.", - lakeSnapshotId, tableId), + snapshotId, tableId), ExceptionUtils.stripExecutionException(exception)); } } @@ -197,17 +273,22 @@ private PrepareLakeTableSnapshotRequest toPrepareLakeTableSnapshotRequest( * * @param tableId the table ID * @param snapshotId the lake snapshot ID - * @param bucketOffsetsPath the file path where the bucket offsets is stored + * @param tieredBucketOffsetsPath the file path where the tiered bucket offsets is stored + * @param readableBucketTieredOffsetsPath the file path where the readable bucket offsets is + * stored * @param logEndOffsets the log end offsets for each bucket * @param logMaxTieredTimestamps the max tiered timestamps for each bucket + * @param earliestSnapshotIDToKeep the earliest snapshot id to keep * @return the commit request */ private CommitLakeTableSnapshotRequest toCommitLakeTableSnapshotRequest( long tableId, long snapshotId, - String bucketOffsetsPath, + String tieredBucketOffsetsPath, + @Nullable String readableBucketTieredOffsetsPath, Map logEndOffsets, - Map logMaxTieredTimestamps) { + Map logMaxTieredTimestamps, + @Nullable Long earliestSnapshotIDToKeep) { CommitLakeTableSnapshotRequest commitLakeTableSnapshotRequest = new CommitLakeTableSnapshotRequest(); @@ -217,8 +298,14 @@ private CommitLakeTableSnapshotRequest toCommitLakeTableSnapshotRequest( pbLakeTableSnapshotMetadata.setSnapshotId(snapshotId); pbLakeTableSnapshotMetadata.setTableId(tableId); // tiered snapshot file path is equal to readable snapshot currently - pbLakeTableSnapshotMetadata.setTieredBucketOffsetsFilePath(bucketOffsetsPath); - pbLakeTableSnapshotMetadata.setReadableBucketOffsetsFilePath(bucketOffsetsPath); + pbLakeTableSnapshotMetadata.setTieredBucketOffsetsFilePath(tieredBucketOffsetsPath); + if (readableBucketTieredOffsetsPath != null) { + pbLakeTableSnapshotMetadata.setReadableBucketOffsetsFilePath( + readableBucketTieredOffsetsPath); + } + if (earliestSnapshotIDToKeep != null) { + pbLakeTableSnapshotMetadata.setEarliestSnapshotIdToKeep(earliestSnapshotIDToKeep); + } // Add PbLakeTableSnapshotInfo for metrics reporting (to notify tablet servers about // synchronized log end offsets and max timestamps) diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperator.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperator.java index 79bcc5f85e..1561bf9149 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperator.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperator.java @@ -30,6 +30,7 @@ import org.apache.fluss.flink.tiering.source.TableBucketWriteResult; import org.apache.fluss.flink.tiering.source.TieringSource; import org.apache.fluss.lake.committer.CommittedLakeSnapshot; +import org.apache.fluss.lake.committer.LakeCommitResult; import org.apache.fluss.lake.committer.LakeCommitter; import org.apache.fluss.lake.writer.LakeTieringFactory; import org.apache.fluss.lake.writer.LakeWriter; @@ -217,7 +218,10 @@ private Committable commitWriteResults( try (LakeCommitter lakeCommitter = lakeTieringFactory.createLakeCommitter( new TieringCommitterInitContext( - tablePath, currentTableInfo, lakeTieringConfig))) { + tablePath, + admin.getTableInfo(tablePath).get(), + lakeTieringConfig, + flussConfig))) { List writeResults = committableWriteResults.stream() .map(TableBucketWriteResult::writeResult) @@ -245,26 +249,95 @@ private Committable commitWriteResults( : flussCurrentLakeSnapshot.getSnapshotId()); // get the lake bucket offsets file storing the log end offsets - String lakeBucketOffsetsFile = + String lakeBucketTieredOffsetsFile = flussTableLakeSnapshotCommitter.prepareLakeSnapshot( tableId, tablePath, logEndOffsets); // record the lake snapshot bucket offsets file to snapshot property - long committedSnapshotId = - lakeCommitter.commit( - committable, - Collections.singletonMap( - FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY, lakeBucketOffsetsFile)); - flussTableLakeSnapshotCommitter.commit( + Map snapshotProperties = + Collections.singletonMap( + FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY, lakeBucketTieredOffsetsFile); + LakeCommitResult lakeCommitResult = + lakeCommitter.commit(committable, snapshotProperties); + // commit to fluss + commitToFluss( tableId, - committedSnapshotId, - lakeBucketOffsetsFile, + tablePath, + lakeCommitResult, + lakeBucketTieredOffsetsFile, logEndOffsets, logMaxTieredTimestamps); return committable; } } + private void commitToFluss( + long tableId, + TablePath tablePath, + LakeCommitResult lakeCommitResult, + String lakeBucketTieredOffsetsFile, + Map logTieredEndOffsets, + Map logMaxTieredTimestamps) + throws Exception { + Long earliestSnapshotIDToKeep = lakeCommitResult.getEarliestSnapshotIDToKeep(); + if (lakeCommitResult.committedIsReadable()) { + flussTableLakeSnapshotCommitter.commit( + tableId, + lakeCommitResult.getCommittedSnapshotId(), + lakeBucketTieredOffsetsFile, + lakeBucketTieredOffsetsFile, + logTieredEndOffsets, + logMaxTieredTimestamps, + earliestSnapshotIDToKeep); + } else { + LakeCommitResult.ReadableSnapshot readableSnapshot = + lakeCommitResult.getReadableSnapshot(); + long committedSnapshotId = lakeCommitResult.getCommittedSnapshotId(); + if (readableSnapshot == null) { + flussTableLakeSnapshotCommitter.commit( + tableId, + committedSnapshotId, + lakeBucketTieredOffsetsFile, + null, + logTieredEndOffsets, + logMaxTieredTimestamps, + earliestSnapshotIDToKeep); + } else { + // commit readable snapshot + String readableSnapshotTieredLogEndOffsetFile = + readableSnapshot.getReadableSnapshotId() == committedSnapshotId + ? lakeBucketTieredOffsetsFile + : flussTableLakeSnapshotCommitter.prepareLakeSnapshot( + tableId, + tablePath, + readableSnapshot.getTieredLogEndOffsets()); + + String readableSnapshotReadableLogEndOffsetFile = + flussTableLakeSnapshotCommitter.prepareLakeSnapshot( + tableId, tablePath, readableSnapshot.getReadableLogEndOffsets()); + flussTableLakeSnapshotCommitter.commit( + tableId, + readableSnapshot.getReadableSnapshotId(), + readableSnapshotTieredLogEndOffsetFile, + readableSnapshotReadableLogEndOffsetFile, + Collections.emptyMap(), + Collections.emptyMap(), + earliestSnapshotIDToKeep); + + // commit tiered snapshot + flussTableLakeSnapshotCommitter.commit( + tableId, + lakeCommitResult.getCommittedSnapshotId(), + lakeBucketTieredOffsetsFile, + // no readable snapshot + null, + logTieredEndOffsets, + logMaxTieredTimestamps, + earliestSnapshotIDToKeep); + } + } + } + @Nullable private LakeSnapshot getLatestLakeSnapshot(TablePath tablePath) throws Exception { LakeSnapshot flussCurrentLakeSnapshot; @@ -332,11 +405,14 @@ private void checkFlussNotMissingLakeSnapshot( tableId, missingCommittedSnapshot.getLakeSnapshotId(), lakeSnapshotOffsetPath, + // don't care readable snapshot and offsets, + null, // use empty log offsets, log max timestamp, since we can't know that // in last tiering, it doesn't matter for they are just used to // report metrics Collections.emptyMap(), - Collections.emptyMap()); + Collections.emptyMap(), + LakeCommitResult.KEEP_ALL); // abort this committable to delete the written files lakeCommitter.abort(committable); throw new IllegalStateException( diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/TieringCommitterInitContext.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/TieringCommitterInitContext.java index 3cae145f06..54dae1817f 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/TieringCommitterInitContext.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/TieringCommitterInitContext.java @@ -29,12 +29,17 @@ public class TieringCommitterInitContext implements CommitterInitContext { private final TablePath tablePath; private final TableInfo tableInfo; private final Configuration lakeTieringConfig; + private final Configuration flussConfig; public TieringCommitterInitContext( - TablePath tablePath, TableInfo tableInfo, Configuration lakeTieringConfig) { + TablePath tablePath, + TableInfo tableInfo, + Configuration lakeTieringConfig, + Configuration flussConfig) { this.tablePath = tablePath; this.tableInfo = tableInfo; this.lakeTieringConfig = lakeTieringConfig; + this.flussConfig = flussConfig; } @Override @@ -51,4 +56,9 @@ public TableInfo tableInfo() { public Configuration lakeTieringConfig() { return lakeTieringConfig; } + + @Override + public Configuration flussConfig() { + return flussConfig; + } } diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/TestingLakeTieringFactory.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/TestingLakeTieringFactory.java index f096b372b2..92f7c562a6 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/TestingLakeTieringFactory.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/TestingLakeTieringFactory.java @@ -21,6 +21,7 @@ import org.apache.fluss.flink.tiering.source.TestingWriteResultSerializer; import org.apache.fluss.lake.committer.CommittedLakeSnapshot; import org.apache.fluss.lake.committer.CommitterInitContext; +import org.apache.fluss.lake.committer.LakeCommitResult; import org.apache.fluss.lake.committer.LakeCommitter; import org.apache.fluss.lake.serializer.SimpleVersionedSerializer; import org.apache.fluss.lake.writer.LakeTieringFactory; @@ -120,9 +121,10 @@ public TestingCommittable toCommittable(List testingWriteRes } @Override - public long commit(TestingCommittable committable, Map snapshotProperties) + public LakeCommitResult commit( + TestingCommittable committable, Map snapshotProperties) throws IOException { - return ++currentSnapshot; + return LakeCommitResult.committedIsReadable(++currentSnapshot); } @Override diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/committer/FlussTableLakeSnapshotCommitterTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/committer/FlussTableLakeSnapshotCommitterTest.java index abb74ebf67..7a5c3f86fd 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/committer/FlussTableLakeSnapshotCommitterTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/committer/FlussTableLakeSnapshotCommitterTest.java @@ -18,7 +18,9 @@ package org.apache.fluss.flink.tiering.committer; import org.apache.fluss.client.metadata.LakeSnapshot; +import org.apache.fluss.exception.LakeTableSnapshotNotExistException; import org.apache.fluss.flink.utils.FlinkTestBase; +import org.apache.fluss.lake.committer.LakeCommitResult; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.rpc.messages.CommitLakeTableSnapshotRequest; @@ -42,6 +44,7 @@ import static org.apache.fluss.record.TestData.DATA1_PARTITIONED_TABLE_DESCRIPTOR; import static org.apache.fluss.record.TestData.DATA1_TABLE_DESCRIPTOR; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Test for {@link FlussTableLakeSnapshotCommitter}. */ class FlussTableLakeSnapshotCommitterTest extends FlinkTestBase { @@ -73,26 +76,95 @@ void testCommit(boolean isPartitioned) throws Exception { Map expectedOffsets = mockLogEndOffsets(tableId, partitions); - long lakeSnapshotId = 3; + long snapshotId1 = 1; - String lakeSnapshotFilePath = + String lakeSnapshotFilePath1 = flussTableLakeSnapshotCommitter.prepareLakeSnapshot( tableId, tablePath, expectedOffsets); // commit offsets flussTableLakeSnapshotCommitter.commit( tableId, - lakeSnapshotId, - lakeSnapshotFilePath, + snapshotId1, + lakeSnapshotFilePath1, + // don't care readable snapshot and offsets, + null, // don't care end offsets, maxTieredTimestamps Collections.emptyMap(), - Collections.emptyMap()); + Collections.emptyMap(), + // test null which only keep one snapshot + null); LakeSnapshot lakeSnapshot = admin.getLatestLakeSnapshot(tablePath).get(); - assertThat(lakeSnapshot.getSnapshotId()).isEqualTo(3); - + assertThat(lakeSnapshot.getSnapshotId()).isEqualTo(snapshotId1); // get and check the offsets Map bucketLogOffsets = lakeSnapshot.getTableBucketsOffset(); assertThat(bucketLogOffsets).isEqualTo(expectedOffsets); + + // verify no any readable lake snapshot + assertThatThrownBy(() -> admin.getReadableLakeSnapshot(tablePath).get()) + .rootCause() + .isInstanceOf(LakeTableSnapshotNotExistException.class) + .hasMessageContaining( + "Lake table readable snapshot doesn't exist for table: fluss.test_commit"); + + // commit with readable snapshot + long snapshotId2 = 2; + String lakeSnapshotFilePath2 = + flussTableLakeSnapshotCommitter.prepareLakeSnapshot( + tableId, tablePath, expectedOffsets); + flussTableLakeSnapshotCommitter.commit( + tableId, + snapshotId2, + lakeSnapshotFilePath2, + // make readable file path is same to lakeSnapshotFilePath + lakeSnapshotFilePath2, + // don't care end offsets, maxTieredTimestamps + Collections.emptyMap(), + Collections.emptyMap(), + LakeCommitResult.KEEP_ALL); + + // test get readable snapshot + LakeSnapshot readLakeSnapshot = admin.getLatestLakeSnapshot(tablePath).get(); + assertThat(readLakeSnapshot.getSnapshotId()).isEqualTo(snapshotId2); + assertThat(readLakeSnapshot.getTableBucketsOffset()).isEqualTo(bucketLogOffsets); + + // verify we can still get snapshot id 1 + LakeSnapshot lakeSnapshot1 = admin.getLakeSnapshot(tablePath, snapshotId1).get(); + assertThat(lakeSnapshot1.getSnapshotId()).isEqualTo(snapshotId1); + assertThat(lakeSnapshot1.getTableBucketsOffset()).isEqualTo(bucketLogOffsets); + + // commit with readable snapshot again, but update earliest + long snapshotId3 = 3L; + String lakeSnapshotFilePath3 = + flussTableLakeSnapshotCommitter.prepareLakeSnapshot( + tableId, tablePath, expectedOffsets); + flussTableLakeSnapshotCommitter.commit( + tableId, + snapshotId3, + lakeSnapshotFilePath3, + // make readable file path null + null, + // don't care end offsets, maxTieredTimestamps + Collections.emptyMap(), + Collections.emptyMap(), + snapshotId2); + + // now, verify we can't get snapshot 1 + assertThatThrownBy(() -> admin.getLakeSnapshot(tablePath, snapshotId1).get()) + .rootCause() + .isInstanceOf(LakeTableSnapshotNotExistException.class) + .hasMessageContaining( + "Lake table snapshot doesn't exist for table: fluss.test_commit") + .hasMessageContaining(", snapshot id: 1"); + + // verify latest snapshot + LakeSnapshot latestLakeSnapshot = admin.getLatestLakeSnapshot(tablePath).get(); + assertThat(latestLakeSnapshot.getSnapshotId()).isEqualTo(snapshotId3); + + // verify readable snapshot + readLakeSnapshot = admin.getReadableLakeSnapshot(tablePath).get(); + assertThat(readLakeSnapshot.getSnapshotId()).isEqualTo(snapshotId2); + assertThat(readLakeSnapshot.getTableBucketsOffset()).isEqualTo(bucketLogOffsets); } @ParameterizedTest diff --git a/fluss-lake/fluss-lake-iceberg/src/main/java/org/apache/fluss/lake/iceberg/tiering/IcebergLakeCommitter.java b/fluss-lake/fluss-lake-iceberg/src/main/java/org/apache/fluss/lake/iceberg/tiering/IcebergLakeCommitter.java index d07c9eb915..e96a8be336 100644 --- a/fluss-lake/fluss-lake-iceberg/src/main/java/org/apache/fluss/lake/iceberg/tiering/IcebergLakeCommitter.java +++ b/fluss-lake/fluss-lake-iceberg/src/main/java/org/apache/fluss/lake/iceberg/tiering/IcebergLakeCommitter.java @@ -18,6 +18,7 @@ package org.apache.fluss.lake.iceberg.tiering; import org.apache.fluss.lake.committer.CommittedLakeSnapshot; +import org.apache.fluss.lake.committer.LakeCommitResult; import org.apache.fluss.lake.committer.LakeCommitter; import org.apache.fluss.lake.iceberg.maintenance.RewriteDataFileResult; import org.apache.fluss.metadata.TablePath; @@ -50,7 +51,6 @@ import static org.apache.fluss.lake.iceberg.utils.IcebergConversions.toIceberg; import static org.apache.fluss.lake.writer.LakeTieringFactory.FLUSS_LAKE_TIERING_COMMIT_USER; -import static org.apache.fluss.utils.Preconditions.checkNotNull; /** Implementation of {@link LakeCommitter} for Iceberg. */ public class IcebergLakeCommitter implements LakeCommitter { @@ -98,7 +98,8 @@ public IcebergCommittable toCommittable(List icebergWriteRes } @Override - public long commit(IcebergCommittable committable, Map snapshotProperties) + public LakeCommitResult commit( + IcebergCommittable committable, Map snapshotProperties) throws IOException { try { // Refresh table to get latest metadata @@ -140,7 +141,7 @@ public long commit(IcebergCommittable committable, Map snapshotP snapshotId = rewriteCommitSnapshotId; } } - return checkNotNull(snapshotId, "Iceberg committed snapshot id must be non-null."); + return LakeCommitResult.committedIsReadable(snapshotId); } catch (Exception e) { throw new IOException("Failed to commit to Iceberg table.", e); } diff --git a/fluss-lake/fluss-lake-iceberg/src/test/java/org/apache/fluss/lake/iceberg/tiering/IcebergTieringTest.java b/fluss-lake/fluss-lake-iceberg/src/test/java/org/apache/fluss/lake/iceberg/tiering/IcebergTieringTest.java index 6443e85294..a29d0c8767 100644 --- a/fluss-lake/fluss-lake-iceberg/src/test/java/org/apache/fluss/lake/iceberg/tiering/IcebergTieringTest.java +++ b/fluss-lake/fluss-lake-iceberg/src/test/java/org/apache/fluss/lake/iceberg/tiering/IcebergTieringTest.java @@ -195,7 +195,9 @@ void testTieringWriteTable(boolean isPrimaryKeyTable, boolean isPartitionedTable committableSerializer.deserialize( committableSerializer.getVersion(), serialized); long snapshot = - lakeCommitter.commit(icebergCommittable, Collections.singletonMap("k1", "v1")); + lakeCommitter + .commit(icebergCommittable, Collections.singletonMap("k1", "v1")) + .getCommittedSnapshotId(); icebergTable.refresh(); Snapshot icebergSnapshot = icebergTable.currentSnapshot(); assertThat(snapshot).isEqualTo(icebergSnapshot.snapshotId()); @@ -264,6 +266,11 @@ public TableInfo tableInfo() { public Configuration lakeTieringConfig() { return new Configuration(); } + + @Override + public Configuration flussConfig() { + return new Configuration(); + } }); } diff --git a/fluss-lake/fluss-lake-lance/src/main/java/org/apache/fluss/lake/lance/tiering/LanceLakeCommitter.java b/fluss-lake/fluss-lake-lance/src/main/java/org/apache/fluss/lake/lance/tiering/LanceLakeCommitter.java index 1aed7810ec..47c163cdb2 100644 --- a/fluss-lake/fluss-lake-lance/src/main/java/org/apache/fluss/lake/lance/tiering/LanceLakeCommitter.java +++ b/fluss-lake/fluss-lake-lance/src/main/java/org/apache/fluss/lake/lance/tiering/LanceLakeCommitter.java @@ -19,6 +19,7 @@ import org.apache.fluss.config.Configuration; import org.apache.fluss.lake.committer.CommittedLakeSnapshot; +import org.apache.fluss.lake.committer.LakeCommitResult; import org.apache.fluss.lake.committer.LakeCommitter; import org.apache.fluss.lake.lance.LanceConfig; import org.apache.fluss.lake.lance.utils.LanceDatasetAdapter; @@ -70,11 +71,14 @@ public LanceCommittable toCommittable(List lanceWriteResults) } @Override - public long commit(LanceCommittable committable, Map snapshotProperties) + public LakeCommitResult commit( + LanceCommittable committable, Map snapshotProperties) throws IOException { Map properties = new HashMap<>(snapshotProperties); properties.put(committerName, FLUSS_LAKE_TIERING_COMMIT_USER); - return LanceDatasetAdapter.commitAppend(config, committable.committable(), properties); + long snapshotId = + LanceDatasetAdapter.commitAppend(config, committable.committable(), properties); + return LakeCommitResult.committedIsReadable(snapshotId); } @Override diff --git a/fluss-lake/fluss-lake-lance/src/test/java/org/apache/fluss/lake/lance/tiering/LanceTieringTest.java b/fluss-lake/fluss-lake-lance/src/test/java/org/apache/fluss/lake/lance/tiering/LanceTieringTest.java index fa4ddfb492..2aefffe278 100644 --- a/fluss-lake/fluss-lake-lance/src/test/java/org/apache/fluss/lake/lance/tiering/LanceTieringTest.java +++ b/fluss-lake/fluss-lake-lance/src/test/java/org/apache/fluss/lake/lance/tiering/LanceTieringTest.java @@ -167,7 +167,10 @@ void testTieringWriteTable(boolean isPartitioned) throws Exception { committableSerializer.getVersion(), serialized); Map snapshotProperties = Collections.singletonMap(FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY, "offsets"); - long snapshot = lakeCommitter.commit(lanceCommittable, snapshotProperties); + long snapshot = + lakeCommitter + .commit(lanceCommittable, snapshotProperties) + .getCommittedSnapshotId(); // lance dataset version starts from 1 assertThat(snapshot).isEqualTo(2); } @@ -251,6 +254,11 @@ public TableInfo tableInfo() { public Configuration lakeTieringConfig() { return new Configuration(); } + + @Override + public Configuration flussConfig() { + return new Configuration(); + } }); } diff --git a/fluss-lake/fluss-lake-paimon/pom.xml b/fluss-lake/fluss-lake-paimon/pom.xml index 7d66e8a9dc..c2cf6f5617 100644 --- a/fluss-lake/fluss-lake-paimon/pom.xml +++ b/fluss-lake/fluss-lake-paimon/pom.xml @@ -43,17 +43,17 @@ ${project.version} - - org.apache.paimon - paimon-bundle - ${paimon.version} - - org.apache.fluss fluss-client ${project.version} - test + provided + + + + org.apache.paimon + paimon-bundle + ${paimon.version} diff --git a/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/tiering/PaimonLakeCommitter.java b/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/tiering/PaimonLakeCommitter.java index d0b81b3be9..c3c6721419 100644 --- a/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/tiering/PaimonLakeCommitter.java +++ b/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/tiering/PaimonLakeCommitter.java @@ -18,9 +18,12 @@ package org.apache.fluss.lake.paimon.tiering; import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.config.Configuration; import org.apache.fluss.lake.committer.CommittedLakeSnapshot; import org.apache.fluss.lake.committer.CommitterInitContext; +import org.apache.fluss.lake.committer.LakeCommitResult; import org.apache.fluss.lake.committer.LakeCommitter; +import org.apache.fluss.lake.paimon.utils.DvTableReadableSnapshotRetriever; import org.apache.fluss.metadata.TablePath; import org.apache.paimon.CoreOptions; @@ -34,6 +37,8 @@ import org.apache.paimon.table.sink.CommitCallback; import org.apache.paimon.table.sink.TableCommitImpl; import org.apache.paimon.utils.SnapshotManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import javax.annotation.Nullable; @@ -50,8 +55,13 @@ /** Implementation of {@link LakeCommitter} for Paimon. */ public class PaimonLakeCommitter implements LakeCommitter { + private static final Logger LOG = LoggerFactory.getLogger(PaimonLakeCommitter.class); + private final Catalog paimonCatalog; private final FileStoreTable fileStoreTable; + private final TablePath tablePath; + private final long tableId; + private final Configuration flussConfig; private TableCommitImpl tableCommit; private static final ThreadLocal currentCommitSnapshotId = new ThreadLocal<>(); @@ -60,6 +70,9 @@ public PaimonLakeCommitter( PaimonCatalogProvider paimonCatalogProvider, CommitterInitContext committerInitContext) throws IOException { this.paimonCatalog = paimonCatalogProvider.get(); + this.tablePath = committerInitContext.tablePath(); + this.tableId = committerInitContext.tableInfo().getTableId(); + this.flussConfig = committerInitContext.flussConfig(); this.fileStoreTable = getTable( committerInitContext.tablePath(), @@ -83,7 +96,8 @@ public PaimonCommittable toCommittable(List paimonWriteResult } @Override - public long commit(PaimonCommittable committable, Map snapshotProperties) + public LakeCommitResult commit( + PaimonCommittable committable, Map snapshotProperties) throws IOException { ManifestCommittable manifestCommittable = committable.manifestCommittable(); snapshotProperties.forEach(manifestCommittable::addProperty); @@ -92,10 +106,44 @@ public long commit(PaimonCommittable committable, Map snapshotPr tableCommit = fileStoreTable.newCommit(FLUSS_LAKE_TIERING_COMMIT_USER); tableCommit.commit(manifestCommittable); - Long commitSnapshotId = currentCommitSnapshotId.get(); + long committedSnapshotId = + checkNotNull( + currentCommitSnapshotId.get(), + "Paimon committed snapshot id must be non-null."); currentCommitSnapshotId.remove(); - return checkNotNull(commitSnapshotId, "Paimon committed snapshot id must be non-null."); + // deletion vector is disabled, committed snapshot is readable + if (!fileStoreTable.coreOptions().deletionVectorsEnabled()) { + return LakeCommitResult.committedIsReadable(committedSnapshotId); + } else { + // retrive the readable snapshot during commit + try (DvTableReadableSnapshotRetriever retriever = + new DvTableReadableSnapshotRetriever( + tablePath, tableId, fileStoreTable, flussConfig)) { + DvTableReadableSnapshotRetriever.ReadableSnapshotResult readableSnapshotResult = + retriever.getReadableSnapshotAndOffsets(committedSnapshotId); + if (readableSnapshotResult == null) { + return LakeCommitResult.unknownReadableSnapshot(committedSnapshotId); + } else { + long earliestSnapshotIdToKeep = + readableSnapshotResult.getEarliestSnapshotIdToKeep(); + if (earliestSnapshotIdToKeep >= 0) { + LOG.info( + "earliest snapshot ID to keep for table {} is {}. " + + "Snapshots before this ID can be safely deleted from Fluss.", + tablePath, + earliestSnapshotIdToKeep); + } + return LakeCommitResult.withReadableSnapshot( + committedSnapshotId, + readableSnapshotResult.getReadableSnapshotId(), + readableSnapshotResult.getTieredOffsets(), + readableSnapshotResult.getReadableOffsets(), + earliestSnapshotIdToKeep); + } + } + } + } catch (Throwable t) { if (tableCommit != null) { // if any error happen while commit, abort the commit to clean committable diff --git a/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/utils/DvTableReadableSnapshotRetriever.java b/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/utils/DvTableReadableSnapshotRetriever.java new file mode 100644 index 0000000000..2393474cfb --- /dev/null +++ b/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/utils/DvTableReadableSnapshotRetriever.java @@ -0,0 +1,605 @@ +/* + * 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.lake.paimon.utils; + +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.config.Configuration; +import org.apache.fluss.lake.committer.LakeCommitResult; +import org.apache.fluss.metadata.PartitionInfo; +import org.apache.fluss.metadata.ResolvedPartitionSpec; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.utils.types.Tuple2; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.Snapshot; +import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.manifest.FileKind; +import org.apache.paimon.manifest.ManifestEntry; +import org.apache.paimon.operation.FileStoreScan; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.source.ScanMode; +import org.apache.paimon.utils.SnapshotManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.apache.fluss.lake.paimon.utils.PaimonDvTableUtils.findLatestSnapshotExactlyHoldingL0Files; +import static org.apache.fluss.utils.Preconditions.checkNotNull; +import static org.apache.fluss.utils.Preconditions.checkState; + +/** + * A retriever to retrieve the readable snapshot and offsets for Paimon deletion vector enabled + * table. + */ +public class DvTableReadableSnapshotRetriever implements AutoCloseable { + + private static final Logger LOG = + LoggerFactory.getLogger(DvTableReadableSnapshotRetriever.class); + + private final TablePath tablePath; + private final long tableId; + private final FileStoreTable fileStoreTable; + private final Admin flussAdmin; + private final Connection flussConnection; + private final SnapshotManager snapshotManager; + + public DvTableReadableSnapshotRetriever( + TablePath tablePath, + long tableId, + FileStoreTable paimonFileStoreTable, + Configuration flussConfig) { + this.tablePath = tablePath; + this.tableId = tableId; + this.fileStoreTable = paimonFileStoreTable; + this.flussConnection = ConnectionFactory.createConnection(flussConfig); + this.flussAdmin = flussConnection.getAdmin(); + this.snapshotManager = fileStoreTable.snapshotManager(); + } + + /** + * Get readable offsets for DV tables based on the latest compacted snapshot. + * + *

For Paimon DV tables, when an appended snapshot is committed, we need to check the latest + * compacted snapshot to determine readable offsets for each bucket. This method implements + * incremental advancement of readable_snapshot per bucket: + * + *

    + *
  • For buckets without L0 files: use offsets from the latest tiered snapshot. These + * buckets can advance their readable offsets since all their data is in base files (L1+). + *
  • For buckets with L0 files: traverse backwards through compacted snapshots to find the + * latest one that flushed this bucket's L0 files. Then find the latest snapshot that + * exactly holds those flushed L0 files, and use the previous APPEND snapshot's offset for + * that bucket. + *
+ * + *

Algorithm: + * + *

    + *
  1. Find the latest compacted snapshot before the given tiered snapshot + *
  2. Check which buckets have no L0 files and which have L0 files in the compacted snapshot + *
  3. For buckets without L0 files: use offsets from the latest tiered snapshot (all data is + * in base files, safe to advance) + *
  4. For buckets with L0 files: + *
      + *
    1. Traverse backwards through compacted snapshots starting from the latest one + *
    2. For each compacted snapshot, check which buckets had their L0 files flushed + *
    3. For each flushed bucket, find the latest snapshot that exactly holds those L0 + * files using {@link PaimonDvTableUtils#findLatestSnapshotExactlyHoldingL0Files} + *
    4. Find the previous APPEND snapshot before that snapshot + *
    5. Use that APPEND snapshot's offset for the bucket + *
    + *
  5. Return readable offsets for all buckets, allowing incremental advancement + *
+ * + *

Note: This allows readable_snapshot to advance incrementally per bucket. Each bucket's + * readable offset is set to the maximum offset that is actually readable in the compacted + * snapshot, ensuring no data duplication or loss. The readable_snapshot is set to the latest + * compacted snapshot ID, and each bucket continues reading from its respective readable offset. + * + *

Example: If bucket0's L0 files were flushed in snapshot5 (which compacted snapshot1's L0 + * files), and snapshot4 is the latest snapshot that exactly holds those L0 files, then + * bucket0's readable offset will be set to snapshot4's previous APPEND snapshot's offset. + * + * @param tieredSnapshotId the tiered snapshot ID (the appended snapshot that was just + * committed) + * @return a tuple containing the readable snapshot ID (the latest compacted snapshot) and a map + * of TableBucket to readable offset for all buckets, or null if: + *

    + *
  • No compacted snapshot exists before the tiered snapshot + *
  • Cannot find the latest snapshot holding flushed L0 files for some buckets + *
  • Cannot find the previous APPEND snapshot for some buckets + *
  • Cannot find offsets in Fluss for some buckets + *
+ * The map contains offsets for ALL buckets, allowing incremental advancement. + * @throws IOException if an error occurs reading snapshots or offsets from Fluss + */ + @Nullable + public ReadableSnapshotResult getReadableSnapshotAndOffsets(long tieredSnapshotId) + throws IOException { + // Find the latest compacted snapshot + Snapshot latestCompactedSnapshot = + findPreviousSnapshot(tieredSnapshotId, Snapshot.CommitKind.COMPACT); + if (latestCompactedSnapshot == null) { + // No compacted snapshot found, may happen when no compaction happens or snapshot + // expiration, we can't update readable offsets, return null directly + LOG.info( + "Can't find latest compacted snapshot before snapshot {}, skip get readable snapshot.", + tieredSnapshotId); + return null; + } + + Map readableOffsets = new HashMap<>(); + + FlussTableBucketMapper flussTableBucketMapper = new FlussTableBucketMapper(); + + // get all the bucket without l0 files and with l0 files + Tuple2, Set> bucketsWithoutL0AndWithL0 = + getBucketsWithAndWithoutL0AndWithL0(latestCompactedSnapshot); + Set bucketsWithoutL0 = bucketsWithoutL0AndWithL0.f0; + Set bucketsWithL0 = bucketsWithoutL0AndWithL0.f1; + + // Track the earliest previousAppendSnapshot ID that was accessed + // This represents the oldest snapshot that might still be needed + long earliestSnapshotIdToKeep = LakeCommitResult.KEEP_ALL; + + if (!bucketsWithoutL0.isEmpty()) { + // Get latest tiered offsets + LakeSnapshot latestTieredSnapshot; + try { + latestTieredSnapshot = flussAdmin.getLatestLakeSnapshot(tablePath).get(); + } catch (Exception e) { + throw new IOException( + "Failed to read lake snapshot from Fluss server for snapshot " + + latestCompactedSnapshot.id(), + e); + } + // for all buckets without l0, we can use the latest tiered offsets + for (PartitionBucket bucket : bucketsWithoutL0) { + TableBucket tableBucket = flussTableBucketMapper.toTableBucket(bucket); + if (tableBucket == null) { + // can't map such paimon bucket to fluss, just ignore + continue; + } + readableOffsets.put( + tableBucket, latestTieredSnapshot.getTableBucketsOffset().get(tableBucket)); + } + } + + // for all buckets with l0, we need to find the latest compacted snapshot which flushed + // the buckets, the per-bucket offset should be updated to the corresponding compacted + // snapshot offsets + Set allBucketsToAdvance = new HashSet<>(bucketsWithL0); + + long earliestSnapshotId = checkNotNull(snapshotManager.earliestSnapshotId()); + // From latestCompacted forward traverse compacted snapshots + for (long currentSnapshotId = latestCompactedSnapshot.id(); + currentSnapshotId >= earliestSnapshotId; + currentSnapshotId--) { + // no any buckets to advance, break directly + if (allBucketsToAdvance.isEmpty()) { + break; + } + Snapshot currentSnapshot = snapshotManager.tryGetSnapshot(currentSnapshotId); + if (currentSnapshot == null + || currentSnapshot.commitKind() != Snapshot.CommitKind.COMPACT) { + continue; + } + // Get buckets flushed by current compacted snapshot + Set flushedBuckets = getBucketsWithFlushedL0(currentSnapshot); + // For each flushed bucket, if offset not set yet, set it + for (PartitionBucket partitionBucket : flushedBuckets) { + TableBucket tb = flussTableBucketMapper.toTableBucket(partitionBucket); + if (tb == null) { + // can't map such paimon bucket to fluss,just ignore + // don't need to advance offset for the bucket + allBucketsToAdvance.remove(partitionBucket); + continue; + } + if (!readableOffsets.containsKey(tb)) { + Snapshot sourceSnapshot = + findLatestSnapshotExactlyHoldingL0Files( + fileStoreTable, currentSnapshot); + // it happens if there is a compacted snapshot flush l0 files for a bucket, + // but the snapshot from which the compacted snapshot compact is expired + // it should happen rarely, we can't determine the readable offsets for this + // bucket, currently, we just return null to stop readable offset advance + // if it happen, compaction should work unexpected, warn it and reminds to + // increase snapshot retention + if (sourceSnapshot == null) { + LOG.warn( + "Cannot find snapshot holding L0 files flushed by compacted snapshot {} for bucket {}, " + + "the snapshot may have been expired. Consider increasing snapshot retention.", + currentSnapshot.id(), + tb); + return null; + } + + // we already find that for this bucket, which snapshot do the latest flush, + // the offset for the previous one append snapshot should be the readable + // offset + Snapshot previousAppendSnapshot = + sourceSnapshot.commitKind() == Snapshot.CommitKind.APPEND + ? sourceSnapshot + : findPreviousSnapshot( + sourceSnapshot.id(), Snapshot.CommitKind.APPEND); + + // Can't find previous APPEND snapshot, likely due to snapshot expiration. + // This happens when the snapshot holding flushed L0 files is a COMPACT + // snapshot, + // and all APPEND snapshots before it have been expired. + // + // TODO: Optimization - Store compacted snapshot offsets in Fluss + // Currently, we rely on Paimon to find the previous APPEND snapshot to get its + // offset. If Fluss stores offsets for all snapshots (including COMPACT + // snapshots), + // we could: + // 1. Use the sourceSnapshot's offset directly if it's stored in Fluss + // 2. Find any previous snapshot (COMPACT or APPEND) and use its offset + // 3. This would make the system more resilient to snapshot expiration + if (previousAppendSnapshot == null) { + LOG.warn( + "Cannot find previous APPEND snapshot before snapshot {} for bucket {}. " + + "This may be due to snapshot expiration. Consider increasing paimon snapshot retention.", + sourceSnapshot.id(), + tb); + return null; + } + + // Track the minimum previousAppendSnapshot ID + // This snapshot will be accessed via getLakeSnapshot, so we need to keep it + if (earliestSnapshotIdToKeep <= 0 + || previousAppendSnapshot.id() < earliestSnapshotIdToKeep) { + earliestSnapshotIdToKeep = previousAppendSnapshot.id(); + } + + try { + LakeSnapshot lakeSnapshot = + flussAdmin + .getLakeSnapshot(tablePath, previousAppendSnapshot.id()) + .get(); + Long offset = lakeSnapshot.getTableBucketsOffset().get(tb); + if (offset != null) { + readableOffsets.put(tb, offset); + allBucketsToAdvance.remove(partitionBucket); + } else { + LOG.error( + "Could not find offset for bucket {} in snapshot {}, skip advancing readable snapshot.", + tb, + previousAppendSnapshot.id()); + return null; + } + } catch (Exception e) { + LOG.error( + "Failed to read lake snapshot {} from Fluss server, skip update readable snapshot and offset.", + previousAppendSnapshot.id(), + e); + return null; + } + } + } + } + + // This happens when there are writes to a bucket, but no compaction has happened for that + // bucket from the earliest snapshot to the latest compacted snapshot. + // This should happen rarely in practice, as compaction typically processes all buckets over + // time. + // + // TODO: Optimization - Handle buckets without flushed L0 files + // We can optimize this case in two ways: + // 1. If a previous readable snapshot exists between earliest and latest snapshot: + // - Reuse the readable snapshot's offset for this bucket (safe since no L0 was flushed) + // 2. If the earliest snapshot is the first snapshot committed by Fluss: + // - Set the readable offset to 0 for this bucket (no data was readable before) + // These optimizations would allow readable_snapshot to advance even when some buckets + // haven't been compacted yet, improving overall system progress. + if (!allBucketsToAdvance.isEmpty()) { + LOG.warn( + "Could not find flushed snapshots for buckets with L0: {}. " + + "These buckets have L0 files but no found compaction snapshot has flushed them yet." + + " Consider increasing paimon snapshot retention.", + allBucketsToAdvance); + return null; + } + + // Return the latest compacted snapshot ID as the unified readable snapshot + // All buckets can read from this snapshot's base files, then continue from their + // respective readable offsets + // Also return the minimum previousAppendSnapshot ID that was accessed + // Snapshots before this ID can potentially be safely deleted from Fluss + + Snapshot previousSnapshot = + findPreviousSnapshot(latestCompactedSnapshot.id(), Snapshot.CommitKind.APPEND); + if (previousSnapshot == null) { + LOG.warn("todo...."); + return null; + } + + try { + LakeSnapshot lakeSnapshot = + flussAdmin.getLakeSnapshot(tablePath, previousSnapshot.id()).get(); + return new ReadableSnapshotResult( + latestCompactedSnapshot.id(), + lakeSnapshot.getTableBucketsOffset(), + readableOffsets, + earliestSnapshotIdToKeep); + } catch (Exception e) { + LOG.warn("todo"); + return null; + } + } + + /** + * Get buckets (with partition info) that have no L0 files in the given snapshot. + * + *

For Paimon DV tables, we check the snapshot's data files to determine which buckets have + * no L0 delta files. A bucket has no L0 files if all its data is in base files. + * + *

For partitioned tables, we include partition information in the returned TableBucket + * objects. + * + * @param snapshot the snapshot to check + * @return set of TableBucket that have no L0 files + */ + private Tuple2, Set> getBucketsWithAndWithoutL0AndWithL0( + Snapshot snapshot) { + Set bucketsWithoutL0 = new HashSet<>(); + Set bucketsWithL0 = new HashSet<>(); + + // Scan the snapshot to get all splits including level0 + Map scanOptions = new HashMap<>(); + scanOptions.put(CoreOptions.SCAN_SNAPSHOT_ID.key(), String.valueOf(snapshot.id())); + // hacky: set batch scan mode to compact to make sure we can get l0 level files + scanOptions.put( + CoreOptions.BATCH_SCAN_MODE.key(), CoreOptions.BatchScanMode.COMPACT.getValue()); + + Map>> manifestsByBucket = + FileStoreScan.Plan.groupByPartFiles( + fileStoreTable.copy(scanOptions).store().newScan().plan().files()); + + for (Map.Entry>> manifestsByBucketEntry : + manifestsByBucket.entrySet()) { + BinaryRow partition = manifestsByBucketEntry.getKey(); + Map> buckets = manifestsByBucketEntry.getValue(); + for (Map.Entry> bucketEntry : buckets.entrySet()) { + // no l0 file + if (bucketEntry.getValue().stream() + .allMatch( + manifestEntry -> + manifestEntry.kind() != FileKind.DELETE + && manifestEntry.file().level() > 0)) { + bucketsWithoutL0.add(new PartitionBucket(partition, bucketEntry.getKey())); + } else { + bucketsWithL0.add(new PartitionBucket(partition, bucketEntry.getKey())); + } + } + } + return Tuple2.of(bucketsWithoutL0, bucketsWithL0); + } + + /** + * Get buckets (with partition info) whose L0 files were flushed (deleted) in a compacted + * snapshot's delta. + * + * @param compactedSnapshot the compacted snapshot to check + * @return set of PartitionBucket whose L0 files were flushed + */ + private Set getBucketsWithFlushedL0(Snapshot compactedSnapshot) { + checkState(compactedSnapshot.commitKind() == Snapshot.CommitKind.COMPACT); + Set flushedBuckets = new HashSet<>(); + + // Scan the compacted snapshot's delta to find deleted L0 files + List manifestEntries = + fileStoreTable + .store() + .newScan() + .withSnapshot(compactedSnapshot.id()) + .withKind(ScanMode.DELTA) + .plan() + .files(FileKind.DELETE); + + for (ManifestEntry manifestEntry : manifestEntries) { + if (manifestEntry.level() == 0) { + flushedBuckets.add( + new PartitionBucket(manifestEntry.partition(), manifestEntry.bucket())); + } + } + + return flushedBuckets; + } + + @Nullable + private Snapshot findPreviousSnapshot(long beforeSnapshotId, Snapshot.CommitKind commitKind) + throws IOException { + SnapshotManager snapshotManager = fileStoreTable.snapshotManager(); + long earliestSnapshotId = checkNotNull(snapshotManager.earliestSnapshotId()); + for (long currentSnapshotId = beforeSnapshotId - 1; + currentSnapshotId >= earliestSnapshotId; + currentSnapshotId--) { + Snapshot snapshot = snapshotManager.tryGetSnapshot(currentSnapshotId); + if (snapshot != null && snapshot.commitKind() == commitKind) { + return snapshot; + } + } + return null; + } + + /** + * Get partition name to partition id mapping for the table. + * + * @return map from partition name to partition id + */ + private Map getPartitionNameToIdMapping() throws IOException { + try { + List partitionInfos = flussAdmin.listPartitionInfos(tablePath).get(); + return partitionInfos.stream() + .collect( + Collectors.toMap( + PartitionInfo::getPartitionName, + PartitionInfo::getPartitionId)); + } catch (Exception e) { + throw new IOException("Fail to list partitions", e); + } + } + + /** + * Convert Paimon BinaryRow partition to Fluss partition name, whose format is: + * value1$value2$...$valueN. + * + * @param partition the BinaryRow partition from Paimon + * @return partition name string + */ + private String getPartitionNameFromBinaryRow(BinaryRow partition) { + List partitionValues = new ArrayList<>(); + for (int i = 0; i < partition.getFieldCount(); i++) { + // todo: consider other partition type + BinaryString binaryString = partition.getString(i); + partitionValues.add(binaryString.toString()); + } + return String.join(ResolvedPartitionSpec.PARTITION_SPEC_SEPARATOR, partitionValues); + } + + @Override + public void close() throws Exception { + if (flussAdmin != null) { + flussAdmin.close(); + } + if (flussConnection != null) { + flussConnection.close(); + } + } + + /** + * Result of {@link #getReadableSnapshotAndOffsets}, containing readable snapshot information + * and the minimum snapshot ID that can be safely deleted. + */ + public static class ReadableSnapshotResult { + private final long readableSnapshotId; + private final Map tieredOffsets; + private final Map readableOffsets; + private final long earliestSnapshotIdToKeep; + + public ReadableSnapshotResult( + long readableSnapshotId, + Map tieredOffsets, + Map readableOffsets, + long earliestSnapshotIdToKeep) { + this.readableSnapshotId = readableSnapshotId; + this.tieredOffsets = tieredOffsets; + this.readableOffsets = readableOffsets; + this.earliestSnapshotIdToKeep = earliestSnapshotIdToKeep; + } + + public Map getTieredOffsets() { + return tieredOffsets; + } + + public long getReadableSnapshotId() { + return readableSnapshotId; + } + + public Map getReadableOffsets() { + return readableOffsets; + } + + /** + * Returns the earliest snapshot ID that should keep in Fluss. + * + *

This is the earliest ID among all snapshot that were accessed via {@code + * getLakeSnapshot} during the retrieve readable offset. Snapshots before this ID can + * potentially be safely deleted. + */ + public long getEarliestSnapshotIdToKeep() { + return earliestSnapshotIdToKeep; + } + } + + private static final class PartitionBucket { + private final BinaryRow partition; + private final Integer bucket; + + public PartitionBucket(BinaryRow partition, Integer bucket) { + this.partition = partition; + this.bucket = bucket; + } + + @Override + public boolean equals(Object o) { + if (o == null || getClass() != o.getClass()) { + return false; + } + PartitionBucket that = (PartitionBucket) o; + return Objects.equals(partition, that.partition) && Objects.equals(bucket, that.bucket); + } + + @Override + public int hashCode() { + return Objects.hash(partition, bucket); + } + } + + private final class FlussTableBucketMapper { + private final Map partitionNameToIdMapping; + + private FlussTableBucketMapper() throws IOException { + if (!fileStoreTable.partitionKeys().isEmpty()) { + partitionNameToIdMapping = getPartitionNameToIdMapping(); + } else { + partitionNameToIdMapping = null; + } + } + + @Nullable + private TableBucket toTableBucket(PartitionBucket partitionBucket) { + if (partitionBucket.partition.getFieldCount() == 0) { + // Non-partitioned table: BinaryRow.EMPTY_ROW has 0 fields + return new TableBucket(tableId, partitionBucket.bucket); + } else { + // Partitioned table: convert partition name to partition id + String partitionName = getPartitionNameFromBinaryRow(partitionBucket.partition); + Long partitionId = partitionNameToIdMapping.get(partitionName); + if (partitionId == null) { + LOG.warn( + "Partition name '{}' not found in Fluss for table {}. " + + "Available partitions: {}", + partitionName, + tablePath, + partitionNameToIdMapping.keySet()); + return null; + } + return new TableBucket(tableId, partitionId, partitionBucket.bucket); + } + } + } +} diff --git a/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/utils/PaimonConversions.java b/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/utils/PaimonConversions.java index ded40ac59b..4f2003a58f 100644 --- a/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/utils/PaimonConversions.java +++ b/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/utils/PaimonConversions.java @@ -27,6 +27,7 @@ import org.apache.fluss.record.ChangeType; import org.apache.fluss.row.GenericRow; import org.apache.fluss.row.InternalRow; +import org.apache.fluss.types.DataTypeRoot; import org.apache.paimon.CoreOptions; import org.apache.paimon.catalog.Identifier; @@ -41,6 +42,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import static org.apache.fluss.lake.paimon.PaimonLakeCatalog.SYSTEM_COLUMNS; @@ -219,6 +221,7 @@ public static Schema toPaimonSchema(TableDescriptor tableDescriptor) { CoreOptions.CHANGELOG_PRODUCER.key(), CoreOptions.ChangelogProducer.INPUT.toString()); } + // set partition keys schemaBuilder.partitionKeys(tableDescriptor.getPartitionKeys()); @@ -229,6 +232,25 @@ public static Schema toPaimonSchema(TableDescriptor tableDescriptor) { .forEach((k, v) -> setFlussPropertyToPaimon(k, v, options)); schemaBuilder.options(options.toMap()); + // currently we only support string type, todo + // consider to support other types + if (options.get(CoreOptions.DELETION_VECTORS_ENABLED)) { + org.apache.fluss.types.RowType rowType = tableDescriptor.getSchema().getRowType(); + Optional invalidKey = + tableDescriptor.getPartitionKeys().stream() + .filter( + key -> + rowType.getField(key).getType().getTypeRoot() + != DataTypeRoot.STRING) + .findFirst(); + if (invalidKey.isPresent()) { + throw new UnsupportedOperationException( + String.format( + "Only support String type as partitioned key when 'deletion-vectors.enabled' is set to true for paimon, found '%s' is not String type.", + invalidKey.get())); + } + } + // set comment tableDescriptor.getComment().ifPresent(schemaBuilder::comment); diff --git a/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/utils/PaimonDvTableUtils.java b/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/utils/PaimonDvTableUtils.java new file mode 100644 index 0000000000..424e7290f5 --- /dev/null +++ b/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/utils/PaimonDvTableUtils.java @@ -0,0 +1,193 @@ +/* + * 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.lake.paimon.utils; + +import org.apache.fluss.utils.types.Tuple2; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.Snapshot; +import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.io.DataFileMeta; +import org.apache.paimon.manifest.FileKind; +import org.apache.paimon.manifest.ManifestEntry; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.source.DataSplit; +import org.apache.paimon.table.source.ScanMode; +import org.apache.paimon.table.source.Split; +import org.apache.paimon.utils.SnapshotManager; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.apache.fluss.utils.Preconditions.checkNotNull; +import static org.apache.fluss.utils.Preconditions.checkState; + +/** Utils for Paimon delete-vector enabled table. */ +public class PaimonDvTableUtils { + + /** + * Find the latest snapshot that still holds the L0 files flushed by the given compacted + * snapshot. + * + *

The method works by: + * + *

    + *
  1. Getting the delta of the compacted snapshot to find deleted L0 files + *
  2. Grouping deleted L0 files by bucket + *
  3. Searching backwards through previous snapshots to find the latest one whose L0 files + * exactly match the deleted L0 files for each bucket + *
+ * + *

This snapshot is the most recent snapshot that still contains all the L0 files that were + * flushed (deleted) by the compacted snapshot. + * + * @param fileStoreTable the FileStoreTable instance + * @param compactedSnapshot the compacted snapshot whose flushed L0 files to search for + * @return the latest snapshot that still holds these L0 files, or null if not found + * @throws IOException if an error occurs + */ + @Nullable + public static Snapshot findLatestSnapshotExactlyHoldingL0Files( + FileStoreTable fileStoreTable, Snapshot compactedSnapshot) throws IOException { + SnapshotManager snapshotManager = fileStoreTable.snapshotManager(); + checkState(compactedSnapshot.commitKind() == Snapshot.CommitKind.COMPACT); + // Get deleted L0 files from the compacted snapshot's delta + Map, Set> deletedL0FilesByBucket = + getDeletedL0FilesByBucket(fileStoreTable, compactedSnapshot); + + if (deletedL0FilesByBucket.isEmpty()) { + // No L0 files were deleted, can't find a snapshot holding these L0 files, + // return null directly + return null; + } + + // Search backwards from the compacted snapshot to find the latest snapshot + // that still holds these L0 files + long earliestSnapshot = checkNotNull(snapshotManager.earliestSnapshotId()); + for (long snapshot = compactedSnapshot.id() - 1; snapshot >= earliestSnapshot; snapshot--) { + Snapshot candidateSnapshot = snapshotManager.tryGetSnapshot(snapshot); + if (candidateSnapshot == null) { + // no such snapshot in paimon, skip + continue; + } + if (matchesDeletedL0Files(fileStoreTable, candidateSnapshot, deletedL0FilesByBucket)) { + return candidateSnapshot; + } + } + return null; + } + + /** + * Get deleted L0 files grouped by bucket from a compacted snapshot's delta. + * + * @param compactedSnapshot the compacted snapshot + * @return a map from bucket ID to set of deleted L0 file names + */ + private static Map, Set> getDeletedL0FilesByBucket( + FileStoreTable fileStoreTable, Snapshot compactedSnapshot) { + Map, Set> deletedL0FilesByBucket = new HashMap<>(); + List manifestEntries = + fileStoreTable + .store() + .newScan() + .withSnapshot(compactedSnapshot.id()) + .withKind(ScanMode.DELTA) + .plan() + .files(FileKind.DELETE); + for (ManifestEntry manifestEntry : manifestEntries) { + if (manifestEntry.level() == 0) { + deletedL0FilesByBucket + .computeIfAbsent( + Tuple2.of(manifestEntry.partition(), manifestEntry.bucket()), + k -> new HashSet<>()) + .add(manifestEntry.fileName()); + } + } + return deletedL0FilesByBucket; + } + + /** + * Check if a candidate snapshot's L0 files exactly match the deleted L0 files for the relevant + * buckets. + * + * @param candidateSnapshot the candidate snapshot to check + * @param deletedL0FilesByBucket the deleted L0 files grouped by bucket + * @return true if the candidate snapshot's L0 files match the deleted L0 files per-bucket + */ + private static boolean matchesDeletedL0Files( + FileStoreTable fileStoreTable, + Snapshot candidateSnapshot, + Map, Set> deletedL0FilesByBucket) { + // Get L0 files from the candidate snapshot, grouped by bucket + Map, Set> candidateL0FilesByBucket = + getL0FilesByBucket(fileStoreTable, candidateSnapshot); + + for (Map.Entry, Set> deleteL0Entry : + deletedL0FilesByBucket.entrySet()) { + Set deleteL0Files = candidateL0FilesByBucket.get(deleteL0Entry.getKey()); + if (deleteL0Files == null || !deleteL0Files.equals(deleteL0Entry.getValue())) { + return false; + } + } + return true; + } + + /** + * Get L0 files from a snapshot, grouped by bucket. + * + *

This method uses the scan API to get all L0 files that exist in the snapshot. + * + * @param snapshot the snapshot to get L0 files from + * @return a map from bucket ID to set of L0 file names that exist in the snapshot + */ + private static Map, Set> getL0FilesByBucket( + FileStoreTable fileStoreTable, Snapshot snapshot) { + Map, Set> l0FilesByBucket = new HashMap<>(); + + // Use scan API to get all splits including L0 files + Map scanOptions = new HashMap<>(); + scanOptions.put(CoreOptions.SCAN_SNAPSHOT_ID.key(), String.valueOf(snapshot.id())); + // Set batch scan mode to compact to make sure we can get L0 level files + scanOptions.put( + CoreOptions.BATCH_SCAN_MODE.key(), CoreOptions.BatchScanMode.COMPACT.getValue()); + List splits = fileStoreTable.copy(scanOptions).newScan().plan().splits(); + + // Check each split to get L0 files + for (Split split : splits) { + DataSplit dataSplit = (DataSplit) split; + for (DataFileMeta dataFileMeta : dataSplit.dataFiles()) { + if (dataFileMeta.level() == 0) { + l0FilesByBucket + .computeIfAbsent( + Tuple2.of(dataSplit.partition(), dataSplit.bucket()), + k -> new HashSet<>()) + .add(dataFileMeta.fileName()); + } + } + } + + return l0FilesByBucket; + } +} diff --git a/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/LakeEnabledTableCreateITCase.java b/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/LakeEnabledTableCreateITCase.java index 6fb7003223..4b9f7b2db7 100644 --- a/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/LakeEnabledTableCreateITCase.java +++ b/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/LakeEnabledTableCreateITCase.java @@ -971,6 +971,29 @@ void testEnableLakeTableWithLegacySystemTimestampColumn() throws Exception { .isTrue(); } + @Test + void testCreatePaimonDvTableWithNonStringPartitionColumn() throws Exception { + TablePath tablePath = TablePath.of(DATABASE, "invalid_dv_table"); + TableDescriptor tableDescriptor = + TableDescriptor.builder() + .schema( + Schema.newBuilder() + .column("c1", DataTypes.INT()) + .column("c2", DataTypes.STRING()) + .column("c3", DataTypes.INT()) + .primaryKey("c1", "c3") + .build()) + .property(ConfigOptions.TABLE_DATALAKE_ENABLED, true) + .property("paimon.deletion-vectors.enabled", "true") + .partitionedBy("c3") + .build(); + + assertThatThrownBy(() -> admin.createTable(tablePath, tableDescriptor, false).get()) + .rootCause() + .hasMessageContaining( + "Only support String type as partitioned key when 'deletion-vectors.enabled' is set to true for paimon, found 'c3' is not String type."); + } + private void verifyPaimonTable( Table paimonTable, TableDescriptor flussTable, diff --git a/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/flink/FlinkUnionReadDvTableITCase.java b/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/flink/FlinkUnionReadDvTableITCase.java new file mode 100644 index 0000000000..40de438874 --- /dev/null +++ b/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/flink/FlinkUnionReadDvTableITCase.java @@ -0,0 +1,242 @@ +/* + * 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.lake.paimon.flink; + +import org.apache.fluss.client.metadata.LakeSnapshot; +import org.apache.fluss.config.AutoPartitionTimeUnit; +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.exception.LakeTableSnapshotNotExistException; +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.InternalRow; +import org.apache.fluss.server.replica.Replica; +import org.apache.fluss.types.DataTypes; + +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.table.FileStoreTable; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import javax.annotation.Nullable; + +import java.io.File; +import java.time.Duration; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.fluss.flink.source.testutils.FlinkRowAssertionsUtils.assertResultsIgnoreOrder; +import static org.apache.fluss.lake.paimon.utils.PaimonTestUtils.CompactCommitter; +import static org.apache.fluss.lake.paimon.utils.PaimonTestUtils.CompactHelper; +import static org.apache.fluss.testutils.DataTestUtils.row; +import static org.apache.fluss.utils.Preconditions.checkNotNull; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** + * The IT case for Flink union data in lake and fluss for paimon deletion-vectors enabled tables. + */ +class FlinkUnionReadDvTableITCase extends FlinkUnionReadTestBase { + + @TempDir private File tempDir; + + @BeforeAll + protected static void beforeAll() { + FlinkUnionReadTestBase.beforeAll(); + } + + @Test + void testUnionReadDvTable() throws Exception { + // first of all, start tiering + JobClient jobClient = buildTieringJob(execEnv); + try { + String tableName = "testUnionReadDvTable"; + TablePath tablePath = TablePath.of(DEFAULT_DB, tableName); + int bucketNum = 3; + long tableId = createDvEnabledTable(tablePath, bucketNum, false); + + List writtenRows = writeRows(tablePath, 0, 10); + Map bucketLogEndOffset = + getBucketLogEndOffset(tableId, bucketNum, null); + waitUntilBucketSynced(tablePath, tableId, DEFAULT_BUCKET_NUM, false); + assertReplicaStatus(bucketLogEndOffset); + + assertThatThrownBy(() -> admin.getReadableLakeSnapshot(tablePath).get()) + .rootCause() + .isInstanceOf(LakeTableSnapshotNotExistException.class); + + // although the readable lake snapshot doesn't exist, we can still + // so streaming reading + CloseableIterator rowIter = + streamTEnv.executeSql("select * from " + tableName).collect(); + + assertResultsIgnoreOrder(rowIter, toString(writtenRows), true); + + // Step 2: Compact bucket 0 and bucket 1 + FileStoreTable fileStoreTable = getPaimonTable(tablePath); + CompactHelper compactHelper = new CompactHelper(fileStoreTable, tempDir); + CompactCommitter compactBucket0 = compactHelper.compactBucket(0); + CompactCommitter compactBucket1 = compactHelper.compactBucket(1); + compactBucket0.commit(); + compactBucket1.commit(); + + // Step 3: write more records + writtenRows.addAll(writeRows(tablePath, 10, 20)); + + // Step 4: Wait until bucket synced and verify result + bucketLogEndOffset = getBucketLogEndOffset(tableId, bucketNum, null); + waitUntilBucketSynced(tablePath, tableId, DEFAULT_BUCKET_NUM, false); + assertReplicaStatus(bucketLogEndOffset); + + // still no readable snapshot + assertThatThrownBy(() -> admin.getReadableLakeSnapshot(tablePath).get()) + .rootCause() + .isInstanceOf(LakeTableSnapshotNotExistException.class); + + CloseableIterator rowIter2 = + streamTEnv.executeSql("select * from " + tableName).collect(); + assertResultsIgnoreOrder(rowIter2, toString(writtenRows), true); + + // Step 5: Compact bucket 2 + CompactCommitter compactBucket2 = compactHelper.compactBucket(2); + compactBucket2.commit(); + long expectedReadableSnapshot = + checkNotNull(fileStoreTable.snapshotManager().latestSnapshotId()); + + // Step 6: Write more data + writtenRows.addAll(writeRows(tablePath, 20, 25)); + + // Step 7: Query to verify result + bucketLogEndOffset = getBucketLogEndOffset(tableId, bucketNum, null); + waitUntilBucketSynced(tablePath, tableId, DEFAULT_BUCKET_NUM, false); + assertReplicaStatus(bucketLogEndOffset); + + LakeSnapshot lakeSnapshot = admin.getReadableLakeSnapshot(tablePath).get(); + assertThat(lakeSnapshot.getSnapshotId()).isEqualTo(expectedReadableSnapshot); + Map expectedEndOffset = new HashMap<>(); + expectedEndOffset.put(new TableBucket(tableId, 0), 4L); + expectedEndOffset.put(new TableBucket(tableId, 1), 3L); + expectedEndOffset.put(new TableBucket(tableId, 2), 7L); + assertThat(lakeSnapshot.getTableBucketsOffset()).isEqualTo(expectedEndOffset); + + CloseableIterator rowIter3 = + streamTEnv.executeSql("select * from " + tableName).collect(); + assertResultsIgnoreOrder(rowIter3, toString(writtenRows), true); + } finally { + jobClient.cancel().get(); + } + } + + private FileStoreTable getPaimonTable(TablePath tablePath) throws Exception { + Identifier identifier = + Identifier.create(tablePath.getDatabaseName(), tablePath.getTableName()); + return (FileStoreTable) paimonCatalog.getTable(identifier); + } + + /** + * Write rows to the table through Fluss API. + * + *

This method writes data through Fluss, which will then be synced to Paimon via tiering + * job. The data will be distributed to buckets based on the primary key hash. + * + * @param tablePath the table path + * @param from the starting value (inclusive) for the primary key + * @param to the ending value (exclusive) for the primary key + * @return the list of Flink Row objects that were written + */ + private List writeRows(TablePath tablePath, int from, int to) throws Exception { + List rows = new ArrayList<>(); + List flinkRows = new ArrayList<>(); + for (int i = from; i < to; i++) { + rows.add(row(i, "value1_" + i, "value2_" + i)); + flinkRows.add(Row.of(i, "value1_" + i, "value2_" + i)); + } + writeRows(tablePath, rows, false); + return flinkRows; + } + + private long createDvEnabledTable(TablePath tablePath, int bucketNum, boolean isPartitioned) + throws Exception { + Schema.Builder schemaBuilder = + Schema.newBuilder() + .column("c1", DataTypes.INT()) + .column("c2", DataTypes.STRING()) + .column("c3", DataTypes.STRING()); + + TableDescriptor.Builder tableBuilder = TableDescriptor.builder().distributedBy(bucketNum); + tableBuilder + .property(ConfigOptions.TABLE_DATALAKE_ENABLED.key(), "true") + .property(ConfigOptions.TABLE_DATALAKE_FRESHNESS, Duration.ofMillis(500)); + + // enabled dv + tableBuilder.customProperty("paimon.deletion-vectors.enabled", "true"); + + if (isPartitioned) { + tableBuilder.property(ConfigOptions.TABLE_AUTO_PARTITION_ENABLED, true); + tableBuilder.partitionedBy("c3"); + schemaBuilder.primaryKey("c1", "c3"); + tableBuilder.property( + ConfigOptions.TABLE_AUTO_PARTITION_TIME_UNIT, AutoPartitionTimeUnit.YEAR); + } else { + schemaBuilder.primaryKey("c1"); + } + tableBuilder.schema(schemaBuilder.build()); + return createTable(tablePath, tableBuilder.build()); + } + + private List writeRows(TablePath tablePath, int rowCount, @Nullable String partition) + throws Exception { + List rows = new ArrayList<>(); + List flinkRows = new ArrayList<>(); + for (int i = 0; i < rowCount; i++) { + if (partition == null) { + rows.add(row(i, "value1_" + i, "value2_" + i)); + flinkRows.add(Row.of(i, "value1_" + i, "value2_" + i)); + } else { + rows.add(row(i, "value1_" + i, partition)); + flinkRows.add(Row.of(i, "value1_" + i, partition)); + } + } + writeRows(tablePath, rows, false); + return flinkRows; + } + + private Map getBucketLogEndOffset( + long tableId, int bucketNum, Long partitionId) { + Map bucketLogEndOffsets = new HashMap<>(); + for (int i = 0; i < bucketNum; i++) { + TableBucket tableBucket = new TableBucket(tableId, partitionId, i); + Replica replica = getLeaderReplica(tableBucket); + bucketLogEndOffsets.put(tableBucket, replica.getLocalLogEndOffset()); + } + return bucketLogEndOffsets; + } + + private List toString(List rows) { + return rows.stream().map(Row::toString).collect(Collectors.toList()); + } +} diff --git a/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/tiering/PaimonTieringTest.java b/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/tiering/PaimonTieringTest.java index 56b3f533d5..8083058066 100644 --- a/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/tiering/PaimonTieringTest.java +++ b/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/tiering/PaimonTieringTest.java @@ -258,7 +258,10 @@ void testMultiPartitionTiering() throws Exception { try (LakeCommitter lakeCommitter = createLakeCommitter(tablePath, tableInfo, new Configuration())) { PaimonCommittable committable = lakeCommitter.toCommittable(paimonWriteResults); - long snapshot = lakeCommitter.commit(committable, Collections.emptyMap()); + long snapshot = + lakeCommitter + .commit(committable, Collections.emptyMap()) + .getCommittedSnapshotId(); assertThat(snapshot).isEqualTo(1); } @@ -330,7 +333,8 @@ void testThreePartitionTiering() throws Exception { try (LakeCommitter lakeCommitter = createLakeCommitter(tablePath, tableInfo, new Configuration())) { PaimonCommittable committable = lakeCommitter.toCommittable(paimonWriteResults); - snapshot = lakeCommitter.commit(committable, snapshotProperties); + snapshot = + lakeCommitter.commit(committable, snapshotProperties).getCommittedSnapshotId(); assertThat(snapshot).isEqualTo(1); } @@ -752,6 +756,12 @@ public TableInfo tableInfo() { public Configuration lakeTieringConfig() { return lakeTieringConfig; } + + @Override + public Configuration flussConfig() { + // don't care about fluss config + return new Configuration(); + } }); } diff --git a/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/utils/DvTableReadableSnapshotRetrieverTest.java b/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/utils/DvTableReadableSnapshotRetrieverTest.java new file mode 100644 index 0000000000..912c282859 --- /dev/null +++ b/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/utils/DvTableReadableSnapshotRetrieverTest.java @@ -0,0 +1,871 @@ +/* + * 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.lake.paimon.utils; + +import org.apache.fluss.client.Connection; +import org.apache.fluss.client.ConnectionFactory; +import org.apache.fluss.client.admin.Admin; +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.exception.FlussRuntimeException; +import org.apache.fluss.flink.tiering.committer.FlussTableLakeSnapshotCommitter; +import org.apache.fluss.lake.committer.LakeCommitResult; +import org.apache.fluss.metadata.PartitionInfo; +import org.apache.fluss.metadata.PartitionSpec; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TableDescriptor; +import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.server.testutils.FlussClusterExtension; +import org.apache.fluss.types.DataTypes; + +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogContext; +import org.apache.paimon.catalog.CatalogFactory; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.options.Options; +import org.apache.paimon.table.FileStoreTable; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +import javax.annotation.Nullable; + +import java.io.File; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.fluss.lake.paimon.utils.PaimonConversions.toPaimon; +import static org.apache.fluss.lake.paimon.utils.PaimonTestUtils.CompactCommitter; +import static org.apache.fluss.lake.paimon.utils.PaimonTestUtils.CompactHelper; +import static org.apache.fluss.lake.paimon.utils.PaimonTestUtils.writeAndCommitData; +import static org.apache.fluss.server.utils.LakeStorageUtils.extractLakeProperties; +import static org.assertj.core.api.Assertions.assertThat; + +/** Unit test for {@link DvTableReadableSnapshotRetriever} using real Paimon tables. */ +class DvTableReadableSnapshotRetrieverTest { + + @RegisterExtension + public static final FlussClusterExtension FLUSS_CLUSTER_EXTENSION = + FlussClusterExtension.builder() + .setNumOfTabletServers(3) + .setClusterConf(initConfig()) + .build(); + + private static Configuration initConfig() { + Configuration conf = new Configuration(); + conf.setString("datalake.format", "paimon"); + conf.setString("datalake.paimon.metastore", "filesystem"); + String warehousePath; + try { + warehousePath = + Files.createTempDirectory("fluss-testing-readable-snapshot") + .resolve("warehouse") + .toString(); + } catch (Exception e) { + throw new FlussRuntimeException("Failed to create warehouse path"); + } + conf.setString("datalake.paimon.warehouse", warehousePath); + paimonCatalog = + CatalogFactory.createCatalog( + CatalogContext.create(Options.fromMap(extractLakeProperties(conf)))); + + return conf; + } + + private static final String DEFAULT_DB = "fluss"; + + @TempDir private File compactionTempDir; + private static Catalog paimonCatalog; + private Connection flussConnection; + private Admin flussAdmin; + private long tableId; + private Configuration flussConf; + + private FlussTableLakeSnapshotCommitter lakeSnapshotCommitter; + + @BeforeEach + void setUp() { + flussConnection = + ConnectionFactory.createConnection(FLUSS_CLUSTER_EXTENSION.getClientConfig()); + flussAdmin = flussConnection.getAdmin(); + flussConf = FLUSS_CLUSTER_EXTENSION.getClientConfig(); + lakeSnapshotCommitter = new FlussTableLakeSnapshotCommitter(flussConf); + lakeSnapshotCommitter.open(); + } + + @AfterEach + void tearDown() throws Exception { + if (flussAdmin != null) { + flussAdmin.close(); + } + if (flussConnection != null) { + flussConnection.close(); + } + if (paimonCatalog != null) { + paimonCatalog.close(); + } + } + + @Test + void testGetReadableSnapshotAndOffsets() throws Exception { + int bucket0 = 0; + int bucket1 = 1; + int bucket2 = 2; + + // Given: create a DV table with 3 buckets: b1, b2, b3 + TablePath tablePath = TablePath.of(DEFAULT_DB, "test_dv_flow"); + tableId = createDvTable(tablePath, 3); + FileStoreTable fileStoreTable = getPaimonTable(tablePath); + + CompactHelper compactHelper = new CompactHelper(fileStoreTable, compactionTempDir); + TableBucket tb0 = new TableBucket(tableId, bucket0); + TableBucket tb1 = new TableBucket(tableId, bucket1); + TableBucket tb2 = new TableBucket(tableId, bucket2); + + // Step 1: APPEND snapshot 1 - write data to all buckets + // Snapshot 1 state: + // ┌─────────┬─────────────────────────────┐ + // │ Bucket │ Files │ + // ├─────────┼─────────────────────────────┤ + // │ bucket0 │ L0: [rows 0-2] │ + // │ bucket1 │ L0: [rows 0-2] │ + // │ bucket2 │ L0: [rows 0-2] │ + // └─────────┴─────────────────────────────┘ + Map> appendRows = new HashMap<>(); + appendRows.put(bucket0, generateRows(bucket0, 0, 3)); + appendRows.put(bucket1, generateRows(bucket1, 0, 3)); + appendRows.put(bucket2, generateRows(bucket2, 0, 3)); + long snapshot1 = writeAndCommitData(fileStoreTable, appendRows); + Map tieredLakeSnapshotEndOffset = new HashMap<>(); + tieredLakeSnapshotEndOffset.put(tb0, 3L); + tieredLakeSnapshotEndOffset.put(tb1, 3L); + tieredLakeSnapshotEndOffset.put(tb2, 3L); + + // No readable_snapshot yet (all buckets have L0 files) + DvTableReadableSnapshotRetriever.ReadableSnapshotResult readableSnapshotAndOffsets = + retriveReadableSnapshotAndOffsets(tablePath, fileStoreTable, snapshot1); + assertThat(readableSnapshotAndOffsets).isNull(); + // commit tiered snapshot and readable snapshot to fluss + // (simulate TieringCommitOperator) behavior + commitSnapshot( + tableId, + tablePath, + snapshot1, + tieredLakeSnapshotEndOffset, + readableSnapshotAndOffsets); + + // Step 2: COMPACT snapshot 2, 3 - compact bucket1 and bucket2 + // Prepare compactions for all buckets (b0, b1, b2), but only commit b1 and b2 + CompactCommitter compactCommitter2B0 = compactHelper.compactBucket(bucket0); // compact b0 + CompactCommitter compactCommitter2B1 = compactHelper.compactBucket(bucket1); // compact b1 + CompactCommitter compactCommitter2B2 = compactHelper.compactBucket(bucket2); // compact b2 + + // Commit compact b1 -> snapshot 2 + // Snapshot 2 state (after compacting bucket1): + // ┌─────────┬─────────────────────────────┐ + // │ Bucket │ Files │ + // ├─────────┼─────────────────────────────┤ + // │ bucket0 │ L0: [rows 0-2] │ ← unchanged + // │ bucket1 │ L1: [compacted from s1 L0] │ ← L0 flushed to L1 (rows 0-2) + // │ bucket2 │ L0: [rows 0-2] │ ← unchanged + // └─────────┴─────────────────────────────┘ + compactCommitter2B1.commit(); + // Commit compact b2 -> snapshot 3 + // Snapshot 3 state (after compacting bucket2): + // ┌─────────┬─────────────────────────────┐ + // │ Bucket │ Files │ + // ├─────────┼─────────────────────────────┤ + // │ bucket0 │ L0: [rows 0-2] │ ← unchanged + // │ bucket1 │ L1: [from s2 (rows 0-2)] │ ← unchanged + // │ bucket2 │ L1: [compacted from s1 L0] │ ← L0 flushed to L1 (rows 0-2) + // └─────────┴─────────────────────────────┘ + compactCommitter2B2.commit(); + + // Step 3: APPEND snapshot 4 - write more data to bucket0 + // Snapshot 4 state: + // ┌─────────┬─────────────────────────────┐ + // │ Bucket │ Files │ + // ├─────────┼─────────────────────────────┤ + // │ bucket0 │ L0: [rows 0-2 from s1] │ ← from snapshot1 + // │ │ L0: [rows 3-7] ← new │ ← added in snapshot4 + // │ bucket1 │ L1: [from s2 (rows 0-2)] │ ← unchanged + // │ bucket2 │ L1: [from s3 (rows 0-2)] │ ← unchanged + // └─────────┴─────────────────────────────┘ + long snapshot4 = + writeAndCommitData( + fileStoreTable, + Collections.singletonMap(bucket0, generateRows(bucket0, 3, 8))); + tieredLakeSnapshotEndOffset.put(tb0, 8L); + // retrive readable snapshot and offsets + readableSnapshotAndOffsets = + retriveReadableSnapshotAndOffsets(tablePath, fileStoreTable, snapshot4); + // (simulate TieringCommitOperator) behavior + commitSnapshot( + tableId, + tablePath, + snapshot4, + tieredLakeSnapshotEndOffset, + readableSnapshotAndOffsets); + + // Should still be null since bucket0 still has L0 files, + // and no L0 files were flushed until snapshot3 (which only flushed b1 and b2) + // we don't just use 0 as the log end offset for safety, + // todo: maybe consider use 0 as the log end offset for bucket0 no any L0 files + // flushed from the first tiered snapshot to latest compacted snapshot + assertThat(readableSnapshotAndOffsets).isNull(); + + // Step 4: COMPACT snapshot 5 - commit b0 compact (prepared based on snapshot1) + // Snapshot 5 state (after compacting bucket0): + // ┌─────────┬─────────────────────────────┐ + // │ Bucket │ Files │ + // ├─────────┼─────────────────────────────┤ + // │ bucket0 │ L1: [compacted from s1 L0] │ ← s1's L0 (rows 0-2) flushed to L1 + // │ │ L0: [rows 3-7 from s4] │ ← s4's L0 still remains + // │ bucket1 │ L1: [from s2 (rows 0-2)] │ ← unchanged + // │ bucket2 │ L1: [from s3 (rows 0-2)] │ ← unchanged + // └─────────┴─────────────────────────────┘ + compactCommitter2B0.commit(); + long snapshot5 = latestSnapshot(fileStoreTable); + + // Step 5: APPEND snapshot 6 - write more data to bucket1 + // Snapshot 6 state: + // ┌─────────┬─────────────────────────────┐ + // │ Bucket │ Files │ + // ├─────────┼─────────────────────────────┤ + // │ bucket0 │ L1: [from s5 (rows 0-2)] │ + // │ │ L0: [rows 3-7 from s4] │ ← unchanged + // │ bucket1 │ L1: [from s2 (rows 0-2)] │ + // │ │ L0: [rows 3-9] ← new │ ← added in snapshot6 + // │ bucket2 │ L1: [from s3 (rows 0-2)] │ ← unchanged + // └─────────┴─────────────────────────────┘ + // Now readable_snapshot can advance: bucket0 and bucket1 have L0, but bucket2 has no L0 + // However, bucket0 and bucket1's L0 files were flushed in snapshot5 and snapshot2 + // respectively, so we can use their base snapshots' offsets + long snapshot6 = + writeAndCommitData( + fileStoreTable, + Collections.singletonMap(bucket1, generateRows(bucket1, 3, 10))); + tieredLakeSnapshotEndOffset.put(tb1, 10L); + readableSnapshotAndOffsets = + retriveReadableSnapshotAndOffsets(tablePath, fileStoreTable, snapshot6); + // readable_snapshot = snapshot5 (latest compacted snapshot) + // readable_offsets: all buckets use snapshot1's offsets (base snapshot for flushed L0) + assertThat(readableSnapshotAndOffsets.getReadableSnapshotId()).isEqualTo(snapshot5); + Map expectedReadableOffsets = new HashMap<>(); + expectedReadableOffsets.put(tb0, 3L); + expectedReadableOffsets.put(tb1, 3L); + expectedReadableOffsets.put(tb2, 3L); + assertThat(readableSnapshotAndOffsets.getReadableOffsets()) + .isEqualTo(expectedReadableOffsets); + commitSnapshot( + tableId, + tablePath, + snapshot6, + tieredLakeSnapshotEndOffset, + readableSnapshotAndOffsets); + + // Step 6: COMPACT snapshot 7 - compact bucket0 again (flushes snapshot4's L0) + // Snapshot 7 state (after compacting bucket0): + // ┌─────────┬─────────────────────────────┐ + // │ Bucket │ Files │ + // ├─────────┼─────────────────────────────┤ + // │ bucket0 │ L1: [merged s5 L1 + s4 L0] │ ← s4's L0 (rows 3-7) flushed to L1 + // │ │ [rows 0-7 total] │ + // │ bucket1 │ L1: [from s2 (rows 0-2)] │ + // │ │ L0: [rows 3-9 from s6] │ ← unchanged + // │ bucket2 │ L1: [from s3 (rows 0-2)] │ ← unchanged + // └─────────┴─────────────────────────────┘ + // readable_snapshot advances: bucket0's L0 flushed, use snapshot4's offset (8L) + compactHelper.compactBucket(bucket0).commit(); + long snapshot7 = latestSnapshot(fileStoreTable); + + // Create an empty tiered snapshot (snapshot8) to simulate tiered snapshot commit + long snapshot8 = writeAndCommitData(fileStoreTable, Collections.emptyMap()); + + // retrieve the readable snapshot and offsets + readableSnapshotAndOffsets = + retriveReadableSnapshotAndOffsets(tablePath, fileStoreTable, snapshot8); + // readable_snapshot = snapshot7 + // readable_offsets: bucket0 uses snapshot4's offset (8L), others use snapshot1's offset + assertThat(readableSnapshotAndOffsets.getReadableSnapshotId()).isEqualTo(snapshot7); + expectedReadableOffsets = new HashMap<>(); + expectedReadableOffsets.put(tb0, 8L); + expectedReadableOffsets.put(tb1, 3L); + expectedReadableOffsets.put(tb2, 3L); + assertThat(readableSnapshotAndOffsets.getReadableOffsets()) + .isEqualTo(expectedReadableOffsets); + commitSnapshot( + tableId, tablePath, snapshot8, Collections.emptyMap(), readableSnapshotAndOffsets); + + // Step 7: COMPACT snapshot 9, 10, 11 - compact all buckets + // Snapshot 9: compact bucket0 (no new L0 to flush) + // Snapshot 10: compact bucket1 (flushes snapshot6's L0) + // Snapshot 11: compact bucket2 (no new L0 to flush) + // Snapshot 11 state (after compacting all buckets): + // ┌─────────┬─────────────────────────────┐ + // │ Bucket │ Files │ + // ├─────────┼─────────────────────────────┤ + // │ bucket0 │ L1: [from s7 (rows 0-7)] │ ← unchanged + // │ bucket1 │ L1: [merged s2 L1 + s6 L0] │ ← s6's L0 (rows 3-9) flushed to L1 + // │ │ [rows 0-9 total] │ + // │ bucket2 │ L1: [from s3 (rows 0-2)] │ ← unchanged + // └─────────┴─────────────────────────────┘ + compactHelper.compactBucket(bucket0).commit(); + compactHelper.compactBucket(bucket1).commit(); + compactHelper.compactBucket(bucket2).commit(); + + long snapshot11 = latestSnapshot(fileStoreTable); + + // Step 8: APPEND snapshot 12 - write data to all buckets + // Snapshot 12 state: + // ┌─────────┬─────────────────────────────┐ + // │ Bucket │ Files │ + // ├─────────┼─────────────────────────────┤ + // │ bucket0 │ L1: [from s11 (rows 0-7)] │ + // │ │ L0: [rows 8-12] ← new │ ← added in snapshot12 + // │ bucket1 │ L1: [from s11 (rows 0-9)] │ + // │ │ L0: [rows 10-19] ← new │ ← added in snapshot12 + // │ bucket2 │ L1: [from s11 (rows 0-2)] │ + // │ │ L0: [rows 3-10] ← new │ ← added in snapshot12 + // └─────────┴─────────────────────────────┘ + // readable_snapshot = snapshot11 (latest compacted snapshot) + // readable_offsets: bucket0 uses snapshot4's offset (8L), bucket1 uses snapshot6's offset + // (10L), bucket2 uses snapshot1's offset (3L) + Map> appendRows12 = new HashMap<>(); + appendRows12.put(bucket0, generateRows(bucket0, 8, 13)); + appendRows12.put(bucket1, generateRows(bucket1, 10, 20)); + appendRows12.put(bucket2, generateRows(bucket2, 3, 11)); + long snapshot12 = writeAndCommitData(fileStoreTable, appendRows12); + readableSnapshotAndOffsets = + retriveReadableSnapshotAndOffsets(tablePath, fileStoreTable, snapshot12); + assertThat(readableSnapshotAndOffsets.getReadableSnapshotId()).isEqualTo(snapshot11); + tieredLakeSnapshotEndOffset.put(tb0, 13L); + tieredLakeSnapshotEndOffset.put(tb1, 20L); + tieredLakeSnapshotEndOffset.put(tb2, 11L); + expectedReadableOffsets = new HashMap<>(); + expectedReadableOffsets.put(tb0, 8L); + expectedReadableOffsets.put(tb1, 10L); + expectedReadableOffsets.put(tb2, 3L); + assertThat(readableSnapshotAndOffsets.getReadableOffsets()) + .isEqualTo(expectedReadableOffsets); + commitSnapshot( + tableId, + tablePath, + snapshot12, + tieredLakeSnapshotEndOffset, + readableSnapshotAndOffsets); + + // Step 9: COMPACT snapshot 13 - compact bucket2 (flushes snapshot12's L0) + // Snapshot 13 state (after compacting bucket2): + // ┌─────────┬─────────────────────────────┐ + // │ Bucket │ Files │ + // ├─────────┼─────────────────────────────┤ + // │ bucket0 │ L1: [from s11 (rows 0-7)] │ + // │ │ L0: [rows 8-12 from s12] │ ← unchanged + // │ bucket1 │ L1: [from s11 (rows 0-9)] │ + // │ │ L0: [rows 10-19 from s12] │ ← unchanged + // │ bucket2 │ L1: [merged s11 L1 + s12 L0]│ ← s12's L0 (rows 3-10) flushed to L1 + // │ │ [rows 0-10 total] │ + // └─────────┴─────────────────────────────┘ + // readable_snapshot advances: bucket2's L0 flushed, use snapshot12's offset (11L) + compactHelper.compactBucket(bucket2).commit(); + long snapshot13 = latestSnapshot(fileStoreTable); + // Create an empty tiered snapshot (snapshot14) to simulate tiered snapshot commit + long snapshot14 = writeAndCommitData(fileStoreTable, Collections.emptyMap()); + readableSnapshotAndOffsets = + retriveReadableSnapshotAndOffsets(tablePath, fileStoreTable, snapshot14); + // readable_snapshot = snapshot13 + // readable_offsets: bucket0 uses snapshot4's offset (8L), bucket1 uses snapshot6's offset + // (10L), bucket2 uses snapshot12's offset (11L) + assertThat(readableSnapshotAndOffsets.getReadableSnapshotId()).isEqualTo(snapshot13); + expectedReadableOffsets = new HashMap<>(); + expectedReadableOffsets.put(tb0, 8L); + expectedReadableOffsets.put(tb1, 10L); + expectedReadableOffsets.put(tb2, 11L); + assertThat(readableSnapshotAndOffsets.getReadableOffsets()) + .isEqualTo(expectedReadableOffsets); + // all buckets L0 level in snapshot6 has been flushed, we can delete all snapshots prior to + // snapshot6 safely since we won't need to search for any earlier snapshots to get readable + // offsets + assertThat(readableSnapshotAndOffsets.getEarliestSnapshotIdToKeep()).isEqualTo(6); + } + + @Test + void testGetReadableSnapshotAndOffsetsForPartitionedTable() throws Exception { + String partition0 = "p0"; + String partition1 = "p1"; + int bucket0 = 0; + int bucket1 = 1; + + // Given: create a partitioned DV table with 2 buckets and partition column "dt" + TablePath tablePath = TablePath.of(DEFAULT_DB, "test_dv_flow_partitioned"); + tableId = createPartitionedDvTable(tablePath, 2); + FileStoreTable fileStoreTable = getPaimonTable(tablePath); + + // Get partition IDs from Fluss + Map partitionIdToName = createTwoPartitions(tablePath); + Long partitionId0 = getPartitionIdByName(partitionIdToName, partition0); + Long partitionId1 = getPartitionIdByName(partitionIdToName, partition1); + + PaimonTestUtils.CompactHelper compactHelper = + new PaimonTestUtils.CompactHelper(fileStoreTable, compactionTempDir); + TableBucket tbP0B0 = new TableBucket(tableId, partitionId0, bucket0); + TableBucket tbP0B1 = new TableBucket(tableId, partitionId0, bucket1); + TableBucket tbP1B0 = new TableBucket(tableId, partitionId1, bucket0); + TableBucket tbP1B1 = new TableBucket(tableId, partitionId1, bucket1); + + // Step 1: APPEND snapshot 1 - write data to partition0, bucket0 and bucket1 + // Snapshot 1 state: + // ┌─────────────┬─────────┬─────────────────────────────┐ + // │ Partition │ Bucket │ Files │ + // ├─────────────┼─────────┼─────────────────────────────┤ + // │ partition0 │ bucket0 │ L0: [rows 0-2] │ + // │ partition0 │ bucket1 │ L0: [rows 0-2] │ + // │ partition1 │ (empty) │ │ + // └─────────────┴─────────┴─────────────────────────────┘ + Map> appendRowsP0 = new HashMap<>(); + appendRowsP0.put(bucket0, generateRowsForPartition(partition0, bucket0, 0, 3)); + appendRowsP0.put(bucket1, generateRowsForPartition(partition0, bucket1, 0, 3)); + long snapshot1 = writeAndCommitData(fileStoreTable, appendRowsP0); + Map tieredLakeSnapshotEndOffset = new HashMap<>(); + tieredLakeSnapshotEndOffset.put(tbP0B0, 3L); + tieredLakeSnapshotEndOffset.put(tbP0B1, 3L); + + // No readable_snapshot yet (all buckets have L0 files) + DvTableReadableSnapshotRetriever.ReadableSnapshotResult readableSnapshotAndOffsets = + retriveReadableSnapshotAndOffsets(tablePath, fileStoreTable, snapshot1); + assertThat(readableSnapshotAndOffsets).isNull(); + commitSnapshot( + tableId, + tablePath, + snapshot1, + tieredLakeSnapshotEndOffset, + readableSnapshotAndOffsets); + + // Step 2: COMPACT snapshot 2 - compact partition0, bucket0 + // Snapshot 2 state (after compacting partition0, bucket0): + // ┌─────────────┬─────────┬─────────────────────────────┐ + // │ Partition │ Bucket │ Files │ + // ├─────────────┼─────────┼─────────────────────────────┤ + // │ partition0 │ bucket0 │ L1: [compacted from s1 L0] │ ← L0 flushed to L1 + // │ │ │ [rows 0-2] │ + // │ partition0 │ bucket1 │ L0: [rows 0-2] │ ← unchanged + // │ partition1 │ (empty) │ │ + // └─────────────┴─────────┴─────────────────────────────┘ + BinaryRow partition0BinaryRow = toPartitionBinaryRow(partition0); + compactHelper.compactBucket(partition0BinaryRow, bucket0).commit(); + long snapshot2 = latestSnapshot(fileStoreTable); + + // Step 3: APPEND snapshot 3 - write data to partition1, bucket0 + // Snapshot 3 state: + // ┌─────────────┬─────────┬─────────────────────────────┐ + // │ Partition │ Bucket │ Files │ + // ├─────────────┼─────────┼─────────────────────────────┤ + // │ partition0 │ bucket0 │ L1: [from s2 (rows 0-2)] │ ← unchanged + // │ partition0 │ bucket1 │ L0: [rows 0-2] │ ← unchanged + // │ partition1 │ bucket0 │ L0: [rows 0-2] │ ← new partition + // └─────────────┴─────────┴─────────────────────────────┘ + Map> appendRowsP1 = + Collections.singletonMap( + bucket0, generateRowsForPartition(partition1, bucket0, 0, 3)); + long snapshot3 = writeAndCommitData(fileStoreTable, appendRowsP1); + tieredLakeSnapshotEndOffset.put(tbP1B0, 3L); + + // Should still be null since partition0 bucket1 still has L0 files, + // and no L0 files for this bucket were flushed until snapshot3 + // todo: maybe consider use 0 as the log end offset for partition0 bucket1 if no any L0 + // files flushed from the first tiered snapshot to latest compacted snapshot + assertThat(readableSnapshotAndOffsets).isNull(); + readableSnapshotAndOffsets = + retriveReadableSnapshotAndOffsets(tablePath, fileStoreTable, snapshot3); + assertThat(readableSnapshotAndOffsets).isNull(); + commitSnapshot( + tableId, + tablePath, + snapshot3, + tieredLakeSnapshotEndOffset, + readableSnapshotAndOffsets); + + // Step 4: COMPACT snapshot 5 - compact partition0, bucket1 and partition1, bucket0 + // Snapshot 4 state (after compacting partition0, bucket1 and partition1, bucket0): + // ┌─────────────┬─────────┬─────────────────────────────┐ + // │ Partition │ Bucket │ Files │ + // ├─────────────┼─────────┼─────────────────────────────┤ + // │ partition0 │ bucket0 │ L1: [from s2 (rows 0-2)] │ ← unchanged + // │ partition0 │ bucket1 │ L1: [compacted from s1 L0] │ ← L0 flushed to L1 + // │ │ │ [rows 0-2] │ + // │ partition1 │ bucket0 │ L1: [compacted from s3 L0] │ ← L0 flushed to L1 + // │ │ │ [rows 0-2] │ + // └─────────────┴─────────┴─────────────────────────────┘ + compactHelper.compactBucket(partition0BinaryRow, bucket1).commit(); + BinaryRow partition1BinaryRow = toPartitionBinaryRow(partition1); + compactHelper.compactBucket(partition1BinaryRow, bucket0).commit(); + long snapshot5 = latestSnapshot(fileStoreTable); + + // Step 5: APPEND snapshot 6 - write more data to partition0, bucket0 + // Snapshot 5 state: + // ┌─────────────┬─────────┬─────────────────────────────┐ + // │ Partition │ Bucket │ Files │ + // ├─────────────┼─────────┼─────────────────────────────┤ + // │ partition0 │ bucket0 │ L1: [from s4] │ + // │ │ │ L0: [rows 3-5] ← new │ ← added in snapshot5 + // │ partition0 │ bucket1 │ L1: [from s4 (rows 0-2)] │ ← unchanged + // │ partition1 │ bucket0 │ L1: [from s4 (rows 0-2)] │ ← unchanged + // └─────────────┴─────────┴─────────────────────────────┘ + Map> appendRowsP0More = + Collections.singletonMap( + bucket0, generateRowsForPartition(partition0, bucket0, 3, 6)); + long snapshot6 = writeAndCommitData(fileStoreTable, appendRowsP0More); + tieredLakeSnapshotEndOffset.put(tbP0B0, 6L); + readableSnapshotAndOffsets = + retriveReadableSnapshotAndOffsets(tablePath, fileStoreTable, snapshot6); + // readable_snapshot = snapshot5 + // readable_offsets: partition0/bucket0 uses snapshot1's offset (3L), + // partition0/bucket1 uses snapshot1's offset (3L), + // partition1/bucket0 uses snapshot3's offset (3L) + assertThat(readableSnapshotAndOffsets.getReadableSnapshotId()).isEqualTo(snapshot5); + Map expectedReadableOffsets = new HashMap<>(); + expectedReadableOffsets.put(tbP0B0, 3L); + expectedReadableOffsets.put(tbP0B1, 3L); + expectedReadableOffsets.put(tbP1B0, 3L); + assertThat(readableSnapshotAndOffsets.getReadableOffsets()) + .isEqualTo(expectedReadableOffsets); + commitSnapshot( + tableId, + tablePath, + snapshot6, + tieredLakeSnapshotEndOffset, + readableSnapshotAndOffsets); + + // Step 6: COMPACT snapshot 7 - compact partition0, bucket0 again (flushes snapshot6's L0) + // Snapshot 7 state (after compacting partition0, bucket0): + // ┌─────────────┬─────────┬─────────────────────────────┐ + // │ Partition │ Bucket │ Files │ + // ├─────────────┼─────────┼─────────────────────────────┤ + // │ partition0 │ bucket0 │ L1: [merged s5 L1 + s6 L0] │ ← s6's L0 flushed to L1 + // │ │ │ [rows 0-5 total] │ + // │ partition0 │ bucket1 │ L1: [from s5 (rows 0-2)] │ ← unchanged + // │ partition1 │ bucket0 │ L1: [from s5 (rows 0-2)] │ ← unchanged (already L1 from + // s5) + // └─────────────┴─────────┴─────────────────────────────┘ + compactHelper.compactBucket(partition0BinaryRow, bucket0).commit(); + long snapshot7 = latestSnapshot(fileStoreTable); + + // Create an empty tiered snapshot (snapshot8) to simulate tiered snapshot commit + long snapshot8 = writeAndCommitData(fileStoreTable, Collections.emptyMap()); + + readableSnapshotAndOffsets = + retriveReadableSnapshotAndOffsets(tablePath, fileStoreTable, snapshot8); + // readable_snapshot = snapshot7 + // readable_offsets: partition0/bucket0 uses snapshot6's offset (6L), + // partition0/bucket1 uses snapshot1's offset (3L), + // partition1/bucket0 uses snapshot3's offset (3L) - from snapshot5 + // compaction + assertThat(readableSnapshotAndOffsets.getReadableSnapshotId()).isEqualTo(snapshot7); + expectedReadableOffsets = new HashMap<>(); + expectedReadableOffsets.put(tbP0B0, 6L); + expectedReadableOffsets.put(tbP0B1, 3L); + expectedReadableOffsets.put(tbP1B0, 3L); + assertThat(readableSnapshotAndOffsets.getReadableOffsets()) + .isEqualTo(expectedReadableOffsets); + commitSnapshot( + tableId, + tablePath, + snapshot8, + tieredLakeSnapshotEndOffset, + readableSnapshotAndOffsets); + + // Step 7: APPEND snapshot 9 - write more data to partition1, bucket0 + // Snapshot 9 state: + // ┌─────────────┬─────────┬─────────────────────────────┐ + // │ Partition │ Bucket │ Files │ + // ├─────────────┼─────────┼─────────────────────────────┤ + // │ partition0 │ bucket0 │ L1: [from s7 (rows 0-5)] │ ← unchanged + // │ partition0 │ bucket1 │ L1: [from s7 (rows 0-2)] │ ← unchanged + // │ partition1 │ bucket0 │ L1: [from s5 (rows 0-2)] │ + // │ │ │ L0: [rows 3-5] ← new │ ← added in snapshot9 + // └─────────────┴─────────┴─────────────────────────────┘ + Map> appendRowsP1More = + Collections.singletonMap( + bucket0, generateRowsForPartition(partition1, bucket0, 3, 6)); + long snapshot9 = writeAndCommitData(fileStoreTable, appendRowsP1More); + tieredLakeSnapshotEndOffset.put(tbP1B0, 6L); + readableSnapshotAndOffsets = + retriveReadableSnapshotAndOffsets(tablePath, fileStoreTable, snapshot9); + // readable_snapshot = snapshot7 (unchanged, partition1/bucket0 still has L0 from snapshot9) + // readable_offsets: partition0/bucket0 uses snapshot6's offset (6L), + // partition0/bucket1 uses snapshot1's offset (3L), + // partition1/bucket0 uses snapshot3's offset (3L) - from snapshot5 + // compaction + assertThat(readableSnapshotAndOffsets.getReadableSnapshotId()).isEqualTo(snapshot7); + expectedReadableOffsets = new HashMap<>(); + expectedReadableOffsets.put(tbP0B0, 6L); + expectedReadableOffsets.put(tbP0B1, 3L); + expectedReadableOffsets.put(tbP1B0, 3L); + assertThat(readableSnapshotAndOffsets.getReadableOffsets()) + .isEqualTo(expectedReadableOffsets); + // all buckets L0 level has been flushed in snapshot3, we can delete all snapshots prior to + // snapshot3 safely since we won't need to search for any earlier snapshots to get readable + // offsets + assertThat(readableSnapshotAndOffsets.getEarliestSnapshotIdToKeep()).isEqualTo(3); + commitSnapshot( + tableId, + tablePath, + snapshot9, + tieredLakeSnapshotEndOffset, + readableSnapshotAndOffsets); + + // Step 8: COMPACT snapshot 10 - compact partition1, bucket0 again (flushes snapshot9's L0) + // Snapshot 10 state (after compacting partition1, bucket0): + // ┌─────────────┬─────────┬─────────────────────────────┐ + // │ Partition │ Bucket │ Files │ + // ├─────────────┼─────────┼─────────────────────────────┤ + // │ partition0 │ bucket0 │ L1: [from s7 (rows 0-5)] │ ← unchanged + // │ partition0 │ bucket1 │ L1: [from s7 (rows 0-2)] │ ← unchanged + // │ partition1 │ bucket0 │ L1: [merged s5 L1 + s9 L0] │ ← s9's L0 flushed to L1 + // │ │ │ [rows 0-5 total] │ + // └─────────────┴─────────┴─────────────────────────────┘ + compactHelper.compactBucket(partition1BinaryRow, bucket0).commit(); + long snapshot10 = latestSnapshot(fileStoreTable); + + // Create an empty tiered snapshot (snapshot11) to simulate tiered snapshot commit + long snapshot11 = writeAndCommitData(fileStoreTable, Collections.emptyMap()); + + readableSnapshotAndOffsets = + retriveReadableSnapshotAndOffsets(tablePath, fileStoreTable, snapshot11); + // readable_snapshot = snapshot10 + // readable_offsets: partition0/bucket0 uses snapshot6's offset (6L), + // partition0/bucket1 uses snapshot1's offset (3L), + // partition1/bucket0 uses snapshot9's offset (6L) + assertThat(readableSnapshotAndOffsets.getReadableSnapshotId()).isEqualTo(snapshot10); + expectedReadableOffsets = new HashMap<>(); + expectedReadableOffsets.put(tbP0B0, 6L); + expectedReadableOffsets.put(tbP0B1, 3L); + expectedReadableOffsets.put(tbP1B0, 6L); + assertThat(readableSnapshotAndOffsets.getReadableOffsets()) + .isEqualTo(expectedReadableOffsets); + } + + private long latestSnapshot(FileStoreTable fileStoreTable) { + return fileStoreTable.latestSnapshot().get().id(); + } + + // Helper methods + private long createDvTable(TablePath tablePath, int numBuckets) throws Exception { + TableDescriptor tableDescriptor = + TableDescriptor.builder() + .schema( + org.apache.fluss.metadata.Schema.newBuilder() + .column("c1", DataTypes.INT()) + .column("c2", DataTypes.STRING()) + .primaryKey("c1") + .build()) + .distributedBy(numBuckets) + .property(ConfigOptions.TABLE_DATALAKE_ENABLED, true) + .customProperty("paimon.deletion-vectors.enabled", "true") + .build(); + flussAdmin.createTable(tablePath, tableDescriptor, false).get(); + + return flussAdmin.getTableInfo(tablePath).get().getTableId(); + } + + /** + * Create a partitioned DV table with partition column "dt". + * + * @param tablePath the table path + * @param numBuckets the number of buckets + * @return the table ID + */ + private long createPartitionedDvTable(TablePath tablePath, int numBuckets) throws Exception { + TableDescriptor tableDescriptor = + TableDescriptor.builder() + .schema( + org.apache.fluss.metadata.Schema.newBuilder() + .column("c1", DataTypes.INT()) + .column("c2", DataTypes.STRING()) + .column("dt", DataTypes.STRING()) + .primaryKey("c1", "dt") + .build()) + .distributedBy(numBuckets) + .partitionedBy("dt") + .property(ConfigOptions.TABLE_DATALAKE_ENABLED, true) + .customProperty("paimon.deletion-vectors.enabled", "true") + .build(); + flussAdmin.createTable(tablePath, tableDescriptor, false).get(); + + return flussAdmin.getTableInfo(tablePath).get().getTableId(); + } + + /** + * Wait until partitions are created and return the mapping from partition ID to partition name. + * + * @param tablePath the table path + * @return map from partition ID to partition name + */ + private Map createTwoPartitions(TablePath tablePath) throws Exception { + // For partitioned tables, partitions are created dynamically when data is written + // We need to manually create partitions first + flussAdmin + .createPartition( + tablePath, new PartitionSpec(Collections.singletonMap("dt", "p0")), true) + .get(); + flussAdmin + .createPartition( + tablePath, new PartitionSpec(Collections.singletonMap("dt", "p1")), true) + .get(); + + List partitionInfos = flussAdmin.listPartitionInfos(tablePath).get(); + Map partitionIdToName = new HashMap<>(); + for (PartitionInfo partitionInfo : partitionInfos) { + partitionIdToName.put(partitionInfo.getPartitionId(), partitionInfo.getPartitionName()); + } + return partitionIdToName; + } + + /** + * Get partition ID by partition name. + * + * @param partitionIdToName map from partition ID to partition name + * @param partitionName the partition name to look up + * @return the partition ID + */ + private Long getPartitionIdByName(Map partitionIdToName, String partitionName) { + return partitionIdToName.entrySet().stream() + .filter(entry -> entry.getValue().equals(partitionName)) + .map(Map.Entry::getKey) + .findFirst() + .orElseThrow( + () -> + new IllegalStateException( + "Partition name " + partitionName + " not found")); + } + + /** + * Generate rows for a specific partition with schema: (c1: INT, c2: STRING, dt: STRING, bucket: + * INT, c4: BIGINT, ts: TIMESTAMP). + * + * @param partitionName the partition name (value for "dt" column) + * @param bucket the bucket number + * @param from the starting value (inclusive) + * @param to the ending value (exclusive) + * @return a list of GenericRow instances + */ + private List generateRowsForPartition( + String partitionName, int bucket, int from, int to) { + List rows = new ArrayList<>(); + for (int i = from; i < to; i++) { + rows.add( + GenericRow.of( + i, + BinaryString.fromString("value" + i), + BinaryString.fromString(partitionName), + bucket, + (long) i, + org.apache.paimon.data.Timestamp.now())); + } + return rows; + } + + private BinaryRow toPartitionBinaryRow(String partitionName) { + return BinaryRow.singleColumn(partitionName); + } + + private DvTableReadableSnapshotRetriever.ReadableSnapshotResult + retriveReadableSnapshotAndOffsets( + TablePath tablePath, FileStoreTable fileStoreTable, long tieredSnapshot) + throws Exception { + try (DvTableReadableSnapshotRetriever retriever = + new DvTableReadableSnapshotRetriever( + tablePath, tableId, fileStoreTable, flussConf)) { + return retriever.getReadableSnapshotAndOffsets(tieredSnapshot); + } + } + + private FileStoreTable getPaimonTable(TablePath tablePath) throws Exception { + Identifier identifier = toPaimon(tablePath); + return (FileStoreTable) paimonCatalog.getTable(identifier); + } + + private void commitSnapshot( + long tableId, + TablePath tablePath, + long tieredSnapshot, + Map lakeSnapshotTieredEndOffset, + @Nullable + DvTableReadableSnapshotRetriever.ReadableSnapshotResult + readableSnapshotAndOffsets) + throws Exception { + String tieredSnapshotOffsetPath = + lakeSnapshotCommitter.prepareLakeSnapshot( + tableId, tablePath, lakeSnapshotTieredEndOffset); + LakeCommitResult lakeCommitResult; + if (readableSnapshotAndOffsets != null) { + lakeCommitResult = + LakeCommitResult.withReadableSnapshot( + tieredSnapshot, + readableSnapshotAndOffsets.getReadableSnapshotId(), + lakeSnapshotTieredEndOffset, + readableSnapshotAndOffsets.getReadableOffsets(), + readableSnapshotAndOffsets.getEarliestSnapshotIdToKeep()); + } else { + lakeCommitResult = LakeCommitResult.unknownReadableSnapshot(tieredSnapshot); + } + lakeSnapshotCommitter.commit( + tableId, + tablePath, + lakeCommitResult, + tieredSnapshotOffsetPath, + lakeSnapshotTieredEndOffset, + Collections.emptyMap()); + } + + /** + * Generate rows for testing with a schema that includes bucket information. + * + *

This is useful for tests that need to verify bucket-specific behavior. The generated rows + * have the schema: (c1: INT, c2: STRING, bucket: INT, c4: BIGINT, ts: TIMESTAMP). + * + * @param bucket the bucket number + * @param from the starting value (inclusive) + * @param to the ending value (exclusive) + * @return a list of GenericRow instances + */ + public static List generateRows(int bucket, int from, int to) { + List rows = new java.util.ArrayList<>(); + for (int i = from; i < to; i++) { + rows.add( + GenericRow.of( + i, + BinaryString.fromString("value" + i), + bucket, + (long) i, + org.apache.paimon.data.Timestamp.now())); + } + return rows; + } +} diff --git a/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/utils/PaimonDvTableUtilsTest.java b/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/utils/PaimonDvTableUtilsTest.java new file mode 100644 index 0000000000..89e4eabb65 --- /dev/null +++ b/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/utils/PaimonDvTableUtilsTest.java @@ -0,0 +1,241 @@ +/* + * 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.lake.paimon.utils; + +import org.apache.paimon.Snapshot; +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogContext; +import org.apache.paimon.catalog.CatalogFactory; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.options.ExpireConfig; +import org.apache.paimon.options.Options; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.types.DataTypes; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.nio.file.Path; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.fluss.lake.paimon.utils.PaimonDvTableUtils.findLatestSnapshotExactlyHoldingL0Files; +import static org.apache.fluss.lake.paimon.utils.PaimonTestUtils.CompactCommitter; +import static org.apache.fluss.lake.paimon.utils.PaimonTestUtils.writeAndCommitData; +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link PaimonDvTableUtils}. */ +class PaimonDvTableUtilsTest { + + private static final String DEFAULT_DB = "test_db"; + + @TempDir private Path tempDir; + @TempDir private File compactionTempDir; + private Catalog paimonCatalog; + + @BeforeEach + void setUp() throws Exception { + // Create a local filesystem catalog for testing + Map options = new HashMap<>(); + options.put("warehouse", tempDir.resolve("warehouse").toString()); + options.put("metastore", "filesystem"); + paimonCatalog = + CatalogFactory.createCatalog(CatalogContext.create(Options.fromMap(options))); + paimonCatalog.createDatabase(DEFAULT_DB, false); + } + + @AfterEach + void tearDown() throws Exception { + if (paimonCatalog != null) { + paimonCatalog.close(); + } + } + + @Test + void testFindLatestSnapshotExactlyHoldingL0Files() throws Exception { + int bucket0 = 0; + int bucket1 = 1; + + // Given: create a DV table with 2 buckets + Identifier tableIdentifier = Identifier.create(DEFAULT_DB, "test_find_base_snapshot"); + FileStoreTable fileStoreTable = createDvTable(tableIdentifier, 2); + + PaimonTestUtils.CompactHelper compactHelper = + new PaimonTestUtils.CompactHelper(fileStoreTable, compactionTempDir); + + // Step 1: APPEND snapshot 1 - write data to bucket0 and bucket1 + // Snapshot 1 state: + // ┌─────────┬─────────────────────────────┐ + // │ Bucket │ Files │ + // ├─────────┼─────────────────────────────┤ + // │ bucket0 │ L0: [rows 0-2] │ + // │ bucket1 │ L0: [rows 0-2] │ + // └─────────┴─────────────────────────────┘ + Map> appendRows = new HashMap<>(); + appendRows.put(bucket0, generateRows(0, 3)); + appendRows.put(bucket1, generateRows(0, 3)); + long snapshot1 = writeAndCommitData(fileStoreTable, appendRows); + + // Step 2: COMPACT snapshot 2 - compact bucket0 + // Snapshot 2 state (after compacting bucket0): + // ┌─────────┬─────────────────────────────┐ + // │ Bucket │ Files │ + // ├─────────┼─────────────────────────────┤ + // │ bucket0 │ L1: [compacted from s1 L0] │ ← L0 flushed to L1 + // │ bucket1 │ L0: [rows 0-2] │ ← unchanged + // └─────────┴─────────────────────────────┘ + CompactCommitter compactCommitter1Bucket0 = compactHelper.compactBucket(bucket0); + compactCommitter1Bucket0.commit(); + + // Prepare compact for bucket1 (not committed yet) + CompactCommitter compactCommitter1Bucket1 = compactHelper.compactBucket(bucket1); + + // Verify: snapshot2 flushed bucket0's L0 files, which were added in snapshot1 + // So snapshot1 is the latest snapshot holding those L0 files + long snapshot2 = fileStoreTable.snapshotManager().latestSnapshot().id(); + Snapshot snapshot2Obj = fileStoreTable.snapshotManager().snapshot(snapshot2); + Snapshot latestHoldL0Snapshot = + findLatestSnapshotExactlyHoldingL0Files(fileStoreTable, snapshot2Obj); + assertThat(latestHoldL0Snapshot).isNotNull(); + assertThat(latestHoldL0Snapshot.id()).isEqualTo(snapshot1); + + // Step 3: APPEND snapshot 3 - write more data to bucket0 + // Snapshot 3 state: + // ┌─────────┬─────────────────────────────┐ + // │ Bucket │ Files │ + // ├─────────┼─────────────────────────────┤ + // │ bucket0 │ L1: [from s2] │ + // │ │ L0: [rows 3-5] ← new │ + // │ bucket1 │ L0: [rows 0-2] ← unchanged │ + // └─────────┴─────────────────────────────┘ + long snapshot3 = + writeAndCommitData( + fileStoreTable, Collections.singletonMap(bucket0, generateRows(3, 6))); + fileStoreTable.snapshotManager().snapshot(snapshot3); + + // Step 4: COMPACT snapshot 4 - compact bucket0 again (flushes snapshot3's L0) + // Snapshot 4 state (after compacting bucket0): + // ┌─────────┬─────────────────────────────┐ + // │ Bucket │ Files │ + // ├─────────┼─────────────────────────────┤ + // │ bucket0 │ L1: [merged s2 L1 + s3 L0] │ ← L0 flushed to L1 + // │ bucket1 │ L0: [rows 0-2] │ ← unchanged + // └─────────┴─────────────────────────────┘ + CompactCommitter compactCommitter3Bucket0 = compactHelper.compactBucket(bucket0); + compactCommitter3Bucket0.commit(); + + // Verify: snapshot4 flushed bucket0's L0 files, which were added in snapshot3 + // So snapshot3 is the latest snapshot holding those L0 files + long snapshot4 = fileStoreTable.snapshotManager().latestSnapshot().id(); + Snapshot snapshot4Obj = fileStoreTable.snapshotManager().snapshot(snapshot4); + latestHoldL0Snapshot = + findLatestSnapshotExactlyHoldingL0Files(fileStoreTable, snapshot4Obj); + assertThat(latestHoldL0Snapshot).isNotNull(); + assertThat(latestHoldL0Snapshot.id()).isEqualTo(snapshot3); + + // Step 5: APPEND snapshot 5 - write more data to bucket1 + // Snapshot 5 state: + // ┌─────────┬─────────────────────────────┐ + // │ Bucket │ Files │ + // ├─────────┼─────────────────────────────┤ + // │ bucket0 │ L1: [from s4] │ ← unchanged + // │ bucket1 │ L0: [rows 0-2 from s1] │ ← from snapshot1 + // │ │ L0: [rows 3-5] ← new │ ← added in snapshot5 + // └─────────┴─────────────────────────────┘ + long snapshot5 = + writeAndCommitData( + fileStoreTable, Collections.singletonMap(bucket1, generateRows(3, 6))); + + // Step 6: COMPACT snapshot 6 - commit compactCommitter1_b1 (compact bucket1) + // Note: compactCommitter1_b1 was prepared based on snapshot1, so it will only compact + // bucket1's L0 files from snapshot1 (rows 0-2), not the new L0 files added in snapshot5. + // Snapshot 6 state (after compacting bucket1): + // ┌─────────┬─────────────────────────────┐ + // │ Bucket │ Files │ + // ├─────────┼─────────────────────────────┤ + // │ bucket0 │ L1: [from s4] │ ← unchanged + // │ bucket1 │ L1: [compacted from s1 L0] │ ← s1's L0 (rows 0-2) flushed to L1 + // │ │ L0: [rows 3-5 from s5] │ ← s5's L0 still remains + // └─────────┴─────────────────────────────┘ + compactCommitter1Bucket1.commit(); + long snapshot6 = fileStoreTable.snapshotManager().latestSnapshot().id(); + Snapshot snapshot6Obj = fileStoreTable.snapshotManager().snapshot(snapshot6); + latestHoldL0Snapshot = + findLatestSnapshotExactlyHoldingL0Files(fileStoreTable, snapshot6Obj); + // Verify: snapshot6 flushed bucket1's L0 files from snapshot1 (rows 0-2). + // These L0 files were added in snapshot1, and they still exist(with all L0 files match) in + // snapshot4, (snapshot5 doesn't exactly match flushed L0 files, with extra L0 files). + // So snapshot4 is the latest snapshot exactly holding those L0 files. + assertThat(latestHoldL0Snapshot).isNotNull(); + assertThat(latestHoldL0Snapshot.id()).isEqualTo(snapshot4); + + // test snapshot expiration case + fileStoreTable + .newExpireSnapshots() + .config(ExpireConfig.builder().snapshotRetainMax(1).build()) + .expire(); + latestHoldL0Snapshot = + findLatestSnapshotExactlyHoldingL0Files(fileStoreTable, snapshot6Obj); + // the snapshot4 is expired, should get null now + assertThat(latestHoldL0Snapshot).isNull(); + } + + /** + * Create a DV table using Paimon catalog directly. + * + * @param tableIdentifier the table identifier + * @param numBuckets the number of buckets + * @return the FileStoreTable instance + */ + private FileStoreTable createDvTable(Identifier tableIdentifier, int numBuckets) + throws Exception { + Schema schema = + Schema.newBuilder() + .column("c1", DataTypes.INT()) + .column("c2", DataTypes.STRING()) + .primaryKey("c1") + .option("bucket", String.valueOf(numBuckets)) + .option("deletion-vectors.enabled", "true") + .build(); + paimonCatalog.createTable(tableIdentifier, schema, false); + return (FileStoreTable) paimonCatalog.getTable(tableIdentifier); + } + + /** + * Generate rows for testing with a simple schema (c1: INT, c2: STRING). + * + * @param from the starting value (inclusive) + * @param to the ending value (exclusive) + * @return a list of GenericRow instances + */ + public static List generateRows(int from, int to) { + List rows = new java.util.ArrayList<>(); + for (int i = from; i < to; i++) { + rows.add(GenericRow.of(i, BinaryString.fromString("value" + i))); + } + return rows; + } +} diff --git a/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/utils/PaimonTestUtils.java b/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/utils/PaimonTestUtils.java new file mode 100644 index 0000000000..485af4d4d6 --- /dev/null +++ b/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/utils/PaimonTestUtils.java @@ -0,0 +1,133 @@ +/* + * 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.lake.paimon.utils; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.disk.IOManager; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.sink.BatchTableCommit; +import org.apache.paimon.table.sink.BatchTableWrite; +import org.apache.paimon.table.sink.BatchWriteBuilder; +import org.apache.paimon.table.sink.CommitMessage; + +import java.io.File; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Utility class for Paimon test helpers that can be shared across test classes. + * + *

This class provides common methods for creating tables, writing data, and triggering + * compactions in Paimon tests. + */ +public class PaimonTestUtils { + + /** + * Write data to multiple buckets in a single commit, creating one snapshot. + * + * @param fileStoreTable the FileStoreTable instance + * @param bucketRows map of bucket -> rows to append to the bucket + * @return the snapshot ID after commit + */ + public static long writeAndCommitData( + FileStoreTable fileStoreTable, Map> bucketRows) + throws Exception { + Map options = new HashMap<>(); + options.put(CoreOptions.WRITE_ONLY.key(), "true"); + options.put(CoreOptions.SNAPSHOT_IGNORE_EMPTY_COMMIT.key(), "false"); + FileStoreTable writeTable = fileStoreTable.copy(options); + BatchWriteBuilder writeBuilder = writeTable.newBatchWriteBuilder(); + try (BatchTableWrite write = writeBuilder.newWrite(); + BatchTableCommit commit = writeBuilder.newCommit()) { + for (Map.Entry> entry : bucketRows.entrySet()) { + int bucket = entry.getKey(); + for (GenericRow row : entry.getValue()) { + write.write(row, bucket); + } + } + List messages = write.prepareCommit(); + commit.commit(messages); + } + return fileStoreTable.snapshotManager().latestSnapshot().id(); + } + + /** Helper class for compacting buckets in tests. */ + public static class CompactHelper { + private final FileStoreTable fileStoreTable; + private final File compactionTempDir; + + public CompactHelper(FileStoreTable fileStoreTable, File compactionTempDir) { + this.fileStoreTable = fileStoreTable; + this.compactionTempDir = compactionTempDir; + } + + public CompactCommitter compactBucket(BinaryRow partition, int bucket) throws Exception { + Map options = new HashMap<>(); + options.put(CoreOptions.WRITE_ONLY.key(), String.valueOf(false)); + FileStoreTable compactTable = fileStoreTable.copy(options); + BatchWriteBuilder writeBuilder = compactTable.newBatchWriteBuilder(); + try (BatchTableWrite write = + (BatchTableWrite) + writeBuilder + .newWrite() + .withIOManager( + IOManager.create(compactionTempDir.toString()))) { + BatchTableCommit batchTableCommit = writeBuilder.newCommit(); + write.compact(partition, bucket, false); + // All compaction operations are prepared together and will be committed atomically + List messages = write.prepareCommit(); + return new CompactCommitter(batchTableCommit, messages); + } + } + + public CompactCommitter compactBucket(int bucket) throws Exception { + return compactBucket(BinaryRow.EMPTY_ROW, bucket); + } + } + + /** + * A helper class to commit compaction operations later. + * + *

This allows preparing compaction operations and committing them at a specific point in the + * test, which is useful for testing scenarios where multiple compactions are prepared but + * committed in a specific order. + */ + public static class CompactCommitter { + private final BatchTableCommit tableCommit; + private final List messages; + + private CompactCommitter(BatchTableCommit batchTableCommit, List messages) { + this.tableCommit = batchTableCommit; + this.messages = messages; + } + + /** + * Commit the compaction operation. + * + * @throws Exception if the commit fails + */ + public void commit() throws Exception { + tableCommit.commit(messages); + tableCommit.close(); + } + } +} diff --git a/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/AdminReadOnlyGateway.java b/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/AdminReadOnlyGateway.java index 6a6e21149f..272d1b4a11 100644 --- a/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/AdminReadOnlyGateway.java +++ b/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/AdminReadOnlyGateway.java @@ -28,10 +28,10 @@ import org.apache.fluss.rpc.messages.GetFileSystemSecurityTokenResponse; import org.apache.fluss.rpc.messages.GetKvSnapshotMetadataRequest; import org.apache.fluss.rpc.messages.GetKvSnapshotMetadataResponse; +import org.apache.fluss.rpc.messages.GetLakeSnapshotRequest; +import org.apache.fluss.rpc.messages.GetLakeSnapshotResponse; import org.apache.fluss.rpc.messages.GetLatestKvSnapshotsRequest; import org.apache.fluss.rpc.messages.GetLatestKvSnapshotsResponse; -import org.apache.fluss.rpc.messages.GetLatestLakeSnapshotRequest; -import org.apache.fluss.rpc.messages.GetLatestLakeSnapshotResponse; import org.apache.fluss.rpc.messages.GetTableInfoRequest; import org.apache.fluss.rpc.messages.GetTableInfoResponse; import org.apache.fluss.rpc.messages.GetTableSchemaRequest; @@ -177,9 +177,8 @@ CompletableFuture listPartitionInfos( * @param request request that specifies that table path. * @return a future returns the lake snapshot of the table. */ - @RPC(api = ApiKeys.GET_LATEST_LAKE_SNAPSHOT) - CompletableFuture getLatestLakeSnapshot( - GetLatestLakeSnapshotRequest request); + @RPC(api = ApiKeys.GET_LAKE_SNAPSHOT) + CompletableFuture getLakeSnapshot(GetLakeSnapshotRequest request); /** * List acls for a table. diff --git a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java index cc033ba8a9..151c657a30 100644 --- a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java +++ b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java @@ -60,7 +60,7 @@ public enum ApiKeys { NOTIFY_KV_SNAPSHOT_OFFSET(1029, 0, 0, PRIVATE), COMMIT_LAKE_TABLE_SNAPSHOT(1030, 0, 0, PRIVATE), NOTIFY_LAKE_TABLE_OFFSET(1031, 0, 0, PRIVATE), - GET_LATEST_LAKE_SNAPSHOT(1032, 0, 0, PUBLIC), + GET_LAKE_SNAPSHOT(1032, 0, 0, PUBLIC), LIMIT_SCAN(1033, 0, 0, PUBLIC), PREFIX_LOOKUP(1034, 0, 0, PUBLIC), GET_DATABASE_INFO(1035, 0, 0, PUBLIC), diff --git a/fluss-rpc/src/main/proto/FlussApi.proto b/fluss-rpc/src/main/proto/FlussApi.proto index 6698d98d58..43c3532063 100644 --- a/fluss-rpc/src/main/proto/FlussApi.proto +++ b/fluss-rpc/src/main/proto/FlussApi.proto @@ -364,11 +364,13 @@ message GetKvSnapshotMetadataResponse { repeated PbRemotePathAndLocalFile snapshot_files = 2; } -message GetLatestLakeSnapshotRequest { +message GetLakeSnapshotRequest { required PbTablePath table_path = 1; + optional int64 snapshot_id = 2; // If not set, returns the latest snapshot + optional bool readable = 3; // If true, returns the latest readable snapshot } -message GetLatestLakeSnapshotResponse { +message GetLakeSnapshotResponse { required int64 table_id = 1; required int64 snapshotId = 2; repeated PbLakeSnapshotForBucket bucket_snapshots = 3; @@ -451,6 +453,11 @@ message NotifyRemoteLogOffsetsResponse { message PrepareLakeTableSnapshotRequest { repeated PbTableOffsets bucket_offsets = 1; + // If true, ignore previous table offsets and do not merge with existing lake table offsets. + // If false (default), the system will merge the provided bucket offsets with the previous + // lake table offsets to ensure continuity. This is useful when you want to replace all offsets + // instead of merging with existing ones. + optional bool ignore_previous_table_offsets = 2; } message PrepareLakeTableSnapshotResponse { @@ -1021,6 +1028,10 @@ message PbLakeTableSnapshotMetadata { required int64 snapshot_id = 2; required string tiered_bucket_offsets_file_path = 3; optional string readable_bucket_offsets_file_path = 4; + // Snapshot retention policy: The earliest snapshot ID to retain. + // 1. If set, the system will keep all snapshots in the range [earliest_snapshot_id_to_keep, current_snapshot_id]. + // 2. If not set, the system defaults to a "Single Snapshot Retention" policy, keeping only the snapshot specified in this request. + optional int64 earliest_snapshot_id_to_keep = 5; } message PbLakeTableSnapshotInfo { diff --git a/fluss-rpc/src/test/java/org/apache/fluss/rpc/TestingTabletGatewayService.java b/fluss-rpc/src/test/java/org/apache/fluss/rpc/TestingTabletGatewayService.java index 7db3654383..405e971141 100644 --- a/fluss-rpc/src/test/java/org/apache/fluss/rpc/TestingTabletGatewayService.java +++ b/fluss-rpc/src/test/java/org/apache/fluss/rpc/TestingTabletGatewayService.java @@ -31,10 +31,10 @@ import org.apache.fluss.rpc.messages.GetFileSystemSecurityTokenResponse; import org.apache.fluss.rpc.messages.GetKvSnapshotMetadataRequest; import org.apache.fluss.rpc.messages.GetKvSnapshotMetadataResponse; +import org.apache.fluss.rpc.messages.GetLakeSnapshotRequest; +import org.apache.fluss.rpc.messages.GetLakeSnapshotResponse; import org.apache.fluss.rpc.messages.GetLatestKvSnapshotsRequest; import org.apache.fluss.rpc.messages.GetLatestKvSnapshotsResponse; -import org.apache.fluss.rpc.messages.GetLatestLakeSnapshotRequest; -import org.apache.fluss.rpc.messages.GetLatestLakeSnapshotResponse; import org.apache.fluss.rpc.messages.GetTableInfoRequest; import org.apache.fluss.rpc.messages.GetTableInfoResponse; import org.apache.fluss.rpc.messages.GetTableSchemaRequest; @@ -234,8 +234,8 @@ public CompletableFuture listPartitionInfos( } @Override - public CompletableFuture getLatestLakeSnapshot( - GetLatestLakeSnapshotRequest request) { + public CompletableFuture getLakeSnapshot( + GetLakeSnapshotRequest request) { return null; } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/RpcServiceBase.java b/fluss-server/src/main/java/org/apache/fluss/server/RpcServiceBase.java index b12bb787c7..3436334f02 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/RpcServiceBase.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/RpcServiceBase.java @@ -52,10 +52,10 @@ import org.apache.fluss.rpc.messages.GetFileSystemSecurityTokenResponse; import org.apache.fluss.rpc.messages.GetKvSnapshotMetadataRequest; import org.apache.fluss.rpc.messages.GetKvSnapshotMetadataResponse; +import org.apache.fluss.rpc.messages.GetLakeSnapshotRequest; +import org.apache.fluss.rpc.messages.GetLakeSnapshotResponse; import org.apache.fluss.rpc.messages.GetLatestKvSnapshotsRequest; import org.apache.fluss.rpc.messages.GetLatestKvSnapshotsResponse; -import org.apache.fluss.rpc.messages.GetLatestLakeSnapshotRequest; -import org.apache.fluss.rpc.messages.GetLatestLakeSnapshotResponse; import org.apache.fluss.rpc.messages.GetTableInfoRequest; import org.apache.fluss.rpc.messages.GetTableInfoResponse; import org.apache.fluss.rpc.messages.GetTableSchemaRequest; @@ -115,8 +115,8 @@ import static org.apache.fluss.rpc.util.CommonRpcMessageUtils.toResolvedPartitionSpec; import static org.apache.fluss.security.acl.Resource.TABLE_SPLITTER; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.buildMetadataResponse; +import static org.apache.fluss.server.utils.ServerRpcMessageUtils.makeGetLakeSnapshotResponse; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.makeGetLatestKvSnapshotsResponse; -import static org.apache.fluss.server.utils.ServerRpcMessageUtils.makeGetLatestLakeSnapshotResponse; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.makeKvSnapshotMetadataResponse; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.makeListAclsResponse; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.toGetFileSystemSecurityTokenResponse; @@ -428,29 +428,47 @@ public CompletableFuture listPartitionInfos( } @Override - public CompletableFuture getLatestLakeSnapshot( - GetLatestLakeSnapshotRequest request) { + public CompletableFuture getLakeSnapshot( + GetLakeSnapshotRequest request) { // get table info TablePath tablePath = toTablePath(request.getTablePath()); TableInfo tableInfo = metadataManager.getTable(tablePath); // get table id long tableId = tableInfo.getTableId(); - CompletableFuture resultFuture = new CompletableFuture<>(); + CompletableFuture resultFuture = new CompletableFuture<>(); + boolean readableSnapshot = request.hasReadable() && request.isReadable(); ioExecutor.execute( () -> { Optional optLakeTableSnapshot; try { - optLakeTableSnapshot = zkClient.getLakeTableSnapshot(tableId); + if (readableSnapshot) { + optLakeTableSnapshot = + zkClient.getLatestReadableLakeTableSnapshot(tableId); + } else { + optLakeTableSnapshot = + request.hasSnapshotId() + ? zkClient.getLakeTableSnapshot( + tableId, request.getSnapshotId()) + : zkClient.getLakeTableSnapshot(tableId, null); + } if (!optLakeTableSnapshot.isPresent()) { + String errorMsg = + readableSnapshot + ? String.format( + "Lake table readable snapshot doesn't exist for table: %s, table id: %d", + tablePath, tableId) + : String.format( + "Lake table snapshot doesn't exist for table: %s, table id: %d", + tablePath, tableId); + if (request.hasSnapshotId()) { + errorMsg = errorMsg + ", snapshot id: " + request.getSnapshotId(); + } resultFuture.completeExceptionally( - new LakeTableSnapshotNotExistException( - String.format( - "Lake table snapshot not exist for table: %s, table id: %d", - tablePath, tableId))); + new LakeTableSnapshotNotExistException(errorMsg)); } else { LakeTableSnapshot lakeTableSnapshot = optLakeTableSnapshot.get(); resultFuture.complete( - makeGetLatestLakeSnapshotResponse(tableId, lakeTableSnapshot)); + makeGetLakeSnapshotResponse(tableId, lakeTableSnapshot)); } } catch (Exception e) { resultFuture.completeExceptionally( diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java index bd709262cc..0e2a284ee5 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java @@ -80,7 +80,7 @@ import org.apache.fluss.server.coordinator.statemachine.ReplicaStateMachine; import org.apache.fluss.server.coordinator.statemachine.TableBucketStateMachine; import org.apache.fluss.server.entity.AdjustIsrResultForBucket; -import org.apache.fluss.server.entity.CommitLakeTableSnapshotData; +import org.apache.fluss.server.entity.CommitLakeTableSnapshotsData; import org.apache.fluss.server.entity.CommitRemoteLogManifestData; import org.apache.fluss.server.entity.DeleteReplicaResultForBucket; import org.apache.fluss.server.entity.NotifyLeaderAndIsrResultForBucket; @@ -100,7 +100,6 @@ import org.apache.fluss.server.zk.data.TabletServerRegistration; import org.apache.fluss.server.zk.data.ZkData.PartitionIdsZNode; import org.apache.fluss.server.zk.data.ZkData.TableIdsZNode; -import org.apache.fluss.server.zk.data.lake.LakeTable; import org.apache.fluss.server.zk.data.lake.LakeTableHelper; import org.apache.fluss.server.zk.data.lake.LakeTableSnapshot; import org.apache.fluss.utils.types.Tuple2; @@ -1366,9 +1365,9 @@ private void processAccessContext(AccessContextEvent event) { private void tryProcessCommitLakeTableSnapshot( CommitLakeTableSnapshotEvent commitLakeTableSnapshotEvent, CompletableFuture callback) { - CommitLakeTableSnapshotData commitLakeTableSnapshotData = + CommitLakeTableSnapshotsData commitLakeTableSnapshotsData = commitLakeTableSnapshotEvent.getCommitLakeTableSnapshotData(); - if (commitLakeTableSnapshotData.getLakeTableSnapshotMetadatas().isEmpty()) { + if (commitLakeTableSnapshotsData.getLakeTableSnapshotMetadatas().isEmpty()) { handleCommitLakeTableSnapshotV1(commitLakeTableSnapshotEvent, callback); } else { handleCommitLakeTableSnapshotV2(commitLakeTableSnapshotEvent, callback); @@ -1379,10 +1378,10 @@ private void handleCommitLakeTableSnapshotV1( CommitLakeTableSnapshotEvent commitLakeTableSnapshotEvent, CompletableFuture callback) { // commit the lake table snapshot asynchronously - CommitLakeTableSnapshotData commitLakeTableSnapshotData = + CommitLakeTableSnapshotsData commitLakeTableSnapshotsData = commitLakeTableSnapshotEvent.getCommitLakeTableSnapshotData(); Map lakeTableSnapshots = - commitLakeTableSnapshotData.getLakeTableSnapshot(); + commitLakeTableSnapshotsData.getLakeTableSnapshot(); Map tablePathById = new HashMap<>(); for (Map.Entry lakeTableSnapshotEntry : lakeTableSnapshots.entrySet()) { @@ -1426,17 +1425,10 @@ private void handleCommitLakeTableSnapshotV1( } } - // remove failed tables - Map commitlakeTableSnapshots = - commitLakeTableSnapshotData.getLakeTableSnapshot(); - commitlakeTableSnapshots.keySet().removeAll(failedTableIds); - Map> tableMaxTieredTimestamps = - commitLakeTableSnapshotData.getTableMaxTieredTimestamps(); - tableMaxTieredTimestamps.keySet().removeAll(failedTableIds); - - coordinatorEventManager.put( - new NotifyLakeTableOffsetEvent( - commitlakeTableSnapshots, tableMaxTieredTimestamps)); + notifyLakeTableOffsets( + commitLakeTableSnapshotsData.getLakeTableSnapshot(), + commitLakeTableSnapshotsData.getTableMaxTieredTimestamps(), + failedTableIds); callback.complete(response); } catch (Exception e) { callback.completeExceptionally(e); @@ -1444,45 +1436,60 @@ private void handleCommitLakeTableSnapshotV1( }); } + private void notifyLakeTableOffsets( + Map committedLakeTableSnapshots, + Map> tableMaxTieredTimestamps, + Set failedTableIds) { + committedLakeTableSnapshots.keySet().removeAll(failedTableIds); + tableMaxTieredTimestamps.keySet().removeAll(failedTableIds); + if (!committedLakeTableSnapshots.isEmpty()) { + coordinatorEventManager.put( + new NotifyLakeTableOffsetEvent( + committedLakeTableSnapshots, tableMaxTieredTimestamps)); + } + } + private void handleCommitLakeTableSnapshotV2( CommitLakeTableSnapshotEvent commitLakeTableSnapshotEvent, CompletableFuture callback) { - CommitLakeTableSnapshotData commitLakeTableSnapshotData = + CommitLakeTableSnapshotsData commitLakeTableSnapshotsData = commitLakeTableSnapshotEvent.getCommitLakeTableSnapshotData(); - Map lakeSnapshotMetadatas = - commitLakeTableSnapshotData.getLakeTableSnapshotMetadatas(); ioExecutor.execute( () -> { try { CommitLakeTableSnapshotResponse response = new CommitLakeTableSnapshotResponse(); Set failedTableIds = new HashSet<>(); - for (Map.Entry - lakeSnapshotMetadataEntry : lakeSnapshotMetadatas.entrySet()) { + for (Map.Entry + entry : + commitLakeTableSnapshotsData + .getCommitLakeTableSnapshotByTableId() + .entrySet()) { PbCommitLakeTableSnapshotRespForTable tableResp = response.addTableResp(); - long tableId = lakeSnapshotMetadataEntry.getKey(); + long tableId = entry.getKey(); tableResp.setTableId(tableId); try { + CommitLakeTableSnapshotsData.CommitLakeTableSnapshot snapshot = + entry.getValue(); + if (snapshot.getLakeSnapshotMetadata() == null) { + throw new FlussRuntimeException( + "Lake snapshot metadata is null for table " + tableId); + } lakeTableHelper.registerLakeTableSnapshotV2( - tableId, lakeSnapshotMetadataEntry.getValue()); + tableId, + snapshot.getLakeSnapshotMetadata(), + snapshot.getEarliestSnapshotIDToKeep()); } catch (Exception e) { failedTableIds.add(tableId); ApiError error = ApiError.fromThrowable(e); tableResp.setError(error.error().code(), error.message()); } } - // remove failed tables - Map lakeTableSnapshots = - commitLakeTableSnapshotData.getLakeTableSnapshot(); - lakeTableSnapshots.keySet().removeAll(failedTableIds); - Map> tableMaxTieredTimestamps = - commitLakeTableSnapshotData.getTableMaxTieredTimestamps(); - tableMaxTieredTimestamps.keySet().removeAll(failedTableIds); - - coordinatorEventManager.put( - new NotifyLakeTableOffsetEvent( - lakeTableSnapshots, tableMaxTieredTimestamps)); + notifyLakeTableOffsets( + commitLakeTableSnapshotsData.getLakeTableSnapshot(), + commitLakeTableSnapshotsData.getTableMaxTieredTimestamps(), + failedTableIds); callback.complete(response); } catch (Exception e) { callback.completeExceptionally(e); 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 7fc53b96f8..2eaf51e378 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 @@ -652,6 +652,8 @@ public CompletableFuture dropAcls(DropAclsRequest request) { public CompletableFuture prepareLakeTableSnapshot( PrepareLakeTableSnapshotRequest request) { CompletableFuture future = new CompletableFuture<>(); + boolean ignorePreviousBucketOffsets = + request.hasIgnorePreviousTableOffsets() && request.isIgnorePreviousTableOffsets(); ioExecutor.submit( () -> { PrepareLakeTableSnapshotResponse response = @@ -664,14 +666,17 @@ public CompletableFuture prepareLakeTableSnaps long tableId = bucketOffsets.getTableId(); TableBucketOffsets tableBucketOffsets = toTableBucketOffsets(bucketOffsets); - // get previous lake tables - Optional optPreviousLakeTable = - zkClient.getLakeTable(tableId); - if (optPreviousLakeTable.isPresent()) { - // need to merge with previous lake table - tableBucketOffsets = - lakeTableHelper.mergeTableBucketOffsets( - optPreviousLakeTable.get(), tableBucketOffsets); + if (!ignorePreviousBucketOffsets) { + // get previous lake tables + Optional optPreviousLakeTable = + zkClient.getLakeTable(tableId); + if (optPreviousLakeTable.isPresent()) { + // need to merge with previous lake table + tableBucketOffsets = + lakeTableHelper.mergeTableBucketOffsets( + optPreviousLakeTable.get(), + tableBucketOffsets); + } } TablePath tablePath = toTablePath(bucketOffsets.getTablePath()); FsPath fsPath = diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/CommitLakeTableSnapshotEvent.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/CommitLakeTableSnapshotEvent.java index 8290e655e6..7eaca6f1fc 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/CommitLakeTableSnapshotEvent.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/CommitLakeTableSnapshotEvent.java @@ -18,26 +18,26 @@ package org.apache.fluss.server.coordinator.event; import org.apache.fluss.rpc.messages.CommitLakeTableSnapshotResponse; -import org.apache.fluss.server.entity.CommitLakeTableSnapshotData; +import org.apache.fluss.server.entity.CommitLakeTableSnapshotsData; import java.util.concurrent.CompletableFuture; /** An event for receiving the request of commit lakehouse data to coordinator server. */ public class CommitLakeTableSnapshotEvent implements CoordinatorEvent { - private final CommitLakeTableSnapshotData commitLakeTableSnapshotData; + private final CommitLakeTableSnapshotsData commitLakeTableSnapshotsData; private final CompletableFuture respCallback; public CommitLakeTableSnapshotEvent( - CommitLakeTableSnapshotData commitLakeTableSnapshotData, + CommitLakeTableSnapshotsData commitLakeTableSnapshotsData, CompletableFuture respCallback) { - this.commitLakeTableSnapshotData = commitLakeTableSnapshotData; + this.commitLakeTableSnapshotsData = commitLakeTableSnapshotsData; this.respCallback = respCallback; } - public CommitLakeTableSnapshotData getCommitLakeTableSnapshotData() { - return commitLakeTableSnapshotData; + public CommitLakeTableSnapshotsData getCommitLakeTableSnapshotData() { + return commitLakeTableSnapshotsData; } public CompletableFuture getRespCallback() { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/entity/CommitLakeTableSnapshotData.java b/fluss-server/src/main/java/org/apache/fluss/server/entity/CommitLakeTableSnapshotData.java deleted file mode 100644 index 3ff7e3e2b3..0000000000 --- a/fluss-server/src/main/java/org/apache/fluss/server/entity/CommitLakeTableSnapshotData.java +++ /dev/null @@ -1,102 +0,0 @@ -/* - * 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.server.entity; - -import org.apache.fluss.metadata.TableBucket; -import org.apache.fluss.rpc.messages.CommitLakeTableSnapshotRequest; -import org.apache.fluss.server.zk.data.lake.LakeTable; -import org.apache.fluss.server.zk.data.lake.LakeTableSnapshot; - -import java.util.Map; -import java.util.Objects; - -/** The data for request {@link CommitLakeTableSnapshotRequest}. */ -public class CommitLakeTableSnapshotData { - - /** - * Since 0.9, this field is only used to allow the coordinator to send requests to tablet - * servers, enabling tablet servers to report metrics about synchronized log end offsets. In the - * future, we plan to have the tiering service directly report metrics, and this field will be - * removed. - */ - private final Map lakeTableSnapshots; - - /** - * Since 0.9, this field is only used to allow the coordinator to send requests to tablet - * servers, enabling tablet servers to report metrics about max tiered timestamps. In the - * future, we plan to have the tiering service directly report metrics, and this field will be - * removed. - */ - private final Map> tableMaxTieredTimestamps; - - // the following field only non-empty since 0.9 - private final Map lakeTableSnapshotMetadatas; - - public CommitLakeTableSnapshotData( - Map lakeTableSnapshots, - Map> tableMaxTieredTimestamps, - Map lakeTableSnapshotMetadatas) { - this.lakeTableSnapshots = lakeTableSnapshots; - this.tableMaxTieredTimestamps = tableMaxTieredTimestamps; - this.lakeTableSnapshotMetadatas = lakeTableSnapshotMetadatas; - } - - public Map getLakeTableSnapshot() { - return lakeTableSnapshots; - } - - public Map> getTableMaxTieredTimestamps() { - return tableMaxTieredTimestamps; - } - - public Map getLakeTableSnapshotMetadatas() { - return lakeTableSnapshotMetadatas; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - CommitLakeTableSnapshotData that = (CommitLakeTableSnapshotData) o; - return Objects.equals(lakeTableSnapshots, that.lakeTableSnapshots) - && Objects.equals(tableMaxTieredTimestamps, that.tableMaxTieredTimestamps) - && Objects.equals(lakeTableSnapshotMetadatas, that.lakeTableSnapshotMetadatas); - } - - @Override - public int hashCode() { - return Objects.hash( - lakeTableSnapshots, tableMaxTieredTimestamps, lakeTableSnapshotMetadatas); - } - - @Override - public String toString() { - return "CommitLakeTableSnapshotData{" - + "lakeTableSnapshots=" - + lakeTableSnapshots - + ", tableMaxTieredTimestamps=" - + tableMaxTieredTimestamps - + ", lakeTableSnapshotMetadatas=" - + lakeTableSnapshotMetadatas - + '}'; - } -} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/entity/CommitLakeTableSnapshotsData.java b/fluss-server/src/main/java/org/apache/fluss/server/entity/CommitLakeTableSnapshotsData.java new file mode 100644 index 0000000000..494585d345 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/entity/CommitLakeTableSnapshotsData.java @@ -0,0 +1,178 @@ +/* + * 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.server.entity; + +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.rpc.messages.CommitLakeTableSnapshotRequest; +import org.apache.fluss.server.zk.data.lake.LakeTable; +import org.apache.fluss.server.zk.data.lake.LakeTableSnapshot; + +import javax.annotation.Nullable; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.stream.Collectors; + +/** The data for request {@link CommitLakeTableSnapshotRequest}. */ +public class CommitLakeTableSnapshotsData { + + private final Map commitLakeTableSnapshotByTableId; + + private CommitLakeTableSnapshotsData( + Map commitLakeTableSnapshotByTableId) { + this.commitLakeTableSnapshotByTableId = + Collections.unmodifiableMap(commitLakeTableSnapshotByTableId); + } + + public static Builder builder() { + return new Builder(); + } + + /** Builder for {@link CommitLakeTableSnapshotsData}. */ + public static class Builder { + private final Map snapshotMap = new HashMap<>(); + + /** + * Add a table snapshot entry. + * + * @param tableId the table ID + * @param lakeTableSnapshot the lake table snapshot (for V1 format, can be null in V2) + * @param tableMaxTieredTimestamps the max tiered timestamps for metrics (can be null when + * tiered timestamps is unknown) + * @param lakeSnapshotMetadata the lake snapshot metadata (for V2 format, can be null in v1) + * @param earliestSnapshotIDToKeep the earliest snapshot ID to keep (can be null in v1 or + * not to keep previous snapshot) + */ + public void addTableSnapshot( + long tableId, + @Nullable LakeTableSnapshot lakeTableSnapshot, + @Nullable Map tableMaxTieredTimestamps, + @Nullable LakeTable.LakeSnapshotMetadata lakeSnapshotMetadata, + @Nullable Long earliestSnapshotIDToKeep) { + snapshotMap.put( + tableId, + new CommitLakeTableSnapshot( + lakeTableSnapshot, + tableMaxTieredTimestamps != null + ? tableMaxTieredTimestamps + : Collections.emptyMap(), + lakeSnapshotMetadata, + earliestSnapshotIDToKeep)); + } + + /** + * Build the {@link CommitLakeTableSnapshotsData} instance. + * + * @return the built instance + */ + public CommitLakeTableSnapshotsData build() { + return new CommitLakeTableSnapshotsData(snapshotMap); + } + } + + public Map getCommitLakeTableSnapshotByTableId() { + return commitLakeTableSnapshotByTableId; + } + + // Backward compatibility methods + public Map getLakeTableSnapshot() { + return commitLakeTableSnapshotByTableId.entrySet().stream() + .filter(entry -> entry.getValue().lakeTableSnapshot != null) + .collect( + Collectors.toMap( + Map.Entry::getKey, entry -> entry.getValue().lakeTableSnapshot)); + } + + public Map> getTableMaxTieredTimestamps() { + return commitLakeTableSnapshotByTableId.entrySet().stream() + .filter( + entry -> + entry.getValue().tableMaxTieredTimestamps != null + && !entry.getValue().tableMaxTieredTimestamps.isEmpty()) + .collect( + Collectors.toMap( + Map.Entry::getKey, + entry -> entry.getValue().tableMaxTieredTimestamps)); + } + + public Map getLakeTableSnapshotMetadatas() { + return commitLakeTableSnapshotByTableId.entrySet().stream() + .filter(entry -> entry.getValue().lakeSnapshotMetadata != null) + .collect( + Collectors.toMap( + Map.Entry::getKey, entry -> entry.getValue().lakeSnapshotMetadata)); + } + + /** + * Data container for a single table's lake snapshot commit. + * + *

This class bridges legacy V1 reporting (used for metrics) and the V2 snapshot metadata + * persistence required for the tiering service. + */ + public static class CommitLakeTableSnapshot { + + /** + * Since 0.9, this field is only used to allow the coordinator to send requests to tablet + * servers, enabling tablet servers to report metrics about synchronized log end offsets. In + * the future, we plan to have the tiering service directly report metrics, and this field + * will be removed. + */ + @Nullable private final LakeTableSnapshot lakeTableSnapshot; + + /** + * Since 0.9, this field is only used to allow the coordinator to send requests to tablet + * servers, enabling tablet servers to report metrics about max tiered timestamps. In the + * future, we plan to have the tiering service directly report metrics, and this field will + * be removed. + */ + @Nullable private final Map tableMaxTieredTimestamps; + + // the following field only non-empty since 0.9 + @Nullable private final LakeTable.LakeSnapshotMetadata lakeSnapshotMetadata; + + // The earliest snapshot ID to keep for DV tables. Null for non-DV tables. + @Nullable private final Long earliestSnapshotIDToKeep; + + public CommitLakeTableSnapshot( + @Nullable LakeTableSnapshot lakeTableSnapshot, + @Nullable Map tableMaxTieredTimestamps, + @Nullable LakeTable.LakeSnapshotMetadata lakeSnapshotMetadata, + @Nullable Long earliestSnapshotIDToKeep) { + this.lakeTableSnapshot = lakeTableSnapshot; + this.tableMaxTieredTimestamps = tableMaxTieredTimestamps; + this.lakeSnapshotMetadata = lakeSnapshotMetadata; + this.earliestSnapshotIDToKeep = earliestSnapshotIDToKeep; + } + + @Nullable + public LakeTableSnapshot getLakeTableSnapshot() { + return lakeTableSnapshot; + } + + @Nullable + public LakeTable.LakeSnapshotMetadata getLakeSnapshotMetadata() { + return lakeSnapshotMetadata; + } + + @Nullable + public Long getEarliestSnapshotIDToKeep() { + return earliestSnapshotIDToKeep; + } + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java index d9350257dc..7e593a4e38 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java @@ -833,7 +833,7 @@ private void makeLeaders( private void updateWithLakeTableSnapshot(Replica replica) throws Exception { TableBucket tb = replica.getTableBucket(); Optional optLakeTableSnapshot = - zkClient.getLakeTableSnapshot(replica.getTableBucket().getTableId()); + zkClient.getLakeTableSnapshot(replica.getTableBucket().getTableId(), null); if (optLakeTableSnapshot.isPresent()) { LakeTableSnapshot lakeTableSnapshot = optLakeTableSnapshot.get(); long snapshotId = optLakeTableSnapshot.get().getSnapshotId(); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java b/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java index 8ee46545b0..8ec72c2ea6 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java @@ -63,8 +63,8 @@ import org.apache.fluss.rpc.messages.FetchLogResponse; import org.apache.fluss.rpc.messages.GetFileSystemSecurityTokenResponse; import org.apache.fluss.rpc.messages.GetKvSnapshotMetadataResponse; +import org.apache.fluss.rpc.messages.GetLakeSnapshotResponse; import org.apache.fluss.rpc.messages.GetLatestKvSnapshotsResponse; -import org.apache.fluss.rpc.messages.GetLatestLakeSnapshotResponse; import org.apache.fluss.rpc.messages.InitWriterResponse; import org.apache.fluss.rpc.messages.LakeTieringHeartbeatResponse; import org.apache.fluss.rpc.messages.LimitScanResponse; @@ -147,7 +147,7 @@ import org.apache.fluss.server.authorizer.AclCreateResult; import org.apache.fluss.server.authorizer.AclDeleteResult; import org.apache.fluss.server.entity.AdjustIsrResultForBucket; -import org.apache.fluss.server.entity.CommitLakeTableSnapshotData; +import org.apache.fluss.server.entity.CommitLakeTableSnapshotsData; import org.apache.fluss.server.entity.CommitRemoteLogManifestData; import org.apache.fluss.server.entity.FetchReqInfo; import org.apache.fluss.server.entity.LakeBucketOffset; @@ -1547,7 +1547,7 @@ public static PbPartitionSpec makePbPartitionSpec(ResolvedPartitionSpec spec) { return pbPartitionSpec; } - public static CommitLakeTableSnapshotData getCommitLakeTableSnapshotData( + public static CommitLakeTableSnapshotsData getCommitLakeTableSnapshotData( CommitLakeTableSnapshotRequest request) { // handle rpc before 0.9 Map lakeTableInfoByTableId = new HashMap<>(); @@ -1581,12 +1581,25 @@ public static CommitLakeTableSnapshotData getCommitLakeTableSnapshotData( tableBucketsMaxTimestamp.put(tableId, bucketLogMaxTimestamp); } - // handle rpc since 0.9 - Map lakeSnapshotMetadatas = new HashMap<>(); + // Build using Builder pattern for cleaner code + CommitLakeTableSnapshotsData.Builder builder = CommitLakeTableSnapshotsData.builder(); + + // Add V1 format snapshots (legacy) + for (Map.Entry entry : lakeTableInfoByTableId.entrySet()) { + long tableId = entry.getKey(); + builder.addTableSnapshot( + tableId, + entry.getValue(), + tableBucketsMaxTimestamp.get(tableId), + null, // no metadata for V1 + null); // no earliestSnapshotIDToKeep for V1 + } + + // Add V2 format snapshots (current) for (PbLakeTableSnapshotMetadata pbLakeTableSnapshotMetadata : request.getLakeTableSnapshotMetadatasList()) { - lakeSnapshotMetadatas.put( - pbLakeTableSnapshotMetadata.getTableId(), + long tableId = pbLakeTableSnapshotMetadata.getTableId(); + LakeTable.LakeSnapshotMetadata lakeSnapshotMetadata = new LakeTable.LakeSnapshotMetadata( pbLakeTableSnapshotMetadata.getSnapshotId(), new FsPath( @@ -1595,10 +1608,22 @@ public static CommitLakeTableSnapshotData getCommitLakeTableSnapshotData( ? new FsPath( pbLakeTableSnapshotMetadata .getReadableBucketOffsetsFilePath()) - : null)); + : null); + Long earliestSnapshotIDToKeep = + pbLakeTableSnapshotMetadata.hasEarliestSnapshotIdToKeep() + ? pbLakeTableSnapshotMetadata.getEarliestSnapshotIdToKeep() + : null; + + // If this table already exists in builder (from V1), update it; otherwise add new + builder.addTableSnapshot( + tableId, + lakeTableInfoByTableId.get(tableId), // may be null for V2-only + tableBucketsMaxTimestamp.get(tableId), // may be null + lakeSnapshotMetadata, + earliestSnapshotIDToKeep); } - return new CommitLakeTableSnapshotData( - lakeTableInfoByTableId, tableBucketsMaxTimestamp, lakeSnapshotMetadatas); + + return builder.build(); } public static TableBucketOffsets toTableBucketOffsets(PbTableOffsets pbTableOffsets) { @@ -1667,10 +1692,9 @@ public static NotifyLakeTableOffsetData getNotifyLakeTableOffset( notifyLakeTableOffsetRequest.getCoordinatorEpoch(), lakeBucketOffsetMap); } - public static GetLatestLakeSnapshotResponse makeGetLatestLakeSnapshotResponse( + public static GetLakeSnapshotResponse makeGetLakeSnapshotResponse( long tableId, LakeTableSnapshot lakeTableSnapshot) { - GetLatestLakeSnapshotResponse getLakeTableSnapshotResponse = - new GetLatestLakeSnapshotResponse(); + GetLakeSnapshotResponse getLakeTableSnapshotResponse = new GetLakeSnapshotResponse(); getLakeTableSnapshotResponse.setTableId(tableId); getLakeTableSnapshotResponse.setSnapshotId(lakeTableSnapshot.getSnapshotId()); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java index 40bffe8171..a5068aa658 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java @@ -1058,14 +1058,42 @@ public Optional getLakeTable(long tableId) throws Exception { * Gets the {@link LakeTableSnapshot} for the given table ID. * * @param tableId the table ID + * @param snapshotId the snapshot id for the snapshot to get, null means to get latest snapshot + * id * @return an Optional containing the LakeTableSnapshot if the table exists, empty otherwise * @throws Exception if the operation fails */ - public Optional getLakeTableSnapshot(long tableId) throws Exception { + public Optional getLakeTableSnapshot(long tableId, @Nullable Long snapshotId) + throws Exception { Optional optLakeTable = getLakeTable(tableId); if (optLakeTable.isPresent()) { // always get the latest snapshot - return Optional.of(optLakeTable.get().getOrReadLatestTableSnapshot()); + if (snapshotId == null) { + return Optional.ofNullable(optLakeTable.get().getOrReadLatestTableSnapshot()); + } else { + return Optional.ofNullable(optLakeTable.get().getOrReadTableSnapshot(snapshotId)); + } + + } else { + return Optional.empty(); + } + } + + /** + * Gets the latest readable {@link LakeTableSnapshot} for the given table ID. + * + * @param tableId the table ID + * @return an Optional containing the latest readable LakeTableSnapshot if found, empty + * otherwise + * @throws Exception if the operation fails + */ + public Optional getLatestReadableLakeTableSnapshot(long tableId) + throws Exception { + Optional optLakeTable = getLakeTable(tableId); + if (optLakeTable.isPresent()) { + LakeTableSnapshot readableSnapshot = + optLakeTable.get().getOrReadLatestReadableTableSnapshot(); + return Optional.ofNullable(readableSnapshot); } else { return Optional.empty(); } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTable.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTable.java index c5ae253ecc..10e7f97486 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTable.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTable.java @@ -97,7 +97,19 @@ private LakeTable( @Nullable public LakeSnapshotMetadata getLatestLakeSnapshotMetadata() { if (lakeSnapshotMetadatas != null && !lakeSnapshotMetadatas.isEmpty()) { - return lakeSnapshotMetadatas.get(0); + return lakeSnapshotMetadatas.get(lakeSnapshotMetadatas.size() - 1); + } + return null; + } + + @Nullable + private LakeSnapshotMetadata getLakeSnapshotMetadata(long snapshotId) { + if (lakeSnapshotMetadatas != null) { + for (LakeSnapshotMetadata lakeSnapshotMetadata : lakeSnapshotMetadatas) { + if (lakeSnapshotMetadata.snapshotId == snapshotId) { + return lakeSnapshotMetadata; + } + } } return null; } @@ -124,15 +136,68 @@ public LakeTableSnapshot getOrReadLatestTableSnapshot() throws IOException { if (lakeTableSnapshot != null) { return lakeTableSnapshot; } - LakeSnapshotMetadata lakeSnapshotMetadata = getLatestLakeSnapshotMetadata(); + return toLakeTableSnapshot(getLatestLakeSnapshotMetadata()); + } + + /** + * Gets the table snapshot for the given ID. * + * + *

Returns the cached snapshot if the ID matches; otherwise, reads and reconstructs it from + * lake metadata. + * + * @return the {@link LakeTableSnapshot}, or null if not found. + */ + @Nullable + public LakeTableSnapshot getOrReadTableSnapshot(long snapshotId) throws IOException { + // only happen in v1 + if (lakeTableSnapshot != null && lakeTableSnapshot.getSnapshotId() == snapshotId) { + return lakeTableSnapshot; + } + LakeSnapshotMetadata lakeSnapshotMetadata = getLakeSnapshotMetadata(snapshotId); + if (lakeSnapshotMetadata == null) { + return null; + } + return toLakeTableSnapshot(lakeSnapshotMetadata); + } + + /** + * Gets the latest readable table snapshot. * + * + *

Searches metadata in reverse order to find the newest snapshot with valid readable + * offsets. + * + * @return the latest readable {@link LakeTableSnapshot}, or null if none exist. + */ + @Nullable + public LakeTableSnapshot getOrReadLatestReadableTableSnapshot() throws IOException { + // only happen in v1 + if (lakeTableSnapshot != null) { + return lakeTableSnapshot; + } + for (int i = checkNotNull(lakeSnapshotMetadatas).size() - 1; i >= 0; i--) { + LakeSnapshotMetadata snapshotMetadata = lakeSnapshotMetadatas.get(i); + if (snapshotMetadata.readableOffsetsFilePath != null) { + return toLakeTableSnapshot( + snapshotMetadata.snapshotId, snapshotMetadata.readableOffsetsFilePath); + } + } + return null; + } + + private LakeTableSnapshot toLakeTableSnapshot(LakeSnapshotMetadata lakeSnapshotMetadata) + throws IOException { FsPath tieredOffsetsFilePath = checkNotNull(lakeSnapshotMetadata).tieredOffsetsFilePath; - FSDataInputStream inputStream = - tieredOffsetsFilePath.getFileSystem().open(tieredOffsetsFilePath); + return toLakeTableSnapshot(lakeSnapshotMetadata.snapshotId, tieredOffsetsFilePath); + } + + private LakeTableSnapshot toLakeTableSnapshot(long snapshotId, FsPath offsetFilePath) + throws IOException { + FSDataInputStream inputStream = offsetFilePath.getFileSystem().open(offsetFilePath); try (ByteArrayOutputStream outputStream = new ByteArrayOutputStream()) { IOUtils.copyBytes(inputStream, outputStream, true); Map logOffsets = TableBucketOffsets.fromJsonBytes(outputStream.toByteArray()).getOffsets(); - return new LakeTableSnapshot(lakeSnapshotMetadata.snapshotId, logOffsets); + return new LakeTableSnapshot(snapshotId, logOffsets); } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableHelper.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableHelper.java index e3b46d04e3..8eb851ab36 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableHelper.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableHelper.java @@ -26,7 +26,12 @@ import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.utils.FlussPaths; import org.apache.fluss.utils.json.TableBucketOffsets; +import org.apache.fluss.utils.types.Tuple2; +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -71,28 +76,93 @@ public void registerLakeTableSnapshotV1(long tableId, LakeTableSnapshot lakeTabl public void registerLakeTableSnapshotV2( long tableId, LakeTable.LakeSnapshotMetadata lakeSnapshotMetadata) throws Exception { - Optional optPreviousLakeTable = zkClient.getLakeTable(tableId); - List previousLakeSnapshotMetadatas = null; - if (optPreviousLakeTable.isPresent()) { - previousLakeSnapshotMetadatas = optPreviousLakeTable.get().getLakeSnapshotMetadatas(); - } - LakeTable lakeTable = new LakeTable(lakeSnapshotMetadata); + registerLakeTableSnapshotV2(tableId, lakeSnapshotMetadata, null); + } + + /** + * Register a lake table snapshot and clean up old snapshots based on the table type. + * + * @param tableId the table ID + * @param lakeSnapshotMetadata the new snapshot metadata to register + * @param earliestSnapshotIDToKeep the earliest snapshot ID to keep. If null, only the latest + * snapshot will be kept. + * @throws Exception if the operation fails + */ + public void registerLakeTableSnapshotV2( + long tableId, + LakeTable.LakeSnapshotMetadata lakeSnapshotMetadata, + @Nullable Long earliestSnapshotIDToKeep) + throws Exception { + Optional optPreviousTable = zkClient.getLakeTable(tableId); + List previousMetadatas = + optPreviousTable + .map(LakeTable::getLakeSnapshotMetadatas) + .orElse(Collections.emptyList()); + + // Determine which snapshots to keep and which to discard (but don't discard yet) + + Tuple2, List> result = + determineSnapshotsToKeepAndDiscard( + previousMetadatas, lakeSnapshotMetadata, earliestSnapshotIDToKeep); + + List keptSnapshots = result.f0; + List snapshotsToDiscard = result.f1; + + LakeTable lakeTable = new LakeTable(keptSnapshots); try { - zkClient.upsertLakeTable(tableId, lakeTable, optPreviousLakeTable.isPresent()); + // First, upsert to ZK. Only after success, we discard old snapshots. + zkClient.upsertLakeTable(tableId, lakeTable, optPreviousTable.isPresent()); } catch (Exception e) { LOG.warn("Failed to upsert lake table snapshot to zk.", e); throw e; } - // currently, we keep only one lake snapshot metadata in zk, - // todo: in solve paimon dv union read issue #2121, we'll keep multiple lake snapshot - // metadata - // discard previous lake snapshot metadata - if (previousLakeSnapshotMetadatas != null) { - previousLakeSnapshotMetadatas.forEach(LakeTable.LakeSnapshotMetadata::discard); + // After successful upsert, discard snapshots + for (LakeTable.LakeSnapshotMetadata metadata : snapshotsToDiscard) { + metadata.discard(); } } + /** + * Determines which snapshots should be retained or discarded based on the timeline according to + * {@code earliestSnapshotIDToKeep}. + */ + private Tuple2, List> + determineSnapshotsToKeepAndDiscard( + List previousMetadatas, + LakeTable.LakeSnapshotMetadata newSnapshotMetadata, + @Nullable Long earliestSnapshotIDToKeep) { + // Scenario 1: No retention boundary or no history -> Keep only latest + if (earliestSnapshotIDToKeep == null || previousMetadatas.isEmpty()) { + return new Tuple2<>( + Collections.singletonList(newSnapshotMetadata), + new ArrayList<>(previousMetadatas)); + } + + // Scenario 2: Find the split point based on position (for non-monotonic IDs, like iceberg) + int splitIndex = -1; + for (int i = 0; i < previousMetadatas.size(); i++) { + if (previousMetadatas.get(i).getSnapshotId() == earliestSnapshotIDToKeep) { + splitIndex = i; + break; + } + } + + // If ID not found, play safe: keep everything + if (splitIndex == -1) { + List kept = new ArrayList<>(previousMetadatas); + kept.add(newSnapshotMetadata); + return new Tuple2<>(kept, Collections.emptyList()); + } + + List toDiscard = + new ArrayList<>(previousMetadatas.subList(0, splitIndex)); + List toKeep = + new ArrayList<>(previousMetadatas.subList(splitIndex, previousMetadatas.size())); + toKeep.add(newSnapshotMetadata); + return new Tuple2<>(toKeep, toDiscard); + } + public TableBucketOffsets mergeTableBucketOffsets( LakeTable previousLakeTable, TableBucketOffsets newTableBucketOffsets) throws Exception { diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TestCoordinatorGateway.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TestCoordinatorGateway.java index 06d15d95ed..e5c35e0f45 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TestCoordinatorGateway.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TestCoordinatorGateway.java @@ -68,10 +68,10 @@ import org.apache.fluss.rpc.messages.GetFileSystemSecurityTokenResponse; import org.apache.fluss.rpc.messages.GetKvSnapshotMetadataRequest; import org.apache.fluss.rpc.messages.GetKvSnapshotMetadataResponse; +import org.apache.fluss.rpc.messages.GetLakeSnapshotRequest; +import org.apache.fluss.rpc.messages.GetLakeSnapshotResponse; import org.apache.fluss.rpc.messages.GetLatestKvSnapshotsRequest; import org.apache.fluss.rpc.messages.GetLatestKvSnapshotsResponse; -import org.apache.fluss.rpc.messages.GetLatestLakeSnapshotRequest; -import org.apache.fluss.rpc.messages.GetLatestLakeSnapshotResponse; import org.apache.fluss.rpc.messages.GetTableInfoRequest; import org.apache.fluss.rpc.messages.GetTableInfoResponse; import org.apache.fluss.rpc.messages.GetTableSchemaRequest; @@ -181,12 +181,6 @@ public CompletableFuture dropPartition(DropPartitionReque throw new UnsupportedOperationException(); } - @Override - public CompletableFuture getLatestLakeSnapshot( - GetLatestLakeSnapshotRequest request) { - throw new UnsupportedOperationException(); - } - @Override public CompletableFuture listDatabases(ListDatabasesRequest request) { throw new UnsupportedOperationException(); @@ -252,6 +246,12 @@ public CompletableFuture listPartitionInfos( return null; } + @Override + public CompletableFuture getLakeSnapshot( + GetLakeSnapshotRequest request) { + return null; + } + @Override public CompletableFuture adjustIsr(AdjustIsrRequest request) { if (networkIssueEnable) { diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/CommitLakeTableSnapshotITCase.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/CommitLakeTableSnapshotITCase.java index 2810ce9d24..e5788f32b6 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/CommitLakeTableSnapshotITCase.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/CommitLakeTableSnapshotITCase.java @@ -132,7 +132,7 @@ void testCommitDataLakeData() throws Exception { } private void checkLakeTableDataInZk(long tableId, LakeTableSnapshot expected) throws Exception { - LakeTableSnapshot lakeTableSnapshot = zkClient.getLakeTableSnapshot(tableId).get(); + LakeTableSnapshot lakeTableSnapshot = zkClient.getLakeTableSnapshot(tableId, null).get(); assertThat(lakeTableSnapshot).isEqualTo(expected); } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/tablet/TestTabletServerGateway.java b/fluss-server/src/test/java/org/apache/fluss/server/tablet/TestTabletServerGateway.java index 500d197fcf..617026fa26 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/tablet/TestTabletServerGateway.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/tablet/TestTabletServerGateway.java @@ -36,10 +36,10 @@ import org.apache.fluss.rpc.messages.GetFileSystemSecurityTokenResponse; import org.apache.fluss.rpc.messages.GetKvSnapshotMetadataRequest; import org.apache.fluss.rpc.messages.GetKvSnapshotMetadataResponse; +import org.apache.fluss.rpc.messages.GetLakeSnapshotRequest; +import org.apache.fluss.rpc.messages.GetLakeSnapshotResponse; import org.apache.fluss.rpc.messages.GetLatestKvSnapshotsRequest; import org.apache.fluss.rpc.messages.GetLatestKvSnapshotsResponse; -import org.apache.fluss.rpc.messages.GetLatestLakeSnapshotRequest; -import org.apache.fluss.rpc.messages.GetLatestLakeSnapshotResponse; import org.apache.fluss.rpc.messages.GetTableInfoRequest; import org.apache.fluss.rpc.messages.GetTableInfoResponse; import org.apache.fluss.rpc.messages.GetTableSchemaRequest; @@ -157,9 +157,9 @@ public CompletableFuture listPartitionInfos( } @Override - public CompletableFuture getLatestLakeSnapshot( - GetLatestLakeSnapshotRequest request) { - throw new UnsupportedOperationException(); + public CompletableFuture getLakeSnapshot( + GetLakeSnapshotRequest request) { + return null; } @Override diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableHelperTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableHelperTest.java index f55c1b94ad..bca0008466 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableHelperTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableHelperTest.java @@ -22,6 +22,7 @@ import org.apache.fluss.config.Configuration; import org.apache.fluss.fs.FileSystem; import org.apache.fluss.fs.FsPath; +import org.apache.fluss.fs.local.LocalFileSystem; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableDescriptor; import org.apache.fluss.metadata.TablePath; @@ -43,6 +44,7 @@ import java.nio.file.Path; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Optional; @@ -88,17 +90,7 @@ void testRegisterLakeTableSnapshotCompatibility(@TempDir Path tempDir) throws Ex // first create a table long tableId = 1; TablePath tablePath = TablePath.of("test_db", "test_table"); - TableRegistration tableReg = - new TableRegistration( - tableId, - "test table", - Collections.emptyList(), - new TableDescriptor.TableDistribution( - 1, Collections.singletonList("a")), - Collections.emptyMap(), - Collections.emptyMap(), - System.currentTimeMillis(), - System.currentTimeMillis()); + TableRegistration tableReg = createTableReg(tableId); zookeeperClient.registerTable(tablePath, tableReg); // Create a legacy version 1 LakeTableSnapshot (full data in ZK) @@ -148,7 +140,7 @@ void testRegisterLakeTableSnapshotCompatibility(@TempDir Path tempDir) throws Ex assertThat(fileSystem.exists(snapshot2FileHandle)).isTrue(); Optional optMergedSnapshot = - zooKeeperClient.getLakeTableSnapshot(tableId); + zooKeeperClient.getLakeTableSnapshot(tableId, null); assertThat(optMergedSnapshot).isPresent(); LakeTableSnapshot mergedSnapshot = optMergedSnapshot.get(); @@ -172,4 +164,105 @@ void testRegisterLakeTableSnapshotCompatibility(@TempDir Path tempDir) throws Ex assertThat(fileSystem.exists(snapshot2FileHandle)).isFalse(); } } + + @Test + void testRegisterLakeTableSnapshotWithRetention(@TempDir Path tempDir) throws Exception { + LakeTableHelper lakeTableHelper = new LakeTableHelper(zookeeperClient, tempDir.toString()); + long tableId = 1L; + TablePath tablePath = TablePath.of("test_db", "retention_test"); + + // --- Setup: Register table and initialize filesystem --- + zookeeperClient.registerTable(tablePath, createTableReg(tableId)); + FileSystem fs = LocalFileSystem.getSharedInstance(); + + // 1. Create Snapshot 1 (ID=1) + FsPath path1 = storeOffsetFile(lakeTableHelper, tablePath, tableId, 100L); + LakeTable.LakeSnapshotMetadata meta1 = new LakeTable.LakeSnapshotMetadata(1L, path1, path1); + lakeTableHelper.registerLakeTableSnapshotV2(tableId, meta1, -1L); // -1: Keep all history + + // 2. Create Snapshot 2 (ID=2) + FsPath path2 = storeOffsetFile(lakeTableHelper, tablePath, tableId, 200L); + LakeTable.LakeSnapshotMetadata meta2 = new LakeTable.LakeSnapshotMetadata(2L, path2, path2); + lakeTableHelper.registerLakeTableSnapshotV2(tableId, meta2, -1L); // -1: Keep all history + + List metadatasAfterStep2 = + zookeeperClient.getLakeTable(tableId).get().getLakeSnapshotMetadatas(); + assertThat(metadatasAfterStep2).hasSize(2); + assertThat(metadatasAfterStep2) + .extracting(LakeTable.LakeSnapshotMetadata::getSnapshotId) + .containsExactly(1L, 2L); + + // --- Scenario A: earliestSnapshotIDToKeep = null (Aggressive Cleanup) --- + // Expected behavior: Only the latest snapshot is retained; all previous ones are discarded. + FsPath path3 = storeOffsetFile(lakeTableHelper, tablePath, tableId, 300L); + LakeTable.LakeSnapshotMetadata meta3 = new LakeTable.LakeSnapshotMetadata(3L, path3, path3); + + lakeTableHelper.registerLakeTableSnapshotV2(tableId, meta3, null); + + // Verify physical files: 1 and 2 should be deleted, 3 must exist. + assertThat(fs.exists(path1)).isFalse(); + assertThat(fs.exists(path2)).isFalse(); + assertThat(fs.exists(path3)).isTrue(); + // Verify metadata: Only 1 entry should remain in ZK. + assertThat(zookeeperClient.getLakeTable(tableId).get().getLakeSnapshotMetadatas()) + .hasSize(1); + + // --- Scenario B: earliestSnapshotIDToKeep = -1 (Infinite Retention) --- + // Expected behavior: No previous snapshots are discarded regardless of history size. + FsPath path4 = storeOffsetFile(lakeTableHelper, tablePath, tableId, 400L); + LakeTable.LakeSnapshotMetadata meta4 = new LakeTable.LakeSnapshotMetadata(4L, path4, path4); + + lakeTableHelper.registerLakeTableSnapshotV2(tableId, meta4, -1L); + + // Verify both snapshots 3 and 4 are preserved. + assertThat(fs.exists(path3)).isTrue(); + assertThat(fs.exists(path4)).isTrue(); + assertThat(zookeeperClient.getLakeTable(tableId).get().getLakeSnapshotMetadatas()) + .hasSize(2); + + // --- Scenario C: earliestSnapshotIDToKeep = Specific ID (Positional Slicing) --- + // Setup: Current history [3, 4], adding [5, 6]. + FsPath path5 = storeOffsetFile(lakeTableHelper, tablePath, tableId, 500L); + LakeTable.LakeSnapshotMetadata meta5 = new LakeTable.LakeSnapshotMetadata(5L, path5, path5); + lakeTableHelper.registerLakeTableSnapshotV2(tableId, meta5, -1L); + + FsPath path6 = storeOffsetFile(lakeTableHelper, tablePath, tableId, 600L); + LakeTable.LakeSnapshotMetadata meta6 = new LakeTable.LakeSnapshotMetadata(6L, path6, path6); + + // Action: Set retention boundary to Snapshot 4. + // Expected behavior: Snapshot 3 (before the boundary) is discarded; 4, 5, and 6 are kept. + lakeTableHelper.registerLakeTableSnapshotV2(tableId, meta6, 4L); + + // Verify physical cleanup. + assertThat(fs.exists(path3)).isFalse(); + assertThat(fs.exists(path4)).isTrue(); + assertThat(fs.exists(path5)).isTrue(); + assertThat(fs.exists(path6)).isTrue(); + + // Verify metadata sequence in Zookeeper. + assertThat(zookeeperClient.getLakeTable(tableId).get().getLakeSnapshotMetadatas()) + .extracting(LakeTable.LakeSnapshotMetadata::getSnapshotId) + .containsExactly(4L, 5L, 6L); + } + + /** Helper to store offset files and return the FsPath. */ + private FsPath storeOffsetFile( + LakeTableHelper helper, TablePath path, long tableId, long offset) throws Exception { + Map offsets = new HashMap<>(); + offsets.put(new TableBucket(tableId, 0), offset); + return helper.storeLakeTableOffsetsFile(path, new TableBucketOffsets(tableId, offsets)); + } + + /** Helper to create a basic TableRegistration. */ + private TableRegistration createTableReg(long tableId) { + return new TableRegistration( + tableId, + "test", + Collections.emptyList(), + new TableDescriptor.TableDistribution(1, Collections.singletonList("a")), + Collections.emptyMap(), + Collections.emptyMap(), + System.currentTimeMillis(), + System.currentTimeMillis()); + } } diff --git a/fluss-test-coverage/pom.xml b/fluss-test-coverage/pom.xml index 16e942860e..abf633face 100644 --- a/fluss-test-coverage/pom.xml +++ b/fluss-test-coverage/pom.xml @@ -464,6 +464,7 @@ org.apache.fluss.flink.tiering.LakeTieringJobBuilder + org.apache.fluss.lake.committer.LakeCommitResult* org.apache.fluss.flink.tiering.FlussLakeTieringEntrypoint