diff --git a/fluss-client/src/main/java/org/apache/fluss/client/metadata/LakeSnapshot.java b/fluss-client/src/main/java/org/apache/fluss/client/metadata/LakeSnapshot.java index 8c74a5eb39..89c7864106 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/metadata/LakeSnapshot.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/metadata/LakeSnapshot.java @@ -37,17 +37,9 @@ public class LakeSnapshot { // the specific log offset of the snapshot private final Map tableBucketsOffset; - // the partition name by partition id of this lake snapshot if - // is a partitioned table, empty if not a partitioned table - private final Map partitionNameById; - - public LakeSnapshot( - long snapshotId, - Map tableBucketsOffset, - Map partitionNameById) { + public LakeSnapshot(long snapshotId, Map tableBucketsOffset) { this.snapshotId = snapshotId; this.tableBucketsOffset = tableBucketsOffset; - this.partitionNameById = partitionNameById; } public long getSnapshotId() { @@ -58,10 +50,6 @@ public Map getTableBucketsOffset() { return Collections.unmodifiableMap(tableBucketsOffset); } - public Map getPartitionNameById() { - return Collections.unmodifiableMap(partitionNameById); - } - @Override public String toString() { return "LakeSnapshot{" @@ -69,8 +57,6 @@ public String toString() { + snapshotId + ", tableBucketsOffset=" + tableBucketsOffset - + ", partitionNameById=" - + partitionNameById + '}'; } } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java b/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java index 58efb7f471..3e349cca3a 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java @@ -215,7 +215,6 @@ public static LakeSnapshot toLakeTableSnapshotInfo(GetLatestLakeSnapshotResponse long snapshotId = response.getSnapshotId(); Map tableBucketsOffset = new HashMap<>(response.getBucketSnapshotsCount()); - Map partitionNameById = new HashMap<>(); for (PbLakeSnapshotForBucket pbLakeSnapshotForBucket : response.getBucketSnapshotsList()) { Long partitionId = pbLakeSnapshotForBucket.hasPartitionId() @@ -223,12 +222,9 @@ public static LakeSnapshot toLakeTableSnapshotInfo(GetLatestLakeSnapshotResponse : null; TableBucket tableBucket = new TableBucket(tableId, partitionId, pbLakeSnapshotForBucket.getBucketId()); - if (partitionId != null && pbLakeSnapshotForBucket.hasPartitionName()) { - partitionNameById.put(partitionId, pbLakeSnapshotForBucket.getPartitionName()); - } tableBucketsOffset.put(tableBucket, pbLakeSnapshotForBucket.getLogOffset()); } - return new LakeSnapshot(snapshotId, tableBucketsOffset, partitionNameById); + return new LakeSnapshot(snapshotId, tableBucketsOffset); } public static List toFsPathAndFileName( diff --git a/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java b/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java index d571db2703..b33ffc9831 100644 --- a/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java +++ b/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java @@ -267,6 +267,17 @@ public class ConfigOptions { public static final ConfigOption> SERVER_SASL_ENABLED_MECHANISMS_CONFIG = key("security.sasl.enabled.mechanisms").stringType().asList().noDefaultValue(); + public static final ConfigOption SERVER_IO_POOL_SIZE = + key("server.io-pool.size") + .intType() + .defaultValue(10) + .withDescription( + "The size of the IO thread pool to run blocking operations for both coordinator and tablet servers. " + + "This includes discard unnecessary snapshot files, transfer kv snapshot files, " + + "and transfer remote log files. Increase this value if you experience slow IO operations. " + + "The default value is 10.") + .withDeprecatedKeys("coordinator.io-pool.size"); + // ------------------------------------------------------------------------ // ConfigOptions for Coordinator Server // ------------------------------------------------------------------------ @@ -324,6 +335,11 @@ public class ConfigOptions { + " (“50100,50101”), ranges (“50100-50200”) or a combination of both." + "This option is deprecated. Please use bind.listeners instead, which provides a more flexible configuration for multiple ports"); + /** + * @deprecated This option is deprecated. Please use {@link ConfigOptions#SERVER_IO_POOL_SIZE} + * instead. + */ + @Deprecated public static final ConfigOption COORDINATOR_IO_POOL_SIZE = key("coordinator.io-pool.size") .intType() @@ -332,7 +348,8 @@ public class ConfigOptions { "The size of the IO thread pool to run blocking operations for coordinator server. " + "This includes discard unnecessary snapshot files. " + "Increase this value if you experience slow unnecessary snapshot files clean. " - + "The default value is 10."); + + "The default value is 10. " + + "This option is deprecated. Please use server.io-pool.size instead."); // ------------------------------------------------------------------------ // ConfigOptions for Tablet Server @@ -769,13 +786,19 @@ public class ConfigOptions { "Size of the thread pool used in scheduling tasks to copy segments, " + "fetch remote log indexes and clean up remote log segments."); + /** + * @deprecated This option is deprecated. Please use {@link ConfigOptions#SERVER_IO_POOL_SIZE} + * instead. + */ + @Deprecated public static final ConfigOption REMOTE_LOG_DATA_TRANSFER_THREAD_NUM = key("remote.log.data-transfer-thread-num") .intType() .defaultValue(4) .withDescription( "The number of threads the server uses to transfer (download and upload) " - + "remote log file can be data file, index file and remote log metadata file."); + + "remote log file can be data file, index file and remote log metadata file. " + + "This option is deprecated. Please use server.io-pool.size instead."); // ------------------------------------------------------------------------ // Netty Settings @@ -1441,12 +1464,18 @@ public class ConfigOptions { .withDescription( "The number of threads that the server uses to schedule snapshot kv data for all the replicas in the server."); + /** + * @deprecated This option is deprecated. Please use {@link ConfigOptions#SERVER_IO_POOL_SIZE} + * instead. + */ + @Deprecated public static final ConfigOption KV_SNAPSHOT_TRANSFER_THREAD_NUM = key("kv.snapshot.transfer-thread-num") .intType() .defaultValue(4) .withDescription( - "The number of threads the server uses to transfer (download and upload) kv snapshot files."); + "The number of threads the server uses to transfer (download and upload) kv snapshot files. " + + "This option is deprecated. Please use server.io-pool.size instead."); public static final ConfigOption KV_MAX_RETAINED_SNAPSHOTS = key("kv.snapshot.num-retained") 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 index 1c8faf5d8d..a8731da155 100644 --- 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 @@ -31,17 +31,11 @@ public class BucketOffset implements Serializable { private final long logOffset; private final int bucket; private final @Nullable Long partitionId; - private final @Nullable String partitionQualifiedName; - public BucketOffset( - long logOffset, - int bucket, - @Nullable Long partitionId, - @Nullable String partitionQualifiedName) { + public BucketOffset(long logOffset, int bucket, @Nullable Long partitionId) { this.logOffset = logOffset; this.bucket = bucket; this.partitionId = partitionId; - this.partitionQualifiedName = partitionQualifiedName; } public long getLogOffset() { @@ -57,11 +51,6 @@ public Long getPartitionId() { return partitionId; } - @Nullable - public String getPartitionQualifiedName() { - return partitionQualifiedName; - } - @Override public boolean equals(Object o) { if (this == o) { @@ -71,9 +60,13 @@ public boolean equals(Object o) { return false; } BucketOffset that = (BucketOffset) o; - return bucket == that.bucket - && logOffset == that.logOffset - && Objects.equals(partitionId, that.partitionId) - && Objects.equals(partitionQualifiedName, that.partitionQualifiedName); + 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/lake/committer/CommittedLakeSnapshot.java b/fluss-common/src/main/java/org/apache/fluss/lake/committer/CommittedLakeSnapshot.java index fee80c3901..7b9e2f4cce 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 @@ -34,10 +34,6 @@ public class CommittedLakeSnapshot { // partition bucket private final Map, Long> logEndOffsets = new HashMap<>(); - // partition id -> partition name, will be empty if is not a partitioned table - // the partition name is a qualified name in the format: key1=value1/key2=value2 - private final Map qualifiedPartitionNameById = new HashMap<>(); - public CommittedLakeSnapshot(long lakeSnapshotId) { this.lakeSnapshotId = lakeSnapshotId; } @@ -50,34 +46,30 @@ public void addBucket(int bucketId, long offset) { logEndOffsets.put(Tuple2.of(null, bucketId), offset); } - public void addPartitionBucket( - Long partitionId, String partitionQualifiedName, int bucketId, long offset) { + public void addPartitionBucket(Long partitionId, int bucketId, long offset) { logEndOffsets.put(Tuple2.of(partitionId, bucketId), offset); - qualifiedPartitionNameById.put(partitionId, partitionQualifiedName); } public Map, Long> getLogEndOffsets() { return logEndOffsets; } - public Map getQualifiedPartitionNameById() { - return qualifiedPartitionNameById; - } - @Override public boolean equals(Object o) { + if (this == o) { + return true; + } if (o == null || getClass() != o.getClass()) { return false; } CommittedLakeSnapshot that = (CommittedLakeSnapshot) o; return lakeSnapshotId == that.lakeSnapshotId - && Objects.equals(logEndOffsets, that.logEndOffsets) - && Objects.equals(qualifiedPartitionNameById, that.qualifiedPartitionNameById); + && Objects.equals(logEndOffsets, that.logEndOffsets); } @Override public int hashCode() { - return Objects.hash(lakeSnapshotId, logEndOffsets, qualifiedPartitionNameById); + return Objects.hash(lakeSnapshotId, logEndOffsets); } @Override @@ -87,8 +79,6 @@ public String toString() { + lakeSnapshotId + ", logEndOffsets=" + logEndOffsets - + ", partitionNameById=" - + qualifiedPartitionNameById + '}'; } } 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 e58a1e364e..8c63113780 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 @@ -91,6 +91,8 @@ public class FlussPaths { /** The name of the directory for shared remote snapshot kv files. */ public static final String REMOTE_KV_SNAPSHOT_SHARED_DIR = "shared"; + private static final String REMOTE_LAKE_DIR_NAME = "lake"; + // ---------------------------------------------------------------------------------------- // LOG/KV Tablet Paths // ---------------------------------------------------------------------------------------- @@ -681,6 +683,45 @@ public static FsPath remoteKvSnapshotDir(FsPath remoteKvTabletDir, long snapshot return new FsPath(remoteKvTabletDir, REMOTE_KV_SNAPSHOT_DIR_PREFIX + snapshotId); } + /** + * Returns the remote path for storing lake snapshot required by Fluss for a table. + * + *

The path contract: + * + *

+     * {$remote.data.dir}/lake/{databaseName}/{tableName}-{tableId}
+     * 
+ */ + public static FsPath remoteLakeTableSnapshotDir( + String remoteDataDir, TablePath tablePath, long tableId) { + return new FsPath( + String.format( + "%s/%s/%s/%s-%d", + remoteDataDir, + REMOTE_LAKE_DIR_NAME, + tablePath.getDatabaseName(), + tablePath.getTableName(), + tableId)); + } + + /** + * Returns a remote path for storing lake snapshot metadata required by Fluss for a table. + * + *

The path contract: + * + *

+     * {$remoteLakeTableSnapshotMetadataDir}/metadata/{uuid}.manifest
+     * 
+ */ + public static FsPath remoteLakeTableSnapshotManifestPath( + String remoteDataDir, TablePath tablePath, long tableId) { + return new FsPath( + String.format( + "%s/metadata/%s.manifest", + remoteLakeTableSnapshotDir(remoteDataDir, tablePath, tableId), + UUID.randomUUID())); + } + /** * Returns the remote directory path for storing kv snapshot shared files (SST files with UUID * prefix). 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 index 0331a4628d..235268e682 100644 --- 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 @@ -30,7 +30,6 @@ public class BucketOffsetJsonSerde 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 PARTITION_NAME = "partition_name"; private static final String LOG_OFFSET = "offset"; @Override @@ -39,14 +38,10 @@ public BucketOffset deserialize(JsonNode node) { Long partitionId = partitionIdNode == null ? null : partitionIdNode.asLong(); int bucketId = node.get(BUCKET_ID).asInt(); - // deserialize partition name - JsonNode partitionNameNode = node.get(PARTITION_NAME); - String partitionName = partitionNameNode == null ? null : partitionNameNode.asText(); - // deserialize log offset long logOffset = node.get(LOG_OFFSET).asLong(); - return new BucketOffset(logOffset, bucketId, partitionId, partitionName); + return new BucketOffset(logOffset, bucketId, partitionId); } @Override @@ -59,11 +54,6 @@ public void serialize(BucketOffset bucketOffset, JsonGenerator generator) throws } generator.writeNumberField(BUCKET_ID, bucketOffset.getBucket()); - // serialize partition name - if (bucketOffset.getPartitionQualifiedName() != null) { - generator.writeStringField(PARTITION_NAME, bucketOffset.getPartitionQualifiedName()); - } - // serialize bucket offset generator.writeNumberField(LOG_OFFSET, bucketOffset.getLogOffset()); 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 index 3fc49774ee..7c3c26d391 100644 --- 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 @@ -28,17 +28,13 @@ public class BucketOffsetJsonSerdeTest extends JsonSerdeTestBase { @Override protected BucketOffset[] createObjects() { - return new BucketOffset[] { - new BucketOffset(10, 1, 1L, "country=eu-central/year=2023/month=12"), - new BucketOffset(20, 2, null, null) - }; + 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,\"partition_name\":\"country=eu-central/year=2023/month=12\",\"offset\":10}", - "{\"bucket\":2,\"offset\":20}" + "{\"partition_id\":1,\"bucket\":1,\"offset\":10}", "{\"bucket\":2,\"offset\":20}" }; } } 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 index 5109499024..b40a04461e 100644 --- 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 @@ -18,7 +18,6 @@ package org.apache.fluss.flink.tiering.committer; import org.apache.fluss.metadata.TableBucket; -import org.apache.fluss.utils.types.Tuple2; import java.util.HashMap; import java.util.Map; @@ -31,19 +30,13 @@ class FlussTableLakeSnapshot { private final long lakeSnapshotId; - // -> log end offsets, - // if the bucket is not of a partition, the partition_name is null - private final Map, Long> logEndOffsets; - - // -> max timestamps, - // if the bucket is not of a partition, the partition_name is null - private final Map, Long> maxTimestamps; + // table_bucket -> log end offsets, + private final Map logEndOffsets; FlussTableLakeSnapshot(long tableId, long lakeSnapshotId) { this.tableId = tableId; this.lakeSnapshotId = lakeSnapshotId; this.logEndOffsets = new HashMap<>(); - this.maxTimestamps = new HashMap<>(); } public long tableId() { @@ -54,27 +47,16 @@ public long lakeSnapshotId() { return lakeSnapshotId; } - public Set> tablePartitionBuckets() { + public Set tableBuckets() { return logEndOffsets.keySet(); } - public void addBucketOffsetAndTimestamp(TableBucket bucket, long offset, long timestamp) { - logEndOffsets.put(Tuple2.of(bucket, null), offset); - maxTimestamps.put(Tuple2.of(bucket, null), timestamp); - } - - public void addPartitionBucketOffsetAndTimestamp( - TableBucket bucket, String partitionName, long offset, long timestamp) { - logEndOffsets.put(Tuple2.of(bucket, partitionName), offset); - maxTimestamps.put(Tuple2.of(bucket, partitionName), timestamp); - } - - public long getLogEndOffset(Tuple2 bucketPartition) { - return logEndOffsets.get(bucketPartition); + public void addBucketOffset(TableBucket bucket, long offset) { + logEndOffsets.put(bucket, offset); } - public long getMaxTimestamp(Tuple2 bucketPartition) { - return maxTimestamps.get(bucketPartition); + public long getLogEndOffset(TableBucket bucket) { + return logEndOffsets.get(bucket); } @Override @@ -86,8 +68,6 @@ public String toString() { + lakeSnapshotId + ", logEndOffsets=" + logEndOffsets - + ", maxTimestamps=" - + maxTimestamps + '}'; } } 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 a6cc477672..b78614dc44 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,7 +21,6 @@ import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; import org.apache.fluss.lake.committer.CommittedLakeSnapshot; -import org.apache.fluss.metadata.ResolvedPartitionSpec; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metrics.registry.MetricRegistry; import org.apache.fluss.rpc.GatewayClientProxy; @@ -84,31 +83,9 @@ public void commit(long tableId, CommittedLakeSnapshot committedLakeSnapshot) for (Map.Entry, Long> entry : committedLakeSnapshot.getLogEndOffsets().entrySet()) { Tuple2 partitionBucket = entry.getKey(); - TableBucket tableBucket; Long partitionId = partitionBucket.f0; - if (partitionId == null) { - tableBucket = new TableBucket(tableId, partitionBucket.f1); - // we use -1 since we don't store timestamp in lake snapshot property for - // simplicity, it may cause the timestamp to be -1 during constructing lake - // snapshot to commit to Fluss. - // But it should happen rarely and should be a normal value after next tiering. - flussTableLakeSnapshot.addBucketOffsetAndTimestamp( - tableBucket, entry.getValue(), -1); - } else { - tableBucket = new TableBucket(tableId, partitionId, partitionBucket.f1); - // the partition name is qualified partition name in format: - // key1=value1/key2=value2. - // we need to convert to partition name in format: value1$value2$ - String qualifiedPartitionName = - committedLakeSnapshot.getQualifiedPartitionNameById().get(partitionId); - ResolvedPartitionSpec resolvedPartitionSpec = - ResolvedPartitionSpec.fromPartitionQualifiedName(qualifiedPartitionName); - flussTableLakeSnapshot.addPartitionBucketOffsetAndTimestamp( - tableBucket, - resolvedPartitionSpec.getPartitionName(), - entry.getValue(), - -1); - } + TableBucket tableBucket = new TableBucket(tableId, partitionId, partitionBucket.f1); + flussTableLakeSnapshot.addBucketOffset(tableBucket, entry.getValue()); } commit(flussTableLakeSnapshot); } @@ -122,23 +99,15 @@ private CommitLakeTableSnapshotRequest toCommitLakeTableSnapshotRequest( pbLakeTableSnapshotInfo.setTableId(flussTableLakeSnapshot.tableId()); pbLakeTableSnapshotInfo.setSnapshotId(flussTableLakeSnapshot.lakeSnapshotId()); - for (Tuple2 bucketPartition : - flussTableLakeSnapshot.tablePartitionBuckets()) { + for (TableBucket tableBucket : flussTableLakeSnapshot.tableBuckets()) { PbLakeTableOffsetForBucket pbLakeTableOffsetForBucket = pbLakeTableSnapshotInfo.addBucketsReq(); - TableBucket tableBucket = bucketPartition.f0; - String partitionName = bucketPartition.f1; - long endOffset = flussTableLakeSnapshot.getLogEndOffset(bucketPartition); - long maxTimestamp = flussTableLakeSnapshot.getMaxTimestamp(bucketPartition); + long endOffset = flussTableLakeSnapshot.getLogEndOffset(tableBucket); if (tableBucket.getPartitionId() != null) { pbLakeTableOffsetForBucket.setPartitionId(tableBucket.getPartitionId()); } - if (partitionName != null) { - pbLakeTableOffsetForBucket.setPartitionName(partitionName); - } pbLakeTableOffsetForBucket.setBucketId(tableBucket.getBucket()); pbLakeTableOffsetForBucket.setLogEndOffset(endOffset); - 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 0d3fe32d29..ebe4a1ae68 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 @@ -33,7 +33,6 @@ 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.ResolvedPartitionSpec; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePath; @@ -213,8 +212,7 @@ private Committable commitWriteResults( LakeSnapshot flussCurrentLakeSnapshot = getLatestLakeSnapshot(tablePath); Map logOffsetsProperty = - toBucketOffsetsProperty( - tablePath, flussCurrentLakeSnapshot, committableWriteResults); + toBucketOffsetsProperty(flussCurrentLakeSnapshot, committableWriteResults); // to committable Committable committable = lakeCommitter.toCommittable(writeResults); // before commit to lake, check fluss not missing any lake snapshot committed by fluss @@ -231,16 +229,7 @@ private Committable commitWriteResults( new FlussTableLakeSnapshot(tableId, committedSnapshotId); for (TableBucketWriteResult writeResult : committableWriteResults) { TableBucket tableBucket = writeResult.tableBucket(); - if (writeResult.tableBucket().getPartitionId() == null) { - flussTableLakeSnapshot.addBucketOffsetAndTimestamp( - tableBucket, writeResult.logEndOffset(), writeResult.maxTimestamp()); - } else { - flussTableLakeSnapshot.addPartitionBucketOffsetAndTimestamp( - tableBucket, - writeResult.partitionName(), - writeResult.logEndOffset(), - writeResult.maxTimestamp()); - } + flussTableLakeSnapshot.addBucketOffset(tableBucket, writeResult.logEndOffset()); } flussTableLakeSnapshotCommitter.commit(flussTableLakeSnapshot); return committable; @@ -252,63 +241,33 @@ private Committable commitWriteResults( * offsets. */ private Map toBucketOffsetsProperty( - TablePath tablePath, @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<>(); - Map partitionNameById = new HashMap<>(); if (latestLakeSnapshot != null) { tableBucketOffsets = new HashMap<>(latestLakeSnapshot.getTableBucketsOffset()); - partitionNameById = new HashMap<>(latestLakeSnapshot.getPartitionNameById()); } for (TableBucketWriteResult tableBucketWriteResult : currentWriteResults) { tableBucketOffsets.put( tableBucketWriteResult.tableBucket(), tableBucketWriteResult.logEndOffset()); - if (tableBucketWriteResult.tableBucket().getPartitionId() != null - && tableBucketWriteResult.partitionName() != null) { - partitionNameById.put( - tableBucketWriteResult.tableBucket().getPartitionId(), - tableBucketWriteResult.partitionName()); - } - } - - List partitionKeys = new ArrayList<>(); - if (!partitionNameById.isEmpty()) { - partitionKeys = admin.getTableInfo(tablePath).get().getPartitionKeys(); } // then, serialize the bucket offsets, partition name by id - return toBucketOffsetsProperty(tableBucketOffsets, partitionNameById, partitionKeys); + return toBucketOffsetsProperty(tableBucketOffsets); } public static Map toBucketOffsetsProperty( - Map tableBucketOffsets, - Map partitionNameById, - List partitionKeys) - throws IOException { + 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(); - String partitionQualifiedName = null; - if (partitionId != null) { - // the partitionName is 2025$12$03, we need to convert to - // qualified name year=2025/month=12/day=03 - partitionQualifiedName = - ResolvedPartitionSpec.fromPartitionName( - partitionKeys, partitionNameById.get(partitionId)) - .getPartitionQualifiedName(); - } BucketOffsetJsonSerde.INSTANCE.serialize( - new BucketOffset( - entry.getValue(), - entry.getKey().getBucket(), - entry.getKey().getPartitionId(), - partitionQualifiedName), + new BucketOffset(entry.getValue(), entry.getKey().getBucket(), partitionId), gen); } gen.writeEndArray(); 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 0061e2abb4..8b67927af4 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 @@ -39,7 +39,7 @@ public class TestingLakeTieringFactory implements LakeTieringFactory { - @Nullable private final TestingLakeCommitter testingLakeCommitter; + @Nullable private TestingLakeCommitter testingLakeCommitter; public TestingLakeTieringFactory(@Nullable TestingLakeCommitter testingLakeCommitter) { this.testingLakeCommitter = testingLakeCommitter; @@ -63,7 +63,10 @@ public SimpleVersionedSerializer getWriteResultSerializer() @Override public LakeCommitter createLakeCommitter( CommitterInitContext committerInitContext) throws IOException { - return testingLakeCommitter == null ? new TestingLakeCommitter() : testingLakeCommitter; + if (testingLakeCommitter == null) { + this.testingLakeCommitter = new TestingLakeCommitter(); + } + return testingLakeCommitter; } @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 d1b3ddc6d8..927d69e2b1 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 @@ -20,7 +20,6 @@ import org.apache.fluss.client.metadata.LakeSnapshot; import org.apache.fluss.flink.utils.FlinkTestBase; import org.apache.fluss.lake.committer.CommittedLakeSnapshot; -import org.apache.fluss.metadata.ResolvedPartitionSpec; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TablePath; @@ -92,13 +91,7 @@ void testCommit(boolean isPartitioned) throws Exception { expectedOffsets.put(new TableBucket(tableId, bucket), bucketOffset); } else { long partitionId = partitionNameAndIds.get(partitionName); - committedLakeSnapshot.addPartitionBucket( - partitionId, - ResolvedPartitionSpec.fromPartitionName( - Collections.singletonList("a"), partitionName) - .getPartitionQualifiedName(), - bucket, - bucketOffset); + committedLakeSnapshot.addPartitionBucket(partitionId, bucket, bucketOffset); expectedOffsets.put( new TableBucket(tableId, partitionId, bucket), bucketOffset); expectedPartitionNameById.put(partitionId, partitionName); @@ -114,9 +107,5 @@ void testCommit(boolean isPartitioned) throws Exception { // get and check the offsets Map bucketLogOffsets = lakeSnapshot.getTableBucketsOffset(); assertThat(bucketLogOffsets).isEqualTo(expectedOffsets); - - // check partition name - Map partitionNameById = lakeSnapshot.getPartitionNameById(); - assertThat(partitionNameById).isEqualTo(expectedPartitionNameById); } } 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 08dc96468b..593dcadd18 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 @@ -26,11 +26,8 @@ import org.apache.fluss.flink.tiering.source.TableBucketWriteResult; import org.apache.fluss.flink.utils.FlinkTestBase; import org.apache.fluss.lake.committer.CommittedLakeSnapshot; -import org.apache.fluss.metadata.ResolvedPartitionSpec; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TablePath; -import org.apache.fluss.server.zk.ZooKeeperClient; -import org.apache.fluss.server.zk.data.LakeTableSnapshot; import org.apache.fluss.utils.types.Tuple2; import org.apache.flink.configuration.Configuration; @@ -58,7 +55,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; import static org.apache.fluss.record.TestData.DATA1_PARTITIONED_TABLE_DESCRIPTOR; import static org.assertj.core.api.Assertions.assertThat; @@ -142,11 +138,7 @@ void testCommitNonPartitionedTable() throws Exception { expectedLogEndOffsets.put(t1b0, 11L); expectedLogEndOffsets.put(t1b1, 12L); expectedLogEndOffsets.put(t1b2, 13L); - Map expectedMaxTimestamps = new HashMap<>(); - expectedMaxTimestamps.put(t1b0, 21L); - expectedMaxTimestamps.put(t1b1, 22L); - expectedMaxTimestamps.put(t1b2, 23L); - verifyLakeSnapshot(tablePath1, tableId1, 1, expectedLogEndOffsets, expectedMaxTimestamps); + verifyLakeSnapshot(tablePath1, tableId1, 1, expectedLogEndOffsets); // addColumn table2, bucket2 TableBucket t2b2 = new TableBucket(tableId2, 2); @@ -157,15 +149,10 @@ void testCommitNonPartitionedTable() throws Exception { expectedLogEndOffsets.put(t2b0, 21L); expectedLogEndOffsets.put(t2b1, 22L); expectedLogEndOffsets.put(t2b2, 23L); - expectedMaxTimestamps = new HashMap<>(); - expectedMaxTimestamps.put(t2b0, 31L); - expectedMaxTimestamps.put(t2b1, 32L); - expectedMaxTimestamps.put(t2b2, 33L); - verifyLakeSnapshot(tablePath2, tableId2, 1, expectedLogEndOffsets, expectedMaxTimestamps); + verifyLakeSnapshot(tablePath2, tableId2, 2, expectedLogEndOffsets); // let's process one round of TableBucketWriteResult again expectedLogEndOffsets = new HashMap<>(); - expectedMaxTimestamps = new HashMap<>(); for (int bucket = 0; bucket < 3; bucket++) { TableBucket tableBucket = new TableBucket(tableId1, bucket); long offset = bucket * bucket; @@ -179,9 +166,8 @@ void testCommitNonPartitionedTable() throws Exception { timestamp, numberOfWriteResults)); expectedLogEndOffsets.put(tableBucket, offset); - expectedMaxTimestamps.put(tableBucket, timestamp); } - verifyLakeSnapshot(tablePath1, tableId1, 1, expectedLogEndOffsets, expectedMaxTimestamps); + verifyLakeSnapshot(tablePath1, tableId1, 3, expectedLogEndOffsets); } @Test @@ -214,8 +200,7 @@ void testCommitPartitionedTable() throws Exception { expectedMaxTimestamps.put(tableBucket, currentTimestamp); } if (bucket == 2) { - verifyLakeSnapshot( - tablePath, tableId, 1, expectedLogEndOffsets, expectedMaxTimestamps); + verifyLakeSnapshot(tablePath, tableId, 1, expectedLogEndOffsets); } else { verifyNoLakeSnapshot(tablePath); } @@ -263,10 +248,7 @@ void testCommitMeetsEmptyWriteResult() throws Exception { Map expectedLogEndOffsets = new HashMap<>(); expectedLogEndOffsets.put(new TableBucket(tableId, 1), 1L); expectedLogEndOffsets.put(new TableBucket(tableId, 2), 2L); - Map expectedMaxTimestamps = new HashMap<>(); - expectedMaxTimestamps.put(new TableBucket(tableId, 1), 1L); - expectedMaxTimestamps.put(new TableBucket(tableId, 2), 2L); - verifyLakeSnapshot(tablePath1, tableId, 1, expectedLogEndOffsets, expectedMaxTimestamps); + verifyLakeSnapshot(tablePath1, tableId, 1, expectedLogEndOffsets); } @Test @@ -298,8 +280,6 @@ void testTableCommitWhenFlussMissingLakeSnapshot() throws Exception { tableId, 2, getExpectedLogEndOffsets(tableId, mockCommittedSnapshot), - getExpectedMaxTimestamps(tableId, mockCommittedSnapshot), - mockCommittedSnapshot.getQualifiedPartitionNameById(), 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.", @@ -310,7 +290,6 @@ void testTableCommitWhenFlussMissingLakeSnapshot() throws Exception { mockCommittedSnapshot)); Map expectedLogEndOffsets = new HashMap<>(); - Map expectedMaxTimestamps = new HashMap<>(); for (int bucket = 0; bucket < 3; bucket++) { TableBucket tableBucket = new TableBucket(tableId, bucket); long offset = bucket * bucket; @@ -319,10 +298,9 @@ void testTableCommitWhenFlussMissingLakeSnapshot() throws Exception { createTableBucketWriteResultStreamRecord( tablePath, tableBucket, 3, offset, timestamp, numberOfWriteResults)); expectedLogEndOffsets.put(tableBucket, offset); - expectedMaxTimestamps.put(tableBucket, timestamp); } - verifyLakeSnapshot(tablePath, tableId, 3, expectedLogEndOffsets, expectedMaxTimestamps); + verifyLakeSnapshot(tablePath, tableId, 3, expectedLogEndOffsets); } @Test @@ -369,8 +347,6 @@ void testPartitionedTableCommitWhenFlussMissingLakeSnapshot() throws Exception { tableId, 3, getExpectedLogEndOffsets(tableId, mockCommittedSnapshot), - getExpectedMaxTimestamps(tableId, mockCommittedSnapshot), - mockCommittedSnapshot.getQualifiedPartitionNameById(), 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, @@ -387,13 +363,7 @@ private CommittedLakeSnapshot mockCommittedLakeSnapshot(List partitions, i if (partition == null) { mockCommittedSnapshot.addBucket(bucket, bucket + 1); } else { - mockCommittedSnapshot.addPartitionBucket( - partition, - ResolvedPartitionSpec.fromPartitionValue( - "partition_key", "partition-" + partition) - .getPartitionQualifiedName(), - bucket, - bucket + 1); + mockCommittedSnapshot.addPartitionBucket(partition, bucket, bucket + 1); } } } @@ -418,22 +388,6 @@ private Map getExpectedLogEndOffsets( return expectedLogEndOffsets; } - private Map getExpectedMaxTimestamps( - long tableId, CommittedLakeSnapshot committedLakeSnapshot) { - Map expectedMaxTimestamps = new HashMap<>(); - for (Map.Entry, Long> entry : - committedLakeSnapshot.getLogEndOffsets().entrySet()) { - Tuple2 partitionBucket = entry.getKey(); - if (partitionBucket.f0 == null) { - expectedMaxTimestamps.put(new TableBucket(tableId, partitionBucket.f1), -1L); - } else { - expectedMaxTimestamps.put( - new TableBucket(tableId, partitionBucket.f0, partitionBucket.f1), -1L); - } - } - return expectedMaxTimestamps; - } - private StreamRecord> createTableBucketWriteResultStreamRecord( TablePath tablePath, @@ -483,16 +437,12 @@ private void verifyLakeSnapshot( TablePath tablePath, long tableId, long expectedSnapshotId, - Map expectedLogEndOffsets, - Map expectedMaxTimestamp) + Map expectedLogEndOffsets) throws Exception { LakeSnapshot lakeSnapshot = admin.getLatestLakeSnapshot(tablePath).get(); assertThat(lakeSnapshot.getSnapshotId()).isEqualTo(expectedSnapshotId); assertThat(lakeSnapshot.getTableBucketsOffset()).isEqualTo(expectedLogEndOffsets); - // TODO: replace with LakeSnapshot when support max timestamp - verifyLakeSnapshotMaxTimestamp(tableId, expectedMaxTimestamp); - // check the tableId has been sent to mark finished List operatorEvents = mockOperatorEventGateway.getEventsSent(); SourceEventWrapper sourceEventWrapper = @@ -507,17 +457,11 @@ private void verifyLakeSnapshot( long tableId, long expectedSnapshotId, Map expectedLogEndOffsets, - Map expectedMaxTimestamp, - Map expectedPartitionIdByName, String failedReason) throws Exception { LakeSnapshot lakeSnapshot = admin.getLatestLakeSnapshot(tablePath).get(); assertThat(lakeSnapshot.getSnapshotId()).isEqualTo(expectedSnapshotId); assertThat(lakeSnapshot.getTableBucketsOffset()).isEqualTo(expectedLogEndOffsets); - assertThat(lakeSnapshot.getPartitionNameById()).isEqualTo(expectedPartitionIdByName); - - // TODO: replace with LakeSnapshot when support max timestamp - verifyLakeSnapshotMaxTimestamp(tableId, expectedMaxTimestamp); // check the tableId has been sent to mark failed List operatorEvents = mockOperatorEventGateway.getEventsSent(); @@ -529,15 +473,6 @@ private void verifyLakeSnapshot( assertThat(finishTieringEvent.failReason()).contains(failedReason); } - private void verifyLakeSnapshotMaxTimestamp( - long tableId, Map expectedMaxTimestamp) throws Exception { - ZooKeeperClient zkClient = FLUSS_CLUSTER_EXTENSION.getZooKeeperClient(); - Optional lakeTableSnapshotOpt = zkClient.getLakeTableSnapshot(tableId); - assertThat(lakeTableSnapshotOpt).isNotEmpty(); - LakeTableSnapshot lakeTableSnapshot = lakeTableSnapshotOpt.get(); - assertThat(lakeTableSnapshot.getBucketMaxTimestamp()).isEqualTo(expectedMaxTimestamp); - } - private static class MockOperatorEventDispatcher implements OperatorEventDispatcher { private final OperatorEventGateway operatorEventGateway; diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumeratorTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumeratorTest.java index b93791660a..ba8c5b17a8 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumeratorTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumeratorTest.java @@ -102,21 +102,15 @@ void testPrimaryKeyTableWithNoSnapshotSplits() throws Throwable { // mock finished tiered this round, check second round context.getSplitsAssignmentSequence().clear(); - final Map bucketOffsetOfEarliest = new HashMap<>(); final Map bucketOffsetOfInitialWrite = new HashMap<>(); for (int tableBucket = 0; tableBucket < DEFAULT_BUCKET_NUM; tableBucket++) { - bucketOffsetOfEarliest.put(tableBucket, EARLIEST_OFFSET); bucketOffsetOfInitialWrite.put(tableBucket, 0L); } // commit and notify this table tiering task finished coordinatorGateway .commitLakeTableSnapshot( genCommitLakeTableSnapshotRequest( - tableId, - null, - 0, - bucketOffsetOfEarliest, - bucketOffsetOfInitialWrite)) + tableId, null, 0, bucketOffsetOfInitialWrite)) .get(); enumerator.handleSourceEvent(1, new FinishedTieringEvent(tableId)); @@ -211,11 +205,7 @@ void testPrimaryKeyTableWithSnapshotSplits() throws Throwable { coordinatorGateway .commitLakeTableSnapshot( genCommitLakeTableSnapshotRequest( - tableId, - null, - 1, - initialBucketOffsets, - bucketOffsetOfInitialWrite)) + tableId, null, 1, bucketOffsetOfInitialWrite)) .get(); enumerator.handleSourceEvent(1, new FinishedTieringEvent(tableId)); @@ -312,11 +302,7 @@ void testLogTableSplits() throws Throwable { coordinatorGateway .commitLakeTableSnapshot( genCommitLakeTableSnapshotRequest( - tableId, - null, - 0, - bucketOffsetOfEarliest, - bucketOffsetOfInitialWrite)) + tableId, null, 0, bucketOffsetOfInitialWrite)) .get(); enumerator.handleSourceEvent(1, new FinishedTieringEvent(tableId)); @@ -393,6 +379,7 @@ void testPartitionedPrimaryKeyTable() throws Throwable { // mock finished tiered this round, check second round context.getSplitsAssignmentSequence().clear(); + long snapshotId = 1; final Map> bucketOffsetOfInitialWrite = new HashMap<>(); for (Map.Entry partitionNameById : partitionNameByIds.entrySet()) { Map partitionBucketOffsetOfEarliest = new HashMap<>(); @@ -409,8 +396,7 @@ void testPartitionedPrimaryKeyTable() throws Throwable { genCommitLakeTableSnapshotRequest( tableId, partitionNameById.getValue(), - 1, - partitionBucketOffsetOfEarliest, + snapshotId++, bucketOffsetOfInitialWrite.get( partitionNameById.getValue()))) .get(); @@ -421,8 +407,7 @@ void testPartitionedPrimaryKeyTable() throws Throwable { Map> bucketOffsetOfSecondWrite = upsertRowForPartitionedTable( tablePath, DEFAULT_PK_TABLE_DESCRIPTOR, partitionNameByIds, 10, 20); - long snapshotId = 0; - waitUntilPartitionTableSnapshot(tableId, partitionNameByIds, snapshotId); + waitUntilPartitionTableSnapshot(tableId, partitionNameByIds, 0); // request tiering table splits for (int subtaskId = 0; subtaskId < numSubtasks; subtaskId++) { @@ -518,6 +503,7 @@ void testPartitionedLogTableSplits() throws Throwable { // mock finished tiered this round, check second round context.getSplitsAssignmentSequence().clear(); final Map> bucketOffsetOfInitialWrite = new HashMap<>(); + long snapshot = 1; for (Map.Entry partitionNameById : partitionNameByIds.entrySet()) { long partitionId = partitionNameById.getValue(); Map partitionInitialBucketOffsets = new HashMap<>(); @@ -537,8 +523,7 @@ void testPartitionedLogTableSplits() throws Throwable { genCommitLakeTableSnapshotRequest( tableId, partitionId, - 1, - partitionInitialBucketOffsets, + snapshot++, bucketOffsetOfInitialWrite.get(partitionId))) .get(); } @@ -692,23 +677,21 @@ private static CommitLakeTableSnapshotRequest genCommitLakeTableSnapshotRequest( long tableId, @Nullable Long partitionId, long snapshotId, - Map bucketLogStartOffsets, Map bucketLogEndOffsets) { CommitLakeTableSnapshotRequest commitLakeTableSnapshotRequest = new CommitLakeTableSnapshotRequest(); PbLakeTableSnapshotInfo reqForTable = commitLakeTableSnapshotRequest.addTablesReq(); reqForTable.setTableId(tableId); reqForTable.setSnapshotId(snapshotId); - for (Map.Entry bucketLogStartOffset : bucketLogStartOffsets.entrySet()) { - int bucketId = bucketLogStartOffset.getKey(); + for (Map.Entry bucketLogEndOffset : bucketLogEndOffsets.entrySet()) { + int bucketId = bucketLogEndOffset.getKey(); TableBucket tb = new TableBucket(tableId, partitionId, bucketId); PbLakeTableOffsetForBucket lakeTableOffsetForBucket = reqForTable.addBucketsReq(); if (tb.getPartitionId() != null) { lakeTableOffsetForBucket.setPartitionId(tb.getPartitionId()); } lakeTableOffsetForBucket.setBucketId(tb.getBucket()); - lakeTableOffsetForBucket.setLogStartOffset(bucketLogStartOffset.getValue()); - lakeTableOffsetForBucket.setLogEndOffset(bucketLogEndOffsets.get(bucketId)); + lakeTableOffsetForBucket.setLogEndOffset(bucketLogEndOffset.getValue()); } return commitLakeTableSnapshotRequest; } diff --git a/fluss-lake/fluss-lake-iceberg/src/main/java/org/apache/fluss/lake/iceberg/tiering/IcebergLakeCommitter.java b/fluss-lake/fluss-lake-iceberg/src/main/java/org/apache/fluss/lake/iceberg/tiering/IcebergLakeCommitter.java index 0444244a7f..935d062316 100644 --- a/fluss-lake/fluss-lake-iceberg/src/main/java/org/apache/fluss/lake/iceberg/tiering/IcebergLakeCommitter.java +++ b/fluss-lake/fluss-lake-iceberg/src/main/java/org/apache/fluss/lake/iceberg/tiering/IcebergLakeCommitter.java @@ -268,7 +268,6 @@ public CommittedLakeSnapshot getMissingLakeSnapshot(@Nullable Long latestLakeSna if (bucketOffset.getPartitionId() != null) { committedLakeSnapshot.addPartitionBucket( bucketOffset.getPartitionId(), - bucketOffset.getPartitionQualifiedName(), bucketOffset.getBucket(), bucketOffset.getLogOffset()); } else { 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 e150fb3fee..55be048c99 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 @@ -402,8 +402,8 @@ private void testPartitionedTableTiering() throws Exception { put( FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY, "[" - + "{\"partition_id\":0,\"bucket\":0,\"partition_name\":\"date=2025\",\"offset\":3}," - + "{\"partition_id\":1,\"bucket\":0,\"partition_name\":\"date=2026\",\"offset\":3}" + + "{\"partition_id\":0,\"bucket\":0,\"offset\":3}," + + "{\"partition_id\":1,\"bucket\":0,\"offset\":3}" + "]"); } }; diff --git a/fluss-lake/fluss-lake-lance/src/main/java/org/apache/fluss/lake/lance/tiering/LanceLakeCommitter.java b/fluss-lake/fluss-lake-lance/src/main/java/org/apache/fluss/lake/lance/tiering/LanceLakeCommitter.java index a9f8f87c99..13cb112db5 100644 --- a/fluss-lake/fluss-lake-lance/src/main/java/org/apache/fluss/lake/lance/tiering/LanceLakeCommitter.java +++ b/fluss-lake/fluss-lake-lance/src/main/java/org/apache/fluss/lake/lance/tiering/LanceLakeCommitter.java @@ -121,7 +121,6 @@ public CommittedLakeSnapshot getMissingLakeSnapshot(@Nullable Long latestLakeSna if (bucketOffset.getPartitionId() != null) { committedLakeSnapshot.addPartitionBucket( bucketOffset.getPartitionId(), - bucketOffset.getPartitionQualifiedName(), bucketOffset.getBucket(), bucketOffset.getLogOffset()); } else { 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 9571f651a5..642bd7afd1 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 @@ -57,7 +57,6 @@ import java.io.File; import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -131,7 +130,6 @@ void testTieringWriteTable(boolean isPartitioned) throws Exception { } } : Collections.singletonMap(null, null); - List partitionKeys = isPartitioned ? Arrays.asList("c3") : null; Map tableBucketOffsets = new HashMap<>(); // first, write data for (int bucket = 0; bucket < bucketNum; bucket++) { @@ -170,9 +168,7 @@ void testTieringWriteTable(boolean isPartitioned) throws Exception { committableSerializer.getVersion(), serialized); long snapshot = lakeCommitter.commit( - lanceCommittable, - toBucketOffsetsProperty( - tableBucketOffsets, partitionIdAndName, partitionKeys)); + lanceCommittable, toBucketOffsetsProperty(tableBucketOffsets)); // lance dataset version starts from 1 assertThat(snapshot).isEqualTo(2); } @@ -305,7 +301,7 @@ private Tuple2, List> genLogTableRecords( return Tuple2.of(logRecords, logRecords); } - private Schema createTable(LanceConfig config) throws Exception { + private Schema createTable(LanceConfig config) { List columns = new ArrayList<>(); columns.add(new Schema.Column("c1", DataTypes.INT())); columns.add(new Schema.Column("c2", DataTypes.STRING())); 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 8a7c2716b6..8351a377fe 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 @@ -153,7 +153,6 @@ public CommittedLakeSnapshot getMissingLakeSnapshot(@Nullable Long latestLakeSna if (bucketOffset.getPartitionId() != null) { committedLakeSnapshot.addPartitionBucket( bucketOffset.getPartitionId(), - bucketOffset.getPartitionQualifiedName(), bucketOffset.getBucket(), bucketOffset.getLogOffset()); } else { 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 18748c1bfa..d725038d61 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 @@ -335,7 +335,7 @@ void testTieringForAllTypes(boolean isPrimaryKeyTable) throws Exception { put( FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY, String.format( - "[{\"partition_id\":%d,\"bucket\":0,\"partition_name\":\"c1=true/c2=1/c3=2/c4=3/c5=4/c6=5_0/c7=6_0/c9=v1/c10=v2/c11=7633/c12=0102030405/c13=2025-10-16/c14=10-10-10_123/c15=2025-10-16-10-10-10_123/c16=2025-10-16-10-10-10_123\",\"offset\":1}]", + "[{\"partition_id\":%d,\"bucket\":0,\"offset\":1}]", partitionId)); } }; @@ -475,8 +475,7 @@ void testTieringForAlterTable() throws Exception { } private String getPartitionOffsetStr(Map partitionNameByIds) { - String raw = - "{\"partition_id\":%s,\"bucket\":0,\"partition_name\":\"date=%s\",\"offset\":3}"; + String raw = "{\"partition_id\":%s,\"bucket\":0,\"offset\":3}"; List partitionIds = new ArrayList<>(partitionNameByIds.keySet()); Collections.sort(partitionIds); List partitionOffsetStrs = new ArrayList<>(); 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 6d341c6840..afc43b86c1 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 @@ -195,11 +195,7 @@ void testTieringWriteTable(boolean isPrimaryKeyTable, boolean isPartitioned) thr committableSerializer.getVersion(), serialized); long snapshot = lakeCommitter.commit( - paimonCommittable, - toBucketOffsetsProperty( - tableBucketOffsets, - partitionIdAndName, - getPartitionKeys(tablePath))); + paimonCommittable, toBucketOffsetsProperty(tableBucketOffsets)); assertThat(snapshot).isEqualTo(1); } @@ -299,12 +295,7 @@ void testMultiPartitionTiering() throws Exception { createLakeCommitter(tablePath)) { PaimonCommittable committable = lakeCommitter.toCommittable(paimonWriteResults); long snapshot = - lakeCommitter.commit( - committable, - toBucketOffsetsProperty( - tableBucketOffsets, - partitionIdAndName, - getPartitionKeys(tablePath))); + lakeCommitter.commit(committable, toBucketOffsetsProperty(tableBucketOffsets)); assertThat(snapshot).isEqualTo(1); } @@ -376,12 +367,7 @@ void testThreePartitionTiering() throws Exception { createLakeCommitter(tablePath)) { PaimonCommittable committable = lakeCommitter.toCommittable(paimonWriteResults); snapshot = - lakeCommitter.commit( - committable, - toBucketOffsetsProperty( - tableBucketOffsets, - partitionIdAndName, - getPartitionKeys(tablePath))); + lakeCommitter.commit(committable, toBucketOffsetsProperty(tableBucketOffsets)); assertThat(snapshot).isEqualTo(1); } @@ -389,8 +375,8 @@ void testThreePartitionTiering() throws Exception { String offsetProperty = getSnapshotLogOffsetProperty(tablePath, snapshot); assertThat(offsetProperty) .isEqualTo( - "[{\"partition_id\":1,\"bucket\":0,\"partition_name\":\"region=us-east/year=2024/month=01\",\"offset\":2}," - + "{\"partition_id\":2,\"bucket\":0,\"partition_name\":\"region=eu-central/year=2023/month=12\",\"offset\":2}]"); + "[{\"partition_id\":1,\"bucket\":0,\"offset\":2}," + + "{\"partition_id\":2,\"bucket\":0,\"offset\":2}]"); // Verify data for each partition for (String partition : partitionIdAndName.values()) { @@ -798,10 +784,4 @@ private String getSnapshotLogOffsetProperty(TablePath tablePath, long snapshotId .properties() .get(FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY); } - - private List getPartitionKeys(TablePath tablePath) throws Exception { - Identifier identifier = toPaimon(tablePath); - FileStoreTable fileStoreTable = (FileStoreTable) paimonCatalog.getTable(identifier); - return fileStoreTable.partitionKeys(); - } } diff --git a/fluss-rpc/src/main/proto/FlussApi.proto b/fluss-rpc/src/main/proto/FlussApi.proto index e6cb83b04e..d2917734ce 100644 --- a/fluss-rpc/src/main/proto/FlussApi.proto +++ b/fluss-rpc/src/main/proto/FlussApi.proto @@ -462,9 +462,11 @@ message PbLakeTableSnapshotInfo { message PbLakeTableOffsetForBucket { optional int64 partition_id = 1; required int32 bucket_id = 2; - optional int64 log_start_offset = 3; + // 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; - optional string partition_name = 5; + // 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; } 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 82f3d5ec25..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 @@ -93,7 +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.LakeTableSnapshot; +import org.apache.fluss.server.zk.data.lake.LakeTableSnapshot; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -108,6 +108,7 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; import java.util.stream.Collectors; import static org.apache.fluss.rpc.util.CommonRpcMessageUtils.toAclFilter; @@ -144,13 +145,16 @@ public abstract class RpcServiceBase extends RpcGatewayService implements AdminR private long tokenLastUpdateTimeMs = 0; private ObtainedSecurityToken securityToken = null; + private final ExecutorService ioExecutor; + public RpcServiceBase( FileSystem remoteFileSystem, ServerType provider, ZooKeeperClient zkClient, MetadataManager metadataManager, @Nullable Authorizer authorizer, - DynamicConfigManager dynamicConfigManager) { + DynamicConfigManager dynamicConfigManager, + ExecutorService ioExecutor) { this.remoteFileSystem = remoteFileSystem; this.provider = provider; this.apiManager = new ApiManager(provider); @@ -158,6 +162,7 @@ public RpcServiceBase( this.metadataManager = metadataManager; this.authorizer = authorizer; this.dynamicConfigManager = dynamicConfigManager; + this.ioExecutor = ioExecutor; } @Override @@ -430,28 +435,33 @@ public CompletableFuture getLatestLakeSnapshot( TableInfo tableInfo = metadataManager.getTable(tablePath); // get table id long tableId = tableInfo.getTableId(); - - Optional optLakeTableSnapshot; - try { - optLakeTableSnapshot = zkClient.getLakeTableSnapshot(tableId); - } catch (Exception e) { - throw new FlussRuntimeException( - String.format( - "Failed to get lake table snapshot for table: %s, table id: %d", - tablePath, tableId), - e); - } - - if (!optLakeTableSnapshot.isPresent()) { - throw new LakeTableSnapshotNotExistException( - String.format( - "Lake table snapshot not exist for table: %s, table id: %d", - tablePath, tableId)); - } - - LakeTableSnapshot lakeTableSnapshot = optLakeTableSnapshot.get(); - return CompletableFuture.completedFuture( - makeGetLatestLakeSnapshotResponse(tableId, lakeTableSnapshot)); + CompletableFuture resultFuture = new CompletableFuture<>(); + ioExecutor.execute( + () -> { + Optional optLakeTableSnapshot; + try { + optLakeTableSnapshot = zkClient.getLakeTableSnapshot(tableId); + if (!optLakeTableSnapshot.isPresent()) { + resultFuture.completeExceptionally( + new LakeTableSnapshotNotExistException( + String.format( + "Lake table snapshot not exist for table: %s, table id: %d", + tablePath, tableId))); + } else { + LakeTableSnapshot lakeTableSnapshot = optLakeTableSnapshot.get(); + resultFuture.complete( + makeGetLatestLakeSnapshotResponse(tableId, lakeTableSnapshot)); + } + } catch (Exception e) { + resultFuture.completeExceptionally( + new FlussRuntimeException( + String.format( + "Failed to get lake table snapshot for table: %s, table id: %d", + tablePath, tableId), + e)); + } + }); + return resultFuture; } @Override 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 d1d5511214..4a6fd89f06 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 @@ -28,6 +28,7 @@ import org.apache.fluss.exception.IneligibleReplicaException; import org.apache.fluss.exception.InvalidCoordinatorException; import org.apache.fluss.exception.InvalidUpdateVersionException; +import org.apache.fluss.exception.TableNotExistException; import org.apache.fluss.exception.TabletServerNotAvailableException; import org.apache.fluss.exception.UnknownTableOrBucketException; import org.apache.fluss.metadata.PhysicalTablePath; @@ -62,6 +63,7 @@ import org.apache.fluss.server.coordinator.event.FencedCoordinatorEvent; import org.apache.fluss.server.coordinator.event.NewTabletServerEvent; import org.apache.fluss.server.coordinator.event.NotifyKvSnapshotOffsetEvent; +import org.apache.fluss.server.coordinator.event.NotifyLakeTableOffsetEvent; import org.apache.fluss.server.coordinator.event.NotifyLeaderAndIsrResponseReceivedEvent; import org.apache.fluss.server.coordinator.event.SchemaChangeEvent; import org.apache.fluss.server.coordinator.event.watcher.TableChangeWatcher; @@ -81,7 +83,6 @@ import org.apache.fluss.server.utils.ServerRpcMessageUtils; import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.server.zk.data.BucketAssignment; -import org.apache.fluss.server.zk.data.LakeTableSnapshot; import org.apache.fluss.server.zk.data.LeaderAndIsr; import org.apache.fluss.server.zk.data.PartitionAssignment; import org.apache.fluss.server.zk.data.RemoteLogManifestHandle; @@ -89,6 +90,8 @@ 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.LakeTableHelper; +import org.apache.fluss.server.zk.data.lake.LakeTableSnapshot; import org.apache.fluss.utils.types.Tuple2; import org.slf4j.Logger; @@ -144,6 +147,7 @@ public class CoordinatorEventProcessor implements EventProcessor { private final String internalListenerName; private final CompletedSnapshotStoreManager completedSnapshotStoreManager; + private final LakeTableHelper lakeTableHelper; public CoordinatorEventProcessor( ZooKeeperClient zooKeeperClient, @@ -204,6 +208,8 @@ public CoordinatorEventProcessor( this.coordinatorMetricGroup = coordinatorMetricGroup; this.internalListenerName = conf.getString(ConfigOptions.INTERNAL_LISTENER_NAME); this.ioExecutor = ioExecutor; + this.lakeTableHelper = + new LakeTableHelper(zooKeeperClient, conf.getString(ConfigOptions.REMOTE_DATA_DIR)); } public CoordinatorEventManager getCoordinatorEventManager() { @@ -529,6 +535,8 @@ public void process(CoordinatorEvent event) { commitKvSnapshotEvent, commitKvSnapshotEvent.getRespCallback()); } else if (event instanceof NotifyKvSnapshotOffsetEvent) { processNotifyKvSnapshotOffsetEvent((NotifyKvSnapshotOffsetEvent) event); + } else if (event instanceof NotifyLakeTableOffsetEvent) { + processNotifyLakeTableOffsetEvent((NotifyLakeTableOffsetEvent) event); } else if (event instanceof CommitRemoteLogManifestEvent) { CommitRemoteLogManifestEvent commitRemoteLogManifestEvent = (CommitRemoteLogManifestEvent) event; @@ -538,9 +546,8 @@ public void process(CoordinatorEvent event) { } else if (event instanceof CommitLakeTableSnapshotEvent) { CommitLakeTableSnapshotEvent commitLakeTableSnapshotEvent = (CommitLakeTableSnapshotEvent) event; - completeFromCallable( - commitLakeTableSnapshotEvent.getRespCallback(), - () -> tryProcessCommitLakeTableSnapshot(commitLakeTableSnapshotEvent)); + tryProcessCommitLakeTableSnapshot( + commitLakeTableSnapshotEvent, commitLakeTableSnapshotEvent.getRespCallback()); } else if (event instanceof ControlledShutdownEvent) { ControlledShutdownEvent controlledShutdownEvent = (ControlledShutdownEvent) event; completeFromCallable( @@ -1140,6 +1147,31 @@ private void processNotifyKvSnapshotOffsetEvent(NotifyKvSnapshotOffsetEvent even coordinatorContext.getCoordinatorEpoch()); } + private void processNotifyLakeTableOffsetEvent(NotifyLakeTableOffsetEvent event) { + Map lakeTableSnapshots = event.getLakeTableSnapshots(); + Map tableBucketMaxTieredTimestamps = + event.getTableBucketMaxTieredTimestamps(); + coordinatorRequestBatch.newBatch(); + for (Map.Entry lakeTableSnapshotEntry : + lakeTableSnapshots.entrySet()) { + LakeTableSnapshot lakeTableSnapshot = lakeTableSnapshotEntry.getValue(); + for (TableBucket tb : lakeTableSnapshot.getBucketLogEndOffset().keySet()) { + coordinatorContext + .getBucketLeaderAndIsr(tb) + .ifPresent( + leaderAndIsr -> + coordinatorRequestBatch + .addNotifyLakeTableOffsetRequestForTableServers( + coordinatorContext.getAssignment(tb), + tb, + lakeTableSnapshot, + tableBucketMaxTieredTimestamps.get(tb))); + } + } + coordinatorRequestBatch.sendNotifyLakeTableOffsetRequest( + coordinatorContext.getCoordinatorEpoch()); + } + private CommitRemoteLogManifestResponse tryProcessCommitRemoteLogManifest( CommitRemoteLogManifestEvent event) { CommitRemoteLogManifestData manifestData = event.getCommitRemoteLogManifestData(); @@ -1189,50 +1221,66 @@ private void processAccessContext(AccessContextEvent event) { } } - private CommitLakeTableSnapshotResponse tryProcessCommitLakeTableSnapshot( - CommitLakeTableSnapshotEvent commitLakeTableSnapshotEvent) { + private void tryProcessCommitLakeTableSnapshot( + CommitLakeTableSnapshotEvent commitLakeTableSnapshotEvent, + CompletableFuture callback) { + // commit the lake table snapshot asynchronously CommitLakeTableSnapshotData commitLakeTableSnapshotData = commitLakeTableSnapshotEvent.getCommitLakeTableSnapshotData(); - CommitLakeTableSnapshotResponse response = new CommitLakeTableSnapshotResponse(); Map lakeTableSnapshots = commitLakeTableSnapshotData.getLakeTableSnapshot(); + Map tablePathById = new HashMap<>(); for (Map.Entry lakeTableSnapshotEntry : lakeTableSnapshots.entrySet()) { Long tableId = lakeTableSnapshotEntry.getKey(); - - PbCommitLakeTableSnapshotRespForTable tableResp = response.addTableResp(); - tableResp.setTableId(tableId); - - try { - zooKeeperClient.upsertLakeTableSnapshot(tableId, lakeTableSnapshotEntry.getValue()); - } catch (Exception e) { - ApiError error = ApiError.fromThrowable(e); - tableResp.setError(error.error().code(), error.message()); + TablePath tablePath = coordinatorContext.getTablePathById(tableId); + if (tablePath != null) { + tablePathById.put(tableId, tablePath); } } - // send notify lakehouse data request to all replicas. - coordinatorRequestBatch.newBatch(); - for (Map.Entry lakeTableSnapshotEntry : - lakeTableSnapshots.entrySet()) { - LakeTableSnapshot lakeTableSnapshot = lakeTableSnapshotEntry.getValue(); - for (Map.Entry bucketLogEndOffsetEntry : - lakeTableSnapshot.getBucketLogEndOffset().entrySet()) { - TableBucket tb = bucketLogEndOffsetEntry.getKey(); - coordinatorContext - .getBucketLeaderAndIsr(bucketLogEndOffsetEntry.getKey()) - .ifPresent( - leaderAndIsr -> - coordinatorRequestBatch - .addNotifyLakeTableOffsetRequestForTableServers( - coordinatorContext.getAssignment(tb), - tb, - lakeTableSnapshot)); - } - } - coordinatorRequestBatch.sendNotifyLakeTableOffsetRequest( - coordinatorContext.getCoordinatorEpoch()); - return response; + ioExecutor.execute( + () -> { + try { + CommitLakeTableSnapshotResponse response = + new CommitLakeTableSnapshotResponse(); + for (Map.Entry lakeTableSnapshotEntry : + lakeTableSnapshots.entrySet()) { + Long tableId = lakeTableSnapshotEntry.getKey(); + + PbCommitLakeTableSnapshotRespForTable tableResp = + response.addTableResp(); + tableResp.setTableId(tableId); + + try { + TablePath tablePath = tablePathById.get(tableId); + if (tablePath == null) { + throw new TableNotExistException( + "Table " + + tableId + + " not found in coordinator context."); + } + + // this involves IO operation (ZK), so we do it in ioExecutor + lakeTableHelper.upsertLakeTable( + tableId, tablePath, lakeTableSnapshotEntry.getValue()); + } catch (Exception e) { + ApiError error = ApiError.fromThrowable(e); + tableResp.setError(error.error().code(), error.message()); + } + } + + // send notify lakehouse data request to all replicas via coordinator event + coordinatorEventManager.put( + new NotifyLakeTableOffsetEvent( + lakeTableSnapshots, + commitLakeTableSnapshotData + .getTableBucketsMaxTieredTimestamp())); + callback.complete(response); + } catch (Exception e) { + callback.completeExceptionally(e); + } + }); } private ControlledShutdownResponse tryProcessControlledShutdown( diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorRequestBatch.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorRequestBatch.java index 982995d21d..35527805c0 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorRequestBatch.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorRequestBatch.java @@ -43,8 +43,8 @@ import org.apache.fluss.server.metadata.BucketMetadata; import org.apache.fluss.server.metadata.PartitionMetadata; import org.apache.fluss.server.metadata.TableMetadata; -import org.apache.fluss.server.zk.data.LakeTableSnapshot; import org.apache.fluss.server.zk.data.LeaderAndIsr; +import org.apache.fluss.server.zk.data.lake.LakeTableSnapshot; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -378,7 +378,8 @@ public void addNotifyKvSnapshotOffsetRequestForTabletServers( public void addNotifyLakeTableOffsetRequestForTableServers( List tabletServers, TableBucket tableBucket, - LakeTableSnapshot lakeTableSnapshot) { + LakeTableSnapshot lakeTableSnapshot, + @Nullable Long maxTieredTimestamp) { tabletServers.stream() .filter(s -> s >= 0) .forEach( @@ -390,7 +391,7 @@ public void addNotifyLakeTableOffsetRequestForTableServers( notifyLakeTableOffsetReqForBucketMap.put( tableBucket, makeNotifyLakeTableOffsetForBucket( - tableBucket, lakeTableSnapshot)); + tableBucket, lakeTableSnapshot, maxTieredTimestamp)); }); } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java index e3a10a7309..83bba2c52f 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java @@ -188,6 +188,10 @@ protected void startServices() throws Exception { MetadataManager metadataManager = new MetadataManager(zkClient, conf, lakeCatalogDynamicLoader); + this.ioExecutor = + Executors.newFixedThreadPool( + conf.get(ConfigOptions.SERVER_IO_POOL_SIZE), + new ExecutorThreadFactory("coordinator-io")); this.coordinatorService = new CoordinatorService( conf, @@ -199,7 +203,8 @@ protected void startServices() throws Exception { authorizer, lakeCatalogDynamicLoader, lakeTableTieringManager, - dynamicConfigManager); + dynamicConfigManager, + ioExecutor); this.rpcServer = RpcServer.create( @@ -225,11 +230,6 @@ protected void startServices() throws Exception { new AutoPartitionManager(metadataCache, metadataManager, conf); autoPartitionManager.start(); - int ioExecutorPoolSize = conf.get(ConfigOptions.COORDINATOR_IO_POOL_SIZE); - this.ioExecutor = - Executors.newFixedThreadPool( - ioExecutorPoolSize, new ExecutorThreadFactory("coordinator-io")); - // start coordinator event processor after we register coordinator leader to zk // so that the event processor can get the coordinator leader node from zk during start // up. @@ -366,15 +366,6 @@ CompletableFuture stopServices() { exception = ExceptionUtils.firstOrSuppressed(t, exception); } - try { - if (ioExecutor != null) { - // shutdown io executor - ExecutorUtils.gracefulShutdown(5, TimeUnit.SECONDS, ioExecutor); - } - } catch (Throwable t) { - exception = ExceptionUtils.firstOrSuppressed(t, exception); - } - try { if (coordinatorEventProcessor != null) { coordinatorEventProcessor.shutdown(); @@ -407,6 +398,15 @@ CompletableFuture stopServices() { exception = ExceptionUtils.firstOrSuppressed(t, exception); } + try { + if (ioExecutor != null) { + // shutdown io executor + ExecutorUtils.gracefulShutdown(5, TimeUnit.SECONDS, ioExecutor); + } + } catch (Throwable t) { + exception = ExceptionUtils.firstOrSuppressed(t, exception); + } + try { if (coordinatorContext != null) { // then reset coordinatorContext @@ -519,11 +519,11 @@ private static void validateConfigs(Configuration conf) { ConfigOptions.KV_MAX_RETAINED_SNAPSHOTS.key())); } - if (conf.get(ConfigOptions.COORDINATOR_IO_POOL_SIZE) < 1) { + if (conf.get(ConfigOptions.SERVER_IO_POOL_SIZE) < 1) { throw new IllegalConfigurationException( String.format( "Invalid configuration for %s, it must be greater than or equal 1.", - ConfigOptions.COORDINATOR_IO_POOL_SIZE.key())); + ConfigOptions.SERVER_IO_POOL_SIZE.key())); } if (conf.get(ConfigOptions.REMOTE_DATA_DIR) == null) { 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 73a7603379..3d58112070 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 @@ -122,6 +122,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -166,14 +167,16 @@ public CoordinatorService( @Nullable Authorizer authorizer, LakeCatalogDynamicLoader lakeCatalogDynamicLoader, LakeTableTieringManager lakeTableTieringManager, - DynamicConfigManager dynamicConfigManager) { + DynamicConfigManager dynamicConfigManager, + ExecutorService ioExecutor) { super( remoteFileSystem, ServerType.COORDINATOR, zkClient, metadataManager, authorizer, - dynamicConfigManager); + dynamicConfigManager, + 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); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/RemoteStorageCleaner.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/RemoteStorageCleaner.java index 1ef36ed4d4..d206f07d2c 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/RemoteStorageCleaner.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/RemoteStorageCleaner.java @@ -17,6 +17,7 @@ package org.apache.fluss.server.coordinator; +import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; import org.apache.fluss.exception.FlussRuntimeException; import org.apache.fluss.fs.FileSystem; @@ -41,6 +42,8 @@ public class RemoteStorageCleaner { private final FsPath remoteLogDir; + private final String remoteDataDir; + private final FileSystem remoteFileSystem; private final ExecutorService ioExecutor; @@ -48,6 +51,7 @@ public class RemoteStorageCleaner { public RemoteStorageCleaner(Configuration configuration, ExecutorService ioExecutor) { this.remoteKvDir = FlussPaths.remoteKvDir(configuration); this.remoteLogDir = FlussPaths.remoteLogDir(configuration); + this.remoteDataDir = configuration.getString(ConfigOptions.REMOTE_DATA_DIR); this.ioExecutor = ioExecutor; try { this.remoteFileSystem = remoteKvDir.getFileSystem(); @@ -62,6 +66,13 @@ public void asyncDeleteTableRemoteDir(TablePath tablePath, boolean isKvTable, lo asyncDeleteDir(FlussPaths.remoteTableDir(remoteKvDir, tablePath, tableId)); } asyncDeleteDir(FlussPaths.remoteTableDir(remoteLogDir, tablePath, tableId)); + + // Always delete lake snapshot metadata directory, regardless of isLakeEnabled flag. + // This is because if a table was enabled datalake but turned off later, and then the table + // was deleted, we may leave the lake snapshot metadata files behind if we only delete when + // isLakeEnabled is true. By always deleting, we ensure cleanup of any existing metadata + // files. + asyncDeleteDir(FlussPaths.remoteLakeTableSnapshotDir(remoteDataDir, tablePath, tableId)); } public void asyncDeletePartitionRemoteDir( 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 new file mode 100644 index 0000000000..ba039f2b6b --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/NotifyLakeTableOffsetEvent.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.coordinator.event; + +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.server.zk.data.lake.LakeTableSnapshot; + +import java.util.Map; + +/** An event for notify lake table offset to local tablet servers. */ +public class NotifyLakeTableOffsetEvent implements CoordinatorEvent { + + private final Map lakeTableSnapshots; + private final Map tableBucketMaxTieredTimestamps; + + public NotifyLakeTableOffsetEvent( + Map lakeTableSnapshots, + Map tableBucketMaxTieredTimestamps) { + this.lakeTableSnapshots = lakeTableSnapshots; + this.tableBucketMaxTieredTimestamps = tableBucketMaxTieredTimestamps; + } + + public Map getLakeTableSnapshots() { + return lakeTableSnapshots; + } + + public Map getTableBucketMaxTieredTimestamps() { + return tableBucketMaxTieredTimestamps; + } +} 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 95be1b15bd..21ac1b340f 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 @@ -17,8 +17,9 @@ package org.apache.fluss.server.entity; +import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.rpc.messages.CommitLakeTableSnapshotRequest; -import org.apache.fluss.server.zk.data.LakeTableSnapshot; +import org.apache.fluss.server.zk.data.lake.LakeTableSnapshot; import java.util.Map; import java.util.Objects; @@ -27,34 +28,49 @@ public class CommitLakeTableSnapshotData { private final Map lakeTableSnapshots; + private final Map tableBucketsMaxTieredTimestamp; - public CommitLakeTableSnapshotData(Map lakeTableSnapshots) { + public CommitLakeTableSnapshotData( + Map lakeTableSnapshots, + Map tableBucketsMaxTieredTimestamp) { this.lakeTableSnapshots = lakeTableSnapshots; + this.tableBucketsMaxTieredTimestamp = tableBucketsMaxTieredTimestamp; } public Map getLakeTableSnapshot() { return lakeTableSnapshots; } + public Map getTableBucketsMaxTieredTimestamp() { + return tableBucketsMaxTieredTimestamp; + } + @Override public boolean equals(Object o) { if (this == o) { return true; } - if (!(o instanceof CommitLakeTableSnapshotData)) { + if (o == null || getClass() != o.getClass()) { return false; } CommitLakeTableSnapshotData that = (CommitLakeTableSnapshotData) o; - return Objects.equals(lakeTableSnapshots, that.lakeTableSnapshots); + return Objects.equals(lakeTableSnapshots, that.lakeTableSnapshots) + && Objects.equals( + tableBucketsMaxTieredTimestamp, that.tableBucketsMaxTieredTimestamp); } @Override public int hashCode() { - return Objects.hashCode(lakeTableSnapshots); + return Objects.hash(lakeTableSnapshots, tableBucketsMaxTieredTimestamp); } @Override public String toString() { - return "CommitLakeTableSnapshotData{" + "lakeTableInfos=" + lakeTableSnapshots + '}'; + return "CommitLakeTableSnapshotData{" + + "lakeTableSnapshots=" + + lakeTableSnapshots + + ", tableBucketsMaxTieredTimestamp=" + + tableBucketsMaxTieredTimestamp + + '}'; } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/KvSnapshotResource.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvSnapshotResource.java index 98d40991a8..0f8f748fb6 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/KvSnapshotResource.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvSnapshotResource.java @@ -46,11 +46,6 @@ public class KvSnapshotResource { /** Thread pool for async snapshot workers. */ private final ExecutorService asyncOperationsThreadPool; - /** - * The executor service that the snapshot data uploader/downloader to upload and download data. - */ - private final ExecutorService snapshotDataTransferService; - /** A uploader to upload snapshot data in the async phase of kv snapshot. */ private final KvSnapshotDataUploader kvSnapshotDataUploader; @@ -59,12 +54,10 @@ public class KvSnapshotResource { private KvSnapshotResource( ScheduledExecutorService kvSnapshotScheduler, - ExecutorService snapshotDataTransferService, KvSnapshotDataUploader kvSnapshotDataUploader, KvSnapshotDataDownloader kvSnapshotDataDownloader, ExecutorService asyncOperationsThreadPool) { this.kvSnapshotScheduler = kvSnapshotScheduler; - this.snapshotDataTransferService = snapshotDataTransferService; this.kvSnapshotDataUploader = kvSnapshotDataUploader; this.kvSnapshotDataDownloader = kvSnapshotDataDownloader; this.asyncOperationsThreadPool = asyncOperationsThreadPool; @@ -86,17 +79,12 @@ public KvSnapshotDataDownloader getKvSnapshotDataDownloader() { return kvSnapshotDataDownloader; } - public static KvSnapshotResource create(int serverId, Configuration conf) { - ExecutorService dataTransferThreadPool = - Executors.newFixedThreadPool( - conf.getInt(ConfigOptions.KV_SNAPSHOT_TRANSFER_THREAD_NUM), - new ExecutorThreadFactory("fluss-kv-snapshot-data-transfer")); - - KvSnapshotDataUploader kvSnapshotDataUploader = - new KvSnapshotDataUploader(dataTransferThreadPool); + public static KvSnapshotResource create( + int serverId, Configuration conf, ExecutorService ioExecutor) { + KvSnapshotDataUploader kvSnapshotDataUploader = new KvSnapshotDataUploader(ioExecutor); KvSnapshotDataDownloader kvSnapshotDataDownloader = - new KvSnapshotDataDownloader(dataTransferThreadPool); + new KvSnapshotDataDownloader(ioExecutor); ScheduledExecutorService kvSnapshotScheduler = Executors.newScheduledThreadPool( @@ -116,17 +104,12 @@ public static KvSnapshotResource create(int serverId, Configuration conf) { new ExecutorThreadFactory("fluss-kv-snapshot-async-operations")); return new KvSnapshotResource( kvSnapshotScheduler, - dataTransferThreadPool, kvSnapshotDataUploader, kvSnapshotDataDownloader, asyncOperationsThreadPool); } public void close() { - // both kvSnapshotDataUploader and kvSnapshotDataDownloader use snapshotDataTransferService - // so, we only need to close snapshotDataTransferService - snapshotDataTransferService.shutdownNow(); - // shutdown asyncOperationsThreadPool now asyncOperationsThreadPool.shutdownNow(); // close kvSnapshotScheduler, also stop any actively executing task immediately diff --git a/fluss-server/src/main/java/org/apache/fluss/server/log/remote/DefaultRemoteLogStorage.java b/fluss-server/src/main/java/org/apache/fluss/server/log/remote/DefaultRemoteLogStorage.java index f5f0e7ce3e..2dedc01da0 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/log/remote/DefaultRemoteLogStorage.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/log/remote/DefaultRemoteLogStorage.java @@ -30,7 +30,6 @@ import org.apache.fluss.utils.ExceptionUtils; import org.apache.fluss.utils.FlussPaths; import org.apache.fluss.utils.IOUtils; -import org.apache.fluss.utils.concurrent.ExecutorThreadFactory; import org.apache.fluss.utils.concurrent.FutureUtils; import org.apache.fluss.utils.function.ThrowingRunnable; @@ -52,7 +51,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import static org.apache.fluss.utils.FlussPaths.INDEX_FILE_SUFFIX; import static org.apache.fluss.utils.FlussPaths.TIME_INDEX_FILE_SUFFIX; @@ -69,17 +67,15 @@ public class DefaultRemoteLogStorage implements RemoteLogStorage { private final FsPath remoteLogDir; private final FileSystem fileSystem; - private final ExecutorService dataTransferThreadPool; + private final ExecutorService ioExecutor; private final int writeBufferSize; - public DefaultRemoteLogStorage(Configuration conf) throws IOException { + public DefaultRemoteLogStorage(Configuration conf, ExecutorService ioExecutor) + throws IOException { this.remoteLogDir = FlussPaths.remoteLogDir(conf); this.fileSystem = remoteLogDir.getFileSystem(); this.writeBufferSize = (int) conf.get(ConfigOptions.REMOTE_FS_WRITE_BUFFER_SIZE).getBytes(); - this.dataTransferThreadPool = - Executors.newFixedThreadPool( - conf.getInt(ConfigOptions.REMOTE_LOG_DATA_TRANSFER_THREAD_NUM), - new ExecutorThreadFactory("fluss-remote-log-data-transfer")); + this.ioExecutor = ioExecutor; } @Override @@ -281,7 +277,7 @@ private List> createUploadFutures( Files.newInputStream(localFile), rlsPath, localFile.getFileName().toString())), - dataTransferThreadPool); + ioExecutor); list.add(voidCompletableFuture); } return list; @@ -333,8 +329,6 @@ private FsPath createRemoteLogSegmentDir(RemoteLogSegment remoteLogSegment) thro @Override public void close() throws IOException { - if (dataTransferThreadPool != null) { - dataTransferThreadPool.shutdownNow(); - } + // do nothing } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/log/remote/RemoteLogManager.java b/fluss-server/src/main/java/org/apache/fluss/server/log/remote/RemoteLogManager.java index 85f77e2b36..9f30067473 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/log/remote/RemoteLogManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/log/remote/RemoteLogManager.java @@ -83,13 +83,14 @@ public RemoteLogManager( Configuration conf, ZooKeeperClient zkClient, CoordinatorGateway coordinatorGateway, - Clock clock) + Clock clock, + ExecutorService ioExecutor) throws IOException { this( conf, zkClient, coordinatorGateway, - new DefaultRemoteLogStorage(conf), + new DefaultRemoteLogStorage(conf, ioExecutor), Executors.newScheduledThreadPool( conf.getInt(ConfigOptions.REMOTE_LOG_MANAGER_THREAD_POOL_SIZE), new ExecutorThreadFactory(RLM_SCHEDULED_THREAD_PREFIX)), diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java index 5cb526a8e8..5eb19da65b 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java @@ -96,7 +96,7 @@ import org.apache.fluss.server.replica.fetcher.ReplicaFetcherManager; import org.apache.fluss.server.utils.FatalErrorHandler; import org.apache.fluss.server.zk.ZooKeeperClient; -import org.apache.fluss.server.zk.data.LakeTableSnapshot; +import org.apache.fluss.server.zk.data.lake.LakeTableSnapshot; import org.apache.fluss.utils.FileUtils; import org.apache.fluss.utils.FlussPaths; import org.apache.fluss.utils.MapUtils; @@ -120,6 +120,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; @@ -150,6 +151,7 @@ public class ReplicaManager { private final Map allReplicas = MapUtils.newConcurrentHashMap(); private final TabletServerMetadataCache metadataCache; + private final ExecutorService ioExecutor; private final ProjectionPushdownCache projectionsCache = new ProjectionPushdownCache(); private final Lock replicaStateChangeLock = new ReentrantLock(); @@ -201,7 +203,8 @@ public ReplicaManager( CompletedKvSnapshotCommitter completedKvSnapshotCommitter, FatalErrorHandler fatalErrorHandler, TabletServerMetricGroup serverMetricGroup, - Clock clock) + Clock clock, + ExecutorService ioExecutor) throws IOException { this( conf, @@ -216,8 +219,9 @@ public ReplicaManager( completedKvSnapshotCommitter, fatalErrorHandler, serverMetricGroup, - new RemoteLogManager(conf, zkClient, coordinatorGateway, clock), - clock); + new RemoteLogManager(conf, zkClient, coordinatorGateway, clock, ioExecutor), + clock, + ioExecutor); } @VisibleForTesting @@ -235,7 +239,8 @@ public ReplicaManager( FatalErrorHandler fatalErrorHandler, TabletServerMetricGroup serverMetricGroup, RemoteLogManager remoteLogManager, - Clock clock) + Clock clock, + ExecutorService ioExecutor) throws IOException { this.conf = conf; this.zkClient = zkClient; @@ -273,13 +278,14 @@ public ReplicaManager( this.fatalErrorHandler = fatalErrorHandler; // for kv snapshot - this.kvSnapshotResource = KvSnapshotResource.create(serverId, conf); + this.kvSnapshotResource = KvSnapshotResource.create(serverId, conf, ioExecutor); this.kvSnapshotContext = DefaultSnapshotContext.create( zkClient, completedKvSnapshotCommitter, kvSnapshotResource, conf); this.remoteLogManager = remoteLogManager; this.serverMetricGroup = serverMetricGroup; this.clock = clock; + this.ioExecutor = ioExecutor; registerMetrics(); } @@ -812,6 +818,7 @@ private void makeLeaders( } } + // NOTE: This method can be removed when fetchFromLake is deprecated private void updateWithLakeTableSnapshot(Replica replica) throws Exception { TableBucket tb = replica.getTableBucket(); Optional optLakeTableSnapshot = @@ -820,18 +827,9 @@ private void updateWithLakeTableSnapshot(Replica replica) throws Exception { LakeTableSnapshot lakeTableSnapshot = optLakeTableSnapshot.get(); long snapshotId = optLakeTableSnapshot.get().getSnapshotId(); replica.getLogTablet().updateLakeTableSnapshotId(snapshotId); - - lakeTableSnapshot - .getLogStartOffset(tb) - .ifPresent(replica.getLogTablet()::updateLakeLogStartOffset); - lakeTableSnapshot .getLogEndOffset(tb) .ifPresent(replica.getLogTablet()::updateLakeLogEndOffset); - - lakeTableSnapshot - .getMaxTimestamp(tb) - .ifPresent(replica.getLogTablet()::updateLakeMaxTimestamp); } } 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 56d1367413..88fa36ff46 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 @@ -53,8 +53,10 @@ import org.apache.fluss.server.zk.data.TabletServerRegistration; import org.apache.fluss.shaded.zookeeper3.org.apache.zookeeper.KeeperException; import org.apache.fluss.utils.ExceptionUtils; +import org.apache.fluss.utils.ExecutorUtils; import org.apache.fluss.utils.clock.Clock; import org.apache.fluss.utils.clock.SystemClock; +import org.apache.fluss.utils.concurrent.ExecutorThreadFactory; import org.apache.fluss.utils.concurrent.FlussScheduler; import org.apache.fluss.utils.concurrent.FutureUtils; import org.apache.fluss.utils.concurrent.Scheduler; @@ -70,6 +72,9 @@ import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.fluss.config.ConfigOptions.BACKGROUND_THREADS; @@ -159,6 +164,9 @@ public class TabletServer extends ServerBase { @GuardedBy("lock") private CoordinatorGateway coordinatorGateway; + @GuardedBy("lock") + private ExecutorService ioExecutor; + public TabletServer(Configuration conf) { this(conf, SystemClock.getInstance()); } @@ -234,6 +242,11 @@ protected void startServices() throws Exception { rpcClient, CoordinatorGateway.class); + this.ioExecutor = + Executors.newFixedThreadPool( + conf.get(ConfigOptions.SERVER_IO_POOL_SIZE), + new ExecutorThreadFactory("tablet-server-io")); + this.replicaManager = new ReplicaManager( conf, @@ -249,7 +262,8 @@ protected void startServices() throws Exception { rpcClient, metadataCache, interListenerName), this, tabletServerMetricGroup, - clock); + clock, + ioExecutor); replicaManager.startup(); this.tabletService = @@ -261,7 +275,8 @@ protected void startServices() throws Exception { metadataCache, metadataManager, authorizer, - dynamicConfigManager); + dynamicConfigManager, + ioExecutor); RequestsMetrics requestsMetrics = RequestsMetrics.createTabletServerRequestMetrics(tabletServerMetricGroup); @@ -430,6 +445,11 @@ CompletableFuture stopServices() { if (zkClient != null) { zkClient.close(); } + + if (ioExecutor != null) { + // shutdown io executor + ExecutorUtils.gracefulShutdown(5, TimeUnit.SECONDS, ioExecutor); + } } catch (Throwable t) { exception = ExceptionUtils.firstOrSuppressed(t, exception); } 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 b60bacec4a..837bc1b23a 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 @@ -87,6 +87,7 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; import java.util.function.BiFunction; import java.util.stream.Collectors; @@ -134,14 +135,16 @@ public TabletService( TabletServerMetadataCache metadataCache, MetadataManager metadataManager, @Nullable Authorizer authorizer, - DynamicConfigManager dynamicConfigManager) { + DynamicConfigManager dynamicConfigManager, + ExecutorService ioExecutor) { super( remoteFileSystem, ServerType.TABLET_SERVER, zkClient, metadataManager, authorizer, - dynamicConfigManager); + dynamicConfigManager, + ioExecutor); 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 8157711669..ed1051af94 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 @@ -164,8 +164,8 @@ import org.apache.fluss.server.metadata.ServerInfo; import org.apache.fluss.server.metadata.TableMetadata; import org.apache.fluss.server.zk.data.BucketSnapshot; -import org.apache.fluss.server.zk.data.LakeTableSnapshot; import org.apache.fluss.server.zk.data.LeaderAndIsr; +import org.apache.fluss.server.zk.data.lake.LakeTableSnapshot; import org.apache.fluss.utils.json.DataTypeJsonSerde; import org.apache.fluss.utils.json.JsonSerdeUtils; @@ -1545,13 +1545,11 @@ public static PbPartitionSpec makePbPartitionSpec(ResolvedPartitionSpec spec) { public static CommitLakeTableSnapshotData getCommitLakeTableSnapshotData( CommitLakeTableSnapshotRequest request) { Map lakeTableInfoByTableId = new HashMap<>(); + Map tableBucketsMaxTimestamp = new HashMap<>(); for (PbLakeTableSnapshotInfo pdLakeTableSnapshotInfo : request.getTablesReqsList()) { long tableId = pdLakeTableSnapshotInfo.getTableId(); long snapshotId = pdLakeTableSnapshotInfo.getSnapshotId(); - Map bucketLogStartOffset = new HashMap<>(); Map bucketLogEndOffset = new HashMap<>(); - Map bucketMaxTimestamp = new HashMap<>(); - Map partitionNameByPartitionId = new HashMap<>(); for (PbLakeTableOffsetForBucket lakeTableOffsetForBucket : pdLakeTableSnapshotInfo.getBucketsReqsList()) { @@ -1562,42 +1560,27 @@ public static CommitLakeTableSnapshotData getCommitLakeTableSnapshotData( int bucketId = lakeTableOffsetForBucket.getBucketId(); TableBucket tableBucket = new TableBucket(tableId, partitionId, bucketId); - Long logStartOffset = - lakeTableOffsetForBucket.hasLogStartOffset() - ? lakeTableOffsetForBucket.getLogStartOffset() - : null; Long logEndOffset = lakeTableOffsetForBucket.hasLogEndOffset() ? lakeTableOffsetForBucket.getLogEndOffset() : null; - Long logMaxTimestamp = - lakeTableOffsetForBucket.hasMaxTimestamp() - ? lakeTableOffsetForBucket.getMaxTimestamp() - : null; - bucketLogStartOffset.put(tableBucket, logStartOffset); bucketLogEndOffset.put(tableBucket, logEndOffset); - bucketMaxTimestamp.put(tableBucket, logMaxTimestamp); - if (lakeTableOffsetForBucket.hasPartitionName()) { - partitionNameByPartitionId.put( - partitionId, lakeTableOffsetForBucket.getPartitionName()); + if (lakeTableOffsetForBucket.hasMaxTimestamp()) { + tableBucketsMaxTimestamp.put( + tableBucket, lakeTableOffsetForBucket.getMaxTimestamp()); } } lakeTableInfoByTableId.put( - tableId, - new LakeTableSnapshot( - snapshotId, - tableId, - bucketLogStartOffset, - bucketLogEndOffset, - bucketMaxTimestamp, - partitionNameByPartitionId)); + tableId, new LakeTableSnapshot(snapshotId, bucketLogEndOffset)); } - return new CommitLakeTableSnapshotData(lakeTableInfoByTableId); + return new CommitLakeTableSnapshotData(lakeTableInfoByTableId, tableBucketsMaxTimestamp); } public static PbNotifyLakeTableOffsetReqForBucket makeNotifyLakeTableOffsetForBucket( - TableBucket tableBucket, LakeTableSnapshot lakeTableSnapshot) { + TableBucket tableBucket, + LakeTableSnapshot lakeTableSnapshot, + @Nullable Long maxTimestamp) { PbNotifyLakeTableOffsetReqForBucket reqForBucket = new PbNotifyLakeTableOffsetReqForBucket(); if (tableBucket.getPartitionId() != null) { @@ -1608,11 +1591,11 @@ public static PbNotifyLakeTableOffsetReqForBucket makeNotifyLakeTableOffsetForBu .setBucketId(tableBucket.getBucket()) .setSnapshotId(lakeTableSnapshot.getSnapshotId()); - lakeTableSnapshot.getLogStartOffset(tableBucket).ifPresent(reqForBucket::setLogStartOffset); - lakeTableSnapshot.getLogEndOffset(tableBucket).ifPresent(reqForBucket::setLogEndOffset); - lakeTableSnapshot.getMaxTimestamp(tableBucket).ifPresent(reqForBucket::setMaxTimestamp); + if (maxTimestamp != null) { + reqForBucket.setMaxTimestamp(maxTimestamp); + } return reqForBucket; } @@ -1662,13 +1645,6 @@ public static GetLatestLakeSnapshotResponse makeGetLatestLakeSnapshotResponse( .setLogOffset(logEndLogOffsetEntry.getValue()); if (tableBucket.getPartitionId() != null) { pbLakeSnapshotForBucket.setPartitionId(tableBucket.getPartitionId()); - String partitionName = - lakeTableSnapshot - .getPartitionNameIdByPartitionId() - .get(tableBucket.getPartitionId()); - if (partitionName != null) { - pbLakeSnapshotForBucket.setPartitionName(partitionName); - } } } 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 75ca2b7b4b..dfc2c004d0 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 @@ -40,7 +40,6 @@ import org.apache.fluss.server.zk.data.BucketSnapshot; import org.apache.fluss.server.zk.data.CoordinatorAddress; import org.apache.fluss.server.zk.data.DatabaseRegistration; -import org.apache.fluss.server.zk.data.LakeTableSnapshot; import org.apache.fluss.server.zk.data.LeaderAndIsr; import org.apache.fluss.server.zk.data.PartitionAssignment; import org.apache.fluss.server.zk.data.RemoteLogManifestHandle; @@ -74,6 +73,8 @@ import org.apache.fluss.server.zk.data.ZkData.TableZNode; import org.apache.fluss.server.zk.data.ZkData.TablesZNode; import org.apache.fluss.server.zk.data.ZkData.WriterIdZNode; +import org.apache.fluss.server.zk.data.lake.LakeTable; +import org.apache.fluss.server.zk.data.lake.LakeTableSnapshot; import org.apache.fluss.shaded.curator5.org.apache.curator.framework.CuratorFramework; import org.apache.fluss.shaded.curator5.org.apache.curator.framework.api.BackgroundCallback; import org.apache.fluss.shaded.curator5.org.apache.curator.framework.api.CuratorEvent; @@ -133,6 +134,7 @@ public class ZooKeeperClient implements AutoCloseable { private final ZkSequenceIDCounter writerIdCounter; private final Semaphore inFlightRequests; + private final Configuration configuration; public ZooKeeperClient( CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper, @@ -147,6 +149,7 @@ public ZooKeeperClient( int maxInFlightRequests = configuration.getInt(ConfigOptions.ZOOKEEPER_MAX_INFLIGHT_REQUESTS); this.inFlightRequests = new Semaphore(maxInFlightRequests); + this.configuration = configuration; } public Optional getOrEmpty(String path) throws Exception { @@ -1023,55 +1026,45 @@ public Optional getRemoteLogManifestHandle(TableBucket return getOrEmpty(path).map(BucketRemoteLogsZNode::decode); } - public void upsertLakeTableSnapshot(long tableId, LakeTableSnapshot lakeTableSnapshot) + /** Upsert the {@link LakeTable} to Zk Node. */ + public void upsertLakeTable(long tableId, LakeTable lakeTable, boolean isUpdate) throws Exception { - String path = LakeTableZNode.path(tableId); - Optional optLakeTableSnapshot = getLakeTableSnapshot(tableId); - if (optLakeTableSnapshot.isPresent()) { - // we need to merge current lake table snapshot with previous - // since the current lake table snapshot request won't carry all - // the bucket for the table. It will only carry the bucket that is written - // after the previous commit - LakeTableSnapshot previous = optLakeTableSnapshot.get(); - - // merge log startup offset, current will override the previous - Map bucketLogStartOffset = - new HashMap<>(previous.getBucketLogStartOffset()); - bucketLogStartOffset.putAll(lakeTableSnapshot.getBucketLogStartOffset()); - - // merge log end offsets, current will override the previous - Map bucketLogEndOffset = - new HashMap<>(previous.getBucketLogEndOffset()); - bucketLogEndOffset.putAll(lakeTableSnapshot.getBucketLogEndOffset()); - - // merge max timestamp, current will override the previous - Map bucketMaxTimestamp = - new HashMap<>(previous.getBucketMaxTimestamp()); - bucketMaxTimestamp.putAll(lakeTableSnapshot.getBucketMaxTimestamp()); - - Map partitionNameById = - new HashMap<>(previous.getPartitionNameIdByPartitionId()); - partitionNameById.putAll(lakeTableSnapshot.getPartitionNameIdByPartitionId()); - - lakeTableSnapshot = - new LakeTableSnapshot( - lakeTableSnapshot.getSnapshotId(), - lakeTableSnapshot.getTableId(), - bucketLogStartOffset, - bucketLogEndOffset, - bucketMaxTimestamp, - partitionNameById); - zkClient.setData().forPath(path, LakeTableZNode.encode(lakeTableSnapshot)); + byte[] zkData = LakeTableZNode.encode(lakeTable); + String zkPath = LakeTableZNode.path(tableId); + if (isUpdate) { + zkClient.setData().forPath(zkPath, zkData); } else { - zkClient.create() - .creatingParentsIfNeeded() - .forPath(path, LakeTableZNode.encode(lakeTableSnapshot)); + zkClient.create().creatingParentsIfNeeded().forPath(zkPath, zkData); } } + /** + * Gets the {@link LakeTable} for the given table ID. + * + * @param tableId the table ID + * @return an Optional containing the LakeTable if it exists, empty otherwise + * @throws Exception if the operation fails + */ + public Optional getLakeTable(long tableId) throws Exception { + String zkPath = LakeTableZNode.path(tableId); + return getOrEmpty(zkPath).map(LakeTableZNode::decode); + } + + /** + * Gets the {@link LakeTableSnapshot} for the given table ID. + * + * @param tableId the table ID + * @return an Optional containing the LakeTableSnapshot if the table exists, empty otherwise + * @throws Exception if the operation fails + */ public Optional getLakeTableSnapshot(long tableId) throws Exception { - String path = LakeTableZNode.path(tableId); - return getOrEmpty(path).map(LakeTableZNode::decode); + Optional optLakeTable = getLakeTable(tableId); + if (optLakeTable.isPresent()) { + // always get the latest snapshot + return Optional.of(optLakeTable.get().getLatestTableSnapshot()); + } else { + return Optional.empty(); + } } /** diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/LakeTableSnapshot.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/LakeTableSnapshot.java deleted file mode 100644 index 9d65e3bf0f..0000000000 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/LakeTableSnapshot.java +++ /dev/null @@ -1,146 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.fluss.server.zk.data; - -import org.apache.fluss.metadata.TableBucket; - -import java.util.Map; -import java.util.Objects; -import java.util.Optional; - -/** The snapshot info for a table. */ -public class LakeTableSnapshot { - - // the last committed snapshot id in lake - private final long snapshotId; - private final long tableId; - - // the log offset of the bucket - - // mapping from bucket id to log start/end offset or max timestamp, - // will be null if log offset is unknown such as reading the snapshot of primary key table - private final Map bucketLogStartOffset; - private final Map bucketLogEndOffset; - private final Map bucketMaxTimestamp; - - // mapping from partition id to partition name, will be empty if the table is not partitioned - // table - private final Map partitionNameIdByPartitionId; - - public LakeTableSnapshot( - long snapshotId, - long tableId, - Map bucketLogStartOffset, - Map bucketLogEndOffset, - Map bucketMaxTimestamp, - Map partitionNameIdByPartitionId) { - this.snapshotId = snapshotId; - this.tableId = tableId; - this.bucketLogStartOffset = bucketLogStartOffset; - this.bucketLogEndOffset = bucketLogEndOffset; - this.bucketMaxTimestamp = bucketMaxTimestamp; - this.partitionNameIdByPartitionId = partitionNameIdByPartitionId; - } - - public long getSnapshotId() { - return snapshotId; - } - - public long getTableId() { - return tableId; - } - - public void putLogStartOffset(TableBucket tableBucket, Long logStartOffset) { - bucketLogStartOffset.put(tableBucket, logStartOffset); - } - - public void putLogEndOffset(TableBucket tableBucket, Long logEndOffset) { - bucketLogEndOffset.put(tableBucket, logEndOffset); - } - - public Optional getLogStartOffset(TableBucket tableBucket) { - return Optional.ofNullable(bucketLogStartOffset.get(tableBucket)); - } - - public Optional getLogEndOffset(TableBucket tableBucket) { - return Optional.ofNullable(bucketLogEndOffset.get(tableBucket)); - } - - public Optional getMaxTimestamp(TableBucket tableBucket) { - return Optional.ofNullable(bucketMaxTimestamp.get(tableBucket)); - } - - public Map getBucketLogEndOffset() { - return bucketLogEndOffset; - } - - public Map getBucketLogStartOffset() { - return bucketLogStartOffset; - } - - public Map getBucketMaxTimestamp() { - return bucketMaxTimestamp; - } - - public Map getPartitionNameIdByPartitionId() { - return partitionNameIdByPartitionId; - } - - @Override - public boolean equals(Object o) { - if (o == null || getClass() != o.getClass()) { - return false; - } - LakeTableSnapshot that = (LakeTableSnapshot) o; - return snapshotId == that.snapshotId - && tableId == that.tableId - && Objects.equals(bucketLogStartOffset, that.bucketLogStartOffset) - && Objects.equals(bucketLogEndOffset, that.bucketLogEndOffset) - && Objects.equals(bucketMaxTimestamp, that.bucketMaxTimestamp) - && Objects.equals(partitionNameIdByPartitionId, that.partitionNameIdByPartitionId); - } - - @Override - public int hashCode() { - return Objects.hash( - snapshotId, - tableId, - bucketLogStartOffset, - bucketLogEndOffset, - bucketMaxTimestamp, - partitionNameIdByPartitionId); - } - - @Override - public String toString() { - return "LakeTableSnapshot{" - + "snapshotId=" - + snapshotId - + ", tableId=" - + tableId - + ", bucketLogStartOffset=" - + bucketLogStartOffset - + ", bucketLogEndOffset=" - + bucketLogEndOffset - + ", bucketMaxTimestamp=" - + bucketMaxTimestamp - + ", partitionNameIdByPartitionId=" - + partitionNameIdByPartitionId - + '}'; - } -} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/LakeTableSnapshotJsonSerde.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/LakeTableSnapshotJsonSerde.java deleted file mode 100644 index 539da5e029..0000000000 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/LakeTableSnapshotJsonSerde.java +++ /dev/null @@ -1,149 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.fluss.server.zk.data; - -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.JsonSerializer; - -import java.io.IOException; -import java.util.HashMap; -import java.util.Iterator; -import java.util.Map; - -/** Json serializer and deserializer for {@link LakeTableSnapshot}. */ -public class LakeTableSnapshotJsonSerde - implements JsonSerializer, JsonDeserializer { - - public static final LakeTableSnapshotJsonSerde INSTANCE = new LakeTableSnapshotJsonSerde(); - - 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_ID = "bucket_id"; - private static final String LOG_START_OFFSET = "log_start_offset"; - private static final String LOG_END_OFFSET = "log_end_offset"; - private static final String MAX_TIMESTAMP = "max_timestamp"; - private static final String PARTITION_NAME = "partition_name"; - - private static final int VERSION = 1; - - @Override - public void serialize(LakeTableSnapshot lakeTableSnapshot, JsonGenerator generator) - throws IOException { - generator.writeStartObject(); - generator.writeNumberField(VERSION_KEY, VERSION); - generator.writeNumberField(SNAPSHOT_ID, lakeTableSnapshot.getSnapshotId()); - generator.writeNumberField(TABLE_ID, lakeTableSnapshot.getTableId()); - - generator.writeArrayFieldStart(BUCKETS); - for (TableBucket tableBucket : lakeTableSnapshot.getBucketLogEndOffset().keySet()) { - generator.writeStartObject(); - - if (tableBucket.getPartitionId() != null) { - generator.writeNumberField(PARTITION_ID, tableBucket.getPartitionId()); - // have partition name - String partitionName = - lakeTableSnapshot - .getPartitionNameIdByPartitionId() - .get(tableBucket.getPartitionId()); - if (partitionName != null) { - generator.writeStringField(PARTITION_NAME, partitionName); - } - } - generator.writeNumberField(BUCKET_ID, tableBucket.getBucket()); - - if (lakeTableSnapshot.getLogStartOffset(tableBucket).isPresent()) { - generator.writeNumberField( - LOG_START_OFFSET, lakeTableSnapshot.getLogStartOffset(tableBucket).get()); - } - - if (lakeTableSnapshot.getLogEndOffset(tableBucket).isPresent()) { - generator.writeNumberField( - LOG_END_OFFSET, lakeTableSnapshot.getLogEndOffset(tableBucket).get()); - } - - if (lakeTableSnapshot.getMaxTimestamp(tableBucket).isPresent()) { - generator.writeNumberField( - MAX_TIMESTAMP, lakeTableSnapshot.getMaxTimestamp(tableBucket).get()); - } - - generator.writeEndObject(); - } - generator.writeEndArray(); - - generator.writeEndObject(); - } - - @Override - public LakeTableSnapshot deserialize(JsonNode node) { - if (node.get(VERSION_KEY).asInt() != VERSION) { - throw new IllegalArgumentException( - "Unsupported version: " + node.get(VERSION_KEY).asInt()); - } - long snapshotId = node.get(SNAPSHOT_ID).asLong(); - long tableId = node.get(TABLE_ID).asLong(); - Iterator buckets = node.get(BUCKETS).elements(); - Map bucketLogStartOffset = new HashMap<>(); - Map bucketLogEndOffset = new HashMap<>(); - Map bucketMaxTimestamp = new HashMap<>(); - Map partitionNameIdByPartitionId = new HashMap<>(); - while (buckets.hasNext()) { - JsonNode bucket = buckets.next(); - TableBucket tableBucket; - Long partitionId = - bucket.get(PARTITION_ID) != null ? bucket.get(PARTITION_ID).asLong() : null; - tableBucket = new TableBucket(tableId, partitionId, bucket.get(BUCKET_ID).asInt()); - - if (bucket.get(LOG_START_OFFSET) != null) { - bucketLogStartOffset.put(tableBucket, bucket.get(LOG_START_OFFSET).asLong()); - } else { - bucketLogStartOffset.put(tableBucket, null); - } - - if (bucket.get(LOG_END_OFFSET) != null) { - bucketLogEndOffset.put(tableBucket, bucket.get(LOG_END_OFFSET).asLong()); - } else { - bucketLogEndOffset.put(tableBucket, null); - } - - if (bucket.get(MAX_TIMESTAMP) != null) { - bucketMaxTimestamp.put(tableBucket, bucket.get(MAX_TIMESTAMP).asLong()); - } else { - bucketMaxTimestamp.put(tableBucket, null); - } - - if (partitionId != null && bucket.get(PARTITION_NAME) != null) { - partitionNameIdByPartitionId.put( - tableBucket.getPartitionId(), bucket.get(PARTITION_NAME).asText()); - } - } - return new LakeTableSnapshot( - snapshotId, - tableId, - bucketLogStartOffset, - bucketLogEndOffset, - bucketMaxTimestamp, - partitionNameIdByPartitionId); - } -} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java index 702400184d..bb33e17ff5 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java @@ -24,6 +24,8 @@ import org.apache.fluss.metadata.TablePath; import org.apache.fluss.security.acl.Resource; import org.apache.fluss.security.acl.ResourceType; +import org.apache.fluss.server.zk.data.lake.LakeTable; +import org.apache.fluss.server.zk.data.lake.LakeTableJsonSerde; import org.apache.fluss.utils.json.JsonSerdeUtils; import org.apache.fluss.utils.types.Tuple2; @@ -564,22 +566,50 @@ public static RemoteLogManifestHandle decode(byte[] json) { } /** - * The znode for the info of lake data for a table. The znode path is: + * The znode for lake table snapshot information. The znode path is: * *

/tabletservers/tables/[tableId]/laketable + * + *

This znode stores {@link LakeTable} in: + * + *

    + *
  • Version 1 (legacy): Full snapshot data stored directly in ZK + *
  • Version 2 (current): A list of snapshot metadata, with metadata file path stored in ZK, + * actual data in remote file + *
*/ public static final class LakeTableZNode { + /** + * Returns the ZK path for the lake table znode of the given table. + * + * @param tableId the table ID + * @return the ZK path + */ public static String path(long tableId) { return TableIdZNode.path(tableId) + "/laketable"; } - public static byte[] encode(LakeTableSnapshot lakeTableSnapshot) { - return JsonSerdeUtils.writeValueAsBytes( - lakeTableSnapshot, LakeTableSnapshotJsonSerde.INSTANCE); + /** + * Encodes a LakeTable to JSON bytes for storage in ZK. + * + * @param lakeTable the LakeTable to encode + * @return the encoded bytes + */ + public static byte[] encode(LakeTable lakeTable) { + return JsonSerdeUtils.writeValueAsBytes(lakeTable, LakeTableJsonSerde.INSTANCE); } - public static LakeTableSnapshot decode(byte[] json) { - return JsonSerdeUtils.readValue(json, LakeTableSnapshotJsonSerde.INSTANCE); + /** + * Decodes JSON bytes from ZK to a LakeTable. + * + *

This method handles both version 1 (legacy) and version 2 (current) formats + * automatically through {@link LakeTableJsonSerde}. + * + * @param json the JSON bytes from ZK + * @return the decoded LakeTable + */ + public static LakeTable decode(byte[] json) { + return JsonSerdeUtils.readValue(json, LakeTableJsonSerde.INSTANCE); } } 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 new file mode 100644 index 0000000000..5df57acd57 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTable.java @@ -0,0 +1,230 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.zk.data.lake; + +import org.apache.fluss.fs.FSDataInputStream; +import org.apache.fluss.fs.FileSystem; +import org.apache.fluss.fs.FsPath; +import org.apache.fluss.server.zk.data.ZkData; +import org.apache.fluss.utils.IOUtils; + +import javax.annotation.Nullable; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Objects; + +import static org.apache.fluss.metrics.registry.MetricRegistry.LOG; +import static org.apache.fluss.utils.Preconditions.checkNotNull; + +/** + * Represents lake table snapshot information stored in {@link ZkData.LakeTableZNode}. + * + *

This class supports two storage formats: + * + *

    + *
  • Version 1 (legacy): Contains the full {@link LakeTableSnapshot} data directly + *
  • Version 2 (current): Contains a list of lake snapshot, recording the metadata file path for + * different lake snapshots, with actual metadata storing in file to reduce zk pressure + *
+ * + * @see LakeTableJsonSerde for JSON serialization and deserialization + */ +public class LakeTable { + + // Version 2 (current): + // a list of lake snapshot metadata, record the metadata for different lake snapshots + @Nullable private final List lakeSnapshotMetadatas; + + // Version 1 (legacy): the full lake table snapshot info stored in ZK, will be null in version2 + @Nullable private final LakeTableSnapshot lakeTableSnapshot; + + /** + * Creates a LakeTable from a LakeTableSnapshot (version 1 format). + * + * @param lakeTableSnapshot the snapshot data + */ + public LakeTable(LakeTableSnapshot lakeTableSnapshot) { + this(lakeTableSnapshot, null); + } + + /** + * Creates a LakeTable with a lake snapshot metadata (version 2 format). + * + * @param lakeSnapshotMetadata the metadata containing the file path to the snapshot data + */ + public LakeTable(LakeSnapshotMetadata lakeSnapshotMetadata) { + this(null, Collections.singletonList(lakeSnapshotMetadata)); + } + + /** + * Creates a LakeTable with a list of lake snapshot metadata (version 2 format). + * + * @param lakeSnapshotMetadatas the list of lake snapshot metadata + */ + public LakeTable(List lakeSnapshotMetadatas) { + this(null, lakeSnapshotMetadatas); + } + + private LakeTable( + @Nullable LakeTableSnapshot lakeTableSnapshot, + List lakeSnapshotMetadatas) { + this.lakeTableSnapshot = lakeTableSnapshot; + this.lakeSnapshotMetadatas = lakeSnapshotMetadatas; + } + + @Nullable + public LakeSnapshotMetadata getLatestLakeSnapshotMetadata() { + if (lakeSnapshotMetadatas != null && !lakeSnapshotMetadatas.isEmpty()) { + return lakeSnapshotMetadatas.get(0); + } + return null; + } + + @Nullable + public List getLakeSnapshotMetadatas() { + return lakeSnapshotMetadatas; + } + + /** + * Get 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 Exception { + if (lakeTableSnapshot != null) { + return lakeTableSnapshot; + } + FsPath tieredOffsetsFilePath = + checkNotNull(getLatestLakeSnapshotMetadata()).tieredOffsetsFilePath; + FSDataInputStream inputStream = + tieredOffsetsFilePath.getFileSystem().open(tieredOffsetsFilePath); + try (ByteArrayOutputStream outputStream = new ByteArrayOutputStream()) { + IOUtils.copyBytes(inputStream, outputStream, true); + return LakeTableSnapshotJsonSerde.fromJson(outputStream.toByteArray()); + } + } + + /** The lake snapshot metadata entry stored in zk lake table. */ + public static class LakeSnapshotMetadata { + private final long snapshotId; + + // the file path to file storing the tiered offsets, + // it points a file storing LakeTableSnapshot which includes tiered offsets + private final FsPath tieredOffsetsFilePath; + + // the file path to file storing the readable offsets, + // will be null if we don't now the readable offsets for this snapshot + @Nullable private final FsPath readableOffsetsFilePath; + + public LakeSnapshotMetadata( + long snapshotId, + FsPath tieredOffsetsFilePath, + @Nullable FsPath readableOffsetsFilePath) { + this.snapshotId = snapshotId; + this.tieredOffsetsFilePath = tieredOffsetsFilePath; + this.readableOffsetsFilePath = readableOffsetsFilePath; + } + + public long getSnapshotId() { + return snapshotId; + } + + public FsPath getTieredOffsetsFilePath() { + return tieredOffsetsFilePath; + } + + public FsPath getReadableOffsetsFilePath() { + return readableOffsetsFilePath; + } + + public void discard() { + if (tieredOffsetsFilePath != null) { + delete(tieredOffsetsFilePath); + } + if (readableOffsetsFilePath != null + && readableOffsetsFilePath != tieredOffsetsFilePath) { + delete(readableOffsetsFilePath); + } + } + + private void delete(FsPath fsPath) { + try { + FileSystem fileSystem = fsPath.getFileSystem(); + if (fileSystem.exists(fsPath)) { + fileSystem.delete(fsPath, false); + } + } catch (IOException e) { + LOG.warn("Error deleting filePath at {}", fsPath, e); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof LakeSnapshotMetadata)) { + return false; + } + LakeSnapshotMetadata that = (LakeSnapshotMetadata) o; + return snapshotId == that.snapshotId + && Objects.equals(tieredOffsetsFilePath, that.tieredOffsetsFilePath) + && Objects.equals(readableOffsetsFilePath, that.readableOffsetsFilePath); + } + + @Override + public int hashCode() { + return Objects.hash(snapshotId, tieredOffsetsFilePath, readableOffsetsFilePath); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof LakeTable)) { + return false; + } + LakeTable lakeTable = (LakeTable) o; + return Objects.equals(lakeSnapshotMetadatas, lakeTable.lakeSnapshotMetadatas) + && Objects.equals(lakeTableSnapshot, lakeTable.lakeTableSnapshot); + } + + @Override + public int hashCode() { + return Objects.hash(lakeSnapshotMetadatas, lakeTableSnapshot); + } + + @Override + public String toString() { + return "LakeTable{" + + "lakeSnapshotMetadatas=" + + lakeSnapshotMetadatas + + ", lakeTableSnapshot=" + + 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 new file mode 100644 index 0000000000..8031b9ddeb --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableHelper.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.zk.data.lake; + +import org.apache.fluss.fs.FSDataOutputStream; +import org.apache.fluss.fs.FileSystem; +import org.apache.fluss.fs.FsPath; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.server.zk.ZooKeeperClient; +import org.apache.fluss.utils.FlussPaths; + +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +import static org.apache.fluss.metrics.registry.MetricRegistry.LOG; + +/** The helper to handle {@link LakeTable}. */ +public class LakeTableHelper { + + private final ZooKeeperClient zkClient; + private final String remoteDataDir; + + public LakeTableHelper(ZooKeeperClient zkClient, String remoteDataDir) { + this.zkClient = zkClient; + this.remoteDataDir = remoteDataDir; + } + + /** + * 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(); + } + } + } + + private LakeTableSnapshot mergeLakeTable( + LakeTableSnapshot previousLakeTableSnapshot, LakeTableSnapshot newLakeTableSnapshot) { + // Merge current snapshot with previous one since the current snapshot 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); + } + + private 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); + // check whether the parent directory exists, if not, create the directory + FileSystem fileSystem = remoteLakeTableSnapshotManifestPath.getFileSystem(); + if (!fileSystem.exists(remoteLakeTableSnapshotManifestPath.getParent())) { + fileSystem.mkdirs(remoteLakeTableSnapshotManifestPath.getParent()); + } + // serialize table snapshot to json bytes, and write to file + byte[] jsonBytes = LakeTableSnapshotJsonSerde.toJson(lakeTableSnapshot); + try (FSDataOutputStream outputStream = + fileSystem.create( + remoteLakeTableSnapshotManifestPath, FileSystem.WriteMode.OVERWRITE)) { + outputStream.write(jsonBytes); + } + return remoteLakeTableSnapshotManifestPath; + } +} 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 new file mode 100644 index 0000000000..f272361bc6 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerde.java @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.zk.data.lake; + +import org.apache.fluss.fs.FsPath; +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.JsonSerializer; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +import static org.apache.fluss.utils.Preconditions.checkNotNull; + +/** + * Json serializer and deserializer for {@link LakeTable}. + * + *

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

    + *
  • 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}. + *
  • 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. + *
+ */ +public class LakeTableJsonSerde implements JsonSerializer, JsonDeserializer { + + public static final LakeTableJsonSerde INSTANCE = new LakeTableJsonSerde(); + + private static final String VERSION_KEY = "version"; + private static final String LAKE_SNAPSHOTS = "lake_snapshots"; + + private static final String SNAPSHOT_ID_KEY = "snapshot_id"; + private static final String TIERED_OFFSETS_KEY = "tiered_offsets"; + private static final String READABLE_OFFSETS_KEY = "readable_offsets"; + + private static final int VERSION_1 = 1; + private static final int VERSION_2 = 2; + private static final int CURRENT_VERSION = VERSION_2; + + @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())) { + generator.writeStartObject(); + + generator.writeNumberField(SNAPSHOT_ID_KEY, lakeSnapshotMetadata.getSnapshotId()); + generator.writeStringField( + TIERED_OFFSETS_KEY, lakeSnapshotMetadata.getTieredOffsetsFilePath().toString()); + if (lakeSnapshotMetadata.getReadableOffsetsFilePath() != null) { + generator.writeStringField( + READABLE_OFFSETS_KEY, + lakeSnapshotMetadata.getReadableOffsetsFilePath().toString()); + } + 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); + } 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); + } else { + throw new IllegalArgumentException("Unsupported version: " + version); + } + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableSnapshot.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableSnapshot.java new file mode 100644 index 0000000000..f567a19d36 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableSnapshot.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.zk.data.lake; + +import org.apache.fluss.metadata.TableBucket; + +import java.util.Map; +import java.util.Objects; +import java.util.Optional; + +/** The snapshot info for a table. */ +public class LakeTableSnapshot { + + // the last committed snapshot id in lake + private final long snapshotId; + + // the log offset of the bucket + // mapping from bucket id to log end offset or max timestamp, + // will be null if log offset is unknown such as reading the snapshot of primary key table + private final Map bucketLogEndOffset; + + public LakeTableSnapshot(long snapshotId, Map bucketLogEndOffset) { + this.snapshotId = snapshotId; + this.bucketLogEndOffset = bucketLogEndOffset; + } + + public long getSnapshotId() { + return snapshotId; + } + + public Optional getLogEndOffset(TableBucket tableBucket) { + return Optional.ofNullable(bucketLogEndOffset.get(tableBucket)); + } + + public Map getBucketLogEndOffset() { + return bucketLogEndOffset; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + LakeTableSnapshot that = (LakeTableSnapshot) o; + return snapshotId == that.snapshotId + && Objects.equals(bucketLogEndOffset, that.bucketLogEndOffset); + } + + @Override + public int hashCode() { + return Objects.hash(snapshotId, bucketLogEndOffset); + } + + @Override + public String toString() { + return "LakeTableSnapshot{" + + "snapshotId=" + + snapshotId + + ", bucketLogEndOffset=" + + bucketLogEndOffset + + '}'; + } +} 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 new file mode 100644 index 0000000000..2543dff59e --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableSnapshotJsonSerde.java @@ -0,0 +1,312 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.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. + *
+ */ +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; + + @Override + public void serialize(LakeTableSnapshot lakeTableSnapshot, JsonGenerator generator) + throws IOException { + generator.writeStartObject(); + generator.writeNumberField(VERSION_KEY, CURRENT_VERSION); + generator.writeNumberField(SNAPSHOT_ID, lakeTableSnapshot.getSnapshotId()); + + Map bucketLogEndOffset = lakeTableSnapshot.getBucketLogEndOffset(); + + 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(); + 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++; + } + long logEndOffset = checkNotNull(bucketLogEndOffset.get(tableBucket)); + generator.writeNumber(logEndOffset); + currentBucketId++; + } + } + + @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); + } + } + + /** 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(); + TableBucket tableBucket; + Long partitionId = + bucket.get(PARTITION_ID) != null ? bucket.get(PARTITION_ID).asLong() : null; + tableBucket = new TableBucket(tableId, partitionId, bucket.get(BUCKET_ID).asInt()); + if (bucket.get(LOG_END_OFFSET) != null) { + bucketLogEndOffset.put(tableBucket, bucket.get(LOG_END_OFFSET).asLong()); + } else { + bucketLogEndOffset.put(tableBucket, null); + } + } + 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/log/remote/DefaultRemoteLogStorageTest.java b/fluss-server/src/test/java/org/apache/fluss/server/log/remote/DefaultRemoteLogStorageTest.java index 207bf16545..a450295a7f 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/log/remote/DefaultRemoteLogStorageTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/log/remote/DefaultRemoteLogStorageTest.java @@ -37,6 +37,8 @@ import java.nio.file.StandardCopyOption; import java.util.Collections; import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -44,16 +46,21 @@ /** Test for {@link DefaultRemoteLogStorage}. */ class DefaultRemoteLogStorageTest extends RemoteLogTestBase { private DefaultRemoteLogStorage remoteLogStorageManager; + private ExecutorService ioExecutor; @BeforeEach public void setup() throws Exception { super.setup(); - remoteLogStorageManager = new DefaultRemoteLogStorage(conf); + ioExecutor = Executors.newSingleThreadExecutor(); + remoteLogStorageManager = new DefaultRemoteLogStorage(conf, ioExecutor); } @AfterEach public void teardown() throws Exception { remoteLogStorageManager.close(); + if (ioExecutor != null) { + ioExecutor.shutdown(); + } } @ParameterizedTest diff --git a/fluss-server/src/test/java/org/apache/fluss/server/log/remote/TestingRemoteLogStorage.java b/fluss-server/src/test/java/org/apache/fluss/server/log/remote/TestingRemoteLogStorage.java index 688e2d320a..75802dc028 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/log/remote/TestingRemoteLogStorage.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/log/remote/TestingRemoteLogStorage.java @@ -22,6 +22,7 @@ import org.apache.fluss.fs.FsPath; import java.io.IOException; +import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicBoolean; /** @@ -32,8 +33,9 @@ public class TestingRemoteLogStorage extends DefaultRemoteLogStorage { public final AtomicBoolean writeManifestFail = new AtomicBoolean(false); - public TestingRemoteLogStorage(Configuration conf) throws IOException { - super(conf); + public TestingRemoteLogStorage(Configuration conf, ExecutorService ioExecutor) + throws IOException { + super(conf, ioExecutor); } @Override diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/CommitLakeTableSnapshotITCase.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/CommitLakeTableSnapshotITCase.java index 88f3008fa1..2810ce9d24 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/CommitLakeTableSnapshotITCase.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/CommitLakeTableSnapshotITCase.java @@ -31,14 +31,13 @@ import org.apache.fluss.server.testutils.FlussClusterExtension; import org.apache.fluss.server.testutils.RpcMessageTestUtils; import org.apache.fluss.server.zk.ZooKeeperClient; -import org.apache.fluss.server.zk.data.LakeTableSnapshot; +import org.apache.fluss.server.zk.data.lake.LakeTableSnapshot; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; import java.time.Duration; -import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -102,7 +101,6 @@ void testCommitDataLakeData() throws Exception { // now, let's commit the lake table snapshot CoordinatorGateway coordinatorGateway = FLUSS_CLUSTER_EXTENSION.newCoordinatorClient(); long snapshotId = 1; - long dataLakeLogStartOffset = 0; long dataLakeLogEndOffset = 50; long dataLakeMaxTimestamp = System.currentTimeMillis(); CommitLakeTableSnapshotRequest commitLakeTableSnapshotRequest = @@ -110,39 +108,26 @@ void testCommitDataLakeData() throws Exception { tableId, BUCKET_NUM, snapshotId, - dataLakeLogStartOffset, dataLakeLogEndOffset, dataLakeMaxTimestamp); coordinatorGateway.commitLakeTableSnapshot(commitLakeTableSnapshotRequest).get(); - Map bucketsLogStartOffset = new HashMap<>(); Map bucketsLogEndOffset = new HashMap<>(); - Map bucketsMaxTimestamp = new HashMap<>(); for (int bucket = 0; bucket < BUCKET_NUM; bucket++) { TableBucket tb = new TableBucket(tableId, bucket); - bucketsLogStartOffset.put(tb, dataLakeLogStartOffset); bucketsLogEndOffset.put(tb, dataLakeLogEndOffset); - bucketsMaxTimestamp.put(tb, dataLakeMaxTimestamp); Replica replica = FLUSS_CLUSTER_EXTENSION.waitAndGetLeaderReplica(tb); retry( Duration.ofMinutes(2), () -> { LogTablet logTablet = replica.getLogTablet(); - assertThat(logTablet.getLakeLogStartOffset()) - .isEqualTo(dataLakeLogStartOffset); assertThat(logTablet.getLakeLogEndOffset()).isEqualTo(dataLakeLogEndOffset); assertThat(logTablet.getLakeMaxTimestamp()).isEqualTo(dataLakeMaxTimestamp); }); } LakeTableSnapshot expectedDataLakeTieredInfo = - new LakeTableSnapshot( - snapshotId, - tableId, - bucketsLogStartOffset, - bucketsLogEndOffset, - bucketsMaxTimestamp, - Collections.emptyMap()); + new LakeTableSnapshot(snapshotId, bucketsLogEndOffset); checkLakeTableDataInZk(tableId, expectedDataLakeTieredInfo); } @@ -152,12 +137,7 @@ private void checkLakeTableDataInZk(long tableId, LakeTableSnapshot expected) th } private static CommitLakeTableSnapshotRequest genCommitLakeTableSnapshotRequest( - long tableId, - int buckets, - long snapshotId, - long logStartOffset, - long logEndOffset, - long maxTimestamp) { + long tableId, int buckets, long snapshotId, long logEndOffset, long maxTimestamp) { CommitLakeTableSnapshotRequest commitLakeTableSnapshotRequest = new CommitLakeTableSnapshotRequest(); PbLakeTableSnapshotInfo reqForTable = commitLakeTableSnapshotRequest.addTablesReq(); @@ -170,7 +150,6 @@ private static CommitLakeTableSnapshotRequest genCommitLakeTableSnapshotRequest( lakeTableOffsetForBucket.setPartitionId(tb.getPartitionId()); } lakeTableOffsetForBucket.setBucketId(tb.getBucket()); - lakeTableOffsetForBucket.setLogStartOffset(logStartOffset); lakeTableOffsetForBucket.setLogEndOffset(logEndOffset); lakeTableOffsetForBucket.setMaxTimestamp(maxTimestamp); } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTestBase.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTestBase.java index ae7e14a874..81cc34aad2 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTestBase.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTestBase.java @@ -141,6 +141,7 @@ public class ReplicaTestBase { protected TestingCompletedKvSnapshotCommitter snapshotReporter; protected TestCoordinatorGateway testCoordinatorGateway; private FlussScheduler scheduler; + private ExecutorService ioExecutor; // remote log related protected TestingRemoteLogStorage remoteLogStorage; @@ -167,7 +168,7 @@ public void setup() throws Exception { conf.setString(ConfigOptions.DATA_DIR, tempDir.getAbsolutePath()); conf.setString(ConfigOptions.COORDINATOR_HOST, "localhost"); conf.set(ConfigOptions.REMOTE_DATA_DIR, tempDir.getAbsolutePath() + "/remote_data_dir"); - conf.set(ConfigOptions.REMOTE_LOG_DATA_TRANSFER_THREAD_NUM, 1); + conf.set(ConfigOptions.SERVER_IO_POOL_SIZE, 2); // set snapshot interval to 1 seconds for test purpose conf.set(ConfigOptions.KV_SNAPSHOT_INTERVAL, Duration.ofSeconds(1)); @@ -179,6 +180,7 @@ public void setup() throws Exception { scheduler = new FlussScheduler(2); scheduler.startup(); + ioExecutor = Executors.newSingleThreadExecutor(); manualClock = new ManualClock(System.currentTimeMillis()); logManager = @@ -302,7 +304,8 @@ protected ReplicaManager buildReplicaManager(CoordinatorGateway coordinatorGatew NOPErrorHandler.INSTANCE, TestingMetricGroups.TABLET_SERVER_METRICS, remoteLogManager, - manualClock); + manualClock, + ioExecutor); } @AfterEach @@ -337,6 +340,10 @@ void tearDown() throws Exception { scheduler.shutdown(); } + if (ioExecutor != null) { + ioExecutor.shutdown(); + } + // clear zk environment. ZOO_KEEPER_EXTENSION_WRAPPER.getCustomExtension().cleanupRoot(); } @@ -477,7 +484,7 @@ private Replica makeReplica( } private void initRemoteLogEnv() throws Exception { - remoteLogStorage = new TestingRemoteLogStorage(conf); + remoteLogStorage = new TestingRemoteLogStorage(conf, ioExecutor); remoteLogTaskScheduler = new ManuallyTriggeredScheduledExecutorService(); remoteLogManager = new RemoteLogManager( diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherThreadTest.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherThreadTest.java index b1fa9067ef..57f6be4848 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherThreadTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherThreadTest.java @@ -51,6 +51,7 @@ import org.apache.fluss.utils.concurrent.FlussScheduler; import org.apache.fluss.utils.concurrent.Scheduler; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -65,6 +66,8 @@ import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.function.Consumer; import static org.apache.fluss.record.TestData.DATA1; @@ -96,6 +99,7 @@ public class ReplicaFetcherThreadTest { private ServerNode leader; private ReplicaManager followerRM; private ReplicaFetcherThread followerFetcher; + private ExecutorService ioExecutor; @BeforeAll static void baseBeforeAll() { @@ -126,12 +130,20 @@ public void setup() throws Exception { followerRM, new TestingLeaderEndpoint(conf, leaderRM, follower), 1000); + ioExecutor = Executors.newSingleThreadExecutor(); registerTableInZkClient(); // make the tb(table, 0) to be leader in leaderRM and to be follower in followerRM. makeLeaderAndFollower(); } + @AfterEach + public void tearDown() throws Exception { + if (ioExecutor != null) { + ioExecutor.shutdownNow(); + } + } + @Test void testSimpleFetch() throws Exception { // append records to leader. @@ -420,7 +432,8 @@ private ReplicaManager createReplicaManager(int serverId) throws Exception { new LakeCatalogDynamicLoader(conf, null, true))), RpcClient.create(conf, TestingClientMetricGroup.newInstance(), false), TestingMetricGroups.TABLET_SERVER_METRICS, - manualClock); + manualClock, + ioExecutor); replicaManager.startup(); return replicaManager; } @@ -440,7 +453,8 @@ public TestingReplicaManager( TabletServerMetadataCache metadataCache, RpcClient rpcClient, TabletServerMetricGroup serverMetricGroup, - Clock clock) + Clock clock, + ExecutorService ioExecutor) throws IOException { super( conf, @@ -455,7 +469,8 @@ public TestingReplicaManager( new TestingCompletedKvSnapshotCommitter(), NOPErrorHandler.INSTANCE, serverMetricGroup, - clock); + clock, + ioExecutor); } @Override 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 282c00d4b9..e00f349021 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 @@ -18,12 +18,20 @@ package org.apache.fluss.server.zk.data; 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.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 { @@ -33,75 +41,104 @@ class LakeTableSnapshotJsonSerdeTest extends JsonSerdeTestBase bucketLogStartOffset = new HashMap<>(); - bucketLogStartOffset.put(new TableBucket(tableId, 1), 1L); - bucketLogStartOffset.put(new TableBucket(tableId, 2), 2L); Map bucketLogEndOffset = new HashMap<>(); - bucketLogEndOffset.put(new TableBucket(tableId, 1), 3L); - bucketLogEndOffset.put(new TableBucket(tableId, 2), 4L); - Map bucketMaxTimestamp = new HashMap<>(); - bucketMaxTimestamp.put(new TableBucket(tableId, 1), 5L); - bucketMaxTimestamp.put(new TableBucket(tableId, 2), 6L); - - LakeTableSnapshot lakeTableSnapshot2 = - new LakeTableSnapshot( - 2, - tableId, - bucketLogStartOffset, - bucketLogEndOffset, - bucketMaxTimestamp, - Collections.emptyMap()); + 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); + // Test case 3: Non-partition table with missing bucket ids (0, 2, 4 - missing 1 and 3) tableId = 5; - bucketLogStartOffset = new HashMap<>(); - Map partitionNameIdByPartitionId = new HashMap<>(); - partitionNameIdByPartitionId.put(1L, "partition1"); - partitionNameIdByPartitionId.put(2L, "partition2"); - bucketLogStartOffset.put(new TableBucket(tableId, 1L, 1), 1L); - bucketLogStartOffset.put(new TableBucket(tableId, 2L, 1), 2L); + 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); + // Test case 5: Partition table with missing bucket ids + tableId = 7; bucketLogEndOffset = new HashMap<>(); - bucketLogEndOffset.put(new TableBucket(tableId, 1L, 1), 3L); - bucketLogEndOffset.put(new TableBucket(tableId, 2L, 1), 4L); - - bucketMaxTimestamp = new HashMap<>(); - bucketMaxTimestamp.put(new TableBucket(tableId, 1L, 1), 5L); - bucketMaxTimestamp.put(new TableBucket(tableId, 2L, 1), 6L); - - LakeTableSnapshot lakeTableSnapshot3 = - new LakeTableSnapshot( - 3, - tableId, - bucketLogStartOffset, - bucketLogEndOffset, - bucketMaxTimestamp, - partitionNameIdByPartitionId); + 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); return new LakeTableSnapshot[] { - lakeTableSnapshot1, lakeTableSnapshot2, lakeTableSnapshot3, + lakeTableSnapshot1, + lakeTableSnapshot2, + lakeTableSnapshot3, + lakeTableSnapshot4, + lakeTableSnapshot5, }; } @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[] { - "{\"version\":1,\"snapshot_id\":1,\"table_id\":1,\"buckets\":[]}", - "{\"version\":1,\"snapshot_id\":2,\"table_id\":4," - + "\"buckets\":[{\"bucket_id\":2,\"log_start_offset\":2,\"log_end_offset\":4,\"max_timestamp\":6}," - + "{\"bucket_id\":1,\"log_start_offset\":1,\"log_end_offset\":3,\"max_timestamp\":5}]}", - "{\"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}," - + "{\"partition_id\":2,\"partition_name\":\"partition2\",\"bucket_id\":1,\"log_start_offset\":2,\"log_end_offset\":4,\"max_timestamp\":6}]}" + // 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]}}" }; } + + @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 new file mode 100644 index 0000000000..6dd8497d58 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableHelperTest.java @@ -0,0 +1,172 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.zk.data.lake; + +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.fs.FileSystem; +import org.apache.fluss.fs.FsPath; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TableDescriptor; +import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.server.zk.NOPErrorHandler; +import org.apache.fluss.server.zk.ZooKeeperClient; +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; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +import java.nio.file.Path; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +import static org.assertj.core.api.Assertions.assertThat; + +/** The UT for {@link LakeTableHelper}. */ +class LakeTableHelperTest { + + @RegisterExtension + public static final AllCallbackWrapper ZOO_KEEPER_EXTENSION_WRAPPER = + new AllCallbackWrapper<>(new ZooKeeperExtension()); + + private static ZooKeeperClient zookeeperClient; + + @BeforeAll + static void beforeAll() { + zookeeperClient = + ZOO_KEEPER_EXTENSION_WRAPPER + .getCustomExtension() + .getZooKeeperClient(NOPErrorHandler.INSTANCE); + } + + @AfterEach + void afterEach() { + ZOO_KEEPER_EXTENSION_WRAPPER.getCustomExtension().cleanupRoot(); + } + + @AfterAll + static void afterAll() { + zookeeperClient.close(); + } + + @Test + void testUpsertLakeTableCompatible(@TempDir Path tempDir) throws Exception { + // Create a ZooKeeperClient with REMOTE_DATA_DIR configuration + Configuration conf = new Configuration(); + conf.setString( + ConfigOptions.ZOOKEEPER_ADDRESS, + ZOO_KEEPER_EXTENSION_WRAPPER.getCustomExtension().getConnectString()); + LakeTableHelper lakeTableHelper = new LakeTableHelper(zookeeperClient, tempDir.toString()); + try (ZooKeeperClient zooKeeperClient = + ZooKeeperUtils.startZookeeperClient(conf, NOPErrorHandler.INSTANCE)) { + // first create a table + long tableId = 1; + TablePath tablePath = TablePath.of("test_db", "test_table"); + TableRegistration tableReg = + new TableRegistration( + tableId, + "test table", + Collections.emptyList(), + new TableDescriptor.TableDistribution( + 1, Collections.singletonList("a")), + Collections.emptyMap(), + Collections.emptyMap(), + System.currentTimeMillis(), + System.currentTimeMillis()); + zookeeperClient.registerTable(tablePath, tableReg); + + // Create a legacy version 1 LakeTableSnapshot (full data in ZK) + long snapshotId = 1L; + + Map bucketLogEndOffset = new HashMap<>(); + bucketLogEndOffset.put(new TableBucket(tableId, 0), 100L); + bucketLogEndOffset.put(new TableBucket(tableId, 1), 200L); + + 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); + + // Verify version 1 data can be read + Optional optionalLakeTable = zooKeeperClient.getLakeTable(tableId); + assertThat(optionalLakeTable).isPresent(); + LakeTable lakeTable = optionalLakeTable.get(); + assertThat(lakeTable.getLatestTableSnapshot()).isEqualTo(lakeTableSnapshot); + + // Test: Call upsertLakeTableSnapshot with new snapshot data + // This should read the old version 1 data, merge it, and write as version 2 + Map newBucketLogEndOffset = new HashMap<>(); + newBucketLogEndOffset.put(new TableBucket(tableId, 0), 1500L); // Updated offset + newBucketLogEndOffset.put(new TableBucket(tableId, 1), 2000L); // new offset + + long snapshot2Id = 2L; + LakeTableSnapshot snapshot2 = new LakeTableSnapshot(snapshot2Id, newBucketLogEndOffset); + lakeTableHelper.upsertLakeTable(tableId, tablePath, snapshot2); + + // Verify: New version 2 data can be read + Optional optLakeTableAfter = zooKeeperClient.getLakeTable(tableId); + assertThat(optLakeTableAfter).isPresent(); + LakeTable lakeTableAfter = optLakeTableAfter.get(); + assertThat(lakeTableAfter.getLatestLakeSnapshotMetadata()) + .isNotNull(); // Version 2 has file path + + // Verify: The lake snapshot file exists + FsPath snapshot2FileHandle = + lakeTableAfter.getLatestLakeSnapshotMetadata().getReadableOffsetsFilePath(); + FileSystem fileSystem = snapshot2FileHandle.getFileSystem(); + assertThat(fileSystem.exists(snapshot2FileHandle)).isTrue(); + + Optional optMergedSnapshot = + zooKeeperClient.getLakeTableSnapshot(tableId); + assertThat(optMergedSnapshot).isPresent(); + LakeTableSnapshot mergedSnapshot = optMergedSnapshot.get(); + + // verify the snapshot should merge previous snapshot + assertThat(mergedSnapshot.getSnapshotId()).isEqualTo(snapshot2Id); + Map expectedBucketLogEndOffset = new HashMap<>(bucketLogEndOffset); + expectedBucketLogEndOffset.putAll(newBucketLogEndOffset); + assertThat(mergedSnapshot.getBucketLogEndOffset()) + .isEqualTo(expectedBucketLogEndOffset); + + // add a new snapshot 3 again, verify snapshot + long snapshot3Id = 3L; + LakeTableSnapshot snapshot3 = new LakeTableSnapshot(snapshot3Id, newBucketLogEndOffset); + lakeTableHelper.upsertLakeTable(tableId, tablePath, snapshot3); + // verify snapshot 3 is discarded + assertThat(fileSystem.exists(snapshot2FileHandle)).isFalse(); + } + } +} 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 new file mode 100644 index 0000000000..c2598e41c0 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerdeTest.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.zk.data.lake; + +import org.apache.fluss.fs.FsPath; +import org.apache.fluss.utils.json.JsonSerdeTestBase; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +/** Test for {@link LakeTableJsonSerde}. */ +class LakeTableJsonSerdeTest extends JsonSerdeTestBase { + + LakeTableJsonSerdeTest() { + super(LakeTableJsonSerde.INSTANCE); + } + + @Override + protected LakeTable[] createObjects() { + // Test case 1: Empty lake snapshots list + LakeTable lakeTable1 = new LakeTable(Collections.emptyList()); + + // Test case 2: Single snapshot metadata with readable offsets + 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 + 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 + List metadatas = new ArrayList<>(); + metadatas.add( + new LakeTable.LakeSnapshotMetadata( + 3L, new FsPath("/path/to/tiered3"), new FsPath("/path/to/readable3"))); + metadatas.add( + new LakeTable.LakeSnapshotMetadata( + 4L, new FsPath("/path/to/tiered4"), new FsPath("/path/to/readable4"))); + metadatas.add(new LakeTable.LakeSnapshotMetadata(5L, new FsPath("/path/to/tiered5"), null)); + LakeTable lakeTable4 = new LakeTable(metadatas); + + return new LakeTable[] {lakeTable1, lakeTable2, lakeTable3, lakeTable4}; + } + + @Override + protected String[] expectedJsons() { + return new String[] { + // Test case 1: Empty lake snapshots list + "{\"version\":2,\"lake_snapshots\":[]}", + // Test case 2: Single snapshot metadata with readable offsets + "{\"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 + "{\"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\"}]}" + }; + } +} diff --git a/website/docs/maintenance/configuration.md b/website/docs/maintenance/configuration.md index 4b8d07bb0b..f6988aacef 100644 --- a/website/docs/maintenance/configuration.md +++ b/website/docs/maintenance/configuration.md @@ -26,35 +26,36 @@ during the Fluss cluster working. ## Common -| Option | Type | Default | Description | -|-----------------------------------------------------|--------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| bind.listeners | String | (None) | The network address and port to which the server binds for accepting connections. This defines the interface and port where the server will listen for incoming requests. The format is `{listener_name}://{host}:{port}`, and multiple addresses can be specified, separated by commas. Use `0.0.0.0` for the `host` to bind to all available interfaces which is dangerous on production and not suggested for production usage. The `listener_name` serves as an identifier for the address in the configuration. For example, `internal.listener.name` specifies the address used for internal server communication. If multiple addresses are configured, ensure that the `listener_name` values are unique. | -| advertised.listeners | String | (None) | The externally advertised address and port for client connections. Required in distributed environments when the bind address is not publicly reachable. Format matches `bind.listeners` (`{listener_name}://{host}:{port}`). Defaults to the value of `bind.listeners` if not explicitly configured. | -| internal.listener.name | String | FLUSS | The listener for server internal communication. | -| security.protocol.map | Map | (none) | A map defining the authentication protocol for each listener. The format is `listenerName1:protocol1,listenerName2:protocol2`, e.g., `INTERNAL:PLAINTEXT,CLIENT:GSSAPI`. Each listener can be associated with a specific authentication protocol. Listeners not included in the map will use PLAINTEXT by default, which does not require authentication. | -| `security.${protocol}.*` | String | (none) | Protocol-specific configuration properties. For example, security.sasl.jaas.config for SASL authentication settings. | -| default.bucket.number | Integer | 1 | The default number of buckets for a table in Fluss cluster. It's a cluster-level parameter and all the tables without specifying bucket number in the cluster will use the value as the bucket number. | -| default.replication.factor | Integer | 1 | The default replication factor for the log of a table in Fluss cluster. It's a cluster-level parameter, and all the tables without specifying replication factor in the cluster will use the value as replication factor. | -| remote.data.dir | String | (None) | The directory used for storing the kv snapshot data files and remote log for log tiered storage in a Fluss supported filesystem. | -| remote.fs.write-buffer-size | MemorySize | 4kb | The default size of the write buffer for writing the local files to remote file systems. | -| plugin.classloader.parent-first-patterns.additional | List<String> | (None) | A (semicolon-separated) list of patterns that specifies which classes should always be resolved through the plugin parent ClassLoader first. A pattern is a simple prefix that is checked against the fully qualified class name. These patterns are appended to `classloader.parent-first-patterns.default`. | -| plugin.classloader.parent-first-patterns.default | String | java.,
org.apache.fluss.,
javax.annotation.,
org.slf4j,
org.apache.log4j,
org.apache.logging,
org.apache.commons.logging,
ch.qos.logback | A (semicolon-separated) list of patterns that specifies which classes should always be resolved through the plugin parent ClassLoader first. A pattern is a simple prefix that is checked against the fully qualified class name. This setting should generally not be modified. | -| auto-partition.check.interval | Duration | 10min | The interval of auto partition check. The default value is 10 minutes. | -| allow.create.log.tables | Boolean | true | Whether to allow creation of log tables. When set to false, attempts to create log tables (tables without primary key) will be rejected. The default value is true. | -| allow.create.kv.tables | Boolean | true | Whether to allow creation of kv tables (primary key tables). When set to false, attempts to create kv tables (tables with primary key) will be rejected. The default value is true. | -| max.partition.num | Integer | 1000 | Limits the maximum number of partitions that can be created for a partitioned table to avoid creating too many partitions. | -| max.bucket.num | Integer | 128000 | The maximum number of buckets that can be created for a table. The default value is 128000. | -| acl.notification.expiration-time | Duration | 15min | The duration for which ACL notifications are valid before they expire. This configuration determines the time window during which an ACL notification is considered active. After this duration, the notification will no longer be valid and will be discarded. The default value is 15 minutes. This setting is important to ensure that ACL changes are propagated in a timely manner and do not remain active longer than necessary. | -| authorizer.enabled | Boolean | false | Specifies whether to enable the authorization feature. If enabled, access control is enforced based on the authorization rules defined in the configuration. If disabled, all operations and resources are accessible to all users. | -| authorizer.type | String | default | Specifies the type of authorizer to be used for access control. This value corresponds to the identifier of the authorization plugin. The default value is `default`, which indicates the built-in authorizer implementation. Custom authorizers can be implemented by providing a matching plugin identifier. | -| super.users | String | (None) | A semicolon-separated list of superusers who have unrestricted access to all operations and resources. Note that the delimiter is semicolon since SSL user names may contain comma, and each super user should be specified in the format `principal_type:principal_name`, e.g., `User:admin;User:bob`. This configuration is critical for defining administrative privileges in the system. | +| Option | Type | Default | Description | +|-----------------------------------------------------|--------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| bind.listeners | String | (None) | The network address and port to which the server binds for accepting connections. This defines the interface and port where the server will listen for incoming requests. The format is `{listener_name}://{host}:{port}`, and multiple addresses can be specified, separated by commas. Use `0.0.0.0` for the `host` to bind to all available interfaces which is dangerous on production and not suggested for production usage. The `listener_name` serves as an identifier for the address in the configuration. For example, `internal.listener.name` specifies the address used for internal server communication. If multiple addresses are configured, ensure that the `listener_name` values are unique. | +| advertised.listeners | String | (None) | The externally advertised address and port for client connections. Required in distributed environments when the bind address is not publicly reachable. Format matches `bind.listeners` (`{listener_name}://{host}:{port}`). Defaults to the value of `bind.listeners` if not explicitly configured. | +| internal.listener.name | String | FLUSS | The listener for server internal communication. | +| security.protocol.map | Map | (none) | A map defining the authentication protocol for each listener. The format is `listenerName1:protocol1,listenerName2:protocol2`, e.g., `INTERNAL:PLAINTEXT,CLIENT:GSSAPI`. Each listener can be associated with a specific authentication protocol. Listeners not included in the map will use PLAINTEXT by default, which does not require authentication. | +| `security.${protocol}.*` | String | (none) | Protocol-specific configuration properties. For example, security.sasl.jaas.config for SASL authentication settings. | +| default.bucket.number | Integer | 1 | The default number of buckets for a table in Fluss cluster. It's a cluster-level parameter and all the tables without specifying bucket number in the cluster will use the value as the bucket number. | +| default.replication.factor | Integer | 1 | The default replication factor for the log of a table in Fluss cluster. It's a cluster-level parameter, and all the tables without specifying replication factor in the cluster will use the value as replication factor. | +| remote.data.dir | String | (None) | The directory used for storing the kv snapshot data files and remote log for log tiered storage in a Fluss supported filesystem. | +| remote.fs.write-buffer-size | MemorySize | 4kb | The default size of the write buffer for writing the local files to remote file systems. | +| plugin.classloader.parent-first-patterns.additional | List<String> | (None) | A (semicolon-separated) list of patterns that specifies which classes should always be resolved through the plugin parent ClassLoader first. A pattern is a simple prefix that is checked against the fully qualified class name. These patterns are appended to `classloader.parent-first-patterns.default`. | +| plugin.classloader.parent-first-patterns.default | String | java.,
org.apache.fluss.,
javax.annotation.,
org.slf4j,
org.apache.log4j,
org.apache.logging,
org.apache.commons.logging,
ch.qos.logback | A (semicolon-separated) list of patterns that specifies which classes should always be resolved through the plugin parent ClassLoader first. A pattern is a simple prefix that is checked against the fully qualified class name. This setting should generally not be modified. | +| auto-partition.check.interval | Duration | 10min | The interval of auto partition check. The default value is 10 minutes. | +| allow.create.log.tables | Boolean | true | Whether to allow creation of log tables. When set to false, attempts to create log tables (tables without primary key) will be rejected. The default value is true. | +| allow.create.kv.tables | Boolean | true | Whether to allow creation of kv tables (primary key tables). When set to false, attempts to create kv tables (tables with primary key) will be rejected. The default value is true. | +| max.partition.num | Integer | 1000 | Limits the maximum number of partitions that can be created for a partitioned table to avoid creating too many partitions. | +| max.bucket.num | Integer | 128000 | The maximum number of buckets that can be created for a table. The default value is 128000. | +| acl.notification.expiration-time | Duration | 15min | The duration for which ACL notifications are valid before they expire. This configuration determines the time window during which an ACL notification is considered active. After this duration, the notification will no longer be valid and will be discarded. The default value is 15 minutes. This setting is important to ensure that ACL changes are propagated in a timely manner and do not remain active longer than necessary. | +| authorizer.enabled | Boolean | false | Specifies whether to enable the authorization feature. If enabled, access control is enforced based on the authorization rules defined in the configuration. If disabled, all operations and resources are accessible to all users. | +| authorizer.type | String | default | Specifies the type of authorizer to be used for access control. This value corresponds to the identifier of the authorization plugin. The default value is `default`, which indicates the built-in authorizer implementation. Custom authorizers can be implemented by providing a matching plugin identifier. | +| super.users | String | (None) | A semicolon-separated list of superusers who have unrestricted access to all operations and resources. Note that the delimiter is semicolon since SSL user names may contain comma, and each super user should be specified in the format `principal_type:principal_name`, e.g., `User:admin;User:bob`. This configuration is critical for defining administrative privileges in the system. | +| server.io-pool.size | Integer | 10 | The size of the IO thread pool to run blocking operations for both coordinator and tablet servers. This includes discard unnecessary snapshot files, transfer kv snapshot files, and transfer remote log files. Increase this value if you experience slow IO operations. The default value is 10. | ## CoordinatorServer -| Option | Type | Default | Description | -|--------------------------|---------|---------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| coordinator.io-pool.size | Integer | 10 | The size of the IO thread pool to run blocking operations for coordinator server. This includes discard unnecessary snapshot files. Increase this value if you experience slow unnecessary snapshot files clean. The default value is 10. | +| Option | Type | Default | Description | +|--------------------------|---------|---------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| coordinator.io-pool.size | Integer | 10 | **Deprecated**: This option is deprecated. Please use `server.io-pool.size` instead. The size of the IO thread pool to run blocking operations for coordinator server. This includes discard unnecessary snapshot files. Increase this value if you experience slow unnecessary snapshot files clean. The default value is 10. | ## TabletServer @@ -121,12 +122,12 @@ during the Fluss cluster working. ## Log Tiered Storage -| Option | Type | Default | Description | -|-------------------------------------|------------|---------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| remote.log.task-interval-duration | Duration | 1min | Interval at which remote log manager runs the scheduled tasks like copy segments, clean up remote log segments, delete local log segments etc. If the value is set to 0s, it means that the remote log storage is disabled. | -| remote.log.index-file-cache-size | MemorySize | 1gb | The total size of the space allocated to store index files fetched from remote storage in the local storage. | -| remote.log-manager.thread-pool-size | Integer | 4 | Size of the thread pool used in scheduling tasks to copy segments, fetch remote log indexes and clean up remote log segments. | -| remote.log.data-transfer-thread-num | Integer | 4 | The number of threads the server uses to transfer (download and upload) remote log file can be data file, index file and remote log metadata file. | +| Option | Type | Default | Description | +|-------------------------------------|------------|---------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| remote.log.task-interval-duration | Duration | 1min | Interval at which remote log manager runs the scheduled tasks like copy segments, clean up remote log segments, delete local log segments etc. If the value is set to 0s, it means that the remote log storage is disabled. | +| remote.log.index-file-cache-size | MemorySize | 1gb | The total size of the space allocated to store index files fetched from remote storage in the local storage. | +| remote.log-manager.thread-pool-size | Integer | 4 | Size of the thread pool used in scheduling tasks to copy segments, fetch remote log indexes and clean up remote log segments. | +| remote.log.data-transfer-thread-num | Integer | 4 | **Deprecated**: This option is deprecated. Please use `server.io-pool.size` instead. The number of threads the server uses to transfer (download and upload) remote log file can be data file, index file and remote log metadata file. | ## Kv @@ -134,7 +135,7 @@ during the Fluss cluster working. |---------------------------------------------------|------------|-------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | kv.snapshot.interval | Duration | 10min | The interval to perform periodic snapshot for kv data. The default setting is 10 minutes. | | kv.snapshot.scheduler-thread-num | Integer | 1 | The number of threads that the server uses to schedule snapshot kv data for all the replicas in the server. | -| kv.snapshot.transfer-thread-num | Integer | 4 | The number of threads the server uses to transfer (download and upload) kv snapshot files. | +| kv.snapshot.transfer-thread-num | Integer | 4 | **Deprecated**: This option is deprecated. Please use `server.io-pool.size` instead. The number of threads the server uses to transfer (download and upload) kv snapshot files. | | kv.snapshot.num-retained | Integer | 1 | The maximum number of completed snapshots to retain. | | kv.rocksdb.thread.num | Integer | 2 | The maximum number of concurrent background flush and compaction jobs (per bucket of table). The default value is `2`. | | kv.rocksdb.files.open | Integer | -1 | The maximum number of open files (per bucket of table) that can be used by the DB, `-1` means no limit. The default value is `-1`. | diff --git a/website/docs/maintenance/operations/upgrade-notes-0.9.md b/website/docs/maintenance/operations/upgrade-notes-0.9.md index b8b54df840..c9c7ab66f4 100644 --- a/website/docs/maintenance/operations/upgrade-notes-0.9.md +++ b/website/docs/maintenance/operations/upgrade-notes-0.9.md @@ -10,4 +10,34 @@ These upgrade notes discuss important aspects, such as configuration, behavior, ## Deprecation / End of Support -TODO \ No newline at end of file +### Configuration Options Deprecated + +Several configuration options have been deprecated in Fluss 0.9 and replaced with a unified `server.io-pool.size` option. This change simplifies configuration management by consolidating IO thread pool settings across different components. + +🔧 **Action Required**: Update your configuration files to use the new option. + +#### Deprecated Options + +The following options are deprecated and will be removed in a future version: + +| Deprecated Option | Replacement | Description | +|---------------------------------------|-----------------------|------------------------------------------------------------------| +| `coordinator.io-pool.size` | `server.io-pool.size` | The size of the IO thread pool for coordinator server operations | +| `remote.log.data-transfer-thread-num` | `server.io-pool.size` | The number of threads for transferring remote log files | +| `kv.snapshot.transfer-thread-num` | `server.io-pool.size` | The number of threads for transferring KV snapshot files | + +#### Migration Steps + +1. **Identify deprecated options in your configuration**: + - Check your `server.yaml` configuration file for any of the deprecated options listed above + +2. **Replace with the unified option**: + - Remove the deprecated options from your configuration + - Add or update `server.io-pool.size` with an appropriate value + - The default value is `10`, which should work for most use cases + +#### Benefits of the Change + +- **Simplified Configuration**: One option instead of multiple options for IO thread pool management +- **Better Resource Management**: Unified thread pool allows better resource sharing across different IO operations +- **Consistent Behavior**: All IO operations (remote log, KV snapshot, etc.) now use the same thread pool configuration