diff --git a/fluss-common/src/main/java/org/apache/fluss/lake/committer/BucketOffset.java b/fluss-common/src/main/java/org/apache/fluss/lake/committer/BucketOffset.java deleted file mode 100644 index a8731da155..0000000000 --- a/fluss-common/src/main/java/org/apache/fluss/lake/committer/BucketOffset.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.fluss.lake.committer; - -import javax.annotation.Nullable; - -import java.io.Serializable; -import java.util.Objects; - -/** The bucket offset information to be expected to be stored in Lake's snapshot property. */ -public class BucketOffset implements Serializable { - - private static final long serialVersionUID = 1L; - public static final String FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY = "fluss-offsets"; - - private final long logOffset; - private final int bucket; - private final @Nullable Long partitionId; - - public BucketOffset(long logOffset, int bucket, @Nullable Long partitionId) { - this.logOffset = logOffset; - this.bucket = bucket; - this.partitionId = partitionId; - } - - public long getLogOffset() { - return logOffset; - } - - public int getBucket() { - return bucket; - } - - @Nullable - public Long getPartitionId() { - return partitionId; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - BucketOffset that = (BucketOffset) o; - return logOffset == that.logOffset - && bucket == that.bucket - && Objects.equals(partitionId, that.partitionId); - } - - @Override - public int hashCode() { - return Objects.hash(logOffset, bucket, partitionId); - } -} diff --git a/fluss-common/src/main/java/org/apache/fluss/lake/committer/LakeCommitter.java b/fluss-common/src/main/java/org/apache/fluss/lake/committer/LakeCommitter.java index 5f87d7f5a2..607dc78667 100644 --- a/fluss-common/src/main/java/org/apache/fluss/lake/committer/LakeCommitter.java +++ b/fluss-common/src/main/java/org/apache/fluss/lake/committer/LakeCommitter.java @@ -36,6 +36,12 @@ @PublicEvolving public interface LakeCommitter extends AutoCloseable { + /** + * The property key used to store the file path of lake table bucket offsets in snapshot + * properties. + */ + String FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY = "fluss-offsets"; + /** * Converts a list of write results to a committable object. * diff --git a/fluss-common/src/main/java/org/apache/fluss/utils/FlussPaths.java b/fluss-common/src/main/java/org/apache/fluss/utils/FlussPaths.java index 8c63113780..d04268fbaa 100644 --- a/fluss-common/src/main/java/org/apache/fluss/utils/FlussPaths.java +++ b/fluss-common/src/main/java/org/apache/fluss/utils/FlussPaths.java @@ -710,14 +710,14 @@ public static FsPath remoteLakeTableSnapshotDir( *

The path contract: * *

-     * {$remoteLakeTableSnapshotMetadataDir}/metadata/{uuid}.manifest
+     * {$remoteLakeTableSnapshotMetadataDir}/metadata/{UUID}.offsets
      * 
*/ - public static FsPath remoteLakeTableSnapshotManifestPath( + public static FsPath remoteLakeTableSnapshotOffsetPath( String remoteDataDir, TablePath tablePath, long tableId) { return new FsPath( String.format( - "%s/metadata/%s.manifest", + "%s/metadata/%s.offsets", remoteLakeTableSnapshotDir(remoteDataDir, tablePath, tableId), UUID.randomUUID())); } diff --git a/fluss-common/src/main/java/org/apache/fluss/utils/json/BucketOffsetJsonSerde.java b/fluss-common/src/main/java/org/apache/fluss/utils/json/BucketOffsetJsonSerde.java deleted file mode 100644 index 235268e682..0000000000 --- a/fluss-common/src/main/java/org/apache/fluss/utils/json/BucketOffsetJsonSerde.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.fluss.utils.json; - -import org.apache.fluss.lake.committer.BucketOffset; -import org.apache.fluss.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; -import org.apache.fluss.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; - -import java.io.IOException; - -/** Json serializer and deserializer for {@link BucketOffset}. */ -public class BucketOffsetJsonSerde - implements JsonSerializer, JsonDeserializer { - - public static final BucketOffsetJsonSerde INSTANCE = new BucketOffsetJsonSerde(); - private static final String PARTITION_ID = "partition_id"; - private static final String BUCKET_ID = "bucket"; - private static final String LOG_OFFSET = "offset"; - - @Override - public BucketOffset deserialize(JsonNode node) { - JsonNode partitionIdNode = node.get(PARTITION_ID); - Long partitionId = partitionIdNode == null ? null : partitionIdNode.asLong(); - int bucketId = node.get(BUCKET_ID).asInt(); - - // deserialize log offset - long logOffset = node.get(LOG_OFFSET).asLong(); - - return new BucketOffset(logOffset, bucketId, partitionId); - } - - @Override - public void serialize(BucketOffset bucketOffset, JsonGenerator generator) throws IOException { - generator.writeStartObject(); - - // write partition id - if (bucketOffset.getPartitionId() != null) { - generator.writeNumberField(PARTITION_ID, bucketOffset.getPartitionId()); - } - generator.writeNumberField(BUCKET_ID, bucketOffset.getBucket()); - - // serialize bucket offset - generator.writeNumberField(LOG_OFFSET, bucketOffset.getLogOffset()); - - generator.writeEndObject(); - } -} diff --git a/fluss-common/src/main/java/org/apache/fluss/utils/json/TableBucketOffsets.java b/fluss-common/src/main/java/org/apache/fluss/utils/json/TableBucketOffsets.java new file mode 100644 index 0000000000..030d3da180 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/utils/json/TableBucketOffsets.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.utils.json; + +import org.apache.fluss.metadata.TableBucket; + +import java.util.Map; +import java.util.Objects; + +/** + * Represents the offsets for all buckets of a table. This class stores the mapping from {@link + * TableBucket} to their corresponding offsets. + * + *

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

The offsets map contains entries for each bucket that has a valid offset. Missing buckets are + * not included in the map. + * + * @see TableBucketOffsetsJsonSerde for JSON serialization and deserialization. + */ +public class TableBucketOffsets { + + /** The table ID that all buckets belong to. */ + private final long tableId; + + /** + * The mapping from {@link TableBucket} to their offsets. The map contains entries only for + * buckets that have valid offsets. + */ + private final Map offsets; + + /** + * Creates a new {@link TableBucketOffsets} instance. + * + * @param tableId the table ID that all buckets belong to + * @param offsets the mapping from {@link TableBucket} to their offsets + */ + public TableBucketOffsets(long tableId, Map offsets) { + this.tableId = tableId; + this.offsets = offsets; + } + + /** + * Returns the table ID that all buckets belong to. + * + * @return the table ID + */ + public long getTableId() { + return tableId; + } + + /** + * Returns the mapping from {@link TableBucket} to their offsets. + * + * @return the offsets map + */ + public Map getOffsets() { + return offsets; + } + + /** + * Serialize to a JSON byte array. + * + * @see TableBucketOffsetsJsonSerde + */ + public byte[] toJsonBytes() { + return JsonSerdeUtils.writeValueAsBytes(this, TableBucketOffsetsJsonSerde.INSTANCE); + } + + /** + * Deserialize from JSON byte array to an instance of {@link TableBucketOffsets}. + * + * @see TableBucketOffsets + */ + public static TableBucketOffsets fromJsonBytes(byte[] json) { + return JsonSerdeUtils.readValue(json, TableBucketOffsetsJsonSerde.INSTANCE); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TableBucketOffsets that = (TableBucketOffsets) o; + return tableId == that.tableId && Objects.equals(offsets, that.offsets); + } + + @Override + public int hashCode() { + return Objects.hash(tableId, offsets); + } + + @Override + public String toString() { + return "TableBucketOffsets{" + "tableId=" + tableId + ", offsets=" + offsets + '}'; + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/utils/json/TableBucketOffsetsJsonSerde.java b/fluss-common/src/main/java/org/apache/fluss/utils/json/TableBucketOffsetsJsonSerde.java new file mode 100644 index 0000000000..a177c53f66 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/utils/json/TableBucketOffsetsJsonSerde.java @@ -0,0 +1,237 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.utils.json; + +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; +import org.apache.fluss.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +import static org.apache.fluss.utils.Preconditions.checkNotNull; +import static org.apache.fluss.utils.Preconditions.checkState; + +/** + * Json serializer and deserializer for {@link TableBucketOffsets}. + * + *

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

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

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

The serialized format includes: + * + *

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

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

This method validates that all buckets in the offsets map have the same table_id as the + * {@link TableBucketOffsets#getTableId()}. + * + * @param tableBucketOffsets the {@link TableBucketOffsets} object to serialize + * @param generator the JSON generator to write to + * @throws IOException if an I/O error occurs during serialization + * @throws IllegalStateException if buckets have inconsistent table IDs + */ + @Override + public void serialize(TableBucketOffsets tableBucketOffsets, JsonGenerator generator) + throws IOException { + generator.writeStartObject(); + long expectedTableId = tableBucketOffsets.getTableId(); + generator.writeNumberField(VERSION_KEY, VERSION); + generator.writeNumberField(TABLE_ID_KEY, expectedTableId); + + Map offsets = tableBucketOffsets.getOffsets(); + if (!offsets.isEmpty()) { + // Group buckets by partition_id and validate table_id consistency + Map> partitionBuckets = new TreeMap<>(); + List nonPartitionBuckets = new ArrayList<>(); + + for (TableBucket tableBucket : offsets.keySet()) { + // Check that all buckets have the same table_id + checkState( + tableBucket.getTableId() == expectedTableId, + "All buckets must have the same table_id. Expected: %d, but found: %d in bucket: %s", + expectedTableId, + tableBucket.getTableId(), + tableBucket); + + if (tableBucket.getPartitionId() != null) { + partitionBuckets + .computeIfAbsent(tableBucket.getPartitionId(), k -> new ArrayList<>()) + .add(tableBucket); + } else { + nonPartitionBuckets.add(tableBucket); + } + } + + // Serialize non-partitioned table bucket offsets + if (!nonPartitionBuckets.isEmpty()) { + checkState( + partitionBuckets.isEmpty(), + "partitionBuckets must be empty when nonPartitionBuckets is not empty"); + generator.writeArrayFieldStart(BUCKET_OFFSETS_KEY); + serializeBucketLogEndOffset(offsets, nonPartitionBuckets, generator); + generator.writeEndArray(); + } else { + // nonPartitionBuckets is empty, partitionBuckets is must not empty + checkState( + !partitionBuckets.isEmpty(), + "partitionBuckets must be not empty when nonPartitionBuckets is empty"); + generator.writeArrayFieldStart(PARTITION_OFFSETS_KEY); + for (Map.Entry> entry : partitionBuckets.entrySet()) { + Long partitionId = entry.getKey(); + List buckets = entry.getValue(); + generator.writeStartObject(); + generator.writeNumberField(PARTITION_ID_KEY, partitionId); + generator.writeArrayFieldStart(BUCKET_OFFSETS_KEY); + serializeBucketLogEndOffset(offsets, buckets, generator); + generator.writeEndArray(); + generator.writeEndObject(); + } + generator.writeEndArray(); + } + } + + generator.writeEndObject(); + } + + /** + * Deserializes a JSON node to a {@link TableBucketOffsets} object. + * + *

This method reads the JSON format and reconstructs the table bucket offsets map. The array + * index in "bucket_offsets" represents the bucket id, and the value represents the offset. + * + * @param node the JSON node to deserialize + * @return the deserialized {@link TableBucketOffsets} object + * @throws IllegalArgumentException if the version is not supported + */ + @Override + public TableBucketOffsets deserialize(JsonNode node) { + int version = node.get(VERSION_KEY).asInt(); + if (version != VERSION) { + throw new IllegalArgumentException("Unsupported version: " + version); + } + + long tableId = node.get(TABLE_ID_KEY).asLong(); + Map offsets = new HashMap<>(); + + // Deserialize non-partitioned table bucket offsets + JsonNode bucketOffsetsNode = node.get(BUCKET_OFFSETS_KEY); + JsonNode partitionBucketOffsetsNode = node.get(PARTITION_OFFSETS_KEY); + if (bucketOffsetsNode != null || partitionBucketOffsetsNode != null) { + if (bucketOffsetsNode != null && partitionBucketOffsetsNode != null) { + throw new IllegalArgumentException( + "Both bucket_offsets and partition_bucket_offsets cannot be present at the same time"); + } + + if (bucketOffsetsNode != null) { + int bucketId = 0; + for (JsonNode bucketOffsetNode : bucketOffsetsNode) { + long offset = bucketOffsetNode.asLong(); + // Ignore unknown offsets (filled for missing bucket ids) + if (offset != UNKNOWN_OFFSET) { + TableBucket tableBucket = new TableBucket(tableId, bucketId); + offsets.put(tableBucket, offset); + } + bucketId++; + } + } else { + for (JsonNode partitionOffsetNode : partitionBucketOffsetsNode) { + long partitionId = partitionOffsetNode.get(PARTITION_ID_KEY).asLong(); + JsonNode bucketOffsetsArray = partitionOffsetNode.get(BUCKET_OFFSETS_KEY); + int bucketId = 0; + for (JsonNode bucketOffsetNode : bucketOffsetsArray) { + long offset = bucketOffsetNode.asLong(); + // Ignore unknown offsets (filled for missing bucket ids) + if (offset != UNKNOWN_OFFSET) { + TableBucket tableBucket = + new TableBucket(tableId, partitionId, bucketId); + offsets.put(tableBucket, offset); + } + bucketId++; + } + } + } + } + + return new TableBucketOffsets(tableId, offsets); + } + + private void serializeBucketLogEndOffset( + Map bucketLogEndOffset, + List buckets, + JsonGenerator generator) + throws IOException { + // sort by bucket id + buckets.sort(Comparator.comparingInt(TableBucket::getBucket)); + int currentBucketId = 0; + for (TableBucket tableBucket : buckets) { + int bucketId = tableBucket.getBucket(); + // Fill null values for missing bucket ids + while (currentBucketId < bucketId) { + generator.writeNumber(UNKNOWN_OFFSET); + currentBucketId++; + } + long logEndOffset = checkNotNull(bucketLogEndOffset.get(tableBucket)); + generator.writeNumber(logEndOffset); + currentBucketId++; + } + } +} diff --git a/fluss-common/src/test/java/org/apache/fluss/utils/json/BucketOffsetJsonSerdeTest.java b/fluss-common/src/test/java/org/apache/fluss/utils/json/BucketOffsetJsonSerdeTest.java deleted file mode 100644 index 7c3c26d391..0000000000 --- a/fluss-common/src/test/java/org/apache/fluss/utils/json/BucketOffsetJsonSerdeTest.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.fluss.utils.json; - -import org.apache.fluss.lake.committer.BucketOffset; - -/** Test for {@link BucketOffset}. */ -public class BucketOffsetJsonSerdeTest extends JsonSerdeTestBase { - - BucketOffsetJsonSerdeTest() { - super(BucketOffsetJsonSerde.INSTANCE); - } - - @Override - protected BucketOffset[] createObjects() { - return new BucketOffset[] {new BucketOffset(10, 1, 1L), new BucketOffset(20, 2, null)}; - } - - @Override - protected String[] expectedJsons() { - return new String[] { - "{\"partition_id\":1,\"bucket\":1,\"offset\":10}", "{\"bucket\":2,\"offset\":20}" - }; - } -} diff --git a/fluss-common/src/test/java/org/apache/fluss/utils/json/TableBucketOffsetsJsonSerdeTest.java b/fluss-common/src/test/java/org/apache/fluss/utils/json/TableBucketOffsetsJsonSerdeTest.java new file mode 100644 index 0000000000..831171af2f --- /dev/null +++ b/fluss-common/src/test/java/org/apache/fluss/utils/json/TableBucketOffsetsJsonSerdeTest.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.utils.json; + +import org.apache.fluss.metadata.TableBucket; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +/** Test for {@link TableBucketOffsetsJsonSerde}. */ +class TableBucketOffsetsJsonSerdeTest extends JsonSerdeTestBase { + + public TableBucketOffsetsJsonSerdeTest() { + super(TableBucketOffsetsJsonSerde.INSTANCE); + } + + @Override + protected TableBucketOffsets[] createObjects() { + // Test case 1: Empty offsets + TableBucketOffsets tableBucketOffsets1 = new TableBucketOffsets(1L, Collections.emptyMap()); + + // Test case 2: Non-partition table with consecutive bucket ids (0, 1, 2) + long tableId = 4; + Map bucketLogEndOffset = new HashMap<>(); + bucketLogEndOffset.put(new TableBucket(tableId, 0), 100L); + bucketLogEndOffset.put(new TableBucket(tableId, 1), 200L); + bucketLogEndOffset.put(new TableBucket(tableId, 2), 300L); + TableBucketOffsets tableBucketOffsets2 = + new TableBucketOffsets(tableId, bucketLogEndOffset); + + // Test case 3: Non-partition table with missing bucket ids (0, 2, 4 - missing 1 and 3) + tableId = 5; + bucketLogEndOffset = new HashMap<>(); + bucketLogEndOffset.put(new TableBucket(tableId, 0), 100L); + bucketLogEndOffset.put(new TableBucket(tableId, 2), 300L); + bucketLogEndOffset.put(new TableBucket(tableId, 4), 500L); + TableBucketOffsets tableBucketOffsets3 = + new TableBucketOffsets(tableId, bucketLogEndOffset); + + // Test case 4: Partition table with consecutive bucket ids + tableId = 6; + bucketLogEndOffset = new HashMap<>(); + bucketLogEndOffset.put(new TableBucket(tableId, 1L, 0), 100L); + bucketLogEndOffset.put(new TableBucket(tableId, 1L, 1), 200L); + bucketLogEndOffset.put(new TableBucket(tableId, 2L, 0), 300L); + bucketLogEndOffset.put(new TableBucket(tableId, 2L, 1), 400L); + TableBucketOffsets tableBucketOffsets4 = + new TableBucketOffsets(tableId, bucketLogEndOffset); + + // Test case 5: Partition table with missing bucket ids + tableId = 7; + bucketLogEndOffset = new HashMap<>(); + bucketLogEndOffset.put(new TableBucket(tableId, 1L, 0), 100L); + bucketLogEndOffset.put(new TableBucket(tableId, 1L, 2), 300L); // missing bucket 1 + bucketLogEndOffset.put(new TableBucket(tableId, 2L, 1), 400L); + bucketLogEndOffset.put(new TableBucket(tableId, 2L, 3), 600L); // missing bucket 0 and 2 + TableBucketOffsets tableBucketOffsets5 = + new TableBucketOffsets(tableId, bucketLogEndOffset); + + return new TableBucketOffsets[] { + tableBucketOffsets1, + tableBucketOffsets2, + tableBucketOffsets3, + tableBucketOffsets4, + tableBucketOffsets5, + }; + } + + @Override + protected String[] expectedJsons() { + // Format: + // - Non-partition table: "bucket_offsets": [100, 200, 300], array index = bucket id, + // value = offset. Missing buckets are filled with -1. + // - Partition table: "partition_offsets": [{"partition_id": 1, "bucket_offsets": [100, + // 200]}, ...], + // array index in bucket_offsets = bucket id, value = offset. Missing buckets are filled + // with -1. + return new String[] { + // Test case 1: Empty offsets + "{\"version\":1,\"table_id\":1}", + // Test case 2: Non-partition table with consecutive bucket ids [0, 1, 2] + "{\"version\":1,\"table_id\":4,\"bucket_offsets\":[100,200,300]}", + // Test case 3: Non-partition table with missing bucket ids [0, -1, 2, -1, 4] + "{\"version\":1,\"table_id\":5,\"bucket_offsets\":[100,-1,300,-1,500]}", + // Test case 4: Partition table with consecutive bucket ids + "{\"version\":1,\"table_id\":6,\"partition_offsets\":[{\"partition_id\":1,\"bucket_offsets\":[100,200]},{\"partition_id\":2,\"bucket_offsets\":[300,400]}]}", + // Test case 5: Partition table with missing bucket ids + "{\"version\":1,\"table_id\":7,\"partition_offsets\":[{\"partition_id\":1,\"bucket_offsets\":[100,-1,300]},{\"partition_id\":2,\"bucket_offsets\":[-1,400,-1,600]}]}" + }; + } +} diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/FlussTableLakeSnapshot.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/FlussTableLakeSnapshot.java deleted file mode 100644 index b40a04461e..0000000000 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/FlussTableLakeSnapshot.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.fluss.flink.tiering.committer; - -import org.apache.fluss.metadata.TableBucket; - -import java.util.HashMap; -import java.util.Map; -import java.util.Set; - -/** A lake snapshot for a Fluss table. */ -class FlussTableLakeSnapshot { - - private final long tableId; - - private final long lakeSnapshotId; - - // table_bucket -> log end offsets, - private final Map logEndOffsets; - - FlussTableLakeSnapshot(long tableId, long lakeSnapshotId) { - this.tableId = tableId; - this.lakeSnapshotId = lakeSnapshotId; - this.logEndOffsets = new HashMap<>(); - } - - public long tableId() { - return tableId; - } - - public long lakeSnapshotId() { - return lakeSnapshotId; - } - - public Set tableBuckets() { - return logEndOffsets.keySet(); - } - - public void addBucketOffset(TableBucket bucket, long offset) { - logEndOffsets.put(bucket, offset); - } - - public long getLogEndOffset(TableBucket bucket) { - return logEndOffsets.get(bucket); - } - - @Override - public String toString() { - return "FlussTableLakeSnapshot{" - + "tableId=" - + tableId - + ", lakeSnapshotId=" - + lakeSnapshotId - + ", logEndOffsets=" - + logEndOffsets - + '}'; - } -} diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/FlussTableLakeSnapshotCommitter.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/FlussTableLakeSnapshotCommitter.java index f4be12593a..fe37c91e5b 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/FlussTableLakeSnapshotCommitter.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/FlussTableLakeSnapshotCommitter.java @@ -17,24 +17,52 @@ package org.apache.fluss.flink.tiering.committer; +import org.apache.fluss.annotation.VisibleForTesting; import org.apache.fluss.client.metadata.MetadataUpdater; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TablePath; import org.apache.fluss.metrics.registry.MetricRegistry; import org.apache.fluss.rpc.GatewayClientProxy; import org.apache.fluss.rpc.RpcClient; import org.apache.fluss.rpc.gateway.CoordinatorGateway; import org.apache.fluss.rpc.messages.CommitLakeTableSnapshotRequest; +import org.apache.fluss.rpc.messages.PbBucketOffset; +import org.apache.fluss.rpc.messages.PbCommitLakeTableSnapshotRespForTable; import org.apache.fluss.rpc.messages.PbLakeTableOffsetForBucket; import org.apache.fluss.rpc.messages.PbLakeTableSnapshotInfo; +import org.apache.fluss.rpc.messages.PbLakeTableSnapshotMetadata; +import org.apache.fluss.rpc.messages.PbPrepareLakeTableRespForTable; +import org.apache.fluss.rpc.messages.PbTableOffsets; +import org.apache.fluss.rpc.messages.PrepareLakeTableSnapshotRequest; +import org.apache.fluss.rpc.messages.PrepareLakeTableSnapshotResponse; import org.apache.fluss.rpc.metrics.ClientMetricGroup; +import org.apache.fluss.rpc.protocol.ApiError; import org.apache.fluss.utils.ExceptionUtils; import java.io.IOException; +import java.util.List; import java.util.Map; -/** Committer to commit {@link FlussTableLakeSnapshot} of lake to Fluss. */ +import static org.apache.fluss.utils.Preconditions.checkNotNull; +import static org.apache.fluss.utils.Preconditions.checkState; + +/** + * Committer to commit lake table snapshots to Fluss cluster. + * + *

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

    + *
  • Prepare phase ({@link #prepareLakeSnapshot}): Sends log end offsets to the Fluss + * cluster, which merges them with the previous log end offsets and stores the merged snapshot + * data in a file. Returns the file path where the snapshot metadata is stored. + *
  • Commit phase ({@link #commit}): Sends the lake snapshot metadata (including snapshot + * ID and file paths) to the coordinator to finalize the commit. Also includes log end offsets + * and max tiered timestamps for metrics reporting to tablet servers. + *
+ */ public class FlussTableLakeSnapshotCommitter implements AutoCloseable { private final Configuration flussConf; @@ -59,53 +87,188 @@ public void open() { metadataUpdater::getCoordinatorServer, rpcClient, CoordinatorGateway.class); } - void commit(FlussTableLakeSnapshot flussTableLakeSnapshot) throws IOException { + String prepareLakeSnapshot( + long tableId, TablePath tablePath, Map logEndOffsets) + throws IOException { + PbPrepareLakeTableRespForTable prepareResp; try { - CommitLakeTableSnapshotRequest request = - toCommitLakeTableSnapshotRequest(flussTableLakeSnapshot); - coordinatorGateway.commitLakeTableSnapshot(request).get(); + PrepareLakeTableSnapshotRequest prepareLakeTableSnapshotRequest = + toPrepareLakeTableSnapshotRequest(tableId, tablePath, logEndOffsets); + PrepareLakeTableSnapshotResponse prepareLakeTableSnapshotResponse = + coordinatorGateway + .prepareLakeTableSnapshot(prepareLakeTableSnapshotRequest) + .get(); + List pbPrepareLakeTableRespForTables = + prepareLakeTableSnapshotResponse.getPrepareLakeTableRespsList(); + checkState(pbPrepareLakeTableRespForTables.size() == 1); + prepareResp = pbPrepareLakeTableRespForTables.get(0); + checkState( + prepareResp.getTableId() == tableId, + "TableId does not match, table id in request is %s, but got %s in response.", + tableId, + prepareResp.getTableId()); + if (prepareResp.hasErrorCode()) { + throw ApiError.fromErrorMessage(prepareResp).exception(); + } else { + return checkNotNull(prepareResp).getLakeTableOffsetsPath(); + } } catch (Exception e) { throw new IOException( String.format( - "Fail to commit table lake snapshot %s to Fluss.", - flussTableLakeSnapshot), + "Fail to prepare commit table lake snapshot for %s to Fluss.", + tablePath), ExceptionUtils.stripExecutionException(e)); } } - public void commit(long tableId, long snapshotId, Map logEndOffsets) + void commit( + long tableId, + long lakeSnapshotId, + String lakeBucketOffsetsPath, + Map logEndOffsets, + Map logMaxTieredTimestamps) throws IOException { - // construct lake snapshot to commit to Fluss - FlussTableLakeSnapshot flussTableLakeSnapshot = - new FlussTableLakeSnapshot(tableId, snapshotId); - for (Map.Entry entry : logEndOffsets.entrySet()) { - flussTableLakeSnapshot.addBucketOffset(entry.getKey(), entry.getValue()); + try { + CommitLakeTableSnapshotRequest request = + toCommitLakeTableSnapshotRequest( + tableId, + lakeSnapshotId, + lakeBucketOffsetsPath, + logEndOffsets, + logMaxTieredTimestamps); + List commitLakeTableSnapshotRespForTables = + coordinatorGateway.commitLakeTableSnapshot(request).get().getTableRespsList(); + checkState(commitLakeTableSnapshotRespForTables.size() == 1); + PbCommitLakeTableSnapshotRespForTable commitLakeTableSnapshotRes = + commitLakeTableSnapshotRespForTables.get(0); + if (commitLakeTableSnapshotRes.hasErrorCode()) { + throw ApiError.fromErrorMessage(commitLakeTableSnapshotRes).exception(); + } + } catch (Exception exception) { + throw new IOException( + String.format( + "Fail to commit table lake snapshot id %d of table %d to Fluss.", + lakeSnapshotId, tableId), + ExceptionUtils.stripExecutionException(exception)); + } + } + + /** + * Converts the prepare commit parameters to a {@link PrepareLakeTableSnapshotRequest}. + * + * @param tableId the table ID + * @param tablePath the table path + * @param logEndOffsets the log end offsets for each bucket + * @return the prepared commit request + */ + private PrepareLakeTableSnapshotRequest toPrepareLakeTableSnapshotRequest( + long tableId, TablePath tablePath, Map logEndOffsets) { + PrepareLakeTableSnapshotRequest prepareLakeTableSnapshotRequest = + new PrepareLakeTableSnapshotRequest(); + PbTableOffsets pbTableOffsets = prepareLakeTableSnapshotRequest.addBucketOffset(); + pbTableOffsets.setTableId(tableId); + pbTableOffsets + .setTablePath() + .setDatabaseName(tablePath.getDatabaseName()) + .setTableName(tablePath.getTableName()); + + for (Map.Entry logEndOffsetEntry : logEndOffsets.entrySet()) { + PbBucketOffset pbBucketOffset = pbTableOffsets.addBucketOffset(); + TableBucket tableBucket = logEndOffsetEntry.getKey(); + if (tableBucket.getPartitionId() != null) { + pbBucketOffset.setPartitionId(tableBucket.getPartitionId()); + } + pbBucketOffset.setBucketId(tableBucket.getBucket()); + pbBucketOffset.setLogEndOffset(logEndOffsetEntry.getValue()); } - commit(flussTableLakeSnapshot); + return prepareLakeTableSnapshotRequest; } + /** + * Converts the commit parameters to a {@link CommitLakeTableSnapshotRequest}. + * + *

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

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

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

    + *
  • Merges the new snapshot with the previous latest snapshot (if exists) to ensure + * completeness + *
  • Stores the merged snapshot to the remote file system. The stored file contains the log + * end offset information for each bucket in the table + *
  • Returns the file path where the snapshot is stored + *
  • Call {@link #commitLakeTableSnapshot(CommitLakeTableSnapshotRequest)} with the offset + * file path to finalize the snapshot commit to ZooKeeper in the second phase. + *
+ * + * @param request the request containing lake table snapshot information for one or more tables + * @return a future that completes with a response containing the file paths where snapshots + * (containing bucket log end offset information) are stored, or error information for + * tables that failed to process + */ + @RPC(api = ApiKeys.PREPARE_LAKE_TABLE_SNAPSHOT) + CompletableFuture prepareLakeTableSnapshot( + PrepareLakeTableSnapshotRequest request); + /** * Commit lakehouse table snapshot to Fluss. * diff --git a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java index 388ec081dd..cc033ba8a9 100644 --- a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java +++ b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java @@ -79,7 +79,8 @@ public enum ApiKeys { REMOVE_SERVER_TAG(1048, 0, 0, PUBLIC), REBALANCE(1049, 0, 0, PUBLIC), LIST_REBALANCE_PROGRESS(1050, 0, 0, PUBLIC), - CANCEL_REBALANCE(1051, 0, 0, PUBLIC); + CANCEL_REBALANCE(1051, 0, 0, PUBLIC), + PREPARE_LAKE_TABLE_SNAPSHOT(1052, 0, 0, PRIVATE); private static final Map ID_TO_TYPE = Arrays.stream(ApiKeys.values()) diff --git a/fluss-rpc/src/main/proto/FlussApi.proto b/fluss-rpc/src/main/proto/FlussApi.proto index f5dae90c7e..6698d98d58 100644 --- a/fluss-rpc/src/main/proto/FlussApi.proto +++ b/fluss-rpc/src/main/proto/FlussApi.proto @@ -449,25 +449,27 @@ message NotifyRemoteLogOffsetsRequest { message NotifyRemoteLogOffsetsResponse { } -message CommitLakeTableSnapshotRequest { - repeated PbLakeTableSnapshotInfo tables_req = 1; +message PrepareLakeTableSnapshotRequest { + repeated PbTableOffsets bucket_offsets = 1; } -message PbLakeTableSnapshotInfo { - optional int64 table_id = 1; - required int64 snapshot_id = 2; - repeated PbLakeTableOffsetForBucket buckets_req = 3; +message PrepareLakeTableSnapshotResponse { + repeated PbPrepareLakeTableRespForTable prepare_lake_table_resp = 1; } -message PbLakeTableOffsetForBucket { - optional int64 partition_id = 1; - required int32 bucket_id = 2; - // Deprecated: log_start_offset is no longer used. Field number 3 is reserved for protocol compatibility. - // optional int64 log_start_offset = 3; - optional int64 log_end_offset = 4; - // Deprecated: partition_name is no longer used. Field number 5 is reserved for protocol compatibility. - // optional string partition_name = 5; - optional int64 max_timestamp = 6; +message CommitLakeTableSnapshotRequest { + // Deprecated: PbLakeTableSnapshotInfo is no longer used for committing lake table snapshots. + // Currently, it is only used to allow the coordinator to notify tablet servers about the current + // synchronized log end offsets, which are then reported to metrics. In the future, we plan to + // have tiering directly report to metrics, and this field will be fully removed. + // Still reserve it for protocol compatibility. + repeated PbLakeTableSnapshotInfo tables_req = 1; + // The metadata for lake table snapshots to be committed. Each entry contains the table ID, + // snapshot ID, and the file paths where the snapshot data (containing bucket log end offset + // information) is stored. The tiered_snapshot_file_path points to the file storing tiered log + // end offsets, while readable_snapshot_file_path (if present) points to the file storing readable + // log end offsets. + repeated PbLakeTableSnapshotMetadata lake_table_snapshot_metadata = 2; } message CommitLakeTableSnapshotResponse { @@ -475,9 +477,9 @@ message CommitLakeTableSnapshotResponse { } message PbCommitLakeTableSnapshotRespForTable { - required int64 table_id = 1; - optional int32 error_code = 2; - optional string error_message = 3; + optional int32 error_code = 1; + optional string error_message = 2; + required int64 table_id = 3; } message NotifyLakeTableOffsetRequest { @@ -1012,4 +1014,49 @@ message PbRebalancePlanForBucket { optional int32 new_leader = 4; repeated int32 original_replicas = 5 [packed = true]; repeated int32 new_replicas = 6 [packed = true]; +} + +message PbLakeTableSnapshotMetadata { + required int64 table_id = 1; + required int64 snapshot_id = 2; + required string tiered_bucket_offsets_file_path = 3; + optional string readable_bucket_offsets_file_path = 4; +} + +message PbLakeTableSnapshotInfo { + optional int64 table_id = 1; + required int64 snapshot_id = 2; + repeated PbLakeTableOffsetForBucket buckets_req = 3; + // add table path to reduce get table_path by table id + optional PbTablePath table_path = 4; +} + +message PbLakeTableOffsetForBucket { + optional int64 partition_id = 1; + required int32 bucket_id = 2; + // Deprecated: log_start_offset is no longer used. Field number 3 is reserved for protocol compatibility. + // optional int64 log_start_offset = 3; + optional int64 log_end_offset = 4; + // Deprecated: partition_name is no longer used. Field number 5 is reserved for protocol compatibility. + // optional string partition_name = 5; + optional int64 max_timestamp = 6; +} + +message PbPrepareLakeTableRespForTable { + optional int32 error_code = 1; + optional string error_message = 2; + optional int64 table_id = 3; + optional string lake_table_offsets_path = 4; +} + +message PbTableOffsets { + required int64 table_id = 1; + required PbTablePath table_path = 2; + repeated PbBucketOffset bucket_offsets = 3; +} + +message PbBucketOffset { + optional int64 partition_id = 1; + required int32 bucket_id = 2; + optional int64 log_end_offset = 4; } \ No newline at end of file diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java index 4a6fd89f06..5c429c379a 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java @@ -90,6 +90,7 @@ import org.apache.fluss.server.zk.data.TabletServerRegistration; import org.apache.fluss.server.zk.data.ZkData.PartitionIdsZNode; import org.apache.fluss.server.zk.data.ZkData.TableIdsZNode; +import org.apache.fluss.server.zk.data.lake.LakeTable; import org.apache.fluss.server.zk.data.lake.LakeTableHelper; import org.apache.fluss.server.zk.data.lake.LakeTableSnapshot; import org.apache.fluss.utils.types.Tuple2; @@ -1149,12 +1150,15 @@ private void processNotifyKvSnapshotOffsetEvent(NotifyKvSnapshotOffsetEvent even private void processNotifyLakeTableOffsetEvent(NotifyLakeTableOffsetEvent event) { Map lakeTableSnapshots = event.getLakeTableSnapshots(); - Map tableBucketMaxTieredTimestamps = - event.getTableBucketMaxTieredTimestamps(); + Map> tableMaxTieredTimestamps = + event.getTableMaxTieredTimestamps(); coordinatorRequestBatch.newBatch(); for (Map.Entry lakeTableSnapshotEntry : lakeTableSnapshots.entrySet()) { LakeTableSnapshot lakeTableSnapshot = lakeTableSnapshotEntry.getValue(); + Map tableBucketMaxTieredTimestamps = + tableMaxTieredTimestamps.getOrDefault( + lakeTableSnapshotEntry.getKey(), Collections.emptyMap()); for (TableBucket tb : lakeTableSnapshot.getBucketLogEndOffset().keySet()) { coordinatorContext .getBucketLeaderAndIsr(tb) @@ -1224,6 +1228,18 @@ private void processAccessContext(AccessContextEvent event) { private void tryProcessCommitLakeTableSnapshot( CommitLakeTableSnapshotEvent commitLakeTableSnapshotEvent, CompletableFuture callback) { + CommitLakeTableSnapshotData commitLakeTableSnapshotData = + commitLakeTableSnapshotEvent.getCommitLakeTableSnapshotData(); + if (commitLakeTableSnapshotData.getLakeTableSnapshotMetadatas().isEmpty()) { + handleCommitLakeTableSnapshotV1(commitLakeTableSnapshotEvent, callback); + } else { + handleCommitLakeTableSnapshotV2(commitLakeTableSnapshotEvent, callback); + } + } + + private void handleCommitLakeTableSnapshotV1( + CommitLakeTableSnapshotEvent commitLakeTableSnapshotEvent, + CompletableFuture callback) { // commit the lake table snapshot asynchronously CommitLakeTableSnapshotData commitLakeTableSnapshotData = commitLakeTableSnapshotEvent.getCommitLakeTableSnapshotData(); @@ -1244,6 +1260,7 @@ private void tryProcessCommitLakeTableSnapshot( try { CommitLakeTableSnapshotResponse response = new CommitLakeTableSnapshotResponse(); + Set failedTableIds = new HashSet<>(); for (Map.Entry lakeTableSnapshotEntry : lakeTableSnapshots.entrySet()) { Long tableId = lakeTableSnapshotEntry.getKey(); @@ -1262,20 +1279,72 @@ private void tryProcessCommitLakeTableSnapshot( } // this involves IO operation (ZK), so we do it in ioExecutor - lakeTableHelper.upsertLakeTable( - tableId, tablePath, lakeTableSnapshotEntry.getValue()); + lakeTableHelper.registerLakeTableSnapshotV1( + tableId, lakeTableSnapshotEntry.getValue()); + } catch (Exception e) { + failedTableIds.add(tableId); + ApiError error = ApiError.fromThrowable(e); + tableResp.setError(error.error().code(), error.message()); + } + } + + // remove failed tables + Map commitlakeTableSnapshots = + commitLakeTableSnapshotData.getLakeTableSnapshot(); + commitlakeTableSnapshots.keySet().removeAll(failedTableIds); + Map> tableMaxTieredTimestamps = + commitLakeTableSnapshotData.getTableMaxTieredTimestamps(); + tableMaxTieredTimestamps.keySet().removeAll(failedTableIds); + + coordinatorEventManager.put( + new NotifyLakeTableOffsetEvent( + commitlakeTableSnapshots, tableMaxTieredTimestamps)); + callback.complete(response); + } catch (Exception e) { + callback.completeExceptionally(e); + } + }); + } + + private void handleCommitLakeTableSnapshotV2( + CommitLakeTableSnapshotEvent commitLakeTableSnapshotEvent, + CompletableFuture callback) { + CommitLakeTableSnapshotData commitLakeTableSnapshotData = + commitLakeTableSnapshotEvent.getCommitLakeTableSnapshotData(); + Map lakeSnapshotMetadatas = + commitLakeTableSnapshotData.getLakeTableSnapshotMetadatas(); + ioExecutor.execute( + () -> { + try { + CommitLakeTableSnapshotResponse response = + new CommitLakeTableSnapshotResponse(); + Set failedTableIds = new HashSet<>(); + for (Map.Entry + lakeSnapshotMetadataEntry : lakeSnapshotMetadatas.entrySet()) { + PbCommitLakeTableSnapshotRespForTable tableResp = + response.addTableResp(); + long tableId = lakeSnapshotMetadataEntry.getKey(); + tableResp.setTableId(tableId); + try { + lakeTableHelper.registerLakeTableSnapshotV2( + tableId, lakeSnapshotMetadataEntry.getValue()); } catch (Exception e) { + failedTableIds.add(tableId); ApiError error = ApiError.fromThrowable(e); tableResp.setError(error.error().code(), error.message()); } } + // remove failed tables + Map lakeTableSnapshots = + commitLakeTableSnapshotData.getLakeTableSnapshot(); + lakeTableSnapshots.keySet().removeAll(failedTableIds); + Map> tableMaxTieredTimestamps = + commitLakeTableSnapshotData.getTableMaxTieredTimestamps(); + tableMaxTieredTimestamps.keySet().removeAll(failedTableIds); - // send notify lakehouse data request to all replicas via coordinator event coordinatorEventManager.put( new NotifyLakeTableOffsetEvent( - lakeTableSnapshots, - commitLakeTableSnapshotData - .getTableBucketsMaxTieredTimestamp())); + lakeTableSnapshots, tableMaxTieredTimestamps)); callback.complete(response); } catch (Exception e) { callback.completeExceptionally(e); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorService.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorService.java index 9eee45e600..f7e565a7b0 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorService.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorService.java @@ -33,6 +33,7 @@ import org.apache.fluss.exception.TableAlreadyExistException; import org.apache.fluss.exception.TableNotPartitionedException; import org.apache.fluss.fs.FileSystem; +import org.apache.fluss.fs.FsPath; import org.apache.fluss.lake.lakestorage.LakeCatalog; import org.apache.fluss.metadata.DataLakeFormat; import org.apache.fluss.metadata.DatabaseDescriptor; @@ -87,12 +88,17 @@ import org.apache.fluss.rpc.messages.PbAlterConfig; import org.apache.fluss.rpc.messages.PbHeartbeatReqForTable; import org.apache.fluss.rpc.messages.PbHeartbeatRespForTable; +import org.apache.fluss.rpc.messages.PbPrepareLakeTableRespForTable; +import org.apache.fluss.rpc.messages.PbTableOffsets; +import org.apache.fluss.rpc.messages.PrepareLakeTableSnapshotRequest; +import org.apache.fluss.rpc.messages.PrepareLakeTableSnapshotResponse; import org.apache.fluss.rpc.messages.RebalanceRequest; import org.apache.fluss.rpc.messages.RebalanceResponse; import org.apache.fluss.rpc.messages.RemoveServerTagRequest; import org.apache.fluss.rpc.messages.RemoveServerTagResponse; import org.apache.fluss.rpc.netty.server.Session; import org.apache.fluss.rpc.protocol.ApiError; +import org.apache.fluss.rpc.protocol.Errors; import org.apache.fluss.security.acl.AclBinding; import org.apache.fluss.security.acl.AclBindingFilter; import org.apache.fluss.security.acl.FlussPrincipal; @@ -122,8 +128,11 @@ import org.apache.fluss.server.zk.data.PartitionAssignment; import org.apache.fluss.server.zk.data.TableAssignment; import org.apache.fluss.server.zk.data.TableRegistration; +import org.apache.fluss.server.zk.data.lake.LakeTable; +import org.apache.fluss.server.zk.data.lake.LakeTableHelper; import org.apache.fluss.utils.IOUtils; import org.apache.fluss.utils.concurrent.FutureUtils; +import org.apache.fluss.utils.json.TableBucketOffsets; import javax.annotation.Nullable; @@ -131,6 +140,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.function.Supplier; @@ -148,6 +158,7 @@ import static org.apache.fluss.server.utils.ServerRpcMessageUtils.makeDropAclsResponse; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.toAlterTableConfigChanges; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.toAlterTableSchemaChanges; +import static org.apache.fluss.server.utils.ServerRpcMessageUtils.toTableBucketOffsets; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.toTablePath; import static org.apache.fluss.server.utils.TableAssignmentUtils.generateAssignment; import static org.apache.fluss.utils.PartitionUtils.validatePartitionSpec; @@ -166,6 +177,8 @@ public final class CoordinatorService extends RpcServiceBase implements Coordina private final LakeTableTieringManager lakeTableTieringManager; private final LakeCatalogDynamicLoader lakeCatalogDynamicLoader; + private final ExecutorService ioExecutor; + private final LakeTableHelper lakeTableHelper; public CoordinatorService( Configuration conf, @@ -198,6 +211,9 @@ public CoordinatorService( this.lakeTableTieringManager = lakeTableTieringManager; this.metadataCache = metadataCache; this.lakeCatalogDynamicLoader = lakeCatalogDynamicLoader; + this.ioExecutor = ioExecutor; + this.lakeTableHelper = + new LakeTableHelper(zkClient, conf.getString(ConfigOptions.REMOTE_DATA_DIR)); } @Override @@ -626,6 +642,52 @@ public CompletableFuture dropAcls(DropAclsRequest request) { return CompletableFuture.completedFuture(makeDropAclsResponse(aclDeleteResults)); } + @Override + public CompletableFuture prepareLakeTableSnapshot( + PrepareLakeTableSnapshotRequest request) { + CompletableFuture future = new CompletableFuture<>(); + ioExecutor.submit( + () -> { + PrepareLakeTableSnapshotResponse response = + new PrepareLakeTableSnapshotResponse(); + try { + for (PbTableOffsets bucketOffsets : request.getBucketOffsetsList()) { + PbPrepareLakeTableRespForTable pbPrepareLakeTableRespForTable = + response.addPrepareLakeTableResp(); + try { + long tableId = bucketOffsets.getTableId(); + TableBucketOffsets tableBucketOffsets = + toTableBucketOffsets(bucketOffsets); + // get previous lake tables + Optional optPreviousLakeTable = + zkClient.getLakeTable(tableId); + if (optPreviousLakeTable.isPresent()) { + // need to merge with previous lake table + tableBucketOffsets = + lakeTableHelper.mergeTableBucketOffsets( + optPreviousLakeTable.get(), tableBucketOffsets); + } + TablePath tablePath = toTablePath(bucketOffsets.getTablePath()); + FsPath fsPath = + lakeTableHelper.storeLakeTableOffsetsFile( + tablePath, tableBucketOffsets); + pbPrepareLakeTableRespForTable.setTableId(tableId); + pbPrepareLakeTableRespForTable.setLakeTableOffsetsPath( + fsPath.toString()); + } catch (Exception e) { + Errors error = ApiError.fromThrowable(e).error(); + pbPrepareLakeTableRespForTable.setError( + error.code(), error.message()); + } + } + future.complete(response); + } catch (Exception e) { + future.completeExceptionally(e); + } + }); + return future; + } + @Override public CompletableFuture commitLakeTableSnapshot( CommitLakeTableSnapshotRequest request) { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/NotifyLakeTableOffsetEvent.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/NotifyLakeTableOffsetEvent.java index ba039f2b6b..5dcb224a79 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/NotifyLakeTableOffsetEvent.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/NotifyLakeTableOffsetEvent.java @@ -26,20 +26,20 @@ public class NotifyLakeTableOffsetEvent implements CoordinatorEvent { private final Map lakeTableSnapshots; - private final Map tableBucketMaxTieredTimestamps; + private final Map> tableMaxTieredTimestamps; public NotifyLakeTableOffsetEvent( Map lakeTableSnapshots, - Map tableBucketMaxTieredTimestamps) { + Map> tableMaxTieredTimestamps) { this.lakeTableSnapshots = lakeTableSnapshots; - this.tableBucketMaxTieredTimestamps = tableBucketMaxTieredTimestamps; + this.tableMaxTieredTimestamps = tableMaxTieredTimestamps; } public Map getLakeTableSnapshots() { return lakeTableSnapshots; } - public Map getTableBucketMaxTieredTimestamps() { - return tableBucketMaxTieredTimestamps; + public Map> getTableMaxTieredTimestamps() { + return tableMaxTieredTimestamps; } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/entity/CommitLakeTableSnapshotData.java b/fluss-server/src/main/java/org/apache/fluss/server/entity/CommitLakeTableSnapshotData.java index 21ac1b340f..3ff7e3e2b3 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/entity/CommitLakeTableSnapshotData.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/entity/CommitLakeTableSnapshotData.java @@ -19,6 +19,7 @@ import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.rpc.messages.CommitLakeTableSnapshotRequest; +import org.apache.fluss.server.zk.data.lake.LakeTable; import org.apache.fluss.server.zk.data.lake.LakeTableSnapshot; import java.util.Map; @@ -27,22 +28,44 @@ /** The data for request {@link CommitLakeTableSnapshotRequest}. */ public class CommitLakeTableSnapshotData { + /** + * Since 0.9, this field is only used to allow the coordinator to send requests to tablet + * servers, enabling tablet servers to report metrics about synchronized log end offsets. In the + * future, we plan to have the tiering service directly report metrics, and this field will be + * removed. + */ private final Map lakeTableSnapshots; - private final Map tableBucketsMaxTieredTimestamp; + + /** + * Since 0.9, this field is only used to allow the coordinator to send requests to tablet + * servers, enabling tablet servers to report metrics about max tiered timestamps. In the + * future, we plan to have the tiering service directly report metrics, and this field will be + * removed. + */ + private final Map> tableMaxTieredTimestamps; + + // the following field only non-empty since 0.9 + private final Map lakeTableSnapshotMetadatas; public CommitLakeTableSnapshotData( Map lakeTableSnapshots, - Map tableBucketsMaxTieredTimestamp) { + Map> tableMaxTieredTimestamps, + Map lakeTableSnapshotMetadatas) { this.lakeTableSnapshots = lakeTableSnapshots; - this.tableBucketsMaxTieredTimestamp = tableBucketsMaxTieredTimestamp; + this.tableMaxTieredTimestamps = tableMaxTieredTimestamps; + this.lakeTableSnapshotMetadatas = lakeTableSnapshotMetadatas; } public Map getLakeTableSnapshot() { return lakeTableSnapshots; } - public Map getTableBucketsMaxTieredTimestamp() { - return tableBucketsMaxTieredTimestamp; + public Map> getTableMaxTieredTimestamps() { + return tableMaxTieredTimestamps; + } + + public Map getLakeTableSnapshotMetadatas() { + return lakeTableSnapshotMetadatas; } @Override @@ -55,13 +78,14 @@ public boolean equals(Object o) { } CommitLakeTableSnapshotData that = (CommitLakeTableSnapshotData) o; return Objects.equals(lakeTableSnapshots, that.lakeTableSnapshots) - && Objects.equals( - tableBucketsMaxTieredTimestamp, that.tableBucketsMaxTieredTimestamp); + && Objects.equals(tableMaxTieredTimestamps, that.tableMaxTieredTimestamps) + && Objects.equals(lakeTableSnapshotMetadatas, that.lakeTableSnapshotMetadatas); } @Override public int hashCode() { - return Objects.hash(lakeTableSnapshots, tableBucketsMaxTieredTimestamp); + return Objects.hash( + lakeTableSnapshots, tableMaxTieredTimestamps, lakeTableSnapshotMetadatas); } @Override @@ -69,8 +93,10 @@ public String toString() { return "CommitLakeTableSnapshotData{" + "lakeTableSnapshots=" + lakeTableSnapshots - + ", tableBucketsMaxTieredTimestamp=" - + tableBucketsMaxTieredTimestamp + + ", tableMaxTieredTimestamps=" + + tableMaxTieredTimestamps + + ", lakeTableSnapshotMetadatas=" + + lakeTableSnapshotMetadatas + '}'; } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java b/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java index ed1051af94..8ee46545b0 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java @@ -88,6 +88,7 @@ import org.apache.fluss.rpc.messages.PbAdjustIsrRespForTable; import org.apache.fluss.rpc.messages.PbAlterConfig; import org.apache.fluss.rpc.messages.PbBucketMetadata; +import org.apache.fluss.rpc.messages.PbBucketOffset; import org.apache.fluss.rpc.messages.PbCreateAclRespInfo; import org.apache.fluss.rpc.messages.PbDescribeConfig; import org.apache.fluss.rpc.messages.PbDropAclsFilterResult; @@ -102,6 +103,7 @@ import org.apache.fluss.rpc.messages.PbLakeSnapshotForBucket; import org.apache.fluss.rpc.messages.PbLakeTableOffsetForBucket; import org.apache.fluss.rpc.messages.PbLakeTableSnapshotInfo; +import org.apache.fluss.rpc.messages.PbLakeTableSnapshotMetadata; import org.apache.fluss.rpc.messages.PbListOffsetsRespForBucket; import org.apache.fluss.rpc.messages.PbLookupReqForBucket; import org.apache.fluss.rpc.messages.PbLookupRespForBucket; @@ -126,6 +128,7 @@ import org.apache.fluss.rpc.messages.PbStopReplicaRespForBucket; import org.apache.fluss.rpc.messages.PbTableBucket; import org.apache.fluss.rpc.messages.PbTableMetadata; +import org.apache.fluss.rpc.messages.PbTableOffsets; import org.apache.fluss.rpc.messages.PbTablePath; import org.apache.fluss.rpc.messages.PbValue; import org.apache.fluss.rpc.messages.PbValueList; @@ -165,9 +168,11 @@ import org.apache.fluss.server.metadata.TableMetadata; import org.apache.fluss.server.zk.data.BucketSnapshot; import org.apache.fluss.server.zk.data.LeaderAndIsr; +import org.apache.fluss.server.zk.data.lake.LakeTable; import org.apache.fluss.server.zk.data.lake.LakeTableSnapshot; import org.apache.fluss.utils.json.DataTypeJsonSerde; import org.apache.fluss.utils.json.JsonSerdeUtils; +import org.apache.fluss.utils.json.TableBucketOffsets; import javax.annotation.Nullable; @@ -1544,37 +1549,72 @@ public static PbPartitionSpec makePbPartitionSpec(ResolvedPartitionSpec spec) { public static CommitLakeTableSnapshotData getCommitLakeTableSnapshotData( CommitLakeTableSnapshotRequest request) { + // handle rpc before 0.9 Map lakeTableInfoByTableId = new HashMap<>(); - Map tableBucketsMaxTimestamp = new HashMap<>(); - for (PbLakeTableSnapshotInfo pdLakeTableSnapshotInfo : request.getTablesReqsList()) { - long tableId = pdLakeTableSnapshotInfo.getTableId(); - long snapshotId = pdLakeTableSnapshotInfo.getSnapshotId(); + Map> tableBucketsMaxTimestamp = new HashMap<>(); + for (PbLakeTableSnapshotInfo pbLakeTableSnapshotInfo : request.getTablesReqsList()) { + long tableId = pbLakeTableSnapshotInfo.getTableId(); + long snapshotId = pbLakeTableSnapshotInfo.getSnapshotId(); Map bucketLogEndOffset = new HashMap<>(); - + Map bucketLogMaxTimestamp = new HashMap<>(); for (PbLakeTableOffsetForBucket lakeTableOffsetForBucket : - pdLakeTableSnapshotInfo.getBucketsReqsList()) { + pbLakeTableSnapshotInfo.getBucketsReqsList()) { Long partitionId = lakeTableOffsetForBucket.hasPartitionId() ? lakeTableOffsetForBucket.getPartitionId() : null; int bucketId = lakeTableOffsetForBucket.getBucketId(); - TableBucket tableBucket = new TableBucket(tableId, partitionId, bucketId); Long logEndOffset = lakeTableOffsetForBucket.hasLogEndOffset() ? lakeTableOffsetForBucket.getLogEndOffset() : null; - bucketLogEndOffset.put(tableBucket, logEndOffset); - if (lakeTableOffsetForBucket.hasMaxTimestamp()) { - tableBucketsMaxTimestamp.put( + bucketLogMaxTimestamp.put( tableBucket, lakeTableOffsetForBucket.getMaxTimestamp()); } + bucketLogEndOffset.put(tableBucket, logEndOffset); } - lakeTableInfoByTableId.put( - tableId, new LakeTableSnapshot(snapshotId, bucketLogEndOffset)); + LakeTableSnapshot lakeTableSnapshot = + new LakeTableSnapshot(snapshotId, bucketLogEndOffset); + lakeTableInfoByTableId.put(tableId, lakeTableSnapshot); + tableBucketsMaxTimestamp.put(tableId, bucketLogMaxTimestamp); + } + + // handle rpc since 0.9 + Map lakeSnapshotMetadatas = new HashMap<>(); + for (PbLakeTableSnapshotMetadata pbLakeTableSnapshotMetadata : + request.getLakeTableSnapshotMetadatasList()) { + lakeSnapshotMetadatas.put( + pbLakeTableSnapshotMetadata.getTableId(), + new LakeTable.LakeSnapshotMetadata( + pbLakeTableSnapshotMetadata.getSnapshotId(), + new FsPath( + pbLakeTableSnapshotMetadata.getTieredBucketOffsetsFilePath()), + pbLakeTableSnapshotMetadata.hasReadableBucketOffsetsFilePath() + ? new FsPath( + pbLakeTableSnapshotMetadata + .getReadableBucketOffsetsFilePath()) + : null)); + } + return new CommitLakeTableSnapshotData( + lakeTableInfoByTableId, tableBucketsMaxTimestamp, lakeSnapshotMetadatas); + } + + public static TableBucketOffsets toTableBucketOffsets(PbTableOffsets pbTableOffsets) { + Map bucketOffsets = new HashMap<>(); + long tableId = pbTableOffsets.getTableId(); + for (PbBucketOffset pbBucketOffset : pbTableOffsets.getBucketOffsetsList()) { + TableBucket tableBucket = + new TableBucket( + tableId, + pbBucketOffset.hasPartitionId() + ? pbBucketOffset.getPartitionId() + : null, + pbBucketOffset.getBucketId()); + bucketOffsets.put(tableBucket, pbBucketOffset.getLogEndOffset()); } - return new CommitLakeTableSnapshotData(lakeTableInfoByTableId, tableBucketsMaxTimestamp); + return new TableBucketOffsets(tableId, bucketOffsets); } public static PbNotifyLakeTableOffsetReqForBucket makeNotifyLakeTableOffsetForBucket( diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java index a9e0af4d95..0af2124674 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java @@ -1065,7 +1065,7 @@ public Optional getLakeTableSnapshot(long tableId) throws Exc Optional optLakeTable = getLakeTable(tableId); if (optLakeTable.isPresent()) { // always get the latest snapshot - return Optional.of(optLakeTable.get().getLatestTableSnapshot()); + return Optional.of(optLakeTable.get().getOrReadLatestTableSnapshot()); } else { return Optional.empty(); } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTable.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTable.java index 5df57acd57..c5ae253ecc 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTable.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTable.java @@ -21,8 +21,10 @@ import org.apache.fluss.fs.FSDataInputStream; import org.apache.fluss.fs.FileSystem; import org.apache.fluss.fs.FsPath; +import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.server.zk.data.ZkData; import org.apache.fluss.utils.IOUtils; +import org.apache.fluss.utils.json.TableBucketOffsets; import javax.annotation.Nullable; @@ -30,6 +32,7 @@ import java.io.IOException; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Objects; import static org.apache.fluss.metrics.registry.MetricRegistry.LOG; @@ -104,25 +107,32 @@ public List getLakeSnapshotMetadatas() { return lakeSnapshotMetadatas; } + @Nullable + public LakeTableSnapshot getLakeTableSnapshot() { + return lakeTableSnapshot; + } + /** - * Get the latest table snapshot for the lake table. + * Get or read the latest table snapshot for the lake table. * *

If this LakeTable was created from a LakeTableSnapshot (version 1), returns it directly. * Otherwise, reads the snapshot data from the lake snapshot file. * * @return the LakeTableSnapshot */ - public LakeTableSnapshot getLatestTableSnapshot() throws Exception { + public LakeTableSnapshot getOrReadLatestTableSnapshot() throws IOException { if (lakeTableSnapshot != null) { return lakeTableSnapshot; } - FsPath tieredOffsetsFilePath = - checkNotNull(getLatestLakeSnapshotMetadata()).tieredOffsetsFilePath; + LakeSnapshotMetadata lakeSnapshotMetadata = getLatestLakeSnapshotMetadata(); + FsPath tieredOffsetsFilePath = checkNotNull(lakeSnapshotMetadata).tieredOffsetsFilePath; FSDataInputStream inputStream = tieredOffsetsFilePath.getFileSystem().open(tieredOffsetsFilePath); try (ByteArrayOutputStream outputStream = new ByteArrayOutputStream()) { IOUtils.copyBytes(inputStream, outputStream, true); - return LakeTableSnapshotJsonSerde.fromJson(outputStream.toByteArray()); + Map logOffsets = + TableBucketOffsets.fromJsonBytes(outputStream.toByteArray()).getOffsets(); + return new LakeTableSnapshot(lakeSnapshotMetadata.snapshotId, logOffsets); } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableHelper.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableHelper.java index 8031b9ddeb..e3b46d04e3 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableHelper.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableHelper.java @@ -25,8 +25,10 @@ import org.apache.fluss.metadata.TablePath; import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.utils.FlussPaths; +import org.apache.fluss.utils.json.TableBucketOffsets; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Optional; @@ -44,99 +46,87 @@ public LakeTableHelper(ZooKeeperClient zkClient, String 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). + * Upserts a lake table snapshot for the given table, stored in v1 format. Note: this method is + * just for back compatibility. * * @param tableId the table ID - * @param 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) + public void registerLakeTableSnapshotV1(long tableId, LakeTableSnapshot lakeTableSnapshot) throws Exception { Optional optPreviousLakeTable = zkClient.getLakeTable(tableId); // Merge with previous snapshot if exists if (optPreviousLakeTable.isPresent()) { - lakeTableSnapshot = - mergeLakeTable( - optPreviousLakeTable.get().getLatestTableSnapshot(), lakeTableSnapshot); + TableBucketOffsets tableBucketOffsets = + mergeTableBucketOffsets( + optPreviousLakeTable.get(), + new TableBucketOffsets( + tableId, lakeTableSnapshot.getBucketLogEndOffset())); + lakeTableSnapshot = new LakeTableSnapshot(tableId, tableBucketOffsets.getOffsets()); } + zkClient.upsertLakeTable( + tableId, new LakeTable(lakeTableSnapshot), optPreviousLakeTable.isPresent()); + } - // 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 + public void registerLakeTableSnapshotV2( + long tableId, LakeTable.LakeSnapshotMetadata lakeSnapshotMetadata) throws Exception { + Optional optPreviousLakeTable = zkClient.getLakeTable(tableId); + List previousLakeSnapshotMetadatas = null; + if (optPreviousLakeTable.isPresent()) { + previousLakeSnapshotMetadatas = optPreviousLakeTable.get().getLakeSnapshotMetadatas(); + } LakeTable lakeTable = new LakeTable(lakeSnapshotMetadata); 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(); - } + // currently, we keep only one lake snapshot metadata in zk, + // todo: in solve paimon dv union read issue #2121, we'll keep multiple lake snapshot + // metadata + // discard previous lake snapshot metadata + if (previousLakeSnapshotMetadatas != null) { + previousLakeSnapshotMetadatas.forEach(LakeTable.LakeSnapshotMetadata::discard); } } - private LakeTableSnapshot mergeLakeTable( - LakeTableSnapshot previousLakeTableSnapshot, LakeTableSnapshot newLakeTableSnapshot) { - // Merge current snapshot with previous one since the current snapshot request + public TableBucketOffsets mergeTableBucketOffsets( + LakeTable previousLakeTable, TableBucketOffsets newTableBucketOffsets) + throws Exception { + // Merge current with previous one since the current request // may not carry all buckets for the table. It typically only carries buckets // that were written after the previous commit. // merge log end offsets, current will override the previous Map bucketLogEndOffset = - new HashMap<>(previousLakeTableSnapshot.getBucketLogEndOffset()); - bucketLogEndOffset.putAll(newLakeTableSnapshot.getBucketLogEndOffset()); - - return new LakeTableSnapshot(newLakeTableSnapshot.getSnapshotId(), bucketLogEndOffset); + new HashMap<>( + previousLakeTable.getOrReadLatestTableSnapshot().getBucketLogEndOffset()); + bucketLogEndOffset.putAll(newTableBucketOffsets.getOffsets()); + return new TableBucketOffsets(newTableBucketOffsets.getTableId(), 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); + public FsPath storeLakeTableOffsetsFile( + TablePath tablePath, TableBucketOffsets tableBucketOffsets) throws Exception { + // get the remote file path to store the lake table snapshot offset information + long tableId = tableBucketOffsets.getTableId(); + FsPath remoteLakeTableSnapshotOffsetPath = + FlussPaths.remoteLakeTableSnapshotOffsetPath(remoteDataDir, tablePath, tableId); // check whether the parent directory exists, if not, create the directory - FileSystem fileSystem = remoteLakeTableSnapshotManifestPath.getFileSystem(); - if (!fileSystem.exists(remoteLakeTableSnapshotManifestPath.getParent())) { - fileSystem.mkdirs(remoteLakeTableSnapshotManifestPath.getParent()); + FileSystem fileSystem = remoteLakeTableSnapshotOffsetPath.getFileSystem(); + if (!fileSystem.exists(remoteLakeTableSnapshotOffsetPath.getParent())) { + fileSystem.mkdirs(remoteLakeTableSnapshotOffsetPath.getParent()); } - // serialize table snapshot to json bytes, and write to file - byte[] jsonBytes = LakeTableSnapshotJsonSerde.toJson(lakeTableSnapshot); + // serialize table offsets to json bytes, and write to file + byte[] jsonBytes = tableBucketOffsets.toJsonBytes(); + try (FSDataOutputStream outputStream = fileSystem.create( - remoteLakeTableSnapshotManifestPath, FileSystem.WriteMode.OVERWRITE)) { + remoteLakeTableSnapshotOffsetPath, FileSystem.WriteMode.OVERWRITE)) { outputStream.write(jsonBytes); } - return remoteLakeTableSnapshotManifestPath; + return remoteLakeTableSnapshotOffsetPath; } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerde.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerde.java index f272361bc6..9e72239395 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerde.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerde.java @@ -38,8 +38,8 @@ * *

    *
  • Version 1 (legacy): ZK node contains full {@link LakeTableSnapshot} data. During - * deserialization, it uses {@link LakeTableSnapshotJsonSerde} to deserialize and wraps the - * result in a {@link LakeTable}. + * deserialization, it uses {@link LakeTableSnapshotLegacyJsonSerde} to deserialize and wraps + * the result in a {@link LakeTable}. *
  • Version 2 (current): ZK node contains only the lake table snapshot file paths. The actual * snapshot data is stored in a remote file pointed by the lake table snapshot file path. *
@@ -61,6 +61,21 @@ public class LakeTableJsonSerde implements JsonSerializer, JsonDeseri @Override public void serialize(LakeTable lakeTable, JsonGenerator generator) throws IOException { + // if lake table snapshot metadata is null, it must be version 1 + if (lakeTable.getLakeSnapshotMetadatas() == null) { + serializeV1(lakeTable, generator); + } else { + serializeV2(lakeTable, generator); + } + } + + private void serializeV1(LakeTable lakeTable, JsonGenerator generator) throws IOException { + // Version 1: ZK node contains full snapshot data, use LakeTableSnapshotJsonSerde + LakeTableSnapshotLegacyJsonSerde.INSTANCE.serialize( + lakeTable.getOrReadLatestTableSnapshot(), generator); + } + + private void serializeV2(LakeTable lakeTable, JsonGenerator generator) throws IOException { generator.writeStartObject(); generator.writeNumberField(VERSION_KEY, CURRENT_VERSION); @@ -79,7 +94,6 @@ public void serialize(LakeTable lakeTable, JsonGenerator generator) throws IOExc } generator.writeEndObject(); } - generator.writeEndArray(); generator.writeEndObject(); @@ -89,37 +103,42 @@ public void serialize(LakeTable lakeTable, JsonGenerator generator) throws IOExc public LakeTable deserialize(JsonNode node) { int version = node.get(VERSION_KEY).asInt(); if (version == VERSION_1) { - // Version 1: ZK node contains full snapshot data, use LakeTableSnapshotJsonSerde - LakeTableSnapshot snapshot = LakeTableSnapshotJsonSerde.INSTANCE.deserialize(node); - return new LakeTable(snapshot); + return deserializeV1(node); } else if (version == VERSION_2) { - // Version 2: ZK node contains lake snapshot file paths - JsonNode lakeSnapshotsNode = node.get(LAKE_SNAPSHOTS); - if (lakeSnapshotsNode == null || !lakeSnapshotsNode.isArray()) { - throw new IllegalArgumentException( - "Invalid lake_snapshots field in version 2 format"); - } - - List lakeSnapshotMetadatas = new ArrayList<>(); - Iterator elements = lakeSnapshotsNode.elements(); - while (elements.hasNext()) { - JsonNode snapshotNode = elements.next(); - long snapshotId = snapshotNode.get(SNAPSHOT_ID_KEY).asLong(); - String tieredOffsetsPath = snapshotNode.get(TIERED_OFFSETS_KEY).asText(); - JsonNode readableOffsetsNode = snapshotNode.get(READABLE_OFFSETS_KEY); - FsPath readableOffsetsPath = - readableOffsetsNode != null - ? new FsPath(readableOffsetsNode.asText()) - : null; - - LakeTable.LakeSnapshotMetadata metadata = - new LakeTable.LakeSnapshotMetadata( - snapshotId, new FsPath(tieredOffsetsPath), readableOffsetsPath); - lakeSnapshotMetadatas.add(metadata); - } - return new LakeTable(lakeSnapshotMetadatas); + return deserializeV2(node); } else { throw new IllegalArgumentException("Unsupported version: " + version); } } + + private LakeTable deserializeV1(JsonNode node) { + // Version 1: ZK node contains full snapshot data, use LakeTableSnapshotJsonSerde + LakeTableSnapshot snapshot = LakeTableSnapshotLegacyJsonSerde.INSTANCE.deserialize(node); + return new LakeTable(snapshot); + } + + private LakeTable deserializeV2(JsonNode node) { + // Version 2: ZK node contains lake snapshot file paths + JsonNode lakeSnapshotsNode = node.get(LAKE_SNAPSHOTS); + if (lakeSnapshotsNode == null || !lakeSnapshotsNode.isArray()) { + throw new IllegalArgumentException("Invalid lake_snapshots field in version 2 format"); + } + + List lakeSnapshotMetadatas = new ArrayList<>(); + Iterator elements = lakeSnapshotsNode.elements(); + while (elements.hasNext()) { + JsonNode snapshotNode = elements.next(); + long snapshotId = snapshotNode.get(SNAPSHOT_ID_KEY).asLong(); + String tieredOffsetsPath = snapshotNode.get(TIERED_OFFSETS_KEY).asText(); + JsonNode readableOffsetsNode = snapshotNode.get(READABLE_OFFSETS_KEY); + FsPath readableOffsetsPath = + readableOffsetsNode != null ? new FsPath(readableOffsetsNode.asText()) : null; + + LakeTable.LakeSnapshotMetadata metadata = + new LakeTable.LakeSnapshotMetadata( + snapshotId, new FsPath(tieredOffsetsPath), readableOffsetsPath); + lakeSnapshotMetadatas.add(metadata); + } + return new LakeTable(lakeSnapshotMetadatas); + } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableSnapshotJsonSerde.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableSnapshotJsonSerde.java deleted file mode 100644 index 2543dff59e..0000000000 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableSnapshotJsonSerde.java +++ /dev/null @@ -1,312 +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.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/main/java/org/apache/fluss/server/zk/data/lake/LakeTableSnapshotLegacyJsonSerde.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableSnapshotLegacyJsonSerde.java new file mode 100644 index 0000000000..d1e5a52589 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lake/LakeTableSnapshotLegacyJsonSerde.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.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.Map; + +/** + * Json serializer and deserializer for {@link LakeTableSnapshot}. + * + *

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

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

    + *
  • version: 1 + *
  • snapshot_id: the snapshot ID + *
  • table_id: the table ID (derived from the first bucket) + *
  • buckets: array of bucket objects, each containing bucket_id, optional partition_id, and + * log_end_offset + *
+ * + * @see LakeTableJsonSerde for the current format (version 2) that uses this serde for legacy + * compatibility + */ +public class LakeTableSnapshotLegacyJsonSerde + implements JsonSerializer, JsonDeserializer { + + public static final LakeTableSnapshotLegacyJsonSerde INSTANCE = + new LakeTableSnapshotLegacyJsonSerde(); + + 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_END_OFFSET = "log_end_offset"; + + private static final int VERSION_1 = 1; + private static final int CURRENT_VERSION = VERSION_1; + + @Override + public void serialize(LakeTableSnapshot lakeTableSnapshot, JsonGenerator generator) + throws IOException { + generator.writeStartObject(); + generator.writeNumberField(VERSION_KEY, VERSION_1); + generator.writeNumberField(SNAPSHOT_ID, lakeTableSnapshot.getSnapshotId()); + + Map bucketLogEndOffset = lakeTableSnapshot.getBucketLogEndOffset(); + // Get table id from the first table bucket, all buckets should have the same table id + if (!bucketLogEndOffset.isEmpty()) { + TableBucket firstBucket = bucketLogEndOffset.keySet().iterator().next(); + long tableId = firstBucket.getTableId(); + generator.writeNumberField(TABLE_ID, tableId); + } + + generator.writeArrayFieldStart(BUCKETS); + for (Map.Entry tableBucketOffsetEntry : bucketLogEndOffset.entrySet()) { + generator.writeStartObject(); + TableBucket tableBucket = tableBucketOffsetEntry.getKey(); + if (tableBucket.getPartitionId() != null) { + generator.writeNumberField(PARTITION_ID, tableBucket.getPartitionId()); + } + generator.writeNumberField(BUCKET_ID, tableBucket.getBucket()); + generator.writeNumberField(LOG_END_OFFSET, tableBucketOffsetEntry.getValue()); + generator.writeEndObject(); + } + generator.writeEndArray(); + + generator.writeEndObject(); + } + + @Override + public LakeTableSnapshot deserialize(JsonNode node) { + int version = node.get(VERSION_KEY).asInt(); + if (version != CURRENT_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(); + Map bucketLogEndOffset = new HashMap<>(); + for (JsonNode bucket : node.get(BUCKETS)) { + 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); + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TestCoordinatorGateway.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TestCoordinatorGateway.java index c8500645e6..06d15d95ed 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TestCoordinatorGateway.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TestCoordinatorGateway.java @@ -90,6 +90,8 @@ import org.apache.fluss.rpc.messages.ListTablesResponse; import org.apache.fluss.rpc.messages.MetadataRequest; import org.apache.fluss.rpc.messages.MetadataResponse; +import org.apache.fluss.rpc.messages.PrepareLakeTableSnapshotRequest; +import org.apache.fluss.rpc.messages.PrepareLakeTableSnapshotResponse; import org.apache.fluss.rpc.messages.RebalanceRequest; import org.apache.fluss.rpc.messages.RebalanceResponse; import org.apache.fluss.rpc.messages.RemoveServerTagRequest; @@ -337,6 +339,12 @@ public CompletableFuture commitRemoteLogManifes return CompletableFuture.completedFuture(response.setCommitSuccess(true)); } + @Override + public CompletableFuture prepareLakeTableSnapshot( + PrepareLakeTableSnapshotRequest request) { + throw new UnsupportedOperationException(); + } + @Override public CompletableFuture commitLakeTableSnapshot( CommitLakeTableSnapshotRequest request) { 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 deleted file mode 100644 index e00f349021..0000000000 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/LakeTableSnapshotJsonSerdeTest.java +++ /dev/null @@ -1,144 +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.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 { - - LakeTableSnapshotJsonSerdeTest() { - super(LakeTableSnapshotJsonSerde.INSTANCE); - } - - @Override - protected LakeTableSnapshot[] createObjects() { - // Test case 1: Empty snapshot - LakeTableSnapshot lakeTableSnapshot1 = new LakeTableSnapshot(1L, Collections.emptyMap()); - - // Test case 2: Non-partition table with consecutive bucket ids (0, 1, 2) - long tableId = 4; - Map bucketLogEndOffset = new HashMap<>(); - bucketLogEndOffset.put(new TableBucket(tableId, 0), 100L); - bucketLogEndOffset.put(new TableBucket(tableId, 1), 200L); - bucketLogEndOffset.put(new TableBucket(tableId, 2), 300L); - 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; - 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, 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, - 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[] { - // 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/LakeTableSnapshotLegacyJsonSerdeTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/LakeTableSnapshotLegacyJsonSerdeTest.java new file mode 100644 index 0000000000..369a53062d --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/LakeTableSnapshotLegacyJsonSerdeTest.java @@ -0,0 +1,107 @@ +/* + * 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.server.zk.data.lake.LakeTableSnapshot; +import org.apache.fluss.server.zk.data.lake.LakeTableSnapshotLegacyJsonSerde; +import org.apache.fluss.utils.json.JsonSerdeTestBase; +import org.apache.fluss.utils.json.JsonSerdeUtils; + +import org.junit.jupiter.api.Test; + +import java.nio.charset.StandardCharsets; +import java.util.HashMap; +import java.util.Map; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link LakeTableSnapshotLegacyJsonSerde}. */ +class LakeTableSnapshotLegacyJsonSerdeTest extends JsonSerdeTestBase { + + LakeTableSnapshotLegacyJsonSerdeTest() { + super(LakeTableSnapshotLegacyJsonSerde.INSTANCE); + } + + @Override + protected LakeTableSnapshot[] createObjects() { + + long tableId = 4; + + Map bucketLogEndOffset = new HashMap<>(); + bucketLogEndOffset.put(new TableBucket(tableId, 1), 3L); + bucketLogEndOffset.put(new TableBucket(tableId, 2), 4L); + + LakeTableSnapshot lakeTableSnapshot1 = new LakeTableSnapshot(2, bucketLogEndOffset); + + tableId = 5; + + bucketLogEndOffset = new HashMap<>(); + bucketLogEndOffset.put(new TableBucket(tableId, 1L, 1), 3L); + bucketLogEndOffset.put(new TableBucket(tableId, 2L, 1), 4L); + + LakeTableSnapshot lakeTableSnapshot2 = new LakeTableSnapshot(3, bucketLogEndOffset); + + return new LakeTableSnapshot[] { + lakeTableSnapshot1, lakeTableSnapshot2, + }; + } + + @Override + protected String[] expectedJsons() { + return new String[] { + "{\"version\":1,\"snapshot_id\":2,\"table_id\":4," + + "\"buckets\":[{\"bucket_id\":2,\"log_end_offset\":4}," + + "{\"bucket_id\":1,\"log_end_offset\":3}]}", + "{\"version\":1,\"snapshot_id\":3,\"table_id\":5," + + "\"buckets\":[{\"partition_id\":1,\"bucket_id\":1,\"log_end_offset\":3}," + + "{\"partition_id\":2,\"bucket_id\":1,\"log_end_offset\":4}]}" + }; + } + + @Test + void testBackwardCompatibility() { + // Test that Version 1 format can still be deserialized + String version1Json1 = "{\"version\":1,\"snapshot_id\":1,\"table_id\":1,\"buckets\":[]}"; + LakeTableSnapshot snapshot1 = + JsonSerdeUtils.readValue( + version1Json1.getBytes(StandardCharsets.UTF_8), + LakeTableSnapshotLegacyJsonSerde.INSTANCE); + assertThat(snapshot1.getSnapshotId()).isEqualTo(1); + assertThat(snapshot1.getBucketLogEndOffset()).isEmpty(); + + String version1Json2 = + "{\"version\":1,\"snapshot_id\":2,\"table_id\":4," + + "\"buckets\":[{\"bucket_id\":1,\"log_start_offset\":1,\"log_end_offset\":3,\"max_timestamp\":5}]}"; + LakeTableSnapshot snapshot2 = + JsonSerdeUtils.readValue( + version1Json2.getBytes(StandardCharsets.UTF_8), + LakeTableSnapshotLegacyJsonSerde.INSTANCE); + assertThat(snapshot2.getSnapshotId()).isEqualTo(2); + assertThat(snapshot2.getBucketLogEndOffset()).hasSize(1); + + String version1Json3 = + "{\"version\":1,\"snapshot_id\":3,\"table_id\":5," + + "\"buckets\":[{\"partition_id\":1,\"partition_name\":\"partition1\",\"bucket_id\":1,\"log_start_offset\":1,\"log_end_offset\":3,\"max_timestamp\":5}]}"; + LakeTableSnapshot snapshot3 = + JsonSerdeUtils.readValue( + version1Json3.getBytes(StandardCharsets.UTF_8), + LakeTableSnapshotLegacyJsonSerde.INSTANCE); + assertThat(snapshot3.getSnapshotId()).isEqualTo(3); + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableHelperTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableHelperTest.java index 6dd8497d58..f55c1b94ad 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableHelperTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableHelperTest.java @@ -30,8 +30,8 @@ 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.apache.fluss.utils.json.TableBucketOffsets; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; @@ -76,7 +76,7 @@ static void afterAll() { } @Test - void testUpsertLakeTableCompatible(@TempDir Path tempDir) throws Exception { + void testRegisterLakeTableSnapshotCompatibility(@TempDir Path tempDir) throws Exception { // Create a ZooKeeperClient with REMOTE_DATA_DIR configuration Configuration conf = new Configuration(); conf.setString( @@ -110,21 +110,14 @@ void testUpsertLakeTableCompatible(@TempDir Path tempDir) throws Exception { LakeTableSnapshot lakeTableSnapshot = new LakeTableSnapshot(snapshotId, bucketLogEndOffset); - // Write version 1 format data directly to ZK (simulating old system behavior) - String zkPath = ZkData.LakeTableZNode.path(tableId); - byte[] version1Data = - LakeTableSnapshotJsonSerde.toJsonVersion1(lakeTableSnapshot, tableId); - zooKeeperClient - .getCuratorClient() - .create() - .creatingParentsIfNeeded() - .forPath(zkPath, version1Data); + // Write version 1 format data(simulating old system behavior) + lakeTableHelper.registerLakeTableSnapshotV1(tableId, lakeTableSnapshot); // Verify version 1 data can be read Optional optionalLakeTable = zooKeeperClient.getLakeTable(tableId); assertThat(optionalLakeTable).isPresent(); LakeTable lakeTable = optionalLakeTable.get(); - assertThat(lakeTable.getLatestTableSnapshot()).isEqualTo(lakeTableSnapshot); + assertThat(lakeTable.getOrReadLatestTableSnapshot()).isEqualTo(lakeTableSnapshot); // Test: Call upsertLakeTableSnapshot with new snapshot data // This should read the old version 1 data, merge it, and write as version 2 @@ -133,8 +126,13 @@ void testUpsertLakeTableCompatible(@TempDir Path tempDir) throws Exception { newBucketLogEndOffset.put(new TableBucket(tableId, 1), 2000L); // new offset long snapshot2Id = 2L; - LakeTableSnapshot snapshot2 = new LakeTableSnapshot(snapshot2Id, newBucketLogEndOffset); - lakeTableHelper.upsertLakeTable(tableId, tablePath, snapshot2); + FsPath tieredOffsetsPath = + lakeTableHelper.storeLakeTableOffsetsFile( + tablePath, new TableBucketOffsets(tableId, newBucketLogEndOffset)); + lakeTableHelper.registerLakeTableSnapshotV2( + tableId, + new LakeTable.LakeSnapshotMetadata( + snapshot2Id, tieredOffsetsPath, tieredOffsetsPath)); // Verify: New version 2 data can be read Optional optLakeTableAfter = zooKeeperClient.getLakeTable(tableId); @@ -163,8 +161,13 @@ void testUpsertLakeTableCompatible(@TempDir Path tempDir) throws Exception { // add a new snapshot 3 again, verify snapshot long snapshot3Id = 3L; - LakeTableSnapshot snapshot3 = new LakeTableSnapshot(snapshot3Id, newBucketLogEndOffset); - lakeTableHelper.upsertLakeTable(tableId, tablePath, snapshot3); + tieredOffsetsPath = + lakeTableHelper.storeLakeTableOffsetsFile( + tablePath, new TableBucketOffsets(tableId, newBucketLogEndOffset)); + lakeTableHelper.registerLakeTableSnapshotV2( + tableId, + new LakeTable.LakeSnapshotMetadata( + snapshot3Id, tieredOffsetsPath, tieredOffsetsPath)); // 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 index c2598e41c0..2dbc2280cf 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerdeTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableJsonSerdeTest.java @@ -18,11 +18,21 @@ package org.apache.fluss.server.zk.data.lake; import org.apache.fluss.fs.FsPath; +import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.utils.json.JsonSerdeTestBase; +import org.apache.fluss.utils.json.JsonSerdeUtils; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; + +import static org.assertj.core.api.Assertions.assertThat; /** Test for {@link LakeTableJsonSerde}. */ class LakeTableJsonSerdeTest extends JsonSerdeTestBase { @@ -33,21 +43,21 @@ class LakeTableJsonSerdeTest extends JsonSerdeTestBase { @Override protected LakeTable[] createObjects() { - // Test case 1: Empty lake snapshots list + // Test case 1: Empty lake snapshots list (v2) LakeTable lakeTable1 = new LakeTable(Collections.emptyList()); - // Test case 2: Single snapshot metadata with readable offsets + // Test case 2: Single snapshot metadata with readable offsets (v2) LakeTable.LakeSnapshotMetadata metadata1 = new LakeTable.LakeSnapshotMetadata( 1L, new FsPath("/path/to/tiered1"), new FsPath("/path/to/readable1")); LakeTable lakeTable2 = new LakeTable(Collections.singletonList(metadata1)); - // Test case 3: Single snapshot metadata without readable offsets + // Test case 3: Single snapshot metadata without readable offsets (v2) LakeTable.LakeSnapshotMetadata metadata2 = new LakeTable.LakeSnapshotMetadata(2L, new FsPath("/path/to/tiered2"), null); LakeTable lakeTable3 = new LakeTable(Collections.singletonList(metadata2)); - // Test case 4: Multiple snapshot metadata + // Test case 4: Multiple snapshot metadata (v2) List metadatas = new ArrayList<>(); metadatas.add( new LakeTable.LakeSnapshotMetadata( @@ -58,20 +68,82 @@ protected LakeTable[] createObjects() { metadatas.add(new LakeTable.LakeSnapshotMetadata(5L, new FsPath("/path/to/tiered5"), null)); LakeTable lakeTable4 = new LakeTable(metadatas); - return new LakeTable[] {lakeTable1, lakeTable2, lakeTable3, lakeTable4}; + // Test case 5: Version 1 format - non-partition table + long tableId = 4; + Map bucketLogEndOffset = new HashMap<>(); + bucketLogEndOffset.put(new TableBucket(tableId, 0), 100L); + bucketLogEndOffset.put(new TableBucket(tableId, 1), 200L); + LakeTableSnapshot snapshot1 = new LakeTableSnapshot(10L, bucketLogEndOffset); + LakeTable lakeTable5 = new LakeTable(snapshot1); + + // Test case 6: Version 1 format - partition table + tableId = 5; + bucketLogEndOffset = new HashMap<>(); + bucketLogEndOffset.put(new TableBucket(tableId, 1L, 0), 100L); + bucketLogEndOffset.put(new TableBucket(tableId, 1L, 1), 200L); + bucketLogEndOffset.put(new TableBucket(tableId, 2L, 0), 300L); + LakeTableSnapshot snapshot2 = new LakeTableSnapshot(11L, bucketLogEndOffset); + LakeTable lakeTable6 = new LakeTable(snapshot2); + + return new LakeTable[] { + lakeTable1, lakeTable2, lakeTable3, lakeTable4, lakeTable5, lakeTable6 + }; } @Override protected String[] expectedJsons() { return new String[] { - // Test case 1: Empty lake snapshots list + // Test case 1: Empty lake snapshots list (v2) "{\"version\":2,\"lake_snapshots\":[]}", - // Test case 2: Single snapshot metadata with readable offsets + // Test case 2: Single snapshot metadata with readable offsets (v2) "{\"version\":2,\"lake_snapshots\":[{\"snapshot_id\":1,\"tiered_offsets\":\"/path/to/tiered1\",\"readable_offsets\":\"/path/to/readable1\"}]}", - // Test case 3: Single snapshot metadata without readable offsets + // Test case 3: Single snapshot metadata without readable offsets (v2) "{\"version\":2,\"lake_snapshots\":[{\"snapshot_id\":2,\"tiered_offsets\":\"/path/to/tiered2\"}]}", - // Test case 4: Multiple snapshot metadata - "{\"version\":2,\"lake_snapshots\":[{\"snapshot_id\":3,\"tiered_offsets\":\"/path/to/tiered3\",\"readable_offsets\":\"/path/to/readable3\"},{\"snapshot_id\":4,\"tiered_offsets\":\"/path/to/tiered4\",\"readable_offsets\":\"/path/to/readable4\"},{\"snapshot_id\":5,\"tiered_offsets\":\"/path/to/tiered5\"}]}" + // Test case 4: Multiple snapshot metadata (v2) + "{\"version\":2,\"lake_snapshots\":[{\"snapshot_id\":3,\"tiered_offsets\":\"/path/to/tiered3\",\"readable_offsets\":\"/path/to/readable3\"},{\"snapshot_id\":4,\"tiered_offsets\":\"/path/to/tiered4\",\"readable_offsets\":\"/path/to/readable4\"},{\"snapshot_id\":5,\"tiered_offsets\":\"/path/to/tiered5\"}]}", + // Test case 5: Version 1 format - non-partition table + "{\"version\":1,\"snapshot_id\":10,\"table_id\":4,\"buckets\":[{\"bucket_id\":1,\"log_end_offset\":200},{\"bucket_id\":0,\"log_end_offset\":100}]}", + // Test case 6: Version 1 format - partition table + "{\"version\":1,\"snapshot_id\":11,\"table_id\":5,\"buckets\":[{\"partition_id\":1,\"bucket_id\":1,\"log_end_offset\":200},{\"partition_id\":1,\"bucket_id\":0,\"log_end_offset\":100},{\"partition_id\":2,\"bucket_id\":0,\"log_end_offset\":300}]}", }; } + + @Test + void testVersion1Compatibility() throws IOException { + // Test that Version 1 format can be deserialized correctly + // Test case 1: Non-partition table + String version1Json1 = + "{\"version\":1,\"snapshot_id\":10,\"table_id\":4," + + "\"buckets\":[{\"bucket_id\":0,\"log_end_offset\":100},{\"bucket_id\":1,\"log_end_offset\":200}]}"; + LakeTable actual1 = + JsonSerdeUtils.readValue( + version1Json1.getBytes(StandardCharsets.UTF_8), + LakeTableJsonSerde.INSTANCE); + + // Create expected LakeTableSnapshot + Map expectedBuckets1 = new HashMap<>(); + expectedBuckets1.put(new TableBucket(4L, 0), 100L); + expectedBuckets1.put(new TableBucket(4L, 1), 200L); + LakeTableSnapshot expectedSnapshot1 = new LakeTableSnapshot(10L, expectedBuckets1); + assertThat(actual1.getOrReadLatestTableSnapshot()).isEqualTo(expectedSnapshot1); + + // Test case 2: Partition table + String version1Json2 = + "{\"version\":1,\"snapshot_id\":11,\"table_id\":5," + + "\"buckets\":[{\"partition_id\":1,\"bucket_id\":0,\"log_end_offset\":100}," + + "{\"partition_id\":1,\"bucket_id\":1,\"log_end_offset\":200}," + + "{\"partition_id\":2,\"bucket_id\":0,\"log_end_offset\":300}]}"; + LakeTable actual2 = + JsonSerdeUtils.readValue( + version1Json2.getBytes(StandardCharsets.UTF_8), + LakeTableJsonSerde.INSTANCE); + + // Create expected LakeTableSnapshot + Map expectedBuckets2 = new HashMap<>(); + expectedBuckets2.put(new TableBucket(5L, 1L, 0), 100L); + expectedBuckets2.put(new TableBucket(5L, 1L, 1), 200L); + expectedBuckets2.put(new TableBucket(5L, 2L, 0), 300L); + LakeTableSnapshot expectedSnapshot2 = new LakeTableSnapshot(11L, expectedBuckets2); + assertThat(actual2.getOrReadLatestTableSnapshot()).isEqualTo(expectedSnapshot2); + } }