From 036e1c1b53140720f6a59a4e91696a39d50baa29 Mon Sep 17 00:00:00 2001 From: luoyuxia Date: Fri, 19 Dec 2025 10:19:02 +0800 Subject: [PATCH 1/7] [lake] Only record lake snapshot file path in lake snapshots --- .../lake/committer/CommittedLakeSnapshot.java | 1 - .../committer/FlussTableLakeSnapshot.java | 73 ------- .../FlussTableLakeSnapshotCommitter.java | 196 +++++++++++++++--- .../committer/TieringCommitOperator.java | 155 ++++++-------- .../tiering/TestingLakeTieringFactory.java | 15 +- .../FlussTableLakeSnapshotCommitterTest.java | 23 +- .../committer/TieringCommitOperatorTest.java | 96 +++++---- .../FlinkIcebergTieringTestBase.java | 18 +- .../iceberg/tiering/IcebergTieringITCase.java | 29 +-- .../lance/tiering/LanceTieringITCase.java | 29 ++- .../lake/lance/tiering/LanceTieringTest.java | 25 +-- .../paimon/tiering/PaimonLakeCommitter.java | 17 +- .../testutils/FlinkPaimonTieringTestBase.java | 18 +- .../paimon/tiering/PaimonTieringITCase.java | 79 ++----- .../paimon/tiering/PaimonTieringTest.java | 48 ++--- .../rpc/gateway/AdminReadOnlyGateway.java | 32 +++ .../apache/fluss/rpc/protocol/ApiKeys.java | 3 +- fluss-rpc/src/main/proto/FlussApi.proto | 34 +++ .../rpc/TestingTabletGatewayService.java | 8 + .../apache/fluss/server/RpcServiceBase.java | 63 +++++- .../CoordinatorEventProcessor.java | 43 ++++ .../coordinator/CoordinatorService.java | 3 +- .../entity/CommitLakeTableSnapshotData.java | 33 ++- .../fluss/server/tablet/TabletServer.java | 3 +- .../fluss/server/tablet/TabletService.java | 7 +- .../server/utils/ServerRpcMessageUtils.java | 98 ++++++--- .../fluss/server/zk/data/lake/LakeTable.java | 11 +- .../server/zk/data/lake/LakeTableHelper.java | 36 +++- .../coordinator/TestCoordinatorGateway.java | 8 + .../tablet/TestTabletServerGateway.java | 8 + 30 files changed, 732 insertions(+), 480 deletions(-) delete mode 100644 fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/FlussTableLakeSnapshot.java diff --git a/fluss-common/src/main/java/org/apache/fluss/lake/committer/CommittedLakeSnapshot.java b/fluss-common/src/main/java/org/apache/fluss/lake/committer/CommittedLakeSnapshot.java index dfc67b2e49..6d3b144fa5 100644 --- a/fluss-common/src/main/java/org/apache/fluss/lake/committer/CommittedLakeSnapshot.java +++ b/fluss-common/src/main/java/org/apache/fluss/lake/committer/CommittedLakeSnapshot.java @@ -27,7 +27,6 @@ public class CommittedLakeSnapshot { private final long lakeSnapshotId; - private final Map snapshotProperties; public CommittedLakeSnapshot(long lakeSnapshotId, Map snapshotProperties) { diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/FlussTableLakeSnapshot.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/FlussTableLakeSnapshot.java deleted file mode 100644 index b40a04461e..0000000000 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/FlussTableLakeSnapshot.java +++ /dev/null @@ -1,73 +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.flink.tiering.committer; - -import org.apache.fluss.metadata.TableBucket; - -import java.util.HashMap; -import java.util.Map; -import java.util.Set; - -/** A lake snapshot for a Fluss table. */ -class FlussTableLakeSnapshot { - - private final long tableId; - - private final long lakeSnapshotId; - - // table_bucket -> log end offsets, - private final Map logEndOffsets; - - FlussTableLakeSnapshot(long tableId, long lakeSnapshotId) { - this.tableId = tableId; - this.lakeSnapshotId = lakeSnapshotId; - this.logEndOffsets = new HashMap<>(); - } - - public long tableId() { - return tableId; - } - - public long lakeSnapshotId() { - return lakeSnapshotId; - } - - public Set tableBuckets() { - return logEndOffsets.keySet(); - } - - public void addBucketOffset(TableBucket bucket, long offset) { - logEndOffsets.put(bucket, offset); - } - - public long getLogEndOffset(TableBucket bucket) { - return logEndOffsets.get(bucket); - } - - @Override - public String toString() { - return "FlussTableLakeSnapshot{" - + "tableId=" - + tableId - + ", lakeSnapshotId=" - + lakeSnapshotId - + ", logEndOffsets=" - + logEndOffsets - + '}'; - } -} 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 f4be12593a..e41be49f64 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,20 +21,45 @@ import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TablePath; import org.apache.fluss.metrics.registry.MetricRegistry; import org.apache.fluss.rpc.GatewayClientProxy; import org.apache.fluss.rpc.RpcClient; import org.apache.fluss.rpc.gateway.CoordinatorGateway; import org.apache.fluss.rpc.messages.CommitLakeTableSnapshotRequest; +import org.apache.fluss.rpc.messages.PbCommitLakeTableSnapshotRespForTable; import org.apache.fluss.rpc.messages.PbLakeTableOffsetForBucket; import org.apache.fluss.rpc.messages.PbLakeTableSnapshotInfo; +import org.apache.fluss.rpc.messages.PbLakeTableSnapshotMetadata; +import org.apache.fluss.rpc.messages.PbPrepareCommitLakeTableRespForTable; +import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotRequest; +import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotResponse; import org.apache.fluss.rpc.metrics.ClientMetricGroup; +import org.apache.fluss.rpc.protocol.ApiError; import org.apache.fluss.utils.ExceptionUtils; import java.io.IOException; +import java.util.List; import java.util.Map; -/** Committer to commit {@link FlussTableLakeSnapshot} of lake to Fluss. */ +import static org.apache.fluss.utils.Preconditions.checkNotNull; +import static org.apache.fluss.utils.Preconditions.checkState; + +/** + * Committer to commit lake table snapshots to Fluss cluster. + * + *

This committer implements a two-phase commit protocol to record lake table snapshot + * information in Fluss: + * + *

    + *
  • Prepare phase ({@link #prepareCommit}): Sends log end offsets to the FLuss cluster, + * which merges them with the previous log end offsets and stores the merged snapshot data in + * a file. Returns the file path where the snapshot metadata is stored. + *
  • Commit phase ({@link #commit}): Sends the lake snapshot metadata (including snapshot + * ID and file paths) to the coordinator to finalize the commit. Also includes log end offsets + * and max tiered timestamps for metrics reporting to tablet servers. + *
+ */ public class FlussTableLakeSnapshotCommitter implements AutoCloseable { private final Configuration flussConf; @@ -59,49 +84,170 @@ public void open() { metadataUpdater::getCoordinatorServer, rpcClient, CoordinatorGateway.class); } - void commit(FlussTableLakeSnapshot flussTableLakeSnapshot) throws IOException { + String prepareCommit(long tableId, TablePath tablePath, Map logEndOffsets) + throws IOException { + PbPrepareCommitLakeTableRespForTable prepareCommitResp = null; + Exception exception = null; try { - CommitLakeTableSnapshotRequest request = - toCommitLakeTableSnapshotRequest(flussTableLakeSnapshot); - coordinatorGateway.commitLakeTableSnapshot(request).get(); + PrepareCommitLakeTableSnapshotRequest prepareCommitLakeTableSnapshotRequest = + toPrepareCommitLakeTableSnapshotRequest(tableId, tablePath, logEndOffsets); + PrepareCommitLakeTableSnapshotResponse prepareCommitLakeTableSnapshotResponse = + coordinatorGateway + .prepareCommitLakeTableSnapshot(prepareCommitLakeTableSnapshotRequest) + .get(); + List pbPrepareCommitLakeTableRespForTables = + prepareCommitLakeTableSnapshotResponse.getPrepareCommitLakeTableRespsList(); + checkState(pbPrepareCommitLakeTableRespForTables.size() == 1); + prepareCommitResp = pbPrepareCommitLakeTableRespForTables.get(0); + if (prepareCommitResp.hasErrorCode()) { + exception = ApiError.fromErrorMessage(prepareCommitResp).exception(); + } } catch (Exception e) { + exception = e; + } + + if (exception != null) { throw new IOException( String.format( - "Fail to commit table lake snapshot %s to Fluss.", - flussTableLakeSnapshot), - ExceptionUtils.stripExecutionException(e)); + "Fail to prepare commit table lake snapshot for %s to Fluss.", + tablePath), + ExceptionUtils.stripExecutionException(exception)); } + return checkNotNull(prepareCommitResp).getLakeTableSnapshotFilePath(); } - public void commit(long tableId, long snapshotId, Map logEndOffsets) + void commit( + long tableId, + long lakeSnapshotId, + String lakeSnapshotPath, + Map logEndOffsets, + Map logMaxTieredTimestamps) throws IOException { - // construct lake snapshot to commit to Fluss - FlussTableLakeSnapshot flussTableLakeSnapshot = - new FlussTableLakeSnapshot(tableId, snapshotId); - for (Map.Entry entry : logEndOffsets.entrySet()) { - flussTableLakeSnapshot.addBucketOffset(entry.getKey(), entry.getValue()); + Exception exception = null; + try { + CommitLakeTableSnapshotRequest request = + toCommitLakeTableSnapshotRequest( + tableId, + lakeSnapshotId, + lakeSnapshotPath, + logEndOffsets, + logMaxTieredTimestamps); + List commitLakeTableSnapshotRespForTables = + coordinatorGateway.commitLakeTableSnapshot(request).get().getTableRespsList(); + checkState(commitLakeTableSnapshotRespForTables.size() == 1); + PbCommitLakeTableSnapshotRespForTable commitLakeTableSnapshotRes = + commitLakeTableSnapshotRespForTables.get(0); + if (commitLakeTableSnapshotRes.hasErrorCode()) { + exception = ApiError.fromErrorMessage(commitLakeTableSnapshotRes).exception(); + } + } catch (Exception e) { + exception = e; + } + + if (exception != null) { + throw new IOException( + String.format( + "Fail to commit table lake snapshot id %d of table %d to Fluss.", + lakeSnapshotId, tableId), + ExceptionUtils.stripExecutionException(exception)); } - commit(flussTableLakeSnapshot); } - private CommitLakeTableSnapshotRequest toCommitLakeTableSnapshotRequest( - FlussTableLakeSnapshot flussTableLakeSnapshot) { - CommitLakeTableSnapshotRequest commitLakeTableSnapshotRequest = - new CommitLakeTableSnapshotRequest(); + /** + * Converts the prepare commit parameters to a {@link PrepareCommitLakeTableSnapshotRequest}. + * + * @param tableId the table ID + * @param tablePath the table path + * @param logEndOffsets the log end offsets for each bucket + * @return the prepared commit request + */ + private PrepareCommitLakeTableSnapshotRequest toPrepareCommitLakeTableSnapshotRequest( + long tableId, TablePath tablePath, Map logEndOffsets) { + PrepareCommitLakeTableSnapshotRequest prepareCommitLakeTableSnapshotRequest = + new PrepareCommitLakeTableSnapshotRequest(); PbLakeTableSnapshotInfo pbLakeTableSnapshotInfo = - commitLakeTableSnapshotRequest.addTablesReq(); + prepareCommitLakeTableSnapshotRequest.addTablesReq(); + pbLakeTableSnapshotInfo.setTableId(tableId); - pbLakeTableSnapshotInfo.setTableId(flussTableLakeSnapshot.tableId()); - pbLakeTableSnapshotInfo.setSnapshotId(flussTableLakeSnapshot.lakeSnapshotId()); - for (TableBucket tableBucket : flussTableLakeSnapshot.tableBuckets()) { + // in prepare phase, we don't know the snapshot id, + // set -1 since the field is required + pbLakeTableSnapshotInfo.setSnapshotId(-1L); + for (Map.Entry logEndOffsetEntry : logEndOffsets.entrySet()) { PbLakeTableOffsetForBucket pbLakeTableOffsetForBucket = pbLakeTableSnapshotInfo.addBucketsReq(); - long endOffset = flussTableLakeSnapshot.getLogEndOffset(tableBucket); + TableBucket tableBucket = logEndOffsetEntry.getKey(); + pbLakeTableSnapshotInfo + .setTablePath() + .setDatabaseName(tablePath.getDatabaseName()) + .setTableName(tablePath.getTableName()); if (tableBucket.getPartitionId() != null) { pbLakeTableOffsetForBucket.setPartitionId(tableBucket.getPartitionId()); } pbLakeTableOffsetForBucket.setBucketId(tableBucket.getBucket()); - pbLakeTableOffsetForBucket.setLogEndOffset(endOffset); + pbLakeTableOffsetForBucket.setLogEndOffset(logEndOffsetEntry.getValue()); + } + return prepareCommitLakeTableSnapshotRequest; + } + + /** + * Converts the commit parameters to a {@link CommitLakeTableSnapshotRequest}. + * + *

This method creates a request that includes: + * + *

    + *
  • Lake table snapshot metadata (snapshot ID, table ID, file paths) + *
  • PbLakeTableSnapshotInfo for metrics reporting (log end offsets and max tiered + * timestamps) + *
+ * + * @param tableId the table ID + * @param snapshotId the lake snapshot ID + * @param lakeSnapshotPath the file path where the snapshot metadata is stored + * @param logEndOffsets the log end offsets for each bucket + * @param logMaxTieredTimestamps the max tiered timestamps for each bucket + * @return the commit request + */ + private CommitLakeTableSnapshotRequest toCommitLakeTableSnapshotRequest( + long tableId, + long snapshotId, + String lakeSnapshotPath, + Map logEndOffsets, + Map logMaxTieredTimestamps) { + CommitLakeTableSnapshotRequest commitLakeTableSnapshotRequest = + new CommitLakeTableSnapshotRequest(); + + // Add lake table snapshot metadata + PbLakeTableSnapshotMetadata pbLakeTableSnapshotMetadata = + commitLakeTableSnapshotRequest.addLakeTableSnapshotMetadata(); + pbLakeTableSnapshotMetadata.setSnapshotId(snapshotId); + pbLakeTableSnapshotMetadata.setTableId(tableId); + // tiered snapshot file path is equal to readable snapshot currently + pbLakeTableSnapshotMetadata.setTieredSnapshotFilePath(lakeSnapshotPath); + pbLakeTableSnapshotMetadata.setReadableSnapshotFilePath(lakeSnapshotPath); + + // Add PbLakeTableSnapshotInfo for metrics reporting (to notify tablet servers about + // synchronized log end offsets and max timestamps) + if (!logEndOffsets.isEmpty()) { + PbLakeTableSnapshotInfo pbLakeTableSnapshotInfo = + commitLakeTableSnapshotRequest.addTablesReq(); + pbLakeTableSnapshotInfo.setTableId(tableId); + pbLakeTableSnapshotInfo.setSnapshotId(snapshotId); + for (Map.Entry logEndOffsetEntry : logEndOffsets.entrySet()) { + TableBucket tableBucket = logEndOffsetEntry.getKey(); + PbLakeTableOffsetForBucket pbLakeTableOffsetForBucket = + pbLakeTableSnapshotInfo.addBucketsReq(); + + if (tableBucket.getPartitionId() != null) { + pbLakeTableOffsetForBucket.setPartitionId(tableBucket.getPartitionId()); + } + pbLakeTableOffsetForBucket.setBucketId(tableBucket.getBucket()); + pbLakeTableOffsetForBucket.setLogEndOffset(logEndOffsetEntry.getValue()); + + Long maxTimestamp = logMaxTieredTimestamps.get(tableBucket); + if (maxTimestamp != null) { + pbLakeTableOffsetForBucket.setMaxTimestamp(maxTimestamp); + } + } } return commitLakeTableSnapshotRequest; } 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 8954ed6f58..fb0a4bf1be 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 @@ -29,19 +29,13 @@ import org.apache.fluss.flink.tiering.event.TieringFailOverEvent; import org.apache.fluss.flink.tiering.source.TableBucketWriteResult; import org.apache.fluss.flink.tiering.source.TieringSource; -import org.apache.fluss.lake.committer.BucketOffset; import org.apache.fluss.lake.committer.CommittedLakeSnapshot; import org.apache.fluss.lake.committer.LakeCommitter; import org.apache.fluss.lake.writer.LakeTieringFactory; import org.apache.fluss.lake.writer.LakeWriter; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TablePath; -import org.apache.fluss.shaded.jackson2.com.fasterxml.jackson.core.JsonFactory; -import org.apache.fluss.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; -import org.apache.fluss.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; -import org.apache.fluss.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.fluss.utils.ExceptionUtils; -import org.apache.fluss.utils.json.BucketOffsetJsonSerde; import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; import org.apache.flink.runtime.source.event.SourceEventWrapper; @@ -55,9 +49,8 @@ import javax.annotation.Nullable; -import java.io.IOException; -import java.io.StringWriter; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -88,8 +81,6 @@ public class TieringCommitOperator implements OneInputStreamOperator< TableBucketWriteResult, CommittableMessage> { - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - private static final long serialVersionUID = 1L; private final Configuration flussConfig; @@ -98,7 +89,6 @@ public class TieringCommitOperator private final FlussTableLakeSnapshotCommitter flussTableLakeSnapshotCommitter; private Connection connection; private Admin admin; - private static final JsonFactory JACKSON_FACTORY = new JsonFactory(); // gateway to send event to flink source coordinator private final OperatorEventGateway operatorEventGateway; @@ -220,74 +210,47 @@ private Committable commitWriteResults( .map(TableBucketWriteResult::writeResult) .collect(Collectors.toList()); - LakeSnapshot flussCurrentLakeSnapshot = getLatestLakeSnapshot(tablePath); - Map logOffsetsProperty = - toBucketOffsetsProperty(flussCurrentLakeSnapshot, committableWriteResults); + Map logEndOffsets = new HashMap<>(); + Map logMaxTieredTimestamps = new HashMap<>(); + for (TableBucketWriteResult writeResult : committableWriteResults) { + TableBucket tableBucket = writeResult.tableBucket(); + logEndOffsets.put(tableBucket, writeResult.logEndOffset()); + logMaxTieredTimestamps.put(tableBucket, writeResult.maxTimestamp()); + } + // to committable Committable committable = lakeCommitter.toCommittable(writeResults); // before commit to lake, check fluss not missing any lake snapshot committed by fluss + LakeSnapshot flussCurrentLakeSnapshot = getLatestLakeSnapshot(tablePath); checkFlussNotMissingLakeSnapshot( - tableId, tablePath, + tableId, lakeCommitter, committable, flussCurrentLakeSnapshot == null ? null : flussCurrentLakeSnapshot.getSnapshotId()); - long committedSnapshotId = lakeCommitter.commit(committable, logOffsetsProperty); - // commit to fluss - FlussTableLakeSnapshot flussTableLakeSnapshot = - new FlussTableLakeSnapshot(tableId, committedSnapshotId); - for (TableBucketWriteResult writeResult : committableWriteResults) { - TableBucket tableBucket = writeResult.tableBucket(); - flussTableLakeSnapshot.addBucketOffset(tableBucket, writeResult.logEndOffset()); - } - flussTableLakeSnapshotCommitter.commit(flussTableLakeSnapshot); - return committable; - } - } - - /** - * Merge the log offsets of latest snapshot with current written bucket offsets to get full log - * offsets. - */ - private Map toBucketOffsetsProperty( - @Nullable LakeSnapshot latestLakeSnapshot, - List> currentWriteResults) - throws Exception { - // first of all, we need to merge latest lake snapshot with current write results - Map tableBucketOffsets = new HashMap<>(); - if (latestLakeSnapshot != null) { - tableBucketOffsets = new HashMap<>(latestLakeSnapshot.getTableBucketsOffset()); - } - for (TableBucketWriteResult tableBucketWriteResult : currentWriteResults) { - tableBucketOffsets.put( - tableBucketWriteResult.tableBucket(), tableBucketWriteResult.logEndOffset()); - } - - // then, serialize the bucket offsets, partition name by id - return toBucketOffsetsProperty(tableBucketOffsets); - } - - public static Map toBucketOffsetsProperty( - Map tableBucketOffsets) throws IOException { - StringWriter sw = new StringWriter(); - try (JsonGenerator gen = JACKSON_FACTORY.createGenerator(sw)) { - gen.writeStartArray(); - for (Map.Entry entry : tableBucketOffsets.entrySet()) { - Long partitionId = entry.getKey().getPartitionId(); - BucketOffsetJsonSerde.INSTANCE.serialize( - new BucketOffset(entry.getValue(), entry.getKey().getBucket(), partitionId), - gen); - } - gen.writeEndArray(); + // get the lake snapshot file storing the log end offsets + String lakeSnapshotMetadataFile = + flussTableLakeSnapshotCommitter.prepareCommit( + tableId, tablePath, logEndOffsets); + + // record the lake snapshot metadata file to snapshot property + long committedSnapshotId = + lakeCommitter.commit( + committable, + Collections.singletonMap( + FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY, + lakeSnapshotMetadataFile)); + flussTableLakeSnapshotCommitter.commit( + tableId, + committedSnapshotId, + lakeSnapshotMetadataFile, + logEndOffsets, + logMaxTieredTimestamps); + return committable; } - return new HashMap() { - { - put(FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY, sw.toString()); - } - }; } @Nullable @@ -308,13 +271,12 @@ private LakeSnapshot getLatestLakeSnapshot(TablePath tablePath) throws Exception } private void checkFlussNotMissingLakeSnapshot( - long tableId, TablePath tablePath, + long tableId, LakeCommitter lakeCommitter, Committable committable, Long flussCurrentLakeSnapshot) throws Exception { - // get Fluss missing lake snapshot in Lake CommittedLakeSnapshot missingCommittedSnapshot = lakeCommitter.getMissingLakeSnapshot(flussCurrentLakeSnapshot); @@ -324,33 +286,45 @@ private void checkFlussNotMissingLakeSnapshot( // known lake snapshot, which means the data already has been committed to lake, // not to commit to lake to avoid data duplicated if (missingCommittedSnapshot != null) { - if (missingCommittedSnapshot.getSnapshotProperties() == null - || missingCommittedSnapshot - .getSnapshotProperties() - .get(FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY) - == null) { + String lakeSnapshotOffsetPath = + missingCommittedSnapshot + .getSnapshotProperties() + .get(FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY); + + // should only will happen in v0.7 which won't put offsets info + // to properties + if (lakeSnapshotOffsetPath == null) { throw new IllegalStateException( String.format( - "Missing required log offsets property '%s' in lake snapshot %d for table: ‘tablePath=%s, tableId=%d’. " - + "This property is required to commit the missing snapshot to Fluss. " - + "The snapshot may have been created by an older version of Fluss that did not store this information, " - + "or the snapshot properties may be corrupted.", + "Can't find %s field from snapshot property.", + FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY)); + } + + // the fluss-offsets will be a json string if it's tiered by v0.8, + // since this code path should be rare, we do not consider backward compatibility + // and throw IllegalStateException directly + String trimmedPath = lakeSnapshotOffsetPath.trim(); + if (trimmedPath.startsWith("{")) { + throw new IllegalStateException( + String.format( + "The %s field in snapshot property is a JSON string (tiered by v0.8), " + + "which is not supported to restore. Snapshot ID: %d, Table: {tablePath=%s, tableId=%d}.", FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY, missingCommittedSnapshot.getLakeSnapshotId(), tablePath, tableId)); } - String logOffsetsProperty = - missingCommittedSnapshot - .getSnapshotProperties() - .get(FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY); - // commit this missing snapshot to fluss flussTableLakeSnapshotCommitter.commit( tableId, missingCommittedSnapshot.getLakeSnapshotId(), - fromLogOffsetProperty(tableId, logOffsetsProperty)); + lakeSnapshotOffsetPath, + // 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()); // abort this committable to delete the written files lakeCommitter.abort(committable); throw new IllegalStateException( @@ -366,19 +340,6 @@ private void checkFlussNotMissingLakeSnapshot( } } - public static Map fromLogOffsetProperty( - long tableId, String logOffsetsProperty) throws IOException { - Map logEndOffsets = new HashMap<>(); - for (JsonNode node : OBJECT_MAPPER.readTree(logOffsetsProperty)) { - BucketOffset bucketOffset = BucketOffsetJsonSerde.INSTANCE.deserialize(node); - TableBucket tableBucket = - new TableBucket( - tableId, bucketOffset.getPartitionId(), bucketOffset.getBucket()); - logEndOffsets.put(tableBucket, bucketOffset.getLogOffset()); - } - return logEndOffsets; - } - private void registerTableBucketWriteResult( long tableId, TableBucketWriteResult tableBucketWriteResult) { collectedTableBucketWriteResults 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 8b67927af4..f096b372b2 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 @@ -99,16 +99,14 @@ public static final class TestingLakeCommitter private long currentSnapshot; - @Nullable private final CommittedLakeSnapshot mockCommittedSnapshot; + @Nullable private final CommittedLakeSnapshot mockMissingCommittedLakeSnapshot; public TestingLakeCommitter() { this(null); } - public TestingLakeCommitter(@Nullable CommittedLakeSnapshot mockCommittedSnapshot) { - this.mockCommittedSnapshot = mockCommittedSnapshot; - this.currentSnapshot = - mockCommittedSnapshot == null ? 0 : mockCommittedSnapshot.getLakeSnapshotId(); + public TestingLakeCommitter(CommittedLakeSnapshot mockMissingCommittedLakeSnapshot) { + this.mockMissingCommittedLakeSnapshot = mockMissingCommittedLakeSnapshot; } @Override @@ -135,11 +133,10 @@ public void abort(TestingCommittable committable) throws IOException { @Override public @Nullable CommittedLakeSnapshot getMissingLakeSnapshot( @Nullable Long knownSnapshotId) throws IOException { - if (knownSnapshotId == null) { - return mockCommittedSnapshot; - } else { - return null; + if (mockMissingCommittedLakeSnapshot != null && knownSnapshotId == null) { + return mockMissingCommittedLakeSnapshot; } + return null; } @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 f8d71e6cf0..d2f9302c12 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 @@ -78,27 +78,38 @@ void testCommit(boolean isPartitioned) throws Exception { partitions = new ArrayList<>(partitionNameAndIds.keySet()); } - Map logEndOffsets = new HashMap<>(); + Map expectedOffsets = new HashMap<>(); for (int bucket = 0; bucket < 3; bucket++) { long bucketOffset = bucket * bucket; for (String partitionName : partitions) { if (partitionName == null) { - logEndOffsets.put(new TableBucket(tableId, bucket), bucketOffset); + expectedOffsets.put(new TableBucket(tableId, bucket), bucketOffset); } else { long partitionId = partitionNameAndIds.get(partitionName); - logEndOffsets.put(new TableBucket(tableId, partitionId, bucket), bucketOffset); + expectedOffsets.put( + new TableBucket(tableId, partitionId, bucket), bucketOffset); } } } - long snapshotId = 3; + long lakeSnapshotId = 3; + + String lakeSnapshotFilePath = + flussTableLakeSnapshotCommitter.prepareCommit(tableId, tablePath, expectedOffsets); + // commit offsets - flussTableLakeSnapshotCommitter.commit(tableId, snapshotId, logEndOffsets); + flussTableLakeSnapshotCommitter.commit( + tableId, + lakeSnapshotId, + lakeSnapshotFilePath, + // don't care end offsets, maxTieredTimestamps + Collections.emptyMap(), + Collections.emptyMap()); LakeSnapshot lakeSnapshot = admin.getLatestLakeSnapshot(tablePath).get(); assertThat(lakeSnapshot.getSnapshotId()).isEqualTo(3); // get and check the offsets Map bucketLogOffsets = lakeSnapshot.getTableBucketsOffset(); - assertThat(bucketLogOffsets).isEqualTo(logEndOffsets); + assertThat(bucketLogOffsets).isEqualTo(expectedOffsets); } } diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperatorTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperatorTest.java index 618d454453..290b99244f 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperatorTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperatorTest.java @@ -50,15 +50,12 @@ import javax.annotation.Nullable; -import java.io.IOException; 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.flink.tiering.committer.TieringCommitOperator.fromLogOffsetProperty; -import static org.apache.fluss.flink.tiering.committer.TieringCommitOperator.toBucketOffsetsProperty; import static org.apache.fluss.lake.committer.BucketOffset.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY; import static org.apache.fluss.record.TestData.DATA1_PARTITIONED_TABLE_DESCRIPTOR; import static org.assertj.core.api.Assertions.assertThat; @@ -182,7 +179,6 @@ void testCommitPartitionedTable() throws Exception { Map partitionIdByNames = FLUSS_CLUSTER_EXTENSION.waitUntilPartitionAllReady(tablePath); Map expectedLogEndOffsets = new HashMap<>(); - Map expectedMaxTimestamps = new HashMap<>(); int numberOfWriteResults = 3 * partitionIdByNames.size(); long offset = 0; long timestamp = System.currentTimeMillis(); @@ -202,7 +198,6 @@ void testCommitPartitionedTable() throws Exception { currentTimestamp, numberOfWriteResults)); expectedLogEndOffsets.put(tableBucket, currentOffset); - expectedMaxTimestamps.put(tableBucket, currentTimestamp); } if (bucket == 2) { verifyLakeSnapshot(tablePath, tableId, 1, expectedLogEndOffsets); @@ -262,10 +257,17 @@ void testTableCommitWhenFlussMissingLakeSnapshot() throws Exception { long tableId = createTable(tablePath, DEFAULT_PK_TABLE_DESCRIPTOR); int numberOfWriteResults = 3; - CommittedLakeSnapshot mockCommittedSnapshot = - mockCommittedLakeSnapshot(Collections.singletonList(null), tableId, 2); + Map expectedLogEndOffsets = new HashMap<>(); + for (int bucket = 0; bucket < 3; bucket++) { + TableBucket tableBucket = new TableBucket(tableId, bucket); + expectedLogEndOffsets.put(tableBucket, 3L); + } + + CommittedLakeSnapshot mockMissingCommittedLakeSnapshot = + mockCommittedLakeSnapshot(tableId, tablePath, 0, expectedLogEndOffsets); TestingLakeTieringFactory.TestingLakeCommitter testingLakeCommitter = - new TestingLakeTieringFactory.TestingLakeCommitter(mockCommittedSnapshot); + new TestingLakeTieringFactory.TestingLakeCommitter( + mockMissingCommittedLakeSnapshot); committerOperator = new TieringCommitOperator<>( parameters, @@ -284,18 +286,18 @@ void testTableCommitWhenFlussMissingLakeSnapshot() throws Exception { verifyLakeSnapshot( tablePath, tableId, - 2, - getExpectedLogEndOffsets(tableId, mockCommittedSnapshot), + 0, + expectedLogEndOffsets, String.format( "The current Fluss's lake snapshot %s is less than lake actual snapshot %d committed by Fluss for table: {tablePath=%s, tableId=%d}," + " missing snapshot: %s.", null, - mockCommittedSnapshot.getLakeSnapshotId(), + mockMissingCommittedLakeSnapshot.getLakeSnapshotId(), tablePath, tableId, - mockCommittedSnapshot)); + mockMissingCommittedLakeSnapshot)); - Map expectedLogEndOffsets = new HashMap<>(); + expectedLogEndOffsets = new HashMap<>(); for (int bucket = 0; bucket < 3; bucket++) { TableBucket tableBucket = new TableBucket(tableId, bucket); long offset = bucket * bucket; @@ -306,7 +308,7 @@ void testTableCommitWhenFlussMissingLakeSnapshot() throws Exception { expectedLogEndOffsets.put(tableBucket, offset); } - verifyLakeSnapshot(tablePath, tableId, 3, expectedLogEndOffsets); + verifyLakeSnapshot(tablePath, tableId, 1, expectedLogEndOffsets); } @Test @@ -319,10 +321,21 @@ void testPartitionedTableCommitWhenFlussMissingLakeSnapshot() throws Exception { Map partitionIdByNames = FLUSS_CLUSTER_EXTENSION.waitUntilPartitionAllReady(tablePath); - CommittedLakeSnapshot mockCommittedSnapshot = - mockCommittedLakeSnapshot(Collections.singletonList(null), tableId, 3); + Map expectedLogEndOffsets = new HashMap<>(); + for (int bucket = 0; bucket < 3; bucket++) { + for (String partitionName : partitionIdByNames.keySet()) { + long partitionId = partitionIdByNames.get(partitionName); + TableBucket tableBucket = new TableBucket(tableId, partitionId, bucket); + expectedLogEndOffsets.put(tableBucket, 3L); + } + } + + CommittedLakeSnapshot mockMissingCommittedLakeSnapshot = + mockCommittedLakeSnapshot(tableId, tablePath, 0, expectedLogEndOffsets); + TestingLakeTieringFactory.TestingLakeCommitter testingLakeCommitter = - new TestingLakeTieringFactory.TestingLakeCommitter(mockCommittedSnapshot); + new TestingLakeTieringFactory.TestingLakeCommitter( + mockMissingCommittedLakeSnapshot); committerOperator = new TieringCommitOperator<>( parameters, @@ -352,35 +365,15 @@ void testPartitionedTableCommitWhenFlussMissingLakeSnapshot() throws Exception { verifyLakeSnapshot( tablePath, tableId, - 3, - getExpectedLogEndOffsets(tableId, mockCommittedSnapshot), + 0, + expectedLogEndOffsets, String.format( "The current Fluss's lake snapshot %s is less than lake actual snapshot %d committed by Fluss for table: {tablePath=%s, tableId=%d}, missing snapshot: %s.", null, - mockCommittedSnapshot.getLakeSnapshotId(), + mockMissingCommittedLakeSnapshot.getLakeSnapshotId(), tablePath, tableId, - mockCommittedSnapshot)); - } - - private CommittedLakeSnapshot mockCommittedLakeSnapshot( - List partitions, long tableId, int snapshotId) throws IOException { - Map logEndOffsets = new HashMap<>(); - for (Long partition : partitions) { - for (int bucket = 0; bucket < DEFAULT_BUCKET_NUM; bucket++) { - logEndOffsets.put(new TableBucket(tableId, partition, bucket), bucket + 1L); - } - } - return new CommittedLakeSnapshot(snapshotId, toBucketOffsetsProperty(logEndOffsets)); - } - - private Map getExpectedLogEndOffsets( - long tableId, CommittedLakeSnapshot committedLakeSnapshot) throws IOException { - return fromLogOffsetProperty( - tableId, - committedLakeSnapshot - .getSnapshotProperties() - .get(FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY)); + mockMissingCommittedLakeSnapshot)); } private StreamRecord> @@ -462,10 +455,25 @@ private void verifyLakeSnapshot( List operatorEvents = mockOperatorEventGateway.getEventsSent(); SourceEventWrapper sourceEventWrapper = (SourceEventWrapper) operatorEvents.get(operatorEvents.size() - 1); - FailedTieringEvent finishTieringEvent = + FailedTieringEvent failedTieringEvent = (FailedTieringEvent) sourceEventWrapper.getSourceEvent(); - assertThat(finishTieringEvent.getTableId()).isEqualTo(tableId); - assertThat(finishTieringEvent.failReason()).contains(failedReason); + assertThat(failedTieringEvent.getTableId()).isEqualTo(tableId); + assertThat(failedTieringEvent.failReason()).contains(failedReason); + } + + private CommittedLakeSnapshot mockCommittedLakeSnapshot( + long tableId, TablePath tablePath, int snapshotId, Map logEndOffsets) + throws Exception { + try (FlussTableLakeSnapshotCommitter lakeSnapshotCommitter = + new FlussTableLakeSnapshotCommitter(FLUSS_CLUSTER_EXTENSION.getClientConfig())) { + lakeSnapshotCommitter.open(); + String lakeSnapshotFile = + lakeSnapshotCommitter.prepareCommit(tableId, tablePath, logEndOffsets); + return new CommittedLakeSnapshot( + snapshotId, + Collections.singletonMap( + FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY, lakeSnapshotFile)); + } } private static class MockOperatorEventDispatcher implements OperatorEventDispatcher { diff --git a/fluss-lake/fluss-lake-iceberg/src/test/java/org/apache/fluss/lake/iceberg/testutils/FlinkIcebergTieringTestBase.java b/fluss-lake/fluss-lake-iceberg/src/test/java/org/apache/fluss/lake/iceberg/testutils/FlinkIcebergTieringTestBase.java index a7c922ba5a..d6408384c3 100644 --- a/fluss-lake/fluss-lake-iceberg/src/test/java/org/apache/fluss/lake/iceberg/testutils/FlinkIcebergTieringTestBase.java +++ b/fluss-lake/fluss-lake-iceberg/src/test/java/org/apache/fluss/lake/iceberg/testutils/FlinkIcebergTieringTestBase.java @@ -28,6 +28,7 @@ import org.apache.fluss.config.Configuration; import org.apache.fluss.exception.FlussRuntimeException; import org.apache.fluss.flink.tiering.LakeTieringJobBuilder; +import org.apache.fluss.fs.FsPath; import org.apache.fluss.metadata.DataLakeFormat; import org.apache.fluss.metadata.Schema; import org.apache.fluss.metadata.TableBucket; @@ -37,6 +38,7 @@ import org.apache.fluss.server.replica.Replica; import org.apache.fluss.server.testutils.FlussClusterExtension; import org.apache.fluss.server.zk.ZooKeeperClient; +import org.apache.fluss.server.zk.data.lake.LakeTable; import org.apache.flink.api.common.RuntimeExecutionMode; import org.apache.flink.core.execution.JobClient; @@ -76,6 +78,7 @@ import java.util.TreeSet; import static org.apache.fluss.flink.tiering.source.TieringSourceOptions.POLL_TIERING_TABLE_INTERVAL; +import static org.apache.fluss.lake.committer.BucketOffset.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY; import static org.apache.fluss.lake.iceberg.utils.IcebergConversions.toIceberg; import static org.apache.fluss.metadata.TableDescriptor.OFFSET_COLUMN_NAME; import static org.apache.fluss.testutils.DataTestUtils.row; @@ -471,11 +474,20 @@ private TableScan filterByPartition(TableScan tableScan, Map par return tableScan; } - protected void checkSnapshotPropertyInIceberg( - TablePath tablePath, Map expectedProperties) { + protected void checkFlussOffsetsInSnapshot( + TablePath tablePath, Map expectedOffsets) throws Exception { org.apache.iceberg.Table table = icebergCatalog.loadTable(toIceberg(tablePath)); Snapshot snapshot = table.currentSnapshot(); - assertThat(snapshot.summary()).containsAllEntriesOf(expectedProperties); + + String offsetFile = snapshot.summary().get(FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY); + Map recordedOffsets = + new LakeTable( + new LakeTable.LakeSnapshotMetadata( + // don't care about snapshot id + -1, new FsPath(offsetFile), null)) + .getLatestTableSnapshot() + .getBucketLogEndOffset(); + assertThat(recordedOffsets).isEqualTo(expectedOffsets); } protected Map> writeRowsIntoPartitionedTable( diff --git a/fluss-lake/fluss-lake-iceberg/src/test/java/org/apache/fluss/lake/iceberg/tiering/IcebergTieringITCase.java b/fluss-lake/fluss-lake-iceberg/src/test/java/org/apache/fluss/lake/iceberg/tiering/IcebergTieringITCase.java index 55be048c99..49983b603f 100644 --- a/fluss-lake/fluss-lake-iceberg/src/test/java/org/apache/fluss/lake/iceberg/tiering/IcebergTieringITCase.java +++ b/fluss-lake/fluss-lake-iceberg/src/test/java/org/apache/fluss/lake/iceberg/tiering/IcebergTieringITCase.java @@ -54,7 +54,6 @@ import java.util.List; import java.util.Map; -import static org.apache.fluss.lake.committer.BucketOffset.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY; import static org.apache.fluss.testutils.DataTestUtils.row; import static org.assertj.core.api.Assertions.assertThat; @@ -182,16 +181,7 @@ void testTiering() throws Exception { assertReplicaStatus(t1Bucket, 3); checkDataInIcebergPrimaryKeyTable(t1, rows); - // check snapshot property in iceberg - Map properties = - new HashMap() { - { - put( - FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY, - "[{\"bucket\":0,\"offset\":3}]"); - } - }; - checkSnapshotPropertyInIceberg(t1, properties); + checkFlussOffsetsInSnapshot(t1, Collections.singletonMap(t1Bucket, 3L)); // test log table testLogTableTiering(); @@ -379,14 +369,17 @@ private void testPartitionedTableTiering() throws Exception { partitionedTablePath, partitionedTableDescriptor, partitionNameByIds); long tableId = tableIdAndDescriptor.f0; + Map expectedOffsets = new HashMap<>(); // wait until synced to iceberg for (Long partitionId : partitionNameByIds.keySet()) { TableBucket tableBucket = new TableBucket(tableId, partitionId, 0); assertReplicaStatus(tableBucket, 3); + expectedOffsets.put(tableBucket, 3L); } // now, let's check data in iceberg per partition // check data in iceberg + String partitionCol = partitionedTableDescriptor.getPartitionKeys().get(0); for (String partitionName : partitionNameByIds.values()) { checkDataInIcebergAppendOnlyPartitionedTable( @@ -396,18 +389,6 @@ private void testPartitionedTableTiering() throws Exception { 0); } - Map properties = - new HashMap() { - { - put( - FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY, - "[" - + "{\"partition_id\":0,\"bucket\":0,\"offset\":3}," - + "{\"partition_id\":1,\"bucket\":0,\"offset\":3}" - + "]"); - } - }; - - checkSnapshotPropertyInIceberg(partitionedTablePath, properties); + checkFlussOffsetsInSnapshot(partitionedTablePath, expectedOffsets); } } diff --git a/fluss-lake/fluss-lake-lance/src/test/java/org/apache/fluss/lake/lance/tiering/LanceTieringITCase.java b/fluss-lake/fluss-lake-lance/src/test/java/org/apache/fluss/lake/lance/tiering/LanceTieringITCase.java index b0c5f1606e..877421e257 100644 --- a/fluss-lake/fluss-lake-lance/src/test/java/org/apache/fluss/lake/lance/tiering/LanceTieringITCase.java +++ b/fluss-lake/fluss-lake-lance/src/test/java/org/apache/fluss/lake/lance/tiering/LanceTieringITCase.java @@ -18,11 +18,13 @@ package org.apache.fluss.lake.lance.tiering; import org.apache.fluss.config.Configuration; +import org.apache.fluss.fs.FsPath; import org.apache.fluss.lake.lance.LanceConfig; import org.apache.fluss.lake.lance.testutils.FlinkLanceTieringTestBase; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.row.InternalRow; +import org.apache.fluss.server.zk.data.lake.LakeTable; import com.lancedb.lance.Dataset; import com.lancedb.lance.ReadOptions; @@ -39,13 +41,11 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; import static org.apache.fluss.lake.committer.BucketOffset.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY; -import static org.apache.fluss.lake.writer.LakeTieringFactory.FLUSS_LAKE_TIERING_COMMIT_USER; import static org.apache.fluss.testutils.DataTestUtils.row; import static org.assertj.core.api.Assertions.assertThat; @@ -96,29 +96,28 @@ void testTiering() throws Exception { // check data in lance checkDataInLanceAppendOnlyTable(config, flussRows); - // check snapshot property in lance - Map properties = - new HashMap() { - { - put( - FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY, - "[{\"bucket\":0,\"offset\":30}]"); - put("commit-user", FLUSS_LAKE_TIERING_COMMIT_USER); - } - }; - checkSnapshotPropertyInLance(config, properties); + checkSnapshotPropertyInLance(config, Collections.singletonMap(t1Bucket, 30L)); jobClient.cancel().get(); } private void checkSnapshotPropertyInLance( - LanceConfig config, Map expectedProperties) throws Exception { + LanceConfig config, Map expectedOffsets) throws Exception { ReadOptions.Builder builder = new ReadOptions.Builder(); builder.setStorageOptions(LanceConfig.genStorageOptions(config)); try (Dataset dataset = Dataset.open(allocator, config.getDatasetUri(), builder.build())) { Transaction transaction = dataset.readTransaction().orElse(null); assertThat(transaction).isNotNull(); - assertThat(transaction.transactionProperties()).isEqualTo(expectedProperties); + String offsetFile = + transaction.transactionProperties().get(FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY); + Map recordedOffsets = + new LakeTable( + new LakeTable.LakeSnapshotMetadata( + // don't care about snapshot id + -1, new FsPath(offsetFile), null)) + .getLatestTableSnapshot() + .getBucketLogEndOffset(); + assertThat(recordedOffsets).isEqualTo(expectedOffsets); } } 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 1cfac3723f..019b21f543 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 @@ -64,8 +64,6 @@ import java.util.Map; import java.util.stream.Stream; -import static org.apache.fluss.flink.tiering.committer.TieringCommitOperator.fromLogOffsetProperty; -import static org.apache.fluss.flink.tiering.committer.TieringCommitOperator.toBucketOffsetsProperty; import static org.apache.fluss.lake.committer.BucketOffset.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY; import static org.assertj.core.api.Assertions.assertThat; @@ -133,7 +131,6 @@ void testTieringWriteTable(boolean isPartitioned) throws Exception { } } : Collections.singletonMap(null, null); - Map tableBucketOffsets = new HashMap<>(); // first, write data for (int bucket = 0; bucket < bucketNum; bucket++) { for (Map.Entry entry : partitionIdAndName.entrySet()) { @@ -146,7 +143,6 @@ void testTieringWriteTable(boolean isPartitioned) throws Exception { List writtenRecords = writeAndExpectRecords.f0; List expectRecords = writeAndExpectRecords.f1; recordsByBucket.put(partitionBucket, expectRecords); - tableBucketOffsets.put(new TableBucket(0, entry.getKey(), bucket), 10L); for (LogRecord logRecord : writtenRecords) { lakeWriter.write(logRecord); } @@ -169,9 +165,9 @@ void testTieringWriteTable(boolean isPartitioned) throws Exception { lanceCommittable = committableSerializer.deserialize( committableSerializer.getVersion(), serialized); - long snapshot = - lakeCommitter.commit( - lanceCommittable, toBucketOffsetsProperty(tableBucketOffsets)); + Map snapshotProperties = + Collections.singletonMap(FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY, "offsets"); + long snapshot = lakeCommitter.commit(lanceCommittable, snapshotProperties); // lance dataset version starts from 1 assertThat(snapshot).isEqualTo(2); } @@ -203,21 +199,6 @@ void testTieringWriteTable(boolean isPartitioned) throws Exception { // use snapshot id 1 as the known snapshot id CommittedLakeSnapshot committedLakeSnapshot = lakeCommitter.getMissingLakeSnapshot(1L); assertThat(committedLakeSnapshot).isNotNull(); - long tableId = tableInfo.getTableId(); - Map offsets = - fromLogOffsetProperty( - tableInfo.getTableId(), - committedLakeSnapshot - .getSnapshotProperties() - .get(FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY)); - - for (int bucket = 0; bucket < 3; bucket++) { - for (Long partitionId : partitionIdAndName.keySet()) { - // we only write 10 records, so expected log offset should be 10 - assertThat(offsets.get(new TableBucket(tableId, partitionId, bucket))) - .isEqualTo(10); - } - } assertThat(committedLakeSnapshot.getLakeSnapshotId()).isEqualTo(2L); // use null as the known snapshot id 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 72790bdcb6..d0b81b3be9 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 @@ -133,21 +133,8 @@ public CommittedLakeSnapshot getMissingLakeSnapshot(@Nullable Long latestLakeSna throw new IOException("Failed to load committed lake snapshot properties from Paimon."); } - // if resume from an old tiering service v0.7 without paimon supporting snapshot properties, - // we can't get the properties. But once come into here, it must be that - // tiering service commit snapshot to lake, but fail to commit to fluss, we have to notify - // users to run old tiering service again to commit the snapshot to fluss again, and then - // it can resume tiering with new tiering service - Map lakeSnapshotProperties = latestLakeSnapshotOfLake.properties(); - if (lakeSnapshotProperties == null) { - throw new IllegalArgumentException( - "Cannot resume tiering from an old version(v0.7) of tiering service. " - + "The snapshot was committed to the lake storage but failed to commit to Fluss. " - + "To resolve this:\n" - + "1. Run the old tiering service(v0.7) again to complete the Fluss commit\n" - + "2. Then you can resume tiering with the newer version of tiering service"); - } - return new CommittedLakeSnapshot(latestLakeSnapshotOfLake.id(), lakeSnapshotProperties); + return new CommittedLakeSnapshot( + latestLakeSnapshotOfLake.id(), latestLakeSnapshotOfLake.properties()); } @Nullable diff --git a/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/testutils/FlinkPaimonTieringTestBase.java b/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/testutils/FlinkPaimonTieringTestBase.java index 74cd75b7d1..7d66afd54f 100644 --- a/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/testutils/FlinkPaimonTieringTestBase.java +++ b/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/testutils/FlinkPaimonTieringTestBase.java @@ -29,6 +29,7 @@ import org.apache.fluss.config.Configuration; import org.apache.fluss.exception.FlussRuntimeException; import org.apache.fluss.flink.tiering.LakeTieringJobBuilder; +import org.apache.fluss.fs.FsPath; import org.apache.fluss.metadata.DataLakeFormat; import org.apache.fluss.metadata.Schema; import org.apache.fluss.metadata.TableBucket; @@ -38,6 +39,7 @@ import org.apache.fluss.server.replica.Replica; import org.apache.fluss.server.testutils.FlussClusterExtension; import org.apache.fluss.server.zk.ZooKeeperClient; +import org.apache.fluss.server.zk.data.lake.LakeTable; import org.apache.fluss.types.DataTypes; import org.apache.flink.api.common.RuntimeExecutionMode; @@ -69,6 +71,7 @@ import java.util.Set; import static org.apache.fluss.flink.tiering.source.TieringSourceOptions.POLL_TIERING_TABLE_INTERVAL; +import static org.apache.fluss.lake.committer.BucketOffset.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY; import static org.apache.fluss.testutils.DataTestUtils.row; import static org.apache.fluss.testutils.common.CommonTestUtils.retry; import static org.apache.fluss.testutils.common.CommonTestUtils.waitUntil; @@ -470,8 +473,8 @@ protected CloseableIterator getPaimonRowClos return reader.toCloseableIterator(); } - protected void checkSnapshotPropertyInPaimon( - TablePath tablePath, Map expectedProperties) throws Exception { + protected void checkFlussOffsetsInSnapshot( + TablePath tablePath, Map expectedOffsets) throws Exception { FileStoreTable table = (FileStoreTable) getPaimonCatalog() @@ -481,6 +484,15 @@ protected void checkSnapshotPropertyInPaimon( tablePath.getTableName())); Snapshot snapshot = table.snapshotManager().latestSnapshot(); assertThat(snapshot).isNotNull(); - assertThat(snapshot.properties()).isEqualTo(expectedProperties); + + String offsetFile = snapshot.properties().get(FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY); + Map recordedOffsets = + new LakeTable( + new LakeTable.LakeSnapshotMetadata( + // don't care about snapshot id + -1, new FsPath(offsetFile), null)) + .getLatestTableSnapshot() + .getBucketLogEndOffset(); + assertThat(recordedOffsets).isEqualTo(expectedOffsets); } } diff --git a/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/tiering/PaimonTieringITCase.java b/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/tiering/PaimonTieringITCase.java index 73aba60cba..19cbeb5585 100644 --- a/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/tiering/PaimonTieringITCase.java +++ b/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/tiering/PaimonTieringITCase.java @@ -67,7 +67,6 @@ import java.util.Map; import java.util.stream.Stream; -import static org.apache.fluss.lake.committer.BucketOffset.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY; import static org.apache.fluss.testutils.DataTestUtils.row; import static org.assertj.core.api.Assertions.assertThat; @@ -115,16 +114,7 @@ void testTiering() throws Exception { assertReplicaStatus(t1Bucket, 3); // check data in paimon checkDataInPaimonPrimaryKeyTable(t1, rows); - // check snapshot property in paimon - Map properties = - new HashMap() { - { - put( - FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY, - "[{\"bucket\":0,\"offset\":3}]"); - } - }; - checkSnapshotPropertyInPaimon(t1, properties); + checkFlussOffsetsInSnapshot(t1, Collections.singletonMap(new TableBucket(t1Id, 0), 3L)); // then, create another log table TablePath t2 = TablePath.of(DEFAULT_DB, "logTable"); @@ -141,6 +131,8 @@ void testTiering() throws Exception { // check the status of replica after synced; // note: we can't update log start offset for unaware bucket mode log table assertReplicaStatus(t2Bucket, 30); + assertThat(getLeaderReplica(t2Bucket).getLogTablet().getLakeMaxTimestamp()) + .isGreaterThan(-1); // check data in paimon checkDataInPaimonAppendOnlyTable(t2, flussRows, 0); @@ -171,10 +163,13 @@ void testTiering() throws Exception { partitionedTablePath, partitionedTableDescriptor, partitionNameByIds); long tableId = tableIdAndDescriptor.f0; + Map expectedOffsets = new HashMap<>(); + // wait until synced to paimon for (Long partitionId : partitionNameByIds.keySet()) { TableBucket tableBucket = new TableBucket(tableId, partitionId, 0); assertReplicaStatus(tableBucket, 3); + expectedOffsets.put(tableBucket, 3L); } // now, let's check data in paimon per partition @@ -187,16 +182,7 @@ void testTiering() throws Exception { writtenRowsByPartition.get(partitionName), 0); } - - properties = - new HashMap() { - { - put( - FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY, - getPartitionOffsetStr(partitionNameByIds)); - } - }; - checkSnapshotPropertyInPaimon(partitionedTablePath, properties); + checkFlussOffsetsInSnapshot(partitionedTablePath, expectedOffsets); } finally { jobClient.cancel().get(); } @@ -328,18 +314,7 @@ void testTieringForAllTypes(boolean isPrimaryKeyTable) throws Exception { assertThat(row.getTimestamp(16, 6).getMillisecond()) .isEqualTo(expectedRow.getTimestampLtz(16, 6).getEpochMillisecond()); - // check snapshot in paimon - Map properties = - new HashMap() { - { - put( - FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY, - String.format( - "[{\"partition_id\":%d,\"bucket\":0,\"offset\":1}]", - partitionId)); - } - }; - checkSnapshotPropertyInPaimon(t1, properties); + checkFlussOffsetsInSnapshot(t1, Collections.singletonMap(t1Bucket, 1L)); } } finally { jobClient.cancel().get(); @@ -374,16 +349,7 @@ void testTieringForAlterTable() throws Exception { assertReplicaStatus(t1Bucket, 3); // check data in paimon checkDataInPaimonPrimaryKeyTable(t1, rows); - // check snapshot property in paimon - Map properties = - new HashMap() { - { - put( - FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY, - "[{\"bucket\":0,\"offset\":3}]"); - } - }; - checkSnapshotPropertyInPaimon(t1, properties); + checkFlussOffsetsInSnapshot(t1, Collections.singletonMap(t1Bucket, 3L)); // then, create another log table TablePath t2 = TablePath.of(DEFAULT_DB, "logTableAlter"); @@ -444,9 +410,11 @@ void testTieringForAlterTable() throws Exception { long tableId = tableIdAndDescriptor.f0; // wait until synced to paimon + Map expectedOffset = new HashMap<>(); for (Long partitionId : partitionNameByIds.keySet()) { TableBucket tableBucket = new TableBucket(tableId, partitionId, 0); assertReplicaStatus(tableBucket, 3); + expectedOffset.put(tableBucket, 3L); } // now, let's check data in paimon per partition @@ -460,35 +428,12 @@ void testTieringForAlterTable() throws Exception { 0); } - properties = - new HashMap() { - { - put( - FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY, - getPartitionOffsetStr(partitionNameByIds)); - } - }; - checkSnapshotPropertyInPaimon(partitionedTablePath, properties); + checkFlussOffsetsInSnapshot(partitionedTablePath, expectedOffset); } finally { jobClient.cancel().get(); } } - private String getPartitionOffsetStr(Map partitionNameByIds) { - String raw = "{\"partition_id\":%s,\"bucket\":0,\"offset\":3}"; - List partitionIds = new ArrayList<>(partitionNameByIds.keySet()); - Collections.sort(partitionIds); - List partitionOffsetStrs = new ArrayList<>(); - - for (Long partitionId : partitionIds) { - String partitionName = partitionNameByIds.get(partitionId); - String partitionOffsetStr = String.format(raw, partitionId, partitionName); - partitionOffsetStrs.add(partitionOffsetStr); - } - - return "[" + String.join(",", partitionOffsetStrs) + "]"; - } - @Test void testTieringToDvEnabledTable() throws Exception { TablePath t1 = TablePath.of(DEFAULT_DB, "pkTableWithDv"); 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 b1b4adc9c5..602180b8c1 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 @@ -71,8 +71,6 @@ import java.util.Map; import java.util.stream.Stream; -import static org.apache.fluss.flink.tiering.committer.TieringCommitOperator.fromLogOffsetProperty; -import static org.apache.fluss.flink.tiering.committer.TieringCommitOperator.toBucketOffsetsProperty; import static org.apache.fluss.lake.committer.BucketOffset.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY; import static org.apache.fluss.lake.paimon.utils.PaimonConversions.toPaimon; import static org.apache.fluss.metadata.TableDescriptor.BUCKET_COLUMN_NAME; @@ -189,20 +187,6 @@ void testTieringWriteTable(boolean isPrimaryKeyTable, boolean isPartitioned) thr // use snapshot id 0 as the known snapshot id CommittedLakeSnapshot committedLakeSnapshot = lakeCommitter.getMissingLakeSnapshot(0L); assertThat(committedLakeSnapshot).isNotNull(); - long tableId = tableInfo.getTableId(); - Map offsets = - fromLogOffsetProperty( - tableId, - committedLakeSnapshot - .getSnapshotProperties() - .get(FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY)); - for (int bucket = 0; bucket < 3; bucket++) { - for (Long partitionId : partitionIdAndName.keySet()) { - // we only write 10 records, so expected log offset should be 10 - assertThat(offsets.get(new TableBucket(tableId, partitionId, bucket))) - .isEqualTo(10); - } - } assertThat(committedLakeSnapshot.getLakeSnapshotId()).isOne(); // use null as the known snapshot id @@ -274,8 +258,7 @@ void testMultiPartitionTiering() throws Exception { try (LakeCommitter lakeCommitter = createLakeCommitter(tablePath, tableInfo, new Configuration())) { PaimonCommittable committable = lakeCommitter.toCommittable(paimonWriteResults); - long snapshot = - lakeCommitter.commit(committable, toBucketOffsetsProperty(tableBucketOffsets)); + long snapshot = lakeCommitter.commit(committable, Collections.emptyMap()); assertThat(snapshot).isEqualTo(1); } @@ -310,7 +293,6 @@ void testThreePartitionTiering() throws Exception { TableInfo tableInfo = TableInfo.of(tablePath, 0, 1, descriptor, 1L, 1L); Map> recordsByPartition = new HashMap<>(); List paimonWriteResults = new ArrayList<>(); - Map tableBucketOffsets = new HashMap<>(); // Test data for different three-level partitions using $ separator Map partitionIdAndName = @@ -334,7 +316,6 @@ void testThreePartitionTiering() throws Exception { for (LogRecord logRecord : logRecords) { lakeWriter.write(logRecord); } - tableBucketOffsets.put(new TableBucket(0, entry.getKey(), bucket), 2L); PaimonWriteResult result = lakeWriter.complete(); paimonWriteResults.add(result); @@ -343,20 +324,18 @@ void testThreePartitionTiering() throws Exception { // Commit all data long snapshot; + Map snapshotProperties = + Collections.singletonMap( + FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY, "/path/to/snapshot1"); try (LakeCommitter lakeCommitter = createLakeCommitter(tablePath, tableInfo, new Configuration())) { PaimonCommittable committable = lakeCommitter.toCommittable(paimonWriteResults); - snapshot = - lakeCommitter.commit(committable, toBucketOffsetsProperty(tableBucketOffsets)); + snapshot = lakeCommitter.commit(committable, snapshotProperties); assertThat(snapshot).isEqualTo(1); } // check fluss offsets in paimon snapshot property - String offsetProperty = getSnapshotLogOffsetProperty(tablePath, snapshot); - assertThat(offsetProperty) - .isEqualTo( - "[{\"partition_id\":1,\"bucket\":0,\"offset\":2}," - + "{\"partition_id\":2,\"bucket\":0,\"offset\":2}]"); + assertThat(getSnapshotProperties(tablePath, snapshot)).isEqualTo(snapshotProperties); // Verify data for each partition for (String partition : partitionIdAndName.values()) { @@ -844,15 +823,11 @@ private void doCreatePaimonTable(TablePath tablePath, Schema.Builder paimonSchem paimonCatalog.createTable(toPaimon(tablePath), paimonSchemaBuilder.build(), true); } - private String getSnapshotLogOffsetProperty(TablePath tablePath, long snapshotId) + private Map getSnapshotProperties(TablePath tablePath, long snapshotId) throws Exception { Identifier identifier = toPaimon(tablePath); FileStoreTable fileStoreTable = (FileStoreTable) paimonCatalog.getTable(identifier); - return fileStoreTable - .snapshotManager() - .snapshot(snapshotId) - .properties() - .get(FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY); + return fileStoreTable.snapshotManager().snapshot(snapshotId).properties(); } private void writeData( @@ -879,7 +854,6 @@ private void writeData( SimpleVersionedSerializer committableSerializer = paimonLakeTieringFactory.getCommittableSerializer(); - Map tableBucketOffsets = new HashMap<>(); // first, write data for (int bucket = 0; bucket < bucketNum; bucket++) { for (Map.Entry entry : partitionIdAndName.entrySet()) { @@ -894,7 +868,6 @@ private void writeData( List writtenRecords = writeAndExpectRecords.f0; List expectRecords = writeAndExpectRecords.f1; recordsByBucket.put(partitionBucket, expectRecords); - tableBucketOffsets.put(new TableBucket(0, entry.getKey(), bucket), 10L); for (LogRecord logRecord : writtenRecords) { lakeWriter.write(logRecord); } @@ -917,7 +890,10 @@ private void writeData( paimonCommittable = committableSerializer.deserialize( committableSerializer.getVersion(), serialized); - lakeCommitter.commit(paimonCommittable, toBucketOffsetsProperty(tableBucketOffsets)); + + Map snapshotProperties = + Collections.singletonMap(FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY, "offsets"); + lakeCommitter.commit(paimonCommittable, snapshotProperties); } } } 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..c31e1796fe 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 @@ -46,6 +46,8 @@ import org.apache.fluss.rpc.messages.ListTablesResponse; import org.apache.fluss.rpc.messages.MetadataRequest; import org.apache.fluss.rpc.messages.MetadataResponse; +import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotRequest; +import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotResponse; import org.apache.fluss.rpc.messages.TableExistsRequest; import org.apache.fluss.rpc.messages.TableExistsResponse; import org.apache.fluss.rpc.protocol.ApiKeys; @@ -193,4 +195,34 @@ CompletableFuture getLatestLakeSnapshot( @RPC(api = ApiKeys.DESCRIBE_CLUSTER_CONFIGS) CompletableFuture describeClusterConfigs( DescribeClusterConfigsRequest request); + + /** + * Prepares to commit lake table snapshots by merging them with existing snapshots and storing + * them to the file system. + * + *

This method is called during the two-phase commit process for lake table snapshots. It + * performs the following operations for each table in the request: + * + *

    + *
  • Merges the new snapshot with the previous latest snapshot (if exists) to ensure + * completeness + *
  • Stores the merged snapshot to the remote file system. The stored file contains the log + * end offset information for each bucket in the table + *
  • Returns the file path where the snapshot is stored + *
+ * + *

The returned file path points to a file that stores the bucket log end offset information + * for the table. This file path will be used in the subsequent commit phase to reference the + * stored snapshot. If any error occurs during processing for a specific table, an error + * response will be set for that table in the response, while other tables will continue to be + * processed. + * + * @param request the request containing lake table snapshot information for one or more tables + * @return a future that completes with a response containing the file paths where snapshots + * (containing bucket log end offset information) are stored, or error information for + * tables that failed to process + */ + @RPC(api = ApiKeys.PRECOMMIT_LAKE_TABLE_SNAPSHOT) + CompletableFuture prepareCommitLakeTableSnapshot( + PrepareCommitLakeTableSnapshotRequest request); } 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 388ec081dd..10305da283 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 @@ -79,7 +79,8 @@ public enum ApiKeys { REMOVE_SERVER_TAG(1048, 0, 0, PUBLIC), REBALANCE(1049, 0, 0, PUBLIC), LIST_REBALANCE_PROGRESS(1050, 0, 0, PUBLIC), - CANCEL_REBALANCE(1051, 0, 0, PUBLIC); + CANCEL_REBALANCE(1051, 0, 0, PUBLIC), + PRECOMMIT_LAKE_TABLE_SNAPSHOT(1052, 0, 0, PRIVATE); private static final Map ID_TO_TYPE = Arrays.stream(ApiKeys.values()) diff --git a/fluss-rpc/src/main/proto/FlussApi.proto b/fluss-rpc/src/main/proto/FlussApi.proto index f5dae90c7e..ec3560c81d 100644 --- a/fluss-rpc/src/main/proto/FlussApi.proto +++ b/fluss-rpc/src/main/proto/FlussApi.proto @@ -449,14 +449,48 @@ message NotifyRemoteLogOffsetsRequest { message NotifyRemoteLogOffsetsResponse { } +message PrepareCommitLakeTableSnapshotRequest { + repeated PbLakeTableSnapshotInfo tables_req = 1; +} + +message PrepareCommitLakeTableSnapshotResponse { + repeated PbPrepareCommitLakeTableRespForTable prepare_commit_lake_table_resp = 1; +} + +message PbPrepareCommitLakeTableRespForTable { + optional string lake_table_snapshot_file_path = 1; + optional int32 error_code = 2; + optional string error_message = 3; +} + message CommitLakeTableSnapshotRequest { + // Deprecated: PbLakeTableSnapshotInfo is no longer used for committing lake table snapshots. + // Currently, it is only used to allow the coordinator to notify tablet servers about the current + // synchronized log end offsets, which are then reported to metrics. In the future, we plan to + // have tiering directly report to metrics, and this field will be fully removed. + // Still reserve it for protocol compatibility. repeated PbLakeTableSnapshotInfo tables_req = 1; + // The metadata for lake table snapshots to be committed. Each entry contains the table ID, + // snapshot ID, and the file paths where the snapshot data (containing bucket log end offset + // information) is stored. The tiered_snapshot_file_path points to the file storing tiered log + // end offsets, while readable_snapshot_file_path (if present) points to the file storing readable + // log end offsets. + repeated PbLakeTableSnapshotMetadata lake_table_snapshot_metadata = 2; +} + +message PbLakeTableSnapshotMetadata { + required int64 table_id = 1; + required int64 snapshot_id = 2; + required string tiered_snapshot_file_path = 3; + optional string readable_snapshot_file_path = 4; } message PbLakeTableSnapshotInfo { optional int64 table_id = 1; required int64 snapshot_id = 2; repeated PbLakeTableOffsetForBucket buckets_req = 3; + // add table path to reduce get table_path by table id + optional PbTablePath table_path = 4; } message PbLakeTableOffsetForBucket { 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..0fa36373d4 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 @@ -67,6 +67,8 @@ import org.apache.fluss.rpc.messages.NotifyRemoteLogOffsetsResponse; import org.apache.fluss.rpc.messages.PrefixLookupRequest; import org.apache.fluss.rpc.messages.PrefixLookupResponse; +import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotRequest; +import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotResponse; import org.apache.fluss.rpc.messages.ProduceLogRequest; import org.apache.fluss.rpc.messages.ProduceLogResponse; import org.apache.fluss.rpc.messages.PutKvRequest; @@ -249,4 +251,10 @@ public CompletableFuture describeClusterConfigs( DescribeClusterConfigsRequest request) { return null; } + + @Override + public CompletableFuture prepareCommitLakeTableSnapshot( + PrepareCommitLakeTableSnapshotRequest request) { + throw new UnsupportedOperationException(); + } } 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..0b78264ba3 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 @@ -19,6 +19,8 @@ import org.apache.fluss.cluster.ServerNode; import org.apache.fluss.cluster.ServerType; +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.config.Configuration; import org.apache.fluss.config.cluster.ConfigEntry; import org.apache.fluss.exception.FlussRuntimeException; import org.apache.fluss.exception.KvSnapshotNotExistException; @@ -29,6 +31,7 @@ import org.apache.fluss.exception.SecurityTokenException; import org.apache.fluss.exception.TableNotPartitionedException; import org.apache.fluss.fs.FileSystem; +import org.apache.fluss.fs.FsPath; import org.apache.fluss.fs.token.ObtainedSecurityToken; import org.apache.fluss.metadata.DatabaseInfo; import org.apache.fluss.metadata.PhysicalTablePath; @@ -71,12 +74,18 @@ import org.apache.fluss.rpc.messages.MetadataRequest; import org.apache.fluss.rpc.messages.MetadataResponse; import org.apache.fluss.rpc.messages.PbApiVersion; +import org.apache.fluss.rpc.messages.PbLakeTableSnapshotInfo; +import org.apache.fluss.rpc.messages.PbPrepareCommitLakeTableRespForTable; import org.apache.fluss.rpc.messages.PbTablePath; +import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotRequest; +import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotResponse; import org.apache.fluss.rpc.messages.TableExistsRequest; import org.apache.fluss.rpc.messages.TableExistsResponse; import org.apache.fluss.rpc.netty.server.Session; +import org.apache.fluss.rpc.protocol.ApiError; import org.apache.fluss.rpc.protocol.ApiKeys; import org.apache.fluss.rpc.protocol.ApiManager; +import org.apache.fluss.rpc.protocol.Errors; import org.apache.fluss.security.acl.AclBinding; import org.apache.fluss.security.acl.AclBindingFilter; import org.apache.fluss.security.acl.OperationType; @@ -93,6 +102,7 @@ import org.apache.fluss.server.utils.ServerRpcMessageUtils; import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.server.zk.data.BucketSnapshot; +import org.apache.fluss.server.zk.data.lake.LakeTableHelper; import org.apache.fluss.server.zk.data.lake.LakeTableSnapshot; import org.slf4j.Logger; @@ -120,6 +130,7 @@ 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; +import static org.apache.fluss.server.utils.ServerRpcMessageUtils.toLakeSnapshot; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.toListPartitionInfosResponse; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.toPbConfigEntries; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.toTablePath; @@ -141,6 +152,7 @@ public abstract class RpcServiceBase extends RpcGatewayService implements AdminR protected final MetadataManager metadataManager; protected final @Nullable Authorizer authorizer; protected final DynamicConfigManager dynamicConfigManager; + private final LakeTableHelper lakeTableHelper; private long tokenLastUpdateTimeMs = 0; private ObtainedSecurityToken securityToken = null; @@ -154,7 +166,8 @@ public RpcServiceBase( MetadataManager metadataManager, @Nullable Authorizer authorizer, DynamicConfigManager dynamicConfigManager, - ExecutorService ioExecutor) { + ExecutorService ioExecutor, + Configuration config) { this.remoteFileSystem = remoteFileSystem; this.provider = provider; this.apiManager = new ApiManager(provider); @@ -163,6 +176,8 @@ public RpcServiceBase( this.authorizer = authorizer; this.dynamicConfigManager = dynamicConfigManager; this.ioExecutor = ioExecutor; + this.lakeTableHelper = + new LakeTableHelper(zkClient, config.get(ConfigOptions.REMOTE_DATA_DIR)); } @Override @@ -591,4 +606,50 @@ protected MetadataResponse processMetadataRequest( return buildMetadataResponse( coordinatorServer, aliveTabletServers, tablesMetadata, partitionsMetadata); } + + @Override + public CompletableFuture prepareCommitLakeTableSnapshot( + PrepareCommitLakeTableSnapshotRequest request) { + CompletableFuture future = + new CompletableFuture<>(); + ioExecutor.submit( + () -> { + PrepareCommitLakeTableSnapshotResponse response = + new PrepareCommitLakeTableSnapshotResponse(); + try { + for (PbLakeTableSnapshotInfo pbLakeTableSnapshotInfo : + request.getTablesReqsList()) { + PbPrepareCommitLakeTableRespForTable + pbPrepareCommitLakeTableRespForTable = + response.addPrepareCommitLakeTableResp(); + try { + // upsert lake table snapshot, need to merge the snapshot with + // previous + // latest snapshot + LakeTableSnapshot lakeTableSnapshot = + lakeTableHelper.upsertLakeTableSnapshot( + pbLakeTableSnapshotInfo.getTableId(), + toLakeSnapshot(pbLakeTableSnapshotInfo)); + TablePath tablePath = + toTablePath(pbLakeTableSnapshotInfo.getTablePath()); + FsPath fsPath = + lakeTableHelper.storeLakeTableSnapshot( + pbLakeTableSnapshotInfo.getTableId(), + tablePath, + lakeTableSnapshot); + pbPrepareCommitLakeTableRespForTable.setLakeTableSnapshotFilePath( + fsPath.toString()); + } catch (Exception e) { + Errors error = ApiError.fromThrowable(e).error(); + pbPrepareCommitLakeTableRespForTable.setError( + error.code(), error.message()); + } + } + future.complete(response); + } catch (Exception e) { + future.completeExceptionally(e); + } + }); + return future; + } } 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 4a6fd89f06..2d715faea5 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 @@ -90,6 +90,7 @@ 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; @@ -1224,6 +1225,48 @@ private void processAccessContext(AccessContextEvent event) { private void tryProcessCommitLakeTableSnapshot( CommitLakeTableSnapshotEvent commitLakeTableSnapshotEvent, CompletableFuture callback) { + CommitLakeTableSnapshotData commitLakeTableSnapshotData = + commitLakeTableSnapshotEvent.getCommitLakeTableSnapshotData(); + if (commitLakeTableSnapshotData.getLakeTableSnapshotMetadatas().isEmpty()) { + handleCommitLakeTableSnapshotV1(commitLakeTableSnapshotEvent, callback); + } else { + Map lakeSnapshotMetadatas = + commitLakeTableSnapshotData.getLakeTableSnapshotMetadatas(); + ioExecutor.execute( + () -> { + try { + CommitLakeTableSnapshotResponse response = + new CommitLakeTableSnapshotResponse(); + for (Map.Entry + lakeSnapshotMetadataEntry : lakeSnapshotMetadatas.entrySet()) { + PbCommitLakeTableSnapshotRespForTable tableResp = + response.addTableResp(); + long tableId = lakeSnapshotMetadataEntry.getKey(); + tableResp.setTableId(tableId); + try { + lakeTableHelper.addLakeTableSnapshotMetadata( + tableId, lakeSnapshotMetadataEntry.getValue()); + } catch (Exception e) { + ApiError error = ApiError.fromThrowable(e); + tableResp.setError(error.error().code(), error.message()); + } + } + coordinatorEventManager.put( + new NotifyLakeTableOffsetEvent( + commitLakeTableSnapshotData.getLakeTableSnapshot(), + commitLakeTableSnapshotData + .getTableBucketsMaxTieredTimestamp())); + callback.complete(response); + } catch (Exception e) { + callback.completeExceptionally(e); + } + }); + } + } + + private void handleCommitLakeTableSnapshotV1( + CommitLakeTableSnapshotEvent commitLakeTableSnapshotEvent, + CompletableFuture callback) { // commit the lake table snapshot asynchronously CommitLakeTableSnapshotData commitLakeTableSnapshotData = commitLakeTableSnapshotEvent.getCommitLakeTableSnapshotData(); 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 9eee45e600..11b486bab4 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 @@ -186,7 +186,8 @@ public CoordinatorService( metadataManager, authorizer, dynamicConfigManager, - ioExecutor); + ioExecutor, + conf); this.defaultBucketNumber = conf.getInt(ConfigOptions.DEFAULT_BUCKET_NUMBER); this.defaultReplicationFactor = conf.getInt(ConfigOptions.DEFAULT_REPLICATION_FACTOR); this.logTableAllowCreation = conf.getBoolean(ConfigOptions.LOG_TABLE_ALLOW_CREATION); 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 index 21ac1b340f..9a913cd4aa 100644 --- 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 @@ -19,6 +19,7 @@ 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; @@ -27,14 +28,32 @@ /** 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 tableBucketsMaxTieredTimestamp; + // the following field only non-empty since 0.9 + private final Map lakeTableSnapshotMetadatas; + public CommitLakeTableSnapshotData( Map lakeTableSnapshots, - Map tableBucketsMaxTieredTimestamp) { + Map tableBucketsMaxTieredTimestamp, + Map lakeTableSnapshotMetadatas) { this.lakeTableSnapshots = lakeTableSnapshots; this.tableBucketsMaxTieredTimestamp = tableBucketsMaxTieredTimestamp; + this.lakeTableSnapshotMetadatas = lakeTableSnapshotMetadatas; } public Map getLakeTableSnapshot() { @@ -45,6 +64,10 @@ public Map getTableBucketsMaxTieredTimestamp() { return tableBucketsMaxTieredTimestamp; } + public Map getLakeTableSnapshotMetadatas() { + return lakeTableSnapshotMetadatas; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -56,12 +79,14 @@ public boolean equals(Object o) { CommitLakeTableSnapshotData that = (CommitLakeTableSnapshotData) o; return Objects.equals(lakeTableSnapshots, that.lakeTableSnapshots) && Objects.equals( - tableBucketsMaxTieredTimestamp, that.tableBucketsMaxTieredTimestamp); + tableBucketsMaxTieredTimestamp, that.tableBucketsMaxTieredTimestamp) + && Objects.equals(lakeTableSnapshotMetadatas, that.lakeTableSnapshotMetadatas); } @Override public int hashCode() { - return Objects.hash(lakeTableSnapshots, tableBucketsMaxTieredTimestamp); + return Objects.hash( + lakeTableSnapshots, tableBucketsMaxTieredTimestamp, lakeTableSnapshotMetadatas); } @Override @@ -71,6 +96,8 @@ public String toString() { + lakeTableSnapshots + ", tableBucketsMaxTieredTimestamp=" + tableBucketsMaxTieredTimestamp + + ", lakeTableSnapshotMetadatas=" + + lakeTableSnapshotMetadatas + '}'; } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletServer.java b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletServer.java index 8eed63c844..aa8e6348d7 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletServer.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletServer.java @@ -286,7 +286,8 @@ protected void startServices() throws Exception { metadataManager, authorizer, dynamicConfigManager, - ioExecutor); + ioExecutor, + conf); RequestsMetrics requestsMetrics = RequestsMetrics.createTabletServerRequestMetrics(tabletServerMetricGroup); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java index 9a47b63830..88dfcae96b 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java @@ -18,6 +18,7 @@ package org.apache.fluss.server.tablet; import org.apache.fluss.cluster.ServerType; +import org.apache.fluss.config.Configuration; import org.apache.fluss.exception.AuthorizationException; import org.apache.fluss.exception.UnknownTableOrBucketException; import org.apache.fluss.fs.FileSystem; @@ -137,7 +138,8 @@ public TabletService( MetadataManager metadataManager, @Nullable Authorizer authorizer, DynamicConfigManager dynamicConfigManager, - ExecutorService ioExecutor) { + ExecutorService ioExecutor, + Configuration conf) { super( remoteFileSystem, ServerType.TABLET_SERVER, @@ -145,7 +147,8 @@ public TabletService( metadataManager, authorizer, dynamicConfigManager, - ioExecutor); + ioExecutor, + conf); this.serviceName = "server-" + serverId; this.replicaManager = replicaManager; this.metadataCache = metadataCache; 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 ed1051af94..e2732a8d08 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 @@ -102,6 +102,7 @@ import org.apache.fluss.rpc.messages.PbLakeSnapshotForBucket; import org.apache.fluss.rpc.messages.PbLakeTableOffsetForBucket; import org.apache.fluss.rpc.messages.PbLakeTableSnapshotInfo; +import org.apache.fluss.rpc.messages.PbLakeTableSnapshotMetadata; import org.apache.fluss.rpc.messages.PbListOffsetsRespForBucket; import org.apache.fluss.rpc.messages.PbLookupReqForBucket; import org.apache.fluss.rpc.messages.PbLookupRespForBucket; @@ -165,9 +166,11 @@ import org.apache.fluss.server.metadata.TableMetadata; import org.apache.fluss.server.zk.data.BucketSnapshot; import org.apache.fluss.server.zk.data.LeaderAndIsr; +import org.apache.fluss.server.zk.data.lake.LakeTable; import org.apache.fluss.server.zk.data.lake.LakeTableSnapshot; import org.apache.fluss.utils.json.DataTypeJsonSerde; import org.apache.fluss.utils.json.JsonSerdeUtils; +import org.apache.fluss.utils.types.Tuple2; import javax.annotation.Nullable; @@ -184,6 +187,7 @@ import java.util.Optional; import java.util.OptionalInt; import java.util.Set; +import java.util.function.Consumer; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -1544,37 +1548,77 @@ public static PbPartitionSpec makePbPartitionSpec(ResolvedPartitionSpec spec) { public static CommitLakeTableSnapshotData getCommitLakeTableSnapshotData( CommitLakeTableSnapshotRequest request) { + // handle rpc before 0.9 Map lakeTableInfoByTableId = new HashMap<>(); Map tableBucketsMaxTimestamp = new HashMap<>(); - for (PbLakeTableSnapshotInfo pdLakeTableSnapshotInfo : request.getTablesReqsList()) { - long tableId = pdLakeTableSnapshotInfo.getTableId(); - long snapshotId = pdLakeTableSnapshotInfo.getSnapshotId(); - Map bucketLogEndOffset = new HashMap<>(); - - for (PbLakeTableOffsetForBucket lakeTableOffsetForBucket : - pdLakeTableSnapshotInfo.getBucketsReqsList()) { - Long partitionId = - lakeTableOffsetForBucket.hasPartitionId() - ? lakeTableOffsetForBucket.getPartitionId() - : null; - int bucketId = lakeTableOffsetForBucket.getBucketId(); - - TableBucket tableBucket = new TableBucket(tableId, partitionId, bucketId); - Long logEndOffset = - lakeTableOffsetForBucket.hasLogEndOffset() - ? lakeTableOffsetForBucket.getLogEndOffset() - : null; - bucketLogEndOffset.put(tableBucket, logEndOffset); - - if (lakeTableOffsetForBucket.hasMaxTimestamp()) { - tableBucketsMaxTimestamp.put( - tableBucket, lakeTableOffsetForBucket.getMaxTimestamp()); - } + for (PbLakeTableSnapshotInfo pbLakeTableSnapshotInfo : request.getTablesReqsList()) { + long tableId = pbLakeTableSnapshotInfo.getTableId(); + Consumer> + pbLakeTableOffsetForBucketConsumer = + (tableBucketAndOffset -> { + if (tableBucketAndOffset.f1.hasMaxTimestamp()) { + tableBucketsMaxTimestamp.put( + tableBucketAndOffset.f0, + tableBucketAndOffset.f1.getMaxTimestamp()); + } + }); + LakeTableSnapshot lakeTableSnapshot = + toLakeSnapshot(pbLakeTableSnapshotInfo, pbLakeTableOffsetForBucketConsumer); + lakeTableInfoByTableId.put(tableId, lakeTableSnapshot); + } + + // handle rpc since 0.9 + Map lakeSnapshotMetadatas = new HashMap<>(); + for (PbLakeTableSnapshotMetadata pbLakeTableSnapshotMetadata : + request.getLakeTableSnapshotMetadatasList()) { + lakeSnapshotMetadatas.put( + pbLakeTableSnapshotMetadata.getTableId(), + new LakeTable.LakeSnapshotMetadata( + pbLakeTableSnapshotMetadata.getSnapshotId(), + new FsPath(pbLakeTableSnapshotMetadata.getTieredSnapshotFilePath()), + pbLakeTableSnapshotMetadata.hasReadableSnapshotFilePath() + ? new FsPath( + pbLakeTableSnapshotMetadata + .getReadableSnapshotFilePath()) + : null)); + } + return new CommitLakeTableSnapshotData( + lakeTableInfoByTableId, tableBucketsMaxTimestamp, lakeSnapshotMetadatas); + } + + public static LakeTableSnapshot toLakeSnapshot( + PbLakeTableSnapshotInfo pbLakeTableSnapshotInfo) { + return toLakeSnapshot(pbLakeTableSnapshotInfo, null); + } + + private static LakeTableSnapshot toLakeSnapshot( + PbLakeTableSnapshotInfo pbLakeTableSnapshotInfo, + @Nullable + Consumer> + pbLakeTableOffsetForBucketConsumer) { + long tableId = pbLakeTableSnapshotInfo.getTableId(); + long snapshotId = pbLakeTableSnapshotInfo.getSnapshotId(); + Map bucketLogEndOffset = new HashMap<>(); + for (PbLakeTableOffsetForBucket lakeTableOffsetForBucket : + pbLakeTableSnapshotInfo.getBucketsReqsList()) { + Long partitionId = + lakeTableOffsetForBucket.hasPartitionId() + ? lakeTableOffsetForBucket.getPartitionId() + : null; + int bucketId = lakeTableOffsetForBucket.getBucketId(); + + TableBucket tableBucket = new TableBucket(tableId, partitionId, bucketId); + Long logEndOffset = + lakeTableOffsetForBucket.hasLogEndOffset() + ? lakeTableOffsetForBucket.getLogEndOffset() + : null; + if (pbLakeTableOffsetForBucketConsumer != null) { + pbLakeTableOffsetForBucketConsumer.accept( + Tuple2.of(tableBucket, lakeTableOffsetForBucket)); } - lakeTableInfoByTableId.put( - tableId, new LakeTableSnapshot(snapshotId, bucketLogEndOffset)); + bucketLogEndOffset.put(tableBucket, logEndOffset); } - return new CommitLakeTableSnapshotData(lakeTableInfoByTableId, tableBucketsMaxTimestamp); + return new LakeTableSnapshot(snapshotId, bucketLogEndOffset); } public static PbNotifyLakeTableOffsetReqForBucket makeNotifyLakeTableOffsetForBucket( 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 5df57acd57..1665014042 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 @@ -21,6 +21,7 @@ import org.apache.fluss.fs.FSDataInputStream; import org.apache.fluss.fs.FileSystem; import org.apache.fluss.fs.FsPath; +import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.server.zk.data.ZkData; import org.apache.fluss.utils.IOUtils; @@ -30,6 +31,7 @@ import java.io.IOException; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Objects; import static org.apache.fluss.metrics.registry.MetricRegistry.LOG; @@ -116,13 +118,16 @@ public LakeTableSnapshot getLatestTableSnapshot() throws Exception { if (lakeTableSnapshot != null) { return lakeTableSnapshot; } - FsPath tieredOffsetsFilePath = - checkNotNull(getLatestLakeSnapshotMetadata()).tieredOffsetsFilePath; + LakeSnapshotMetadata lakeSnapshotMetadata = getLatestLakeSnapshotMetadata(); + FsPath tieredOffsetsFilePath = checkNotNull(lakeSnapshotMetadata).tieredOffsetsFilePath; FSDataInputStream inputStream = tieredOffsetsFilePath.getFileSystem().open(tieredOffsetsFilePath); try (ByteArrayOutputStream outputStream = new ByteArrayOutputStream()) { IOUtils.copyBytes(inputStream, outputStream, true); - return LakeTableSnapshotJsonSerde.fromJson(outputStream.toByteArray()); + Map logOffsets = + LakeTableSnapshotJsonSerde.fromJson(outputStream.toByteArray()) + .getBucketLogEndOffset(); + return new LakeTableSnapshot(lakeSnapshotMetadata.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 8031b9ddeb..8f9b352c42 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 @@ -27,6 +27,7 @@ import org.apache.fluss.utils.FlussPaths; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Optional; @@ -105,6 +106,39 @@ public void upsertLakeTable( } } + public void addLakeTableSnapshotMetadata( + long tableId, LakeTable.LakeSnapshotMetadata lakeSnapshotMetadata) throws Exception { + Optional optPreviousLakeTable = zkClient.getLakeTable(tableId); + List previousLakeSnapshotMetadatas = null; + if (optPreviousLakeTable.isPresent()) { + previousLakeSnapshotMetadatas = optPreviousLakeTable.get().getLakeSnapshotMetadatas(); + } + // currently, we always keep only one snapshot metadata + LakeTable lakeTable = new LakeTable(lakeSnapshotMetadata); + try { + zkClient.upsertLakeTable(tableId, lakeTable, optPreviousLakeTable.isPresent()); + } catch (Exception e) { + LOG.warn("Failed to upsert lake table snapshot to zk.", e); + throw e; + } + + // discard previous lake snapshot metadata + if (previousLakeSnapshotMetadatas != null) { + previousLakeSnapshotMetadatas.forEach(LakeTable.LakeSnapshotMetadata::discard); + } + } + + public LakeTableSnapshot upsertLakeTableSnapshot( + long tableId, LakeTableSnapshot newLakeTableSnapshot) throws Exception { + Optional optPreviousLakeTable = zkClient.getLakeTable(tableId); + // Merge with previous snapshot if exists + if (optPreviousLakeTable.isPresent()) { + return mergeLakeTable( + optPreviousLakeTable.get().getLatestTableSnapshot(), newLakeTableSnapshot); + } + return newLakeTableSnapshot; + } + private LakeTableSnapshot mergeLakeTable( LakeTableSnapshot previousLakeTableSnapshot, LakeTableSnapshot newLakeTableSnapshot) { // Merge current snapshot with previous one since the current snapshot request @@ -119,7 +153,7 @@ private LakeTableSnapshot mergeLakeTable( return new LakeTableSnapshot(newLakeTableSnapshot.getSnapshotId(), bucketLogEndOffset); } - private FsPath storeLakeTableSnapshot( + public FsPath storeLakeTableSnapshot( long tableId, TablePath tablePath, LakeTableSnapshot lakeTableSnapshot) throws Exception { // get the remote file path to store the lake table snapshot information 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 c8500645e6..41b0c1d125 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 @@ -94,6 +94,8 @@ import org.apache.fluss.rpc.messages.RebalanceResponse; import org.apache.fluss.rpc.messages.RemoveServerTagRequest; import org.apache.fluss.rpc.messages.RemoveServerTagResponse; +import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotRequest; +import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotResponse; import org.apache.fluss.rpc.messages.TableExistsRequest; import org.apache.fluss.rpc.messages.TableExistsResponse; import org.apache.fluss.rpc.protocol.ApiError; @@ -337,6 +339,12 @@ public CompletableFuture commitRemoteLogManifes return CompletableFuture.completedFuture(response.setCommitSuccess(true)); } + @Override + public CompletableFuture prepareCommitLakeTableSnapshot( + PrepareCommitLakeTableSnapshotRequest request) { + throw new UnsupportedOperationException(); + } + @Override public CompletableFuture commitLakeTableSnapshot( CommitLakeTableSnapshotRequest request) { 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 8ebcc57489..cac7b80c16 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 @@ -77,6 +77,8 @@ import org.apache.fluss.rpc.messages.PbTableBucket; import org.apache.fluss.rpc.messages.PrefixLookupRequest; import org.apache.fluss.rpc.messages.PrefixLookupResponse; +import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotRequest; +import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotResponse; import org.apache.fluss.rpc.messages.ProduceLogRequest; import org.apache.fluss.rpc.messages.ProduceLogResponse; import org.apache.fluss.rpc.messages.PutKvRequest; @@ -333,6 +335,12 @@ public CompletableFuture describeClusterConfigs( throw new UnsupportedOperationException(); } + @Override + public CompletableFuture prepareCommitLakeTableSnapshot( + PrepareCommitLakeTableSnapshotRequest request) { + throw new UnsupportedOperationException(); + } + public int pendingRequestSize() { return requests.size(); } From 2128d14b9aec7fd7ba097d03b5bc5f7a0dfc9550 Mon Sep 17 00:00:00 2001 From: luoyuxia Date: Wed, 24 Dec 2025 19:33:38 +0800 Subject: [PATCH 2/7] minor fix --- .../fluss/lake/committer/BucketOffset.java | 72 ----- .../org/apache/fluss/utils/FlussPaths.java | 6 +- .../utils/json/BucketOffsetJsonSerde.java | 62 ----- .../fluss/utils/json/TableBucketOffsets.java | 100 +++++++ .../json/TableBucketOffsetsJsonSerde.java | 242 +++++++++++++++++ .../utils/json/BucketOffsetJsonSerdeTest.java | 40 --- .../json/TableBucketOffsetsJsonSerdeTest.java | 108 ++++++++ .../FlussTableLakeSnapshotCommitter.java | 57 ++-- .../FlussTableLakeSnapshotCommitterTest.java | 116 +++++--- .../CoordinatorEventProcessor.java | 4 +- .../fluss/server/zk/data/lake/LakeTable.java | 13 +- .../server/zk/data/lake/LakeTableHelper.java | 47 +++- .../zk/data/lake/LakeTableJsonSerde.java | 39 +-- .../data/lake/LakeTableSnapshotJsonSerde.java | 253 ++---------------- .../data/LakeTableSnapshotJsonSerdeTest.java | 104 ++----- .../zk/data/lake/LakeTableHelperTest.java | 12 +- .../zk/data/lake/LakeTableJsonSerdeTest.java | 91 ++++++- 17 files changed, 771 insertions(+), 595 deletions(-) delete mode 100644 fluss-common/src/main/java/org/apache/fluss/lake/committer/BucketOffset.java delete mode 100644 fluss-common/src/main/java/org/apache/fluss/utils/json/BucketOffsetJsonSerde.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/utils/json/TableBucketOffsets.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/utils/json/TableBucketOffsetsJsonSerde.java delete mode 100644 fluss-common/src/test/java/org/apache/fluss/utils/json/BucketOffsetJsonSerdeTest.java create mode 100644 fluss-common/src/test/java/org/apache/fluss/utils/json/TableBucketOffsetsJsonSerdeTest.java diff --git a/fluss-common/src/main/java/org/apache/fluss/lake/committer/BucketOffset.java b/fluss-common/src/main/java/org/apache/fluss/lake/committer/BucketOffset.java deleted file mode 100644 index a8731da155..0000000000 --- a/fluss-common/src/main/java/org/apache/fluss/lake/committer/BucketOffset.java +++ /dev/null @@ -1,72 +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.lake.committer; - -import javax.annotation.Nullable; - -import java.io.Serializable; -import java.util.Objects; - -/** The bucket offset information to be expected to be stored in Lake's snapshot property. */ -public class BucketOffset implements Serializable { - - private static final long serialVersionUID = 1L; - public static final String FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY = "fluss-offsets"; - - private final long logOffset; - private final int bucket; - private final @Nullable Long partitionId; - - public BucketOffset(long logOffset, int bucket, @Nullable Long partitionId) { - this.logOffset = logOffset; - this.bucket = bucket; - this.partitionId = partitionId; - } - - public long getLogOffset() { - return logOffset; - } - - public int getBucket() { - return bucket; - } - - @Nullable - public Long getPartitionId() { - return partitionId; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - BucketOffset that = (BucketOffset) o; - return logOffset == that.logOffset - && bucket == that.bucket - && Objects.equals(partitionId, that.partitionId); - } - - @Override - public int hashCode() { - return Objects.hash(logOffset, bucket, partitionId); - } -} diff --git a/fluss-common/src/main/java/org/apache/fluss/utils/FlussPaths.java b/fluss-common/src/main/java/org/apache/fluss/utils/FlussPaths.java index 8c63113780..d04268fbaa 100644 --- a/fluss-common/src/main/java/org/apache/fluss/utils/FlussPaths.java +++ b/fluss-common/src/main/java/org/apache/fluss/utils/FlussPaths.java @@ -710,14 +710,14 @@ public static FsPath remoteLakeTableSnapshotDir( *

The path contract: * *

-     * {$remoteLakeTableSnapshotMetadataDir}/metadata/{uuid}.manifest
+     * {$remoteLakeTableSnapshotMetadataDir}/metadata/{UUID}.offsets
      * 
*/ - public static FsPath remoteLakeTableSnapshotManifestPath( + public static FsPath remoteLakeTableSnapshotOffsetPath( String remoteDataDir, TablePath tablePath, long tableId) { return new FsPath( String.format( - "%s/metadata/%s.manifest", + "%s/metadata/%s.offsets", remoteLakeTableSnapshotDir(remoteDataDir, tablePath, tableId), UUID.randomUUID())); } diff --git a/fluss-common/src/main/java/org/apache/fluss/utils/json/BucketOffsetJsonSerde.java b/fluss-common/src/main/java/org/apache/fluss/utils/json/BucketOffsetJsonSerde.java deleted file mode 100644 index 235268e682..0000000000 --- a/fluss-common/src/main/java/org/apache/fluss/utils/json/BucketOffsetJsonSerde.java +++ /dev/null @@ -1,62 +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.utils.json; - -import org.apache.fluss.lake.committer.BucketOffset; -import org.apache.fluss.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; -import org.apache.fluss.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; - -import java.io.IOException; - -/** Json serializer and deserializer for {@link BucketOffset}. */ -public class BucketOffsetJsonSerde - implements JsonSerializer, JsonDeserializer { - - public static final BucketOffsetJsonSerde INSTANCE = new BucketOffsetJsonSerde(); - private static final String PARTITION_ID = "partition_id"; - private static final String BUCKET_ID = "bucket"; - private static final String LOG_OFFSET = "offset"; - - @Override - public BucketOffset deserialize(JsonNode node) { - JsonNode partitionIdNode = node.get(PARTITION_ID); - Long partitionId = partitionIdNode == null ? null : partitionIdNode.asLong(); - int bucketId = node.get(BUCKET_ID).asInt(); - - // deserialize log offset - long logOffset = node.get(LOG_OFFSET).asLong(); - - return new BucketOffset(logOffset, bucketId, partitionId); - } - - @Override - public void serialize(BucketOffset bucketOffset, JsonGenerator generator) throws IOException { - generator.writeStartObject(); - - // write partition id - if (bucketOffset.getPartitionId() != null) { - generator.writeNumberField(PARTITION_ID, bucketOffset.getPartitionId()); - } - generator.writeNumberField(BUCKET_ID, bucketOffset.getBucket()); - - // serialize bucket offset - generator.writeNumberField(LOG_OFFSET, bucketOffset.getLogOffset()); - - generator.writeEndObject(); - } -} diff --git a/fluss-common/src/main/java/org/apache/fluss/utils/json/TableBucketOffsets.java b/fluss-common/src/main/java/org/apache/fluss/utils/json/TableBucketOffsets.java new file mode 100644 index 0000000000..d66b524b6d --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/utils/json/TableBucketOffsets.java @@ -0,0 +1,100 @@ +/* + * 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.utils.json; + +import org.apache.fluss.metadata.TableBucket; + +import java.util.Map; +import java.util.Objects; + +/** + * Represents the offsets for all buckets of a table. This class stores the mapping from {@link + * TableBucket} to their corresponding offsets. + * + *

This class is used to track the log end offsets for each bucket in a table. It supports both + * non-partitioned tables (where buckets are identified only by bucket id) and partitioned tables + * (where buckets are identified by partition id and bucket id). + * + *

The offsets map contains entries for each bucket that has a valid offset. Missing buckets are + * not included in the map. + * + * @see TableBucketOffsetsJsonSerde for JSON serialization and deserialization. + */ +public class TableBucketOffsets { + + /** The table ID that all buckets belong to. */ + private final long tableId; + + /** + * The mapping from {@link TableBucket} to their offsets. The map contains entries only for + * buckets that have valid offsets. + */ + private final Map offsets; + + /** + * Creates a new {@link TableBucketOffsets} instance. + * + * @param tableId the table ID that all buckets belong to + * @param offsets the mapping from {@link TableBucket} to their offsets + */ + public TableBucketOffsets(long tableId, Map offsets) { + this.tableId = tableId; + this.offsets = offsets; + } + + /** + * Returns the table ID that all buckets belong to. + * + * @return the table ID + */ + public long getTableId() { + return tableId; + } + + /** + * Returns the mapping from {@link TableBucket} to their offsets. + * + * @return the offsets map + */ + public Map getOffsets() { + return offsets; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TableBucketOffsets that = (TableBucketOffsets) o; + return tableId == that.tableId && Objects.equals(offsets, that.offsets); + } + + @Override + public int hashCode() { + return Objects.hash(tableId, offsets); + } + + @Override + public String toString() { + return "TableBucketOffsets{" + "tableId=" + tableId + ", offsets=" + offsets + '}'; + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/utils/json/TableBucketOffsetsJsonSerde.java b/fluss-common/src/main/java/org/apache/fluss/utils/json/TableBucketOffsetsJsonSerde.java new file mode 100644 index 0000000000..20355810ec --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/utils/json/TableBucketOffsetsJsonSerde.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.utils.json; + +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; +import org.apache.fluss.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +import static org.apache.fluss.utils.Preconditions.checkNotNull; +import static org.apache.fluss.utils.Preconditions.checkState; + +/** + * Json serializer and deserializer for {@link TableBucketOffsets}. + * + *

This serde supports the following JSON format: + * + *

    + *
  • Non-partition table uses "bucket_offsets": [1234, 5678, 1992], where array index represents + * bucket id (0, 1, 2) and value represents the offset. Missing bucket ids in the sequence are + * filled with -1. + *
  • Partition table uses "partition_offsets": [{"partition_id": 3001, "bucket_offsets": [1234, + * 5678, 1992]}, ...], where each element contains a partition_id and a bucket_offsets array. + * The array index in bucket_offsets represents bucket id and value represents the offset. + * Missing bucket ids in the sequence are filled with -1. + *
+ * + *

During deserialization, values of -1 are ignored and not added to the offsets map. + * + *

The serialized format includes: + * + *

    + *
  • "version": 1 - the format version + *
  • "table_id": the table ID that all buckets belong to + *
  • "bucket_offsets": array of offsets for non-partitioned table buckets (optional) + *
  • "partition_offsets": array of partition offset objects for partitioned table buckets + * (optional) + *
+ */ +public class TableBucketOffsetsJsonSerde + implements JsonSerializer, JsonDeserializer { + + public static final TableBucketOffsetsJsonSerde INSTANCE = new TableBucketOffsetsJsonSerde(); + + private static final String VERSION_KEY = "version"; + private static final String TABLE_ID_KEY = "table_id"; + private static final String BUCKET_OFFSETS_KEY = "bucket_offsets"; + private static final String PARTITION_OFFSETS_KEY = "partition_offsets"; + private static final String PARTITION_ID_KEY = "partition_id"; + + private static final int VERSION = 1; + private static final long UNKNOWN_OFFSET = -1; + + /** + * Deserializes a JSON node to a {@link TableBucketOffsets} object. + * + *

This method reads the JSON format and reconstructs the table bucket offsets map. The array + * index in "bucket_offsets" represents the bucket id, and the value represents the offset. + * + * @param node the JSON node to deserialize + * @return the deserialized {@link TableBucketOffsets} object + * @throws IllegalArgumentException if the version is not supported + */ + @Override + public TableBucketOffsets deserialize(JsonNode node) { + int version = node.get(VERSION_KEY).asInt(); + if (version != VERSION) { + throw new IllegalArgumentException("Unsupported version: " + version); + } + + long tableId = node.get(TABLE_ID_KEY).asLong(); + Map offsets = new HashMap<>(); + + // Deserialize non-partitioned table bucket offsets + JsonNode bucketOffsetsNode = node.get(BUCKET_OFFSETS_KEY); + JsonNode partitionBucketOffsetsNode = node.get(PARTITION_OFFSETS_KEY); + if (bucketOffsetsNode != null || partitionBucketOffsetsNode != null) { + if (bucketOffsetsNode != null && partitionBucketOffsetsNode != null) { + throw new IllegalArgumentException( + "Both bucket_offsets and partition_bucket_offsets cannot be present at the same time"); + } + + if (bucketOffsetsNode != null) { + int bucketId = 0; + for (JsonNode bucketOffsetNode : bucketOffsetsNode) { + long offset = bucketOffsetNode.asLong(); + // Ignore unknown offsets (filled for missing bucket ids) + if (offset != UNKNOWN_OFFSET) { + TableBucket tableBucket = new TableBucket(tableId, bucketId); + offsets.put(tableBucket, offset); + } + bucketId++; + } + } else { + for (JsonNode partitionOffsetNode : partitionBucketOffsetsNode) { + long partitionId = partitionOffsetNode.get(PARTITION_ID_KEY).asLong(); + JsonNode bucketOffsetsArray = partitionOffsetNode.get(BUCKET_OFFSETS_KEY); + if (bucketOffsetsArray != null && bucketOffsetsArray.isArray()) { + Iterator elements = bucketOffsetsArray.elements(); + int bucketId = 0; + while (elements.hasNext()) { + JsonNode offsetNode = elements.next(); + long offset = offsetNode.asLong(); + // Ignore unknown offsets (filled for missing bucket ids) + if (offset != UNKNOWN_OFFSET) { + TableBucket tableBucket = + new TableBucket(tableId, partitionId, bucketId); + offsets.put(tableBucket, offset); + } + bucketId++; + } + } + } + } + } + + return new TableBucketOffsets(tableId, offsets); + } + + /** + * Serializes a {@link TableBucketOffsets} object to JSON format. + * + *

This method writes the table bucket offsets in the JSON format. It groups buckets by + * partition_id and writes non-partitioned buckets to "bucket_offsets" array and partitioned + * buckets to "partition_offsets" array. The array index represents the bucket id. + * + *

This method validates that all buckets in the offsets map have the same table_id as the + * {@link TableBucketOffsets#getTableId()}. + * + * @param tableBucketOffsets the {@link TableBucketOffsets} object to serialize + * @param generator the JSON generator to write to + * @throws IOException if an I/O error occurs during serialization + * @throws IllegalStateException if buckets have inconsistent table IDs + */ + @Override + public void serialize(TableBucketOffsets tableBucketOffsets, JsonGenerator generator) + throws IOException { + generator.writeStartObject(); + long expectedTableId = tableBucketOffsets.getTableId(); + generator.writeNumberField(VERSION_KEY, VERSION); + generator.writeNumberField(TABLE_ID_KEY, expectedTableId); + + Map offsets = tableBucketOffsets.getOffsets(); + if (!offsets.isEmpty()) { + // Group buckets by partition_id and validate table_id consistency + Map> partitionBuckets = new TreeMap<>(); + List nonPartitionBuckets = new ArrayList<>(); + + for (TableBucket tableBucket : offsets.keySet()) { + // Check that all buckets have the same table_id + checkState( + tableBucket.getTableId() == expectedTableId, + "All buckets must have the same table_id. Expected: %d, but found: %d in bucket: %s", + expectedTableId, + tableBucket.getTableId(), + tableBucket); + + if (tableBucket.getPartitionId() != null) { + partitionBuckets + .computeIfAbsent(tableBucket.getPartitionId(), k -> new ArrayList<>()) + .add(tableBucket); + } else { + nonPartitionBuckets.add(tableBucket); + } + } + + // Serialize non-partitioned table bucket offsets + if (!nonPartitionBuckets.isEmpty()) { + checkState( + partitionBuckets.isEmpty(), + "partitionBuckets must be empty when nonPartitionBuckets is not empty"); + generator.writeArrayFieldStart(BUCKET_OFFSETS_KEY); + serializeBucketLogEndOffset(offsets, nonPartitionBuckets, generator); + generator.writeEndArray(); + } else { + // nonPartitionBuckets is empty, partitionBuckets is must not empty + checkState( + !partitionBuckets.isEmpty(), + "partitionBuckets must be not empty when nonPartitionBuckets is empty"); + generator.writeArrayFieldStart(PARTITION_OFFSETS_KEY); + for (Map.Entry> entry : partitionBuckets.entrySet()) { + Long partitionId = entry.getKey(); + List buckets = entry.getValue(); + generator.writeStartObject(); + generator.writeNumberField(PARTITION_ID_KEY, partitionId); + generator.writeArrayFieldStart(BUCKET_OFFSETS_KEY); + serializeBucketLogEndOffset(offsets, buckets, generator); + generator.writeEndArray(); + generator.writeEndObject(); + } + generator.writeEndArray(); + } + } + + generator.writeEndObject(); + } + + private void serializeBucketLogEndOffset( + Map bucketLogEndOffset, + List buckets, + JsonGenerator generator) + throws IOException { + // sort by bucket id + buckets.sort(Comparator.comparingInt(TableBucket::getBucket)); + int currentBucketId = 0; + for (TableBucket tableBucket : buckets) { + int bucketId = tableBucket.getBucket(); + // Fill null values for missing bucket ids + while (currentBucketId < bucketId) { + generator.writeNumber(UNKNOWN_OFFSET); + currentBucketId++; + } + long logEndOffset = checkNotNull(bucketLogEndOffset.get(tableBucket)); + generator.writeNumber(logEndOffset); + currentBucketId++; + } + } +} diff --git a/fluss-common/src/test/java/org/apache/fluss/utils/json/BucketOffsetJsonSerdeTest.java b/fluss-common/src/test/java/org/apache/fluss/utils/json/BucketOffsetJsonSerdeTest.java deleted file mode 100644 index 7c3c26d391..0000000000 --- a/fluss-common/src/test/java/org/apache/fluss/utils/json/BucketOffsetJsonSerdeTest.java +++ /dev/null @@ -1,40 +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.utils.json; - -import org.apache.fluss.lake.committer.BucketOffset; - -/** Test for {@link BucketOffset}. */ -public class BucketOffsetJsonSerdeTest extends JsonSerdeTestBase { - - BucketOffsetJsonSerdeTest() { - super(BucketOffsetJsonSerde.INSTANCE); - } - - @Override - protected BucketOffset[] createObjects() { - return new BucketOffset[] {new BucketOffset(10, 1, 1L), new BucketOffset(20, 2, null)}; - } - - @Override - protected String[] expectedJsons() { - return new String[] { - "{\"partition_id\":1,\"bucket\":1,\"offset\":10}", "{\"bucket\":2,\"offset\":20}" - }; - } -} diff --git a/fluss-common/src/test/java/org/apache/fluss/utils/json/TableBucketOffsetsJsonSerdeTest.java b/fluss-common/src/test/java/org/apache/fluss/utils/json/TableBucketOffsetsJsonSerdeTest.java new file mode 100644 index 0000000000..831171af2f --- /dev/null +++ b/fluss-common/src/test/java/org/apache/fluss/utils/json/TableBucketOffsetsJsonSerdeTest.java @@ -0,0 +1,108 @@ +/* + * 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.utils.json; + +import org.apache.fluss.metadata.TableBucket; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +/** Test for {@link TableBucketOffsetsJsonSerde}. */ +class TableBucketOffsetsJsonSerdeTest extends JsonSerdeTestBase { + + public TableBucketOffsetsJsonSerdeTest() { + super(TableBucketOffsetsJsonSerde.INSTANCE); + } + + @Override + protected TableBucketOffsets[] createObjects() { + // Test case 1: Empty offsets + TableBucketOffsets tableBucketOffsets1 = new TableBucketOffsets(1L, Collections.emptyMap()); + + // Test case 2: Non-partition table with consecutive bucket ids (0, 1, 2) + long tableId = 4; + Map bucketLogEndOffset = new HashMap<>(); + bucketLogEndOffset.put(new TableBucket(tableId, 0), 100L); + bucketLogEndOffset.put(new TableBucket(tableId, 1), 200L); + bucketLogEndOffset.put(new TableBucket(tableId, 2), 300L); + TableBucketOffsets tableBucketOffsets2 = + new TableBucketOffsets(tableId, bucketLogEndOffset); + + // Test case 3: Non-partition table with missing bucket ids (0, 2, 4 - missing 1 and 3) + tableId = 5; + bucketLogEndOffset = new HashMap<>(); + bucketLogEndOffset.put(new TableBucket(tableId, 0), 100L); + bucketLogEndOffset.put(new TableBucket(tableId, 2), 300L); + bucketLogEndOffset.put(new TableBucket(tableId, 4), 500L); + TableBucketOffsets tableBucketOffsets3 = + new TableBucketOffsets(tableId, bucketLogEndOffset); + + // Test case 4: Partition table with consecutive bucket ids + tableId = 6; + bucketLogEndOffset = new HashMap<>(); + bucketLogEndOffset.put(new TableBucket(tableId, 1L, 0), 100L); + bucketLogEndOffset.put(new TableBucket(tableId, 1L, 1), 200L); + bucketLogEndOffset.put(new TableBucket(tableId, 2L, 0), 300L); + bucketLogEndOffset.put(new TableBucket(tableId, 2L, 1), 400L); + TableBucketOffsets tableBucketOffsets4 = + new TableBucketOffsets(tableId, bucketLogEndOffset); + + // Test case 5: Partition table with missing bucket ids + tableId = 7; + bucketLogEndOffset = new HashMap<>(); + bucketLogEndOffset.put(new TableBucket(tableId, 1L, 0), 100L); + bucketLogEndOffset.put(new TableBucket(tableId, 1L, 2), 300L); // missing bucket 1 + bucketLogEndOffset.put(new TableBucket(tableId, 2L, 1), 400L); + bucketLogEndOffset.put(new TableBucket(tableId, 2L, 3), 600L); // missing bucket 0 and 2 + TableBucketOffsets tableBucketOffsets5 = + new TableBucketOffsets(tableId, bucketLogEndOffset); + + return new TableBucketOffsets[] { + tableBucketOffsets1, + tableBucketOffsets2, + tableBucketOffsets3, + tableBucketOffsets4, + tableBucketOffsets5, + }; + } + + @Override + protected String[] expectedJsons() { + // Format: + // - Non-partition table: "bucket_offsets": [100, 200, 300], array index = bucket id, + // value = offset. Missing buckets are filled with -1. + // - Partition table: "partition_offsets": [{"partition_id": 1, "bucket_offsets": [100, + // 200]}, ...], + // array index in bucket_offsets = bucket id, value = offset. Missing buckets are filled + // with -1. + return new String[] { + // Test case 1: Empty offsets + "{\"version\":1,\"table_id\":1}", + // Test case 2: Non-partition table with consecutive bucket ids [0, 1, 2] + "{\"version\":1,\"table_id\":4,\"bucket_offsets\":[100,200,300]}", + // Test case 3: Non-partition table with missing bucket ids [0, -1, 2, -1, 4] + "{\"version\":1,\"table_id\":5,\"bucket_offsets\":[100,-1,300,-1,500]}", + // Test case 4: Partition table with consecutive bucket ids + "{\"version\":1,\"table_id\":6,\"partition_offsets\":[{\"partition_id\":1,\"bucket_offsets\":[100,200]},{\"partition_id\":2,\"bucket_offsets\":[300,400]}]}", + // Test case 5: Partition table with missing bucket ids + "{\"version\":1,\"table_id\":7,\"partition_offsets\":[{\"partition_id\":1,\"bucket_offsets\":[100,-1,300]},{\"partition_id\":2,\"bucket_offsets\":[-1,400,-1,600]}]}" + }; + } +} 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 e41be49f64..bfc44b53d9 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 @@ -17,6 +17,7 @@ package org.apache.fluss.flink.tiering.committer; +import org.apache.fluss.annotation.VisibleForTesting; import org.apache.fluss.client.metadata.MetadataUpdater; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; @@ -228,30 +229,52 @@ private CommitLakeTableSnapshotRequest toCommitLakeTableSnapshotRequest( // Add PbLakeTableSnapshotInfo for metrics reporting (to notify tablet servers about // synchronized log end offsets and max timestamps) if (!logEndOffsets.isEmpty()) { - PbLakeTableSnapshotInfo pbLakeTableSnapshotInfo = - commitLakeTableSnapshotRequest.addTablesReq(); - pbLakeTableSnapshotInfo.setTableId(tableId); - pbLakeTableSnapshotInfo.setSnapshotId(snapshotId); - for (Map.Entry logEndOffsetEntry : logEndOffsets.entrySet()) { - TableBucket tableBucket = logEndOffsetEntry.getKey(); - PbLakeTableOffsetForBucket pbLakeTableOffsetForBucket = - pbLakeTableSnapshotInfo.addBucketsReq(); + commitLakeTableSnapshotRequest = + addLogEndOffsets( + commitLakeTableSnapshotRequest, + tableId, + snapshotId, + logEndOffsets, + logMaxTieredTimestamps); + } + return commitLakeTableSnapshotRequest; + } + + @VisibleForTesting + protected CommitLakeTableSnapshotRequest addLogEndOffsets( + CommitLakeTableSnapshotRequest commitLakeTableSnapshotRequest, + long tableId, + long snapshotId, + Map logEndOffsets, + Map logMaxTieredTimestamps) { + PbLakeTableSnapshotInfo pbLakeTableSnapshotInfo = + commitLakeTableSnapshotRequest.addTablesReq(); + pbLakeTableSnapshotInfo.setTableId(tableId); + pbLakeTableSnapshotInfo.setSnapshotId(snapshotId); + for (Map.Entry logEndOffsetEntry : logEndOffsets.entrySet()) { + TableBucket tableBucket = logEndOffsetEntry.getKey(); + PbLakeTableOffsetForBucket pbLakeTableOffsetForBucket = + pbLakeTableSnapshotInfo.addBucketsReq(); - if (tableBucket.getPartitionId() != null) { - pbLakeTableOffsetForBucket.setPartitionId(tableBucket.getPartitionId()); - } - pbLakeTableOffsetForBucket.setBucketId(tableBucket.getBucket()); - pbLakeTableOffsetForBucket.setLogEndOffset(logEndOffsetEntry.getValue()); + if (tableBucket.getPartitionId() != null) { + pbLakeTableOffsetForBucket.setPartitionId(tableBucket.getPartitionId()); + } + pbLakeTableOffsetForBucket.setBucketId(tableBucket.getBucket()); + pbLakeTableOffsetForBucket.setLogEndOffset(logEndOffsetEntry.getValue()); - Long maxTimestamp = logMaxTieredTimestamps.get(tableBucket); - if (maxTimestamp != null) { - pbLakeTableOffsetForBucket.setMaxTimestamp(maxTimestamp); - } + Long maxTimestamp = logMaxTieredTimestamps.get(tableBucket); + if (maxTimestamp != null) { + pbLakeTableOffsetForBucket.setMaxTimestamp(maxTimestamp); } } return commitLakeTableSnapshotRequest; } + @VisibleForTesting + CoordinatorGateway getCoordinatorGateway() { + return coordinatorGateway; + } + @Override public void close() throws Exception { if (rpcClient != null) { 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 d2f9302c12..c85ace01eb 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 @@ -21,16 +21,22 @@ import org.apache.fluss.flink.utils.FlinkTestBase; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.rpc.messages.CommitLakeTableSnapshotRequest; +import org.apache.fluss.server.zk.ZooKeeperClient; +import org.apache.fluss.server.zk.data.ZkData; +import org.apache.fluss.server.zk.data.lake.LakeTableSnapshot; +import org.apache.fluss.server.zk.data.lake.LakeTableSnapshotJsonSerde; +import org.apache.fluss.utils.json.JsonSerdeUtils; +import org.apache.fluss.utils.types.Tuple2; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; -import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; -import java.util.List; import java.util.Map; import static org.apache.fluss.record.TestData.DATA1_PARTITIONED_TABLE_DESCRIPTOR; @@ -61,36 +67,11 @@ void afterEach() throws Exception { void testCommit(boolean isPartitioned) throws Exception { TablePath tablePath = TablePath.of("fluss", "test_commit" + (isPartitioned ? "_partitioned" : "")); - long tableId = - createTable( - tablePath, - isPartitioned - ? DATA1_PARTITIONED_TABLE_DESCRIPTOR - : DATA1_TABLE_DESCRIPTOR); - - List partitions; - Map partitionNameAndIds = new HashMap<>(); - if (!isPartitioned) { - FLUSS_CLUSTER_EXTENSION.waitUntilTableReady(tableId); - partitions = Collections.singletonList(null); - } else { - partitionNameAndIds = FLUSS_CLUSTER_EXTENSION.waitUntilPartitionAllReady(tablePath); - partitions = new ArrayList<>(partitionNameAndIds.keySet()); - } + Tuple2> tableIdAndPartitions = createTable(tablePath, isPartitioned); + long tableId = tableIdAndPartitions.f0; + Collection partitions = tableIdAndPartitions.f1; - Map expectedOffsets = new HashMap<>(); - for (int bucket = 0; bucket < 3; bucket++) { - long bucketOffset = bucket * bucket; - for (String partitionName : partitions) { - if (partitionName == null) { - expectedOffsets.put(new TableBucket(tableId, bucket), bucketOffset); - } else { - long partitionId = partitionNameAndIds.get(partitionName); - expectedOffsets.put( - new TableBucket(tableId, partitionId, bucket), bucketOffset); - } - } - } + Map expectedOffsets = mockLogEndOffsets(tableId, partitions); long lakeSnapshotId = 3; @@ -112,4 +93,77 @@ void testCommit(boolean isPartitioned) throws Exception { Map bucketLogOffsets = lakeSnapshot.getTableBucketsOffset(); assertThat(bucketLogOffsets).isEqualTo(expectedOffsets); } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testCompatibilityWithOldCommitter(boolean isPartitioned) throws Exception { + // test commit lake snapshot with old behavior + TablePath tablePath = + TablePath.of( + "fluss", + "test_legacy_version_commit" + (isPartitioned ? "_partitioned" : "")); + Tuple2> tableIdAndPartitions = createTable(tablePath, isPartitioned); + long tableId = tableIdAndPartitions.f0; + Collection partitions = tableIdAndPartitions.f1; + + Map logEndOffsets = mockLogEndOffsets(tableId, partitions); + long snapshotId = 3; + + // mock old behavior to commit + CommitLakeTableSnapshotRequest commitLakeTableSnapshotRequest = + new CommitLakeTableSnapshotRequest(); + commitLakeTableSnapshotRequest = + flussTableLakeSnapshotCommitter.addLogEndOffsets( + commitLakeTableSnapshotRequest, + tableId, + snapshotId, + logEndOffsets, + Collections.emptyMap()); + flussTableLakeSnapshotCommitter + .getCoordinatorGateway() + .commitLakeTableSnapshot(commitLakeTableSnapshotRequest) + .get(); + + // make sure it can be deserialized with v1 + ZooKeeperClient zkClient = FLUSS_CLUSTER_EXTENSION.getZooKeeperClient(); + byte[] jsonBytes = zkClient.getOrEmpty(ZkData.LakeTableZNode.path(tableId)).get(); + + LakeTableSnapshot lakeTableSnapshot = + JsonSerdeUtils.readValue(jsonBytes, LakeTableSnapshotJsonSerde.INSTANCE); + assertThat(lakeTableSnapshot.getSnapshotId()).isEqualTo(snapshotId); + assertThat(lakeTableSnapshot.getBucketLogEndOffset()).isEqualTo(logEndOffsets); + } + + private Map mockLogEndOffsets(long tableId, Collection partitionsIds) { + Map logEndOffsets = new HashMap<>(); + for (int bucket = 0; bucket < 3; bucket++) { + long bucketOffset = bucket * bucket; + for (Long partitionId : partitionsIds) { + if (partitionId == null) { + logEndOffsets.put(new TableBucket(tableId, bucket), bucketOffset); + } else { + logEndOffsets.put(new TableBucket(tableId, partitionId, bucket), bucketOffset); + } + } + } + return logEndOffsets; + } + + private Tuple2> createTable(TablePath tablePath, boolean isPartitioned) + throws Exception { + long tableId = + createTable( + tablePath, + isPartitioned + ? DATA1_PARTITIONED_TABLE_DESCRIPTOR + : DATA1_TABLE_DESCRIPTOR); + Collection partitions; + if (!isPartitioned) { + partitions = Collections.singletonList(null); + FLUSS_CLUSTER_EXTENSION.waitUntilTableReady(tableId); + } else { + partitions = FLUSS_CLUSTER_EXTENSION.waitUntilPartitionAllReady(tablePath).values(); + } + return new Tuple2<>(tableId, partitions); + } } 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 2d715faea5..f193507a0d 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 @@ -1305,8 +1305,8 @@ private void handleCommitLakeTableSnapshotV1( } // this involves IO operation (ZK), so we do it in ioExecutor - lakeTableHelper.upsertLakeTable( - tableId, tablePath, lakeTableSnapshotEntry.getValue()); + lakeTableHelper.upsertLakeTableV1( + tableId, lakeTableSnapshotEntry.getValue()); } catch (Exception e) { ApiError error = ApiError.fromThrowable(e); tableResp.setError(error.error().code(), error.message()); 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 1665014042..74b7da9013 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 @@ -24,6 +24,8 @@ import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.server.zk.data.ZkData; import org.apache.fluss.utils.IOUtils; +import org.apache.fluss.utils.json.JsonSerdeUtils; +import org.apache.fluss.utils.json.TableBucketOffsetsJsonSerde; import javax.annotation.Nullable; @@ -106,6 +108,11 @@ public List getLakeSnapshotMetadatas() { return lakeSnapshotMetadatas; } + @Nullable + public LakeTableSnapshot getLakeTableSnapshot() { + return lakeTableSnapshot; + } + /** * Get the latest table snapshot for the lake table. * @@ -125,8 +132,10 @@ public LakeTableSnapshot getLatestTableSnapshot() throws Exception { try (ByteArrayOutputStream outputStream = new ByteArrayOutputStream()) { IOUtils.copyBytes(inputStream, outputStream, true); Map logOffsets = - LakeTableSnapshotJsonSerde.fromJson(outputStream.toByteArray()) - .getBucketLogEndOffset(); + JsonSerdeUtils.readValue( + outputStream.toByteArray(), + TableBucketOffsetsJsonSerde.INSTANCE) + .getOffsets(); return new LakeTableSnapshot(lakeSnapshotMetadata.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 8f9b352c42..eb0aecac5b 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 @@ -25,6 +25,9 @@ import org.apache.fluss.metadata.TablePath; import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.utils.FlussPaths; +import org.apache.fluss.utils.json.JsonSerdeUtils; +import org.apache.fluss.utils.json.TableBucketOffsets; +import org.apache.fluss.utils.json.TableBucketOffsetsJsonSerde; import java.util.HashMap; import java.util.List; @@ -44,6 +47,26 @@ public LakeTableHelper(ZooKeeperClient zkClient, String remoteDataDir) { this.remoteDataDir = remoteDataDir; } + /** + * Upserts a lake table snapshot for the given table, stored in v1 format. + * + * @param tableId the table ID + * @param lakeTableSnapshot the new snapshot to upsert + * @throws Exception if the operation fails + */ + public void upsertLakeTableV1(long tableId, LakeTableSnapshot lakeTableSnapshot) + throws Exception { + Optional optPreviousLakeTable = zkClient.getLakeTable(tableId); + // Merge with previous snapshot if exists + if (optPreviousLakeTable.isPresent()) { + lakeTableSnapshot = + mergeLakeTable( + optPreviousLakeTable.get().getLatestTableSnapshot(), lakeTableSnapshot); + } + zkClient.upsertLakeTable( + tableId, new LakeTable(lakeTableSnapshot), optPreviousLakeTable.isPresent()); + } + /** * Upserts a lake table snapshot for the given table. * @@ -156,21 +179,25 @@ private LakeTableSnapshot mergeLakeTable( public FsPath storeLakeTableSnapshot( long tableId, TablePath tablePath, LakeTableSnapshot lakeTableSnapshot) throws Exception { - // get the remote file path to store the lake table snapshot information - FsPath remoteLakeTableSnapshotManifestPath = - FlussPaths.remoteLakeTableSnapshotManifestPath(remoteDataDir, tablePath, tableId); + // get the remote file path to store the lake table snapshot offset information + FsPath remoteLakeTableSnapshotOffsetPath = + FlussPaths.remoteLakeTableSnapshotOffsetPath(remoteDataDir, tablePath, tableId); // check whether the parent directory exists, if not, create the directory - FileSystem fileSystem = remoteLakeTableSnapshotManifestPath.getFileSystem(); - if (!fileSystem.exists(remoteLakeTableSnapshotManifestPath.getParent())) { - fileSystem.mkdirs(remoteLakeTableSnapshotManifestPath.getParent()); + FileSystem fileSystem = remoteLakeTableSnapshotOffsetPath.getFileSystem(); + if (!fileSystem.exists(remoteLakeTableSnapshotOffsetPath.getParent())) { + fileSystem.mkdirs(remoteLakeTableSnapshotOffsetPath.getParent()); } - // serialize table snapshot to json bytes, and write to file - byte[] jsonBytes = LakeTableSnapshotJsonSerde.toJson(lakeTableSnapshot); + // serialize table offsets to json bytes, and write to file + byte[] jsonBytes = + JsonSerdeUtils.writeValueAsBytes( + new TableBucketOffsets(tableId, lakeTableSnapshot.getBucketLogEndOffset()), + TableBucketOffsetsJsonSerde.INSTANCE); + try (FSDataOutputStream outputStream = fileSystem.create( - remoteLakeTableSnapshotManifestPath, FileSystem.WriteMode.OVERWRITE)) { + remoteLakeTableSnapshotOffsetPath, FileSystem.WriteMode.OVERWRITE)) { outputStream.write(jsonBytes); } - return remoteLakeTableSnapshotManifestPath; + return remoteLakeTableSnapshotOffsetPath; } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerde.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerde.java index f272361bc6..e6de853fba 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerde.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerde.java @@ -61,28 +61,35 @@ public class LakeTableJsonSerde implements JsonSerializer, JsonDeseri @Override public void serialize(LakeTable lakeTable, JsonGenerator generator) throws IOException { - generator.writeStartObject(); - generator.writeNumberField(VERSION_KEY, CURRENT_VERSION); - - generator.writeArrayFieldStart(LAKE_SNAPSHOTS); - for (LakeTable.LakeSnapshotMetadata lakeSnapshotMetadata : - checkNotNull(lakeTable.getLakeSnapshotMetadatas())) { + // if lake table snapshot is null, it must be version 1 + if (lakeTable.getLakeTableSnapshot() != null) { + // Version 1: ZK node contains full snapshot data, use LakeTableSnapshotJsonSerde + LakeTableSnapshotJsonSerde.INSTANCE.serialize( + lakeTable.getLakeTableSnapshot(), generator); + } else { generator.writeStartObject(); + generator.writeNumberField(VERSION_KEY, CURRENT_VERSION); + + generator.writeArrayFieldStart(LAKE_SNAPSHOTS); + for (LakeTable.LakeSnapshotMetadata lakeSnapshotMetadata : + checkNotNull(lakeTable.getLakeSnapshotMetadatas())) { + generator.writeStartObject(); - generator.writeNumberField(SNAPSHOT_ID_KEY, lakeSnapshotMetadata.getSnapshotId()); - generator.writeStringField( - TIERED_OFFSETS_KEY, lakeSnapshotMetadata.getTieredOffsetsFilePath().toString()); - if (lakeSnapshotMetadata.getReadableOffsetsFilePath() != null) { + generator.writeNumberField(SNAPSHOT_ID_KEY, lakeSnapshotMetadata.getSnapshotId()); generator.writeStringField( - READABLE_OFFSETS_KEY, - lakeSnapshotMetadata.getReadableOffsetsFilePath().toString()); + TIERED_OFFSETS_KEY, + lakeSnapshotMetadata.getTieredOffsetsFilePath().toString()); + if (lakeSnapshotMetadata.getReadableOffsetsFilePath() != null) { + generator.writeStringField( + READABLE_OFFSETS_KEY, + lakeSnapshotMetadata.getReadableOffsetsFilePath().toString()); + } + generator.writeEndObject(); } + generator.writeEndArray(); + generator.writeEndObject(); } - - generator.writeEndArray(); - - generator.writeEndObject(); } @Override diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableSnapshotJsonSerde.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableSnapshotJsonSerde.java index 2543dff59e..b8d34ed893 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableSnapshotJsonSerde.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableSnapshotJsonSerde.java @@ -18,168 +18,76 @@ package org.apache.fluss.server.zk.data.lake; -import org.apache.fluss.annotation.VisibleForTesting; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; import org.apache.fluss.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.fluss.utils.json.JsonDeserializer; -import org.apache.fluss.utils.json.JsonSerdeUtils; import org.apache.fluss.utils.json.JsonSerializer; import java.io.IOException; -import java.util.ArrayList; -import java.util.Comparator; import java.util.HashMap; -import java.util.Iterator; -import java.util.List; import java.util.Map; -import java.util.TreeMap; -import static org.apache.fluss.utils.Preconditions.checkNotNull; -import static org.apache.fluss.utils.Preconditions.checkState; - -/** - * Json serializer and deserializer for {@link LakeTableSnapshot}. - * - *

This serde supports two storage format versions: - * - *

    - *
  • Version 1 (legacy): Each bucket object contains full information including repeated - * partition names and partition_id in each bucket entry. - *
  • Version 2 (current): Compact format that uses different property keys for partitioned and - * non-partitioned tables to simplify deserialization: - *
      - *
    • Non-partition table uses "bucket_offsets": [100, 200, 300], where array index - * represents bucket id (0, 1, 2) and value represents log_end_offset. For buckets - * without end offset, -1 is written. Missing bucket ids in the sequence are also filled - * with -1. - *
    • Partition table uses "partition_bucket_offsets": {"1": [100, 200], "2": [300, 400]}, - * where key is partition id, array index represents bucket id (0, 1) and value - * represents log_end_offset. For buckets without end offset, -1 is written. Missing - * bucket ids in the sequence are also filled with -1. - *
    - * During deserialization, values of -1 are ignored and not added to the bucket log end offset - * map. - *
- */ +/** Json serializer and deserializer for {@link LakeTableSnapshot}. */ public class LakeTableSnapshotJsonSerde implements JsonSerializer, JsonDeserializer { public static final LakeTableSnapshotJsonSerde INSTANCE = new LakeTableSnapshotJsonSerde(); - private static final long UNKNOWN_LOG_OFFSET = -1; - private static final String VERSION_KEY = "version"; private static final String SNAPSHOT_ID = "snapshot_id"; private static final String TABLE_ID = "table_id"; private static final String PARTITION_ID = "partition_id"; private static final String BUCKETS = "buckets"; - private static final String BUCKET_OFFSETS = "bucket_offsets"; - private static final String PARTITION_BUCKET_OFFSETS = "partition_bucket_offsets"; private static final String BUCKET_ID = "bucket_id"; private static final String LOG_END_OFFSET = "log_end_offset"; private static final int VERSION_1 = 1; - private static final int VERSION_2 = 2; - private static final int CURRENT_VERSION = VERSION_2; + private static final int CURRENT_VERSION = VERSION_1; @Override public void serialize(LakeTableSnapshot lakeTableSnapshot, JsonGenerator generator) throws IOException { generator.writeStartObject(); - generator.writeNumberField(VERSION_KEY, CURRENT_VERSION); + generator.writeNumberField(VERSION_KEY, VERSION_1); generator.writeNumberField(SNAPSHOT_ID, lakeTableSnapshot.getSnapshotId()); Map bucketLogEndOffset = lakeTableSnapshot.getBucketLogEndOffset(); - + // Get table id from the first table bucket, all buckets should have the same table id if (!bucketLogEndOffset.isEmpty()) { - // Get table_id from the first bucket (all buckets should have the same table_id) - long tableId = bucketLogEndOffset.keySet().iterator().next().getTableId(); + TableBucket firstBucket = bucketLogEndOffset.keySet().iterator().next(); + long tableId = firstBucket.getTableId(); generator.writeNumberField(TABLE_ID, tableId); - - // Group buckets by partition_id - Map> partitionBuckets = new TreeMap<>(); - List nonPartitionBuckets = new ArrayList<>(); - - for (TableBucket tableBucket : bucketLogEndOffset.keySet()) { - if (tableBucket.getPartitionId() != null) { - partitionBuckets - .computeIfAbsent(tableBucket.getPartitionId(), k -> new ArrayList<>()) - .add(tableBucket); - } else { - nonPartitionBuckets.add(tableBucket); - } - } - if (!partitionBuckets.isEmpty()) { - checkState( - nonPartitionBuckets.isEmpty(), - "nonPartitionBuckets must be empty when partitionBuckets is not empty"); - // Partition table: object format grouped by partition_id - generator.writeObjectFieldStart(PARTITION_BUCKET_OFFSETS); - for (Map.Entry> entry : partitionBuckets.entrySet()) { - Long partitionId = entry.getKey(); - List buckets = entry.getValue(); - // Write array of log_end_offset values, array index represents bucket id - generator.writeArrayFieldStart(String.valueOf(partitionId)); - serializeBucketLogEndOffset(bucketLogEndOffset, buckets, generator); - generator.writeEndArray(); - } - generator.writeEndObject(); - } else { - checkState( - !nonPartitionBuckets.isEmpty(), - "nonPartitionBuckets must be not empty when partitionBuckets is empty"); - // Non-partition table: array format, array index represents bucket id - generator.writeArrayFieldStart(BUCKET_OFFSETS); - serializeBucketLogEndOffset(bucketLogEndOffset, nonPartitionBuckets, generator); - generator.writeEndArray(); - } } - generator.writeEndObject(); - } - private void serializeBucketLogEndOffset( - Map bucketLogEndOffset, - List buckets, - JsonGenerator generator) - throws IOException { - // sort by bucket id - buckets.sort(Comparator.comparingInt(TableBucket::getBucket)); - int currentBucketId = 0; - for (TableBucket tableBucket : buckets) { - int bucketId = tableBucket.getBucket(); - // Fill null values for missing bucket ids - while (currentBucketId < bucketId) { - generator.writeNumber(UNKNOWN_LOG_OFFSET); - currentBucketId++; + generator.writeArrayFieldStart(BUCKETS); + for (Map.Entry tableBucketOffsetEntry : bucketLogEndOffset.entrySet()) { + generator.writeStartObject(); + TableBucket tableBucket = tableBucketOffsetEntry.getKey(); + if (tableBucket.getPartitionId() != null) { + generator.writeNumberField(PARTITION_ID, tableBucket.getPartitionId()); } - long logEndOffset = checkNotNull(bucketLogEndOffset.get(tableBucket)); - generator.writeNumber(logEndOffset); - currentBucketId++; + generator.writeNumberField(BUCKET_ID, tableBucket.getBucket()); + generator.writeNumberField(LOG_END_OFFSET, tableBucketOffsetEntry.getValue()); + generator.writeEndObject(); } + generator.writeEndArray(); + + generator.writeEndObject(); } @Override public LakeTableSnapshot deserialize(JsonNode node) { int version = node.get(VERSION_KEY).asInt(); - if (version == VERSION_1) { - return deserializeVersion1(node); - } else if (version == VERSION_2) { - return deserializeVersion2(node); - } else { - throw new IllegalArgumentException("Unsupported version: " + version); + if (version != CURRENT_VERSION) { + throw new IllegalArgumentException( + "Unsupported version: " + node.get(VERSION_KEY).asInt()); } - } - - /** Deserialize Version 1 format (legacy). */ - private LakeTableSnapshot deserializeVersion1(JsonNode node) { long snapshotId = node.get(SNAPSHOT_ID).asLong(); long tableId = node.get(TABLE_ID).asLong(); - Iterator buckets = node.get(BUCKETS).elements(); Map bucketLogEndOffset = new HashMap<>(); - while (buckets.hasNext()) { - JsonNode bucket = buckets.next(); + for (JsonNode bucket : node.get(BUCKETS)) { TableBucket tableBucket; Long partitionId = bucket.get(PARTITION_ID) != null ? bucket.get(PARTITION_ID).asLong() : null; @@ -192,121 +100,4 @@ private LakeTableSnapshot deserializeVersion1(JsonNode node) { } return new LakeTableSnapshot(snapshotId, bucketLogEndOffset); } - - /** - * Deserialize Version 2 format (uses different property keys for partitioned and - * non-partitioned tables). - */ - private LakeTableSnapshot deserializeVersion2(JsonNode node) { - long snapshotId = node.get(SNAPSHOT_ID).asLong(); - Map bucketLogEndOffset = new HashMap<>(); - - // Check for bucket_offsets (non-partition table) or partition_bucket_offsets (partition - // table) - JsonNode bucketOffsetsNode = node.get(BUCKET_OFFSETS); - JsonNode partitionBucketOffsetsNode = node.get(PARTITION_BUCKET_OFFSETS); - if (bucketOffsetsNode != null || partitionBucketOffsetsNode != null) { - if (bucketOffsetsNode != null && partitionBucketOffsetsNode != null) { - throw new IllegalArgumentException( - "Both bucket_offsets and partition_bucket_offsets cannot be present at the same time"); - } - JsonNode tableIdNode = node.get(TABLE_ID); - // Non-partition table: array format, array index represents bucket id - if (tableIdNode == null) { - throw new IllegalArgumentException( - "table_id is required when bucket_offsets or partition_bucket_offsets is present in version 2 format"); - } - long tableId = tableIdNode.asLong(); - - if (bucketOffsetsNode != null) { - - Iterator elements = bucketOffsetsNode.elements(); - int bucketId = 0; - while (elements.hasNext()) { - JsonNode logEndOffsetNode = elements.next(); - TableBucket tableBucket = new TableBucket(tableId, bucketId); - long logEndOffset = logEndOffsetNode.asLong(); - if (logEndOffset != UNKNOWN_LOG_OFFSET) { - bucketLogEndOffset.put(tableBucket, logEndOffset); - } - bucketId++; - } - } else { - Iterator> partitions = - partitionBucketOffsetsNode.fields(); - while (partitions.hasNext()) { - Map.Entry entry = partitions.next(); - String partitionKey = entry.getKey(); - Long partitionId = Long.parseLong(partitionKey); - JsonNode logEndOffsetsArray = entry.getValue(); - // Array index represents bucket id, value represents log_end_offset - Iterator elements = logEndOffsetsArray.elements(); - int bucketId = 0; - while (elements.hasNext()) { - JsonNode logEndOffsetNode = elements.next(); - TableBucket tableBucket = new TableBucket(tableId, partitionId, bucketId); - long logEndOffset = logEndOffsetNode.asLong(); - if (logEndOffset != UNKNOWN_LOG_OFFSET) { - bucketLogEndOffset.put(tableBucket, logEndOffset); - } - bucketId++; - } - } - } - } - return new LakeTableSnapshot(snapshotId, bucketLogEndOffset); - } - - /** Serialize the {@link LakeTableSnapshot} to json bytes using current version. */ - public static byte[] toJson(LakeTableSnapshot lakeTableSnapshot) { - return JsonSerdeUtils.writeValueAsBytes(lakeTableSnapshot, INSTANCE); - } - - /** Serialize the {@link LakeTableSnapshot} to json bytes using Version 1 format. */ - @VisibleForTesting - public static byte[] toJsonVersion1(LakeTableSnapshot lakeTableSnapshot, long tableId) { - return JsonSerdeUtils.writeValueAsBytes(lakeTableSnapshot, new Version1Serializer(tableId)); - } - - /** Deserialize the json bytes to {@link LakeTableSnapshot}. */ - public static LakeTableSnapshot fromJson(byte[] json) { - return JsonSerdeUtils.readValue(json, INSTANCE); - } - - /** Version 1 serializer for backward compatibility testing. */ - private static class Version1Serializer implements JsonSerializer { - - private final long tableId; - - private Version1Serializer(long tableId) { - this.tableId = tableId; - } - - @Override - public void serialize(LakeTableSnapshot lakeTableSnapshot, JsonGenerator generator) - throws IOException { - generator.writeStartObject(); - generator.writeNumberField(VERSION_KEY, VERSION_1); - generator.writeNumberField(SNAPSHOT_ID, lakeTableSnapshot.getSnapshotId()); - generator.writeNumberField(TABLE_ID, tableId); - - generator.writeArrayFieldStart(BUCKETS); - for (TableBucket tableBucket : lakeTableSnapshot.getBucketLogEndOffset().keySet()) { - generator.writeStartObject(); - generator.writeNumberField(BUCKET_ID, tableBucket.getBucket()); - if (tableBucket.getPartitionId() != null) { - generator.writeNumberField(PARTITION_ID, tableBucket.getPartitionId()); - } - if (lakeTableSnapshot.getLogEndOffset(tableBucket).isPresent()) { - generator.writeNumberField( - LOG_END_OFFSET, lakeTableSnapshot.getLogEndOffset(tableBucket).get()); - } - - generator.writeEndObject(); - } - generator.writeEndArray(); - - generator.writeEndObject(); - } - } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/LakeTableSnapshotJsonSerdeTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/LakeTableSnapshotJsonSerdeTest.java index e00f349021..119558d882 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/LakeTableSnapshotJsonSerdeTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/LakeTableSnapshotJsonSerdeTest.java @@ -21,17 +21,10 @@ import org.apache.fluss.server.zk.data.lake.LakeTableSnapshot; import org.apache.fluss.server.zk.data.lake.LakeTableSnapshotJsonSerde; import org.apache.fluss.utils.json.JsonSerdeTestBase; -import org.apache.fluss.utils.json.JsonSerdeUtils; -import org.junit.jupiter.api.Test; - -import java.nio.charset.StandardCharsets; -import java.util.Collections; import java.util.HashMap; import java.util.Map; -import static org.assertj.core.api.Assertions.assertThat; - /** Test for {@link LakeTableSnapshotJsonSerde}. */ class LakeTableSnapshotJsonSerdeTest extends JsonSerdeTestBase { @@ -41,104 +34,37 @@ class LakeTableSnapshotJsonSerdeTest extends JsonSerdeTestBase bucketLogEndOffset = new HashMap<>(); - bucketLogEndOffset.put(new TableBucket(tableId, 0), 100L); - bucketLogEndOffset.put(new TableBucket(tableId, 1), 200L); - bucketLogEndOffset.put(new TableBucket(tableId, 2), 300L); - LakeTableSnapshot lakeTableSnapshot2 = new LakeTableSnapshot(2, bucketLogEndOffset); + bucketLogEndOffset.put(new TableBucket(tableId, 1), 3L); + bucketLogEndOffset.put(new TableBucket(tableId, 2), 4L); + + LakeTableSnapshot lakeTableSnapshot1 = new LakeTableSnapshot(2, bucketLogEndOffset); - // Test case 3: Non-partition table with missing bucket ids (0, 2, 4 - missing 1 and 3) tableId = 5; - bucketLogEndOffset = new HashMap<>(); - bucketLogEndOffset.put(new TableBucket(tableId, 0), 100L); - bucketLogEndOffset.put(new TableBucket(tableId, 2), 300L); - bucketLogEndOffset.put(new TableBucket(tableId, 4), 500L); - LakeTableSnapshot lakeTableSnapshot3 = new LakeTableSnapshot(3, bucketLogEndOffset); - // Test case 4: Partition table with consecutive bucket ids - tableId = 6; bucketLogEndOffset = new HashMap<>(); - bucketLogEndOffset.put(new TableBucket(tableId, 1L, 0), 100L); - bucketLogEndOffset.put(new TableBucket(tableId, 1L, 1), 200L); - bucketLogEndOffset.put(new TableBucket(tableId, 2L, 0), 300L); - bucketLogEndOffset.put(new TableBucket(tableId, 2L, 1), 400L); - LakeTableSnapshot lakeTableSnapshot4 = new LakeTableSnapshot(4, bucketLogEndOffset); + bucketLogEndOffset.put(new TableBucket(tableId, 1L, 1), 3L); + bucketLogEndOffset.put(new TableBucket(tableId, 2L, 1), 4L); - // Test case 5: Partition table with missing bucket ids - tableId = 7; - bucketLogEndOffset = new HashMap<>(); - bucketLogEndOffset.put(new TableBucket(tableId, 1L, 0), 100L); - bucketLogEndOffset.put(new TableBucket(tableId, 1L, 2), 300L); // missing bucket 1 - bucketLogEndOffset.put(new TableBucket(tableId, 2L, 1), 400L); - bucketLogEndOffset.put(new TableBucket(tableId, 2L, 3), 600L); // missing bucket 0 and 2 - LakeTableSnapshot lakeTableSnapshot5 = new LakeTableSnapshot(5, bucketLogEndOffset); + LakeTableSnapshot lakeTableSnapshot2 = new LakeTableSnapshot(3, bucketLogEndOffset); return new LakeTableSnapshot[] { - lakeTableSnapshot1, - lakeTableSnapshot2, - lakeTableSnapshot3, - lakeTableSnapshot4, - lakeTableSnapshot5, + lakeTableSnapshot1, lakeTableSnapshot2, }; } @Override protected String[] expectedJsons() { - // Version 2 format (uses different property keys): - // - Non-partition table: "bucket_offsets": [100, 200, 300], array index = bucket id, - // value = log_end_offset. Missing buckets are filled with -1. - // - Partition table: "partition_bucket_offsets": {"1": [100, 200], "2": [300, 400]}, - // key = partition id, array index = bucket id, value = log_end_offset. Missing buckets - // are filled with -1. return new String[] { - // Test case 1: Empty snapshot - "{\"version\":2,\"snapshot_id\":1}", - // Test case 2: Non-partition table with consecutive bucket ids [0, 1, 2] - "{\"version\":2,\"snapshot_id\":2,\"table_id\":4,\"bucket_offsets\":[100,200,300]}", - // Test case 3: Non-partition table with missing bucket ids [0, -1, 2, -1, 4] - "{\"version\":2,\"snapshot_id\":3,\"table_id\":5,\"bucket_offsets\":[100,-1,300,-1,500]}", - // Test case 4: Partition table with consecutive bucket ids - "{\"version\":2,\"snapshot_id\":4,\"table_id\":6," - + "\"partition_bucket_offsets\":{\"1\":[100,200],\"2\":[300,400]}}", - // Test case 5: Partition table with missing bucket ids - "{\"version\":2,\"snapshot_id\":5,\"table_id\":7," - + "\"partition_bucket_offsets\":{\"1\":[100,-1,300],\"2\":[-1,400,-1,600]}}" + "{\"version\":1,\"snapshot_id\":2,\"table_id\":4," + + "\"buckets\":[{\"bucket_id\":2,\"log_end_offset\":4}," + + "{\"bucket_id\":1,\"log_end_offset\":3}]}", + "{\"version\":1,\"snapshot_id\":3,\"table_id\":5," + + "\"buckets\":[{\"partition_id\":1,\"bucket_id\":1,\"log_end_offset\":3}," + + "{\"partition_id\":2,\"bucket_id\":1,\"log_end_offset\":4}]}" }; } - - @Test - void testBackwardCompatibility() { - // Test that Version 1 format can still be deserialized - String version1Json1 = "{\"version\":1,\"snapshot_id\":1,\"table_id\":1,\"buckets\":[]}"; - LakeTableSnapshot snapshot1 = - JsonSerdeUtils.readValue( - version1Json1.getBytes(StandardCharsets.UTF_8), - LakeTableSnapshotJsonSerde.INSTANCE); - assertThat(snapshot1.getSnapshotId()).isEqualTo(1); - assertThat(snapshot1.getBucketLogEndOffset()).isEmpty(); - - String version1Json2 = - "{\"version\":1,\"snapshot_id\":2,\"table_id\":4," - + "\"buckets\":[{\"bucket_id\":1,\"log_start_offset\":1,\"log_end_offset\":3,\"max_timestamp\":5}]}"; - LakeTableSnapshot snapshot2 = - JsonSerdeUtils.readValue( - version1Json2.getBytes(StandardCharsets.UTF_8), - LakeTableSnapshotJsonSerde.INSTANCE); - assertThat(snapshot2.getSnapshotId()).isEqualTo(2); - assertThat(snapshot2.getBucketLogEndOffset()).hasSize(1); - - String version1Json3 = - "{\"version\":1,\"snapshot_id\":3,\"table_id\":5," - + "\"buckets\":[{\"partition_id\":1,\"partition_name\":\"partition1\",\"bucket_id\":1,\"log_start_offset\":1,\"log_end_offset\":3,\"max_timestamp\":5}]}"; - LakeTableSnapshot snapshot3 = - JsonSerdeUtils.readValue( - version1Json3.getBytes(StandardCharsets.UTF_8), - LakeTableSnapshotJsonSerde.INSTANCE); - assertThat(snapshot3.getSnapshotId()).isEqualTo(3); - } } 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 6dd8497d58..d8aad22f6b 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 @@ -30,7 +30,6 @@ import org.apache.fluss.server.zk.ZooKeeperExtension; import org.apache.fluss.server.zk.ZooKeeperUtils; import org.apache.fluss.server.zk.data.TableRegistration; -import org.apache.fluss.server.zk.data.ZkData; import org.apache.fluss.testutils.common.AllCallbackWrapper; import org.junit.jupiter.api.AfterAll; @@ -110,15 +109,8 @@ void testUpsertLakeTableCompatible(@TempDir Path tempDir) throws Exception { LakeTableSnapshot lakeTableSnapshot = new LakeTableSnapshot(snapshotId, bucketLogEndOffset); - // Write version 1 format data directly to ZK (simulating old system behavior) - String zkPath = ZkData.LakeTableZNode.path(tableId); - byte[] version1Data = - LakeTableSnapshotJsonSerde.toJsonVersion1(lakeTableSnapshot, tableId); - zooKeeperClient - .getCuratorClient() - .create() - .creatingParentsIfNeeded() - .forPath(zkPath, version1Data); + // Write version 1 format data(simulating old system behavior) + lakeTableHelper.upsertLakeTableV1(tableId, lakeTableSnapshot); // Verify version 1 data can be read Optional optionalLakeTable = zooKeeperClient.getLakeTable(tableId); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerdeTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerdeTest.java index c2598e41c0..16555ef912 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerdeTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerdeTest.java @@ -18,11 +18,20 @@ package org.apache.fluss.server.zk.data.lake; import org.apache.fluss.fs.FsPath; +import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.utils.json.JsonSerdeTestBase; +import org.apache.fluss.utils.json.JsonSerdeUtils; +import org.junit.jupiter.api.Test; + +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; + +import static org.assertj.core.api.Assertions.assertThat; /** Test for {@link LakeTableJsonSerde}. */ class LakeTableJsonSerdeTest extends JsonSerdeTestBase { @@ -33,21 +42,21 @@ class LakeTableJsonSerdeTest extends JsonSerdeTestBase { @Override protected LakeTable[] createObjects() { - // Test case 1: Empty lake snapshots list + // Test case 1: Empty lake snapshots list (v2) LakeTable lakeTable1 = new LakeTable(Collections.emptyList()); - // Test case 2: Single snapshot metadata with readable offsets + // Test case 2: Single snapshot metadata with readable offsets (v2) LakeTable.LakeSnapshotMetadata metadata1 = new LakeTable.LakeSnapshotMetadata( 1L, new FsPath("/path/to/tiered1"), new FsPath("/path/to/readable1")); LakeTable lakeTable2 = new LakeTable(Collections.singletonList(metadata1)); - // Test case 3: Single snapshot metadata without readable offsets + // Test case 3: Single snapshot metadata without readable offsets (v2) LakeTable.LakeSnapshotMetadata metadata2 = new LakeTable.LakeSnapshotMetadata(2L, new FsPath("/path/to/tiered2"), null); LakeTable lakeTable3 = new LakeTable(Collections.singletonList(metadata2)); - // Test case 4: Multiple snapshot metadata + // Test case 4: Multiple snapshot metadata (v2) List metadatas = new ArrayList<>(); metadatas.add( new LakeTable.LakeSnapshotMetadata( @@ -58,20 +67,82 @@ protected LakeTable[] createObjects() { metadatas.add(new LakeTable.LakeSnapshotMetadata(5L, new FsPath("/path/to/tiered5"), null)); LakeTable lakeTable4 = new LakeTable(metadatas); - return new LakeTable[] {lakeTable1, lakeTable2, lakeTable3, lakeTable4}; + // Test case 5: Version 1 format - non-partition table + long tableId = 4; + Map bucketLogEndOffset = new HashMap<>(); + bucketLogEndOffset.put(new TableBucket(tableId, 0), 100L); + bucketLogEndOffset.put(new TableBucket(tableId, 1), 200L); + LakeTableSnapshot snapshot1 = new LakeTableSnapshot(10L, bucketLogEndOffset); + LakeTable lakeTable5 = new LakeTable(snapshot1); + + // Test case 6: Version 1 format - partition table + tableId = 5; + bucketLogEndOffset = new HashMap<>(); + bucketLogEndOffset.put(new TableBucket(tableId, 1L, 0), 100L); + bucketLogEndOffset.put(new TableBucket(tableId, 1L, 1), 200L); + bucketLogEndOffset.put(new TableBucket(tableId, 2L, 0), 300L); + LakeTableSnapshot snapshot2 = new LakeTableSnapshot(11L, bucketLogEndOffset); + LakeTable lakeTable6 = new LakeTable(snapshot2); + + return new LakeTable[] { + lakeTable1, lakeTable2, lakeTable3, lakeTable4, lakeTable5, lakeTable6 + }; } @Override protected String[] expectedJsons() { return new String[] { - // Test case 1: Empty lake snapshots list + // Test case 1: Empty lake snapshots list (v2) "{\"version\":2,\"lake_snapshots\":[]}", - // Test case 2: Single snapshot metadata with readable offsets + // Test case 2: Single snapshot metadata with readable offsets (v2) "{\"version\":2,\"lake_snapshots\":[{\"snapshot_id\":1,\"tiered_offsets\":\"/path/to/tiered1\",\"readable_offsets\":\"/path/to/readable1\"}]}", - // Test case 3: Single snapshot metadata without readable offsets + // Test case 3: Single snapshot metadata without readable offsets (v2) "{\"version\":2,\"lake_snapshots\":[{\"snapshot_id\":2,\"tiered_offsets\":\"/path/to/tiered2\"}]}", - // Test case 4: Multiple snapshot metadata - "{\"version\":2,\"lake_snapshots\":[{\"snapshot_id\":3,\"tiered_offsets\":\"/path/to/tiered3\",\"readable_offsets\":\"/path/to/readable3\"},{\"snapshot_id\":4,\"tiered_offsets\":\"/path/to/tiered4\",\"readable_offsets\":\"/path/to/readable4\"},{\"snapshot_id\":5,\"tiered_offsets\":\"/path/to/tiered5\"}]}" + // Test case 4: Multiple snapshot metadata (v2) + "{\"version\":2,\"lake_snapshots\":[{\"snapshot_id\":3,\"tiered_offsets\":\"/path/to/tiered3\",\"readable_offsets\":\"/path/to/readable3\"},{\"snapshot_id\":4,\"tiered_offsets\":\"/path/to/tiered4\",\"readable_offsets\":\"/path/to/readable4\"},{\"snapshot_id\":5,\"tiered_offsets\":\"/path/to/tiered5\"}]}", + // Test case 5: Version 1 format - non-partition table + "{\"version\":1,\"snapshot_id\":10,\"table_id\":4,\"buckets\":[{\"bucket_id\":1,\"log_end_offset\":200},{\"bucket_id\":0,\"log_end_offset\":100}]}", + // Test case 6: Version 1 format - partition table + "{\"version\":1,\"snapshot_id\":11,\"table_id\":5,\"buckets\":[{\"partition_id\":1,\"bucket_id\":1,\"log_end_offset\":200},{\"partition_id\":1,\"bucket_id\":0,\"log_end_offset\":100},{\"partition_id\":2,\"bucket_id\":0,\"log_end_offset\":300}]}", }; } + + @Test + void testVersion1Compatibility() { + // Test that Version 1 format can be deserialized correctly + // Test case 1: Non-partition table + String version1Json1 = + "{\"version\":1,\"snapshot_id\":10,\"table_id\":4," + + "\"buckets\":[{\"bucket_id\":0,\"log_end_offset\":100},{\"bucket_id\":1,\"log_end_offset\":200}]}"; + LakeTable actual1 = + JsonSerdeUtils.readValue( + version1Json1.getBytes(StandardCharsets.UTF_8), + LakeTableJsonSerde.INSTANCE); + + // Create expected LakeTableSnapshot + Map expectedBuckets1 = new HashMap<>(); + expectedBuckets1.put(new TableBucket(4L, 0), 100L); + expectedBuckets1.put(new TableBucket(4L, 1), 200L); + LakeTableSnapshot expectedSnapshot1 = new LakeTableSnapshot(10L, expectedBuckets1); + assertThat(actual1.getLakeTableSnapshot()).isEqualTo(expectedSnapshot1); + + // Test case 2: Partition table + String version1Json2 = + "{\"version\":1,\"snapshot_id\":11,\"table_id\":5," + + "\"buckets\":[{\"partition_id\":1,\"bucket_id\":0,\"log_end_offset\":100}," + + "{\"partition_id\":1,\"bucket_id\":1,\"log_end_offset\":200}," + + "{\"partition_id\":2,\"bucket_id\":0,\"log_end_offset\":300}]}"; + LakeTable actual2 = + JsonSerdeUtils.readValue( + version1Json2.getBytes(StandardCharsets.UTF_8), + LakeTableJsonSerde.INSTANCE); + + // Create expected LakeTableSnapshot + Map expectedBuckets2 = new HashMap<>(); + expectedBuckets2.put(new TableBucket(5L, 1L, 0), 100L); + expectedBuckets2.put(new TableBucket(5L, 1L, 1), 200L); + expectedBuckets2.put(new TableBucket(5L, 2L, 0), 300L); + LakeTableSnapshot expectedSnapshot2 = new LakeTableSnapshot(11L, expectedBuckets2); + assertThat(actual2.getLakeTableSnapshot()).isEqualTo(expectedSnapshot2); + } } From 1da58b430f4ee2ef8c28ba85f947599fb613505a Mon Sep 17 00:00:00 2001 From: luoyuxia Date: Wed, 24 Dec 2025 21:40:28 +0800 Subject: [PATCH 3/7] refactor --- .../fluss/utils/json/TableBucketOffsets.java | 18 +++ .../json/TableBucketOffsetsJsonSerde.java | 131 +++++++++--------- .../FlussTableLakeSnapshotCommitter.java | 61 ++++---- .../enumerator/FlinkSourceEnumeratorTest.java | 2 +- fluss-rpc/src/main/proto/FlussApi.proto | 78 ++++++----- .../apache/fluss/server/RpcServiceBase.java | 28 ++-- .../server/utils/ServerRpcMessageUtils.java | 21 ++- .../fluss/server/zk/data/lake/LakeTable.java | 8 +- .../server/zk/data/lake/LakeTableHelper.java | 110 ++++----------- .../zk/data/lake/LakeTableJsonSerde.java | 2 +- .../data/lake/LakeTableSnapshotJsonSerde.java | 23 ++- .../zk/data/lake/LakeTableHelperTest.java | 19 ++- 12 files changed, 250 insertions(+), 251 deletions(-) diff --git a/fluss-common/src/main/java/org/apache/fluss/utils/json/TableBucketOffsets.java b/fluss-common/src/main/java/org/apache/fluss/utils/json/TableBucketOffsets.java index d66b524b6d..030d3da180 100644 --- a/fluss-common/src/main/java/org/apache/fluss/utils/json/TableBucketOffsets.java +++ b/fluss-common/src/main/java/org/apache/fluss/utils/json/TableBucketOffsets.java @@ -76,6 +76,24 @@ public Map getOffsets() { return offsets; } + /** + * Serialize to a JSON byte array. + * + * @see TableBucketOffsetsJsonSerde + */ + public byte[] toJsonBytes() { + return JsonSerdeUtils.writeValueAsBytes(this, TableBucketOffsetsJsonSerde.INSTANCE); + } + + /** + * Deserialize from JSON byte array to an instance of {@link TableBucketOffsets}. + * + * @see TableBucketOffsets + */ + public static TableBucketOffsets fromJsonBytes(byte[] json) { + return JsonSerdeUtils.readValue(json, TableBucketOffsetsJsonSerde.INSTANCE); + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/fluss-common/src/main/java/org/apache/fluss/utils/json/TableBucketOffsetsJsonSerde.java b/fluss-common/src/main/java/org/apache/fluss/utils/json/TableBucketOffsetsJsonSerde.java index 20355810ec..a177c53f66 100644 --- a/fluss-common/src/main/java/org/apache/fluss/utils/json/TableBucketOffsetsJsonSerde.java +++ b/fluss-common/src/main/java/org/apache/fluss/utils/json/TableBucketOffsetsJsonSerde.java @@ -26,7 +26,6 @@ import java.util.ArrayList; import java.util.Comparator; import java.util.HashMap; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.TreeMap; @@ -54,7 +53,7 @@ *

The serialized format includes: * *

    - *
  • "version": 1 - the format version + *
  • "version": the format version *
  • "table_id": the table ID that all buckets belong to *
  • "bucket_offsets": array of offsets for non-partitioned table buckets (optional) *
  • "partition_offsets": array of partition offset objects for partitioned table buckets @@ -75,72 +74,6 @@ public class TableBucketOffsetsJsonSerde private static final int VERSION = 1; private static final long UNKNOWN_OFFSET = -1; - /** - * Deserializes a JSON node to a {@link TableBucketOffsets} object. - * - *

    This method reads the JSON format and reconstructs the table bucket offsets map. The array - * index in "bucket_offsets" represents the bucket id, and the value represents the offset. - * - * @param node the JSON node to deserialize - * @return the deserialized {@link TableBucketOffsets} object - * @throws IllegalArgumentException if the version is not supported - */ - @Override - public TableBucketOffsets deserialize(JsonNode node) { - int version = node.get(VERSION_KEY).asInt(); - if (version != VERSION) { - throw new IllegalArgumentException("Unsupported version: " + version); - } - - long tableId = node.get(TABLE_ID_KEY).asLong(); - Map offsets = new HashMap<>(); - - // Deserialize non-partitioned table bucket offsets - JsonNode bucketOffsetsNode = node.get(BUCKET_OFFSETS_KEY); - JsonNode partitionBucketOffsetsNode = node.get(PARTITION_OFFSETS_KEY); - if (bucketOffsetsNode != null || partitionBucketOffsetsNode != null) { - if (bucketOffsetsNode != null && partitionBucketOffsetsNode != null) { - throw new IllegalArgumentException( - "Both bucket_offsets and partition_bucket_offsets cannot be present at the same time"); - } - - if (bucketOffsetsNode != null) { - int bucketId = 0; - for (JsonNode bucketOffsetNode : bucketOffsetsNode) { - long offset = bucketOffsetNode.asLong(); - // Ignore unknown offsets (filled for missing bucket ids) - if (offset != UNKNOWN_OFFSET) { - TableBucket tableBucket = new TableBucket(tableId, bucketId); - offsets.put(tableBucket, offset); - } - bucketId++; - } - } else { - for (JsonNode partitionOffsetNode : partitionBucketOffsetsNode) { - long partitionId = partitionOffsetNode.get(PARTITION_ID_KEY).asLong(); - JsonNode bucketOffsetsArray = partitionOffsetNode.get(BUCKET_OFFSETS_KEY); - if (bucketOffsetsArray != null && bucketOffsetsArray.isArray()) { - Iterator elements = bucketOffsetsArray.elements(); - int bucketId = 0; - while (elements.hasNext()) { - JsonNode offsetNode = elements.next(); - long offset = offsetNode.asLong(); - // Ignore unknown offsets (filled for missing bucket ids) - if (offset != UNKNOWN_OFFSET) { - TableBucket tableBucket = - new TableBucket(tableId, partitionId, bucketId); - offsets.put(tableBucket, offset); - } - bucketId++; - } - } - } - } - } - - return new TableBucketOffsets(tableId, offsets); - } - /** * Serializes a {@link TableBucketOffsets} object to JSON format. * @@ -219,6 +152,68 @@ public void serialize(TableBucketOffsets tableBucketOffsets, JsonGenerator gener generator.writeEndObject(); } + /** + * Deserializes a JSON node to a {@link TableBucketOffsets} object. + * + *

    This method reads the JSON format and reconstructs the table bucket offsets map. The array + * index in "bucket_offsets" represents the bucket id, and the value represents the offset. + * + * @param node the JSON node to deserialize + * @return the deserialized {@link TableBucketOffsets} object + * @throws IllegalArgumentException if the version is not supported + */ + @Override + public TableBucketOffsets deserialize(JsonNode node) { + int version = node.get(VERSION_KEY).asInt(); + if (version != VERSION) { + throw new IllegalArgumentException("Unsupported version: " + version); + } + + long tableId = node.get(TABLE_ID_KEY).asLong(); + Map offsets = new HashMap<>(); + + // Deserialize non-partitioned table bucket offsets + JsonNode bucketOffsetsNode = node.get(BUCKET_OFFSETS_KEY); + JsonNode partitionBucketOffsetsNode = node.get(PARTITION_OFFSETS_KEY); + if (bucketOffsetsNode != null || partitionBucketOffsetsNode != null) { + if (bucketOffsetsNode != null && partitionBucketOffsetsNode != null) { + throw new IllegalArgumentException( + "Both bucket_offsets and partition_bucket_offsets cannot be present at the same time"); + } + + if (bucketOffsetsNode != null) { + int bucketId = 0; + for (JsonNode bucketOffsetNode : bucketOffsetsNode) { + long offset = bucketOffsetNode.asLong(); + // Ignore unknown offsets (filled for missing bucket ids) + if (offset != UNKNOWN_OFFSET) { + TableBucket tableBucket = new TableBucket(tableId, bucketId); + offsets.put(tableBucket, offset); + } + bucketId++; + } + } else { + for (JsonNode partitionOffsetNode : partitionBucketOffsetsNode) { + long partitionId = partitionOffsetNode.get(PARTITION_ID_KEY).asLong(); + JsonNode bucketOffsetsArray = partitionOffsetNode.get(BUCKET_OFFSETS_KEY); + int bucketId = 0; + for (JsonNode bucketOffsetNode : bucketOffsetsArray) { + long offset = bucketOffsetNode.asLong(); + // Ignore unknown offsets (filled for missing bucket ids) + if (offset != UNKNOWN_OFFSET) { + TableBucket tableBucket = + new TableBucket(tableId, partitionId, bucketId); + offsets.put(tableBucket, offset); + } + bucketId++; + } + } + } + } + + return new TableBucketOffsets(tableId, offsets); + } + private void serializeBucketLogEndOffset( Map bucketLogEndOffset, List buckets, 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 bfc44b53d9..1998219c39 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 @@ -26,13 +26,16 @@ import org.apache.fluss.metrics.registry.MetricRegistry; import org.apache.fluss.rpc.GatewayClientProxy; import org.apache.fluss.rpc.RpcClient; +import org.apache.fluss.rpc.gateway.AdminReadOnlyGateway; import org.apache.fluss.rpc.gateway.CoordinatorGateway; import org.apache.fluss.rpc.messages.CommitLakeTableSnapshotRequest; +import org.apache.fluss.rpc.messages.PbBucketOffset; import org.apache.fluss.rpc.messages.PbCommitLakeTableSnapshotRespForTable; import org.apache.fluss.rpc.messages.PbLakeTableOffsetForBucket; import org.apache.fluss.rpc.messages.PbLakeTableSnapshotInfo; import org.apache.fluss.rpc.messages.PbLakeTableSnapshotMetadata; import org.apache.fluss.rpc.messages.PbPrepareCommitLakeTableRespForTable; +import org.apache.fluss.rpc.messages.PbTableBucketOffsets; import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotRequest; import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotResponse; import org.apache.fluss.rpc.metrics.ClientMetricGroup; @@ -66,6 +69,7 @@ public class FlussTableLakeSnapshotCommitter implements AutoCloseable { private final Configuration flussConf; private CoordinatorGateway coordinatorGateway; + private AdminReadOnlyGateway readOnlyGateway; private RpcClient rpcClient; public FlussTableLakeSnapshotCommitter(Configuration flussConf) { @@ -83,17 +87,22 @@ public void open() { this.coordinatorGateway = GatewayClientProxy.createGatewayProxy( metadataUpdater::getCoordinatorServer, rpcClient, CoordinatorGateway.class); + + this.readOnlyGateway = + GatewayClientProxy.createGatewayProxy( + metadataUpdater::getRandomTabletServer, + rpcClient, + AdminReadOnlyGateway.class); } String prepareCommit(long tableId, TablePath tablePath, Map logEndOffsets) throws IOException { - PbPrepareCommitLakeTableRespForTable prepareCommitResp = null; - Exception exception = null; + PbPrepareCommitLakeTableRespForTable prepareCommitResp; try { PrepareCommitLakeTableSnapshotRequest prepareCommitLakeTableSnapshotRequest = toPrepareCommitLakeTableSnapshotRequest(tableId, tablePath, logEndOffsets); PrepareCommitLakeTableSnapshotResponse prepareCommitLakeTableSnapshotResponse = - coordinatorGateway + readOnlyGateway .prepareCommitLakeTableSnapshot(prepareCommitLakeTableSnapshotRequest) .get(); List pbPrepareCommitLakeTableRespForTables = @@ -101,20 +110,17 @@ String prepareCommit(long tableId, TablePath tablePath, Map l checkState(pbPrepareCommitLakeTableRespForTables.size() == 1); prepareCommitResp = pbPrepareCommitLakeTableRespForTables.get(0); if (prepareCommitResp.hasErrorCode()) { - exception = ApiError.fromErrorMessage(prepareCommitResp).exception(); + throw ApiError.fromErrorMessage(prepareCommitResp).exception(); + } else { + return checkNotNull(prepareCommitResp).getLakeTableSnapshotFilePath(); } } catch (Exception e) { - exception = e; - } - - if (exception != null) { throw new IOException( String.format( "Fail to prepare commit table lake snapshot for %s to Fluss.", tablePath), - ExceptionUtils.stripExecutionException(exception)); + ExceptionUtils.stripExecutionException(e)); } - return checkNotNull(prepareCommitResp).getLakeTableSnapshotFilePath(); } void commit( @@ -124,7 +130,6 @@ void commit( Map logEndOffsets, Map logMaxTieredTimestamps) throws IOException { - Exception exception = null; try { CommitLakeTableSnapshotRequest request = toCommitLakeTableSnapshotRequest( @@ -139,13 +144,9 @@ void commit( PbCommitLakeTableSnapshotRespForTable commitLakeTableSnapshotRes = commitLakeTableSnapshotRespForTables.get(0); if (commitLakeTableSnapshotRes.hasErrorCode()) { - exception = ApiError.fromErrorMessage(commitLakeTableSnapshotRes).exception(); + throw ApiError.fromErrorMessage(commitLakeTableSnapshotRes).exception(); } - } catch (Exception e) { - exception = e; - } - - if (exception != null) { + } catch (Exception exception) { throw new IOException( String.format( "Fail to commit table lake snapshot id %d of table %d to Fluss.", @@ -166,26 +167,22 @@ private PrepareCommitLakeTableSnapshotRequest toPrepareCommitLakeTableSnapshotRe long tableId, TablePath tablePath, Map logEndOffsets) { PrepareCommitLakeTableSnapshotRequest prepareCommitLakeTableSnapshotRequest = new PrepareCommitLakeTableSnapshotRequest(); - PbLakeTableSnapshotInfo pbLakeTableSnapshotInfo = - prepareCommitLakeTableSnapshotRequest.addTablesReq(); - pbLakeTableSnapshotInfo.setTableId(tableId); + PbTableBucketOffsets pbTableBucketOffsets = + prepareCommitLakeTableSnapshotRequest.addBucketOffset(); + pbTableBucketOffsets.setTableId(tableId); + pbTableBucketOffsets + .setTablePath() + .setDatabaseName(tablePath.getDatabaseName()) + .setTableName(tablePath.getTableName()); - // in prepare phase, we don't know the snapshot id, - // set -1 since the field is required - pbLakeTableSnapshotInfo.setSnapshotId(-1L); for (Map.Entry logEndOffsetEntry : logEndOffsets.entrySet()) { - PbLakeTableOffsetForBucket pbLakeTableOffsetForBucket = - pbLakeTableSnapshotInfo.addBucketsReq(); + PbBucketOffset pbBucketOffset = pbTableBucketOffsets.addBucketOffset(); TableBucket tableBucket = logEndOffsetEntry.getKey(); - pbLakeTableSnapshotInfo - .setTablePath() - .setDatabaseName(tablePath.getDatabaseName()) - .setTableName(tablePath.getTableName()); if (tableBucket.getPartitionId() != null) { - pbLakeTableOffsetForBucket.setPartitionId(tableBucket.getPartitionId()); + pbBucketOffset.setPartitionId(tableBucket.getPartitionId()); } - pbLakeTableOffsetForBucket.setBucketId(tableBucket.getBucket()); - pbLakeTableOffsetForBucket.setLogEndOffset(logEndOffsetEntry.getValue()); + pbBucketOffset.setBucketId(tableBucket.getBucket()); + pbBucketOffset.setLogEndOffset(logEndOffsetEntry.getValue()); } return prepareCommitLakeTableSnapshotRequest; } diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumeratorTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumeratorTest.java index 6dfce8e4df..51525da351 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumeratorTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumeratorTest.java @@ -617,7 +617,7 @@ void testPartitionsExpiredInFlussButExistInLake( new TableBucket(tableId, hybridPartitionId, 1), lakeEndOffset, new TableBucket(tableId, hybridPartitionId, 2), lakeEndOffset)); LakeTableHelper lakeTableHelper = new LakeTableHelper(zooKeeperClient, tempDir.toString()); - lakeTableHelper.upsertLakeTable(tableId, DEFAULT_TABLE_PATH, lakeTableSnapshot); + lakeTableHelper.upsertLakeTableV1(tableId, lakeTableSnapshot); // Create PartitionInfo for lake partitions List lakePartitionInfos = new ArrayList<>(); diff --git a/fluss-rpc/src/main/proto/FlussApi.proto b/fluss-rpc/src/main/proto/FlussApi.proto index ec3560c81d..f468f9eaa9 100644 --- a/fluss-rpc/src/main/proto/FlussApi.proto +++ b/fluss-rpc/src/main/proto/FlussApi.proto @@ -450,19 +450,13 @@ message NotifyRemoteLogOffsetsResponse { } message PrepareCommitLakeTableSnapshotRequest { - repeated PbLakeTableSnapshotInfo tables_req = 1; + repeated PbTableBucketOffsets bucket_offsets = 1; } message PrepareCommitLakeTableSnapshotResponse { repeated PbPrepareCommitLakeTableRespForTable prepare_commit_lake_table_resp = 1; } -message PbPrepareCommitLakeTableRespForTable { - optional string lake_table_snapshot_file_path = 1; - optional int32 error_code = 2; - optional string error_message = 3; -} - message CommitLakeTableSnapshotRequest { // Deprecated: PbLakeTableSnapshotInfo is no longer used for committing lake table snapshots. // Currently, it is only used to allow the coordinator to notify tablet servers about the current @@ -478,32 +472,6 @@ message CommitLakeTableSnapshotRequest { repeated PbLakeTableSnapshotMetadata lake_table_snapshot_metadata = 2; } -message PbLakeTableSnapshotMetadata { - required int64 table_id = 1; - required int64 snapshot_id = 2; - required string tiered_snapshot_file_path = 3; - optional string readable_snapshot_file_path = 4; -} - -message PbLakeTableSnapshotInfo { - optional int64 table_id = 1; - required int64 snapshot_id = 2; - repeated PbLakeTableOffsetForBucket buckets_req = 3; - // add table path to reduce get table_path by table id - optional PbTablePath table_path = 4; -} - -message PbLakeTableOffsetForBucket { - optional int64 partition_id = 1; - required int32 bucket_id = 2; - // Deprecated: log_start_offset is no longer used. Field number 3 is reserved for protocol compatibility. - // optional int64 log_start_offset = 3; - optional int64 log_end_offset = 4; - // Deprecated: partition_name is no longer used. Field number 5 is reserved for protocol compatibility. - // optional string partition_name = 5; - optional int64 max_timestamp = 6; -} - message CommitLakeTableSnapshotResponse { repeated PbCommitLakeTableSnapshotRespForTable table_resp = 1; } @@ -1046,4 +1014,48 @@ message PbRebalancePlanForBucket { optional int32 new_leader = 4; repeated int32 original_replicas = 5 [packed = true]; repeated int32 new_replicas = 6 [packed = true]; +} + +message PbLakeTableSnapshotMetadata { + required int64 table_id = 1; + required int64 snapshot_id = 2; + required string tiered_snapshot_file_path = 3; + optional string readable_snapshot_file_path = 4; +} + +message PbLakeTableSnapshotInfo { + optional int64 table_id = 1; + required int64 snapshot_id = 2; + repeated PbLakeTableOffsetForBucket buckets_req = 3; + // add table path to reduce get table_path by table id + optional PbTablePath table_path = 4; +} + +message PbLakeTableOffsetForBucket { + optional int64 partition_id = 1; + required int32 bucket_id = 2; + // Deprecated: log_start_offset is no longer used. Field number 3 is reserved for protocol compatibility. + // optional int64 log_start_offset = 3; + optional int64 log_end_offset = 4; + // Deprecated: partition_name is no longer used. Field number 5 is reserved for protocol compatibility. + // optional string partition_name = 5; + optional int64 max_timestamp = 6; +} + +message PbPrepareCommitLakeTableRespForTable { + optional string lake_table_snapshot_file_path = 1; + optional int32 error_code = 2; + optional string error_message = 3; +} + +message PbTableBucketOffsets { + required int64 table_id = 1; + required PbTablePath table_path = 2; + repeated PbBucketOffset bucket_offsets = 3; +} + +message PbBucketOffset { + optional int64 partition_id = 1; + required int32 bucket_id = 2; + optional int64 log_end_offset = 4; } \ No newline at end of file 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 0b78264ba3..0695c14f41 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 @@ -74,8 +74,8 @@ import org.apache.fluss.rpc.messages.MetadataRequest; import org.apache.fluss.rpc.messages.MetadataResponse; import org.apache.fluss.rpc.messages.PbApiVersion; -import org.apache.fluss.rpc.messages.PbLakeTableSnapshotInfo; import org.apache.fluss.rpc.messages.PbPrepareCommitLakeTableRespForTable; +import org.apache.fluss.rpc.messages.PbTableBucketOffsets; import org.apache.fluss.rpc.messages.PbTablePath; import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotRequest; import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotResponse; @@ -104,6 +104,7 @@ import org.apache.fluss.server.zk.data.BucketSnapshot; import org.apache.fluss.server.zk.data.lake.LakeTableHelper; import org.apache.fluss.server.zk.data.lake.LakeTableSnapshot; +import org.apache.fluss.utils.json.TableBucketOffsets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -130,9 +131,9 @@ 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; -import static org.apache.fluss.server.utils.ServerRpcMessageUtils.toLakeSnapshot; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.toListPartitionInfosResponse; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.toPbConfigEntries; +import static org.apache.fluss.server.utils.ServerRpcMessageUtils.toTableBucketOffsets; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.toTablePath; import static org.apache.fluss.utils.Preconditions.checkState; @@ -617,26 +618,21 @@ public CompletableFuture prepareCommitLa PrepareCommitLakeTableSnapshotResponse response = new PrepareCommitLakeTableSnapshotResponse(); try { - for (PbLakeTableSnapshotInfo pbLakeTableSnapshotInfo : - request.getTablesReqsList()) { + for (PbTableBucketOffsets bucketOffsets : request.getBucketOffsetsList()) { PbPrepareCommitLakeTableRespForTable pbPrepareCommitLakeTableRespForTable = response.addPrepareCommitLakeTableResp(); try { // upsert lake table snapshot, need to merge the snapshot with - // previous - // latest snapshot - LakeTableSnapshot lakeTableSnapshot = - lakeTableHelper.upsertLakeTableSnapshot( - pbLakeTableSnapshotInfo.getTableId(), - toLakeSnapshot(pbLakeTableSnapshotInfo)); - TablePath tablePath = - toTablePath(pbLakeTableSnapshotInfo.getTablePath()); + // previous latest snapshot + TableBucketOffsets tableBucketOffsets = + lakeTableHelper.upsertTableBucketOffsets( + bucketOffsets.getTableId(), + toTableBucketOffsets(bucketOffsets)); + TablePath tablePath = toTablePath(bucketOffsets.getTablePath()); FsPath fsPath = - lakeTableHelper.storeLakeTableSnapshot( - pbLakeTableSnapshotInfo.getTableId(), - tablePath, - lakeTableSnapshot); + lakeTableHelper.storeLakeTableBucketOffsets( + tablePath, tableBucketOffsets); pbPrepareCommitLakeTableRespForTable.setLakeTableSnapshotFilePath( fsPath.toString()); } catch (Exception e) { 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 e2732a8d08..018d2b6ec9 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 @@ -88,6 +88,7 @@ import org.apache.fluss.rpc.messages.PbAdjustIsrRespForTable; import org.apache.fluss.rpc.messages.PbAlterConfig; import org.apache.fluss.rpc.messages.PbBucketMetadata; +import org.apache.fluss.rpc.messages.PbBucketOffset; import org.apache.fluss.rpc.messages.PbCreateAclRespInfo; import org.apache.fluss.rpc.messages.PbDescribeConfig; import org.apache.fluss.rpc.messages.PbDropAclsFilterResult; @@ -126,6 +127,7 @@ import org.apache.fluss.rpc.messages.PbStopReplicaReqForBucket; import org.apache.fluss.rpc.messages.PbStopReplicaRespForBucket; import org.apache.fluss.rpc.messages.PbTableBucket; +import org.apache.fluss.rpc.messages.PbTableBucketOffsets; import org.apache.fluss.rpc.messages.PbTableMetadata; import org.apache.fluss.rpc.messages.PbTablePath; import org.apache.fluss.rpc.messages.PbValue; @@ -170,6 +172,7 @@ import org.apache.fluss.server.zk.data.lake.LakeTableSnapshot; import org.apache.fluss.utils.json.DataTypeJsonSerde; import org.apache.fluss.utils.json.JsonSerdeUtils; +import org.apache.fluss.utils.json.TableBucketOffsets; import org.apache.fluss.utils.types.Tuple2; import javax.annotation.Nullable; @@ -1586,9 +1589,21 @@ public static CommitLakeTableSnapshotData getCommitLakeTableSnapshotData( lakeTableInfoByTableId, tableBucketsMaxTimestamp, lakeSnapshotMetadatas); } - public static LakeTableSnapshot toLakeSnapshot( - PbLakeTableSnapshotInfo pbLakeTableSnapshotInfo) { - return toLakeSnapshot(pbLakeTableSnapshotInfo, null); + public static TableBucketOffsets toTableBucketOffsets( + PbTableBucketOffsets pbTableBucketOffsets) { + Map bucketOffsets = new HashMap<>(); + long tableId = pbTableBucketOffsets.getTableId(); + for (PbBucketOffset pbBucketOffset : pbTableBucketOffsets.getBucketOffsetsList()) { + TableBucket tableBucket = + new TableBucket( + tableId, + pbBucketOffset.hasPartitionId() + ? pbBucketOffset.getPartitionId() + : null, + pbBucketOffset.getBucketId()); + bucketOffsets.put(tableBucket, pbBucketOffset.getLogEndOffset()); + } + return new TableBucketOffsets(tableId, bucketOffsets); } private static LakeTableSnapshot toLakeSnapshot( 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 74b7da9013..6f9d3629f5 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 @@ -24,8 +24,7 @@ import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.server.zk.data.ZkData; import org.apache.fluss.utils.IOUtils; -import org.apache.fluss.utils.json.JsonSerdeUtils; -import org.apache.fluss.utils.json.TableBucketOffsetsJsonSerde; +import org.apache.fluss.utils.json.TableBucketOffsets; import javax.annotation.Nullable; @@ -132,10 +131,7 @@ public LakeTableSnapshot getLatestTableSnapshot() throws Exception { try (ByteArrayOutputStream outputStream = new ByteArrayOutputStream()) { IOUtils.copyBytes(inputStream, outputStream, true); Map logOffsets = - JsonSerdeUtils.readValue( - outputStream.toByteArray(), - TableBucketOffsetsJsonSerde.INSTANCE) - .getOffsets(); + TableBucketOffsets.fromJsonBytes(outputStream.toByteArray()).getOffsets(); return new LakeTableSnapshot(lakeSnapshotMetadata.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 eb0aecac5b..8f3498fb00 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 @@ -25,9 +25,7 @@ import org.apache.fluss.metadata.TablePath; import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.utils.FlussPaths; -import org.apache.fluss.utils.json.JsonSerdeUtils; import org.apache.fluss.utils.json.TableBucketOffsets; -import org.apache.fluss.utils.json.TableBucketOffsetsJsonSerde; import java.util.HashMap; import java.util.List; @@ -59,76 +57,17 @@ public void upsertLakeTableV1(long tableId, LakeTableSnapshot lakeTableSnapshot) Optional optPreviousLakeTable = zkClient.getLakeTable(tableId); // Merge with previous snapshot if exists if (optPreviousLakeTable.isPresent()) { - lakeTableSnapshot = - mergeLakeTable( - optPreviousLakeTable.get().getLatestTableSnapshot(), lakeTableSnapshot); + TableBucketOffsets tableBucketOffsets = + mergeTableBucketOffsets( + optPreviousLakeTable.get(), + new TableBucketOffsets( + tableId, lakeTableSnapshot.getBucketLogEndOffset())); + lakeTableSnapshot = new LakeTableSnapshot(tableId, tableBucketOffsets.getOffsets()); } zkClient.upsertLakeTable( tableId, new LakeTable(lakeTableSnapshot), optPreviousLakeTable.isPresent()); } - /** - * Upserts a lake table snapshot for the given table. - * - *

    This method merges the new snapshot with the existing one (if any) and stores it (data in - * remote file, the remote file path in ZK). - * - * @param tableId the table ID - * @param tablePath the table path - * @param lakeTableSnapshot the new snapshot to upsert - * @throws Exception if the operation fails - */ - public void upsertLakeTable( - long tableId, TablePath tablePath, LakeTableSnapshot lakeTableSnapshot) - throws Exception { - Optional optPreviousLakeTable = zkClient.getLakeTable(tableId); - // Merge with previous snapshot if exists - if (optPreviousLakeTable.isPresent()) { - lakeTableSnapshot = - mergeLakeTable( - optPreviousLakeTable.get().getLatestTableSnapshot(), lakeTableSnapshot); - } - - // store the lake table snapshot into a file - FsPath lakeTableSnapshotFsPath = - storeLakeTableSnapshot(tableId, tablePath, lakeTableSnapshot); - - LakeTable.LakeSnapshotMetadata lakeSnapshotMetadata = - new LakeTable.LakeSnapshotMetadata( - lakeTableSnapshot.getSnapshotId(), - // use the lake table snapshot file as the tiered offsets file since - // the table snapshot file will contain the tiered log end offsets - lakeTableSnapshotFsPath, - // currently, readableOffsetsFilePath is always same with - // tieredOffsetsFilePath, but in the future we'll commit a readable offsets - // separately to mark what the readable offsets are for a snapshot since - // in paimon dv table, tiered log end offsets is not same with readable - // offsets - lakeTableSnapshotFsPath); - - // 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 - LakeTable lakeTable = new LakeTable(lakeSnapshotMetadata); - try { - zkClient.upsertLakeTable(tableId, lakeTable, optPreviousLakeTable.isPresent()); - } catch (Exception e) { - LOG.warn("Failed to upsert lake table snapshot to zk.", e); - // discard the new lake snapshot metadata - lakeSnapshotMetadata.discard(); - throw e; - } - - if (optPreviousLakeTable.isPresent()) { - // discard previous latest lake snapshot - LakeTable.LakeSnapshotMetadata previousLakeSnapshotMetadata = - optPreviousLakeTable.get().getLatestLakeSnapshotMetadata(); - if (previousLakeSnapshotMetadata != null) { - previousLakeSnapshotMetadata.discard(); - } - } - } - public void addLakeTableSnapshotMetadata( long tableId, LakeTable.LakeSnapshotMetadata lakeSnapshotMetadata) throws Exception { Optional optPreviousLakeTable = zkClient.getLakeTable(tableId); @@ -145,41 +84,43 @@ public void addLakeTableSnapshotMetadata( 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); } } - public LakeTableSnapshot upsertLakeTableSnapshot( - long tableId, LakeTableSnapshot newLakeTableSnapshot) throws Exception { + public TableBucketOffsets upsertTableBucketOffsets( + long tableId, TableBucketOffsets newTableBucketOffsets) throws Exception { Optional optPreviousLakeTable = zkClient.getLakeTable(tableId); // Merge with previous snapshot if exists if (optPreviousLakeTable.isPresent()) { - return mergeLakeTable( - optPreviousLakeTable.get().getLatestTableSnapshot(), newLakeTableSnapshot); + return mergeTableBucketOffsets(optPreviousLakeTable.get(), newTableBucketOffsets); } - return newLakeTableSnapshot; + return newTableBucketOffsets; } - private LakeTableSnapshot mergeLakeTable( - LakeTableSnapshot previousLakeTableSnapshot, LakeTableSnapshot newLakeTableSnapshot) { - // Merge current snapshot with previous one since the current snapshot request + private TableBucketOffsets mergeTableBucketOffsets( + LakeTable previousLakeTable, TableBucketOffsets newTableBucketOffsets) + throws Exception { + // Merge current with previous one since the current request // may not carry all buckets for the table. It typically only carries buckets // that were written after the previous commit. // merge log end offsets, current will override the previous Map bucketLogEndOffset = - new HashMap<>(previousLakeTableSnapshot.getBucketLogEndOffset()); - bucketLogEndOffset.putAll(newLakeTableSnapshot.getBucketLogEndOffset()); - - return new LakeTableSnapshot(newLakeTableSnapshot.getSnapshotId(), bucketLogEndOffset); + new HashMap<>(previousLakeTable.getLatestTableSnapshot().getBucketLogEndOffset()); + bucketLogEndOffset.putAll(newTableBucketOffsets.getOffsets()); + return new TableBucketOffsets(newTableBucketOffsets.getTableId(), bucketLogEndOffset); } - public FsPath storeLakeTableSnapshot( - long tableId, TablePath tablePath, LakeTableSnapshot lakeTableSnapshot) - throws Exception { + public FsPath storeLakeTableBucketOffsets( + TablePath tablePath, TableBucketOffsets tableBucketOffsets) throws Exception { // get the remote file path to store the lake table snapshot offset information + long tableId = tableBucketOffsets.getTableId(); FsPath remoteLakeTableSnapshotOffsetPath = FlussPaths.remoteLakeTableSnapshotOffsetPath(remoteDataDir, tablePath, tableId); // check whether the parent directory exists, if not, create the directory @@ -188,10 +129,7 @@ public FsPath storeLakeTableSnapshot( fileSystem.mkdirs(remoteLakeTableSnapshotOffsetPath.getParent()); } // serialize table offsets to json bytes, and write to file - byte[] jsonBytes = - JsonSerdeUtils.writeValueAsBytes( - new TableBucketOffsets(tableId, lakeTableSnapshot.getBucketLogEndOffset()), - TableBucketOffsetsJsonSerde.INSTANCE); + byte[] jsonBytes = tableBucketOffsets.toJsonBytes(); try (FSDataOutputStream outputStream = fileSystem.create( diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerde.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerde.java index e6de853fba..a74e939a94 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerde.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerde.java @@ -61,7 +61,7 @@ public class LakeTableJsonSerde implements JsonSerializer, JsonDeseri @Override public void serialize(LakeTable lakeTable, JsonGenerator generator) throws IOException { - // if lake table snapshot is null, it must be version 1 + // if lake table snapshot is not null, it must be version 1 if (lakeTable.getLakeTableSnapshot() != null) { // Version 1: ZK node contains full snapshot data, use LakeTableSnapshotJsonSerde LakeTableSnapshotJsonSerde.INSTANCE.serialize( diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableSnapshotJsonSerde.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableSnapshotJsonSerde.java index b8d34ed893..7c96c4d45c 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableSnapshotJsonSerde.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableSnapshotJsonSerde.java @@ -28,7 +28,28 @@ import java.util.HashMap; import java.util.Map; -/** Json serializer and deserializer for {@link LakeTableSnapshot}. */ +/** + * Json serializer and deserializer for {@link LakeTableSnapshot}. + * + *

    Note: This class is primarily used for backward compatibility to deserialize legacy + * version 1 lake snapshot data stored in ZooKeeper. The current storage format (version 2) stores + * only file paths in ZooKeeper, with actual snapshot data stored in remote files. This serde is + * used by {@link LakeTableJsonSerde} to handle version 1 format deserialization. + * + *

    The version 1 format stores the full {@link LakeTableSnapshot} data directly in the ZooKeeper + * node, which includes: + * + *

      + *
    • version: 1 + *
    • snapshot_id: the snapshot ID + *
    • table_id: the table ID (derived from the first bucket) + *
    • buckets: array of bucket objects, each containing bucket_id, optional partition_id, and + * log_end_offset + *
    + * + * @see LakeTableJsonSerde for the current format (version 2) that uses this serde for legacy + * compatibility + */ public class LakeTableSnapshotJsonSerde implements JsonSerializer, JsonDeserializer { 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 d8aad22f6b..17f5650786 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 @@ -31,6 +31,7 @@ import org.apache.fluss.server.zk.ZooKeeperUtils; import org.apache.fluss.server.zk.data.TableRegistration; import org.apache.fluss.testutils.common.AllCallbackWrapper; +import org.apache.fluss.utils.json.TableBucketOffsets; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; @@ -125,8 +126,13 @@ void testUpsertLakeTableCompatible(@TempDir Path tempDir) throws Exception { newBucketLogEndOffset.put(new TableBucket(tableId, 1), 2000L); // new offset long snapshot2Id = 2L; - LakeTableSnapshot snapshot2 = new LakeTableSnapshot(snapshot2Id, newBucketLogEndOffset); - lakeTableHelper.upsertLakeTable(tableId, tablePath, snapshot2); + FsPath tieredOffsetsPath = + lakeTableHelper.storeLakeTableBucketOffsets( + tablePath, new TableBucketOffsets(tableId, newBucketLogEndOffset)); + lakeTableHelper.addLakeTableSnapshotMetadata( + tableId, + new LakeTable.LakeSnapshotMetadata( + snapshot2Id, tieredOffsetsPath, tieredOffsetsPath)); // Verify: New version 2 data can be read Optional optLakeTableAfter = zooKeeperClient.getLakeTable(tableId); @@ -155,8 +161,13 @@ void testUpsertLakeTableCompatible(@TempDir Path tempDir) throws Exception { // add a new snapshot 3 again, verify snapshot long snapshot3Id = 3L; - LakeTableSnapshot snapshot3 = new LakeTableSnapshot(snapshot3Id, newBucketLogEndOffset); - lakeTableHelper.upsertLakeTable(tableId, tablePath, snapshot3); + tieredOffsetsPath = + lakeTableHelper.storeLakeTableBucketOffsets( + tablePath, new TableBucketOffsets(tableId, newBucketLogEndOffset)); + lakeTableHelper.addLakeTableSnapshotMetadata( + tableId, + new LakeTable.LakeSnapshotMetadata( + snapshot3Id, tieredOffsetsPath, tieredOffsetsPath)); // verify snapshot 3 is discarded assertThat(fileSystem.exists(snapshot2FileHandle)).isFalse(); } From 68949e18feed5b51e0462ce06cfe376758e5633b Mon Sep 17 00:00:00 2001 From: luoyuxia Date: Thu, 25 Dec 2025 07:56:08 +0800 Subject: [PATCH 4/7] minor fix --- .../lake/committer/CommittedLakeSnapshot.java | 1 + .../fluss/lake/committer/LakeCommitter.java | 6 ++ .../FlussTableLakeSnapshotCommitter.java | 16 ++--- .../committer/TieringCommitOperator.java | 15 ++-- .../enumerator/FlinkSourceEnumeratorTest.java | 2 +- .../committer/TieringCommitOperatorTest.java | 2 +- .../FlinkIcebergTieringTestBase.java | 2 +- .../lance/tiering/LanceTieringITCase.java | 2 +- .../lake/lance/tiering/LanceTieringTest.java | 2 +- .../testutils/FlinkPaimonTieringTestBase.java | 2 +- .../paimon/tiering/PaimonTieringTest.java | 2 +- .../rpc/gateway/AdminReadOnlyGateway.java | 6 -- fluss-rpc/src/main/proto/FlussApi.proto | 6 +- .../apache/fluss/server/RpcServiceBase.java | 2 +- .../CoordinatorEventProcessor.java | 2 +- .../server/utils/ServerRpcMessageUtils.java | 70 +++++++------------ .../fluss/server/zk/data/lake/LakeTable.java | 2 +- .../server/zk/data/lake/LakeTableHelper.java | 6 +- .../zk/data/lake/LakeTableJsonSerde.java | 6 +- .../zk/data/lake/LakeTableHelperTest.java | 2 +- .../zk/data/lake/LakeTableJsonSerdeTest.java | 7 +- 21 files changed, 71 insertions(+), 90 deletions(-) diff --git a/fluss-common/src/main/java/org/apache/fluss/lake/committer/CommittedLakeSnapshot.java b/fluss-common/src/main/java/org/apache/fluss/lake/committer/CommittedLakeSnapshot.java index 6d3b144fa5..dfc67b2e49 100644 --- a/fluss-common/src/main/java/org/apache/fluss/lake/committer/CommittedLakeSnapshot.java +++ b/fluss-common/src/main/java/org/apache/fluss/lake/committer/CommittedLakeSnapshot.java @@ -27,6 +27,7 @@ public class CommittedLakeSnapshot { private final long lakeSnapshotId; + private final Map snapshotProperties; public CommittedLakeSnapshot(long lakeSnapshotId, Map snapshotProperties) { 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 5f87d7f5a2..607dc78667 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 @@ -36,6 +36,12 @@ @PublicEvolving public interface LakeCommitter extends AutoCloseable { + /** + * The property key used to store the file path of lake table bucket offsets in snapshot + * properties. + */ + String FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY = "fluss-offsets"; + /** * Converts a list of write results to a committable object. * 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 1998219c39..61706f99bc 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 @@ -56,7 +56,7 @@ * information in Fluss: * *
      - *
    • Prepare phase ({@link #prepareCommit}): Sends log end offsets to the FLuss cluster, + *
    • Prepare phase ({@link #prepareCommit}): Sends log end offsets to the Fluss cluster, * which merges them with the previous log end offsets and stores the merged snapshot data in * a file. Returns the file path where the snapshot metadata is stored. *
    • Commit phase ({@link #commit}): Sends the lake snapshot metadata (including snapshot @@ -112,7 +112,7 @@ String prepareCommit(long tableId, TablePath tablePath, Map l if (prepareCommitResp.hasErrorCode()) { throw ApiError.fromErrorMessage(prepareCommitResp).exception(); } else { - return checkNotNull(prepareCommitResp).getLakeTableSnapshotFilePath(); + return checkNotNull(prepareCommitResp).getLakeTableBucketOffsetsPath(); } } catch (Exception e) { throw new IOException( @@ -126,7 +126,7 @@ String prepareCommit(long tableId, TablePath tablePath, Map l void commit( long tableId, long lakeSnapshotId, - String lakeSnapshotPath, + String lakeBucketOffsetsPath, Map logEndOffsets, Map logMaxTieredTimestamps) throws IOException { @@ -135,7 +135,7 @@ void commit( toCommitLakeTableSnapshotRequest( tableId, lakeSnapshotId, - lakeSnapshotPath, + lakeBucketOffsetsPath, logEndOffsets, logMaxTieredTimestamps); List commitLakeTableSnapshotRespForTables = @@ -200,7 +200,7 @@ private PrepareCommitLakeTableSnapshotRequest toPrepareCommitLakeTableSnapshotRe * * @param tableId the table ID * @param snapshotId the lake snapshot ID - * @param lakeSnapshotPath the file path where the snapshot metadata is stored + * @param bucketOffsetsPath the file path where the bucket offsets is stored * @param logEndOffsets the log end offsets for each bucket * @param logMaxTieredTimestamps the max tiered timestamps for each bucket * @return the commit request @@ -208,7 +208,7 @@ private PrepareCommitLakeTableSnapshotRequest toPrepareCommitLakeTableSnapshotRe private CommitLakeTableSnapshotRequest toCommitLakeTableSnapshotRequest( long tableId, long snapshotId, - String lakeSnapshotPath, + String bucketOffsetsPath, Map logEndOffsets, Map logMaxTieredTimestamps) { CommitLakeTableSnapshotRequest commitLakeTableSnapshotRequest = @@ -220,8 +220,8 @@ private CommitLakeTableSnapshotRequest toCommitLakeTableSnapshotRequest( pbLakeTableSnapshotMetadata.setSnapshotId(snapshotId); pbLakeTableSnapshotMetadata.setTableId(tableId); // tiered snapshot file path is equal to readable snapshot currently - pbLakeTableSnapshotMetadata.setTieredSnapshotFilePath(lakeSnapshotPath); - pbLakeTableSnapshotMetadata.setReadableSnapshotFilePath(lakeSnapshotPath); + pbLakeTableSnapshotMetadata.setTieredBucketOffsetsFilePath(bucketOffsetsPath); + pbLakeTableSnapshotMetadata.setReadableBucketOffsetsFilePath(bucketOffsetsPath); // 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 fb0a4bf1be..c4f4d9e1c3 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 @@ -58,7 +58,7 @@ import java.util.Set; import java.util.stream.Collectors; -import static org.apache.fluss.lake.committer.BucketOffset.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY; +import static org.apache.fluss.lake.committer.LakeCommitter.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY; import static org.apache.fluss.utils.Preconditions.checkState; /** @@ -231,22 +231,21 @@ private Committable commitWriteResults( ? null : flussCurrentLakeSnapshot.getSnapshotId()); - // get the lake snapshot file storing the log end offsets - String lakeSnapshotMetadataFile = + // get the lake bucket offsets file storing the log end offsets + String lakeBucketOffsetsFile = flussTableLakeSnapshotCommitter.prepareCommit( tableId, tablePath, logEndOffsets); - // record the lake snapshot metadata file to snapshot property + // record the lake snapshot bucket offsets file to snapshot property long committedSnapshotId = lakeCommitter.commit( committable, Collections.singletonMap( - FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY, - lakeSnapshotMetadataFile)); + FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY, lakeBucketOffsetsFile)); flussTableLakeSnapshotCommitter.commit( tableId, committedSnapshotId, - lakeSnapshotMetadataFile, + lakeBucketOffsetsFile, logEndOffsets, logMaxTieredTimestamps); return committable; @@ -304,7 +303,7 @@ private void checkFlussNotMissingLakeSnapshot( // since this code path should be rare, we do not consider backward compatibility // and throw IllegalStateException directly String trimmedPath = lakeSnapshotOffsetPath.trim(); - if (trimmedPath.startsWith("{")) { + if (trimmedPath.contains("{")) { throw new IllegalStateException( String.format( "The %s field in snapshot property is a JSON string (tiered by v0.8), " diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumeratorTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumeratorTest.java index 51525da351..fb92345c78 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumeratorTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumeratorTest.java @@ -617,7 +617,7 @@ void testPartitionsExpiredInFlussButExistInLake( new TableBucket(tableId, hybridPartitionId, 1), lakeEndOffset, new TableBucket(tableId, hybridPartitionId, 2), lakeEndOffset)); LakeTableHelper lakeTableHelper = new LakeTableHelper(zooKeeperClient, tempDir.toString()); - lakeTableHelper.upsertLakeTableV1(tableId, lakeTableSnapshot); + lakeTableHelper.upsertLakeTable(tableId, lakeTableSnapshot); // Create PartitionInfo for lake partitions List lakePartitionInfos = new ArrayList<>(); diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperatorTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperatorTest.java index 290b99244f..d325e0786e 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperatorTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperatorTest.java @@ -56,7 +56,7 @@ import java.util.List; import java.util.Map; -import static org.apache.fluss.lake.committer.BucketOffset.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY; +import static org.apache.fluss.lake.committer.LakeCommitter.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY; import static org.apache.fluss.record.TestData.DATA1_PARTITIONED_TABLE_DESCRIPTOR; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; diff --git a/fluss-lake/fluss-lake-iceberg/src/test/java/org/apache/fluss/lake/iceberg/testutils/FlinkIcebergTieringTestBase.java b/fluss-lake/fluss-lake-iceberg/src/test/java/org/apache/fluss/lake/iceberg/testutils/FlinkIcebergTieringTestBase.java index d6408384c3..29a583f7e8 100644 --- a/fluss-lake/fluss-lake-iceberg/src/test/java/org/apache/fluss/lake/iceberg/testutils/FlinkIcebergTieringTestBase.java +++ b/fluss-lake/fluss-lake-iceberg/src/test/java/org/apache/fluss/lake/iceberg/testutils/FlinkIcebergTieringTestBase.java @@ -78,7 +78,7 @@ import java.util.TreeSet; import static org.apache.fluss.flink.tiering.source.TieringSourceOptions.POLL_TIERING_TABLE_INTERVAL; -import static org.apache.fluss.lake.committer.BucketOffset.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY; +import static org.apache.fluss.lake.committer.LakeCommitter.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY; import static org.apache.fluss.lake.iceberg.utils.IcebergConversions.toIceberg; import static org.apache.fluss.metadata.TableDescriptor.OFFSET_COLUMN_NAME; import static org.apache.fluss.testutils.DataTestUtils.row; diff --git a/fluss-lake/fluss-lake-lance/src/test/java/org/apache/fluss/lake/lance/tiering/LanceTieringITCase.java b/fluss-lake/fluss-lake-lance/src/test/java/org/apache/fluss/lake/lance/tiering/LanceTieringITCase.java index 877421e257..2180554aa3 100644 --- a/fluss-lake/fluss-lake-lance/src/test/java/org/apache/fluss/lake/lance/tiering/LanceTieringITCase.java +++ b/fluss-lake/fluss-lake-lance/src/test/java/org/apache/fluss/lake/lance/tiering/LanceTieringITCase.java @@ -45,7 +45,7 @@ import java.util.List; import java.util.Map; -import static org.apache.fluss.lake.committer.BucketOffset.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY; +import static org.apache.fluss.lake.committer.LakeCommitter.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY; import static org.apache.fluss.testutils.DataTestUtils.row; import static org.assertj.core.api.Assertions.assertThat; 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 019b21f543..fa4ddfb492 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 @@ -64,7 +64,7 @@ import java.util.Map; import java.util.stream.Stream; -import static org.apache.fluss.lake.committer.BucketOffset.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY; +import static org.apache.fluss.lake.committer.LakeCommitter.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY; import static org.assertj.core.api.Assertions.assertThat; /** The UT for tiering to Lance via {@link LanceLakeTieringFactory}. */ diff --git a/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/testutils/FlinkPaimonTieringTestBase.java b/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/testutils/FlinkPaimonTieringTestBase.java index 7d66afd54f..76fc332bed 100644 --- a/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/testutils/FlinkPaimonTieringTestBase.java +++ b/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/testutils/FlinkPaimonTieringTestBase.java @@ -71,7 +71,7 @@ import java.util.Set; import static org.apache.fluss.flink.tiering.source.TieringSourceOptions.POLL_TIERING_TABLE_INTERVAL; -import static org.apache.fluss.lake.committer.BucketOffset.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY; +import static org.apache.fluss.lake.committer.LakeCommitter.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY; import static org.apache.fluss.testutils.DataTestUtils.row; import static org.apache.fluss.testutils.common.CommonTestUtils.retry; import static org.apache.fluss.testutils.common.CommonTestUtils.waitUntil; 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 602180b8c1..cadd2a46ad 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 @@ -71,7 +71,7 @@ import java.util.Map; import java.util.stream.Stream; -import static org.apache.fluss.lake.committer.BucketOffset.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY; +import static org.apache.fluss.lake.committer.LakeCommitter.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY; import static org.apache.fluss.lake.paimon.utils.PaimonConversions.toPaimon; import static org.apache.fluss.metadata.TableDescriptor.BUCKET_COLUMN_NAME; import static org.apache.fluss.metadata.TableDescriptor.OFFSET_COLUMN_NAME; 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 c31e1796fe..8ab16118f2 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 @@ -211,12 +211,6 @@ CompletableFuture describeClusterConfigs( *
    • Returns the file path where the snapshot is stored *
    * - *

    The returned file path points to a file that stores the bucket log end offset information - * for the table. This file path will be used in the subsequent commit phase to reference the - * stored snapshot. If any error occurs during processing for a specific table, an error - * response will be set for that table in the response, while other tables will continue to be - * processed. - * * @param request the request containing lake table snapshot information for one or more tables * @return a future that completes with a response containing the file paths where snapshots * (containing bucket log end offset information) are stored, or error information for diff --git a/fluss-rpc/src/main/proto/FlussApi.proto b/fluss-rpc/src/main/proto/FlussApi.proto index f468f9eaa9..ff0bd8b82f 100644 --- a/fluss-rpc/src/main/proto/FlussApi.proto +++ b/fluss-rpc/src/main/proto/FlussApi.proto @@ -1019,8 +1019,8 @@ message PbRebalancePlanForBucket { message PbLakeTableSnapshotMetadata { required int64 table_id = 1; required int64 snapshot_id = 2; - required string tiered_snapshot_file_path = 3; - optional string readable_snapshot_file_path = 4; + required string tiered_bucket_offsets_file_path = 3; + optional string readable_bucket_offsets_file_path = 4; } message PbLakeTableSnapshotInfo { @@ -1043,7 +1043,7 @@ message PbLakeTableOffsetForBucket { } message PbPrepareCommitLakeTableRespForTable { - optional string lake_table_snapshot_file_path = 1; + optional string lake_table_bucket_offsets_path = 1; optional int32 error_code = 2; optional string error_message = 3; } 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 0695c14f41..ba2646d437 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 @@ -633,7 +633,7 @@ public CompletableFuture prepareCommitLa FsPath fsPath = lakeTableHelper.storeLakeTableBucketOffsets( tablePath, tableBucketOffsets); - pbPrepareCommitLakeTableRespForTable.setLakeTableSnapshotFilePath( + pbPrepareCommitLakeTableRespForTable.setLakeTableBucketOffsetsPath( fsPath.toString()); } catch (Exception e) { Errors error = ApiError.fromThrowable(e).error(); 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 f193507a0d..1a51c0aece 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 @@ -1305,7 +1305,7 @@ private void handleCommitLakeTableSnapshotV1( } // this involves IO operation (ZK), so we do it in ioExecutor - lakeTableHelper.upsertLakeTableV1( + lakeTableHelper.upsertLakeTable( tableId, lakeTableSnapshotEntry.getValue()); } catch (Exception e) { ApiError error = ApiError.fromThrowable(e); 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 018d2b6ec9..05caa40daf 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 @@ -173,7 +173,6 @@ import org.apache.fluss.utils.json.DataTypeJsonSerde; import org.apache.fluss.utils.json.JsonSerdeUtils; import org.apache.fluss.utils.json.TableBucketOffsets; -import org.apache.fluss.utils.types.Tuple2; import javax.annotation.Nullable; @@ -190,7 +189,6 @@ import java.util.Optional; import java.util.OptionalInt; import java.util.Set; -import java.util.function.Consumer; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -1556,17 +1554,28 @@ public static CommitLakeTableSnapshotData getCommitLakeTableSnapshotData( Map tableBucketsMaxTimestamp = new HashMap<>(); for (PbLakeTableSnapshotInfo pbLakeTableSnapshotInfo : request.getTablesReqsList()) { long tableId = pbLakeTableSnapshotInfo.getTableId(); - Consumer> - pbLakeTableOffsetForBucketConsumer = - (tableBucketAndOffset -> { - if (tableBucketAndOffset.f1.hasMaxTimestamp()) { - tableBucketsMaxTimestamp.put( - tableBucketAndOffset.f0, - tableBucketAndOffset.f1.getMaxTimestamp()); - } - }); + long snapshotId = pbLakeTableSnapshotInfo.getSnapshotId(); + Map bucketLogEndOffset = new HashMap<>(); + for (PbLakeTableOffsetForBucket lakeTableOffsetForBucket : + pbLakeTableSnapshotInfo.getBucketsReqsList()) { + Long partitionId = + lakeTableOffsetForBucket.hasPartitionId() + ? lakeTableOffsetForBucket.getPartitionId() + : null; + int bucketId = lakeTableOffsetForBucket.getBucketId(); + TableBucket tableBucket = new TableBucket(tableId, partitionId, bucketId); + Long logEndOffset = + lakeTableOffsetForBucket.hasLogEndOffset() + ? lakeTableOffsetForBucket.getLogEndOffset() + : null; + if (lakeTableOffsetForBucket.hasMaxTimestamp()) { + tableBucketsMaxTimestamp.put( + tableBucket, lakeTableOffsetForBucket.getMaxTimestamp()); + } + bucketLogEndOffset.put(tableBucket, logEndOffset); + } LakeTableSnapshot lakeTableSnapshot = - toLakeSnapshot(pbLakeTableSnapshotInfo, pbLakeTableOffsetForBucketConsumer); + new LakeTableSnapshot(snapshotId, bucketLogEndOffset); lakeTableInfoByTableId.put(tableId, lakeTableSnapshot); } @@ -1578,11 +1587,12 @@ public static CommitLakeTableSnapshotData getCommitLakeTableSnapshotData( pbLakeTableSnapshotMetadata.getTableId(), new LakeTable.LakeSnapshotMetadata( pbLakeTableSnapshotMetadata.getSnapshotId(), - new FsPath(pbLakeTableSnapshotMetadata.getTieredSnapshotFilePath()), - pbLakeTableSnapshotMetadata.hasReadableSnapshotFilePath() + new FsPath( + pbLakeTableSnapshotMetadata.getTieredBucketOffsetsFilePath()), + pbLakeTableSnapshotMetadata.hasReadableBucketOffsetsFilePath() ? new FsPath( pbLakeTableSnapshotMetadata - .getReadableSnapshotFilePath()) + .getReadableBucketOffsetsFilePath()) : null)); } return new CommitLakeTableSnapshotData( @@ -1606,36 +1616,6 @@ public static TableBucketOffsets toTableBucketOffsets( return new TableBucketOffsets(tableId, bucketOffsets); } - private static LakeTableSnapshot toLakeSnapshot( - PbLakeTableSnapshotInfo pbLakeTableSnapshotInfo, - @Nullable - Consumer> - pbLakeTableOffsetForBucketConsumer) { - long tableId = pbLakeTableSnapshotInfo.getTableId(); - long snapshotId = pbLakeTableSnapshotInfo.getSnapshotId(); - Map bucketLogEndOffset = new HashMap<>(); - for (PbLakeTableOffsetForBucket lakeTableOffsetForBucket : - pbLakeTableSnapshotInfo.getBucketsReqsList()) { - Long partitionId = - lakeTableOffsetForBucket.hasPartitionId() - ? lakeTableOffsetForBucket.getPartitionId() - : null; - int bucketId = lakeTableOffsetForBucket.getBucketId(); - - TableBucket tableBucket = new TableBucket(tableId, partitionId, bucketId); - Long logEndOffset = - lakeTableOffsetForBucket.hasLogEndOffset() - ? lakeTableOffsetForBucket.getLogEndOffset() - : null; - if (pbLakeTableOffsetForBucketConsumer != null) { - pbLakeTableOffsetForBucketConsumer.accept( - Tuple2.of(tableBucket, lakeTableOffsetForBucket)); - } - bucketLogEndOffset.put(tableBucket, logEndOffset); - } - return new LakeTableSnapshot(snapshotId, bucketLogEndOffset); - } - public static PbNotifyLakeTableOffsetReqForBucket makeNotifyLakeTableOffsetForBucket( TableBucket tableBucket, LakeTableSnapshot lakeTableSnapshot, 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 6f9d3629f5..f4fd85a138 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 @@ -120,7 +120,7 @@ public LakeTableSnapshot getLakeTableSnapshot() { * * @return the LakeTableSnapshot */ - public LakeTableSnapshot getLatestTableSnapshot() throws Exception { + public LakeTableSnapshot getLatestTableSnapshot() throws IOException { if (lakeTableSnapshot != null) { return lakeTableSnapshot; } 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 8f3498fb00..88f4e3f98a 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 @@ -46,13 +46,14 @@ public LakeTableHelper(ZooKeeperClient zkClient, String remoteDataDir) { } /** - * Upserts a lake table snapshot for the given table, stored in v1 format. + * Upserts a lake table snapshot for the given table, stored in v1 format. Note: this method is + * just for back compatibility. * * @param tableId the table ID * @param lakeTableSnapshot the new snapshot to upsert * @throws Exception if the operation fails */ - public void upsertLakeTableV1(long tableId, LakeTableSnapshot lakeTableSnapshot) + public void upsertLakeTable(long tableId, LakeTableSnapshot lakeTableSnapshot) throws Exception { Optional optPreviousLakeTable = zkClient.getLakeTable(tableId); // Merge with previous snapshot if exists @@ -75,7 +76,6 @@ public void addLakeTableSnapshotMetadata( if (optPreviousLakeTable.isPresent()) { previousLakeSnapshotMetadatas = optPreviousLakeTable.get().getLakeSnapshotMetadatas(); } - // currently, we always keep only one snapshot metadata LakeTable lakeTable = new LakeTable(lakeSnapshotMetadata); try { zkClient.upsertLakeTable(tableId, lakeTable, optPreviousLakeTable.isPresent()); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerde.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerde.java index a74e939a94..db7838c834 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerde.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerde.java @@ -61,11 +61,11 @@ public class LakeTableJsonSerde implements JsonSerializer, JsonDeseri @Override public void serialize(LakeTable lakeTable, JsonGenerator generator) throws IOException { - // if lake table snapshot is not null, it must be version 1 - if (lakeTable.getLakeTableSnapshot() != null) { + // if lake table snapshot metadata is null, it must be version 1 + if (lakeTable.getLakeSnapshotMetadatas() == null) { // Version 1: ZK node contains full snapshot data, use LakeTableSnapshotJsonSerde LakeTableSnapshotJsonSerde.INSTANCE.serialize( - lakeTable.getLakeTableSnapshot(), generator); + lakeTable.getLatestTableSnapshot(), generator); } else { generator.writeStartObject(); generator.writeNumberField(VERSION_KEY, CURRENT_VERSION); 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 17f5650786..855ad7880d 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 @@ -111,7 +111,7 @@ void testUpsertLakeTableCompatible(@TempDir Path tempDir) throws Exception { LakeTableSnapshot lakeTableSnapshot = new LakeTableSnapshot(snapshotId, bucketLogEndOffset); // Write version 1 format data(simulating old system behavior) - lakeTableHelper.upsertLakeTableV1(tableId, lakeTableSnapshot); + lakeTableHelper.upsertLakeTable(tableId, lakeTableSnapshot); // Verify version 1 data can be read Optional optionalLakeTable = zooKeeperClient.getLakeTable(tableId); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerdeTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerdeTest.java index 16555ef912..9bf486ab02 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerdeTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerdeTest.java @@ -24,6 +24,7 @@ import org.junit.jupiter.api.Test; +import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collections; @@ -108,7 +109,7 @@ protected String[] expectedJsons() { } @Test - void testVersion1Compatibility() { + void testVersion1Compatibility() throws IOException { // Test that Version 1 format can be deserialized correctly // Test case 1: Non-partition table String version1Json1 = @@ -124,7 +125,7 @@ void testVersion1Compatibility() { expectedBuckets1.put(new TableBucket(4L, 0), 100L); expectedBuckets1.put(new TableBucket(4L, 1), 200L); LakeTableSnapshot expectedSnapshot1 = new LakeTableSnapshot(10L, expectedBuckets1); - assertThat(actual1.getLakeTableSnapshot()).isEqualTo(expectedSnapshot1); + assertThat(actual1.getLatestTableSnapshot()).isEqualTo(expectedSnapshot1); // Test case 2: Partition table String version1Json2 = @@ -143,6 +144,6 @@ void testVersion1Compatibility() { expectedBuckets2.put(new TableBucket(5L, 1L, 1), 200L); expectedBuckets2.put(new TableBucket(5L, 2L, 0), 300L); LakeTableSnapshot expectedSnapshot2 = new LakeTableSnapshot(11L, expectedBuckets2); - assertThat(actual2.getLakeTableSnapshot()).isEqualTo(expectedSnapshot2); + assertThat(actual2.getLatestTableSnapshot()).isEqualTo(expectedSnapshot2); } } From 716b3e080a1cb2ffd2f7eeb26d13c27c05792a6c Mon Sep 17 00:00:00 2001 From: luoyuxia Date: Fri, 26 Dec 2025 10:18:52 +0800 Subject: [PATCH 5/7] move prepare commit to coordinator gateway --- .../FlussTableLakeSnapshotCommitter.java | 10 +--- .../rpc/gateway/AdminReadOnlyGateway.java | 26 -------- .../fluss/rpc/gateway/CoordinatorGateway.java | 26 ++++++++ .../rpc/TestingTabletGatewayService.java | 8 --- .../apache/fluss/server/RpcServiceBase.java | 59 +------------------ .../coordinator/CoordinatorService.java | 58 +++++++++++++++++- .../fluss/server/tablet/TabletServer.java | 3 +- .../fluss/server/tablet/TabletService.java | 7 +-- .../tablet/TestTabletServerGateway.java | 8 --- 9 files changed, 87 insertions(+), 118 deletions(-) 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 61706f99bc..0b4627ab37 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 @@ -26,7 +26,6 @@ import org.apache.fluss.metrics.registry.MetricRegistry; import org.apache.fluss.rpc.GatewayClientProxy; import org.apache.fluss.rpc.RpcClient; -import org.apache.fluss.rpc.gateway.AdminReadOnlyGateway; import org.apache.fluss.rpc.gateway.CoordinatorGateway; import org.apache.fluss.rpc.messages.CommitLakeTableSnapshotRequest; import org.apache.fluss.rpc.messages.PbBucketOffset; @@ -69,7 +68,6 @@ public class FlussTableLakeSnapshotCommitter implements AutoCloseable { private final Configuration flussConf; private CoordinatorGateway coordinatorGateway; - private AdminReadOnlyGateway readOnlyGateway; private RpcClient rpcClient; public FlussTableLakeSnapshotCommitter(Configuration flussConf) { @@ -87,12 +85,6 @@ public void open() { this.coordinatorGateway = GatewayClientProxy.createGatewayProxy( metadataUpdater::getCoordinatorServer, rpcClient, CoordinatorGateway.class); - - this.readOnlyGateway = - GatewayClientProxy.createGatewayProxy( - metadataUpdater::getRandomTabletServer, - rpcClient, - AdminReadOnlyGateway.class); } String prepareCommit(long tableId, TablePath tablePath, Map logEndOffsets) @@ -102,7 +94,7 @@ String prepareCommit(long tableId, TablePath tablePath, Map l PrepareCommitLakeTableSnapshotRequest prepareCommitLakeTableSnapshotRequest = toPrepareCommitLakeTableSnapshotRequest(tableId, tablePath, logEndOffsets); PrepareCommitLakeTableSnapshotResponse prepareCommitLakeTableSnapshotResponse = - readOnlyGateway + coordinatorGateway .prepareCommitLakeTableSnapshot(prepareCommitLakeTableSnapshotRequest) .get(); List pbPrepareCommitLakeTableRespForTables = 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 8ab16118f2..6a6e21149f 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 @@ -46,8 +46,6 @@ import org.apache.fluss.rpc.messages.ListTablesResponse; import org.apache.fluss.rpc.messages.MetadataRequest; import org.apache.fluss.rpc.messages.MetadataResponse; -import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotRequest; -import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotResponse; import org.apache.fluss.rpc.messages.TableExistsRequest; import org.apache.fluss.rpc.messages.TableExistsResponse; import org.apache.fluss.rpc.protocol.ApiKeys; @@ -195,28 +193,4 @@ CompletableFuture getLatestLakeSnapshot( @RPC(api = ApiKeys.DESCRIBE_CLUSTER_CONFIGS) CompletableFuture describeClusterConfigs( DescribeClusterConfigsRequest request); - - /** - * Prepares to commit lake table snapshots by merging them with existing snapshots and storing - * them to the file system. - * - *

    This method is called during the two-phase commit process for lake table snapshots. It - * performs the following operations for each table in the request: - * - *

      - *
    • Merges the new snapshot with the previous latest snapshot (if exists) to ensure - * completeness - *
    • Stores the merged snapshot to the remote file system. The stored file contains the log - * end offset information for each bucket in the table - *
    • Returns the file path where the snapshot is stored - *
    - * - * @param request the request containing lake table snapshot information for one or more tables - * @return a future that completes with a response containing the file paths where snapshots - * (containing bucket log end offset information) are stored, or error information for - * tables that failed to process - */ - @RPC(api = ApiKeys.PRECOMMIT_LAKE_TABLE_SNAPSHOT) - CompletableFuture prepareCommitLakeTableSnapshot( - PrepareCommitLakeTableSnapshotRequest request); } diff --git a/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/CoordinatorGateway.java b/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/CoordinatorGateway.java index a78c9bdbdb..a6b0f99b38 100644 --- a/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/CoordinatorGateway.java +++ b/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/CoordinatorGateway.java @@ -30,6 +30,8 @@ import org.apache.fluss.rpc.messages.ControlledShutdownResponse; import org.apache.fluss.rpc.messages.LakeTieringHeartbeatRequest; import org.apache.fluss.rpc.messages.LakeTieringHeartbeatResponse; +import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotRequest; +import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotResponse; import org.apache.fluss.rpc.protocol.ApiKeys; import org.apache.fluss.rpc.protocol.RPC; @@ -85,4 +87,28 @@ CompletableFuture lakeTieringHeartbeat( @RPC(api = ApiKeys.CONTROLLED_SHUTDOWN) CompletableFuture controlledShutdown( ControlledShutdownRequest request); + + /** + * Prepares to commit lake table snapshots by merging them with existing snapshots and storing + * them to the file system. + * + *

    This method is called during the two-phase commit process for lake table snapshots. It + * performs the following operations for each table in the request: + * + *

      + *
    • Merges the new snapshot with the previous latest snapshot (if exists) to ensure + * completeness + *
    • Stores the merged snapshot to the remote file system. The stored file contains the log + * end offset information for each bucket in the table + *
    • Returns the file path where the snapshot is stored + *
    + * + * @param request the request containing lake table snapshot information for one or more tables + * @return a future that completes with a response containing the file paths where snapshots + * (containing bucket log end offset information) are stored, or error information for + * tables that failed to process + */ + @RPC(api = ApiKeys.PRECOMMIT_LAKE_TABLE_SNAPSHOT) + CompletableFuture prepareCommitLakeTableSnapshot( + PrepareCommitLakeTableSnapshotRequest request); } 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 0fa36373d4..7db3654383 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 @@ -67,8 +67,6 @@ import org.apache.fluss.rpc.messages.NotifyRemoteLogOffsetsResponse; import org.apache.fluss.rpc.messages.PrefixLookupRequest; import org.apache.fluss.rpc.messages.PrefixLookupResponse; -import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotRequest; -import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotResponse; import org.apache.fluss.rpc.messages.ProduceLogRequest; import org.apache.fluss.rpc.messages.ProduceLogResponse; import org.apache.fluss.rpc.messages.PutKvRequest; @@ -251,10 +249,4 @@ public CompletableFuture describeClusterConfigs( DescribeClusterConfigsRequest request) { return null; } - - @Override - public CompletableFuture prepareCommitLakeTableSnapshot( - PrepareCommitLakeTableSnapshotRequest request) { - throw new UnsupportedOperationException(); - } } 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 ba2646d437..b12bb787c7 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 @@ -19,8 +19,6 @@ import org.apache.fluss.cluster.ServerNode; import org.apache.fluss.cluster.ServerType; -import org.apache.fluss.config.ConfigOptions; -import org.apache.fluss.config.Configuration; import org.apache.fluss.config.cluster.ConfigEntry; import org.apache.fluss.exception.FlussRuntimeException; import org.apache.fluss.exception.KvSnapshotNotExistException; @@ -31,7 +29,6 @@ import org.apache.fluss.exception.SecurityTokenException; import org.apache.fluss.exception.TableNotPartitionedException; import org.apache.fluss.fs.FileSystem; -import org.apache.fluss.fs.FsPath; import org.apache.fluss.fs.token.ObtainedSecurityToken; import org.apache.fluss.metadata.DatabaseInfo; import org.apache.fluss.metadata.PhysicalTablePath; @@ -74,18 +71,12 @@ import org.apache.fluss.rpc.messages.MetadataRequest; import org.apache.fluss.rpc.messages.MetadataResponse; import org.apache.fluss.rpc.messages.PbApiVersion; -import org.apache.fluss.rpc.messages.PbPrepareCommitLakeTableRespForTable; -import org.apache.fluss.rpc.messages.PbTableBucketOffsets; import org.apache.fluss.rpc.messages.PbTablePath; -import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotRequest; -import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotResponse; import org.apache.fluss.rpc.messages.TableExistsRequest; import org.apache.fluss.rpc.messages.TableExistsResponse; import org.apache.fluss.rpc.netty.server.Session; -import org.apache.fluss.rpc.protocol.ApiError; import org.apache.fluss.rpc.protocol.ApiKeys; import org.apache.fluss.rpc.protocol.ApiManager; -import org.apache.fluss.rpc.protocol.Errors; import org.apache.fluss.security.acl.AclBinding; import org.apache.fluss.security.acl.AclBindingFilter; import org.apache.fluss.security.acl.OperationType; @@ -102,9 +93,7 @@ import org.apache.fluss.server.utils.ServerRpcMessageUtils; import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.server.zk.data.BucketSnapshot; -import org.apache.fluss.server.zk.data.lake.LakeTableHelper; import org.apache.fluss.server.zk.data.lake.LakeTableSnapshot; -import org.apache.fluss.utils.json.TableBucketOffsets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -133,7 +122,6 @@ import static org.apache.fluss.server.utils.ServerRpcMessageUtils.toGetFileSystemSecurityTokenResponse; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.toListPartitionInfosResponse; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.toPbConfigEntries; -import static org.apache.fluss.server.utils.ServerRpcMessageUtils.toTableBucketOffsets; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.toTablePath; import static org.apache.fluss.utils.Preconditions.checkState; @@ -153,7 +141,6 @@ public abstract class RpcServiceBase extends RpcGatewayService implements AdminR protected final MetadataManager metadataManager; protected final @Nullable Authorizer authorizer; protected final DynamicConfigManager dynamicConfigManager; - private final LakeTableHelper lakeTableHelper; private long tokenLastUpdateTimeMs = 0; private ObtainedSecurityToken securityToken = null; @@ -167,8 +154,7 @@ public RpcServiceBase( MetadataManager metadataManager, @Nullable Authorizer authorizer, DynamicConfigManager dynamicConfigManager, - ExecutorService ioExecutor, - Configuration config) { + ExecutorService ioExecutor) { this.remoteFileSystem = remoteFileSystem; this.provider = provider; this.apiManager = new ApiManager(provider); @@ -177,8 +163,6 @@ public RpcServiceBase( this.authorizer = authorizer; this.dynamicConfigManager = dynamicConfigManager; this.ioExecutor = ioExecutor; - this.lakeTableHelper = - new LakeTableHelper(zkClient, config.get(ConfigOptions.REMOTE_DATA_DIR)); } @Override @@ -607,45 +591,4 @@ protected MetadataResponse processMetadataRequest( return buildMetadataResponse( coordinatorServer, aliveTabletServers, tablesMetadata, partitionsMetadata); } - - @Override - public CompletableFuture prepareCommitLakeTableSnapshot( - PrepareCommitLakeTableSnapshotRequest request) { - CompletableFuture future = - new CompletableFuture<>(); - ioExecutor.submit( - () -> { - PrepareCommitLakeTableSnapshotResponse response = - new PrepareCommitLakeTableSnapshotResponse(); - try { - for (PbTableBucketOffsets bucketOffsets : request.getBucketOffsetsList()) { - PbPrepareCommitLakeTableRespForTable - pbPrepareCommitLakeTableRespForTable = - response.addPrepareCommitLakeTableResp(); - try { - // upsert lake table snapshot, need to merge the snapshot with - // previous latest snapshot - TableBucketOffsets tableBucketOffsets = - lakeTableHelper.upsertTableBucketOffsets( - bucketOffsets.getTableId(), - toTableBucketOffsets(bucketOffsets)); - TablePath tablePath = toTablePath(bucketOffsets.getTablePath()); - FsPath fsPath = - lakeTableHelper.storeLakeTableBucketOffsets( - tablePath, tableBucketOffsets); - pbPrepareCommitLakeTableRespForTable.setLakeTableBucketOffsetsPath( - fsPath.toString()); - } catch (Exception e) { - Errors error = ApiError.fromThrowable(e).error(); - pbPrepareCommitLakeTableRespForTable.setError( - error.code(), error.message()); - } - } - future.complete(response); - } catch (Exception e) { - future.completeExceptionally(e); - } - }); - return future; - } } 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 11b486bab4..65676821bc 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 @@ -33,6 +33,7 @@ import org.apache.fluss.exception.TableAlreadyExistException; import org.apache.fluss.exception.TableNotPartitionedException; import org.apache.fluss.fs.FileSystem; +import org.apache.fluss.fs.FsPath; import org.apache.fluss.lake.lakestorage.LakeCatalog; import org.apache.fluss.metadata.DataLakeFormat; import org.apache.fluss.metadata.DatabaseDescriptor; @@ -87,12 +88,17 @@ import org.apache.fluss.rpc.messages.PbAlterConfig; import org.apache.fluss.rpc.messages.PbHeartbeatReqForTable; import org.apache.fluss.rpc.messages.PbHeartbeatRespForTable; +import org.apache.fluss.rpc.messages.PbPrepareCommitLakeTableRespForTable; +import org.apache.fluss.rpc.messages.PbTableBucketOffsets; +import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotRequest; +import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotResponse; import org.apache.fluss.rpc.messages.RebalanceRequest; import org.apache.fluss.rpc.messages.RebalanceResponse; import org.apache.fluss.rpc.messages.RemoveServerTagRequest; import org.apache.fluss.rpc.messages.RemoveServerTagResponse; import org.apache.fluss.rpc.netty.server.Session; import org.apache.fluss.rpc.protocol.ApiError; +import org.apache.fluss.rpc.protocol.Errors; import org.apache.fluss.security.acl.AclBinding; import org.apache.fluss.security.acl.AclBindingFilter; import org.apache.fluss.security.acl.FlussPrincipal; @@ -122,8 +128,10 @@ import org.apache.fluss.server.zk.data.PartitionAssignment; import org.apache.fluss.server.zk.data.TableAssignment; import org.apache.fluss.server.zk.data.TableRegistration; +import org.apache.fluss.server.zk.data.lake.LakeTableHelper; import org.apache.fluss.utils.IOUtils; import org.apache.fluss.utils.concurrent.FutureUtils; +import org.apache.fluss.utils.json.TableBucketOffsets; import javax.annotation.Nullable; @@ -148,6 +156,7 @@ import static org.apache.fluss.server.utils.ServerRpcMessageUtils.makeDropAclsResponse; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.toAlterTableConfigChanges; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.toAlterTableSchemaChanges; +import static org.apache.fluss.server.utils.ServerRpcMessageUtils.toTableBucketOffsets; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.toTablePath; import static org.apache.fluss.server.utils.TableAssignmentUtils.generateAssignment; import static org.apache.fluss.utils.PartitionUtils.validatePartitionSpec; @@ -166,6 +175,8 @@ public final class CoordinatorService extends RpcServiceBase implements Coordina private final LakeTableTieringManager lakeTableTieringManager; private final LakeCatalogDynamicLoader lakeCatalogDynamicLoader; + private final ExecutorService ioExecutor; + private final LakeTableHelper lakeTableHelper; public CoordinatorService( Configuration conf, @@ -186,8 +197,7 @@ public CoordinatorService( metadataManager, authorizer, dynamicConfigManager, - ioExecutor, - conf); + ioExecutor); this.defaultBucketNumber = conf.getInt(ConfigOptions.DEFAULT_BUCKET_NUMBER); this.defaultReplicationFactor = conf.getInt(ConfigOptions.DEFAULT_REPLICATION_FACTOR); this.logTableAllowCreation = conf.getBoolean(ConfigOptions.LOG_TABLE_ALLOW_CREATION); @@ -199,6 +209,9 @@ public CoordinatorService( this.lakeTableTieringManager = lakeTableTieringManager; this.metadataCache = metadataCache; this.lakeCatalogDynamicLoader = lakeCatalogDynamicLoader; + this.ioExecutor = ioExecutor; + this.lakeTableHelper = + new LakeTableHelper(zkClient, conf.getString(ConfigOptions.REMOTE_DATA_DIR)); } @Override @@ -712,6 +725,47 @@ public CompletableFuture controlledShutdown( return response; } + @Override + public CompletableFuture prepareCommitLakeTableSnapshot( + PrepareCommitLakeTableSnapshotRequest request) { + CompletableFuture future = + new CompletableFuture<>(); + ioExecutor.submit( + () -> { + PrepareCommitLakeTableSnapshotResponse response = + new PrepareCommitLakeTableSnapshotResponse(); + try { + for (PbTableBucketOffsets bucketOffsets : request.getBucketOffsetsList()) { + PbPrepareCommitLakeTableRespForTable + pbPrepareCommitLakeTableRespForTable = + response.addPrepareCommitLakeTableResp(); + try { + // upsert lake table snapshot, need to merge the snapshot with + // previous latest snapshot + TableBucketOffsets tableBucketOffsets = + lakeTableHelper.upsertTableBucketOffsets( + bucketOffsets.getTableId(), + toTableBucketOffsets(bucketOffsets)); + TablePath tablePath = toTablePath(bucketOffsets.getTablePath()); + FsPath fsPath = + lakeTableHelper.storeLakeTableBucketOffsets( + tablePath, tableBucketOffsets); + pbPrepareCommitLakeTableRespForTable.setLakeTableBucketOffsetsPath( + fsPath.toString()); + } catch (Exception e) { + Errors error = ApiError.fromThrowable(e).error(); + pbPrepareCommitLakeTableRespForTable.setError( + error.code(), error.message()); + } + } + future.complete(response); + } catch (Exception e) { + future.completeExceptionally(e); + } + }); + return future; + } + @Override public CompletableFuture alterClusterConfigs( AlterClusterConfigsRequest request) { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletServer.java b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletServer.java index aa8e6348d7..8eed63c844 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletServer.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletServer.java @@ -286,8 +286,7 @@ protected void startServices() throws Exception { metadataManager, authorizer, dynamicConfigManager, - ioExecutor, - conf); + ioExecutor); RequestsMetrics requestsMetrics = RequestsMetrics.createTabletServerRequestMetrics(tabletServerMetricGroup); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java index 88dfcae96b..9a47b63830 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java @@ -18,7 +18,6 @@ package org.apache.fluss.server.tablet; import org.apache.fluss.cluster.ServerType; -import org.apache.fluss.config.Configuration; import org.apache.fluss.exception.AuthorizationException; import org.apache.fluss.exception.UnknownTableOrBucketException; import org.apache.fluss.fs.FileSystem; @@ -138,8 +137,7 @@ public TabletService( MetadataManager metadataManager, @Nullable Authorizer authorizer, DynamicConfigManager dynamicConfigManager, - ExecutorService ioExecutor, - Configuration conf) { + ExecutorService ioExecutor) { super( remoteFileSystem, ServerType.TABLET_SERVER, @@ -147,8 +145,7 @@ public TabletService( metadataManager, authorizer, dynamicConfigManager, - ioExecutor, - conf); + ioExecutor); this.serviceName = "server-" + serverId; this.replicaManager = replicaManager; this.metadataCache = metadataCache; 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 cac7b80c16..8ebcc57489 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 @@ -77,8 +77,6 @@ import org.apache.fluss.rpc.messages.PbTableBucket; import org.apache.fluss.rpc.messages.PrefixLookupRequest; import org.apache.fluss.rpc.messages.PrefixLookupResponse; -import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotRequest; -import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotResponse; import org.apache.fluss.rpc.messages.ProduceLogRequest; import org.apache.fluss.rpc.messages.ProduceLogResponse; import org.apache.fluss.rpc.messages.PutKvRequest; @@ -335,12 +333,6 @@ public CompletableFuture describeClusterConfigs( throw new UnsupportedOperationException(); } - @Override - public CompletableFuture prepareCommitLakeTableSnapshot( - PrepareCommitLakeTableSnapshotRequest request) { - throw new UnsupportedOperationException(); - } - public int pendingRequestSize() { return requests.size(); } From 97849a91fb4573a3c3baf403a80f9e305a523389 Mon Sep 17 00:00:00 2001 From: luoyuxia Date: Mon, 29 Dec 2025 11:01:44 +0800 Subject: [PATCH 6/7] address comments --- .../FlussTableLakeSnapshotCommitter.java | 65 +++++----- .../committer/TieringCommitOperator.java | 2 +- .../enumerator/FlinkSourceEnumeratorTest.java | 2 +- .../FlussTableLakeSnapshotCommitterTest.java | 7 +- .../committer/TieringCommitOperatorTest.java | 2 +- .../FlinkIcebergTieringTestBase.java | 2 +- .../lance/tiering/LanceTieringITCase.java | 2 +- .../testutils/FlinkPaimonTieringTestBase.java | 2 +- .../fluss/rpc/gateway/CoordinatorGateway.java | 52 ++++---- .../apache/fluss/rpc/protocol/ApiKeys.java | 2 +- fluss-rpc/src/main/proto/FlussApi.proto | 25 ++-- .../CoordinatorEventProcessor.java | 84 +++++++------ .../coordinator/CoordinatorService.java | 97 ++++++++------- .../server/utils/ServerRpcMessageUtils.java | 9 +- .../fluss/server/zk/ZooKeeperClient.java | 2 +- .../fluss/server/zk/data/lake/LakeTable.java | 4 +- .../server/zk/data/lake/LakeTableHelper.java | 21 +--- .../zk/data/lake/LakeTableJsonSerde.java | 112 ++++++++++-------- ... => LakeTableSnapshotLegacyJsonSerde.java} | 5 +- .../coordinator/TestCoordinatorGateway.java | 8 +- ...LakeTableSnapshotLegacyJsonSerdeTest.java} | 47 +++++++- .../zk/data/lake/LakeTableHelperTest.java | 14 +-- .../zk/data/lake/LakeTableJsonSerdeTest.java | 4 +- 23 files changed, 316 insertions(+), 254 deletions(-) rename fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/{LakeTableSnapshotJsonSerde.java => LakeTableSnapshotLegacyJsonSerde.java} (97%) rename fluss-server/src/test/java/org/apache/fluss/server/zk/data/{LakeTableSnapshotJsonSerdeTest.java => LakeTableSnapshotLegacyJsonSerdeTest.java} (52%) 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 0b4627ab37..5ba5b7fc2a 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 @@ -33,10 +33,10 @@ import org.apache.fluss.rpc.messages.PbLakeTableOffsetForBucket; import org.apache.fluss.rpc.messages.PbLakeTableSnapshotInfo; import org.apache.fluss.rpc.messages.PbLakeTableSnapshotMetadata; -import org.apache.fluss.rpc.messages.PbPrepareCommitLakeTableRespForTable; -import org.apache.fluss.rpc.messages.PbTableBucketOffsets; -import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotRequest; -import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotResponse; +import org.apache.fluss.rpc.messages.PbPrepareLakeTableRespForTable; +import org.apache.fluss.rpc.messages.PbTableOffsets; +import org.apache.fluss.rpc.messages.PrepareLakeTableSnapshotRequest; +import org.apache.fluss.rpc.messages.PrepareLakeTableSnapshotResponse; import org.apache.fluss.rpc.metrics.ClientMetricGroup; import org.apache.fluss.rpc.protocol.ApiError; import org.apache.fluss.utils.ExceptionUtils; @@ -55,9 +55,9 @@ * information in Fluss: * *
      - *
    • Prepare phase ({@link #prepareCommit}): Sends log end offsets to the Fluss cluster, - * which merges them with the previous log end offsets and stores the merged snapshot data in - * a file. Returns the file path where the snapshot metadata is stored. + *
    • Prepare phase ({@link #prepareLakeSnapshot}): Sends log end offsets to the Fluss + * cluster, which merges them with the previous log end offsets and stores the merged snapshot + * data in a file. Returns the file path where the snapshot metadata is stored. *
    • Commit phase ({@link #commit}): Sends the lake snapshot metadata (including snapshot * ID and file paths) to the coordinator to finalize the commit. Also includes log end offsets * and max tiered timestamps for metrics reporting to tablet servers. @@ -87,24 +87,30 @@ public void open() { metadataUpdater::getCoordinatorServer, rpcClient, CoordinatorGateway.class); } - String prepareCommit(long tableId, TablePath tablePath, Map logEndOffsets) + String prepareLakeSnapshot( + long tableId, TablePath tablePath, Map logEndOffsets) throws IOException { - PbPrepareCommitLakeTableRespForTable prepareCommitResp; + PbPrepareLakeTableRespForTable prepareResp; try { - PrepareCommitLakeTableSnapshotRequest prepareCommitLakeTableSnapshotRequest = - toPrepareCommitLakeTableSnapshotRequest(tableId, tablePath, logEndOffsets); - PrepareCommitLakeTableSnapshotResponse prepareCommitLakeTableSnapshotResponse = + PrepareLakeTableSnapshotRequest prepareLakeTableSnapshotRequest = + toPrepareLakeTableSnapshotRequest(tableId, tablePath, logEndOffsets); + PrepareLakeTableSnapshotResponse prepareLakeTableSnapshotResponse = coordinatorGateway - .prepareCommitLakeTableSnapshot(prepareCommitLakeTableSnapshotRequest) + .prepareLakeTableSnapshot(prepareLakeTableSnapshotRequest) .get(); - List pbPrepareCommitLakeTableRespForTables = - prepareCommitLakeTableSnapshotResponse.getPrepareCommitLakeTableRespsList(); - checkState(pbPrepareCommitLakeTableRespForTables.size() == 1); - prepareCommitResp = pbPrepareCommitLakeTableRespForTables.get(0); - if (prepareCommitResp.hasErrorCode()) { - throw ApiError.fromErrorMessage(prepareCommitResp).exception(); + List pbPrepareLakeTableRespForTables = + prepareLakeTableSnapshotResponse.getPrepareLakeTableRespsList(); + checkState(pbPrepareLakeTableRespForTables.size() == 1); + prepareResp = pbPrepareLakeTableRespForTables.get(0); + checkState( + prepareResp.getTableId() == tableId, + "TableId does not match, table id in request is %s, but got %s in response.", + tableId, + prepareResp.getTableId()); + if (prepareResp.hasErrorCode()) { + throw ApiError.fromErrorMessage(prepareResp).exception(); } else { - return checkNotNull(prepareCommitResp).getLakeTableBucketOffsetsPath(); + return checkNotNull(prepareResp).getLakeTableBucketOffsetsPath(); } } catch (Exception e) { throw new IOException( @@ -148,27 +154,26 @@ void commit( } /** - * Converts the prepare commit parameters to a {@link PrepareCommitLakeTableSnapshotRequest}. + * Converts the prepare commit parameters to a {@link PrepareLakeTableSnapshotRequest}. * * @param tableId the table ID * @param tablePath the table path * @param logEndOffsets the log end offsets for each bucket * @return the prepared commit request */ - private PrepareCommitLakeTableSnapshotRequest toPrepareCommitLakeTableSnapshotRequest( + private PrepareLakeTableSnapshotRequest toPrepareLakeTableSnapshotRequest( long tableId, TablePath tablePath, Map logEndOffsets) { - PrepareCommitLakeTableSnapshotRequest prepareCommitLakeTableSnapshotRequest = - new PrepareCommitLakeTableSnapshotRequest(); - PbTableBucketOffsets pbTableBucketOffsets = - prepareCommitLakeTableSnapshotRequest.addBucketOffset(); - pbTableBucketOffsets.setTableId(tableId); - pbTableBucketOffsets + PrepareLakeTableSnapshotRequest prepareLakeTableSnapshotRequest = + new PrepareLakeTableSnapshotRequest(); + PbTableOffsets pbTableOffsets = prepareLakeTableSnapshotRequest.addBucketOffset(); + pbTableOffsets.setTableId(tableId); + pbTableOffsets .setTablePath() .setDatabaseName(tablePath.getDatabaseName()) .setTableName(tablePath.getTableName()); for (Map.Entry logEndOffsetEntry : logEndOffsets.entrySet()) { - PbBucketOffset pbBucketOffset = pbTableBucketOffsets.addBucketOffset(); + PbBucketOffset pbBucketOffset = pbTableOffsets.addBucketOffset(); TableBucket tableBucket = logEndOffsetEntry.getKey(); if (tableBucket.getPartitionId() != null) { pbBucketOffset.setPartitionId(tableBucket.getPartitionId()); @@ -176,7 +181,7 @@ private PrepareCommitLakeTableSnapshotRequest toPrepareCommitLakeTableSnapshotRe pbBucketOffset.setBucketId(tableBucket.getBucket()); pbBucketOffset.setLogEndOffset(logEndOffsetEntry.getValue()); } - return prepareCommitLakeTableSnapshotRequest; + return prepareLakeTableSnapshotRequest; } /** 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 c4f4d9e1c3..2992705d9d 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 @@ -233,7 +233,7 @@ private Committable commitWriteResults( // get the lake bucket offsets file storing the log end offsets String lakeBucketOffsetsFile = - flussTableLakeSnapshotCommitter.prepareCommit( + flussTableLakeSnapshotCommitter.prepareLakeSnapshot( tableId, tablePath, logEndOffsets); // record the lake snapshot bucket offsets file to snapshot property diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumeratorTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumeratorTest.java index fb92345c78..0e9260a54c 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumeratorTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumeratorTest.java @@ -617,7 +617,7 @@ void testPartitionsExpiredInFlussButExistInLake( new TableBucket(tableId, hybridPartitionId, 1), lakeEndOffset, new TableBucket(tableId, hybridPartitionId, 2), lakeEndOffset)); LakeTableHelper lakeTableHelper = new LakeTableHelper(zooKeeperClient, tempDir.toString()); - lakeTableHelper.upsertLakeTable(tableId, lakeTableSnapshot); + lakeTableHelper.registerLakeTableSnapshotV1(tableId, lakeTableSnapshot); // Create PartitionInfo for lake partitions List lakePartitionInfos = new ArrayList<>(); 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 c85ace01eb..abb74ebf67 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 @@ -25,7 +25,7 @@ import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.server.zk.data.ZkData; import org.apache.fluss.server.zk.data.lake.LakeTableSnapshot; -import org.apache.fluss.server.zk.data.lake.LakeTableSnapshotJsonSerde; +import org.apache.fluss.server.zk.data.lake.LakeTableSnapshotLegacyJsonSerde; import org.apache.fluss.utils.json.JsonSerdeUtils; import org.apache.fluss.utils.types.Tuple2; @@ -76,7 +76,8 @@ void testCommit(boolean isPartitioned) throws Exception { long lakeSnapshotId = 3; String lakeSnapshotFilePath = - flussTableLakeSnapshotCommitter.prepareCommit(tableId, tablePath, expectedOffsets); + flussTableLakeSnapshotCommitter.prepareLakeSnapshot( + tableId, tablePath, expectedOffsets); // commit offsets flussTableLakeSnapshotCommitter.commit( @@ -129,7 +130,7 @@ void testCompatibilityWithOldCommitter(boolean isPartitioned) throws Exception { byte[] jsonBytes = zkClient.getOrEmpty(ZkData.LakeTableZNode.path(tableId)).get(); LakeTableSnapshot lakeTableSnapshot = - JsonSerdeUtils.readValue(jsonBytes, LakeTableSnapshotJsonSerde.INSTANCE); + JsonSerdeUtils.readValue(jsonBytes, LakeTableSnapshotLegacyJsonSerde.INSTANCE); assertThat(lakeTableSnapshot.getSnapshotId()).isEqualTo(snapshotId); assertThat(lakeTableSnapshot.getBucketLogEndOffset()).isEqualTo(logEndOffsets); } diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperatorTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperatorTest.java index d325e0786e..7313956041 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperatorTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperatorTest.java @@ -468,7 +468,7 @@ private CommittedLakeSnapshot mockCommittedLakeSnapshot( new FlussTableLakeSnapshotCommitter(FLUSS_CLUSTER_EXTENSION.getClientConfig())) { lakeSnapshotCommitter.open(); String lakeSnapshotFile = - lakeSnapshotCommitter.prepareCommit(tableId, tablePath, logEndOffsets); + lakeSnapshotCommitter.prepareLakeSnapshot(tableId, tablePath, logEndOffsets); return new CommittedLakeSnapshot( snapshotId, Collections.singletonMap( diff --git a/fluss-lake/fluss-lake-iceberg/src/test/java/org/apache/fluss/lake/iceberg/testutils/FlinkIcebergTieringTestBase.java b/fluss-lake/fluss-lake-iceberg/src/test/java/org/apache/fluss/lake/iceberg/testutils/FlinkIcebergTieringTestBase.java index 29a583f7e8..99fd340cf4 100644 --- a/fluss-lake/fluss-lake-iceberg/src/test/java/org/apache/fluss/lake/iceberg/testutils/FlinkIcebergTieringTestBase.java +++ b/fluss-lake/fluss-lake-iceberg/src/test/java/org/apache/fluss/lake/iceberg/testutils/FlinkIcebergTieringTestBase.java @@ -485,7 +485,7 @@ protected void checkFlussOffsetsInSnapshot( new LakeTable.LakeSnapshotMetadata( // don't care about snapshot id -1, new FsPath(offsetFile), null)) - .getLatestTableSnapshot() + .getOrReadLatestTableSnapshot() .getBucketLogEndOffset(); assertThat(recordedOffsets).isEqualTo(expectedOffsets); } diff --git a/fluss-lake/fluss-lake-lance/src/test/java/org/apache/fluss/lake/lance/tiering/LanceTieringITCase.java b/fluss-lake/fluss-lake-lance/src/test/java/org/apache/fluss/lake/lance/tiering/LanceTieringITCase.java index 2180554aa3..98dc30dd00 100644 --- a/fluss-lake/fluss-lake-lance/src/test/java/org/apache/fluss/lake/lance/tiering/LanceTieringITCase.java +++ b/fluss-lake/fluss-lake-lance/src/test/java/org/apache/fluss/lake/lance/tiering/LanceTieringITCase.java @@ -115,7 +115,7 @@ private void checkSnapshotPropertyInLance( new LakeTable.LakeSnapshotMetadata( // don't care about snapshot id -1, new FsPath(offsetFile), null)) - .getLatestTableSnapshot() + .getOrReadLatestTableSnapshot() .getBucketLogEndOffset(); assertThat(recordedOffsets).isEqualTo(expectedOffsets); } diff --git a/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/testutils/FlinkPaimonTieringTestBase.java b/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/testutils/FlinkPaimonTieringTestBase.java index 76fc332bed..e98596e9ec 100644 --- a/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/testutils/FlinkPaimonTieringTestBase.java +++ b/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/testutils/FlinkPaimonTieringTestBase.java @@ -491,7 +491,7 @@ protected void checkFlussOffsetsInSnapshot( new LakeTable.LakeSnapshotMetadata( // don't care about snapshot id -1, new FsPath(offsetFile), null)) - .getLatestTableSnapshot() + .getOrReadLatestTableSnapshot() .getBucketLogEndOffset(); assertThat(recordedOffsets).isEqualTo(expectedOffsets); } diff --git a/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/CoordinatorGateway.java b/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/CoordinatorGateway.java index a6b0f99b38..bfe05cde4c 100644 --- a/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/CoordinatorGateway.java +++ b/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/CoordinatorGateway.java @@ -30,8 +30,8 @@ import org.apache.fluss.rpc.messages.ControlledShutdownResponse; import org.apache.fluss.rpc.messages.LakeTieringHeartbeatRequest; import org.apache.fluss.rpc.messages.LakeTieringHeartbeatResponse; -import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotRequest; -import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotResponse; +import org.apache.fluss.rpc.messages.PrepareLakeTableSnapshotRequest; +import org.apache.fluss.rpc.messages.PrepareLakeTableSnapshotResponse; import org.apache.fluss.rpc.protocol.ApiKeys; import org.apache.fluss.rpc.protocol.RPC; @@ -68,6 +68,30 @@ public interface CoordinatorGateway extends RpcGateway, AdminGateway { CompletableFuture commitRemoteLogManifest( CommitRemoteLogManifestRequest request); + /** + * Prepares lake table snapshots by merging them with existing snapshots and storing them to the + * file system. + * + *

      This method is called during the two-phase commit process for lake table snapshots. It + * performs the following operations for each table in the request: + * + *

        + *
      • Merges the new snapshot with the previous latest snapshot (if exists) to ensure + * completeness + *
      • Stores the merged snapshot to the remote file system. The stored file contains the log + * end offset information for each bucket in the table + *
      • Returns the file path where the snapshot is stored + *
      + * + * @param request the request containing lake table snapshot information for one or more tables + * @return a future that completes with a response containing the file paths where snapshots + * (containing bucket log end offset information) are stored, or error information for + * tables that failed to process + */ + @RPC(api = ApiKeys.PRE_LAKE_TABLE_SNAPSHOT) + CompletableFuture prepareLakeTableSnapshot( + PrepareLakeTableSnapshotRequest request); + /** * Commit lakehouse table snapshot to Fluss. * @@ -87,28 +111,4 @@ CompletableFuture lakeTieringHeartbeat( @RPC(api = ApiKeys.CONTROLLED_SHUTDOWN) CompletableFuture controlledShutdown( ControlledShutdownRequest request); - - /** - * Prepares to commit lake table snapshots by merging them with existing snapshots and storing - * them to the file system. - * - *

      This method is called during the two-phase commit process for lake table snapshots. It - * performs the following operations for each table in the request: - * - *

        - *
      • Merges the new snapshot with the previous latest snapshot (if exists) to ensure - * completeness - *
      • Stores the merged snapshot to the remote file system. The stored file contains the log - * end offset information for each bucket in the table - *
      • Returns the file path where the snapshot is stored - *
      - * - * @param request the request containing lake table snapshot information for one or more tables - * @return a future that completes with a response containing the file paths where snapshots - * (containing bucket log end offset information) are stored, or error information for - * tables that failed to process - */ - @RPC(api = ApiKeys.PRECOMMIT_LAKE_TABLE_SNAPSHOT) - CompletableFuture prepareCommitLakeTableSnapshot( - PrepareCommitLakeTableSnapshotRequest request); } 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 10305da283..07a9b44d6f 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 @@ -80,7 +80,7 @@ public enum ApiKeys { REBALANCE(1049, 0, 0, PUBLIC), LIST_REBALANCE_PROGRESS(1050, 0, 0, PUBLIC), CANCEL_REBALANCE(1051, 0, 0, PUBLIC), - PRECOMMIT_LAKE_TABLE_SNAPSHOT(1052, 0, 0, PRIVATE); + PRE_LAKE_TABLE_SNAPSHOT(1052, 0, 0, PRIVATE); private static final Map ID_TO_TYPE = Arrays.stream(ApiKeys.values()) diff --git a/fluss-rpc/src/main/proto/FlussApi.proto b/fluss-rpc/src/main/proto/FlussApi.proto index ff0bd8b82f..07a569c9ca 100644 --- a/fluss-rpc/src/main/proto/FlussApi.proto +++ b/fluss-rpc/src/main/proto/FlussApi.proto @@ -449,12 +449,12 @@ message NotifyRemoteLogOffsetsRequest { message NotifyRemoteLogOffsetsResponse { } -message PrepareCommitLakeTableSnapshotRequest { - repeated PbTableBucketOffsets bucket_offsets = 1; +message PrepareLakeTableSnapshotRequest { + repeated PbTableOffsets bucket_offsets = 1; } -message PrepareCommitLakeTableSnapshotResponse { - repeated PbPrepareCommitLakeTableRespForTable prepare_commit_lake_table_resp = 1; +message PrepareLakeTableSnapshotResponse { + repeated PbPrepareLakeTableRespForTable prepare_lake_table_resp = 1; } message CommitLakeTableSnapshotRequest { @@ -477,9 +477,9 @@ message CommitLakeTableSnapshotResponse { } message PbCommitLakeTableSnapshotRespForTable { - required int64 table_id = 1; - optional int32 error_code = 2; - optional string error_message = 3; + optional int32 error_code = 1; + optional string error_message = 2; + required int64 table_id = 3; } message NotifyLakeTableOffsetRequest { @@ -1042,13 +1042,14 @@ message PbLakeTableOffsetForBucket { optional int64 max_timestamp = 6; } -message PbPrepareCommitLakeTableRespForTable { - optional string lake_table_bucket_offsets_path = 1; - optional int32 error_code = 2; - optional string error_message = 3; +message PbPrepareLakeTableRespForTable { + optional int32 error_code = 1; + optional string error_message = 2; + optional int64 table_id = 3; + optional string lake_table_bucket_offsets_path = 4; } -message PbTableBucketOffsets { +message PbTableOffsets { required int64 table_id = 1; required PbTablePath table_path = 2; repeated PbBucketOffset bucket_offsets = 3; 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 1a51c0aece..d9ed2db983 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 @@ -1230,37 +1230,7 @@ private void tryProcessCommitLakeTableSnapshot( if (commitLakeTableSnapshotData.getLakeTableSnapshotMetadatas().isEmpty()) { handleCommitLakeTableSnapshotV1(commitLakeTableSnapshotEvent, callback); } else { - Map lakeSnapshotMetadatas = - commitLakeTableSnapshotData.getLakeTableSnapshotMetadatas(); - ioExecutor.execute( - () -> { - try { - CommitLakeTableSnapshotResponse response = - new CommitLakeTableSnapshotResponse(); - for (Map.Entry - lakeSnapshotMetadataEntry : lakeSnapshotMetadatas.entrySet()) { - PbCommitLakeTableSnapshotRespForTable tableResp = - response.addTableResp(); - long tableId = lakeSnapshotMetadataEntry.getKey(); - tableResp.setTableId(tableId); - try { - lakeTableHelper.addLakeTableSnapshotMetadata( - tableId, lakeSnapshotMetadataEntry.getValue()); - } catch (Exception e) { - ApiError error = ApiError.fromThrowable(e); - tableResp.setError(error.error().code(), error.message()); - } - } - coordinatorEventManager.put( - new NotifyLakeTableOffsetEvent( - commitLakeTableSnapshotData.getLakeTableSnapshot(), - commitLakeTableSnapshotData - .getTableBucketsMaxTieredTimestamp())); - callback.complete(response); - } catch (Exception e) { - callback.completeExceptionally(e); - } - }); + handleCommitLakeTableSnapshotV2(commitLakeTableSnapshotEvent, callback); } } @@ -1305,20 +1275,58 @@ private void handleCommitLakeTableSnapshotV1( } // this involves IO operation (ZK), so we do it in ioExecutor - lakeTableHelper.upsertLakeTable( + lakeTableHelper.registerLakeTableSnapshotV1( tableId, lakeTableSnapshotEntry.getValue()); + // send notify lakehouse data request to all replicas via + // coordinator event + coordinatorEventManager.put( + new NotifyLakeTableOffsetEvent( + lakeTableSnapshots, + commitLakeTableSnapshotData + .getTableBucketsMaxTieredTimestamp())); } catch (Exception e) { ApiError error = ApiError.fromThrowable(e); tableResp.setError(error.error().code(), error.message()); } } + callback.complete(response); + } catch (Exception e) { + callback.completeExceptionally(e); + } + }); + } - // send notify lakehouse data request to all replicas via coordinator event - coordinatorEventManager.put( - new NotifyLakeTableOffsetEvent( - lakeTableSnapshots, - commitLakeTableSnapshotData - .getTableBucketsMaxTieredTimestamp())); + private void handleCommitLakeTableSnapshotV2( + CommitLakeTableSnapshotEvent commitLakeTableSnapshotEvent, + CompletableFuture callback) { + CommitLakeTableSnapshotData commitLakeTableSnapshotData = + commitLakeTableSnapshotEvent.getCommitLakeTableSnapshotData(); + Map lakeSnapshotMetadatas = + commitLakeTableSnapshotData.getLakeTableSnapshotMetadatas(); + ioExecutor.execute( + () -> { + try { + CommitLakeTableSnapshotResponse response = + new CommitLakeTableSnapshotResponse(); + for (Map.Entry + lakeSnapshotMetadataEntry : lakeSnapshotMetadatas.entrySet()) { + PbCommitLakeTableSnapshotRespForTable tableResp = + response.addTableResp(); + long tableId = lakeSnapshotMetadataEntry.getKey(); + tableResp.setTableId(tableId); + try { + lakeTableHelper.registerLakeTableSnapshotV2( + tableId, lakeSnapshotMetadataEntry.getValue()); + coordinatorEventManager.put( + new NotifyLakeTableOffsetEvent( + commitLakeTableSnapshotData.getLakeTableSnapshot(), + commitLakeTableSnapshotData + .getTableBucketsMaxTieredTimestamp())); + } catch (Exception e) { + ApiError error = ApiError.fromThrowable(e); + tableResp.setError(error.error().code(), error.message()); + } + } 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 65676821bc..1ebd80070e 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 @@ -88,10 +88,10 @@ import org.apache.fluss.rpc.messages.PbAlterConfig; import org.apache.fluss.rpc.messages.PbHeartbeatReqForTable; import org.apache.fluss.rpc.messages.PbHeartbeatRespForTable; -import org.apache.fluss.rpc.messages.PbPrepareCommitLakeTableRespForTable; -import org.apache.fluss.rpc.messages.PbTableBucketOffsets; -import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotRequest; -import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotResponse; +import org.apache.fluss.rpc.messages.PbPrepareLakeTableRespForTable; +import org.apache.fluss.rpc.messages.PbTableOffsets; +import org.apache.fluss.rpc.messages.PrepareLakeTableSnapshotRequest; +import org.apache.fluss.rpc.messages.PrepareLakeTableSnapshotResponse; import org.apache.fluss.rpc.messages.RebalanceRequest; import org.apache.fluss.rpc.messages.RebalanceResponse; import org.apache.fluss.rpc.messages.RemoveServerTagRequest; @@ -128,6 +128,7 @@ import org.apache.fluss.server.zk.data.PartitionAssignment; import org.apache.fluss.server.zk.data.TableAssignment; import org.apache.fluss.server.zk.data.TableRegistration; +import org.apache.fluss.server.zk.data.lake.LakeTable; import org.apache.fluss.server.zk.data.lake.LakeTableHelper; import org.apache.fluss.utils.IOUtils; import org.apache.fluss.utils.concurrent.FutureUtils; @@ -139,6 +140,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.function.Supplier; @@ -640,6 +642,52 @@ public CompletableFuture dropAcls(DropAclsRequest request) { return CompletableFuture.completedFuture(makeDropAclsResponse(aclDeleteResults)); } + @Override + public CompletableFuture prepareLakeTableSnapshot( + PrepareLakeTableSnapshotRequest request) { + CompletableFuture future = new CompletableFuture<>(); + ioExecutor.submit( + () -> { + PrepareLakeTableSnapshotResponse response = + new PrepareLakeTableSnapshotResponse(); + try { + for (PbTableOffsets bucketOffsets : request.getBucketOffsetsList()) { + PbPrepareLakeTableRespForTable pbPrepareLakeTableRespForTable = + response.addPrepareLakeTableResp(); + try { + 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); + } + TablePath tablePath = toTablePath(bucketOffsets.getTablePath()); + FsPath fsPath = + lakeTableHelper.storeLakeTableOffsetsFile( + tablePath, tableBucketOffsets); + pbPrepareLakeTableRespForTable.setTableId(tableId); + pbPrepareLakeTableRespForTable.setLakeTableBucketOffsetsPath( + fsPath.toString()); + } catch (Exception e) { + Errors error = ApiError.fromThrowable(e).error(); + pbPrepareLakeTableRespForTable.setError( + error.code(), error.message()); + } + } + future.complete(response); + } catch (Exception e) { + future.completeExceptionally(e); + } + }); + return future; + } + @Override public CompletableFuture commitLakeTableSnapshot( CommitLakeTableSnapshotRequest request) { @@ -725,47 +773,6 @@ public CompletableFuture controlledShutdown( return response; } - @Override - public CompletableFuture prepareCommitLakeTableSnapshot( - PrepareCommitLakeTableSnapshotRequest request) { - CompletableFuture future = - new CompletableFuture<>(); - ioExecutor.submit( - () -> { - PrepareCommitLakeTableSnapshotResponse response = - new PrepareCommitLakeTableSnapshotResponse(); - try { - for (PbTableBucketOffsets bucketOffsets : request.getBucketOffsetsList()) { - PbPrepareCommitLakeTableRespForTable - pbPrepareCommitLakeTableRespForTable = - response.addPrepareCommitLakeTableResp(); - try { - // upsert lake table snapshot, need to merge the snapshot with - // previous latest snapshot - TableBucketOffsets tableBucketOffsets = - lakeTableHelper.upsertTableBucketOffsets( - bucketOffsets.getTableId(), - toTableBucketOffsets(bucketOffsets)); - TablePath tablePath = toTablePath(bucketOffsets.getTablePath()); - FsPath fsPath = - lakeTableHelper.storeLakeTableBucketOffsets( - tablePath, tableBucketOffsets); - pbPrepareCommitLakeTableRespForTable.setLakeTableBucketOffsetsPath( - fsPath.toString()); - } catch (Exception e) { - Errors error = ApiError.fromThrowable(e).error(); - pbPrepareCommitLakeTableRespForTable.setError( - error.code(), error.message()); - } - } - future.complete(response); - } catch (Exception e) { - future.completeExceptionally(e); - } - }); - return future; - } - @Override public CompletableFuture alterClusterConfigs( AlterClusterConfigsRequest request) { 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 05caa40daf..7601b249ea 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 @@ -127,8 +127,8 @@ import org.apache.fluss.rpc.messages.PbStopReplicaReqForBucket; import org.apache.fluss.rpc.messages.PbStopReplicaRespForBucket; import org.apache.fluss.rpc.messages.PbTableBucket; -import org.apache.fluss.rpc.messages.PbTableBucketOffsets; import org.apache.fluss.rpc.messages.PbTableMetadata; +import org.apache.fluss.rpc.messages.PbTableOffsets; import org.apache.fluss.rpc.messages.PbTablePath; import org.apache.fluss.rpc.messages.PbValue; import org.apache.fluss.rpc.messages.PbValueList; @@ -1599,11 +1599,10 @@ public static CommitLakeTableSnapshotData getCommitLakeTableSnapshotData( lakeTableInfoByTableId, tableBucketsMaxTimestamp, lakeSnapshotMetadatas); } - public static TableBucketOffsets toTableBucketOffsets( - PbTableBucketOffsets pbTableBucketOffsets) { + public static TableBucketOffsets toTableBucketOffsets(PbTableOffsets pbTableOffsets) { Map bucketOffsets = new HashMap<>(); - long tableId = pbTableBucketOffsets.getTableId(); - for (PbBucketOffset pbBucketOffset : pbTableBucketOffsets.getBucketOffsetsList()) { + long tableId = pbTableOffsets.getTableId(); + for (PbBucketOffset pbBucketOffset : pbTableOffsets.getBucketOffsetsList()) { TableBucket tableBucket = new TableBucket( tableId, 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 a9e0af4d95..0af2124674 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 @@ -1065,7 +1065,7 @@ public Optional getLakeTableSnapshot(long tableId) throws Exc Optional optLakeTable = getLakeTable(tableId); if (optLakeTable.isPresent()) { // always get the latest snapshot - return Optional.of(optLakeTable.get().getLatestTableSnapshot()); + return Optional.of(optLakeTable.get().getOrReadLatestTableSnapshot()); } 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 f4fd85a138..c5ae253ecc 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 @@ -113,14 +113,14 @@ public LakeTableSnapshot getLakeTableSnapshot() { } /** - * Get the latest table snapshot for the lake table. + * Get or read the latest table snapshot for the lake table. * *

      If this LakeTable was created from a LakeTableSnapshot (version 1), returns it directly. * Otherwise, reads the snapshot data from the lake snapshot file. * * @return the LakeTableSnapshot */ - public LakeTableSnapshot getLatestTableSnapshot() throws IOException { + public LakeTableSnapshot getOrReadLatestTableSnapshot() throws IOException { if (lakeTableSnapshot != null) { return lakeTableSnapshot; } 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 88f4e3f98a..e3b46d04e3 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 @@ -53,7 +53,7 @@ public LakeTableHelper(ZooKeeperClient zkClient, String remoteDataDir) { * @param lakeTableSnapshot the new snapshot to upsert * @throws Exception if the operation fails */ - public void upsertLakeTable(long tableId, LakeTableSnapshot lakeTableSnapshot) + public void registerLakeTableSnapshotV1(long tableId, LakeTableSnapshot lakeTableSnapshot) throws Exception { Optional optPreviousLakeTable = zkClient.getLakeTable(tableId); // Merge with previous snapshot if exists @@ -69,7 +69,7 @@ public void upsertLakeTable(long tableId, LakeTableSnapshot lakeTableSnapshot) tableId, new LakeTable(lakeTableSnapshot), optPreviousLakeTable.isPresent()); } - public void addLakeTableSnapshotMetadata( + public void registerLakeTableSnapshotV2( long tableId, LakeTable.LakeSnapshotMetadata lakeSnapshotMetadata) throws Exception { Optional optPreviousLakeTable = zkClient.getLakeTable(tableId); List previousLakeSnapshotMetadatas = null; @@ -93,17 +93,7 @@ public void addLakeTableSnapshotMetadata( } } - public TableBucketOffsets upsertTableBucketOffsets( - long tableId, TableBucketOffsets newTableBucketOffsets) throws Exception { - Optional optPreviousLakeTable = zkClient.getLakeTable(tableId); - // Merge with previous snapshot if exists - if (optPreviousLakeTable.isPresent()) { - return mergeTableBucketOffsets(optPreviousLakeTable.get(), newTableBucketOffsets); - } - return newTableBucketOffsets; - } - - private TableBucketOffsets mergeTableBucketOffsets( + public TableBucketOffsets mergeTableBucketOffsets( LakeTable previousLakeTable, TableBucketOffsets newTableBucketOffsets) throws Exception { // Merge current with previous one since the current request @@ -112,12 +102,13 @@ private TableBucketOffsets mergeTableBucketOffsets( // merge log end offsets, current will override the previous Map bucketLogEndOffset = - new HashMap<>(previousLakeTable.getLatestTableSnapshot().getBucketLogEndOffset()); + new HashMap<>( + previousLakeTable.getOrReadLatestTableSnapshot().getBucketLogEndOffset()); bucketLogEndOffset.putAll(newTableBucketOffsets.getOffsets()); return new TableBucketOffsets(newTableBucketOffsets.getTableId(), bucketLogEndOffset); } - public FsPath storeLakeTableBucketOffsets( + public FsPath storeLakeTableOffsetsFile( TablePath tablePath, TableBucketOffsets tableBucketOffsets) throws Exception { // get the remote file path to store the lake table snapshot offset information long tableId = tableBucketOffsets.getTableId(); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerde.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerde.java index db7838c834..9e72239395 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerde.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerde.java @@ -38,8 +38,8 @@ * *

        *
      • Version 1 (legacy): ZK node contains full {@link LakeTableSnapshot} data. During - * deserialization, it uses {@link LakeTableSnapshotJsonSerde} to deserialize and wraps the - * result in a {@link LakeTable}. + * deserialization, it uses {@link LakeTableSnapshotLegacyJsonSerde} to deserialize and wraps + * the result in a {@link LakeTable}. *
      • Version 2 (current): ZK node contains only the lake table snapshot file paths. The actual * snapshot data is stored in a remote file pointed by the lake table snapshot file path. *
      @@ -63,70 +63,82 @@ public class LakeTableJsonSerde implements JsonSerializer, JsonDeseri public void serialize(LakeTable lakeTable, JsonGenerator generator) throws IOException { // if lake table snapshot metadata is null, it must be version 1 if (lakeTable.getLakeSnapshotMetadatas() == null) { - // Version 1: ZK node contains full snapshot data, use LakeTableSnapshotJsonSerde - LakeTableSnapshotJsonSerde.INSTANCE.serialize( - lakeTable.getLatestTableSnapshot(), generator); + serializeV1(lakeTable, generator); } else { - generator.writeStartObject(); - generator.writeNumberField(VERSION_KEY, CURRENT_VERSION); + serializeV2(lakeTable, generator); + } + } + + private void serializeV1(LakeTable lakeTable, JsonGenerator generator) throws IOException { + // Version 1: ZK node contains full snapshot data, use LakeTableSnapshotJsonSerde + LakeTableSnapshotLegacyJsonSerde.INSTANCE.serialize( + lakeTable.getOrReadLatestTableSnapshot(), generator); + } - generator.writeArrayFieldStart(LAKE_SNAPSHOTS); - for (LakeTable.LakeSnapshotMetadata lakeSnapshotMetadata : - checkNotNull(lakeTable.getLakeSnapshotMetadatas())) { - generator.writeStartObject(); + private void serializeV2(LakeTable lakeTable, JsonGenerator generator) throws IOException { + generator.writeStartObject(); + generator.writeNumberField(VERSION_KEY, CURRENT_VERSION); - generator.writeNumberField(SNAPSHOT_ID_KEY, lakeSnapshotMetadata.getSnapshotId()); + generator.writeArrayFieldStart(LAKE_SNAPSHOTS); + for (LakeTable.LakeSnapshotMetadata lakeSnapshotMetadata : + checkNotNull(lakeTable.getLakeSnapshotMetadatas())) { + generator.writeStartObject(); + + generator.writeNumberField(SNAPSHOT_ID_KEY, lakeSnapshotMetadata.getSnapshotId()); + generator.writeStringField( + TIERED_OFFSETS_KEY, lakeSnapshotMetadata.getTieredOffsetsFilePath().toString()); + if (lakeSnapshotMetadata.getReadableOffsetsFilePath() != null) { generator.writeStringField( - TIERED_OFFSETS_KEY, - lakeSnapshotMetadata.getTieredOffsetsFilePath().toString()); - if (lakeSnapshotMetadata.getReadableOffsetsFilePath() != null) { - generator.writeStringField( - READABLE_OFFSETS_KEY, - lakeSnapshotMetadata.getReadableOffsetsFilePath().toString()); - } - generator.writeEndObject(); + READABLE_OFFSETS_KEY, + lakeSnapshotMetadata.getReadableOffsetsFilePath().toString()); } - generator.writeEndArray(); - generator.writeEndObject(); } + generator.writeEndArray(); + + generator.writeEndObject(); } @Override public LakeTable deserialize(JsonNode node) { int version = node.get(VERSION_KEY).asInt(); if (version == VERSION_1) { - // Version 1: ZK node contains full snapshot data, use LakeTableSnapshotJsonSerde - LakeTableSnapshot snapshot = LakeTableSnapshotJsonSerde.INSTANCE.deserialize(node); - return new LakeTable(snapshot); + return deserializeV1(node); } else if (version == VERSION_2) { - // Version 2: ZK node contains lake snapshot file paths - JsonNode lakeSnapshotsNode = node.get(LAKE_SNAPSHOTS); - if (lakeSnapshotsNode == null || !lakeSnapshotsNode.isArray()) { - throw new IllegalArgumentException( - "Invalid lake_snapshots field in version 2 format"); - } - - List lakeSnapshotMetadatas = new ArrayList<>(); - Iterator elements = lakeSnapshotsNode.elements(); - while (elements.hasNext()) { - JsonNode snapshotNode = elements.next(); - long snapshotId = snapshotNode.get(SNAPSHOT_ID_KEY).asLong(); - String tieredOffsetsPath = snapshotNode.get(TIERED_OFFSETS_KEY).asText(); - JsonNode readableOffsetsNode = snapshotNode.get(READABLE_OFFSETS_KEY); - FsPath readableOffsetsPath = - readableOffsetsNode != null - ? new FsPath(readableOffsetsNode.asText()) - : null; - - LakeTable.LakeSnapshotMetadata metadata = - new LakeTable.LakeSnapshotMetadata( - snapshotId, new FsPath(tieredOffsetsPath), readableOffsetsPath); - lakeSnapshotMetadatas.add(metadata); - } - return new LakeTable(lakeSnapshotMetadatas); + return deserializeV2(node); } else { throw new IllegalArgumentException("Unsupported version: " + version); } } + + private LakeTable deserializeV1(JsonNode node) { + // Version 1: ZK node contains full snapshot data, use LakeTableSnapshotJsonSerde + LakeTableSnapshot snapshot = LakeTableSnapshotLegacyJsonSerde.INSTANCE.deserialize(node); + return new LakeTable(snapshot); + } + + private LakeTable deserializeV2(JsonNode node) { + // Version 2: ZK node contains lake snapshot file paths + JsonNode lakeSnapshotsNode = node.get(LAKE_SNAPSHOTS); + if (lakeSnapshotsNode == null || !lakeSnapshotsNode.isArray()) { + throw new IllegalArgumentException("Invalid lake_snapshots field in version 2 format"); + } + + List lakeSnapshotMetadatas = new ArrayList<>(); + Iterator elements = lakeSnapshotsNode.elements(); + while (elements.hasNext()) { + JsonNode snapshotNode = elements.next(); + long snapshotId = snapshotNode.get(SNAPSHOT_ID_KEY).asLong(); + String tieredOffsetsPath = snapshotNode.get(TIERED_OFFSETS_KEY).asText(); + JsonNode readableOffsetsNode = snapshotNode.get(READABLE_OFFSETS_KEY); + FsPath readableOffsetsPath = + readableOffsetsNode != null ? new FsPath(readableOffsetsNode.asText()) : null; + + LakeTable.LakeSnapshotMetadata metadata = + new LakeTable.LakeSnapshotMetadata( + snapshotId, new FsPath(tieredOffsetsPath), readableOffsetsPath); + lakeSnapshotMetadatas.add(metadata); + } + return new LakeTable(lakeSnapshotMetadatas); + } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableSnapshotJsonSerde.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableSnapshotLegacyJsonSerde.java similarity index 97% rename from fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableSnapshotJsonSerde.java rename to fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableSnapshotLegacyJsonSerde.java index 7c96c4d45c..d1e5a52589 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableSnapshotJsonSerde.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableSnapshotLegacyJsonSerde.java @@ -50,10 +50,11 @@ * @see LakeTableJsonSerde for the current format (version 2) that uses this serde for legacy * compatibility */ -public class LakeTableSnapshotJsonSerde +public class LakeTableSnapshotLegacyJsonSerde implements JsonSerializer, JsonDeserializer { - public static final LakeTableSnapshotJsonSerde INSTANCE = new LakeTableSnapshotJsonSerde(); + public static final LakeTableSnapshotLegacyJsonSerde INSTANCE = + new LakeTableSnapshotLegacyJsonSerde(); private static final String VERSION_KEY = "version"; 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 41b0c1d125..06d15d95ed 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 @@ -90,12 +90,12 @@ import org.apache.fluss.rpc.messages.ListTablesResponse; import org.apache.fluss.rpc.messages.MetadataRequest; import org.apache.fluss.rpc.messages.MetadataResponse; +import org.apache.fluss.rpc.messages.PrepareLakeTableSnapshotRequest; +import org.apache.fluss.rpc.messages.PrepareLakeTableSnapshotResponse; import org.apache.fluss.rpc.messages.RebalanceRequest; import org.apache.fluss.rpc.messages.RebalanceResponse; import org.apache.fluss.rpc.messages.RemoveServerTagRequest; import org.apache.fluss.rpc.messages.RemoveServerTagResponse; -import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotRequest; -import org.apache.fluss.rpc.messages.PrepareCommitLakeTableSnapshotResponse; import org.apache.fluss.rpc.messages.TableExistsRequest; import org.apache.fluss.rpc.messages.TableExistsResponse; import org.apache.fluss.rpc.protocol.ApiError; @@ -340,8 +340,8 @@ public CompletableFuture commitRemoteLogManifes } @Override - public CompletableFuture prepareCommitLakeTableSnapshot( - PrepareCommitLakeTableSnapshotRequest request) { + public CompletableFuture prepareLakeTableSnapshot( + PrepareLakeTableSnapshotRequest request) { throw new UnsupportedOperationException(); } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/LakeTableSnapshotJsonSerdeTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/LakeTableSnapshotLegacyJsonSerdeTest.java similarity index 52% rename from fluss-server/src/test/java/org/apache/fluss/server/zk/data/LakeTableSnapshotJsonSerdeTest.java rename to fluss-server/src/test/java/org/apache/fluss/server/zk/data/LakeTableSnapshotLegacyJsonSerdeTest.java index 119558d882..369a53062d 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/LakeTableSnapshotJsonSerdeTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/LakeTableSnapshotLegacyJsonSerdeTest.java @@ -19,17 +19,23 @@ import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.server.zk.data.lake.LakeTableSnapshot; -import org.apache.fluss.server.zk.data.lake.LakeTableSnapshotJsonSerde; +import org.apache.fluss.server.zk.data.lake.LakeTableSnapshotLegacyJsonSerde; import org.apache.fluss.utils.json.JsonSerdeTestBase; +import org.apache.fluss.utils.json.JsonSerdeUtils; +import org.junit.jupiter.api.Test; + +import java.nio.charset.StandardCharsets; import java.util.HashMap; import java.util.Map; -/** Test for {@link LakeTableSnapshotJsonSerde}. */ -class LakeTableSnapshotJsonSerdeTest extends JsonSerdeTestBase { +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link LakeTableSnapshotLegacyJsonSerde}. */ +class LakeTableSnapshotLegacyJsonSerdeTest extends JsonSerdeTestBase { - LakeTableSnapshotJsonSerdeTest() { - super(LakeTableSnapshotJsonSerde.INSTANCE); + LakeTableSnapshotLegacyJsonSerdeTest() { + super(LakeTableSnapshotLegacyJsonSerde.INSTANCE); } @Override @@ -67,4 +73,35 @@ protected String[] expectedJsons() { + "{\"partition_id\":2,\"bucket_id\":1,\"log_end_offset\":4}]}" }; } + + @Test + void testBackwardCompatibility() { + // Test that Version 1 format can still be deserialized + String version1Json1 = "{\"version\":1,\"snapshot_id\":1,\"table_id\":1,\"buckets\":[]}"; + LakeTableSnapshot snapshot1 = + JsonSerdeUtils.readValue( + version1Json1.getBytes(StandardCharsets.UTF_8), + LakeTableSnapshotLegacyJsonSerde.INSTANCE); + assertThat(snapshot1.getSnapshotId()).isEqualTo(1); + assertThat(snapshot1.getBucketLogEndOffset()).isEmpty(); + + String version1Json2 = + "{\"version\":1,\"snapshot_id\":2,\"table_id\":4," + + "\"buckets\":[{\"bucket_id\":1,\"log_start_offset\":1,\"log_end_offset\":3,\"max_timestamp\":5}]}"; + LakeTableSnapshot snapshot2 = + JsonSerdeUtils.readValue( + version1Json2.getBytes(StandardCharsets.UTF_8), + LakeTableSnapshotLegacyJsonSerde.INSTANCE); + assertThat(snapshot2.getSnapshotId()).isEqualTo(2); + assertThat(snapshot2.getBucketLogEndOffset()).hasSize(1); + + String version1Json3 = + "{\"version\":1,\"snapshot_id\":3,\"table_id\":5," + + "\"buckets\":[{\"partition_id\":1,\"partition_name\":\"partition1\",\"bucket_id\":1,\"log_start_offset\":1,\"log_end_offset\":3,\"max_timestamp\":5}]}"; + LakeTableSnapshot snapshot3 = + JsonSerdeUtils.readValue( + version1Json3.getBytes(StandardCharsets.UTF_8), + LakeTableSnapshotLegacyJsonSerde.INSTANCE); + assertThat(snapshot3.getSnapshotId()).isEqualTo(3); + } } 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 855ad7880d..f55c1b94ad 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 @@ -76,7 +76,7 @@ static void afterAll() { } @Test - void testUpsertLakeTableCompatible(@TempDir Path tempDir) throws Exception { + void testRegisterLakeTableSnapshotCompatibility(@TempDir Path tempDir) throws Exception { // Create a ZooKeeperClient with REMOTE_DATA_DIR configuration Configuration conf = new Configuration(); conf.setString( @@ -111,13 +111,13 @@ void testUpsertLakeTableCompatible(@TempDir Path tempDir) throws Exception { LakeTableSnapshot lakeTableSnapshot = new LakeTableSnapshot(snapshotId, bucketLogEndOffset); // Write version 1 format data(simulating old system behavior) - lakeTableHelper.upsertLakeTable(tableId, lakeTableSnapshot); + lakeTableHelper.registerLakeTableSnapshotV1(tableId, lakeTableSnapshot); // Verify version 1 data can be read Optional optionalLakeTable = zooKeeperClient.getLakeTable(tableId); assertThat(optionalLakeTable).isPresent(); LakeTable lakeTable = optionalLakeTable.get(); - assertThat(lakeTable.getLatestTableSnapshot()).isEqualTo(lakeTableSnapshot); + assertThat(lakeTable.getOrReadLatestTableSnapshot()).isEqualTo(lakeTableSnapshot); // Test: Call upsertLakeTableSnapshot with new snapshot data // This should read the old version 1 data, merge it, and write as version 2 @@ -127,9 +127,9 @@ void testUpsertLakeTableCompatible(@TempDir Path tempDir) throws Exception { long snapshot2Id = 2L; FsPath tieredOffsetsPath = - lakeTableHelper.storeLakeTableBucketOffsets( + lakeTableHelper.storeLakeTableOffsetsFile( tablePath, new TableBucketOffsets(tableId, newBucketLogEndOffset)); - lakeTableHelper.addLakeTableSnapshotMetadata( + lakeTableHelper.registerLakeTableSnapshotV2( tableId, new LakeTable.LakeSnapshotMetadata( snapshot2Id, tieredOffsetsPath, tieredOffsetsPath)); @@ -162,9 +162,9 @@ void testUpsertLakeTableCompatible(@TempDir Path tempDir) throws Exception { // add a new snapshot 3 again, verify snapshot long snapshot3Id = 3L; tieredOffsetsPath = - lakeTableHelper.storeLakeTableBucketOffsets( + lakeTableHelper.storeLakeTableOffsetsFile( tablePath, new TableBucketOffsets(tableId, newBucketLogEndOffset)); - lakeTableHelper.addLakeTableSnapshotMetadata( + lakeTableHelper.registerLakeTableSnapshotV2( tableId, new LakeTable.LakeSnapshotMetadata( snapshot3Id, tieredOffsetsPath, tieredOffsetsPath)); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerdeTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerdeTest.java index 9bf486ab02..2dbc2280cf 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerdeTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerdeTest.java @@ -125,7 +125,7 @@ void testVersion1Compatibility() throws IOException { expectedBuckets1.put(new TableBucket(4L, 0), 100L); expectedBuckets1.put(new TableBucket(4L, 1), 200L); LakeTableSnapshot expectedSnapshot1 = new LakeTableSnapshot(10L, expectedBuckets1); - assertThat(actual1.getLatestTableSnapshot()).isEqualTo(expectedSnapshot1); + assertThat(actual1.getOrReadLatestTableSnapshot()).isEqualTo(expectedSnapshot1); // Test case 2: Partition table String version1Json2 = @@ -144,6 +144,6 @@ void testVersion1Compatibility() throws IOException { expectedBuckets2.put(new TableBucket(5L, 1L, 1), 200L); expectedBuckets2.put(new TableBucket(5L, 2L, 0), 300L); LakeTableSnapshot expectedSnapshot2 = new LakeTableSnapshot(11L, expectedBuckets2); - assertThat(actual2.getLatestTableSnapshot()).isEqualTo(expectedSnapshot2); + assertThat(actual2.getOrReadLatestTableSnapshot()).isEqualTo(expectedSnapshot2); } } From ef1f4b99c60cc8c78743029a33533916c98ea7dd Mon Sep 17 00:00:00 2001 From: luoyuxia Date: Mon, 29 Dec 2025 20:51:37 +0800 Subject: [PATCH 7/7] address comments again --- .../FlussTableLakeSnapshotCommitter.java | 2 +- .../fluss/rpc/gateway/CoordinatorGateway.java | 6 ++- .../apache/fluss/rpc/protocol/ApiKeys.java | 2 +- fluss-rpc/src/main/proto/FlussApi.proto | 2 +- .../CoordinatorEventProcessor.java | 46 +++++++++++++------ .../coordinator/CoordinatorService.java | 2 +- .../event/NotifyLakeTableOffsetEvent.java | 10 ++-- .../entity/CommitLakeTableSnapshotData.java | 19 ++++---- .../server/utils/ServerRpcMessageUtils.java | 6 ++- 9 files changed, 58 insertions(+), 37 deletions(-) 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 5ba5b7fc2a..fe37c91e5b 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 @@ -110,7 +110,7 @@ String prepareLakeSnapshot( if (prepareResp.hasErrorCode()) { throw ApiError.fromErrorMessage(prepareResp).exception(); } else { - return checkNotNull(prepareResp).getLakeTableBucketOffsetsPath(); + return checkNotNull(prepareResp).getLakeTableOffsetsPath(); } } catch (Exception e) { throw new IOException( diff --git a/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/CoordinatorGateway.java b/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/CoordinatorGateway.java index bfe05cde4c..92508f8565 100644 --- a/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/CoordinatorGateway.java +++ b/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/CoordinatorGateway.java @@ -70,7 +70,7 @@ CompletableFuture commitRemoteLogManifest( /** * Prepares lake table snapshots by merging them with existing snapshots and storing them to the - * file system. + * remote file system. * *

      This method is called during the two-phase commit process for lake table snapshots. It * performs the following operations for each table in the request: @@ -81,6 +81,8 @@ CompletableFuture commitRemoteLogManifest( *

    • Stores the merged snapshot to the remote file system. The stored file contains the log * end offset information for each bucket in the table *
    • Returns the file path where the snapshot is stored + *
    • Call {@link #commitLakeTableSnapshot(CommitLakeTableSnapshotRequest)} with the offset + * file path to finalize the snapshot commit to ZooKeeper in the second phase. *
    * * @param request the request containing lake table snapshot information for one or more tables @@ -88,7 +90,7 @@ CompletableFuture commitRemoteLogManifest( * (containing bucket log end offset information) are stored, or error information for * tables that failed to process */ - @RPC(api = ApiKeys.PRE_LAKE_TABLE_SNAPSHOT) + @RPC(api = ApiKeys.PREPARE_LAKE_TABLE_SNAPSHOT) CompletableFuture prepareLakeTableSnapshot( PrepareLakeTableSnapshotRequest request); 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 07a9b44d6f..cc033ba8a9 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 @@ -80,7 +80,7 @@ public enum ApiKeys { REBALANCE(1049, 0, 0, PUBLIC), LIST_REBALANCE_PROGRESS(1050, 0, 0, PUBLIC), CANCEL_REBALANCE(1051, 0, 0, PUBLIC), - PRE_LAKE_TABLE_SNAPSHOT(1052, 0, 0, PRIVATE); + PREPARE_LAKE_TABLE_SNAPSHOT(1052, 0, 0, PRIVATE); private static final Map ID_TO_TYPE = Arrays.stream(ApiKeys.values()) diff --git a/fluss-rpc/src/main/proto/FlussApi.proto b/fluss-rpc/src/main/proto/FlussApi.proto index 07a569c9ca..6698d98d58 100644 --- a/fluss-rpc/src/main/proto/FlussApi.proto +++ b/fluss-rpc/src/main/proto/FlussApi.proto @@ -1046,7 +1046,7 @@ message PbPrepareLakeTableRespForTable { optional int32 error_code = 1; optional string error_message = 2; optional int64 table_id = 3; - optional string lake_table_bucket_offsets_path = 4; + optional string lake_table_offsets_path = 4; } message PbTableOffsets { 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 d9ed2db983..5c429c379a 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 @@ -1150,12 +1150,15 @@ private void processNotifyKvSnapshotOffsetEvent(NotifyKvSnapshotOffsetEvent even private void processNotifyLakeTableOffsetEvent(NotifyLakeTableOffsetEvent event) { Map lakeTableSnapshots = event.getLakeTableSnapshots(); - Map tableBucketMaxTieredTimestamps = - event.getTableBucketMaxTieredTimestamps(); + Map> tableMaxTieredTimestamps = + event.getTableMaxTieredTimestamps(); coordinatorRequestBatch.newBatch(); for (Map.Entry lakeTableSnapshotEntry : lakeTableSnapshots.entrySet()) { LakeTableSnapshot lakeTableSnapshot = lakeTableSnapshotEntry.getValue(); + Map tableBucketMaxTieredTimestamps = + tableMaxTieredTimestamps.getOrDefault( + lakeTableSnapshotEntry.getKey(), Collections.emptyMap()); for (TableBucket tb : lakeTableSnapshot.getBucketLogEndOffset().keySet()) { coordinatorContext .getBucketLeaderAndIsr(tb) @@ -1257,6 +1260,7 @@ private void handleCommitLakeTableSnapshotV1( try { CommitLakeTableSnapshotResponse response = new CommitLakeTableSnapshotResponse(); + Set failedTableIds = new HashSet<>(); for (Map.Entry lakeTableSnapshotEntry : lakeTableSnapshots.entrySet()) { Long tableId = lakeTableSnapshotEntry.getKey(); @@ -1277,18 +1281,24 @@ private void handleCommitLakeTableSnapshotV1( // this involves IO operation (ZK), so we do it in ioExecutor lakeTableHelper.registerLakeTableSnapshotV1( tableId, lakeTableSnapshotEntry.getValue()); - // send notify lakehouse data request to all replicas via - // coordinator event - coordinatorEventManager.put( - new NotifyLakeTableOffsetEvent( - lakeTableSnapshots, - commitLakeTableSnapshotData - .getTableBucketsMaxTieredTimestamp())); } catch (Exception e) { + failedTableIds.add(tableId); ApiError error = ApiError.fromThrowable(e); tableResp.setError(error.error().code(), error.message()); } } + + // 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)); callback.complete(response); } catch (Exception e) { callback.completeExceptionally(e); @@ -1308,6 +1318,7 @@ private void handleCommitLakeTableSnapshotV2( try { CommitLakeTableSnapshotResponse response = new CommitLakeTableSnapshotResponse(); + Set failedTableIds = new HashSet<>(); for (Map.Entry lakeSnapshotMetadataEntry : lakeSnapshotMetadatas.entrySet()) { PbCommitLakeTableSnapshotRespForTable tableResp = @@ -1317,16 +1328,23 @@ private void handleCommitLakeTableSnapshotV2( try { lakeTableHelper.registerLakeTableSnapshotV2( tableId, lakeSnapshotMetadataEntry.getValue()); - coordinatorEventManager.put( - new NotifyLakeTableOffsetEvent( - commitLakeTableSnapshotData.getLakeTableSnapshot(), - commitLakeTableSnapshotData - .getTableBucketsMaxTieredTimestamp())); } 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)); 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 1ebd80070e..f7e565a7b0 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 @@ -672,7 +672,7 @@ public CompletableFuture prepareLakeTableSnaps lakeTableHelper.storeLakeTableOffsetsFile( tablePath, tableBucketOffsets); pbPrepareLakeTableRespForTable.setTableId(tableId); - pbPrepareLakeTableRespForTable.setLakeTableBucketOffsetsPath( + pbPrepareLakeTableRespForTable.setLakeTableOffsetsPath( fsPath.toString()); } catch (Exception e) { Errors error = ApiError.fromThrowable(e).error(); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/NotifyLakeTableOffsetEvent.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/NotifyLakeTableOffsetEvent.java index ba039f2b6b..5dcb224a79 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/NotifyLakeTableOffsetEvent.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/NotifyLakeTableOffsetEvent.java @@ -26,20 +26,20 @@ public class NotifyLakeTableOffsetEvent implements CoordinatorEvent { private final Map lakeTableSnapshots; - private final Map tableBucketMaxTieredTimestamps; + private final Map> tableMaxTieredTimestamps; public NotifyLakeTableOffsetEvent( Map lakeTableSnapshots, - Map tableBucketMaxTieredTimestamps) { + Map> tableMaxTieredTimestamps) { this.lakeTableSnapshots = lakeTableSnapshots; - this.tableBucketMaxTieredTimestamps = tableBucketMaxTieredTimestamps; + this.tableMaxTieredTimestamps = tableMaxTieredTimestamps; } public Map getLakeTableSnapshots() { return lakeTableSnapshots; } - public Map getTableBucketMaxTieredTimestamps() { - return tableBucketMaxTieredTimestamps; + public Map> getTableMaxTieredTimestamps() { + return tableMaxTieredTimestamps; } } 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 index 9a913cd4aa..3ff7e3e2b3 100644 --- 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 @@ -42,17 +42,17 @@ public class CommitLakeTableSnapshotData { * future, we plan to have the tiering service directly report metrics, and this field will be * removed. */ - private final Map tableBucketsMaxTieredTimestamp; + private final Map> tableMaxTieredTimestamps; // the following field only non-empty since 0.9 private final Map lakeTableSnapshotMetadatas; public CommitLakeTableSnapshotData( Map lakeTableSnapshots, - Map tableBucketsMaxTieredTimestamp, + Map> tableMaxTieredTimestamps, Map lakeTableSnapshotMetadatas) { this.lakeTableSnapshots = lakeTableSnapshots; - this.tableBucketsMaxTieredTimestamp = tableBucketsMaxTieredTimestamp; + this.tableMaxTieredTimestamps = tableMaxTieredTimestamps; this.lakeTableSnapshotMetadatas = lakeTableSnapshotMetadatas; } @@ -60,8 +60,8 @@ public Map getLakeTableSnapshot() { return lakeTableSnapshots; } - public Map getTableBucketsMaxTieredTimestamp() { - return tableBucketsMaxTieredTimestamp; + public Map> getTableMaxTieredTimestamps() { + return tableMaxTieredTimestamps; } public Map getLakeTableSnapshotMetadatas() { @@ -78,15 +78,14 @@ public boolean equals(Object o) { } CommitLakeTableSnapshotData that = (CommitLakeTableSnapshotData) o; return Objects.equals(lakeTableSnapshots, that.lakeTableSnapshots) - && Objects.equals( - tableBucketsMaxTieredTimestamp, that.tableBucketsMaxTieredTimestamp) + && Objects.equals(tableMaxTieredTimestamps, that.tableMaxTieredTimestamps) && Objects.equals(lakeTableSnapshotMetadatas, that.lakeTableSnapshotMetadatas); } @Override public int hashCode() { return Objects.hash( - lakeTableSnapshots, tableBucketsMaxTieredTimestamp, lakeTableSnapshotMetadatas); + lakeTableSnapshots, tableMaxTieredTimestamps, lakeTableSnapshotMetadatas); } @Override @@ -94,8 +93,8 @@ public String toString() { return "CommitLakeTableSnapshotData{" + "lakeTableSnapshots=" + lakeTableSnapshots - + ", tableBucketsMaxTieredTimestamp=" - + tableBucketsMaxTieredTimestamp + + ", tableMaxTieredTimestamps=" + + tableMaxTieredTimestamps + ", lakeTableSnapshotMetadatas=" + lakeTableSnapshotMetadatas + '}'; 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 7601b249ea..8ee46545b0 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 @@ -1551,11 +1551,12 @@ public static CommitLakeTableSnapshotData getCommitLakeTableSnapshotData( CommitLakeTableSnapshotRequest request) { // handle rpc before 0.9 Map lakeTableInfoByTableId = new HashMap<>(); - Map tableBucketsMaxTimestamp = new HashMap<>(); + Map> tableBucketsMaxTimestamp = new HashMap<>(); for (PbLakeTableSnapshotInfo pbLakeTableSnapshotInfo : request.getTablesReqsList()) { long tableId = pbLakeTableSnapshotInfo.getTableId(); long snapshotId = pbLakeTableSnapshotInfo.getSnapshotId(); Map bucketLogEndOffset = new HashMap<>(); + Map bucketLogMaxTimestamp = new HashMap<>(); for (PbLakeTableOffsetForBucket lakeTableOffsetForBucket : pbLakeTableSnapshotInfo.getBucketsReqsList()) { Long partitionId = @@ -1569,7 +1570,7 @@ public static CommitLakeTableSnapshotData getCommitLakeTableSnapshotData( ? lakeTableOffsetForBucket.getLogEndOffset() : null; if (lakeTableOffsetForBucket.hasMaxTimestamp()) { - tableBucketsMaxTimestamp.put( + bucketLogMaxTimestamp.put( tableBucket, lakeTableOffsetForBucket.getMaxTimestamp()); } bucketLogEndOffset.put(tableBucket, logEndOffset); @@ -1577,6 +1578,7 @@ public static CommitLakeTableSnapshotData getCommitLakeTableSnapshotData( LakeTableSnapshot lakeTableSnapshot = new LakeTableSnapshot(snapshotId, bucketLogEndOffset); lakeTableInfoByTableId.put(tableId, lakeTableSnapshot); + tableBucketsMaxTimestamp.put(tableId, bucketLogMaxTimestamp); } // handle rpc since 0.9