From 99a228b7f9b00ba04878b4751a03e06ed6da8459 Mon Sep 17 00:00:00 2001 From: yunhong <337361684@qq.com> Date: Mon, 15 Dec 2025 13:57:27 +0800 Subject: [PATCH 1/6] [kv] Support kv snapshot lease --- .../org/apache/fluss/client/admin/Admin.java | 53 ++ .../apache/fluss/client/admin/FlussAdmin.java | 33 ++ .../AcquireKvSnapshotLeaseResult.java | 55 ++ .../fluss/client/metadata/KvSnapshots.java | 8 + .../client/utils/ClientRpcMessageUtils.java | 76 +++ .../metadata/TestingClientSchemaGetter.java | 6 +- .../batch/KvSnapshotBatchScannerITCase.java | 115 +++++ .../table/scanner/log/LogFetcherTest.java | 5 +- .../apache/fluss/config/ConfigOptions.java | 10 +- .../KvSnapshotLeaseNotExistException.java | 31 ++ .../metadata/KvSnapshotLeaseForBucket.java | 66 +++ .../org/apache/fluss/metrics/MetricNames.java | 5 + .../org/apache/fluss/utils/FlussPaths.java | 37 ++ .../org/apache/fluss/record/TestData.java | 14 + .../fluss/flink/FlinkConnectorOptions.java | 18 + .../flink/catalog/FlinkTableFactory.java | 15 +- .../flink/procedure/ProcedureManager.java | 4 +- .../ReleaseAllKvSnapshotLeaseProcedure.java | 46 ++ .../fluss/flink/source/FlinkSource.java | 18 +- .../fluss/flink/source/FlinkTableSource.java | 13 +- .../fluss/flink/source/FlussSource.java | 7 +- .../flink/source/FlussSourceBuilder.java | 18 +- .../enumerator/FlinkSourceEnumerator.java | 184 +++++-- .../event/FinishedKvSnapshotConsumeEvent.java | 74 +++ .../source/reader/FlinkSourceReader.java | 51 ++ .../flink/source/reader/LeaseContext.java | 79 +++ .../FlussSourceEnumeratorStateSerializer.java | 68 ++- .../source/state/SourceEnumeratorState.java | 17 +- .../flink/procedure/FlinkProcedureITCase.java | 43 +- .../flink/source/FlinkTableSourceITCase.java | 47 ++ .../fluss/flink/source/FlussSourceITCase.java | 7 + .../enumerator/FlinkSourceEnumeratorTest.java | 34 +- .../SourceEnumeratorStateSerializerTest.java | 28 +- .../fluss/rpc/gateway/AdminGateway.java | 12 + .../apache/fluss/rpc/protocol/ApiKeys.java | 4 +- .../org/apache/fluss/rpc/protocol/Errors.java | 5 +- fluss-rpc/src/main/proto/FlussApi.proto | 44 ++ .../CompletedSnapshotStoreManager.java | 15 +- .../CoordinatorEventProcessor.java | 93 +++- .../server/coordinator/CoordinatorServer.java | 11 +- .../coordinator/CoordinatorService.java | 37 ++ .../coordinator/KvSnapshotLeaseManager.java | 459 +++++++++++++++++ .../event/AcquireKvSnapshotLeaseEvent.java | 60 +++ .../event/ReleaseKvSnapshotLeaseEvent.java | 53 ++ .../kv/snapshot/CompletedSnapshotStore.java | 33 +- .../server/utils/ServerRpcMessageUtils.java | 49 ++ .../fluss/server/zk/ZooKeeperClient.java | 33 ++ .../apache/fluss/server/zk/data/ZkData.java | 36 ++ .../server/zk/data/lease/KvSnapshotLease.java | 198 ++++++++ .../data/lease/KvSnapshotLeaseMetadata.java | 94 ++++ .../KvSnapshotLeaseMetadataJsonSerde.java | 84 ++++ .../lease/KvSnapshotLeaseMetadataManager.java | 226 +++++++++ .../zk/data/lease/KvSnapshotTableLease.java | 188 +++++++ .../lease/KvSnapshotTableLeaseJsonSerde.java | 116 +++++ .../CompletedSnapshotStoreManagerTest.java | 6 +- .../CoordinatorEventProcessorTest.java | 4 +- .../KvSnapshotLeaseManagerTest.java | 470 ++++++++++++++++++ .../coordinator/TestCoordinatorGateway.java | 16 + .../rebalance/RebalanceManagerTest.java | 4 +- .../TableBucketStateMachineTest.java | 4 +- .../snapshot/CompletedSnapshotStoreTest.java | 3 +- .../snapshot/KvTabletSnapshotTargetTest.java | 3 +- .../ZooKeeperCompletedSnapshotStoreTest.java | 3 +- .../testutils/FlussClusterExtension.java | 2 +- .../fluss/server/zk/ZooKeeperClientTest.java | 42 ++ .../KvSnapshotLeaseMetadataJsonSerdeTest.java | 57 +++ .../KvSnapshotLeaseMetadataManagerTest.java | 146 ++++++ .../zk/data/lease/KvSnapshotLeaseTest.java | 207 ++++++++ .../KvSnapshotTableLeaseJsonSerdeTest.java | 54 ++ fluss-test-coverage/pom.xml | 3 + website/docs/engine-flink/options.md | 30 +- website/docs/engine-flink/procedures.md | 34 ++ website/docs/maintenance/configuration.md | 3 +- .../observability/monitor-metrics.md | 12 +- 74 files changed, 4127 insertions(+), 111 deletions(-) create mode 100644 fluss-client/src/main/java/org/apache/fluss/client/metadata/AcquireKvSnapshotLeaseResult.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/exception/KvSnapshotLeaseNotExistException.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/metadata/KvSnapshotLeaseForBucket.java create mode 100644 fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/ReleaseAllKvSnapshotLeaseProcedure.java create mode 100644 fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/event/FinishedKvSnapshotConsumeEvent.java create mode 100644 fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/reader/LeaseContext.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/KvSnapshotLeaseManager.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/AcquireKvSnapshotLeaseEvent.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/ReleaseKvSnapshotLeaseEvent.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLease.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadata.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataJsonSerde.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataManager.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLease.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLeaseJsonSerde.java create mode 100644 fluss-server/src/test/java/org/apache/fluss/server/coordinator/KvSnapshotLeaseManagerTest.java create mode 100644 fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataJsonSerdeTest.java create mode 100644 fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataManagerTest.java create mode 100644 fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseTest.java create mode 100644 fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLeaseJsonSerdeTest.java diff --git a/fluss-client/src/main/java/org/apache/fluss/client/admin/Admin.java b/fluss-client/src/main/java/org/apache/fluss/client/admin/Admin.java index b6e9ea1d69..12d9a9eeb2 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/admin/Admin.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/admin/Admin.java @@ -18,6 +18,7 @@ package org.apache.fluss.client.admin; import org.apache.fluss.annotation.PublicEvolving; +import org.apache.fluss.client.metadata.AcquireKvSnapshotLeaseResult; import org.apache.fluss.client.metadata.KvSnapshotMetadata; import org.apache.fluss.client.metadata.KvSnapshots; import org.apache.fluss.client.metadata.LakeSnapshot; @@ -74,6 +75,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.concurrent.CompletableFuture; /** @@ -428,6 +430,57 @@ CompletableFuture dropPartition( CompletableFuture getKvSnapshotMetadata( TableBucket bucket, long snapshotId); + /** + * Acquires a lease for specific KV snapshots of the given tableBuckets asynchronously. + * + *

Once acquired, the specified KV snapshots will be protected from garbage collection for + * the duration of the {@code leaseDuration}. The client must call {@link + * #releaseKvSnapshotLease} to release the lock early when reading is finished. + * + *

If the lease expires (no renew received within duration), the server is free to delete the + * snapshot files. + * + *

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

+ * + * @param leaseId The unique ID for this lease session (usually a UUID generated by client). + * @param snapshotIds The snapshots to lease, a map from TableBucket to kvSnapshotId. + * @param leaseDuration The duration (in milliseconds) for which the snapshots should be kept. + * @return The result of the acquire operation, containing any buckets that failed to be locked. + */ + CompletableFuture acquireKvSnapshotLease( + String leaseId, Map snapshotIds, long leaseDuration); + + /** + * Releases the lease for specific tableBuckets asynchronously. + * + *

This is typically called when a client finishes reading a specific bucket (or a batch of + * buckets) but is still reading others under the same leaseId. + * + *

If {@code bucketsToRelease} contains all buckets under this leaseId, the lease itself will + * be removed. + * + * @param leaseId The lease id. + * @param bucketsToRelease The specific tableBuckets to release. + */ + CompletableFuture releaseKvSnapshotLease( + String leaseId, Set bucketsToRelease); + + /** + * Releases the entire lease asynchronously for all leased snapshots for all table buckets of + * specified leaseId. + * + *

All snapshots locked under this {@code leaseId} will be released immediately. This is + * equivalent to calling {@link #releaseKvSnapshotLease} with all held buckets. + * + * @param leaseId The lease id to release. + */ + CompletableFuture releaseAllKvSnapshotLease(String leaseId); + /** * Get table lake snapshot info of the given table asynchronously. * diff --git a/fluss-client/src/main/java/org/apache/fluss/client/admin/FlussAdmin.java b/fluss-client/src/main/java/org/apache/fluss/client/admin/FlussAdmin.java index 9df4bf248f..79035f14f9 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/admin/FlussAdmin.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/admin/FlussAdmin.java @@ -18,6 +18,7 @@ package org.apache.fluss.client.admin; import org.apache.fluss.annotation.VisibleForTesting; +import org.apache.fluss.client.metadata.AcquireKvSnapshotLeaseResult; import org.apache.fluss.client.metadata.KvSnapshotMetadata; import org.apache.fluss.client.metadata.KvSnapshots; import org.apache.fluss.client.metadata.LakeSnapshot; @@ -84,6 +85,7 @@ import org.apache.fluss.rpc.messages.PbTablePath; import org.apache.fluss.rpc.messages.RebalanceRequest; import org.apache.fluss.rpc.messages.RebalanceResponse; +import org.apache.fluss.rpc.messages.ReleaseKvSnapshotLeaseRequest; import org.apache.fluss.rpc.messages.RemoveServerTagRequest; import org.apache.fluss.rpc.messages.TableExistsRequest; import org.apache.fluss.rpc.messages.TableExistsResponse; @@ -101,14 +103,17 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.concurrent.CompletableFuture; +import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makeAcquireKvSnapshotLeaseRequest; import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makeAlterTableRequest; import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makeCreatePartitionRequest; import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makeDropPartitionRequest; import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makeListOffsetsRequest; import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makePbPartitionSpec; import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makeRegisterProducerOffsetsRequest; +import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makeReleaseKvSnapshotLeaseRequest; import static org.apache.fluss.client.utils.ClientRpcMessageUtils.toConfigEntries; import static org.apache.fluss.client.utils.MetadataUtils.sendMetadataRequestAndRebuildCluster; import static org.apache.fluss.rpc.util.CommonRpcMessageUtils.toAclBindings; @@ -400,6 +405,34 @@ public CompletableFuture getKvSnapshotMetadata( .thenApply(ClientRpcMessageUtils::toKvSnapshotMetadata); } + @Override + public CompletableFuture acquireKvSnapshotLease( + String leaseId, Map snapshotIds, long leaseDuration) { + if (snapshotIds.isEmpty()) { + throw new IllegalArgumentException( + "The snapshotIds to acquire kv snapshot lease is empty"); + } + + return gateway.acquireKvSnapshotLease( + makeAcquireKvSnapshotLeaseRequest(leaseId, snapshotIds, leaseDuration)) + .thenApply(ClientRpcMessageUtils::toAcquireKvSnapshotLeaseResult); + } + + @Override + public CompletableFuture releaseKvSnapshotLease( + String leaseId, Set bucketsToRelease) { + return gateway.releaseKvSnapshotLease( + makeReleaseKvSnapshotLeaseRequest(leaseId, bucketsToRelease)) + .thenApply(r -> null); + } + + @Override + public CompletableFuture releaseAllKvSnapshotLease(String leaseId) { + ReleaseKvSnapshotLeaseRequest request = + new ReleaseKvSnapshotLeaseRequest().setLeaseId(leaseId); + return gateway.releaseKvSnapshotLease(request).thenApply(r -> null); + } + @Override public CompletableFuture getLatestLakeSnapshot(TablePath tablePath) { GetLatestLakeSnapshotRequest request = new GetLatestLakeSnapshotRequest(); diff --git a/fluss-client/src/main/java/org/apache/fluss/client/metadata/AcquireKvSnapshotLeaseResult.java b/fluss-client/src/main/java/org/apache/fluss/client/metadata/AcquireKvSnapshotLeaseResult.java new file mode 100644 index 0000000000..3852acd78c --- /dev/null +++ b/fluss-client/src/main/java/org/apache/fluss/client/metadata/AcquireKvSnapshotLeaseResult.java @@ -0,0 +1,55 @@ +/* + * 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.client.metadata; + +import org.apache.fluss.annotation.PublicEvolving; +import org.apache.fluss.metadata.TableBucket; + +import java.util.Map; +import java.util.Set; + +/** + * A class to represent the result of acquire kv snapshot lease. It contains: + * + *

    + *
  • A map of unavailable snapshots. Such as the specify snapshotId is not exist for this table + * bucket. + *
+ * + * @since 0.9 + */ +@PublicEvolving +public class AcquireKvSnapshotLeaseResult { + private final Map unavailableSnapshots; + + public AcquireKvSnapshotLeaseResult(Map unavailableSnapshots) { + this.unavailableSnapshots = unavailableSnapshots; + } + + /** + * Returns the set of buckets that could not be locked (e.g., snapshot ID doesn't exist or has + * already been GC'ed). + */ + public Map getUnavailableSnapshots() { + return unavailableSnapshots; + } + + public Set getUnavailableTableBucketSet() { + return unavailableSnapshots.keySet(); + } +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/metadata/KvSnapshots.java b/fluss-client/src/main/java/org/apache/fluss/client/metadata/KvSnapshots.java index 9f01ed8f19..91428dcb5e 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/metadata/KvSnapshots.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/metadata/KvSnapshots.java @@ -18,12 +18,14 @@ package org.apache.fluss.client.metadata; import org.apache.fluss.annotation.PublicEvolving; +import org.apache.fluss.metadata.TableBucket; import javax.annotation.Nullable; import java.util.Map; import java.util.OptionalLong; import java.util.Set; +import java.util.stream.Collectors; /** * A class representing the kv snapshots of a table or a partition. It contains multiple snapshots @@ -71,6 +73,12 @@ public Set getBucketIds() { return snapshotIds.keySet(); } + public Set getTableBuckets() { + return snapshotIds.keySet().stream() + .map(bucketId -> new TableBucket(tableId, partitionId, bucketId)) + .collect(Collectors.toSet()); + } + /** * Get the latest snapshot id for this kv tablet (bucket), or empty if there are no snapshots. */ diff --git a/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java b/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java index 1748e0e43a..34de4e9782 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java @@ -21,6 +21,7 @@ import org.apache.fluss.client.admin.ProducerOffsetsResult; import org.apache.fluss.client.lookup.LookupBatch; import org.apache.fluss.client.lookup.PrefixLookupBatch; +import org.apache.fluss.client.metadata.AcquireKvSnapshotLeaseResult; import org.apache.fluss.client.metadata.KvSnapshotMetadata; import org.apache.fluss.client.metadata.KvSnapshots; import org.apache.fluss.client.metadata.LakeSnapshot; @@ -43,6 +44,8 @@ import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableChange; import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.rpc.messages.AcquireKvSnapshotLeaseRequest; +import org.apache.fluss.rpc.messages.AcquireKvSnapshotLeaseResponse; import org.apache.fluss.rpc.messages.AlterTableRequest; import org.apache.fluss.rpc.messages.CreatePartitionRequest; import org.apache.fluss.rpc.messages.DropPartitionRequest; @@ -59,12 +62,15 @@ import org.apache.fluss.rpc.messages.MetadataRequest; import org.apache.fluss.rpc.messages.PbAddColumn; import org.apache.fluss.rpc.messages.PbAlterConfig; +import org.apache.fluss.rpc.messages.PbBucket; import org.apache.fluss.rpc.messages.PbBucketOffset; import org.apache.fluss.rpc.messages.PbDatabaseSummary; import org.apache.fluss.rpc.messages.PbDescribeConfig; import org.apache.fluss.rpc.messages.PbDropColumn; import org.apache.fluss.rpc.messages.PbKeyValue; import org.apache.fluss.rpc.messages.PbKvSnapshot; +import org.apache.fluss.rpc.messages.PbKvSnapshotLeaseForBucket; +import org.apache.fluss.rpc.messages.PbKvSnapshotLeaseForTable; import org.apache.fluss.rpc.messages.PbLakeSnapshotForBucket; import org.apache.fluss.rpc.messages.PbLookupReqForBucket; import org.apache.fluss.rpc.messages.PbModifyColumn; @@ -82,6 +88,7 @@ import org.apache.fluss.rpc.messages.ProduceLogRequest; import org.apache.fluss.rpc.messages.PutKvRequest; import org.apache.fluss.rpc.messages.RegisterProducerOffsetsRequest; +import org.apache.fluss.rpc.messages.ReleaseKvSnapshotLeaseRequest; import org.apache.fluss.rpc.protocol.MergeMode; import org.apache.fluss.utils.json.DataTypeJsonSerde; import org.apache.fluss.utils.json.JsonSerdeUtils; @@ -411,6 +418,75 @@ public static AlterTableRequest makeAlterTableRequest( return request; } + public static AcquireKvSnapshotLeaseRequest makeAcquireKvSnapshotLeaseRequest( + String leaseId, Map snapshotIds, long leaseDuration) { + AcquireKvSnapshotLeaseRequest request = new AcquireKvSnapshotLeaseRequest(); + request.setLeaseId(leaseId).setLeaseDuration(leaseDuration); + + Map> pbLeaseForTables = new HashMap<>(); + for (Map.Entry entry : snapshotIds.entrySet()) { + TableBucket tableBucket = entry.getKey(); + Long snapshotId = entry.getValue(); + PbKvSnapshotLeaseForBucket pbLeaseForBucket = + new PbKvSnapshotLeaseForBucket() + .setBucketId(tableBucket.getBucket()) + .setSnapshotId(snapshotId); + if (tableBucket.getPartitionId() != null) { + pbLeaseForBucket.setPartitionId(tableBucket.getPartitionId()); + } + pbLeaseForTables + .computeIfAbsent(tableBucket.getTableId(), k -> new ArrayList<>()) + .add(pbLeaseForBucket); + } + + for (Map.Entry> entry : + pbLeaseForTables.entrySet()) { + request.addTableLeaseReq() + .setTableId(entry.getKey()) + .addAllBucketsReqs(entry.getValue()); + } + return request; + } + + public static AcquireKvSnapshotLeaseResult toAcquireKvSnapshotLeaseResult( + AcquireKvSnapshotLeaseResponse response) { + Map unavailableSnapshots = new HashMap<>(); + for (PbKvSnapshotLeaseForTable leaseForTable : response.getTablesLeaseResList()) { + long tableId = leaseForTable.getTableId(); + for (PbKvSnapshotLeaseForBucket leaseForBucket : leaseForTable.getBucketsReqsList()) { + TableBucket tableBucket = + new TableBucket( + tableId, + leaseForBucket.hasPartitionId() + ? leaseForBucket.getPartitionId() + : null, + leaseForBucket.getBucketId()); + unavailableSnapshots.put(tableBucket, leaseForBucket.getSnapshotId()); + } + } + return new AcquireKvSnapshotLeaseResult(unavailableSnapshots); + } + + public static ReleaseKvSnapshotLeaseRequest makeReleaseKvSnapshotLeaseRequest( + String leaseId, Set bucketsToRelease) { + ReleaseKvSnapshotLeaseRequest request = new ReleaseKvSnapshotLeaseRequest(); + request.setLeaseId(leaseId); + + Map> pbLeasedTable = new HashMap<>(); + for (TableBucket tb : bucketsToRelease) { + PbBucket pbBucket = new PbBucket().setBucketId(tb.getBucket()); + if (tb.getPartitionId() != null) { + pbBucket.setPartitionId(tb.getPartitionId()); + } + pbLeasedTable.computeIfAbsent(tb.getTableId(), k -> new ArrayList<>()).add(pbBucket); + } + + for (Map.Entry> entry : pbLeasedTable.entrySet()) { + request.addReleaseTable().setTableId(entry.getKey()).addAllBuckets(entry.getValue()); + } + return request; + } + public static Optional toRebalanceProgress( ListRebalanceProgressResponse response) { if (!response.hasRebalanceId()) { diff --git a/fluss-client/src/test/java/org/apache/fluss/client/metadata/TestingClientSchemaGetter.java b/fluss-client/src/test/java/org/apache/fluss/client/metadata/TestingClientSchemaGetter.java index a983eea11e..ffb2c54dc2 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/metadata/TestingClientSchemaGetter.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/metadata/TestingClientSchemaGetter.java @@ -32,13 +32,13 @@ public class TestingClientSchemaGetter extends ClientSchemaGetter { public TestingClientSchemaGetter( TablePath tablePath, SchemaInfo latestSchemaInfo, - TestingMetadataUpdater metadataUpdater) { + TestingMetadataUpdater metadataUpdater, + Configuration conf) { super( tablePath, latestSchemaInfo, new FlussAdmin( - RpcClient.create( - new Configuration(), TestingClientMetricGroup.newInstance(), false), + RpcClient.create(conf, TestingClientMetricGroup.newInstance(), false), metadataUpdater)); } diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvSnapshotBatchScannerITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvSnapshotBatchScannerITCase.java index 4127608fc9..19cdc28ab7 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvSnapshotBatchScannerITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvSnapshotBatchScannerITCase.java @@ -24,6 +24,7 @@ import org.apache.fluss.client.table.scanner.RemoteFileDownloader; import org.apache.fluss.client.table.writer.UpsertWriter; import org.apache.fluss.client.write.HashBucketAssigner; +import org.apache.fluss.exception.KvSnapshotLeaseNotExistException; import org.apache.fluss.metadata.DataLakeFormat; import org.apache.fluss.metadata.Schema; import org.apache.fluss.metadata.TableBucket; @@ -34,21 +35,28 @@ import org.apache.fluss.row.ProjectedRow; import org.apache.fluss.row.encode.CompactedKeyEncoder; import org.apache.fluss.row.encode.KeyEncoder; +import org.apache.fluss.server.zk.ZooKeeperClient; +import org.apache.fluss.server.zk.data.lease.KvSnapshotLease; +import org.apache.fluss.server.zk.data.lease.KvSnapshotLeaseMetadataManager; +import org.apache.fluss.server.zk.data.lease.KvSnapshotTableLease; import org.apache.fluss.types.DataTypes; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import java.time.Duration; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import static org.apache.fluss.record.TestData.DATA1_ROW_TYPE; import static org.apache.fluss.testutils.DataTestUtils.compactedRow; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** IT Case for {@link KvSnapshotBatchScanner}. */ class KvSnapshotBatchScannerITCase extends ClientToServerITCaseBase { @@ -180,6 +188,97 @@ void testScanSnapshotDuringSchemaChange() throws Exception { testSnapshotRead(tablePath, expectedRowByBuckets); } + @Test + public void testKvSnapshotLease() throws Exception { + TablePath tablePath = TablePath.of(DEFAULT_DB, "test-kv-snapshot-lease"); + long tableId = createTable(tablePath, DEFAULT_TABLE_DESCRIPTOR, true); + + String kvSnapshotLease1 = "test-lease"; + String kvSnapshotLease2 = "test-lease2"; + + // scan the snapshot + Map> expectedRowByBuckets = putRows(tableId, tablePath, 10); + + // wait snapshot finish + FLUSS_CLUSTER_EXTENSION.triggerAndWaitSnapshots(expectedRowByBuckets.keySet()); + + ZooKeeperClient zkClient = FLUSS_CLUSTER_EXTENSION.getZooKeeperClient(); + String remoteDataDir = FLUSS_CLUSTER_EXTENSION.getRemoteDataDir(); + KvSnapshotLeaseMetadataManager metadataManager = + new KvSnapshotLeaseMetadataManager(zkClient, remoteDataDir); + + assertThat(zkClient.getKvSnapshotLeasesList()).isEmpty(); + + // test register kv snapshot lease for snapshot 0. + Map consumeBuckets = new HashMap<>(); + KvSnapshots kvSnapshots = admin.getLatestKvSnapshots(tablePath).get(); + for (int bucketId : kvSnapshots.getBucketIds()) { + TableBucket tableBucket = new TableBucket(kvSnapshots.getTableId(), bucketId); + consumeBuckets.put(tableBucket, kvSnapshots.getSnapshotId(bucketId).getAsLong()); + } + admin.acquireKvSnapshotLease( + kvSnapshotLease1, consumeBuckets, Duration.ofDays(1).toMillis()) + .get(); + checkKvSnapshotLeaseEquals( + metadataManager, kvSnapshotLease1, tableId, new Long[] {0L, 0L, 0L}); + + expectedRowByBuckets = putRows(tableId, tablePath, 10); + // wait snapshot2 finish + FLUSS_CLUSTER_EXTENSION.triggerAndWaitSnapshots(expectedRowByBuckets.keySet()); + + // test register kv snapshot lease for snapshot 1. + consumeBuckets = new HashMap<>(); + kvSnapshots = admin.getLatestKvSnapshots(tablePath).get(); + for (int bucketId : kvSnapshots.getBucketIds()) { + TableBucket tableBucket = new TableBucket(kvSnapshots.getTableId(), bucketId); + consumeBuckets.put(tableBucket, kvSnapshots.getSnapshotId(bucketId).getAsLong()); + } + admin.acquireKvSnapshotLease( + kvSnapshotLease2, consumeBuckets, Duration.ofDays(1).toMillis()) + .get(); + checkKvSnapshotLeaseEquals( + metadataManager, kvSnapshotLease2, tableId, new Long[] {1L, 1L, 1L}); + // check even snapshot1 is generated, snapshot0 also retained as lease exists. + for (TableBucket tb : expectedRowByBuckets.keySet()) { + assertThat(zkClient.getTableBucketSnapshot(tb, 0L).isPresent()).isTrue(); + assertThat(zkClient.getTableBucketSnapshot(tb, 1L).isPresent()).isTrue(); + } + + expectedRowByBuckets = putRows(tableId, tablePath, 10); + // wait snapshot3 finish + FLUSS_CLUSTER_EXTENSION.triggerAndWaitSnapshots(expectedRowByBuckets.keySet()); + + // release lease1. + admin.releaseKvSnapshotLease( + kvSnapshotLease1, Collections.singleton(new TableBucket(tableId, 0))) + .get(); + checkKvSnapshotLeaseEquals( + metadataManager, kvSnapshotLease1, tableId, new Long[] {-1L, 0L, 0L}); + + // release lease2. + admin.releaseKvSnapshotLease(kvSnapshotLease2, consumeBuckets.keySet()).get(); + assertThat(zkClient.getKvSnapshotLeasesList()).doesNotContain(kvSnapshotLease2); + + // release all kv snapshot lease of lease1 + admin.releaseAllKvSnapshotLease(kvSnapshotLease1).get(); + assertThat(zkClient.getKvSnapshotLeasesList()).isEmpty(); + + expectedRowByBuckets = putRows(tableId, tablePath, 10); + // wait snapshot2 finish + FLUSS_CLUSTER_EXTENSION.triggerAndWaitSnapshots(expectedRowByBuckets.keySet()); + // as all leases are dropped, and new snapshot is generated, all old snapshot are + // cleared. + for (TableBucket tb : expectedRowByBuckets.keySet()) { + assertThat(zkClient.getTableBucketSnapshot(tb, 0L).isPresent()).isFalse(); + assertThat(zkClient.getTableBucketSnapshot(tb, 1L).isPresent()).isFalse(); + } + + assertThatThrownBy(() -> admin.releaseAllKvSnapshotLease("no-exist-lease").get()) + .rootCause() + .isInstanceOf(KvSnapshotLeaseNotExistException.class) + .hasMessageContaining("kv snapshot lease 'no-exist-lease' not exits"); + } + private Map> putRows( long tableId, TablePath tablePath, int rowNumber) throws Exception { List rows = new ArrayList<>(); @@ -238,4 +337,20 @@ private static int getBucketId(InternalRow row) { byte[] key = bucketKeyEncoder.encodeKey(row); return function.bucketing(key, DEFAULT_BUCKET_NUM); } + + private void checkKvSnapshotLeaseEquals( + KvSnapshotLeaseMetadataManager metadataManager, + String leaseId, + long tableId, + Long[] expectedBucketIndex) + throws Exception { + assertThat(metadataManager.getLeasesList()).contains(leaseId); + Optional leaseOpt = metadataManager.getLease(leaseId); + assertThat(leaseOpt).isPresent(); + KvSnapshotLease actualLease = leaseOpt.get(); + Map tableIdToTableLease = actualLease.getTableIdToTableLease(); + KvSnapshotTableLease tableLease = tableIdToTableLease.get(tableId); + assertThat(tableLease).isNotNull(); + assertThat(tableLease.getBucketSnapshots()).isEqualTo(expectedBucketIndex); + } } diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherTest.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherTest.java index f06f886142..a01809c58c 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherTest.java @@ -70,7 +70,10 @@ public void setup() { metadataUpdater = initializeMetadataUpdater(); ClientSchemaGetter clientSchemaGetter = new TestingClientSchemaGetter( - DATA1_TABLE_PATH, new SchemaInfo(DATA1_SCHEMA, 0), metadataUpdater); + DATA1_TABLE_PATH, + new SchemaInfo(DATA1_SCHEMA, 0), + metadataUpdater, + new Configuration()); LogScannerStatus logScannerStatus = initializeLogScannerStatus(); logFetcher = new LogFetcher( diff --git a/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java b/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java index f121539224..08742ebc2e 100644 --- a/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java +++ b/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java @@ -1552,9 +1552,17 @@ public class ConfigOptions { public static final ConfigOption KV_MAX_RETAINED_SNAPSHOTS = key("kv.snapshot.num-retained") .intType() - .defaultValue(1) + .defaultValue(2) .withDescription("The maximum number of completed snapshots to retain."); + public static final ConfigOption KV_SNAPSHOT_LEASE_EXPIRATION_CHECK_INTERVAL = + key("kv.snapshot.lease.expiration-check-interval") + .durationType() + .defaultValue(Duration.ofMinutes(10)) + .withDescription( + "The interval to check the expiration of kv snapshot lease. " + + "The default setting is 10 minutes."); + public static final ConfigOption KV_MAX_BACKGROUND_THREADS = key("kv.rocksdb.thread.num") .intType() diff --git a/fluss-common/src/main/java/org/apache/fluss/exception/KvSnapshotLeaseNotExistException.java b/fluss-common/src/main/java/org/apache/fluss/exception/KvSnapshotLeaseNotExistException.java new file mode 100644 index 0000000000..b77660c67a --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/exception/KvSnapshotLeaseNotExistException.java @@ -0,0 +1,31 @@ +/* + * 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.exception; + +/** + * Kv snapshot lease not exist exception. + * + * @since 0.9 + */ +public class KvSnapshotLeaseNotExistException extends ApiException { + private static final long serialVersionUID = 1L; + + public KvSnapshotLeaseNotExistException(String message) { + super(message); + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/metadata/KvSnapshotLeaseForBucket.java b/fluss-common/src/main/java/org/apache/fluss/metadata/KvSnapshotLeaseForBucket.java new file mode 100644 index 0000000000..fc3840bd27 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/metadata/KvSnapshotLeaseForBucket.java @@ -0,0 +1,66 @@ +/* + * 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.metadata; + +import java.util.Objects; + +/** An entity for kv snapshot lease for bucket. */ +public class KvSnapshotLeaseForBucket { + private final TableBucket tableBucket; + private final long kvSnapshotId; + + public KvSnapshotLeaseForBucket(TableBucket tableBucket, long kvSnapshotId) { + this.tableBucket = tableBucket; + this.kvSnapshotId = kvSnapshotId; + } + + public TableBucket getTableBucket() { + return tableBucket; + } + + public long getKvSnapshotId() { + return kvSnapshotId; + } + + @Override + public String toString() { + return "KvSnapshotLeaseForBucket{" + + "tableBucket=" + + tableBucket + + ", kvSnapshotId=" + + kvSnapshotId + + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + KvSnapshotLeaseForBucket that = (KvSnapshotLeaseForBucket) o; + return kvSnapshotId == that.kvSnapshotId && Objects.equals(tableBucket, that.tableBucket); + } + + @Override + public int hashCode() { + return Objects.hash(tableBucket, kvSnapshotId); + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/metrics/MetricNames.java b/fluss-common/src/main/java/org/apache/fluss/metrics/MetricNames.java index 7f4ae5a03b..e1a5e28a6f 100644 --- a/fluss-common/src/main/java/org/apache/fluss/metrics/MetricNames.java +++ b/fluss-common/src/main/java/org/apache/fluss/metrics/MetricNames.java @@ -54,6 +54,11 @@ public class MetricNames { public static final String KV_ALL_SNAPSHOT_SIZE = "allKvSnapshotSize"; public static final String SERVER_PHYSICAL_STORAGE_REMOTE_KV_SIZE = "remoteKvSize"; + // for kv snapshot lease. + // TODO implemented it at the table level. Trace by: https://github.com/apache/fluss/issues/2297 + public static final String KV_SNAPSHOT_LEASE_COUNT = "kvSnapshotLeaseCount"; + public static final String LEASED_KV_SNAPSHOT_COUNT = "leasedKvSnapshotCount"; + // -------------------------------------------------------------------------------------------- // metrics for tablet server // -------------------------------------------------------------------------------------------- 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 c2e796964e..9a0659f180 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 @@ -99,6 +99,8 @@ public class FlussPaths { /** Suffix of a producer offsets file. */ private static final String PRODUCER_OFFSETS_FILE_SUFFIX = ".offsets"; + private static final String REMOTE_LEASE_DIR_NAME = "lease"; + // ---------------------------------------------------------------------------------------- // LOG/KV Tablet Paths // ---------------------------------------------------------------------------------------- @@ -728,6 +730,41 @@ public static FsPath remoteLakeTableSnapshotOffsetPath( UUID.randomUUID())); } + /** + * Returns the remote directory path for storing kv snapshot lease files. + * + *

The path contract: + * + *

+     * {$remote.data.dir}/lease/kv-snapshot/{leaseId}/{tableId}/
+     * 
+ */ + private static FsPath remoteKvSnapshotLeaseDir( + String remoteDataDir, String leaseId, long tableId) { + return new FsPath( + String.format( + "%s/%s/kv-snapshot/%s/%d", + remoteDataDir, REMOTE_LEASE_DIR_NAME, leaseId, tableId)); + } + + /** + * Returns the remote file path for storing kv snapshot lease files. + * + *

The path contract: + * + *

+     * {$remoteKvSnapshotLeaseDir}/{uuid}.metadata
+     * 
+ */ + public static FsPath remoteKvSnapshotLeaseFile( + String remoteDataDir, String leaseId, long tableId) { + return new FsPath( + String.format( + "%s/%s.metadata", + remoteKvSnapshotLeaseDir(remoteDataDir, leaseId, tableId), + UUID.randomUUID())); + } + /** * Returns the remote directory path for storing kv snapshot shared files (SST files with UUID * prefix). diff --git a/fluss-common/src/test/java/org/apache/fluss/record/TestData.java b/fluss-common/src/test/java/org/apache/fluss/record/TestData.java index b67e6443ec..28ab9ececc 100644 --- a/fluss-common/src/test/java/org/apache/fluss/record/TestData.java +++ b/fluss-common/src/test/java/org/apache/fluss/record/TestData.java @@ -97,6 +97,10 @@ public final class TestData { currentMillis); // for log table / partition table + public static final TablePath PARTITION_TABLE_PATH = + new TablePath("test_db_1", "test_partition_table"); + public static final long PARTITION_TABLE_ID = 150008L; + public static final TableDescriptor DATA1_PARTITIONED_TABLE_DESCRIPTOR = TableDescriptor.builder() .schema(DATA1_SCHEMA) @@ -107,6 +111,16 @@ public final class TestData { ConfigOptions.TABLE_AUTO_PARTITION_TIME_UNIT, AutoPartitionTimeUnit.YEAR) .build(); + + public static final TableInfo PARTITION_TABLE_INFO = + TableInfo.of( + PARTITION_TABLE_PATH, + PARTITION_TABLE_ID, + 1, + DATA1_PARTITIONED_TABLE_DESCRIPTOR, + System.currentTimeMillis(), + System.currentTimeMillis()); + public static final PhysicalTablePath DATA1_PHYSICAL_TABLE_PATH_PA_2024 = PhysicalTablePath.of(DATA1_TABLE_PATH, "2024"); diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/FlinkConnectorOptions.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/FlinkConnectorOptions.java index 6fd5d147fc..a00e0ff5e4 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/FlinkConnectorOptions.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/FlinkConnectorOptions.java @@ -31,6 +31,7 @@ import java.time.Duration; import java.util.Arrays; import java.util.List; +import java.util.UUID; import static org.apache.flink.configuration.description.TextElement.text; @@ -76,6 +77,23 @@ public class FlinkConnectorOptions { "A list of host/port pairs to use for establishing the initial connection to the Fluss cluster. " + "The list should be in the form host1:port1,host2:port2,...."); + public static final ConfigOption SCAN_KV_SNAPSHOT_LEASE_ID = + ConfigOptions.key("scan.kv.snapshot.lease.id") + .stringType() + .defaultValue(String.valueOf(UUID.randomUUID())) + .withDescription( + "The lease id to ping kv snapshots. If set, the acquired kv snapshots will not be deleted " + + "until the consumer finished consuming all the snapshots or the lease duration time " + + "is reached. If not set, an UUID will be set."); + + public static final ConfigOption SCAN_KV_SNAPSHOT_LEASE_DURATION = + ConfigOptions.key("scan.kv.snapshot.lease.duration") + .durationType() + .defaultValue(Duration.ofDays(1)) + .withDescription( + "The time period how long to wait before expiring the kv snapshot lease to " + + "avoid kv snapshot blocking to delete."); + // -------------------------------------------------------------------------------------------- // Lookup specific options // -------------------------------------------------------------------------------------------- diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/catalog/FlinkTableFactory.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/catalog/FlinkTableFactory.java index a7a5608de6..ab3b70119a 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/catalog/FlinkTableFactory.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/catalog/FlinkTableFactory.java @@ -27,6 +27,7 @@ import org.apache.fluss.flink.source.BinlogFlinkTableSource; import org.apache.fluss.flink.source.ChangelogFlinkTableSource; import org.apache.fluss.flink.source.FlinkTableSource; +import org.apache.fluss.flink.source.reader.LeaseContext; import org.apache.fluss.flink.utils.FlinkConnectorOptionsUtils; import org.apache.fluss.metadata.TablePath; @@ -142,6 +143,15 @@ public DynamicTableSource createDynamicTableSource(Context context) { .get(FlinkConnectorOptions.SCAN_PARTITION_DISCOVERY_INTERVAL) .toMillis(); + LeaseContext leaseContext = + primaryKeyIndexes.length > 0 + ? new LeaseContext( + tableOptions.get(FlinkConnectorOptions.SCAN_KV_SNAPSHOT_LEASE_ID), + tableOptions + .get(FlinkConnectorOptions.SCAN_KV_SNAPSHOT_LEASE_DURATION) + .toMillis()) + : new LeaseContext(null, null); + return new FlinkTableSource( toFlussTablePath(context.getObjectIdentifier()), toFlussClientConfig( @@ -157,7 +167,8 @@ public DynamicTableSource createDynamicTableSource(Context context) { partitionDiscoveryIntervalMs, tableOptions.get(toFlinkOption(ConfigOptions.TABLE_DATALAKE_ENABLED)), tableOptions.get(toFlinkOption(ConfigOptions.TABLE_MERGE_ENGINE)), - context.getCatalogTable().getOptions()); + context.getCatalogTable().getOptions(), + leaseContext); } @Override @@ -213,6 +224,8 @@ public Set> optionalOptions() { FlinkConnectorOptions.SCAN_STARTUP_MODE, FlinkConnectorOptions.SCAN_STARTUP_TIMESTAMP, FlinkConnectorOptions.SCAN_PARTITION_DISCOVERY_INTERVAL, + FlinkConnectorOptions.SCAN_KV_SNAPSHOT_LEASE_ID, + FlinkConnectorOptions.SCAN_KV_SNAPSHOT_LEASE_DURATION, FlinkConnectorOptions.LOOKUP_ASYNC, FlinkConnectorOptions.SINK_IGNORE_DELETE, FlinkConnectorOptions.SINK_BUCKET_SHUFFLE, diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/ProcedureManager.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/ProcedureManager.java index b97b3cd8e0..a0d0d54e0d 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/ProcedureManager.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/ProcedureManager.java @@ -77,7 +77,9 @@ private enum ProcedureEnum { REMOVE_SERVER_TAG("sys.remove_server_tag", RemoveServerTagProcedure.class), REBALANCE("sys.rebalance", RebalanceProcedure.class), CANCEL_REBALANCE("sys.cancel_rebalance", CancelRebalanceProcedure.class), - LIST_REBALANCE_PROGRESS("sys.list_rebalance", ListRebalanceProcessProcedure.class); + LIST_REBALANCE_PROGRESS("sys.list_rebalance", ListRebalanceProcessProcedure.class), + RELEASE_ALL_KV_SNAPSHOT_LEASE( + "sys.release_all_kv_snapshot_lease", ReleaseAllKvSnapshotLeaseProcedure.class); private final String path; private final Class procedureClass; diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/ReleaseAllKvSnapshotLeaseProcedure.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/ReleaseAllKvSnapshotLeaseProcedure.java new file mode 100644 index 0000000000..ba6e9c76ff --- /dev/null +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/ReleaseAllKvSnapshotLeaseProcedure.java @@ -0,0 +1,46 @@ +/* + * 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.procedure; + +import org.apache.flink.table.annotation.ArgumentHint; +import org.apache.flink.table.annotation.DataTypeHint; +import org.apache.flink.table.annotation.ProcedureHint; +import org.apache.flink.table.procedure.ProcedureContext; + +/** + * Procedure to release all kv snapshots leased of specified leaseId. See {@link + * org.apache.fluss.client.admin.Admin#releaseAllKvSnapshotLease(String)} for more details. + * + *

Usage examples: + * + *

+ * -- Release all kv snapshots leased of specified leaseId
+ * CALL sys.release_all_kv_snapshot_lease('test-lease-id');
+ * 
+ */ +public class ReleaseAllKvSnapshotLeaseProcedure extends ProcedureBase { + + @ProcedureHint( + argument = { + @ArgumentHint(name = "leaseId", type = @DataTypeHint("STRING")), + }) + public String[] call(ProcedureContext context, String leaseId) throws Exception { + admin.releaseAllKvSnapshotLease(leaseId).get(); + return new String[] {"success"}; + } +} diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkSource.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkSource.java index 768fa7ddaa..d8072559dd 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkSource.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkSource.java @@ -25,6 +25,7 @@ import org.apache.fluss.flink.source.enumerator.FlinkSourceEnumerator; import org.apache.fluss.flink.source.metrics.FlinkSourceReaderMetrics; import org.apache.fluss.flink.source.reader.FlinkSourceReader; +import org.apache.fluss.flink.source.reader.LeaseContext; import org.apache.fluss.flink.source.reader.RecordAndPos; import org.apache.fluss.flink.source.split.SourceSplitBase; import org.apache.fluss.flink.source.split.SourceSplitSerializer; @@ -70,6 +71,7 @@ public class FlinkSource private final FlussDeserializationSchema deserializationSchema; @Nullable private final Predicate partitionFilters; @Nullable private final LakeSource lakeSource; + private final LeaseContext leaseContext; public FlinkSource( Configuration flussConf, @@ -82,7 +84,8 @@ public FlinkSource( long scanPartitionDiscoveryIntervalMs, FlussDeserializationSchema deserializationSchema, boolean streaming, - @Nullable Predicate partitionFilters) { + @Nullable Predicate partitionFilters, + LeaseContext leaseContext) { this( flussConf, tablePath, @@ -95,7 +98,8 @@ public FlinkSource( deserializationSchema, streaming, partitionFilters, - null); + null, + leaseContext); } public FlinkSource( @@ -110,7 +114,8 @@ public FlinkSource( FlussDeserializationSchema deserializationSchema, boolean streaming, @Nullable Predicate partitionFilters, - @Nullable LakeSource lakeSource) { + @Nullable LakeSource lakeSource, + LeaseContext leaseContext) { this.flussConf = flussConf; this.tablePath = tablePath; this.hasPrimaryKey = hasPrimaryKey; @@ -123,6 +128,7 @@ public FlinkSource( this.streaming = streaming; this.partitionFilters = partitionFilters; this.lakeSource = lakeSource; + this.leaseContext = leaseContext; } @Override @@ -143,7 +149,8 @@ public SplitEnumerator createEnumerator( scanPartitionDiscoveryIntervalMs, streaming, partitionFilters, - lakeSource); + lakeSource, + leaseContext); } @Override @@ -163,7 +170,8 @@ public SplitEnumerator restoreEnumerator scanPartitionDiscoveryIntervalMs, streaming, partitionFilters, - lakeSource); + lakeSource, + sourceEnumeratorState.getLeaseContext()); } @Override diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java index 8e234d86c5..fb6ec95756 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java @@ -25,6 +25,7 @@ import org.apache.fluss.flink.source.lookup.FlinkAsyncLookupFunction; import org.apache.fluss.flink.source.lookup.FlinkLookupFunction; import org.apache.fluss.flink.source.lookup.LookupNormalizer; +import org.apache.fluss.flink.source.reader.LeaseContext; import org.apache.fluss.flink.utils.FlinkConnectorOptionsUtils; import org.apache.fluss.flink.utils.FlinkConversions; import org.apache.fluss.flink.utils.PushdownUtils; @@ -127,6 +128,8 @@ public class FlinkTableSource private final long scanPartitionDiscoveryIntervalMs; private final boolean isDataLakeEnabled; + private final LeaseContext leaseContext; + @Nullable private final MergeEngineType mergeEngineType; // output type after projection pushdown @@ -165,7 +168,8 @@ public FlinkTableSource( long scanPartitionDiscoveryIntervalMs, boolean isDataLakeEnabled, @Nullable MergeEngineType mergeEngineType, - Map tableOptions) { + Map tableOptions, + LeaseContext leaseContext) { this.tablePath = tablePath; this.flussConfig = flussConfig; this.tableOutputType = tableOutputType; @@ -181,6 +185,7 @@ public FlinkTableSource( this.scanPartitionDiscoveryIntervalMs = scanPartitionDiscoveryIntervalMs; this.isDataLakeEnabled = isDataLakeEnabled; + this.leaseContext = leaseContext; this.mergeEngineType = mergeEngineType; this.tableOptions = tableOptions; if (isDataLakeEnabled) { @@ -337,7 +342,8 @@ public boolean isBounded() { new RowDataDeserializationSchema(), streaming, partitionFilters, - enableLakeSource ? lakeSource : null); + enableLakeSource ? lakeSource : null, + leaseContext); if (!streaming) { // return a bounded source provide to make planner happy, @@ -424,7 +430,8 @@ public DynamicTableSource copy() { scanPartitionDiscoveryIntervalMs, isDataLakeEnabled, mergeEngineType, - tableOptions); + tableOptions, + leaseContext); source.producedDataType = producedDataType; source.projectedFields = projectedFields; source.singleRowFilter = singleRowFilter; diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlussSource.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlussSource.java index 05fe4e2763..1c181aafbc 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlussSource.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlussSource.java @@ -21,6 +21,7 @@ import org.apache.fluss.client.initializer.OffsetsInitializer; import org.apache.fluss.config.Configuration; import org.apache.fluss.flink.source.deserializer.FlussDeserializationSchema; +import org.apache.fluss.flink.source.reader.LeaseContext; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.types.RowType; @@ -67,7 +68,8 @@ public class FlussSource extends FlinkSource { OffsetsInitializer offsetsInitializer, long scanPartitionDiscoveryIntervalMs, FlussDeserializationSchema deserializationSchema, - boolean streaming) { + boolean streaming, + LeaseContext leaseContext) { // TODO: Support partition pushDown in datastream super( flussConf, @@ -80,7 +82,8 @@ public class FlussSource extends FlinkSource { scanPartitionDiscoveryIntervalMs, deserializationSchema, streaming, - null); + null, + leaseContext); } /** diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlussSourceBuilder.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlussSourceBuilder.java index afd955c01f..da1d8f7b7b 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlussSourceBuilder.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlussSourceBuilder.java @@ -25,6 +25,7 @@ import org.apache.fluss.config.Configuration; import org.apache.fluss.flink.FlinkConnectorOptions; import org.apache.fluss.flink.source.deserializer.FlussDeserializationSchema; +import org.apache.fluss.flink.source.reader.LeaseContext; import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.types.RowType; @@ -71,6 +72,8 @@ public class FlussSourceBuilder { private Long scanPartitionDiscoveryIntervalMs; private OffsetsInitializer offsetsInitializer; private FlussDeserializationSchema deserializationSchema; + private String kvSnapshotLeaseId; + private long kvSnapshotLeaseDurationMs; private String bootstrapServers; @@ -174,6 +177,16 @@ public FlussSourceBuilder setProjectedFields(String... projectedFieldNames) return this; } + public FlussSourceBuilder setKvSnapshotLeaseId(String kvSnapshotLeaseId) { + this.kvSnapshotLeaseId = kvSnapshotLeaseId; + return this; + } + + public FlussSourceBuilder setKvSnapshotLeaseDurationMs(long kvSnapshotLeaseDurationMs) { + this.kvSnapshotLeaseDurationMs = kvSnapshotLeaseDurationMs; + return this; + } + /** * Sets custom Fluss configuration properties for the source connector. * @@ -300,6 +313,9 @@ public FlussSource build() { offsetsInitializer, scanPartitionDiscoveryIntervalMs, deserializationSchema, - true); + true, + hasPrimaryKey + ? new LeaseContext(kvSnapshotLeaseId, kvSnapshotLeaseDurationMs) + : new LeaseContext(null, null)); } } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumerator.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumerator.java index a22c61d71c..0ebb808e81 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumerator.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumerator.java @@ -31,8 +31,10 @@ import org.apache.fluss.flink.lake.LakeSplitGenerator; import org.apache.fluss.flink.lake.split.LakeSnapshotAndFlussLogSplit; import org.apache.fluss.flink.lake.split.LakeSnapshotSplit; +import org.apache.fluss.flink.source.event.FinishedKvSnapshotConsumeEvent; import org.apache.fluss.flink.source.event.PartitionBucketsUnsubscribedEvent; import org.apache.fluss.flink.source.event.PartitionsRemovedEvent; +import org.apache.fluss.flink.source.reader.LeaseContext; import org.apache.fluss.flink.source.split.HybridSnapshotLogSplit; import org.apache.fluss.flink.source.split.LogSplit; import org.apache.fluss.flink.source.split.SourceSplitBase; @@ -71,9 +73,11 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.NavigableMap; import java.util.Objects; import java.util.OptionalLong; import java.util.Set; +import java.util.TreeMap; import java.util.stream.Collectors; import static org.apache.fluss.utils.Preconditions.checkNotNull; @@ -132,6 +136,11 @@ public class FlinkSourceEnumerator private final OffsetsInitializer startingOffsetsInitializer; private final OffsetsInitializer stoppingOffsetsInitializer; + private final LeaseContext leaseContext; + + /** checkpointId -> tableBuckets who finished consume kv snapshots. */ + private final TreeMap> consumedKvSnapshotMap = new TreeMap<>(); + // Lazily instantiated or mutable fields. private Connection connection; private Admin flussAdmin; @@ -160,7 +169,8 @@ public FlinkSourceEnumerator( long scanPartitionDiscoveryIntervalMs, boolean streaming, @Nullable Predicate partitionFilters, - @Nullable LakeSource lakeSource) { + @Nullable LakeSource lakeSource, + LeaseContext leaseContext) { this( tablePath, flussConf, @@ -174,7 +184,8 @@ public FlinkSourceEnumerator( scanPartitionDiscoveryIntervalMs, streaming, partitionFilters, - lakeSource); + lakeSource, + leaseContext); } public FlinkSourceEnumerator( @@ -190,7 +201,8 @@ public FlinkSourceEnumerator( long scanPartitionDiscoveryIntervalMs, boolean streaming, @Nullable Predicate partitionFilters, - @Nullable LakeSource lakeSource) { + @Nullable LakeSource lakeSource, + LeaseContext leaseContext) { this( tablePath, flussConf, @@ -205,7 +217,8 @@ public FlinkSourceEnumerator( streaming, partitionFilters, lakeSource, - new WorkerExecutor(context)); + new WorkerExecutor(context), + leaseContext); } FlinkSourceEnumerator( @@ -222,7 +235,8 @@ public FlinkSourceEnumerator( boolean streaming, @Nullable Predicate partitionFilters, @Nullable LakeSource lakeSource, - WorkerExecutor workerExecutor) { + WorkerExecutor workerExecutor, + LeaseContext leaseContext) { this.tablePath = checkNotNull(tablePath); this.flussConf = checkNotNull(flussConf); this.hasPrimaryKey = hasPrimaryKey; @@ -243,6 +257,7 @@ public FlinkSourceEnumerator( streaming ? new NoStoppingOffsetsInitializer() : OffsetsInitializer.latest(); this.lakeSource = lakeSource; this.workerExecutor = workerExecutor; + this.leaseContext = leaseContext; } @Override @@ -349,16 +364,7 @@ private void startInStreamModeForNonPartitionedTable() { private List initNonPartitionedSplits() { if (hasPrimaryKey && startingOffsetsInitializer instanceof SnapshotOffsetsInitializer) { - // get the table snapshot info - final KvSnapshots kvSnapshots; - try { - kvSnapshots = flussAdmin.getLatestKvSnapshots(tablePath).get(); - } catch (Exception e) { - throw new FlinkRuntimeException( - String.format("Failed to get table snapshot for %s", tablePath), - ExceptionUtils.stripCompletionException(e)); - } - return getSnapshotAndLogSplits(kvSnapshots, null); + return getSnapshotAndLogSplits(getLatestKvSnapshotsAndRegister(null), null); } else { return getLogSplit(null, null); } @@ -531,22 +537,88 @@ private List initPrimaryKeyTablePartitionSplits( List splits = new ArrayList<>(); for (Partition partition : newPartitions) { String partitionName = partition.getPartitionName(); - // get the table snapshot info - final KvSnapshots kvSnapshots; - try { - kvSnapshots = flussAdmin.getLatestKvSnapshots(tablePath, partitionName).get(); - } catch (Exception e) { - throw new FlinkRuntimeException( - String.format( - "Failed to get table snapshot for table %s and partition %s", - tablePath, partitionName), - ExceptionUtils.stripCompletionException(e)); - } - splits.addAll(getSnapshotAndLogSplits(kvSnapshots, partitionName)); + splits.addAll( + getSnapshotAndLogSplits( + getLatestKvSnapshotsAndRegister(partitionName), partitionName)); } return splits; } + private KvSnapshots getLatestKvSnapshotsAndRegister(@Nullable String partitionName) { + long tableId; + Long partitionId; + Map snapshotIds = new HashMap<>(); + Map logOffsets = new HashMap<>(); + + // retry to get the latest kv snapshots and acquire kvSnapshot lease. + try { + KvSnapshots kvSnapshots = getLatestKvSnapshots(partitionName); + Set remainingTableBuckets = new HashSet<>(kvSnapshots.getTableBuckets()); + + tableId = kvSnapshots.getTableId(); + partitionId = kvSnapshots.getPartitionId(); + + Set ignoreBuckets = new HashSet<>(); + Map bucketsToLease = new HashMap<>(); + for (TableBucket tb : remainingTableBuckets) { + int bucket = tb.getBucket(); + OptionalLong snapshotIdOpt = kvSnapshots.getSnapshotId(bucket); + OptionalLong logOffsetOpt = kvSnapshots.getLogOffset(bucket); + if (snapshotIdOpt.isPresent() && !ignoreTableBucket(tb)) { + bucketsToLease.put(tb, snapshotIdOpt.getAsLong()); + } else { + ignoreBuckets.add(tb); + } + + snapshotIds.put( + bucket, snapshotIdOpt.isPresent() ? snapshotIdOpt.getAsLong() : null); + logOffsets.put(bucket, logOffsetOpt.isPresent() ? logOffsetOpt.getAsLong() : null); + } + + if (!ignoreBuckets.isEmpty()) { + remainingTableBuckets.removeAll(ignoreBuckets); + } + + if (!bucketsToLease.isEmpty()) { + String kvSnapshotLeaseId = leaseContext.getKvSnapshotLeaseId(); + LOG.info( + "Try to acquire kv snapshot lease {} for table {}", + kvSnapshotLeaseId, + tablePath); + Long kvSnapshotLeaseDurationMs = leaseContext.getKvSnapshotLeaseDurationMs(); + checkNotNull(kvSnapshotLeaseDurationMs, "kv snapshot lease duration is null."); + remainingTableBuckets = + flussAdmin + .acquireKvSnapshotLease( + kvSnapshotLeaseId, + bucketsToLease, + kvSnapshotLeaseDurationMs) + .get() + .getUnavailableTableBucketSet(); + if (!remainingTableBuckets.isEmpty()) { + LOG.info( + "Failed to acquire kv snapshot lease for table {}: {}. Retry to re-acquire", + tablePath, + remainingTableBuckets); + } + } + } catch (Exception e) { + throw new FlinkRuntimeException( + String.format("Failed to get table snapshot for %s", tablePath), + ExceptionUtils.stripCompletionException(e)); + } + + return new KvSnapshots(tableId, partitionId, snapshotIds, logOffsets); + } + + private KvSnapshots getLatestKvSnapshots(@Nullable String partitionName) throws Exception { + if (partitionName == null) { + return flussAdmin.getLatestKvSnapshots(tablePath).get(); + } else { + return flussAdmin.getLatestKvSnapshots(tablePath, partitionName).get(); + } + } + private List getSnapshotAndLogSplits( KvSnapshots snapshots, @Nullable String partitionName) { long tableId = snapshots.getTableId(); @@ -889,6 +961,18 @@ public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { for (Long partitionToRemove : partitionsPendingRemove) { assignedPartitions.remove(partitionToRemove); } + } else if (sourceEvent instanceof FinishedKvSnapshotConsumeEvent) { + FinishedKvSnapshotConsumeEvent event = (FinishedKvSnapshotConsumeEvent) sourceEvent; + long checkpointId = event.getCheckpointId(); + Set tableBuckets = event.getTableBuckets(); + if (!tableBuckets.isEmpty()) { + LOG.info( + "Received finished kv snapshot consumer event for buckets: {}, checkpoint id: {}", + tableBuckets, + checkpointId); + } + + tableBuckets.forEach(tableBucket -> addConsumedBucket(checkpointId, tableBucket)); } } @@ -923,11 +1007,55 @@ public void addReader(int subtaskId) { public SourceEnumeratorState snapshotState(long checkpointId) { final SourceEnumeratorState enumeratorState = new SourceEnumeratorState( - assignedTableBuckets, assignedPartitions, pendingHybridLakeFlussSplits); + assignedTableBuckets, + assignedPartitions, + pendingHybridLakeFlussSplits, + leaseContext); LOG.debug("Source Checkpoint is {}", enumeratorState); return enumeratorState; } + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + // lower than this checkpoint id. + Set consumedKvSnapshots = getAndRemoveConsumedBucketsUpTo(checkpointId); + + LOG.info( + "kv snapshot has already consumed and try to release kv snapshot lease for: {}, checkpoint id: {}", + consumedKvSnapshots, + checkpointId); + + // send request to fluss to unregister the kv snapshot lease. + try { + flussAdmin + .releaseKvSnapshotLease( + leaseContext.getKvSnapshotLeaseId(), consumedKvSnapshots) + .get(); + } catch (Exception e) { + LOG.error("Failed to release kv snapshot lease. These snapshot need to re-enqueue", e); + // use the current checkpoint id to re-enqueue the buckets + consumedKvSnapshots.forEach( + tableBucket -> addConsumedBucket(checkpointId, tableBucket)); + } + } + + /** Add bucket who has been consumed kv snapshot to the consumedKvSnapshotMap. */ + public void addConsumedBucket(long checkpointId, TableBucket tableBucket) { + consumedKvSnapshotMap.computeIfAbsent(checkpointId, k -> new HashSet<>()).add(tableBucket); + } + + /** Get and remove the buckets who have been consumed kv snapshot up to the checkpoint id. */ + public Set getAndRemoveConsumedBucketsUpTo(long checkpointId) { + NavigableMap> toRemove = + consumedKvSnapshotMap.headMap(checkpointId, false); + Set result = new HashSet<>(); + for (Set snapshots : toRemove.values()) { + result.addAll(snapshots); + } + toRemove.clear(); + return result; + } + @Override public void close() throws IOException { try { diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/event/FinishedKvSnapshotConsumeEvent.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/event/FinishedKvSnapshotConsumeEvent.java new file mode 100644 index 0000000000..a5ad1f9021 --- /dev/null +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/event/FinishedKvSnapshotConsumeEvent.java @@ -0,0 +1,74 @@ +/* + * 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.source.event; + +import org.apache.fluss.metadata.TableBucket; + +import org.apache.flink.api.connector.source.SourceEvent; + +import java.util.Objects; +import java.util.Set; + +/** SourceEvent used to represent a Fluss table bucket has complete consume kv snapshot. */ +public class FinishedKvSnapshotConsumeEvent implements SourceEvent { + private static final long serialVersionUID = 1L; + + private final long checkpointId; + /** The tableBucket set who finished consume kv snapshots. */ + private final Set tableBuckets; + + public FinishedKvSnapshotConsumeEvent(long checkpointId, Set tableBuckets) { + this.checkpointId = checkpointId; + this.tableBuckets = tableBuckets; + } + + public long getCheckpointId() { + return checkpointId; + } + + public Set getTableBuckets() { + return tableBuckets; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + FinishedKvSnapshotConsumeEvent that = (FinishedKvSnapshotConsumeEvent) o; + return checkpointId == that.checkpointId && Objects.equals(tableBuckets, that.tableBuckets); + } + + @Override + public int hashCode() { + return Objects.hash(checkpointId, tableBuckets); + } + + @Override + public String toString() { + return "FinishedKvSnapshotConsumeEvent{" + + "checkpointId=" + + checkpointId + + ", tableBuckets=" + + tableBuckets + + '}'; + } +} diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/reader/FlinkSourceReader.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/reader/FlinkSourceReader.java index 6364532bb3..01eadbcea3 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/reader/FlinkSourceReader.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/reader/FlinkSourceReader.java @@ -21,10 +21,12 @@ import org.apache.fluss.flink.adapter.SingleThreadMultiplexSourceReaderBaseAdapter; import org.apache.fluss.flink.lake.LakeSplitStateInitializer; import org.apache.fluss.flink.source.emitter.FlinkRecordEmitter; +import org.apache.fluss.flink.source.event.FinishedKvSnapshotConsumeEvent; import org.apache.fluss.flink.source.event.PartitionBucketsUnsubscribedEvent; import org.apache.fluss.flink.source.event.PartitionsRemovedEvent; import org.apache.fluss.flink.source.metrics.FlinkSourceReaderMetrics; import org.apache.fluss.flink.source.reader.fetcher.FlinkSourceFetcherManager; +import org.apache.fluss.flink.source.split.HybridSnapshotLogSplit; import org.apache.fluss.flink.source.split.HybridSnapshotLogSplitState; import org.apache.fluss.flink.source.split.LogSplitState; import org.apache.fluss.flink.source.split.SourceSplitBase; @@ -39,9 +41,13 @@ import org.apache.flink.api.connector.source.SourceReaderContext; import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import javax.annotation.Nullable; +import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.function.Consumer; @@ -51,6 +57,11 @@ public class FlinkSourceReader extends SingleThreadMultiplexSourceReaderBaseAdapter< RecordAndPos, OUT, SourceSplitBase, SourceSplitState> { + private static final Logger LOG = LoggerFactory.getLogger(FlinkSourceReader.class); + + /** the tableBuckets ignore to send FinishedKvSnapshotConsumeEvent as it already sending. */ + private final Set finishedKvSnapshotConsumeBuckets; + public FlinkSourceReader( FutureCompletingBlockingQueue> elementsQueue, Configuration flussConfig, @@ -77,6 +88,7 @@ public FlinkSourceReader( recordEmitter, context.getConfiguration(), context); + this.finishedKvSnapshotConsumeBuckets = new HashSet<>(); } @Override @@ -84,6 +96,45 @@ protected void onSplitFinished(Map map) { // do nothing } + @Override + public List snapshotState(long checkpointId) { + Set bucketsFinishedConsumeKvSnapshot = new HashSet<>(); + + // do not modify this state. + List sourceSplitBases = super.snapshotState(checkpointId); + for (SourceSplitBase sourceSplitBase : sourceSplitBases) { + TableBucket tableBucket = sourceSplitBase.getTableBucket(); + if (finishedKvSnapshotConsumeBuckets.contains(tableBucket)) { + continue; + } + + if (sourceSplitBase.isHybridSnapshotLogSplit()) { + HybridSnapshotLogSplit hybridSnapshotLogSplit = + sourceSplitBase.asHybridSnapshotLogSplit(); + if (hybridSnapshotLogSplit.isSnapshotFinished()) { + bucketsFinishedConsumeKvSnapshot.add(tableBucket); + } + } + } + + // report finished kv snapshot consume event. + if (!bucketsFinishedConsumeKvSnapshot.isEmpty()) { + LOG.info( + "reader has finished kv snapshot read for bucket: {}, checkpoint id {}", + bucketsFinishedConsumeKvSnapshot, + checkpointId); + + context.sendSourceEventToCoordinator( + new FinishedKvSnapshotConsumeEvent( + checkpointId, bucketsFinishedConsumeKvSnapshot)); + // It won't be sent anymore in the future for this table bucket, but will be resent + // after failover recovery as ignoreBuckets is cleared. + finishedKvSnapshotConsumeBuckets.addAll(bucketsFinishedConsumeKvSnapshot); + } + + return sourceSplitBases; + } + @Override public void handleSourceEvents(SourceEvent sourceEvent) { if (sourceEvent instanceof PartitionsRemovedEvent) { diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/reader/LeaseContext.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/reader/LeaseContext.java new file mode 100644 index 0000000000..5ee12bdfa9 --- /dev/null +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/reader/LeaseContext.java @@ -0,0 +1,79 @@ +/* + * 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.source.reader; + +import javax.annotation.Nullable; + +import java.io.Serializable; +import java.util.Objects; + +/** Context for lease. */ +public class LeaseContext implements Serializable { + + private static final long serialVersionUID = 1L; + + // kv snapshot lease id. null for log table. + private final @Nullable String kvSnapshotLeaseId; + + // kv snapshot lease duration. null for log table. + private final @Nullable Long kvSnapshotLeaseDurationMs; + + public LeaseContext( + @Nullable String kvSnapshotLeaseId, @Nullable Long kvSnapshotLeaseDurationMs) { + this.kvSnapshotLeaseId = kvSnapshotLeaseId; + this.kvSnapshotLeaseDurationMs = kvSnapshotLeaseDurationMs; + } + + public @Nullable String getKvSnapshotLeaseId() { + return kvSnapshotLeaseId; + } + + public @Nullable Long getKvSnapshotLeaseDurationMs() { + return kvSnapshotLeaseDurationMs; + } + + @Override + public String toString() { + return "LeaseContext{" + + "kvSnapshotLeaseId='" + + kvSnapshotLeaseId + + '\'' + + ", kvSnapshotLeaseDurationMs=" + + kvSnapshotLeaseDurationMs + + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + LeaseContext that = (LeaseContext) o; + + return Objects.equals(kvSnapshotLeaseId, that.kvSnapshotLeaseId) + && Objects.equals(kvSnapshotLeaseDurationMs, that.kvSnapshotLeaseDurationMs); + } + + @Override + public int hashCode() { + return Objects.hash(kvSnapshotLeaseId, kvSnapshotLeaseDurationMs); + } +} diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/state/FlussSourceEnumeratorStateSerializer.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/state/FlussSourceEnumeratorStateSerializer.java index b721032d0b..6907b0e271 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/state/FlussSourceEnumeratorStateSerializer.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/state/FlussSourceEnumeratorStateSerializer.java @@ -18,6 +18,7 @@ package org.apache.fluss.flink.source.state; import org.apache.fluss.annotation.VisibleForTesting; +import org.apache.fluss.flink.source.reader.LeaseContext; import org.apache.fluss.flink.source.split.SourceSplitBase; import org.apache.fluss.flink.source.split.SourceSplitSerializer; import org.apache.fluss.lake.source.LakeSource; @@ -68,11 +69,12 @@ public class FlussSourceEnumeratorStateSerializer private static final int VERSION_0 = 0; private static final int VERSION_1 = 1; + private static final int VERSION_2 = 2; private static final ThreadLocal SERIALIZER_CACHE = ThreadLocal.withInitial(() -> new DataOutputSerializer(64)); - private static final int CURRENT_VERSION = VERSION_1; + private static final int CURRENT_VERSION = VERSION_2; public FlussSourceEnumeratorStateSerializer(LakeSource lakeSource) { this.lakeSource = lakeSource; @@ -94,6 +96,9 @@ public byte[] serialize(SourceEnumeratorState state) throws IOException { // serialize remain hybrid lake splits serializeRemainingHybridLakeFlussSplits(out, state); + // write lease context + serializeLeaseContext(out, state); + final byte[] result = out.getCopyOfBuffer(); out.clear(); return result; @@ -164,6 +169,8 @@ protected byte[] serializeV0(SourceEnumeratorState state) throws IOException { @Override public SourceEnumeratorState deserialize(int version, byte[] serialized) throws IOException { switch (version) { + case VERSION_2: + return deserializeV2(serialized); case VERSION_1: return deserializeV1(serialized); case VERSION_0: @@ -187,10 +194,12 @@ private SourceEnumeratorState deserializeV0(byte[] serialized) throws IOExceptio if (lakeSource != null) { remainingHybridLakeFlussSplits = deserializeRemainingHybridLakeFlussSplits(in); } + return new SourceEnumeratorState( assignBucketAndPartitions.f0, assignBucketAndPartitions.f1, - remainingHybridLakeFlussSplits); + remainingHybridLakeFlussSplits, + new LeaseContext(null, null)); } private SourceEnumeratorState deserializeV1(byte[] serialized) throws IOException { @@ -203,10 +212,28 @@ private SourceEnumeratorState deserializeV1(byte[] serialized) throws IOExceptio // splits. The serialized state encodes their presence via a boolean flag, so // this logic no longer depends on the lakeSource flag. This unconditional // deserialization is the intended behavior change compared to VERSION_0. + + return new SourceEnumeratorState( + assignBucketAndPartitions.f0, + assignBucketAndPartitions.f1, + remainingHybridLakeFlussSplits, + new LeaseContext(null, null)); + } + + private SourceEnumeratorState deserializeV2(byte[] serialized) throws IOException { + DataInputDeserializer in = new DataInputDeserializer(serialized); + Tuple2, Map> assignBucketAndPartitions = + deserializeAssignBucketAndPartitions(in); + List remainingHybridLakeFlussSplits = + deserializeRemainingHybridLakeFlussSplits(in); + + // deserialize lease context + LeaseContext leaseContext = deserializeLeaseContext(in); return new SourceEnumeratorState( assignBucketAndPartitions.f0, assignBucketAndPartitions.f1, - remainingHybridLakeFlussSplits); + remainingHybridLakeFlussSplits, + leaseContext); } private Tuple2, Map> deserializeAssignBucketAndPartitions( @@ -260,4 +287,39 @@ private List deserializeRemainingHybridLakeFlussSplits( return null; } } + + private void serializeLeaseContext(final DataOutputSerializer out, SourceEnumeratorState state) + throws IOException { + LeaseContext leaseContext = state.getLeaseContext(); + String kvSnapshotLeaseId = leaseContext.getKvSnapshotLeaseId(); + if (kvSnapshotLeaseId != null) { + out.writeBoolean(true); + out.writeUTF(kvSnapshotLeaseId); + } else { + out.writeBoolean(false); + } + + Long kvSnapshotLeaseDurationMs = leaseContext.getKvSnapshotLeaseDurationMs(); + if (kvSnapshotLeaseDurationMs != null) { + out.writeBoolean(true); + out.writeLong(kvSnapshotLeaseDurationMs); + } else { + out.writeBoolean(false); + } + } + + private LeaseContext deserializeLeaseContext(final DataInputDeserializer in) + throws IOException { + if (in.readBoolean()) { + String kvSnapshotLeaseId = in.readUTF(); + if (in.readBoolean()) { + Long kvSnapshotLeaseDurationMs = in.readLong(); + return new LeaseContext(kvSnapshotLeaseId, kvSnapshotLeaseDurationMs); + } else { + return new LeaseContext(kvSnapshotLeaseId, null); + } + } else { + return new LeaseContext(null, null); + } + } } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/state/SourceEnumeratorState.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/state/SourceEnumeratorState.java index 6042e65f15..a97736f47d 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/state/SourceEnumeratorState.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/state/SourceEnumeratorState.java @@ -17,6 +17,7 @@ package org.apache.fluss.flink.source.state; +import org.apache.fluss.flink.source.reader.LeaseContext; import org.apache.fluss.flink.source.split.SourceSplitBase; import org.apache.fluss.metadata.TableBucket; @@ -41,13 +42,18 @@ public class SourceEnumeratorState { // lake snapshot @Nullable private final List remainingHybridLakeFlussSplits; + // lease context for restore. + private final LeaseContext leaseContext; + public SourceEnumeratorState( Set assignedBuckets, Map assignedPartitions, - @Nullable List remainingHybridLakeFlussSplits) { + @Nullable List remainingHybridLakeFlussSplits, + @Nullable LeaseContext leaseContext) { this.assignedBuckets = assignedBuckets; this.assignedPartitions = assignedPartitions; this.remainingHybridLakeFlussSplits = remainingHybridLakeFlussSplits; + this.leaseContext = leaseContext; } public Set getAssignedBuckets() { @@ -63,6 +69,10 @@ public List getRemainingHybridLakeFlussSplits() { return remainingHybridLakeFlussSplits; } + public LeaseContext getLeaseContext() { + return leaseContext; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -75,7 +85,8 @@ public boolean equals(Object o) { return Objects.equals(assignedBuckets, that.assignedBuckets) && Objects.equals(assignedPartitions, that.assignedPartitions) && Objects.equals( - remainingHybridLakeFlussSplits, that.remainingHybridLakeFlussSplits); + remainingHybridLakeFlussSplits, that.remainingHybridLakeFlussSplits) + && Objects.equals(leaseContext, that.leaseContext); } @Override @@ -92,6 +103,8 @@ public String toString() { + assignedPartitions + ", remainingHybridLakeFlussSplits=" + remainingHybridLakeFlussSplits + + ", leaseContext=" + + leaseContext + '}'; } } diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/procedure/FlinkProcedureITCase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/procedure/FlinkProcedureITCase.java index 5138ca8698..1cd87da56f 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/procedure/FlinkProcedureITCase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/procedure/FlinkProcedureITCase.java @@ -30,6 +30,7 @@ import org.apache.fluss.exception.SecurityDisabledException; import org.apache.fluss.metadata.DataLakeFormat; import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.row.InternalRow; import org.apache.fluss.server.testutils.FlussClusterExtension; import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.server.zk.data.ServerTags; @@ -57,7 +58,9 @@ import static org.apache.fluss.cluster.rebalance.ServerTag.PERMANENT_OFFLINE; import static org.apache.fluss.flink.source.testutils.FlinkRowAssertionsUtils.assertResultsIgnoreOrder; +import static org.apache.fluss.flink.utils.FlinkTestBase.writeRows; import static org.apache.fluss.server.testutils.FlussClusterExtension.BUILTIN_DATABASE; +import static org.apache.fluss.testutils.DataTestUtils.row; import static org.apache.fluss.testutils.common.CommonTestUtils.retry; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -140,7 +143,8 @@ void testShowProcedures() throws Exception { "+I[sys.remove_server_tag]", "+I[sys.rebalance]", "+I[sys.cancel_rebalance]", - "+I[sys.list_rebalance]"); + "+I[sys.list_rebalance]", + "+I[sys.release_all_kv_snapshot_lease]"); // make sure no more results is unread. assertResultsIgnoreOrder(showProceduresIterator, expectedShowProceduresResult, true); } @@ -779,6 +783,43 @@ void testListRebalanceProgress() throws Exception { }); } + @Test + void testReleaseAllKvSnapshotLeaseProcedure() throws Exception { + tEnv.executeSql( + "create table testcatalog.fluss.pk_table_test_kv_snapshot_lease (" + + "a int not null primary key not enforced, b varchar)"); + TablePath tablePath = TablePath.of("fluss", "pk_table_test_kv_snapshot_lease"); + + List rows = Arrays.asList(row(1, "v1"), row(2, "v2"), row(3, "v3")); + + // write records + writeRows(conn, tablePath, rows, false); + + FLUSS_CLUSTER_EXTENSION.triggerAndWaitSnapshot(tablePath); + + List expectedRows = Arrays.asList("+I[1, v1]", "+I[2, v2]", "+I[3, v3]"); + + String leaseId = "test-lease-kjhdds23"; + org.apache.flink.util.CloseableIterator rowIter = + tEnv.executeSql( + "select * from testcatalog.fluss.pk_table_test_kv_snapshot_lease " + + "/*+ OPTIONS('scan.kv.snapshot.lease.id' = '" + + leaseId + + "') */") + .collect(); + assertResultsIgnoreOrder(rowIter, expectedRows, false); + + // Lease will not be dropped automatically as the checkpoint not trigger. + ZooKeeperClient zkClient = FLUSS_CLUSTER_EXTENSION.getZooKeeperClient(); + assertThat(zkClient.getKvSnapshotLeaseMetadata(leaseId)).isPresent(); + tEnv.executeSql( + String.format( + "Call %s.sys.release_all_kv_snapshot_lease('" + leaseId + "' )", + CATALOG_NAME)) + .await(); + assertThat(zkClient.getKvSnapshotLeaseMetadata(leaseId)).isNotPresent(); + } + private static Configuration initConfig() { Configuration conf = new Configuration(); conf.setInt(ConfigOptions.DEFAULT_REPLICATION_FACTOR, 3); diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java index 5d903cb8d6..57beca3a8b 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java @@ -28,6 +28,7 @@ import org.apache.fluss.row.GenericRow; import org.apache.fluss.row.InternalRow; import org.apache.fluss.server.testutils.FlussClusterExtension; +import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.utils.clock.ManualClock; import org.apache.commons.lang3.RandomUtils; @@ -77,6 +78,7 @@ import static org.apache.fluss.flink.utils.FlinkTestBase.writeRowsToPartition; import static org.apache.fluss.server.testutils.FlussClusterExtension.BUILTIN_DATABASE; import static org.apache.fluss.testutils.DataTestUtils.row; +import static org.apache.fluss.testutils.common.CommonTestUtils.retry; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -352,6 +354,51 @@ void testPkTableReadMixSnapshotAndLog() throws Exception { assertResultsIgnoreOrder(rowIter, expectedRows, true); } + @Test + void testPkTableReadWithKvSnapshotLease() throws Exception { + tEnv.executeSql( + "create table pk_table_with_kv_snapshot_lease (a int not null primary key not enforced, b varchar)"); + TablePath tablePath = TablePath.of(DEFAULT_DB, "pk_table_with_kv_snapshot_lease"); + + List rows = Arrays.asList(row(1, "v1"), row(2, "v2"), row(3, "v3")); + + // write records + writeRows(conn, tablePath, rows, false); + + FLUSS_CLUSTER_EXTENSION.triggerAndWaitSnapshot(tablePath); + + // enable checkpoint to make sure the kv snapshot lease will be cleared. + execEnv.enableCheckpointing(100); + + List expectedRows = Arrays.asList("+I[1, v1]", "+I[2, v2]", "+I[3, v3]"); + org.apache.flink.util.CloseableIterator rowIter = + tEnv.executeSql( + "select * from pk_table_with_kv_snapshot_lease " + + "/*+ OPTIONS('scan.kv.snapshot.lease.id' = 'test-lease-10001') */") + .collect(); + assertResultsIgnoreOrder(rowIter, expectedRows, false); + + // now, we put rows to the table again, should read the log + expectedRows = + Arrays.asList( + "-U[1, v1]", + "+U[1, v1]", + "-U[2, v2]", + "+U[2, v2]", + "-U[3, v3]", + "+U[3, v3]"); + writeRows(conn, tablePath, rows, false); + assertResultsIgnoreOrder(rowIter, expectedRows, true); + + // check lease will be dropped after job finished. + ZooKeeperClient zkClient = FLUSS_CLUSTER_EXTENSION.getZooKeeperClient(); + retry( + Duration.ofMinutes(1), + () -> + assertThat(zkClient.getKvSnapshotLeaseMetadata("test-lease-10001")) + .isNotPresent()); + } + // ------------------------------------------------------------------------------------- // Fluss scan start mode tests // ------------------------------------------------------------------------------------- diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlussSourceITCase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlussSourceITCase.java index 9938fcce5e..05ded07232 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlussSourceITCase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlussSourceITCase.java @@ -45,6 +45,7 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import java.time.Duration; import java.util.Arrays; import java.util.List; import java.util.stream.Collectors; @@ -91,6 +92,8 @@ public void testTablePKSource() throws Exception { .setTable(pkTableName) .setStartingOffsets(OffsetsInitializer.earliest()) .setScanPartitionDiscoveryIntervalMs(1000L) + .setKvSnapshotLeaseId("test-lease-1vd7j9") + .setKvSnapshotLeaseDurationMs(Duration.ofDays(1).toMillis()) .setDeserializationSchema(new MockDataUtils.OrderDeserializationSchema()) .build(); @@ -123,6 +126,8 @@ public void testTablePKSourceWithProjectionPushdown() throws Exception { .setTable(pkTableName) .setStartingOffsets(OffsetsInitializer.earliest()) .setScanPartitionDiscoveryIntervalMs(1000L) + .setKvSnapshotLeaseId("test-lease-kj232df") + .setKvSnapshotLeaseDurationMs(Duration.ofDays(1).toMillis()) .setDeserializationSchema(new OrderPartialDeserializationSchema()) .setProjectedFields("orderId", "amount") .build(); @@ -151,6 +156,8 @@ public void testRowDataPKTableSource() throws Exception { .setTable(pkTableName) .setStartingOffsets(OffsetsInitializer.earliest()) .setScanPartitionDiscoveryIntervalMs(1000L) + .setKvSnapshotLeaseId("test-lease-sdafa3") + .setKvSnapshotLeaseDurationMs(Duration.ofDays(1).toMillis()) .setDeserializationSchema(new RowDataDeserializationSchema()) .build(); 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 238accf0bc..ac0393307f 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 @@ -27,6 +27,7 @@ import org.apache.fluss.flink.lake.split.LakeSnapshotSplit; import org.apache.fluss.flink.source.event.PartitionBucketsUnsubscribedEvent; import org.apache.fluss.flink.source.event.PartitionsRemovedEvent; +import org.apache.fluss.flink.source.reader.LeaseContext; import org.apache.fluss.flink.source.split.HybridSnapshotLogSplit; import org.apache.fluss.flink.source.split.LogSplit; import org.apache.fluss.flink.source.split.SnapshotSplit; @@ -113,7 +114,8 @@ void testPkTableNoSnapshotSplits() throws Throwable { DEFAULT_SCAN_PARTITION_DISCOVERY_INTERVAL_MS, streaming, null, - null); + null, + new LeaseContext("kv_snapshot_lease", Duration.ofDays(1).toMillis())); enumerator.start(); @@ -161,7 +163,8 @@ void testPkTableWithSnapshotSplits() throws Throwable { DEFAULT_SCAN_PARTITION_DISCOVERY_INTERVAL_MS, streaming, null, - null); + null, + new LeaseContext("kv_snapshot_lease1", Duration.ofDays(1).toMillis())); enumerator.start(); // register all read for (int i = 0; i < numSubtasks; i++) { @@ -233,7 +236,8 @@ void testNonPkTable() throws Throwable { DEFAULT_SCAN_PARTITION_DISCOVERY_INTERVAL_MS, streaming, null, - null); + null, + new LeaseContext(null, null)); enumerator.start(); @@ -280,7 +284,8 @@ void testReaderRegistrationTriggerAssignments() throws Throwable { DEFAULT_SCAN_PARTITION_DISCOVERY_INTERVAL_MS, streaming, null, - null); + null, + new LeaseContext("kv_snapshot_lease1", Duration.ofDays(1).toMillis())); enumerator.start(); @@ -317,7 +322,8 @@ void testAddSplitBack() throws Throwable { DEFAULT_SCAN_PARTITION_DISCOVERY_INTERVAL_MS, streaming, null, - null); + null, + new LeaseContext("kv_snapshot_lease1", Duration.ofDays(1).toMillis())); enumerator.start(); @@ -378,7 +384,8 @@ void testRestore() throws Throwable { DEFAULT_SCAN_PARTITION_DISCOVERY_INTERVAL_MS, streaming, null, - null); + null, + new LeaseContext("kv_snapshot_lease1", Duration.ofDays(1).toMillis())); enumerator.start(); assertThat(context.getSplitsAssignmentSequence()).isEmpty(); @@ -427,7 +434,12 @@ void testDiscoverPartitionsPeriodically(boolean isPrimaryKeyTable) throws Throwa streaming, null, null, - workExecutor)) { + workExecutor, + isPrimaryKeyTable + ? new LeaseContext( + "kv_snapshot_lease1", Duration.ofDays(1).toMillis()) + : new LeaseContext(null, null))) { + Map partitionNameByIds = waitUntilPartitions(zooKeeperClient, DEFAULT_TABLE_PATH); enumerator.start(); @@ -543,7 +555,9 @@ void testGetSplitOwner() throws Exception { DEFAULT_SCAN_PARTITION_DISCOVERY_INTERVAL_MS, streaming, null, - null)) { + null, + new LeaseContext( + "kv_snapshot_lease1", Duration.ofDays(1).toMillis()))) { // test splits for same non-partitioned bucket, should assign to same task TableBucket t1 = new TableBucket(tableId, 0); @@ -656,7 +670,9 @@ void testPartitionsExpiredInFlussButExistInLake( streaming, null, lakeSource, - workExecutor)) { + workExecutor, + new LeaseContext( + "kv_snapshot_lease1", Duration.ofDays(1).toMillis()))) { enumerator.start(); // Remove the hybrid partition to mock expire after enumerator start diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/state/SourceEnumeratorStateSerializerTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/state/SourceEnumeratorStateSerializerTest.java index 2c30bf086c..9ce6cade2c 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/state/SourceEnumeratorStateSerializerTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/state/SourceEnumeratorStateSerializerTest.java @@ -18,6 +18,7 @@ package org.apache.fluss.flink.source.state; import org.apache.fluss.flink.lake.split.LakeSnapshotAndFlussLogSplit; +import org.apache.fluss.flink.source.reader.LeaseContext; import org.apache.fluss.flink.source.split.HybridSnapshotLogSplit; import org.apache.fluss.flink.source.split.LogSplit; import org.apache.fluss.flink.source.split.SourceSplitBase; @@ -27,6 +28,8 @@ import org.apache.fluss.metadata.TableBucket; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import java.util.ArrayList; import java.util.Arrays; @@ -44,8 +47,9 @@ */ class SourceEnumeratorStateSerializerTest { - @Test - void testPendingSplitsCheckpointSerde() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testPendingSplitsCheckpointSerde(boolean isLogTable) throws Exception { FlussSourceEnumeratorStateSerializer serializer = new FlussSourceEnumeratorStateSerializer(new TestingLakeSource()); @@ -79,9 +83,16 @@ void testPendingSplitsCheckpointSerde() throws Exception { lakeHybridSplitBucket, "2024-01-01", lakeSplits, 300L, Long.MIN_VALUE); remainingHybridLakeFlussSplits.add(lakeHybridSplit); + // Add a LeaseContext + LeaseContext leaseContext = + isLogTable ? new LeaseContext(null, null) : new LeaseContext("leaseId", 1000L); + SourceEnumeratorState sourceEnumeratorState = new SourceEnumeratorState( - assignedBuckets, assignedPartitions, remainingHybridLakeFlussSplits); + assignedBuckets, + assignedPartitions, + remainingHybridLakeFlussSplits, + leaseContext); // serialize state with remaining hybrid lake fluss splits byte[] serialized = serializer.serialize(sourceEnumeratorState); @@ -107,7 +118,8 @@ void testV0Compatibility() throws Exception { assignedPartitions.put(1L, "partition1"); assignedPartitions.put(2L, "partition2"); SourceEnumeratorState sourceEnumeratorState = - new SourceEnumeratorState(assignedBuckets, assignedPartitions, null); + new SourceEnumeratorState( + assignedBuckets, assignedPartitions, null, new LeaseContext(null, null)); byte[] serialized = serializer.serializeV0(sourceEnumeratorState); // then deserialize @@ -124,7 +136,10 @@ void testV0Compatibility() throws Exception { remainingHybridLakeFlussSplits.add(logSplit); sourceEnumeratorState = new SourceEnumeratorState( - assignedBuckets, assignedPartitions, remainingHybridLakeFlussSplits); + assignedBuckets, + assignedPartitions, + remainingHybridLakeFlussSplits, + new LeaseContext(null, null)); serialized = serializer.serializeV0(sourceEnumeratorState); @@ -145,7 +160,8 @@ void testInconsistentLakeSourceSerde() throws Exception { assignedPartitions.put(1L, "partition1"); assignedPartitions.put(2L, "partition2"); SourceEnumeratorState sourceEnumeratorState = - new SourceEnumeratorState(assignedBuckets, assignedPartitions, null); + new SourceEnumeratorState( + assignedBuckets, assignedPartitions, null, new LeaseContext(null, null)); byte[] serialized = serializer.serialize(sourceEnumeratorState); // test deserialize with nonnull lake source diff --git a/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/AdminGateway.java b/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/AdminGateway.java index 3885bb387c..e2ec83ff71 100644 --- a/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/AdminGateway.java +++ b/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/AdminGateway.java @@ -17,6 +17,8 @@ package org.apache.fluss.rpc.gateway; +import org.apache.fluss.rpc.messages.AcquireKvSnapshotLeaseRequest; +import org.apache.fluss.rpc.messages.AcquireKvSnapshotLeaseResponse; import org.apache.fluss.rpc.messages.AddServerTagRequest; import org.apache.fluss.rpc.messages.AddServerTagResponse; import org.apache.fluss.rpc.messages.AlterClusterConfigsRequest; @@ -51,6 +53,8 @@ import org.apache.fluss.rpc.messages.RebalanceResponse; import org.apache.fluss.rpc.messages.RegisterProducerOffsetsRequest; import org.apache.fluss.rpc.messages.RegisterProducerOffsetsResponse; +import org.apache.fluss.rpc.messages.ReleaseKvSnapshotLeaseRequest; +import org.apache.fluss.rpc.messages.ReleaseKvSnapshotLeaseResponse; import org.apache.fluss.rpc.messages.RemoveServerTagRequest; import org.apache.fluss.rpc.messages.RemoveServerTagResponse; import org.apache.fluss.rpc.protocol.ApiKeys; @@ -186,6 +190,14 @@ CompletableFuture getProducerOffsets( CompletableFuture deleteProducerOffsets( DeleteProducerOffsetsRequest request); + @RPC(api = ApiKeys.ACQUIRE_KV_SNAPSHOT_LEASE) + CompletableFuture acquireKvSnapshotLease( + AcquireKvSnapshotLeaseRequest request); + + @RPC(api = ApiKeys.RELEASE_KV_SNAPSHOT_LEASE) + CompletableFuture releaseKvSnapshotLease( + ReleaseKvSnapshotLeaseRequest request); + // todo: rename table & alter table } diff --git a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java index 8aecca25f6..c9bce10ae8 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 @@ -97,7 +97,9 @@ public enum ApiKeys { PREPARE_LAKE_TABLE_SNAPSHOT(1052, 0, 0, PRIVATE), REGISTER_PRODUCER_OFFSETS(1053, 0, 0, PUBLIC), GET_PRODUCER_OFFSETS(1054, 0, 0, PUBLIC), - DELETE_PRODUCER_OFFSETS(1055, 0, 0, PUBLIC); + DELETE_PRODUCER_OFFSETS(1055, 0, 0, PUBLIC), + ACQUIRE_KV_SNAPSHOT_LEASE(1056, 0, 0, PUBLIC), + RELEASE_KV_SNAPSHOT_LEASE(1057, 0, 0, PUBLIC); private static final Map ID_TO_TYPE = Arrays.stream(ApiKeys.values()) diff --git a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java index 36ecd819ff..ffe99ad146 100644 --- a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java +++ b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java @@ -44,6 +44,7 @@ import org.apache.fluss.exception.InvalidTargetColumnException; import org.apache.fluss.exception.InvalidTimestampException; import org.apache.fluss.exception.InvalidUpdateVersionException; +import org.apache.fluss.exception.KvSnapshotLeaseNotExistException; import org.apache.fluss.exception.KvSnapshotNotExistException; import org.apache.fluss.exception.KvStorageException; import org.apache.fluss.exception.LakeStorageNotConfiguredException; @@ -245,7 +246,9 @@ public enum Errors { INVALID_PRODUCER_ID_EXCEPTION( 63, "The client has attempted to perform an operation with an invalid producer ID.", - InvalidProducerIdException::new); + InvalidProducerIdException::new), + KV_SNAPSHOT_LEASE_NOT_EXIST( + 64, "The kv snapshot lease is not exist.", KvSnapshotLeaseNotExistException::new); private static final Logger LOG = LoggerFactory.getLogger(Errors.class); diff --git a/fluss-rpc/src/main/proto/FlussApi.proto b/fluss-rpc/src/main/proto/FlussApi.proto index 99cdf32382..c3ad9840f6 100644 --- a/fluss-rpc/src/main/proto/FlussApi.proto +++ b/fluss-rpc/src/main/proto/FlussApi.proto @@ -378,6 +378,29 @@ message GetKvSnapshotMetadataResponse { repeated PbRemotePathAndLocalFile snapshot_files = 2; } +message AcquireKvSnapshotLeaseRequest { + required string lease_id = 1; + required int64 lease_duration = 2; + repeated PbKvSnapshotLeaseForTable table_lease_req = 3; +} + +message AcquireKvSnapshotLeaseResponse { + repeated PbKvSnapshotLeaseForTable tables_lease_res = 3; +} + +message ReleaseKvSnapshotLeaseRequest { + required string lease_id = 1; + repeated PbTable release_tables = 2; +} + +message ReleaseKvSnapshotLeaseResponse {} + +message DropKvSnapshotLeaseRequest { + required string lease_id = 1; +} + +message DropKvSnapshotLeaseResponse {} + message GetLatestLakeSnapshotRequest { required PbTablePath table_path = 1; } @@ -841,6 +864,16 @@ message PbTableBucket { required int32 bucket_id = 3; } +message PbTable { + required int64 table_id = 1; + repeated PbBucket buckets = 2; +} + +message PbBucket { + optional int64 partition_id = 1; + required int32 bucket_id = 2; +} + message PbAdjustIsrReqForTable { required int64 table_id = 1; repeated PbAdjustIsrReqForBucket buckets_req = 2; @@ -1138,4 +1171,15 @@ message PbDatabaseSummary { required string database_name = 1; required int64 created_time = 2; required int32 table_count = 3; +} + +message PbKvSnapshotLeaseForTable { + required int64 table_id = 1; + repeated PbKvSnapshotLeaseForBucket buckets_req = 2; +} + +message PbKvSnapshotLeaseForBucket { + optional int64 partition_id = 1; + required int32 bucket_id = 2; + required int64 snapshot_id = 3; } \ No newline at end of file diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CompletedSnapshotStoreManager.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CompletedSnapshotStoreManager.java index b7c92289bc..78c719f1e7 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CompletedSnapshotStoreManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CompletedSnapshotStoreManager.java @@ -26,6 +26,7 @@ import org.apache.fluss.server.kv.snapshot.CompletedSnapshotHandle; import org.apache.fluss.server.kv.snapshot.CompletedSnapshotHandleStore; import org.apache.fluss.server.kv.snapshot.CompletedSnapshotStore; +import org.apache.fluss.server.kv.snapshot.CompletedSnapshotStore.SubsumptionChecker; import org.apache.fluss.server.kv.snapshot.SharedKvFileRegistry; import org.apache.fluss.server.kv.snapshot.ZooKeeperCompletedSnapshotHandleStore; import org.apache.fluss.server.metrics.group.CoordinatorMetricGroup; @@ -64,19 +65,22 @@ public class CompletedSnapshotStoreManager { private final Executor ioExecutor; private final Function makeZookeeperCompletedSnapshotHandleStore; + private final SubsumptionChecker subsumptionChecker; private final CoordinatorMetricGroup coordinatorMetricGroup; public CompletedSnapshotStoreManager( int maxNumberOfSnapshotsToRetain, Executor ioExecutor, ZooKeeperClient zooKeeperClient, - CoordinatorMetricGroup coordinatorMetricGroup) { + CoordinatorMetricGroup coordinatorMetricGroup, + SubsumptionChecker subsumptionChecker) { this( maxNumberOfSnapshotsToRetain, ioExecutor, zooKeeperClient, ZooKeeperCompletedSnapshotHandleStore::new, - coordinatorMetricGroup); + coordinatorMetricGroup, + subsumptionChecker); } @VisibleForTesting @@ -86,13 +90,15 @@ public CompletedSnapshotStoreManager( ZooKeeperClient zooKeeperClient, Function makeZookeeperCompletedSnapshotHandleStore, - CoordinatorMetricGroup coordinatorMetricGroup) { + CoordinatorMetricGroup coordinatorMetricGroup, + SubsumptionChecker subsumptionChecker) { checkArgument( maxNumberOfSnapshotsToRetain > 0, "maxNumberOfSnapshotsToRetain must be positive"); this.maxNumberOfSnapshotsToRetain = maxNumberOfSnapshotsToRetain; this.zooKeeperClient = zooKeeperClient; this.bucketCompletedSnapshotStores = MapUtils.newConcurrentHashMap(); this.ioExecutor = ioExecutor; + this.subsumptionChecker = subsumptionChecker; this.makeZookeeperCompletedSnapshotHandleStore = makeZookeeperCompletedSnapshotHandleStore; this.coordinatorMetricGroup = coordinatorMetricGroup; @@ -237,7 +243,8 @@ private CompletedSnapshotStore createCompletedSnapshotStore( sharedKvFileRegistry, retrievedSnapshots, completedSnapshotHandleStore, - ioExecutor); + ioExecutor, + subsumptionChecker); } @VisibleForTesting 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 813c1f10ed..892cf845ae 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 @@ -32,6 +32,7 @@ import org.apache.fluss.exception.IneligibleReplicaException; import org.apache.fluss.exception.InvalidCoordinatorException; import org.apache.fluss.exception.InvalidUpdateVersionException; +import org.apache.fluss.exception.KvSnapshotLeaseNotExistException; import org.apache.fluss.exception.RebalanceFailureException; import org.apache.fluss.exception.ServerNotExistException; import org.apache.fluss.exception.ServerTagAlreadyExistException; @@ -47,6 +48,7 @@ import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePartition; import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.rpc.messages.AcquireKvSnapshotLeaseResponse; import org.apache.fluss.rpc.messages.AddServerTagResponse; import org.apache.fluss.rpc.messages.AdjustIsrResponse; import org.apache.fluss.rpc.messages.CancelRebalanceResponse; @@ -56,10 +58,13 @@ import org.apache.fluss.rpc.messages.ControlledShutdownResponse; import org.apache.fluss.rpc.messages.ListRebalanceProgressResponse; import org.apache.fluss.rpc.messages.PbCommitLakeTableSnapshotRespForTable; +import org.apache.fluss.rpc.messages.PbKvSnapshotLeaseForBucket; import org.apache.fluss.rpc.messages.RebalanceResponse; +import org.apache.fluss.rpc.messages.ReleaseKvSnapshotLeaseResponse; import org.apache.fluss.rpc.messages.RemoveServerTagResponse; import org.apache.fluss.rpc.protocol.ApiError; import org.apache.fluss.server.coordinator.event.AccessContextEvent; +import org.apache.fluss.server.coordinator.event.AcquireKvSnapshotLeaseEvent; import org.apache.fluss.server.coordinator.event.AddServerTagEvent; import org.apache.fluss.server.coordinator.event.AdjustIsrReceivedEvent; import org.apache.fluss.server.coordinator.event.CancelRebalanceEvent; @@ -83,6 +88,7 @@ import org.apache.fluss.server.coordinator.event.NotifyLakeTableOffsetEvent; import org.apache.fluss.server.coordinator.event.NotifyLeaderAndIsrResponseReceivedEvent; import org.apache.fluss.server.coordinator.event.RebalanceEvent; +import org.apache.fluss.server.coordinator.event.ReleaseKvSnapshotLeaseEvent; import org.apache.fluss.server.coordinator.event.RemoveServerTagEvent; import org.apache.fluss.server.coordinator.event.SchemaChangeEvent; import org.apache.fluss.server.coordinator.event.TableRegistrationChangeEvent; @@ -118,6 +124,8 @@ 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.server.zk.data.lease.KvSnapshotLeaseMetadataManager; +import org.apache.fluss.utils.clock.Clock; import org.apache.fluss.utils.types.Tuple2; import org.slf4j.Logger; @@ -179,6 +187,7 @@ public class CoordinatorEventProcessor implements EventProcessor { private final CoordinatorMetricGroup coordinatorMetricGroup; private final RebalanceManager rebalanceManager; + private final KvSnapshotLeaseManager kvSnapshotLeaseManager; private final CompletedSnapshotStoreManager completedSnapshotStoreManager; private final LakeTableHelper lakeTableHelper; @@ -192,7 +201,8 @@ public CoordinatorEventProcessor( CoordinatorMetricGroup coordinatorMetricGroup, Configuration conf, ExecutorService ioExecutor, - MetadataManager metadataManager) { + MetadataManager metadataManager, + Clock clock) { this.zooKeeperClient = zooKeeperClient; this.serverMetadataCache = serverMetadataCache; this.coordinatorChannelManager = coordinatorChannelManager; @@ -230,20 +240,30 @@ public CoordinatorEventProcessor( this.coordinatorRequestBatch = new CoordinatorRequestBatch( coordinatorChannelManager, coordinatorEventManager, coordinatorContext); + + String remoteDataDir = conf.getString(ConfigOptions.REMOTE_DATA_DIR); + this.kvSnapshotLeaseManager = + new KvSnapshotLeaseManager( + conf, + new KvSnapshotLeaseMetadataManager(zooKeeperClient, remoteDataDir), + coordinatorContext, + clock, + coordinatorMetricGroup); + this.completedSnapshotStoreManager = new CompletedSnapshotStoreManager( conf.getInt(ConfigOptions.KV_MAX_RETAINED_SNAPSHOTS), ioExecutor, zooKeeperClient, - coordinatorMetricGroup); + coordinatorMetricGroup, + kvSnapshotLeaseManager::snapshotLeaseNotExist); this.autoPartitionManager = autoPartitionManager; this.lakeTableTieringManager = lakeTableTieringManager; this.coordinatorMetricGroup = coordinatorMetricGroup; this.internalListenerName = conf.getString(ConfigOptions.INTERNAL_LISTENER_NAME); this.rebalanceManager = new RebalanceManager(this, zooKeeperClient); this.ioExecutor = ioExecutor; - this.lakeTableHelper = - new LakeTableHelper(zooKeeperClient, conf.getString(ConfigOptions.REMOTE_DATA_DIR)); + this.lakeTableHelper = new LakeTableHelper(zooKeeperClient, remoteDataDir); } public CoordinatorEventManager getCoordinatorEventManager() { @@ -292,6 +312,9 @@ public void startup() { // start rebalance manager. rebalanceManager.startup(); + + // start kv snapshot lease manager + kvSnapshotLeaseManager.start(); } public void shutdown() { @@ -440,6 +463,9 @@ private void initCoordinatorContext() throws Exception { "Load table and partition assignment success in {}ms when initializing coordinator context.", System.currentTimeMillis() - start4loadAssignment); + // load all kv snapshot lease from zookeeper when starting. + kvSnapshotLeaseManager.initialize(); + long end = System.currentTimeMillis(); LOG.info("Current total {} tables in the cluster.", coordinatorContext.allTables().size()); LOG.info( @@ -633,6 +659,18 @@ public void process(CoordinatorEvent event) { completeFromCallable( listRebalanceProgressEvent.getRespCallback(), () -> processListRebalanceProgress(listRebalanceProgressEvent)); + } else if (event instanceof AcquireKvSnapshotLeaseEvent) { + AcquireKvSnapshotLeaseEvent acquireKvSnapshotLeaseEvent = + (AcquireKvSnapshotLeaseEvent) event; + completeFromCallable( + acquireKvSnapshotLeaseEvent.getRespCallback(), + () -> tryProcessAcquireKvSnapshotLease(acquireKvSnapshotLeaseEvent)); + } else if (event instanceof ReleaseKvSnapshotLeaseEvent) { + ReleaseKvSnapshotLeaseEvent releaseKvSnapshotLeaseEvent = + (ReleaseKvSnapshotLeaseEvent) event; + completeFromCallable( + releaseKvSnapshotLeaseEvent.getRespCallback(), + () -> tryProcessReleaseKvSnapshotLease(releaseKvSnapshotLeaseEvent)); } else if (event instanceof AccessContextEvent) { AccessContextEvent accessContextEvent = (AccessContextEvent) event; processAccessContext(accessContextEvent); @@ -2035,6 +2073,53 @@ private ControlledShutdownResponse tryProcessControlledShutdown( return response; } + private AcquireKvSnapshotLeaseResponse tryProcessAcquireKvSnapshotLease( + AcquireKvSnapshotLeaseEvent event) throws Exception { + AcquireKvSnapshotLeaseResponse response = new AcquireKvSnapshotLeaseResponse(); + Map unavailableSnapshots = + kvSnapshotLeaseManager.acquireLease( + event.getLeaseId(), + event.getLeaseDuration(), + event.getTableIdToLeasedBucket()); + + Map> pbFailedTables = new HashMap<>(); + for (Map.Entry entry : unavailableSnapshots.entrySet()) { + TableBucket tb = entry.getKey(); + Long snapshotId = entry.getValue(); + PbKvSnapshotLeaseForBucket pbBucket = + new PbKvSnapshotLeaseForBucket().setBucketId(tb.getBucket()); + if (tb.getPartitionId() != null) { + pbBucket.setPartitionId(tb.getPartitionId()); + } + pbBucket.setSnapshotId(snapshotId); + pbFailedTables.computeIfAbsent(tb.getTableId(), k -> new ArrayList<>()).add(pbBucket); + } + + for (Map.Entry> entry : pbFailedTables.entrySet()) { + response.addTablesLeaseRe() + .setTableId(entry.getKey()) + .addAllBucketsReqs(entry.getValue()); + } + return response; + } + + private ReleaseKvSnapshotLeaseResponse tryProcessReleaseKvSnapshotLease( + ReleaseKvSnapshotLeaseEvent event) throws Exception { + ReleaseKvSnapshotLeaseResponse response = new ReleaseKvSnapshotLeaseResponse(); + Map> tableIdToReleasedBucket = event.getTableIdToReleasedBucket(); + if (tableIdToReleasedBucket.isEmpty()) { + // release all + boolean exist = kvSnapshotLeaseManager.releaseAll(event.getLeaseId()); + if (!exist) { + throw new KvSnapshotLeaseNotExistException( + "kv snapshot lease '" + event.getLeaseId() + "' not exits."); + } + } else { + kvSnapshotLeaseManager.release(event.getLeaseId(), tableIdToReleasedBucket); + } + return response; + } + private void validateFencedEvent(FencedCoordinatorEvent event) { TableBucket tb = event.getTableBucket(); if (coordinatorContext.getTablePathById(tb.getTableId()) == null) { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java index 7e06f61091..9ae3d6139b 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java @@ -44,6 +44,8 @@ import org.apache.fluss.shaded.zookeeper3.org.apache.zookeeper.KeeperException; import org.apache.fluss.utils.ExceptionUtils; import org.apache.fluss.utils.ExecutorUtils; +import org.apache.fluss.utils.clock.Clock; +import org.apache.fluss.utils.clock.SystemClock; import org.apache.fluss.utils.concurrent.ExecutorThreadFactory; import org.apache.fluss.utils.concurrent.FutureUtils; @@ -85,6 +87,7 @@ public class CoordinatorServer extends ServerBase { private final CompletableFuture terminationFuture; private final AtomicBoolean isShutDown = new AtomicBoolean(false); + private final Clock clock; @GuardedBy("lock") private String serverId; @@ -142,9 +145,14 @@ public class CoordinatorServer extends ServerBase { private LakeCatalogDynamicLoader lakeCatalogDynamicLoader; public CoordinatorServer(Configuration conf) { + this(conf, SystemClock.getInstance()); + } + + public CoordinatorServer(Configuration conf, Clock clock) { super(conf); validateConfigs(conf); this.terminationFuture = new CompletableFuture<>(); + this.clock = clock; } public static void main(String[] args) { @@ -250,7 +258,8 @@ protected void startServices() throws Exception { serverMetricGroup, conf, ioExecutor, - metadataManager); + metadataManager, + clock); coordinatorEventProcessor.startup(); createDefaultDatabase(); 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 b9340efc34..631befb8c3 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 @@ -52,6 +52,8 @@ import org.apache.fluss.metadata.TableDescriptor; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.rpc.gateway.CoordinatorGateway; +import org.apache.fluss.rpc.messages.AcquireKvSnapshotLeaseRequest; +import org.apache.fluss.rpc.messages.AcquireKvSnapshotLeaseResponse; import org.apache.fluss.rpc.messages.AddServerTagRequest; import org.apache.fluss.rpc.messages.AddServerTagResponse; import org.apache.fluss.rpc.messages.AdjustIsrRequest; @@ -108,6 +110,8 @@ import org.apache.fluss.rpc.messages.RebalanceResponse; import org.apache.fluss.rpc.messages.RegisterProducerOffsetsRequest; import org.apache.fluss.rpc.messages.RegisterProducerOffsetsResponse; +import org.apache.fluss.rpc.messages.ReleaseKvSnapshotLeaseRequest; +import org.apache.fluss.rpc.messages.ReleaseKvSnapshotLeaseResponse; import org.apache.fluss.rpc.messages.RemoveServerTagRequest; import org.apache.fluss.rpc.messages.RemoveServerTagResponse; import org.apache.fluss.rpc.netty.server.Session; @@ -124,6 +128,7 @@ import org.apache.fluss.server.authorizer.AclDeleteResult; import org.apache.fluss.server.authorizer.Authorizer; import org.apache.fluss.server.coordinator.event.AccessContextEvent; +import org.apache.fluss.server.coordinator.event.AcquireKvSnapshotLeaseEvent; import org.apache.fluss.server.coordinator.event.AddServerTagEvent; import org.apache.fluss.server.coordinator.event.AdjustIsrReceivedEvent; import org.apache.fluss.server.coordinator.event.CancelRebalanceEvent; @@ -134,6 +139,7 @@ import org.apache.fluss.server.coordinator.event.EventManager; import org.apache.fluss.server.coordinator.event.ListRebalanceProgressEvent; import org.apache.fluss.server.coordinator.event.RebalanceEvent; +import org.apache.fluss.server.coordinator.event.ReleaseKvSnapshotLeaseEvent; import org.apache.fluss.server.coordinator.event.RemoveServerTagEvent; import org.apache.fluss.server.coordinator.producer.ProducerOffsetsManager; import org.apache.fluss.server.coordinator.rebalance.goal.Goal; @@ -183,10 +189,12 @@ import static org.apache.fluss.server.coordinator.rebalance.goal.GoalUtils.getGoalByType; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.addTableOffsetsToResponse; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.fromTablePath; +import static org.apache.fluss.server.utils.ServerRpcMessageUtils.getAcquireKvSnapshotLeaseData; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.getAdjustIsrData; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.getCommitLakeTableSnapshotData; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.getCommitRemoteLogManifestData; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.getPartitionSpec; +import static org.apache.fluss.server.utils.ServerRpcMessageUtils.getReleaseKvSnapshotLeaseData; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.groupOffsetsByTableId; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.makeCreateAclsResponse; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.makeDropAclsResponse; @@ -856,6 +864,35 @@ public CompletableFuture controlledShutdown( return response; } + @Override + public CompletableFuture acquireKvSnapshotLease( + AcquireKvSnapshotLeaseRequest request) { + CompletableFuture response = new CompletableFuture<>(); + eventManagerSupplier + .get() + .put( + new AcquireKvSnapshotLeaseEvent( + request.getLeaseId(), + request.getLeaseDuration(), + getAcquireKvSnapshotLeaseData(request), + response)); + return response; + } + + @Override + public CompletableFuture releaseKvSnapshotLease( + ReleaseKvSnapshotLeaseRequest request) { + CompletableFuture response = new CompletableFuture<>(); + eventManagerSupplier + .get() + .put( + new ReleaseKvSnapshotLeaseEvent( + request.getLeaseId(), + getReleaseKvSnapshotLeaseData(request), + response)); + return response; + } + @Override public CompletableFuture alterClusterConfigs( AlterClusterConfigsRequest request) { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/KvSnapshotLeaseManager.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/KvSnapshotLeaseManager.java new file mode 100644 index 0000000000..60e47d5337 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/KvSnapshotLeaseManager.java @@ -0,0 +1,459 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.coordinator; + +import org.apache.fluss.annotation.VisibleForTesting; +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.metadata.KvSnapshotLeaseForBucket; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TableInfo; +import org.apache.fluss.metrics.MetricNames; +import org.apache.fluss.server.metrics.group.CoordinatorMetricGroup; +import org.apache.fluss.server.zk.data.lease.KvSnapshotLease; +import org.apache.fluss.server.zk.data.lease.KvSnapshotLeaseMetadataManager; +import org.apache.fluss.server.zk.data.lease.KvSnapshotTableLease; +import org.apache.fluss.utils.MapUtils; +import org.apache.fluss.utils.clock.Clock; +import org.apache.fluss.utils.concurrent.ExecutorThreadFactory; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.concurrent.GuardedBy; +import javax.annotation.concurrent.ThreadSafe; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.stream.Collectors; + +import static org.apache.fluss.utils.concurrent.LockUtils.inReadLock; +import static org.apache.fluss.utils.concurrent.LockUtils.inWriteLock; + +/** A manager to manage kv snapshot lease acquire, renew, release and drop. */ +@ThreadSafe +public class KvSnapshotLeaseManager { + private static final Logger LOG = LoggerFactory.getLogger(KvSnapshotLeaseManager.class); + + private final KvSnapshotLeaseMetadataManager metadataManager; + private final CoordinatorContext coordinatorContext; + private final Clock clock; + private final ScheduledExecutorService scheduledExecutor; + private final Configuration conf; + + private final Map leaseLocks = MapUtils.newConcurrentHashMap(); + /** lease id to kv snapshot lease. */ + @GuardedBy("leaseLocks") + private final Map kvSnapshotLeaseMap; + + private final ReadWriteLock refCountLock = new ReentrantReadWriteLock(); + + /** + * KvSnapshotLeaseForBucket to the ref count, which means this table bucket + snapshotId has + * been leased by how many lease id. + */ + private final Map refCount = + MapUtils.newConcurrentHashMap(); + + /** For metrics. */ + private final AtomicInteger leasedBucketCount = new AtomicInteger(0); + + public KvSnapshotLeaseManager( + Configuration conf, + KvSnapshotLeaseMetadataManager metadataManager, + CoordinatorContext coordinatorContext, + Clock clock, + CoordinatorMetricGroup coordinatorMetricGroup) { + this( + conf, + metadataManager, + coordinatorContext, + Executors.newScheduledThreadPool( + 1, new ExecutorThreadFactory("kv-snapshot-lease-cleaner")), + clock, + coordinatorMetricGroup); + } + + @VisibleForTesting + public KvSnapshotLeaseManager( + Configuration conf, + KvSnapshotLeaseMetadataManager metadataManager, + CoordinatorContext coordinatorContext, + ScheduledExecutorService scheduledExecutor, + Clock clock, + CoordinatorMetricGroup coordinatorMetricGroup) { + this.metadataManager = metadataManager; + this.conf = conf; + this.scheduledExecutor = scheduledExecutor; + this.coordinatorContext = coordinatorContext; + this.clock = clock; + this.kvSnapshotLeaseMap = MapUtils.newConcurrentHashMap(); + + registerMetrics(coordinatorMetricGroup); + } + + public void start() { + LOG.info("kv snapshot lease manager has been started."); + scheduledExecutor.scheduleWithFixedDelay( + this::expireLeases, + 0L, + conf.get(ConfigOptions.KV_SNAPSHOT_LEASE_EXPIRATION_CHECK_INTERVAL).toMillis(), + TimeUnit.MILLISECONDS); + } + + public void initialize() throws Exception { + for (String leaseId : metadataManager.getLeasesList()) { + Optional kvSnapshotLeaseOpt = metadataManager.getLease(leaseId); + if (kvSnapshotLeaseOpt.isPresent()) { + KvSnapshotLease kvSnapshotLease = kvSnapshotLeaseOpt.get(); + this.leaseLocks.put(leaseId, new ReentrantReadWriteLock()); + this.kvSnapshotLeaseMap.put(leaseId, kvSnapshotLease); + + initializeRefCount(kvSnapshotLease); + + leasedBucketCount.addAndGet(kvSnapshotLease.getLeasedSnapshotCount()); + } + } + } + + public boolean snapshotLeaseNotExist(KvSnapshotLeaseForBucket kvSnapshotLeaseForBucket) { + return inReadLock( + refCountLock, + () -> { + AtomicInteger count = refCount.get(kvSnapshotLeaseForBucket); + return count == null || count.get() <= 0; + }); + } + + /** + * Acquire kv snapshot lease. + * + * @param leaseId the lease id + * @param leaseDuration the lease duration + * @param tableIdToLeaseBucket the table id to lease bucket + * @return the map of unavailable snapshots that failed to be leased + */ + public Map acquireLease( + String leaseId, + long leaseDuration, + Map> tableIdToLeaseBucket) + throws Exception { + ReadWriteLock lock = leaseLocks.computeIfAbsent(leaseId, k -> new ReentrantReadWriteLock()); + return inWriteLock( + lock, + () -> { + // To record the unavailable snapshots such as the kv snapshotId to lease not + // exists. + Map unavailableSnapshots = new HashMap<>(); + + boolean update = kvSnapshotLeaseMap.containsKey(leaseId); + // set the expiration time as: current time + leaseDuration + long newExpirationTime = clock.milliseconds() + leaseDuration; + KvSnapshotLease kvSnapshotLease = + kvSnapshotLeaseMap.compute( + leaseId, + (key, existingLease) -> { + if (existingLease == null) { + LOG.info( + "kv snapshot lease '{}' has been acquired. The lease expiration " + + "time is {}", + leaseId, + newExpirationTime); + return new KvSnapshotLease(newExpirationTime); + } else { + existingLease.setExpirationTime(newExpirationTime); + return existingLease; + } + }); + + for (Map.Entry> entry : + tableIdToLeaseBucket.entrySet()) { + Long tableId = entry.getKey(); + TableInfo tableInfo = coordinatorContext.getTableInfoById(tableId); + int numBuckets = tableInfo.getNumBuckets(); + List buckets = entry.getValue(); + for (KvSnapshotLeaseForBucket bucket : buckets) { + + TableBucket tableBucket = bucket.getTableBucket(); + long kvSnapshotId = bucket.getKvSnapshotId(); + try { + boolean snapshotExists = + metadataManager.isSnapshotExists(tableBucket, kvSnapshotId); + if (!snapshotExists) { + unavailableSnapshots.put(tableBucket, kvSnapshotId); + continue; + } + } catch (Exception e) { + LOG.error( + "Failed to check snapshotExists for tableBucket when acquire kv " + + "snapshot kvSnapshotLease {}.", + tableBucket, + e); + unavailableSnapshots.put(tableBucket, kvSnapshotId); + continue; + } + + long originalSnapshotId = + kvSnapshotLease.acquireBucket( + tableBucket, kvSnapshotId, numBuckets); + if (originalSnapshotId == -1L) { + leasedBucketCount.incrementAndGet(); + } else { + // clear the original ref. + decrementRefCount( + new KvSnapshotLeaseForBucket( + tableBucket, originalSnapshotId)); + } + incrementRefCount(bucket); + } + } + + if (update) { + metadataManager.updateLease(leaseId, kvSnapshotLease); + } else { + metadataManager.registerLease(leaseId, kvSnapshotLease); + } + + return unavailableSnapshots; + }); + } + + public void release(String leaseId, Map> tableIdToUnregisterBucket) + throws Exception { + ReadWriteLock lock = leaseLocks.get(leaseId); + if (lock == null) { + return; + } + + inWriteLock( + lock, + () -> { + KvSnapshotLease lease = kvSnapshotLeaseMap.get(leaseId); + if (lease == null) { + return; + } + + for (Map.Entry> entry : + tableIdToUnregisterBucket.entrySet()) { + List buckets = entry.getValue(); + for (TableBucket bucket : buckets) { + long snapshotId = lease.releaseBucket(bucket); + if (snapshotId != -1L) { + leasedBucketCount.decrementAndGet(); + decrementRefCount(new KvSnapshotLeaseForBucket(bucket, snapshotId)); + } + } + } + + if (lease.isEmpty()) { + releaseAll(leaseId); + } else { + metadataManager.updateLease(leaseId, lease); + } + }); + } + + /** + * Release kv snapshot lease. + * + * @param leaseId the lease id + * @return true if clear success, false if lease not exist + */ + public boolean releaseAll(String leaseId) throws Exception { + ReadWriteLock lock = leaseLocks.get(leaseId); + if (lock == null) { + return false; + } + + boolean exist = + inWriteLock( + lock, + () -> { + KvSnapshotLease kvSnapshotLease = kvSnapshotLeaseMap.remove(leaseId); + if (kvSnapshotLease == null) { + return false; + } + + clearRefCount(kvSnapshotLease); + metadataManager.deleteLease(leaseId); + + LOG.info( + "kv snapshots of lease '" + + leaseId + + "' has been all released."); + return true; + }); + + leaseLocks.remove(leaseId); + return exist; + } + + private void initializeRefCount(KvSnapshotLease lease) { + for (Map.Entry tableEntry : + lease.getTableIdToTableLease().entrySet()) { + long tableId = tableEntry.getKey(); + KvSnapshotTableLease tableLease = tableEntry.getValue(); + if (tableLease.getBucketSnapshots() != null) { + Long[] snapshots = tableLease.getBucketSnapshots(); + for (int i = 0; i < snapshots.length; i++) { + if (snapshots[i] == -1L) { + continue; + } + + incrementRefCount( + new KvSnapshotLeaseForBucket( + new TableBucket(tableId, i), snapshots[i])); + } + } else { + Map partitionSnapshots = tableLease.getPartitionSnapshots(); + for (Map.Entry entry : partitionSnapshots.entrySet()) { + Long partitionId = entry.getKey(); + Long[] snapshots = entry.getValue(); + for (int i = 0; i < snapshots.length; i++) { + if (snapshots[i] == -1L) { + continue; + } + + incrementRefCount( + new KvSnapshotLeaseForBucket( + new TableBucket(tableId, partitionId, i), snapshots[i])); + } + } + } + } + } + + private void clearRefCount(KvSnapshotLease lease) { + for (Map.Entry tableEntry : + lease.getTableIdToTableLease().entrySet()) { + long tableId = tableEntry.getKey(); + KvSnapshotTableLease tableLease = tableEntry.getValue(); + if (tableLease.getBucketSnapshots() != null) { + Long[] snapshots = tableLease.getBucketSnapshots(); + for (int i = 0; i < snapshots.length; i++) { + if (snapshots[i] == -1L) { + continue; + } + decrementRefCount( + new KvSnapshotLeaseForBucket( + new TableBucket(tableId, i), snapshots[i])); + leasedBucketCount.decrementAndGet(); + } + } else { + Map partitionSnapshots = tableLease.getPartitionSnapshots(); + for (Map.Entry entry : partitionSnapshots.entrySet()) { + Long partitionId = entry.getKey(); + Long[] snapshots = entry.getValue(); + for (int i = 0; i < snapshots.length; i++) { + if (snapshots[i] == -1L) { + continue; + } + + decrementRefCount( + new KvSnapshotLeaseForBucket( + new TableBucket(tableId, partitionId, i), snapshots[i])); + leasedBucketCount.decrementAndGet(); + } + } + } + } + } + + private void incrementRefCount(KvSnapshotLeaseForBucket kvSnapshotLeaseForBucket) { + inWriteLock( + refCountLock, + () -> + refCount.computeIfAbsent( + kvSnapshotLeaseForBucket, k -> new AtomicInteger(0)) + .incrementAndGet()); + } + + private void decrementRefCount(KvSnapshotLeaseForBucket kvSnapshotLeaseForBucket) { + inWriteLock( + refCountLock, + () -> { + AtomicInteger atomicInteger = refCount.get(kvSnapshotLeaseForBucket); + if (atomicInteger != null) { + int decrementAndGet = atomicInteger.decrementAndGet(); + if (decrementAndGet <= 0) { + refCount.remove(kvSnapshotLeaseForBucket); + } + } + }); + } + + private void expireLeases() { + long currentTime = clock.milliseconds(); + // 1. First collect all expired lease IDs + List expiredLeaseIds = + kvSnapshotLeaseMap.entrySet().stream() + .filter(entry -> entry.getValue().getExpirationTime() < currentTime) + .map(Map.Entry::getKey) + .collect(Collectors.toList()); + + // 2. Then process each collected ID + expiredLeaseIds.forEach( + leaseId -> { + try { + releaseAll(leaseId); + } catch (Exception e) { + LOG.error("Failed to clear kv snapshot lease {}", leaseId, e); + } + }); + } + + private void registerMetrics(CoordinatorMetricGroup coordinatorMetricGroup) { + coordinatorMetricGroup.gauge(MetricNames.KV_SNAPSHOT_LEASE_COUNT, this::getLeaseCount); + // TODO register as table or bucket level. + coordinatorMetricGroup.gauge( + MetricNames.LEASED_KV_SNAPSHOT_COUNT, this::getLeasedBucketCount); + } + + @VisibleForTesting + int getLeaseCount() { + return kvSnapshotLeaseMap.size(); + } + + @VisibleForTesting + int getLeasedBucketCount() { + return leasedBucketCount.get(); + } + + @VisibleForTesting + int getRefCount(KvSnapshotLeaseForBucket kvSnapshotLeaseForBucket) { + return inReadLock( + refCountLock, + () -> { + AtomicInteger count = refCount.get(kvSnapshotLeaseForBucket); + return count == null ? 0 : count.get(); + }); + } + + @VisibleForTesting + KvSnapshotLease getKvSnapshotLease(String leaseId) { + return kvSnapshotLeaseMap.get(leaseId); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/AcquireKvSnapshotLeaseEvent.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/AcquireKvSnapshotLeaseEvent.java new file mode 100644 index 0000000000..36ef0f56d2 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/AcquireKvSnapshotLeaseEvent.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.coordinator.event; + +import org.apache.fluss.metadata.KvSnapshotLeaseForBucket; +import org.apache.fluss.rpc.messages.AcquireKvSnapshotLeaseResponse; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; + +/** An event for acquire a kv snapshot lease. */ +public class AcquireKvSnapshotLeaseEvent implements CoordinatorEvent { + private final String leaseId; + private final long leaseDuration; + private final Map> tableIdToLeasedBucket; + private final CompletableFuture respCallback; + + public AcquireKvSnapshotLeaseEvent( + String leaseId, + long leaseDuration, + Map> tableIdToLeasedBucket, + CompletableFuture respCallback) { + this.leaseId = leaseId; + this.leaseDuration = leaseDuration; + this.tableIdToLeasedBucket = tableIdToLeasedBucket; + this.respCallback = respCallback; + } + + public String getLeaseId() { + return leaseId; + } + + public long getLeaseDuration() { + return leaseDuration; + } + + public Map> getTableIdToLeasedBucket() { + return tableIdToLeasedBucket; + } + + public CompletableFuture getRespCallback() { + return respCallback; + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/ReleaseKvSnapshotLeaseEvent.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/ReleaseKvSnapshotLeaseEvent.java new file mode 100644 index 0000000000..9165cd419e --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/ReleaseKvSnapshotLeaseEvent.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.coordinator.event; + +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.rpc.messages.ReleaseKvSnapshotLeaseResponse; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; + +/** An event for release a kv snapshot lease for table buckets. */ +public class ReleaseKvSnapshotLeaseEvent implements CoordinatorEvent { + private final String leaseId; + private final Map> tableIdToReleasedBucket; + private final CompletableFuture respCallback; + + public ReleaseKvSnapshotLeaseEvent( + String leaseId, + Map> tableIdToReleasedBucket, + CompletableFuture respCallback) { + this.leaseId = leaseId; + this.tableIdToReleasedBucket = tableIdToReleasedBucket; + this.respCallback = respCallback; + } + + public String getLeaseId() { + return leaseId; + } + + public Map> getTableIdToReleasedBucket() { + return tableIdToReleasedBucket; + } + + public CompletableFuture getRespCallback() { + return respCallback; + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStore.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStore.java index b67f984668..dc2d9c8fee 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStore.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStore.java @@ -21,6 +21,7 @@ import org.apache.fluss.fs.FSDataOutputStream; import org.apache.fluss.fs.FileSystem; import org.apache.fluss.fs.FsPath; +import org.apache.fluss.metadata.KvSnapshotLeaseForBucket; import org.apache.fluss.metadata.TableBucket; import org.slf4j.Logger; @@ -66,6 +67,7 @@ public class CompletedSnapshotStore { private final Executor ioExecutor; private final SnapshotsCleaner snapshotsCleaner; + private final SubsumptionChecker subsumptionChecker; private final ReentrantLock lock = new ReentrantLock(); @@ -80,12 +82,14 @@ public CompletedSnapshotStore( SharedKvFileRegistry sharedKvFileRegistry, Collection completedSnapshots, CompletedSnapshotHandleStore completedSnapshotHandleStore, - Executor executor) { + Executor executor, + SubsumptionChecker subsumptionChecker) { this.maxNumberOfSnapshotsToRetain = maxNumberOfSnapshotsToRetain; this.sharedKvFileRegistry = sharedKvFileRegistry; this.completedSnapshots = new ArrayDeque<>(); this.completedSnapshots.addAll(completedSnapshots); this.completedSnapshotHandleStore = completedSnapshotHandleStore; + this.subsumptionChecker = subsumptionChecker; this.ioExecutor = executor; this.snapshotsCleaner = new SnapshotsCleaner(); } @@ -144,7 +148,8 @@ CompletedSnapshot addSnapshotAndSubsumeOldestOne( completedSnapshot.getTableBucket(), completedSnapshot.getSnapshotID()); snapshotsCleaner.addSubsumedSnapshot(completedSnapshot); - }); + }, + subsumptionChecker); findLowest(completedSnapshots) .ifPresent( @@ -168,7 +173,10 @@ public List getAllSnapshots() { } private static Optional subsume( - Deque snapshots, int numRetain, SubsumeAction subsumeAction) { + Deque snapshots, + int numRetain, + SubsumeAction subsumeAction, + SubsumptionChecker subsumptionChecker) { if (snapshots.isEmpty()) { return Optional.empty(); } @@ -178,7 +186,7 @@ private static Optional subsume( Iterator iterator = snapshots.iterator(); while (snapshots.size() > numRetain && iterator.hasNext()) { CompletedSnapshot next = iterator.next(); - if (canSubsume(next, latest)) { + if (canSubsume(next, latest, subsumptionChecker)) { // always return the subsumed snapshot with larger snapshot id. if (!lastSubsumedSnapshot.isPresent() || next.getSnapshotID() > lastSubsumedSnapshot.get().getSnapshotID()) { @@ -200,14 +208,23 @@ interface SubsumeAction { void subsume(CompletedSnapshot snapshot) throws Exception; } - private static boolean canSubsume(CompletedSnapshot next, CompletedSnapshot latest) { + /** A function to check whether a snapshot can be subsumed. */ + @FunctionalInterface + public interface SubsumptionChecker { + boolean canSubsume(KvSnapshotLeaseForBucket bucket); + } + + private static boolean canSubsume( + CompletedSnapshot next, + CompletedSnapshot latest, + SubsumptionChecker subsumptionChecker) { // if the snapshot is equal to the latest snapshot, it means it can't be subsumed if (next == latest) { return false; } - // else, we always subsume it as we will only keep single one snapshot currently - // todo: consider some client are pining this snapshot in FLUSS-54730210 - return true; + + return subsumptionChecker.canSubsume( + new KvSnapshotLeaseForBucket(next.getTableBucket(), next.getSnapshotID())); } /** diff --git a/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java b/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java index 3a8dd5c240..968a8b0e55 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 @@ -30,6 +30,7 @@ import org.apache.fluss.fs.FsPath; import org.apache.fluss.fs.token.ObtainedSecurityToken; import org.apache.fluss.metadata.DatabaseSummary; +import org.apache.fluss.metadata.KvSnapshotLeaseForBucket; import org.apache.fluss.metadata.PartitionSpec; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.ResolvedPartitionSpec; @@ -55,6 +56,7 @@ import org.apache.fluss.rpc.entity.PrefixLookupResultForBucket; import org.apache.fluss.rpc.entity.ProduceLogResultForBucket; import org.apache.fluss.rpc.entity.PutKvResultForBucket; +import org.apache.fluss.rpc.messages.AcquireKvSnapshotLeaseRequest; import org.apache.fluss.rpc.messages.AdjustIsrRequest; import org.apache.fluss.rpc.messages.AdjustIsrResponse; import org.apache.fluss.rpc.messages.AlterTableRequest; @@ -93,6 +95,7 @@ import org.apache.fluss.rpc.messages.PbAdjustIsrRespForBucket; import org.apache.fluss.rpc.messages.PbAdjustIsrRespForTable; import org.apache.fluss.rpc.messages.PbAlterConfig; +import org.apache.fluss.rpc.messages.PbBucket; import org.apache.fluss.rpc.messages.PbBucketMetadata; import org.apache.fluss.rpc.messages.PbBucketOffset; import org.apache.fluss.rpc.messages.PbCreateAclRespInfo; @@ -107,6 +110,8 @@ import org.apache.fluss.rpc.messages.PbFetchLogRespForTable; import org.apache.fluss.rpc.messages.PbKeyValue; import org.apache.fluss.rpc.messages.PbKvSnapshot; +import org.apache.fluss.rpc.messages.PbKvSnapshotLeaseForBucket; +import org.apache.fluss.rpc.messages.PbKvSnapshotLeaseForTable; import org.apache.fluss.rpc.messages.PbLakeSnapshotForBucket; import org.apache.fluss.rpc.messages.PbLakeTableOffsetForBucket; import org.apache.fluss.rpc.messages.PbLakeTableSnapshotInfo; @@ -136,6 +141,7 @@ import org.apache.fluss.rpc.messages.PbServerNode; import org.apache.fluss.rpc.messages.PbStopReplicaReqForBucket; import org.apache.fluss.rpc.messages.PbStopReplicaRespForBucket; +import org.apache.fluss.rpc.messages.PbTable; import org.apache.fluss.rpc.messages.PbTableBucket; import org.apache.fluss.rpc.messages.PbTableMetadata; import org.apache.fluss.rpc.messages.PbTableOffsets; @@ -149,6 +155,7 @@ import org.apache.fluss.rpc.messages.PutKvRequest; import org.apache.fluss.rpc.messages.PutKvResponse; import org.apache.fluss.rpc.messages.RebalanceResponse; +import org.apache.fluss.rpc.messages.ReleaseKvSnapshotLeaseRequest; import org.apache.fluss.rpc.messages.StopReplicaRequest; import org.apache.fluss.rpc.messages.StopReplicaResponse; import org.apache.fluss.rpc.messages.UpdateMetadataRequest; @@ -1968,6 +1975,48 @@ private static PbRebalancePlanForBucket toPbRebalancePlanForBucket( return pbRebalancePlanForBucket; } + public static Map> getAcquireKvSnapshotLeaseData( + AcquireKvSnapshotLeaseRequest request) { + Map> tableIdToLeasedBucket = new HashMap<>(); + for (PbKvSnapshotLeaseForTable leaseForTable : request.getTableLeaseReqsList()) { + long tableId = leaseForTable.getTableId(); + List bucketList = new ArrayList<>(); + for (PbKvSnapshotLeaseForBucket leaseForBucket : leaseForTable.getBucketsReqsList()) { + bucketList.add(getKvSnapshotLeaseForBucket(tableId, leaseForBucket)); + } + tableIdToLeasedBucket.put(tableId, bucketList); + } + return tableIdToLeasedBucket; + } + + public static Map> getReleaseKvSnapshotLeaseData( + ReleaseKvSnapshotLeaseRequest request) { + Map> tableIdToReleasedBucket = new HashMap<>(); + for (PbTable pbTable : request.getReleaseTablesList()) { + long tableId = pbTable.getTableId(); + List bucketList = new ArrayList<>(); + for (PbBucket pbBucket : pbTable.getBucketsList()) { + bucketList.add( + new TableBucket( + tableId, + pbBucket.hasPartitionId() ? pbBucket.getPartitionId() : null, + pbBucket.getBucketId())); + } + tableIdToReleasedBucket.put(tableId, bucketList); + } + return tableIdToReleasedBucket; + } + + private static KvSnapshotLeaseForBucket getKvSnapshotLeaseForBucket( + long tableId, PbKvSnapshotLeaseForBucket leaseForBucket) { + return new KvSnapshotLeaseForBucket( + new TableBucket( + tableId, + leaseForBucket.hasPartitionId() ? leaseForBucket.getPartitionId() : null, + leaseForBucket.getBucketId()), + leaseForBucket.getSnapshotId()); + } + private static Map mergeResponse( Map response, Map errors) { if (errors.isEmpty()) { 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 86f3b952f6..ff0e4fde14 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 @@ -63,6 +63,8 @@ import org.apache.fluss.server.zk.data.ZkData.CoordinatorZNode; import org.apache.fluss.server.zk.data.ZkData.DatabaseZNode; import org.apache.fluss.server.zk.data.ZkData.DatabasesZNode; +import org.apache.fluss.server.zk.data.ZkData.KvSnapshotLeaseZNode; +import org.apache.fluss.server.zk.data.ZkData.KvSnapshotLeasesZNode; import org.apache.fluss.server.zk.data.ZkData.LakeTableZNode; import org.apache.fluss.server.zk.data.ZkData.LeaderAndIsrZNode; import org.apache.fluss.server.zk.data.ZkData.PartitionIdZNode; @@ -85,6 +87,7 @@ import org.apache.fluss.server.zk.data.ZkData.WriterIdZNode; import org.apache.fluss.server.zk.data.lake.LakeTable; import org.apache.fluss.server.zk.data.lake.LakeTableSnapshot; +import org.apache.fluss.server.zk.data.lease.KvSnapshotLeaseMetadata; import org.apache.fluss.server.zk.data.producer.ProducerOffsets; import org.apache.fluss.shaded.curator5.org.apache.curator.framework.CuratorFramework; import org.apache.fluss.shaded.curator5.org.apache.curator.framework.api.BackgroundCallback; @@ -1039,6 +1042,36 @@ private Map> getBucketSnapshots( return snapshots; } + public List getKvSnapshotLeasesList() throws Exception { + return getChildren(KvSnapshotLeasesZNode.path()); + } + + public void registerKvSnapshotLeaseMetadata( + String leaseId, KvSnapshotLeaseMetadata leaseMetadata) throws Exception { + String path = KvSnapshotLeaseZNode.path(leaseId); + zkClient.create() + .creatingParentsIfNeeded() + .withMode(CreateMode.PERSISTENT) + .forPath(path, KvSnapshotLeaseZNode.encode(leaseMetadata)); + } + + public void updateKvSnapshotLeaseMetadata(String leaseId, KvSnapshotLeaseMetadata leaseMetadata) + throws Exception { + String path = KvSnapshotLeaseZNode.path(leaseId); + zkClient.setData().forPath(path, KvSnapshotLeaseZNode.encode(leaseMetadata)); + } + + public Optional getKvSnapshotLeaseMetadata(String leaseId) + throws Exception { + String path = KvSnapshotLeaseZNode.path(leaseId); + return getOrEmpty(path).map(KvSnapshotLeaseZNode::decode); + } + + public void deleteKvSnapshotLease(String leaseId) throws Exception { + String path = KvSnapshotLeaseZNode.path(leaseId); + zkClient.delete().forPath(path); + } + // -------------------------------------------------------------------------------------------- // Writer // -------------------------------------------------------------------------------------------- diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java index 43c1acbf44..b1e0e9493c 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java @@ -26,6 +26,8 @@ import org.apache.fluss.security.acl.ResourceType; import org.apache.fluss.server.zk.data.lake.LakeTable; import org.apache.fluss.server.zk.data.lake.LakeTableJsonSerde; +import org.apache.fluss.server.zk.data.lease.KvSnapshotLeaseMetadata; +import org.apache.fluss.server.zk.data.lease.KvSnapshotLeaseMetadataJsonSerde; import org.apache.fluss.server.zk.data.producer.ProducerOffsets; import org.apache.fluss.server.zk.data.producer.ProducerOffsetsJsonSerde; import org.apache.fluss.utils.json.JsonSerdeUtils; @@ -906,4 +908,38 @@ public static ProducerOffsets decode(byte[] json) { return JsonSerdeUtils.readValue(json, ProducerOffsetsJsonSerde.INSTANCE); } } + + // ------------------------------------------------------------------------------------------ + // ZNodes for Consumers. + // ------------------------------------------------------------------------------------------ + + /** The root znode for leases. It will record all the info of fluss leases. */ + public static final class LeasesNode { + public static String path() { + return "/leases"; + } + } + + /** The root znode for kv snapshot leases. */ + public static final class KvSnapshotLeasesZNode { + public static String path() { + return LeasesNode.path() + "/kv_snapshot"; + } + } + + /** The znode for kv snapshot lease zk data. */ + public static final class KvSnapshotLeaseZNode { + public static String path(String leaseId) { + return KvSnapshotLeasesZNode.path() + "/" + leaseId; + } + + public static byte[] encode(KvSnapshotLeaseMetadata kvSnapshotLeaseMetadata) { + return JsonSerdeUtils.writeValueAsBytes( + kvSnapshotLeaseMetadata, KvSnapshotLeaseMetadataJsonSerde.INSTANCE); + } + + public static KvSnapshotLeaseMetadata decode(byte[] json) { + return JsonSerdeUtils.readValue(json, KvSnapshotLeaseMetadataJsonSerde.INSTANCE); + } + } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLease.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLease.java new file mode 100644 index 0000000000..893fe8b530 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLease.java @@ -0,0 +1,198 @@ +/* + * 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.lease; + +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.utils.MapUtils; + +import javax.annotation.concurrent.NotThreadSafe; + +import java.util.Arrays; +import java.util.Map; +import java.util.Objects; + +/** The entity of kv snapshot lease. */ +@NotThreadSafe +public class KvSnapshotLease { + private long expirationTime; + + /** A map from table id to kv snapshot lease for one table. */ + private final Map tableIdToTableLease; + + public KvSnapshotLease(long expirationTime) { + this(expirationTime, MapUtils.newConcurrentHashMap()); + } + + public KvSnapshotLease( + long expirationTime, Map tableIdToTableLease) { + this.expirationTime = expirationTime; + this.tableIdToTableLease = tableIdToTableLease; + } + + public void setExpirationTime(long expirationTime) { + this.expirationTime = expirationTime; + } + + public long getExpirationTime() { + return expirationTime; + } + + public Map getTableIdToTableLease() { + return tableIdToTableLease; + } + + /** + * Acquire a bucket to the lease id. + * + * @param tableBucket table bucket + * @param snapshotId snapshot id + * @param bucketNum bucket number of this table or partition + * @return the original registered snapshotId. if -1 means the bucket is new registered + */ + public long acquireBucket(TableBucket tableBucket, long snapshotId, int bucketNum) { + Long[] bucketSnapshot; + Long partitionId = tableBucket.getPartitionId(); + long tableId = tableBucket.getTableId(); + int bucketId = tableBucket.getBucket(); + if (partitionId == null) { + // For none-partitioned table. + KvSnapshotTableLease tableLease = + tableIdToTableLease.computeIfAbsent( + tableId, + k -> { + Long[] array = new Long[bucketNum]; + Arrays.fill(array, -1L); + return new KvSnapshotTableLease(tableId, array); + }); + bucketSnapshot = tableLease.getBucketSnapshots(); + } else { + // For partitioned table. + + // first add partition to table. + KvSnapshotTableLease tableLease = + tableIdToTableLease.computeIfAbsent( + tableId, k -> new KvSnapshotTableLease(tableId)); + Map partitionSnapshots = tableLease.getPartitionSnapshots(); + // then add bucket to partition. + bucketSnapshot = + partitionSnapshots.computeIfAbsent( + partitionId, + k -> { + Long[] array = new Long[bucketNum]; + Arrays.fill(array, -1L); + return array; + }); + } + + if (bucketSnapshot == null || bucketSnapshot.length != bucketNum) { + throw new IllegalArgumentException( + "Bucket index is null, or input bucket number is not equal to the bucket number of the table."); + } + long originalSnapshotId = bucketSnapshot[bucketId]; + bucketSnapshot[bucketId] = snapshotId; + return originalSnapshotId; + } + + /** + * Release a bucket from the lease id. + * + * @param tableBucket table bucket + * @return the snapshot id of the unregistered bucket + */ + public long releaseBucket(TableBucket tableBucket) { + Long[] bucketIndex; + long tableId = tableBucket.getTableId(); + Long partitionId = tableBucket.getPartitionId(); + int bucketId = tableBucket.getBucket(); + KvSnapshotTableLease tableLease = tableIdToTableLease.get(tableId); + if (partitionId == null) { + // For none-partitioned table. + bucketIndex = tableLease.getBucketSnapshots(); + } else { + // For partitioned table. + bucketIndex = tableLease.getBucketSnapshots(partitionId); + } + + Long snapshotId = -1L; + if (bucketIndex != null) { + snapshotId = bucketIndex[bucketId]; + bucketIndex[bucketId] = -1L; + + boolean needRemove = true; + for (Long bucket : bucketIndex) { + if (bucket != -1L) { + needRemove = false; + break; + } + } + + if (needRemove) { + if (partitionId == null) { + tableIdToTableLease.remove(tableId); + } else { + Map partitionSnapshots = tableLease.getPartitionSnapshots(); + partitionSnapshots.remove(partitionId); + if (partitionSnapshots.isEmpty()) { + tableIdToTableLease.remove(tableId); + } + } + } + } + return snapshotId; + } + + public boolean isEmpty() { + return tableIdToTableLease.isEmpty(); + } + + public int getLeasedSnapshotCount() { + int count = 0; + for (KvSnapshotTableLease tableLease : tableIdToTableLease.values()) { + count += tableLease.getLeasedSnapshotCount(); + } + return count; + } + + @Override + public String toString() { + return "KvSnapshotLease{" + + "expirationTime=" + + expirationTime + + ", tableIdToTableLease=" + + tableIdToTableLease + + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof KvSnapshotLease)) { + return false; + } + KvSnapshotLease that = (KvSnapshotLease) o; + return expirationTime == that.expirationTime + && Objects.equals(tableIdToTableLease, that.tableIdToTableLease); + } + + @Override + public int hashCode() { + return Objects.hash(expirationTime, tableIdToTableLease); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadata.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadata.java new file mode 100644 index 0000000000..2c9a40a9c5 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadata.java @@ -0,0 +1,94 @@ +/* + * 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.lease; + +import org.apache.fluss.fs.FileSystem; +import org.apache.fluss.fs.FsPath; + +import java.io.IOException; +import java.util.Map; +import java.util.Objects; + +import static org.apache.fluss.metrics.registry.MetricRegistry.LOG; + +/** The zkNode data of kv snapshot lease. */ +public class KvSnapshotLeaseMetadata { + private final long expirationTime; + private final Map tableIdToRemoteMetadataFilePath; + + public KvSnapshotLeaseMetadata( + long expirationTime, Map tableIdToRemoteMetadataFilePath) { + this.expirationTime = expirationTime; + this.tableIdToRemoteMetadataFilePath = tableIdToRemoteMetadataFilePath; + } + + public long getExpirationTime() { + return expirationTime; + } + + public Map getTableIdToRemoteMetadataFilePath() { + return tableIdToRemoteMetadataFilePath; + } + + public void discard() { + // delete all remote metadata file. + tableIdToRemoteMetadataFilePath.values().forEach(this::delete); + } + + private void delete(FsPath fsPath) { + try { + FileSystem fileSystem = fsPath.getFileSystem(); + if (fileSystem.exists(fsPath)) { + fileSystem.delete(fsPath, false); + } + } catch (IOException e) { + LOG.warn( + "Error deleting remote file path of kv snapshot lease metadata at {}", + fsPath, + e); + } + } + + @Override + public String toString() { + return "KvSnapshotLeaseMetadata{" + + "expirationTime=" + + expirationTime + + ", tableIdToRemoteMetadataFilePath=" + + tableIdToRemoteMetadataFilePath + + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + KvSnapshotLeaseMetadata that = (KvSnapshotLeaseMetadata) o; + return expirationTime == that.expirationTime + && tableIdToRemoteMetadataFilePath.equals(that.tableIdToRemoteMetadataFilePath); + } + + @Override + public int hashCode() { + return Objects.hash(expirationTime, tableIdToRemoteMetadataFilePath); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataJsonSerde.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataJsonSerde.java new file mode 100644 index 0000000000..30c9367da8 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataJsonSerde.java @@ -0,0 +1,84 @@ +/* + * 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.lease; + +import org.apache.fluss.fs.FsPath; +import org.apache.fluss.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; +import org.apache.fluss.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.fluss.utils.json.JsonDeserializer; +import org.apache.fluss.utils.json.JsonSerializer; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +/** Json serializer and deserializer for {@link KvSnapshotLeaseMetadata}. */ +public class KvSnapshotLeaseMetadataJsonSerde + implements JsonSerializer, + JsonDeserializer { + + public static final KvSnapshotLeaseMetadataJsonSerde INSTANCE = + new KvSnapshotLeaseMetadataJsonSerde(); + + private static final String VERSION_KEY = "version"; + private static final String EXPIRATION_TIME = "expiration_time"; + private static final String TABLES = "tables"; + private static final String TABLE_ID = "table_id"; + private static final String KV_SNAPSHOT_PATH = "lease_metadata_path"; + + private static final int VERSION = 1; + + @Override + public void serialize(KvSnapshotLeaseMetadata kvSnapshotLeaseMetadata, JsonGenerator generator) + throws IOException { + generator.writeStartObject(); + generator.writeNumberField(VERSION_KEY, VERSION); + generator.writeNumberField(EXPIRATION_TIME, kvSnapshotLeaseMetadata.getExpirationTime()); + + generator.writeFieldName(TABLES); + generator.writeStartArray(); + for (Map.Entry entry : + kvSnapshotLeaseMetadata.getTableIdToRemoteMetadataFilePath().entrySet()) { + generator.writeStartObject(); + generator.writeNumberField(TABLE_ID, entry.getKey()); + generator.writeStringField(KV_SNAPSHOT_PATH, entry.getValue().getPath()); + generator.writeEndObject(); + } + // end tables + generator.writeEndArray(); + + // end root + generator.writeEndObject(); + } + + @Override + public KvSnapshotLeaseMetadata deserialize(JsonNode node) { + long expirationTime = node.get(EXPIRATION_TIME).asLong(); + + Map tableIdToRemoteMetadataFilePath = new HashMap<>(); + JsonNode tablesNode = node.get(TABLES); + + for (JsonNode tableNode : tablesNode) { + long tableId = tableNode.get(TABLE_ID).asLong(); + String kvSnapshotPath = tableNode.get(KV_SNAPSHOT_PATH).asText(); + tableIdToRemoteMetadataFilePath.put(tableId, new FsPath(kvSnapshotPath)); + } + + return new KvSnapshotLeaseMetadata(expirationTime, tableIdToRemoteMetadataFilePath); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataManager.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataManager.java new file mode 100644 index 0000000000..d269e8d763 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataManager.java @@ -0,0 +1,226 @@ +/* + * 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.lease; + +import org.apache.fluss.fs.FSDataInputStream; +import org.apache.fluss.fs.FSDataOutputStream; +import org.apache.fluss.fs.FileSystem; +import org.apache.fluss.fs.FsPath; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.server.zk.ZooKeeperClient; +import org.apache.fluss.server.zk.data.BucketSnapshot; +import org.apache.fluss.utils.FlussPaths; +import org.apache.fluss.utils.IOUtils; +import org.apache.fluss.utils.types.Tuple2; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayOutputStream; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static org.apache.fluss.utils.Preconditions.checkNotNull; + +/** + * The manager to handle {@link KvSnapshotLease} to register/update/delete metadata from zk and + * remote fs. + */ +public class KvSnapshotLeaseMetadataManager { + private static final Logger LOG = LoggerFactory.getLogger(KvSnapshotLeaseMetadataManager.class); + + private final ZooKeeperClient zkClient; + private final String remoteDataDir; + + public KvSnapshotLeaseMetadataManager(ZooKeeperClient zkClient, String remoteDataDir) { + this.zkClient = zkClient; + this.remoteDataDir = remoteDataDir; + } + + public List getLeasesList() throws Exception { + return zkClient.getKvSnapshotLeasesList(); + } + + /** + * Register a new kv snapshot lease to zk and remote fs. + * + * @param leaseId the lease id. + * @param lease the kv snapshot lease. + */ + public void registerLease(String leaseId, KvSnapshotLease lease) throws Exception { + Map tableIdToRemoteMetadataFsPath = generateMetadataFile(leaseId, lease); + + // generate remote fsPath of metadata. + KvSnapshotLeaseMetadata leaseMetadata = + new KvSnapshotLeaseMetadata( + lease.getExpirationTime(), tableIdToRemoteMetadataFsPath); + + // register kv snapshot metadata to zk. + try { + zkClient.registerKvSnapshotLeaseMetadata(leaseId, leaseMetadata); + } catch (Exception e) { + LOG.warn("Failed to register kv snapshot lease metadata to zk.", e); + leaseMetadata.discard(); + throw e; + } + } + + /** + * Update a kv snapshot lease to zk and remote fs. + * + * @param leaseId the lease id. + * @param kvSnapshotLease the kv snapshot lease. + */ + public void updateLease(String leaseId, KvSnapshotLease kvSnapshotLease) throws Exception { + // TODO change this to incremental update to avoid create too many remote metadata files. + + Optional originalLeaseMetadata = + zkClient.getKvSnapshotLeaseMetadata(leaseId); + + Map tableIdToNewRemoteMetadataFsPath = + generateMetadataFile(leaseId, kvSnapshotLease); + + // generate new kv snapshot lease metadata. + KvSnapshotLeaseMetadata newLeaseMetadata = + new KvSnapshotLeaseMetadata( + kvSnapshotLease.getExpirationTime(), tableIdToNewRemoteMetadataFsPath); + // register new snapshot metadata to zk. + try { + zkClient.updateKvSnapshotLeaseMetadata(leaseId, newLeaseMetadata); + } catch (Exception e) { + LOG.warn("Failed to update kv snapshot lease metadata to zk.", e); + newLeaseMetadata.discard(); + throw e; + } + + // discard original snapshot metadata. + originalLeaseMetadata.ifPresent(KvSnapshotLeaseMetadata::discard); + } + + /** + * Get a kv snapshot lease from zk and remote fs. + * + * @param leaseId the lease id. + * @return the kv snapshot lease. + */ + public Optional getLease(String leaseId) throws Exception { + Optional kvSnapshotLeaseMetadataOpt = + zkClient.getKvSnapshotLeaseMetadata(leaseId); + if (!kvSnapshotLeaseMetadataOpt.isPresent()) { + return Optional.empty(); + } + + KvSnapshotLeaseMetadata kvSnapshotLeaseMetadata = kvSnapshotLeaseMetadataOpt.get(); + KvSnapshotLease kvSnapshotLease = buildKvSnapshotLease(kvSnapshotLeaseMetadata); + return Optional.of(kvSnapshotLease); + } + + /** + * Delete a kv snapshot lease from zk and remote fs. + * + * @param leaseId the lease id. + */ + public void deleteLease(String leaseId) throws Exception { + Optional leaseMetadataOpt = + zkClient.getKvSnapshotLeaseMetadata(leaseId); + + // delete zk metadata. + zkClient.deleteKvSnapshotLease(leaseId); + + // delete remote metadata file. + leaseMetadataOpt.ifPresent(KvSnapshotLeaseMetadata::discard); + } + + /** + * Check whether the snapshot exists for the bucket in zookeeper. + * + * @param tableBucket the table bucket. + * @param snapshotId the snapshot id. + * @return true if the snapshot exists in the bucket. + */ + public boolean isSnapshotExists(TableBucket tableBucket, long snapshotId) throws Exception { + List> allSnapshotAndIds = + zkClient.getTableBucketAllSnapshotAndIds(tableBucket); + for (Tuple2 snapshotAndId : allSnapshotAndIds) { + if (snapshotAndId.f1 == snapshotId) { + return true; + } + } + return false; + } + + private Map generateMetadataFile(String leaseId, KvSnapshotLease lease) + throws Exception { + Map tableIdToMetadataFile = new HashMap<>(); + for (Map.Entry entry : + lease.getTableIdToTableLease().entrySet()) { + long tableId = entry.getKey(); + tableIdToMetadataFile.put( + tableId, generateMetadataFile(tableId, leaseId, entry.getValue())); + } + return tableIdToMetadataFile; + } + + private FsPath generateMetadataFile( + long tableId, String leaseId, KvSnapshotTableLease tableLease) throws Exception { + // get the remote file path to store the kv snapshot lease metadata of a table + FsPath remoteKvSnapshotLeaseFile = + FlussPaths.remoteKvSnapshotLeaseFile(remoteDataDir, leaseId, tableId); + // check whether the parent directory exists, if not, create the directory + FileSystem fileSystem = remoteKvSnapshotLeaseFile.getFileSystem(); + if (!fileSystem.exists(remoteKvSnapshotLeaseFile.getParent())) { + fileSystem.mkdirs(remoteKvSnapshotLeaseFile.getParent()); + } + + // serialize table lease to json bytes, and write to file. + byte[] jsonBytes = KvSnapshotTableLeaseJsonSerde.toJson(tableLease); + try (FSDataOutputStream outputStream = + fileSystem.create(remoteKvSnapshotLeaseFile, FileSystem.WriteMode.OVERWRITE)) { + outputStream.write(jsonBytes); + } + return remoteKvSnapshotLeaseFile; + } + + private KvSnapshotLease buildKvSnapshotLease(KvSnapshotLeaseMetadata kvSnapshotLeaseMetadata) + throws Exception { + Map tableIdToRemoteMetadataFilePath = + kvSnapshotLeaseMetadata.getTableIdToRemoteMetadataFilePath(); + Map tableIdToTableLease = new HashMap<>(); + for (Map.Entry entry : tableIdToRemoteMetadataFilePath.entrySet()) { + long tableId = entry.getKey(); + FsPath remoteMetadataFilePath = entry.getValue(); + tableIdToTableLease.put(tableId, buildKvSnapshotTableLease(remoteMetadataFilePath)); + } + return new KvSnapshotLease( + kvSnapshotLeaseMetadata.getExpirationTime(), tableIdToTableLease); + } + + private KvSnapshotTableLease buildKvSnapshotTableLease(FsPath remoteMetadataFilePath) + throws Exception { + checkNotNull(remoteMetadataFilePath); + FSDataInputStream inputStream = + remoteMetadataFilePath.getFileSystem().open(remoteMetadataFilePath); + try (ByteArrayOutputStream outputStream = new ByteArrayOutputStream()) { + IOUtils.copyBytes(inputStream, outputStream, true); + return KvSnapshotTableLeaseJsonSerde.fromJson(outputStream.toByteArray()); + } + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLease.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLease.java new file mode 100644 index 0000000000..4cebd27405 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLease.java @@ -0,0 +1,188 @@ +/* + * 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.lease; + +import org.apache.fluss.utils.MapUtils; + +import javax.annotation.Nullable; +import javax.annotation.concurrent.NotThreadSafe; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Map; +import java.util.Objects; + +/** The lease of kv snapshot for a table. */ +@NotThreadSafe +public class KvSnapshotTableLease { + private final long tableId; + private final @Nullable Long[] bucketSnapshots; + private final Map partitionSnapshots; + + public KvSnapshotTableLease(long tableId) { + this(tableId, null, MapUtils.newConcurrentHashMap()); + } + + public KvSnapshotTableLease(long tableId, Long[] bucketSnapshots) { + this(tableId, bucketSnapshots, Collections.emptyMap()); + } + + public KvSnapshotTableLease(long tableId, Map partitionSnapshots) { + this(tableId, null, partitionSnapshots); + } + + public KvSnapshotTableLease( + long tableId, @Nullable Long[] bucketSnapshots, Map partitionSnapshots) { + this.tableId = tableId; + this.bucketSnapshots = bucketSnapshots; + this.partitionSnapshots = partitionSnapshots; + } + + public long getTableId() { + return tableId; + } + + public @Nullable Long[] getBucketSnapshots() { + return bucketSnapshots; + } + + public @Nullable Long[] getBucketSnapshots(long partitionId) { + return partitionSnapshots.get(partitionId); + } + + public Map getPartitionSnapshots() { + return partitionSnapshots; + } + + public void addPartitionSnapshots(long partitionId, Long[] snapshots) { + if (bucketSnapshots != null) { + throw new IllegalStateException("This is an none partition table lease."); + } + partitionSnapshots.put(partitionId, snapshots); + } + + public int getLeasedSnapshotCount() { + int count = 0; + if (bucketSnapshots != null) { + for (Long snapshot : bucketSnapshots) { + if (snapshot != -1L) { + count++; + } + } + } else { + for (Long[] snapshots : partitionSnapshots.values()) { + for (Long snapshot : snapshots) { + if (snapshot != -1L) { + count++; + } + } + } + } + return count; + } + + @Override + public String toString() { + String partitionSnapshotsStr = formatLongArrayMap(partitionSnapshots); + return "KvSnapshotTableLease{" + + "tableId=" + + tableId + + ", bucketSnapshots=" + + Arrays.toString(bucketSnapshots) + + ", partitionSnapshots=" + + partitionSnapshotsStr + + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + KvSnapshotTableLease that = (KvSnapshotTableLease) o; + return tableId == that.tableId + && Arrays.equals(bucketSnapshots, that.bucketSnapshots) + && deepEqualsMapOfArrays(partitionSnapshots, that.partitionSnapshots); + } + + @Override + public int hashCode() { + int result = Objects.hash(tableId); + result = 31 * result + Arrays.hashCode(bucketSnapshots); + result = 31 * result + deepHashCodeMapOfArrays(partitionSnapshots); + return result; + } + + private static String formatLongArrayMap(Map map) { + if (map == null) { + return "null"; + } + StringBuilder sb = new StringBuilder("{"); + boolean first = true; + for (Map.Entry entry : map.entrySet()) { + if (!first) { + sb.append(", "); + } + sb.append(entry.getKey()).append("=").append(Arrays.toString(entry.getValue())); + first = false; + } + sb.append("}"); + return sb.toString(); + } + + private static boolean deepEqualsMapOfArrays(Map map1, Map map2) { + if (map1 == map2) { + return true; + } + if (map1 == null || map2 == null || map1.size() != map2.size()) { + return false; + } + + for (Map.Entry entry : map1.entrySet()) { + Long key = entry.getKey(); + Long[] value1 = entry.getValue(); + Long[] value2 = map2.get(key); + + if (value2 == null) { + return false; + } + + if (!Arrays.equals(value1, value2)) { + return false; + } + } + return true; + } + + private static int deepHashCodeMapOfArrays(Map map) { + if (map == null) { + return 0; + } + int hash = 0; + for (Map.Entry entry : map.entrySet()) { + Long key = entry.getKey(); + Long[] value = entry.getValue(); + // Combine key hash and array content hash + hash = 31 * hash + (Objects.hashCode(key) ^ Arrays.hashCode(value)); + } + return hash; + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLeaseJsonSerde.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLeaseJsonSerde.java new file mode 100644 index 0000000000..09dfbb3f32 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLeaseJsonSerde.java @@ -0,0 +1,116 @@ +/* + * 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.lease; + +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.HashMap; +import java.util.Map; + +/** Json serializer and deserializer for {@link KvSnapshotTableLease}. */ +public class KvSnapshotTableLeaseJsonSerde + implements JsonSerializer, JsonDeserializer { + + public static final KvSnapshotTableLeaseJsonSerde INSTANCE = + new KvSnapshotTableLeaseJsonSerde(); + + private static final String VERSION_KEY = "version"; + private static final String TABLE_ID = "table_id"; + private static final String PARTITION_SNAPSHOTS = "partition_snapshots"; + private static final String PARTITION_ID = "partition_id"; + private static final String BUCKET_SNAPSHOTS = "bucket_snapshots"; + + private static final int VERSION = 1; + + @Override + public void serialize(KvSnapshotTableLease kvSnapshotTableLease, JsonGenerator generator) + throws IOException { + generator.writeStartObject(); + generator.writeNumberField(VERSION_KEY, VERSION); + generator.writeNumberField(TABLE_ID, kvSnapshotTableLease.getTableId()); + + if (kvSnapshotTableLease.getBucketSnapshots() != null) { + // for none-partition table. + generator.writeArrayFieldStart(BUCKET_SNAPSHOTS); + for (Long snapshot : kvSnapshotTableLease.getBucketSnapshots()) { + generator.writeNumber(snapshot); + } + generator.writeEndArray(); + } else { + // for partition table. + Map partitionSnapshots = kvSnapshotTableLease.getPartitionSnapshots(); + if (partitionSnapshots != null && !partitionSnapshots.isEmpty()) { + generator.writeArrayFieldStart(PARTITION_SNAPSHOTS); + for (Map.Entry entry : partitionSnapshots.entrySet()) { + generator.writeStartObject(); + generator.writeNumberField(PARTITION_ID, entry.getKey()); + generator.writeArrayFieldStart(BUCKET_SNAPSHOTS); + for (Long snapshot : entry.getValue()) { + generator.writeNumber(snapshot); + } + generator.writeEndArray(); + generator.writeEndObject(); + } + generator.writeEndArray(); + } + } + } + + @Override + public KvSnapshotTableLease deserialize(JsonNode node) { + long tableId = node.get(TABLE_ID).asLong(); + if (node.has(BUCKET_SNAPSHOTS)) { + // for none-partition table. + Long[] bucketSnapshots = new Long[node.get(BUCKET_SNAPSHOTS).size()]; + for (int i = 0; i < bucketSnapshots.length; i++) { + bucketSnapshots[i] = node.get(BUCKET_SNAPSHOTS).get(i).asLong(); + } + return new KvSnapshotTableLease(tableId, bucketSnapshots); + } else { + // for partition table. + Map partitionSnapshots = new HashMap<>(); + JsonNode partitionSnapshotsNode = node.get(PARTITION_SNAPSHOTS); + for (JsonNode partitionSnapshotNode : partitionSnapshotsNode) { + long partitionId = partitionSnapshotNode.get(PARTITION_ID).asLong(); + Long[] bucketSnapshots = + new Long[partitionSnapshotNode.get(BUCKET_SNAPSHOTS).size()]; + for (int i = 0; i < bucketSnapshots.length; i++) { + bucketSnapshots[i] = + partitionSnapshotNode.get(BUCKET_SNAPSHOTS).get(i).asLong(); + } + partitionSnapshots.put(partitionId, bucketSnapshots); + } + return new KvSnapshotTableLease(tableId, partitionSnapshots); + } + } + + /** Serialize the {@link KvSnapshotTableLease} to json bytes using current version. */ + public static byte[] toJson(KvSnapshotTableLease kvSnapshotTableLease) { + return JsonSerdeUtils.writeValueAsBytes(kvSnapshotTableLease, INSTANCE); + } + + /** Deserialize the json bytes to {@link KvSnapshotTableLease}. */ + public static KvSnapshotTableLease fromJson(byte[] json) { + return JsonSerdeUtils.readValue(json, INSTANCE); + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CompletedSnapshotStoreManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CompletedSnapshotStoreManagerTest.java index b8fec59cab..2558be96f9 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CompletedSnapshotStoreManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CompletedSnapshotStoreManagerTest.java @@ -211,7 +211,8 @@ void testMetadataInconsistencyWithMetadataNotExistsException() throws Exception ioExecutor, zookeeperClient, zooKeeperClient -> completedSnapshotHandleStore, - TestingMetricGroups.COORDINATOR_METRICS); + TestingMetricGroups.COORDINATOR_METRICS, + bucket -> true); // Verify that only the valid snapshot remains CompletedSnapshotStore completedSnapshotStore = @@ -227,7 +228,8 @@ private CompletedSnapshotStoreManager createCompletedSnapshotStoreManager( maxNumberOfSnapshotsToRetain, ioExecutor, zookeeperClient, - TestingMetricGroups.COORDINATOR_METRICS); + TestingMetricGroups.COORDINATOR_METRICS, + bucket -> true); } private CompletedSnapshot getLatestCompletedSnapshot( diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java index d4ec8f8db2..0c1c11c0e6 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java @@ -78,6 +78,7 @@ import org.apache.fluss.testutils.common.AllCallbackWrapper; import org.apache.fluss.types.DataTypes; import org.apache.fluss.utils.ExceptionUtils; +import org.apache.fluss.utils.clock.SystemClock; import org.apache.fluss.utils.concurrent.ExecutorThreadFactory; import org.apache.fluss.utils.types.Tuple2; @@ -1051,7 +1052,8 @@ private CoordinatorEventProcessor buildCoordinatorEventProcessor() { TestingMetricGroups.COORDINATOR_METRICS, new Configuration(), Executors.newFixedThreadPool(1, new ExecutorThreadFactory("test-coordinator-io")), - metadataManager); + metadataManager, + SystemClock.getInstance()); } private void initCoordinatorChannel() throws Exception { diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/KvSnapshotLeaseManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/KvSnapshotLeaseManagerTest.java new file mode 100644 index 0000000000..74f62456b1 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/KvSnapshotLeaseManagerTest.java @@ -0,0 +1,470 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.coordinator; + +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.metadata.KvSnapshotLeaseForBucket; +import org.apache.fluss.metadata.PhysicalTablePath; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.server.metrics.group.TestingMetricGroups; +import org.apache.fluss.server.zk.NOPErrorHandler; +import org.apache.fluss.server.zk.ZooKeeperClient; +import org.apache.fluss.server.zk.ZooKeeperExtension; +import org.apache.fluss.server.zk.data.BucketSnapshot; +import org.apache.fluss.server.zk.data.lease.KvSnapshotLease; +import org.apache.fluss.server.zk.data.lease.KvSnapshotLeaseMetadataManager; +import org.apache.fluss.server.zk.data.lease.KvSnapshotTableLease; +import org.apache.fluss.testutils.common.AllCallbackWrapper; +import org.apache.fluss.testutils.common.ManuallyTriggeredScheduledExecutorService; +import org.apache.fluss.utils.clock.ManualClock; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +import java.nio.file.Path; +import java.time.Duration; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import static org.apache.fluss.record.TestData.DATA1_TABLE_ID_PK; +import static org.apache.fluss.record.TestData.DATA1_TABLE_INFO_PK; +import static org.apache.fluss.record.TestData.DATA1_TABLE_PATH_PK; +import static org.apache.fluss.record.TestData.PARTITION_TABLE_ID; +import static org.apache.fluss.record.TestData.PARTITION_TABLE_INFO; +import static org.apache.fluss.record.TestData.PARTITION_TABLE_PATH; +import static org.apache.fluss.server.coordinator.CoordinatorTestUtils.createServers; +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link KvSnapshotLeaseManager}. */ +public class KvSnapshotLeaseManagerTest { + + @RegisterExtension + public static final AllCallbackWrapper ZOO_KEEPER_EXTENSION_WRAPPER = + new AllCallbackWrapper<>(new ZooKeeperExtension()); + + private static final long PARTITION_ID_1 = 19001L; + private static final PhysicalTablePath PARTITION_TABLE_PATH_1 = + PhysicalTablePath.of(PARTITION_TABLE_PATH, "2024"); + + private static final long PARTITION_ID_2 = 19002L; + private static final PhysicalTablePath PARTITION_TABLE_PATH_2 = + PhysicalTablePath.of(PARTITION_TABLE_PATH, "2025"); + + private static final int NUM_BUCKETS = DATA1_TABLE_INFO_PK.getNumBuckets(); + private static final TableBucket t0b0 = new TableBucket(DATA1_TABLE_ID_PK, 0); + private static final TableBucket t0b1 = new TableBucket(DATA1_TABLE_ID_PK, 1); + private static final TableBucket t1p0b0 = + new TableBucket(PARTITION_TABLE_ID, PARTITION_ID_1, 0); + private static final TableBucket t1p0b1 = + new TableBucket(PARTITION_TABLE_ID, PARTITION_ID_1, 1); + private static final TableBucket t1p1b0 = + new TableBucket(PARTITION_TABLE_ID, PARTITION_ID_2, 0); + + protected static ZooKeeperClient zookeeperClient; + + private CoordinatorContext coordinatorContext; + private ManualClock manualClock; + private ManuallyTriggeredScheduledExecutorService clearLeaseScheduler; + private KvSnapshotLeaseManager kvSnapshotLeaseManager; + private KvSnapshotLeaseMetadataManager metadataManager; + + private @TempDir Path tempDir; + + @BeforeAll + static void beforeAll() { + zookeeperClient = + ZOO_KEEPER_EXTENSION_WRAPPER + .getCustomExtension() + .getZooKeeperClient(NOPErrorHandler.INSTANCE); + } + + @BeforeEach + void beforeEach() throws Exception { + initCoordinatorContext(); + Configuration conf = new Configuration(); + // set a huge expiration check interval to avoid expiration check. + conf.set(ConfigOptions.KV_SNAPSHOT_LEASE_EXPIRATION_CHECK_INTERVAL, Duration.ofDays(7)); + manualClock = new ManualClock(System.currentTimeMillis()); + clearLeaseScheduler = new ManuallyTriggeredScheduledExecutorService(); + metadataManager = new KvSnapshotLeaseMetadataManager(zookeeperClient, tempDir.toString()); + kvSnapshotLeaseManager = + new KvSnapshotLeaseManager( + conf, + metadataManager, + coordinatorContext, + clearLeaseScheduler, + manualClock, + TestingMetricGroups.COORDINATOR_METRICS); + kvSnapshotLeaseManager.start(); + initialZookeeper(); + } + + @AfterEach + void afterEach() { + ZOO_KEEPER_EXTENSION_WRAPPER.getCustomExtension().cleanupRoot(); + } + + private static void initialZookeeper() throws Exception { + List tableBuckets = Arrays.asList(t0b0, t0b1, t1p0b0, t1p0b1, t1p1b0); + for (TableBucket tb : tableBuckets) { + zookeeperClient.registerTableBucketSnapshot( + tb, new BucketSnapshot(0L, 0L, "test-path")); + } + } + + @Test + void testInitialize() throws Exception { + assertThat( + snapshotLeaseNotExists( + Arrays.asList( + new KvSnapshotLeaseForBucket(t0b0, 0L), + new KvSnapshotLeaseForBucket(t0b1, 0L), + new KvSnapshotLeaseForBucket(t1p0b0, 0L), + new KvSnapshotLeaseForBucket(t1p0b1, 0L), + new KvSnapshotLeaseForBucket(t1p1b0, 0L)))) + .isTrue(); + assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(0); + + // test initialize from zookeeper when coordinator is started. + KvSnapshotLease kvSnapshotLease = new KvSnapshotLease(1000L); + acquire(kvSnapshotLease, new KvSnapshotLeaseForBucket(t0b0, 0L)); + acquire(kvSnapshotLease, new KvSnapshotLeaseForBucket(t0b1, 0L)); + acquire(kvSnapshotLease, new KvSnapshotLeaseForBucket(t1p0b0, 0L)); + acquire(kvSnapshotLease, new KvSnapshotLeaseForBucket(t1p0b1, 0L)); + acquire(kvSnapshotLease, new KvSnapshotLeaseForBucket(t1p1b0, 0L)); + metadataManager.registerLease("lease1", kvSnapshotLease); + + kvSnapshotLeaseManager.initialize(); + + assertThat( + snapshotLeaseExists( + Arrays.asList( + new KvSnapshotLeaseForBucket(t0b0, 0L), + new KvSnapshotLeaseForBucket(t0b1, 0L), + new KvSnapshotLeaseForBucket(t1p0b0, 0L), + new KvSnapshotLeaseForBucket(t1p0b1, 0L), + new KvSnapshotLeaseForBucket(t1p1b0, 0L)))) + .isTrue(); + assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(5); + + // check detail content. + Map tableIdToTableLease = new HashMap<>(); + tableIdToTableLease.put( + DATA1_TABLE_ID_PK, + new KvSnapshotTableLease(DATA1_TABLE_ID_PK, new Long[] {0L, 0L, -1L})); + KvSnapshotTableLease leaseForPartitionTable = new KvSnapshotTableLease(PARTITION_TABLE_ID); + leaseForPartitionTable.addPartitionSnapshots(PARTITION_ID_1, new Long[] {0L, 0L, -1L}); + leaseForPartitionTable.addPartitionSnapshots(PARTITION_ID_2, new Long[] {0L, -1L, -1L}); + tableIdToTableLease.put(PARTITION_TABLE_ID, leaseForPartitionTable); + KvSnapshotLease expectedLease = new KvSnapshotLease(1000L, tableIdToTableLease); + assertThat(kvSnapshotLeaseManager.getKvSnapshotLease("lease1")).isEqualTo(expectedLease); + assertThat(metadataManager.getLease("lease1")).hasValue(expectedLease); + } + + @Test + void testAcquireAndRelease() throws Exception { + Map> tableIdToRegisterBucket = initRegisterBuckets(); + acquire("lease1", tableIdToRegisterBucket); + + // first register snapshot to zk. + zookeeperClient.registerTableBucketSnapshot( + t1p0b0, new BucketSnapshot(1L, 10L, "test-path")); + zookeeperClient.registerTableBucketSnapshot( + t1p0b1, new BucketSnapshot(1L, 10L, "test-path")); + + tableIdToRegisterBucket = new HashMap<>(); + tableIdToRegisterBucket.put( + PARTITION_TABLE_ID, + Arrays.asList( + new KvSnapshotLeaseForBucket(t1p0b0, 1L), + new KvSnapshotLeaseForBucket(t1p0b1, 1L))); + acquire("lease2", tableIdToRegisterBucket); + + assertThat( + snapshotLeaseExists( + Arrays.asList( + new KvSnapshotLeaseForBucket(t0b0, 0L), + new KvSnapshotLeaseForBucket(t0b1, 0L), + new KvSnapshotLeaseForBucket(t1p0b0, 0), + new KvSnapshotLeaseForBucket(t1p0b1, 0L), + new KvSnapshotLeaseForBucket(t1p1b0, 0L), + new KvSnapshotLeaseForBucket(t1p0b0, 1L), + new KvSnapshotLeaseForBucket(t1p0b1, 1L)))) + .isTrue(); + assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(7); + assertThat(kvSnapshotLeaseManager.getLeaseCount()).isEqualTo(2); + + // update lease register. + tableIdToRegisterBucket = new HashMap<>(); + zookeeperClient.registerTableBucketSnapshot(t0b0, new BucketSnapshot(1L, 10L, "test-path")); + tableIdToRegisterBucket.put( + DATA1_TABLE_ID_PK, + Collections.singletonList(new KvSnapshotLeaseForBucket(t0b0, 1L))); + acquire("lease1", tableIdToRegisterBucket); + assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(7); + + // new insert. + tableIdToRegisterBucket = new HashMap<>(); + TableBucket newTableBucket = new TableBucket(DATA1_TABLE_ID_PK, 2); + + zookeeperClient.registerTableBucketSnapshot( + newTableBucket, new BucketSnapshot(1L, 10L, "test-path")); + tableIdToRegisterBucket.put( + DATA1_TABLE_ID_PK, + Collections.singletonList(new KvSnapshotLeaseForBucket(newTableBucket, 1L))); + acquire("lease1", tableIdToRegisterBucket); + assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(8); + + // release + Map> tableIdToUnregisterBucket = new HashMap<>(); + tableIdToUnregisterBucket.put(DATA1_TABLE_ID_PK, Collections.singletonList(newTableBucket)); + release("lease1", tableIdToUnregisterBucket); + assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(7); + + // release a non-exist bucket. + tableIdToUnregisterBucket = new HashMap<>(); + tableIdToUnregisterBucket.put( + DATA1_TABLE_ID_PK, + Collections.singletonList(new TableBucket(DATA1_TABLE_ID_PK, PARTITION_ID_1, 2))); + release("lease1", tableIdToUnregisterBucket); + assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(7); + + // check detail content for lease1. + Map tableIdToTableLease = new HashMap<>(); + tableIdToTableLease.put( + DATA1_TABLE_ID_PK, + new KvSnapshotTableLease(DATA1_TABLE_ID_PK, new Long[] {1L, 0L, -1L})); + KvSnapshotTableLease leaseForPartitionTable = new KvSnapshotTableLease(PARTITION_TABLE_ID); + leaseForPartitionTable.addPartitionSnapshots(PARTITION_ID_1, new Long[] {0L, 0L, -1L}); + leaseForPartitionTable.addPartitionSnapshots(PARTITION_ID_2, new Long[] {0L, -1L, -1L}); + tableIdToTableLease.put(PARTITION_TABLE_ID, leaseForPartitionTable); + KvSnapshotLease expectedLease = + new KvSnapshotLease(manualClock.milliseconds() + 1000L, tableIdToTableLease); + assertThat(kvSnapshotLeaseManager.getKvSnapshotLease("lease1")).isEqualTo(expectedLease); + assertThat(metadataManager.getLease("lease1")).hasValue(expectedLease); + + // check detail content for lease2. + tableIdToTableLease = new HashMap<>(); + leaseForPartitionTable = new KvSnapshotTableLease(PARTITION_TABLE_ID); + leaseForPartitionTable.addPartitionSnapshots(PARTITION_ID_1, new Long[] {1L, 1L, -1L}); + tableIdToTableLease.put(PARTITION_TABLE_ID, leaseForPartitionTable); + KvSnapshotLease expectedLease2 = + new KvSnapshotLease(manualClock.milliseconds() + 1000L, tableIdToTableLease); + assertThat(kvSnapshotLeaseManager.getKvSnapshotLease("lease2")).isEqualTo(expectedLease2); + assertThat(metadataManager.getLease("lease2")).hasValue(expectedLease2); + } + + @Test + void testUnregisterAll() throws Exception { + Map> tableIdToRegisterBucket = initRegisterBuckets(); + acquire("lease1", tableIdToRegisterBucket); + assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(5); + assertThat(kvSnapshotLeaseManager.getLeaseCount()).isEqualTo(1); + assertThat(zookeeperClient.getKvSnapshotLeaseMetadata("lease1")).isPresent(); + + Map> tableIdToUnregisterBucket = new HashMap<>(); + tableIdToUnregisterBucket.put(DATA1_TABLE_ID_PK, Arrays.asList(t0b0, t0b1)); + tableIdToUnregisterBucket.put(PARTITION_TABLE_ID, Arrays.asList(t1p0b0, t1p0b1, t1p1b0)); + + // unregister all will clear this lease. + release("lease1", tableIdToUnregisterBucket); + assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(0); + assertThat(kvSnapshotLeaseManager.getLeaseCount()).isEqualTo(0); + assertThat(zookeeperClient.getKvSnapshotLeaseMetadata("lease1")).isNotPresent(); + } + + @Test + void testClear() throws Exception { + Map> tableIdToRegisterBucket = initRegisterBuckets(); + acquire("lease1", tableIdToRegisterBucket); + + // first register snapshot to zk. + zookeeperClient.registerTableBucketSnapshot( + t1p0b0, new BucketSnapshot(1L, 10L, "test-path")); + zookeeperClient.registerTableBucketSnapshot( + t1p0b1, new BucketSnapshot(1L, 10L, "test-path")); + tableIdToRegisterBucket = new HashMap<>(); + tableIdToRegisterBucket.put( + PARTITION_TABLE_ID, + Arrays.asList( + new KvSnapshotLeaseForBucket(t0b0, 0L), // same ref. + new KvSnapshotLeaseForBucket(t1p0b0, 1L), + new KvSnapshotLeaseForBucket(t1p0b1, 1L))); + acquire("lease2", tableIdToRegisterBucket); + + assertThat( + snapshotLeaseExists( + Arrays.asList( + new KvSnapshotLeaseForBucket(t0b0, 0L), + new KvSnapshotLeaseForBucket(t0b1, 0L), + new KvSnapshotLeaseForBucket(t1p0b0, 0L), + new KvSnapshotLeaseForBucket(t1p0b1, 0L), + new KvSnapshotLeaseForBucket(t1p1b0, 0L), + new KvSnapshotLeaseForBucket(t1p0b0, 1L), + new KvSnapshotLeaseForBucket(t1p0b1, 1L)))) + .isTrue(); + assertThat(kvSnapshotLeaseManager.getRefCount(new KvSnapshotLeaseForBucket(t0b0, 0L))) + .isEqualTo(2); + assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(8); + assertThat(kvSnapshotLeaseManager.getLeaseCount()).isEqualTo(2); + + kvSnapshotLeaseManager.releaseAll("lease1"); + assertThat(kvSnapshotLeaseManager.getRefCount(new KvSnapshotLeaseForBucket(t0b0, 0L))) + .isEqualTo(1); + assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(3); + assertThat(kvSnapshotLeaseManager.getLeaseCount()).isEqualTo(1); + assertThat(zookeeperClient.getKvSnapshotLeaseMetadata("lease1")).isEmpty(); + + kvSnapshotLeaseManager.releaseAll("lease2"); + assertThat(kvSnapshotLeaseManager.getRefCount(new KvSnapshotLeaseForBucket(t0b0, 0L))) + .isEqualTo(0); + assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(0); + assertThat(kvSnapshotLeaseManager.getLeaseCount()).isEqualTo(0); + assertThat(zookeeperClient.getKvSnapshotLeaseMetadata("lease2")).isEmpty(); + + assertThat(kvSnapshotLeaseManager.releaseAll("non-exist")).isFalse(); + } + + @Test + void testExpireLeases() throws Exception { + // test lease expire by expire thread. + Map> tableIdToLeaseBucket = initRegisterBuckets(); + + // expire after 1000ms. + kvSnapshotLeaseManager.acquireLease("lease1", 1000L, tableIdToLeaseBucket); + + tableIdToLeaseBucket = new HashMap<>(); + zookeeperClient.registerTableBucketSnapshot( + t1p0b0, new BucketSnapshot(1L, 10L, "test-path")); + zookeeperClient.registerTableBucketSnapshot( + t1p0b1, new BucketSnapshot(1L, 10L, "test-path")); + tableIdToLeaseBucket.put( + PARTITION_TABLE_ID, + Arrays.asList( + new KvSnapshotLeaseForBucket(t0b0, 0L), // same ref. + new KvSnapshotLeaseForBucket(t1p0b0, 1L), + new KvSnapshotLeaseForBucket(t1p0b1, 1L))); + // expire after 2000ms. + kvSnapshotLeaseManager.acquireLease("lease2", 2000L, tableIdToLeaseBucket); + + clearLeaseScheduler.triggerPeriodicScheduledTasks(); + // no lease expire. + assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(8); + assertThat(kvSnapshotLeaseManager.getLeaseCount()).isEqualTo(2); + assertThat(zookeeperClient.getKvSnapshotLeaseMetadata("lease1")).isPresent(); + assertThat(zookeeperClient.getKvSnapshotLeaseMetadata("lease2")).isPresent(); + + manualClock.advanceTime(1005L, TimeUnit.MILLISECONDS); + clearLeaseScheduler.triggerPeriodicScheduledTasks(); + // lease1 expire. + assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(3); + assertThat(kvSnapshotLeaseManager.getLeaseCount()).isEqualTo(1); + assertThat(zookeeperClient.getKvSnapshotLeaseMetadata("lease1")).isNotPresent(); + assertThat(zookeeperClient.getKvSnapshotLeaseMetadata("lease2")).isPresent(); + + manualClock.advanceTime(1005L, TimeUnit.MILLISECONDS); + clearLeaseScheduler.triggerPeriodicScheduledTasks(); + // lease2 expire. + assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(0); + assertThat(kvSnapshotLeaseManager.getLeaseCount()).isEqualTo(0); + assertThat(zookeeperClient.getKvSnapshotLeaseMetadata("lease1")).isNotPresent(); + assertThat(zookeeperClient.getKvSnapshotLeaseMetadata("lease2")).isNotPresent(); + } + + @Test + void registerWithNotExistSnapshotId() throws Exception { + Map> tableIdToRegisterBucket = new HashMap<>(); + tableIdToRegisterBucket.put( + DATA1_TABLE_ID_PK, + Arrays.asList( + new KvSnapshotLeaseForBucket(t0b0, 1000L), + new KvSnapshotLeaseForBucket(t0b1, 1000L))); + assertThat( + kvSnapshotLeaseManager + .acquireLease("lease1", 1000L, tableIdToRegisterBucket) + .keySet()) + .contains(t0b0); + } + + private void initCoordinatorContext() { + coordinatorContext = new CoordinatorContext(); + coordinatorContext.setLiveTabletServers(createServers(Arrays.asList(0, 1, 2))); + + // register an non-partitioned table. + coordinatorContext.putTableInfo(DATA1_TABLE_INFO_PK); + coordinatorContext.putTablePath(DATA1_TABLE_ID_PK, DATA1_TABLE_PATH_PK); + + // register a partitioned table. + coordinatorContext.putTableInfo(PARTITION_TABLE_INFO); + coordinatorContext.putTablePath( + PARTITION_TABLE_INFO.getTableId(), PARTITION_TABLE_INFO.getTablePath()); + coordinatorContext.putPartition(PARTITION_ID_1, PARTITION_TABLE_PATH_1); + coordinatorContext.putPartition(PARTITION_ID_2, PARTITION_TABLE_PATH_2); + } + + private Map> initRegisterBuckets() { + Map> tableIdToRegisterBucket = new HashMap<>(); + tableIdToRegisterBucket.put( + DATA1_TABLE_ID_PK, + Arrays.asList( + new KvSnapshotLeaseForBucket(t0b0, 0L), + new KvSnapshotLeaseForBucket(t0b1, 0L))); + tableIdToRegisterBucket.put( + PARTITION_TABLE_ID, + Arrays.asList( + new KvSnapshotLeaseForBucket(t1p0b0, 0L), + new KvSnapshotLeaseForBucket(t1p0b1, 0L), + new KvSnapshotLeaseForBucket(t1p1b0, 0L))); + return tableIdToRegisterBucket; + } + + private boolean snapshotLeaseNotExists(List bucketList) { + return bucketList.stream() + .allMatch(bucket -> kvSnapshotLeaseManager.snapshotLeaseNotExist(bucket)); + } + + private boolean snapshotLeaseExists(List bucketList) { + return bucketList.stream() + .noneMatch(bucket -> kvSnapshotLeaseManager.snapshotLeaseNotExist(bucket)); + } + + private void acquire( + String leaseId, Map> tableIdToLeaseBucket) + throws Exception { + kvSnapshotLeaseManager.acquireLease(leaseId, 1000L, tableIdToLeaseBucket); + } + + private void release(String leaseId, Map> tableIdToReleaseBucket) + throws Exception { + kvSnapshotLeaseManager.release(leaseId, tableIdToReleaseBucket); + } + + private long acquire(KvSnapshotLease kvSnapshotLease, KvSnapshotLeaseForBucket leaseForBucket) { + return kvSnapshotLease.acquireBucket( + leaseForBucket.getTableBucket(), leaseForBucket.getKvSnapshotId(), NUM_BUCKETS); + } +} 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 a0a49e027b..b6d567b8cd 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 @@ -22,6 +22,8 @@ import org.apache.fluss.exception.NetworkException; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.rpc.gateway.CoordinatorGateway; +import org.apache.fluss.rpc.messages.AcquireKvSnapshotLeaseRequest; +import org.apache.fluss.rpc.messages.AcquireKvSnapshotLeaseResponse; import org.apache.fluss.rpc.messages.AddServerTagRequest; import org.apache.fluss.rpc.messages.AddServerTagResponse; import org.apache.fluss.rpc.messages.AdjustIsrRequest; @@ -100,6 +102,8 @@ import org.apache.fluss.rpc.messages.RebalanceResponse; import org.apache.fluss.rpc.messages.RegisterProducerOffsetsRequest; import org.apache.fluss.rpc.messages.RegisterProducerOffsetsResponse; +import org.apache.fluss.rpc.messages.ReleaseKvSnapshotLeaseRequest; +import org.apache.fluss.rpc.messages.ReleaseKvSnapshotLeaseResponse; import org.apache.fluss.rpc.messages.RemoveServerTagRequest; import org.apache.fluss.rpc.messages.RemoveServerTagResponse; import org.apache.fluss.rpc.messages.TableExistsRequest; @@ -397,6 +401,18 @@ public CompletableFuture controlledShutdown( throw new UnsupportedOperationException(); } + @Override + public CompletableFuture acquireKvSnapshotLease( + AcquireKvSnapshotLeaseRequest request) { + throw new UnsupportedOperationException(); + } + + @Override + public CompletableFuture releaseKvSnapshotLease( + ReleaseKvSnapshotLeaseRequest request) { + throw new UnsupportedOperationException(); + } + @Override public CompletableFuture listAcls(ListAclsRequest request) { throw new UnsupportedOperationException(); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/RebalanceManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/RebalanceManagerTest.java index fc36e3034f..0c8a0e4415 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/RebalanceManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/RebalanceManagerTest.java @@ -33,6 +33,7 @@ import org.apache.fluss.server.zk.ZooKeeperExtension; import org.apache.fluss.server.zk.data.RebalanceTask; import org.apache.fluss.testutils.common.AllCallbackWrapper; +import org.apache.fluss.utils.clock.SystemClock; import org.apache.fluss.utils.concurrent.ExecutorThreadFactory; import org.junit.jupiter.api.AfterEach; @@ -126,6 +127,7 @@ private CoordinatorEventProcessor buildCoordinatorEventProcessor() { TestingMetricGroups.COORDINATOR_METRICS, new Configuration(), Executors.newFixedThreadPool(1, new ExecutorThreadFactory("test-coordinator-io")), - metadataManager); + metadataManager, + SystemClock.getInstance()); } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachineTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachineTest.java index bafb477c54..a34ff1e238 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachineTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachineTest.java @@ -45,6 +45,7 @@ import org.apache.fluss.server.zk.data.LeaderAndIsr; import org.apache.fluss.shaded.guava32.com.google.common.collect.Sets; import org.apache.fluss.testutils.common.AllCallbackWrapper; +import org.apache.fluss.utils.clock.SystemClock; import org.apache.fluss.utils.concurrent.ExecutorThreadFactory; import org.junit.jupiter.api.BeforeAll; @@ -268,7 +269,8 @@ void testStateChangeToOnline() throws Exception { new MetadataManager( zookeeperClient, new Configuration(), - new LakeCatalogDynamicLoader(new Configuration(), null, true))); + new LakeCatalogDynamicLoader(new Configuration(), null, true)), + SystemClock.getInstance()); CoordinatorEventManager eventManager = new CoordinatorEventManager( coordinatorEventProcessor, TestingMetricGroups.COORDINATOR_METRICS); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStoreTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStoreTest.java index 1ab73cfee4..78eaa97506 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStoreTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStoreTest.java @@ -557,7 +557,8 @@ private CompletedSnapshotStore createCompletedSnapshotStore( sharedKvFileRegistry, completedSnapshots, snapshotHandleStore, - executorService); + executorService, + bucket -> true); } private List> createSnapshotHandles(int num) { diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/snapshot/KvTabletSnapshotTargetTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/snapshot/KvTabletSnapshotTargetTest.java index 4077e0e65e..0cd2f92d2e 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/kv/snapshot/KvTabletSnapshotTargetTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/snapshot/KvTabletSnapshotTargetTest.java @@ -480,7 +480,8 @@ private KvTabletSnapshotTarget createSnapshotTarget( sharedKvFileRegistry, Collections.emptyList(), snapshotHandleStore, - executor); + executor, + (consumeKvSnapshotForBucket) -> true); // only retain the latest snapshot. RocksIncrementalSnapshot rocksIncrementalSnapshot = createIncrementalSnapshot(snapshotFailType); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/snapshot/ZooKeeperCompletedSnapshotStoreTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/snapshot/ZooKeeperCompletedSnapshotStoreTest.java index c6e90ff630..e6729f9829 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/kv/snapshot/ZooKeeperCompletedSnapshotStoreTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/snapshot/ZooKeeperCompletedSnapshotStoreTest.java @@ -139,6 +139,7 @@ private CompletedSnapshotStore createZooKeeperSnapshotStore( sharedKvFileRegistry, Collections.emptyList(), snapshotsInZooKeeper, - Executors.directExecutor()); + Executors.directExecutor(), + (consumeKvSnapshotForBucket) -> true); // only retain the latest snapshot. } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java b/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java index 8eaf29a0bb..90f28dd95f 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java @@ -268,7 +268,7 @@ public void startCoordinatorServer() throws Exception { conf.setString(ConfigOptions.ZOOKEEPER_ADDRESS, zooKeeperServer.getConnectString()); conf.setString(ConfigOptions.BIND_LISTENERS, coordinatorServerListeners); setRemoteDataDir(conf); - coordinatorServer = new CoordinatorServer(conf); + coordinatorServer = new CoordinatorServer(conf, clock); coordinatorServer.start(); coordinatorServerInfo = // TODO, Currently, we use 0 as coordinator server id. diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java index 94cb34b233..da4e8ef994 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java @@ -23,6 +23,7 @@ import org.apache.fluss.cluster.rebalance.ServerTag; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; +import org.apache.fluss.fs.FsPath; import org.apache.fluss.metadata.DatabaseSummary; import org.apache.fluss.metadata.Schema; import org.apache.fluss.metadata.SchemaInfo; @@ -41,6 +42,8 @@ import org.apache.fluss.server.zk.data.TableAssignment; import org.apache.fluss.server.zk.data.TableRegistration; import org.apache.fluss.server.zk.data.TabletServerRegistration; +import org.apache.fluss.server.zk.data.lease.KvSnapshotLease; +import org.apache.fluss.server.zk.data.lease.KvSnapshotLeaseMetadata; import org.apache.fluss.shaded.curator5.org.apache.curator.CuratorZookeeperClient; import org.apache.fluss.shaded.curator5.org.apache.curator.framework.CuratorFramework; import org.apache.fluss.shaded.zookeeper3.org.apache.zookeeper.KeeperException; @@ -489,6 +492,36 @@ void testTableBucketSnapshot() throws Exception { assertThat(zookeeperClient.getTableBucketSnapshot(table1Bucket2, 1)).isEmpty(); } + @Test + void testKvSnapshotLease() throws Exception { + Map tableIdToRemotePath = new HashMap<>(); + tableIdToRemotePath.put(150002L, new FsPath("/test/cp1")); + KvSnapshotLeaseMetadata leaseMetadata = + new KvSnapshotLeaseMetadata(1000L, tableIdToRemotePath); + + assertThat(zookeeperClient.getKvSnapshotLeasesList()).isEmpty(); + zookeeperClient.registerKvSnapshotLeaseMetadata("lease1", leaseMetadata); + assertThat(zookeeperClient.getKvSnapshotLeasesList()).containsExactly("lease1"); + + Optional metadataOpt = + zookeeperClient.getKvSnapshotLeaseMetadata("lease1"); + assertThat(metadataOpt.isPresent()).isTrue(); + assertThat(metadataOpt.get()).isEqualTo(leaseMetadata); + + tableIdToRemotePath = new HashMap<>(); + tableIdToRemotePath.put(150002L, new FsPath("/test/cp2")); + leaseMetadata = new KvSnapshotLeaseMetadata(1000L, tableIdToRemotePath); + zookeeperClient.updateKvSnapshotLeaseMetadata("lease1", leaseMetadata); + + metadataOpt = zookeeperClient.getKvSnapshotLeaseMetadata("lease1"); + assertThat(metadataOpt.isPresent()).isTrue(); + assertThat(metadataOpt.get()).isEqualTo(leaseMetadata); + + zookeeperClient.deleteKvSnapshotLease("lease1"); + metadataOpt = zookeeperClient.getKvSnapshotLeaseMetadata("lease1"); + assertThat(metadataOpt).isNotPresent(); + } + @Test void testGetWriterIdAndIncrement() throws Exception { // init @@ -704,4 +737,13 @@ void testGetDatabaseSummary() throws Exception { .isGreaterThanOrEqualTo(beforeCreateTime) .isLessThanOrEqualTo(afterCreateTime); } + + private void registerBucket( + KvSnapshotLease consumer, TableBucket tb, long kvSnapshotId, int bucketNum) { + consumer.acquireBucket(tb, kvSnapshotId, bucketNum); + } + + private void unregisterBucket(KvSnapshotLease consumer, TableBucket tb) { + consumer.releaseBucket(tb); + } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataJsonSerdeTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataJsonSerdeTest.java new file mode 100644 index 0000000000..b4ed0654d8 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataJsonSerdeTest.java @@ -0,0 +1,57 @@ +/* + * 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.lease; + +import org.apache.fluss.fs.FsPath; +import org.apache.fluss.utils.json.JsonSerdeTestBase; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +/** Test for {@link KvSnapshotLeaseMetadataJsonSerde}. */ +public class KvSnapshotLeaseMetadataJsonSerdeTest + extends JsonSerdeTestBase { + + KvSnapshotLeaseMetadataJsonSerdeTest() { + super(KvSnapshotLeaseMetadataJsonSerde.INSTANCE); + } + + @Override + protected KvSnapshotLeaseMetadata[] createObjects() { + KvSnapshotLeaseMetadata[] kvSnapshotLeaseMetadata = new KvSnapshotLeaseMetadata[2]; + Map tableIdToRemoteMetadataFilePath = new HashMap<>(); + tableIdToRemoteMetadataFilePath.put(1L, new FsPath("/path/to/metadata1")); + tableIdToRemoteMetadataFilePath.put(2L, new FsPath("/path/to/metadata2")); + kvSnapshotLeaseMetadata[0] = + new KvSnapshotLeaseMetadata(1735538268L, tableIdToRemoteMetadataFilePath); + kvSnapshotLeaseMetadata[1] = + new KvSnapshotLeaseMetadata(1735538268L, Collections.emptyMap()); + return kvSnapshotLeaseMetadata; + } + + @Override + protected String[] expectedJsons() { + return new String[] { + "{\"version\":1,\"expiration_time\":1735538268,\"tables\":" + + "[{\"table_id\":1,\"lease_metadata_path\":\"/path/to/metadata1\"}," + + "{\"table_id\":2,\"lease_metadata_path\":\"/path/to/metadata2\"}]}", + "{\"version\":1,\"expiration_time\":1735538268,\"tables\":[]}" + }; + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataManagerTest.java new file mode 100644 index 0000000000..1e1ad7fb23 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataManagerTest.java @@ -0,0 +1,146 @@ +/* + * 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.lease; + +import org.apache.fluss.fs.FSDataInputStream; +import org.apache.fluss.fs.FsPath; +import org.apache.fluss.server.zk.NOPErrorHandler; +import org.apache.fluss.server.zk.ZooKeeperClient; +import org.apache.fluss.server.zk.ZooKeeperExtension; +import org.apache.fluss.testutils.common.AllCallbackWrapper; +import org.apache.fluss.utils.IOUtils; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +import java.io.ByteArrayOutputStream; +import java.nio.file.Path; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link KvSnapshotLeaseMetadataManager}. */ +public class KvSnapshotLeaseMetadataManagerTest { + + @RegisterExtension + public static final AllCallbackWrapper ZOO_KEEPER_EXTENSION_WRAPPER = + new AllCallbackWrapper<>(new ZooKeeperExtension()); + + protected static ZooKeeperClient zookeeperClient; + private @TempDir Path tempDir; + private KvSnapshotLeaseMetadataManager metadataManager; + + @BeforeAll + static void beforeAll() { + zookeeperClient = + ZOO_KEEPER_EXTENSION_WRAPPER + .getCustomExtension() + .getZooKeeperClient(NOPErrorHandler.INSTANCE); + } + + @BeforeEach + void beforeEach() { + metadataManager = new KvSnapshotLeaseMetadataManager(zookeeperClient, tempDir.toString()); + } + + @AfterEach + void afterEach() { + ZOO_KEEPER_EXTENSION_WRAPPER.getCustomExtension().cleanupRoot(); + } + + @Test + void testGetLeasesList() throws Exception { + List leasesList = metadataManager.getLeasesList(); + assertThat(leasesList).isEmpty(); + + metadataManager.registerLease("leaseId1", new KvSnapshotLease(1000L)); + + Map tableIdToTableLease = new HashMap<>(); + tableIdToTableLease.put(1L, new KvSnapshotTableLease(1L, new Long[] {100L, -1L})); + metadataManager.registerLease("leaseId2", new KvSnapshotLease(2000L, tableIdToTableLease)); + leasesList = metadataManager.getLeasesList(); + assertThat(leasesList).containsExactlyInAnyOrder("leaseId1", "leaseId2"); + } + + @Test + void testRegisterAndUpdateLease() throws Exception { + Map tableIdToTableLease = new HashMap<>(); + tableIdToTableLease.put(1L, new KvSnapshotTableLease(1L, new Long[] {100L, -1L})); + + Map partitionSnapshots = new HashMap<>(); + partitionSnapshots.put(1000L, new Long[] {111L, 122L}); + partitionSnapshots.put(1001L, new Long[] {122L, -1L}); + tableIdToTableLease.put(2L, new KvSnapshotTableLease(2L, partitionSnapshots)); + + KvSnapshotLease expectedLease = new KvSnapshotLease(1000L, tableIdToTableLease); + metadataManager.registerLease("leaseId1", expectedLease); + + Optional lease = metadataManager.getLease("leaseId1"); + assertThat(lease).hasValue(expectedLease); + // assert zk and remote fs. + assertRemoteFsAndZkEquals("leaseId1", expectedLease); + + // test update lease. + tableIdToTableLease.remove(1L); + expectedLease = new KvSnapshotLease(2000L, tableIdToTableLease); + metadataManager.updateLease("leaseId1", expectedLease); + lease = metadataManager.getLease("leaseId1"); + assertThat(lease).hasValue(expectedLease); + // assert zk and remote fs. + assertRemoteFsAndZkEquals("leaseId1", expectedLease); + + // test delete lease. + metadataManager.deleteLease("leaseId1"); + lease = metadataManager.getLease("leaseId1"); + assertThat(lease).isEmpty(); + } + + private void assertRemoteFsAndZkEquals(String leaseId, KvSnapshotLease expectedLease) + throws Exception { + Optional leaseMetadataOpt = + zookeeperClient.getKvSnapshotLeaseMetadata(leaseId); + assertThat(leaseMetadataOpt).isPresent(); + KvSnapshotLeaseMetadata leaseMetadata = leaseMetadataOpt.get(); + assertThat(leaseMetadata.getExpirationTime()).isEqualTo(expectedLease.getExpirationTime()); + Map actualFsPathSet = leaseMetadata.getTableIdToRemoteMetadataFilePath(); + Map expectedTableLeases = + expectedLease.getTableIdToTableLease(); + assertThat(actualFsPathSet).hasSize(expectedTableLeases.size()); + for (Map.Entry actualEntry : actualFsPathSet.entrySet()) { + long tableId = actualEntry.getKey(); + FsPath actualMetadataPath = actualEntry.getValue(); + assertThat(actualMetadataPath).isNotNull(); + KvSnapshotTableLease expectedTableLease = expectedTableLeases.get(tableId); + assertThat(expectedTableLease).isNotNull(); + FSDataInputStream inputStream = + actualMetadataPath.getFileSystem().open(actualMetadataPath); + try (ByteArrayOutputStream outputStream = new ByteArrayOutputStream()) { + IOUtils.copyBytes(inputStream, outputStream, true); + assertThat(KvSnapshotTableLeaseJsonSerde.fromJson(outputStream.toByteArray())) + .isEqualTo(expectedTableLease); + } + } + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseTest.java new file mode 100644 index 0000000000..1dcf26054b --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseTest.java @@ -0,0 +1,207 @@ +/* + * 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.lease; + +import org.apache.fluss.metadata.TableBucket; + +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.Map; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Test for {@link KvSnapshotLease}. */ +public class KvSnapshotLeaseTest { + + private static final int NUM_BUCKET = 2; + + @Test + void testConstructorAndGetters() { + long expirationTime = 1000L; + KvSnapshotLease kvSnapshotLease = new KvSnapshotLease(expirationTime); + + assertThat(kvSnapshotLease.getExpirationTime()).isEqualTo(expirationTime); + assertThat(kvSnapshotLease.getTableIdToTableLease()).isEmpty(); + assertThat(kvSnapshotLease.getLeasedSnapshotCount()).isEqualTo(0); + } + + @Test + void testRegisterBucketForNonPartitionedTable() { + KvSnapshotLease lease = new KvSnapshotLease(1000L); + long tableId = 1L; + int bucketId = 0; + + long originalSnapshot = acquireBucket(lease, new TableBucket(tableId, bucketId), 123L); + + assertThat(originalSnapshot).isEqualTo(-1L); + assertThat(lease.getTableIdToTableLease()).containsKey(tableId); + KvSnapshotTableLease tableLease = lease.getTableIdToTableLease().get(tableId); + Long[] bucketSnapshots = tableLease.getBucketSnapshots(); + assertThat(bucketSnapshots).isNotNull(); + assertThat(bucketSnapshots).hasSize(NUM_BUCKET); + assertThat(bucketSnapshots[bucketId]).isEqualTo(123L); + assertThat(bucketSnapshots[1]).isEqualTo(-1L); + + // Register again same bucket → should be update + originalSnapshot = acquireBucket(lease, new TableBucket(tableId, bucketId), 456L); + assertThat(originalSnapshot).isEqualTo(123L); + tableLease = lease.getTableIdToTableLease().get(tableId); + bucketSnapshots = tableLease.getBucketSnapshots(); + assertThat(bucketSnapshots).isNotNull(); + assertThat(bucketSnapshots[bucketId]).isEqualTo(456L); + } + + @Test + void testIllegalBucketNum() { + // Currently, for the same table, the bucket num should be the same. + KvSnapshotLease lease = new KvSnapshotLease(1000L); + long tableId = 1L; + int bucketId = 0; + + lease.acquireBucket(new TableBucket(tableId, bucketId), 123L, 10); + assertThatThrownBy(() -> lease.acquireBucket(new TableBucket(tableId, bucketId), 456L, 20)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Bucket index is null, or input bucket number is not equal to the bucket " + + "number of the table."); + } + + @Test + void testRegisterBucketForPartitionedTable() { + KvSnapshotLease lease = new KvSnapshotLease(1000L); + long tableId = 1L; + + long originalSnapshot = acquireBucket(lease, new TableBucket(tableId, 1000L, 0), 111L); + assertThat(originalSnapshot).isEqualTo(-1L); + originalSnapshot = acquireBucket(lease, new TableBucket(tableId, 1000L, 1), 122L); + assertThat(originalSnapshot).isEqualTo(-1L); + originalSnapshot = acquireBucket(lease, new TableBucket(tableId, 1001L, 0), 122L); + assertThat(originalSnapshot).isEqualTo(-1L); + + Map tableIdToTableLease = lease.getTableIdToTableLease(); + assertThat(tableIdToTableLease).containsKey(tableId); + KvSnapshotTableLease tableLease = tableIdToTableLease.get(tableId); + Map partitionSnapshots = tableLease.getPartitionSnapshots(); + assertThat(partitionSnapshots).containsKeys(1000L, 1001L); + assertThat(partitionSnapshots.get(1000L)[0]).isEqualTo(111L); + assertThat(partitionSnapshots.get(1000L)[1]).isEqualTo(122L); + assertThat(partitionSnapshots.get(1001L)[0]).isEqualTo(122L); + assertThat(partitionSnapshots.get(1001L)[1]).isEqualTo(-1L); + + // test update. + originalSnapshot = acquireBucket(lease, new TableBucket(tableId, 1000L, 0), 222L); + assertThat(originalSnapshot).isEqualTo(111L); + assertThat(partitionSnapshots.get(1000L)[0]).isEqualTo(222L); + } + + @Test + void testReleaseBucket() { + KvSnapshotLease lease = new KvSnapshotLease(1000L); + long tableId = 1L; + + // Register + acquireBucket(lease, new TableBucket(tableId, 0), 123L); + assertThat(lease.getLeasedSnapshotCount()).isEqualTo(1); + + // Unregister + long snapshotId = releaseBucket(lease, new TableBucket(tableId, 0)); + assertThat(snapshotId).isEqualTo(123L); + assertThat(lease.getLeasedSnapshotCount()).isEqualTo(0); + assertThat(lease.isEmpty()).isTrue(); + } + + @Test + void testGetLeasedSnapshotCount() { + KvSnapshotLease lease = new KvSnapshotLease(1000L); + + // Non-partitioned + acquireBucket(lease, new TableBucket(1L, 0), 100L); + acquireBucket(lease, new TableBucket(1L, 1), 101L); + + // Partitioned + acquireBucket(lease, new TableBucket(2L, 20L, 0), 200L); + acquireBucket(lease, new TableBucket(2L, 21L, 1), 201L); + + assertThat(lease.getLeasedSnapshotCount()).isEqualTo(4); + + // Unregister one + releaseBucket(lease, new TableBucket(1L, 0)); + assertThat(lease.getLeasedSnapshotCount()).isEqualTo(3); + } + + @Test + void testEqualsAndHashCode() { + KvSnapshotLease lease = new KvSnapshotLease(1000L); + assertThat(lease).isEqualTo(lease); + assertThat(lease.hashCode()).isEqualTo(lease.hashCode()); + + KvSnapshotLease c1 = new KvSnapshotLease(1000L); + KvSnapshotLease c2 = new KvSnapshotLease(2000L); + assertThat(c1).isNotEqualTo(c2); + + // Create two leases with same logical content but different array objects + Map map1 = new HashMap<>(); + Map partitionSnapshots1 = new HashMap<>(); + partitionSnapshots1.put(2001L, new Long[] {100L, -1L}); + partitionSnapshots1.put(2002L, new Long[] {-1L, 101L}); + map1.put(1L, new KvSnapshotTableLease(1L, new Long[] {100L, -1L}, partitionSnapshots1)); + Map map2 = new HashMap<>(); + Map partitionSnapshots2 = new HashMap<>(); + partitionSnapshots2.put(2001L, new Long[] {100L, -1L}); + partitionSnapshots2.put(2002L, new Long[] {-1L, 101L}); + map2.put(1L, new KvSnapshotTableLease(1L, new Long[] {100L, -1L}, partitionSnapshots2)); + c1 = new KvSnapshotLease(1000L, map1); + c2 = new KvSnapshotLease(1000L, map2); + assertThat(c1).isEqualTo(c2); + assertThat(c1.hashCode()).isEqualTo(c2.hashCode()); + + // different array content. + map1 = new HashMap<>(); + map1.put(1L, new KvSnapshotTableLease(1L, new Long[] {100L, -1L})); + map2 = new HashMap<>(); + map2.put(1L, new KvSnapshotTableLease(1L, new Long[] {200L, -1L})); + c1 = new KvSnapshotLease(1000L, map1); + c2 = new KvSnapshotLease(1000L, map2); + assertThat(c1).isNotEqualTo(c2); + } + + @Test + void testToString() { + KvSnapshotLease lease = new KvSnapshotLease(1000L); + acquireBucket(lease, new TableBucket(1L, 0), 100L); + acquireBucket(lease, new TableBucket(1L, 1), 101L); + acquireBucket(lease, new TableBucket(2L, 0L, 0), 200L); + acquireBucket(lease, new TableBucket(2L, 1L, 1), 201L); + assertThat(lease.toString()) + .isEqualTo( + "KvSnapshotLease{expirationTime=1000, tableIdToTableLease={" + + "1=KvSnapshotTableLease{tableId=1, bucketSnapshots=[100, 101], partitionSnapshots={}}, " + + "2=KvSnapshotTableLease{tableId=2, bucketSnapshots=null, partitionSnapshots={" + + "0=[200, -1], 1=[-1, 201]}}}}"); + } + + private long acquireBucket(KvSnapshotLease lease, TableBucket tb, long kvSnapshotId) { + return lease.acquireBucket(tb, kvSnapshotId, NUM_BUCKET); + } + + private long releaseBucket(KvSnapshotLease lease, TableBucket tb) { + return lease.releaseBucket(tb); + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLeaseJsonSerdeTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLeaseJsonSerdeTest.java new file mode 100644 index 0000000000..de77bb7779 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLeaseJsonSerdeTest.java @@ -0,0 +1,54 @@ +/* + * 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.lease; + +import org.apache.fluss.utils.json.JsonSerdeTestBase; + +import java.util.HashMap; +import java.util.Map; + +/** Test for {@link KvSnapshotTableLeaseJsonSerde}. */ +public class KvSnapshotTableLeaseJsonSerdeTest extends JsonSerdeTestBase { + + KvSnapshotTableLeaseJsonSerdeTest() { + super(KvSnapshotTableLeaseJsonSerde.INSTANCE); + } + + @Override + protected KvSnapshotTableLease[] createObjects() { + KvSnapshotTableLease[] kvSnapshotTableLeases = new KvSnapshotTableLease[2]; + kvSnapshotTableLeases[0] = new KvSnapshotTableLease(1L, new Long[] {1L, -1L, 1L, 2L}); + + Map partitionSnapshots = new HashMap<>(); + partitionSnapshots.put(2001L, new Long[] {10L, -1L, 20L, 30L}); + partitionSnapshots.put(2002L, new Long[] {15L, -1L, 25L, 35L}); + kvSnapshotTableLeases[1] = new KvSnapshotTableLease(2L, partitionSnapshots); + + return kvSnapshotTableLeases; + } + + @Override + protected String[] expectedJsons() { + return new String[] { + "{\"version\":1,\"table_id\":1,\"bucket_snapshots\":[1,-1,1,2]}", + "{\"version\":1,\"table_id\":2,\"partition_snapshots\":[" + + "{\"partition_id\":2001,\"bucket_snapshots\":[10,-1,20,30]}," + + "{\"partition_id\":2002,\"bucket_snapshots\":[15,-1,25,35]}]}" + }; + } +} diff --git a/fluss-test-coverage/pom.xml b/fluss-test-coverage/pom.xml index d254372251..91e355410d 100644 --- a/fluss-test-coverage/pom.xml +++ b/fluss-test-coverage/pom.xml @@ -407,6 +407,9 @@ org.apache.fluss.flink.metrics.* + + org.apache.fluss.flink.procedure.* + org.apache.flink.streaming.api.functions.sink.v2.* diff --git a/website/docs/engine-flink/options.md b/website/docs/engine-flink/options.md index 5f18a4b795..d70c71f416 100644 --- a/website/docs/engine-flink/options.md +++ b/website/docs/engine-flink/options.md @@ -93,20 +93,22 @@ See more details about [ALTER TABLE ... SET](engine-flink/ddl.md#set-properties) ## Read Options -| Option | Type | Default | Description | -|-----------------------------------------------------|------------|-------------------------------------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| scan.startup.mode | Enum | full | The scan startup mode enables you to specify the starting point for data consumption. Fluss currently supports the following `scan.startup.mode` options: `full` (default), earliest, latest, timestamp. See the [Start Reading Position](engine-flink/reads.md#start-reading-position) for more details. | -| scan.startup.timestamp | Long | (None) | The timestamp to start reading the data from. This option is only valid when `scan.startup.mode` is set to `timestamp`. The format is 'milli-second-since-epoch' or `yyyy-MM-dd HH:mm:ss`, like `1678883047356` or `2023-12-09 23:09:12`. | -| scan.partition.discovery.interval | Duration | 1min | The time interval for the Fluss source to discover the new partitions for partitioned table while scanning. A non-positive value disables the partition discovery. The default value is 1 minute. Currently, since Fluss Admin#listPartitions(TablePath tablePath) requires a large number of requests to ZooKeeper in server, this option cannot be set too small, as a small value would cause frequent requests and increase server load. In the future, once list partitions is optimized, the default value of this parameter can be reduced. | -| client.scanner.log.check-crc | Boolean | true | Automatically check the CRC3 of the read records for LogScanner. This ensures no on-the-wire or on-disk corruption to the messages occurred. This check adds some overhead, so it may be disabled in cases seeking extreme performance. | -| client.scanner.log.max-poll-records | Integer | 500 | The maximum number of records returned in a single call to poll() for LogScanner. Note that this config doesn't impact the underlying fetching behavior. The Scanner will cache the records from each fetch request and returns them incrementally from each poll. | -| client.scanner.log.fetch.max-bytes | MemorySize | 16mb | The maximum amount of data the server should return for a fetch request from client. Records are fetched in batches, and if the first record batch in the first non-empty bucket of the fetch is larger than this value, the record batch will still be returned to ensure that the fetch can make progress. As such, this is not a absolute maximum. | -| client.scanner.log.fetch.max-bytes-for-bucket | MemorySize | 1mb | The maximum amount of data the server should return for a table bucket in fetch request fom client. Records are fetched in batches, and the max bytes size is config by this option. | -| client.scanner.log.fetch.min-bytes | MemorySize | 1b | The minimum bytes expected for each fetch log request from client to response. If not enough bytes, wait up to client.scanner.log.fetch-wait-max-time time to return. | -| client.scanner.log.fetch.wait-max-time | Duration | 500ms | The maximum time to wait for enough bytes to be available for a fetch log request from client to response. | -| client.scanner.io.tmpdir | String | System.getProperty("java.io.tmpdir") + "/fluss" | Local directory that is used by client for storing the data files (like kv snapshot, log segment files) to read temporarily | -| client.scanner.remote-log.prefetch-num | Integer | 4 | The number of remote log segments to keep in local temp file for LogScanner, which download from remote storage. The default setting is 4. | -| client.remote-file.download-thread-num | Integer | 3 | The number of threads the client uses to download remote files. | +| Option | Type | Default | Description | +|-----------------------------------------------|------------|-------------------------------------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| scan.startup.mode | Enum | full | The scan startup mode enables you to specify the starting point for data consumption. Fluss currently supports the following `scan.startup.mode` options: `full` (default), earliest, latest, timestamp. See the [Start Reading Position](engine-flink/reads.md#start-reading-position) for more details. | +| scan.startup.timestamp | Long | (None) | The timestamp to start reading the data from. This option is only valid when `scan.startup.mode` is set to `timestamp`. The format is 'milli-second-since-epoch' or `yyyy-MM-dd HH:mm:ss`, like `1678883047356` or `2023-12-09 23:09:12`. | +| scan.partition.discovery.interval | Duration | 1min | The time interval for the Fluss source to discover the new partitions for partitioned table while scanning. A non-positive value disables the partition discovery. The default value is 1 minute. Currently, since Fluss Admin#listPartitions(TablePath tablePath) requires a large number of requests to ZooKeeper in server, this option cannot be set too small, as a small value would cause frequent requests and increase server load. In the future, once list partitions is optimized, the default value of this parameter can be reduced. | +| scan.kv.snapshot.lease.id | String | UUID | The lease id to lease kv snapshots. If set, the acquired kv snapshots will not be deleted until the consumer finished consuming all the snapshots or the lease duration time is reached. If not set, an UUID will be set. | +| scan.kv.snapshot.lease.duration | Duration | 1day | The time period how long to wait before expiring the kv snapshot lease to avoid kv snapshot blocking to delete. | +| client.scanner.log.check-crc | Boolean | true | Automatically check the CRC3 of the read records for LogScanner. This ensures no on-the-wire or on-disk corruption to the messages occurred. This check adds some overhead, so it may be disabled in cases seeking extreme performance. | +| client.scanner.log.max-poll-records | Integer | 500 | The maximum number of records returned in a single call to poll() for LogScanner. Note that this config doesn't impact the underlying fetching behavior. The Scanner will cache the records from each fetch request and returns them incrementally from each poll. | +| client.scanner.log.fetch.max-bytes | MemorySize | 16mb | The maximum amount of data the server should return for a fetch request from client. Records are fetched in batches, and if the first record batch in the first non-empty bucket of the fetch is larger than this value, the record batch will still be returned to ensure that the fetch can make progress. As such, this is not a absolute maximum. | +| client.scanner.log.fetch.max-bytes-for-bucket | MemorySize | 1mb | The maximum amount of data the server should return for a table bucket in fetch request fom client. Records are fetched in batches, and the max bytes size is config by this option. | +| client.scanner.log.fetch.min-bytes | MemorySize | 1b | The minimum bytes expected for each fetch log request from client to response. If not enough bytes, wait up to client.scanner.log.fetch-wait-max-time time to return. | +| client.scanner.log.fetch.wait-max-time | Duration | 500ms | The maximum time to wait for enough bytes to be available for a fetch log request from client to response. | +| client.scanner.io.tmpdir | String | System.getProperty("java.io.tmpdir") + "/fluss" | Local directory that is used by client for storing the data files (like kv snapshot, log segment files) to read temporarily | +| client.scanner.remote-log.prefetch-num | Integer | 4 | The number of remote log segments to keep in local temp file for LogScanner, which download from remote storage. The default setting is 4. | +| client.remote-file.download-thread-num | Integer | 3 | The number of threads the client uses to download remote files. | ## Lookup Options diff --git a/website/docs/engine-flink/procedures.md b/website/docs/engine-flink/procedures.md index 182ef033e3..63d64f4866 100644 --- a/website/docs/engine-flink/procedures.md +++ b/website/docs/engine-flink/procedures.md @@ -487,4 +487,38 @@ CALL sys.cancel_rebalance(); -- Cancel a specific rebalance operation by ID CALL sys.cancel_rebalance('rebalance-12345'); +``` + +## kv snapshot lease + +Fluss provides procedures to manage KV snapshot leases, allowing you to release leased kv snapshots. + +### release_all_kv_snapshot_lease + +Release all KV snapshots leased under a specified leaseId. This is typically used for handle the scenario of lease +remnants. After a normal job completion, the registered lease is not released, requiring a manual trigger of the +procedure for cleanup. + +**Syntax:** + +```sql +CALL [catalog_name.]sys.release_all_kv_snapshot_lease( + leaseId => 'STRING' +) +``` + +**Parameters:** + +- `leaseId` (required): The lease identifier of the KV snapshots to release. This should match the lease ID used when acquiring the KV snapshots. + +**Returns:** An array with a single element `'success'` if the operation completes successfully. + +**Example:** + +```sql title="Flink SQL" +-- Use the Fluss catalog (replace 'fluss_catalog' with your catalog name if different) +USE fluss_catalog; + +-- Release all KV snapshots leased under the given leaseId +CALL sys.release_all_kv_snapshot_lease('test-lease-id'); ``` \ No newline at end of file diff --git a/website/docs/maintenance/configuration.md b/website/docs/maintenance/configuration.md index 57f7017c86..80d40fddbc 100644 --- a/website/docs/maintenance/configuration.md +++ b/website/docs/maintenance/configuration.md @@ -139,6 +139,7 @@ during the Fluss cluster working. | kv.snapshot.scheduler-thread-num | Integer | 1 | The number of threads that the server uses to schedule snapshot kv data for all the replicas in the server. | | kv.snapshot.transfer-thread-num | Integer | 4 | **Deprecated**: This option is deprecated. Please use `server.io-pool.size` instead. The number of threads the server uses to transfer (download and upload) kv snapshot files. | | kv.snapshot.num-retained | Integer | 1 | The maximum number of completed snapshots to retain. | +| kv.snapshot.lease-expiration-check-interval | Duration | 10min | The interval to check the expiration of kv snapshot leases. The default setting is 10 minutes. | | kv.rocksdb.thread.num | Integer | 2 | The maximum number of concurrent background flush and compaction jobs (per bucket of table). The default value is `2`. | | kv.rocksdb.files.open | Integer | -1 | The maximum number of open files (per bucket of table) that can be used by the DB, `-1` means no limit. The default value is `-1`. | | kv.rocksdb.log.max-file-size | MemorySize | 25mb | The maximum size of RocksDB's file used for information logging. If the log files becomes larger than this, a new file will be created. If 0, all logs will be written to one log file. The default maximum file size is `25MB`. | @@ -163,7 +164,7 @@ during the Fluss cluster working. | kv.rocksdb.use-bloom-filter | Boolean | true | If true, every newly created SST file will contain a Bloom filter. It is enabled by default. | | kv.rocksdb.bloom-filter.bits-per-key | Double | 10.0 | Bits per key that bloom filter will use, this only take effect when bloom filter is used. The default value is 10.0. | | kv.rocksdb.bloom-filter.block-based-mode | Boolean | false | If true, RocksDB will use block-based filter instead of full filter, this only take effect when bloom filter is used. The default value is `false`. | -| kv.rocksdb.shared-rate-limiter-bytes-per-sec | MemorySize | Long.MAX_VALUE | The bytes per second rate limit for RocksDB flush and compaction operations shared across all RocksDB instances on the TabletServer. The rate limiter is always enabled. The default value is Long.MAX_VALUE (effectively unlimited). Set to a lower value (e.g., 100MB) to limit the rate. This configuration can be updated dynamically without server restart. See [Updating Configs](operations/updating-configs.md) for more details. | +| kv.rocksdb.shared-rate-limiter-bytes-per-sec | MemorySize | Long.MAX_VALUE | The bytes per second rate limit for RocksDB flush and compaction operations shared across all RocksDB instances on the TabletServer. The rate limiter is always enabled. The default value is Long.MAX_VALUE (effectively unlimited). Set to a lower value (e.g., 100MB) to limit the rate. This configuration can be updated dynamically without server restart. See [Updating Configs](operations/updating-configs.md) for more details. | | kv.recover.log-record-batch.max-size | MemorySize | 16mb | The max fetch size for fetching log to apply to kv during recovering kv. | ## Metrics diff --git a/website/docs/maintenance/observability/monitor-metrics.md b/website/docs/maintenance/observability/monitor-metrics.md index 4b082b2f2e..bf62746b47 100644 --- a/website/docs/maintenance/observability/monitor-metrics.md +++ b/website/docs/maintenance/observability/monitor-metrics.md @@ -294,7 +294,7 @@ Some metrics might not be exposed when using other JVM implementations (e.g. IBM - coordinator + coordinator - activeCoordinatorCount The number of active CoordinatorServer in this cluster. @@ -325,6 +325,16 @@ Some metrics might not be exposed when using other JVM implementations (e.g. IBM The total number of partitions in this cluster. Gauge + + kvSnapshotLeaseCount + The total number of kv snapshot leases in this cluster. + Gauge + + + leasedKvSnapshotCount + The total number of leased kv snapshots in this cluster. + Gauge + replicasToDeleteCount The total number of replicas in the progress to be deleted in this cluster. From e4e454de5db7ab3b5521b36e61904f3ae5ad08f5 Mon Sep 17 00:00:00 2001 From: yunhong <337361684@qq.com> Date: Sun, 8 Feb 2026 15:48:00 +0800 Subject: [PATCH 2/6] address jark's comments --- .../org/apache/fluss/client/admin/Admin.java | 54 +-- .../apache/fluss/client/admin/FlussAdmin.java | 32 +- .../fluss/client/admin/KvSnapshotLease.java | 73 ++++ .../client/admin/KvSnapshotLeaseImpl.java | 90 +++++ .../client/utils/ClientRpcMessageUtils.java | 27 +- .../admin/CustomFlussClusterITCase.java | 125 ++++++ .../acl/FlussAuthorizationITCase.java | 115 ++++++ .../batch/KvSnapshotBatchScannerITCase.java | 53 ++- .../KvSnapshotLeaseNotExistException.java | 31 -- ...orBucket.java => TableBucketSnapshot.java} | 26 +- .../fluss/flink/FlinkConnectorOptions.java | 7 +- .../flink/catalog/FlinkTableFactory.java | 10 +- ...java => DropKvSnapshotLeaseProcedure.java} | 16 +- .../flink/procedure/ProcedureManager.java | 3 +- .../fluss/flink/source/FlinkSource.java | 4 +- .../fluss/flink/source/FlussSource.java | 5 +- .../flink/source/FlussSourceBuilder.java | 18 +- .../enumerator/FlinkSourceEnumerator.java | 52 +-- .../source/reader/FlinkSourceReader.java | 2 + .../flink/source/reader/LeaseContext.java | 26 +- .../FlussSourceEnumeratorStateSerializer.java | 38 +- .../source/state/SourceEnumeratorState.java | 17 +- .../flink/procedure/FlinkProcedureITCase.java | 6 +- .../source/FlinkTableSourceBatchITCase.java | 52 +++ .../flink/source/FlinkTableSourceITCase.java | 9 +- .../fluss/flink/source/FlussSourceITCase.java | 7 - .../enumerator/FlinkSourceEnumeratorTest.java | 4 +- .../SourceEnumeratorStateSerializerTest.java | 25 +- .../fluss/rpc/gateway/AdminGateway.java | 6 + .../apache/fluss/rpc/protocol/ApiKeys.java | 3 +- .../org/apache/fluss/rpc/protocol/Errors.java | 5 +- fluss-rpc/src/main/proto/FlussApi.proto | 20 +- .../CoordinatorEventProcessor.java | 90 +---- .../server/coordinator/CoordinatorServer.java | 21 +- .../coordinator/CoordinatorService.java | 112 ++++-- .../event/AcquireKvSnapshotLeaseEvent.java | 60 --- .../event/ReleaseKvSnapshotLeaseEvent.java | 53 --- .../lease/KvSnapshotLeaseHandler.java} | 65 +++- .../{ => lease}/KvSnapshotLeaseManager.java | 298 +++++++-------- .../lease/KvSnapshotLeaseMetadataManager.java | 173 ++++++--- .../kv/snapshot/CompletedSnapshotStore.java | 6 +- .../server/utils/ServerRpcMessageUtils.java | 67 ++-- .../data/lease/KvSnapshotLeaseMetadata.java | 23 -- .../zk/data/lease/KvSnapshotTableLease.java | 18 +- .../lease/KvSnapshotTableLeaseJsonSerde.java | 5 +- .../CoordinatorEventProcessorTest.java | 16 +- .../coordinator/TestCoordinatorGateway.java | 8 + .../lease/KvSnapshotLeaseHandlerTest.java} | 162 ++++++-- .../KvSnapshotLeaseManagerTest.java | 356 ++++++++++-------- .../KvSnapshotLeaseMetadataManagerTest.java | 281 ++++++++++++++ .../rebalance/RebalanceManagerTest.java | 15 +- .../TableBucketStateMachineTest.java | 16 +- .../fluss/server/zk/ZooKeeperClientTest.java | 10 - .../KvSnapshotLeaseMetadataManagerTest.java | 146 ------- .../KvSnapshotTableLeaseJsonSerdeTest.java | 6 +- website/docs/_configs/_partial_config.mdx | 16 +- website/docs/engine-flink/options.md | 2 +- website/docs/engine-flink/procedures.md | 16 +- 58 files changed, 1791 insertions(+), 1211 deletions(-) create mode 100644 fluss-client/src/main/java/org/apache/fluss/client/admin/KvSnapshotLease.java create mode 100644 fluss-client/src/main/java/org/apache/fluss/client/admin/KvSnapshotLeaseImpl.java delete mode 100644 fluss-common/src/main/java/org/apache/fluss/exception/KvSnapshotLeaseNotExistException.java rename fluss-common/src/main/java/org/apache/fluss/metadata/{KvSnapshotLeaseForBucket.java => TableBucketSnapshot.java} (68%) rename fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/{ReleaseAllKvSnapshotLeaseProcedure.java => DropKvSnapshotLeaseProcedure.java} (74%) delete mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/AcquireKvSnapshotLeaseEvent.java delete mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/ReleaseKvSnapshotLeaseEvent.java rename fluss-server/src/main/java/org/apache/fluss/server/{zk/data/lease/KvSnapshotLease.java => coordinator/lease/KvSnapshotLeaseHandler.java} (70%) rename fluss-server/src/main/java/org/apache/fluss/server/coordinator/{ => lease}/KvSnapshotLeaseManager.java (56%) rename fluss-server/src/main/java/org/apache/fluss/server/{zk/data => coordinator}/lease/KvSnapshotLeaseMetadataManager.java (51%) rename fluss-server/src/test/java/org/apache/fluss/server/{zk/data/lease/KvSnapshotLeaseTest.java => coordinator/lease/KvSnapshotLeaseHandlerTest.java} (50%) rename fluss-server/src/test/java/org/apache/fluss/server/coordinator/{ => lease}/KvSnapshotLeaseManagerTest.java (52%) create mode 100644 fluss-server/src/test/java/org/apache/fluss/server/coordinator/lease/KvSnapshotLeaseMetadataManagerTest.java delete mode 100644 fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataManagerTest.java diff --git a/fluss-client/src/main/java/org/apache/fluss/client/admin/Admin.java b/fluss-client/src/main/java/org/apache/fluss/client/admin/Admin.java index 12d9a9eeb2..f2e75ae3e4 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/admin/Admin.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/admin/Admin.java @@ -18,7 +18,6 @@ package org.apache.fluss.client.admin; import org.apache.fluss.annotation.PublicEvolving; -import org.apache.fluss.client.metadata.AcquireKvSnapshotLeaseResult; import org.apache.fluss.client.metadata.KvSnapshotMetadata; import org.apache.fluss.client.metadata.KvSnapshots; import org.apache.fluss.client.metadata.LakeSnapshot; @@ -75,7 +74,6 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.Set; import java.util.concurrent.CompletableFuture; /** @@ -431,55 +429,13 @@ CompletableFuture getKvSnapshotMetadata( TableBucket bucket, long snapshotId); /** - * Acquires a lease for specific KV snapshots of the given tableBuckets asynchronously. + * Creates a new KV snapshot lease with the given ID and duration. * - *

Once acquired, the specified KV snapshots will be protected from garbage collection for - * the duration of the {@code leaseDuration}. The client must call {@link - * #releaseKvSnapshotLease} to release the lock early when reading is finished. - * - *

If the lease expires (no renew received within duration), the server is free to delete the - * snapshot files. - * - *

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

    - *
  • {@link TableNotExistException} if the table does not exist. - *
  • {@link PartitionNotExistException} if the partition does not exist. - *
- * - * @param leaseId The unique ID for this lease session (usually a UUID generated by client). - * @param snapshotIds The snapshots to lease, a map from TableBucket to kvSnapshotId. - * @param leaseDuration The duration (in milliseconds) for which the snapshots should be kept. - * @return The result of the acquire operation, containing any buckets that failed to be locked. - */ - CompletableFuture acquireKvSnapshotLease( - String leaseId, Map snapshotIds, long leaseDuration); - - /** - * Releases the lease for specific tableBuckets asynchronously. - * - *

This is typically called when a client finishes reading a specific bucket (or a batch of - * buckets) but is still reading others under the same leaseId. - * - *

If {@code bucketsToRelease} contains all buckets under this leaseId, the lease itself will - * be removed. - * - * @param leaseId The lease id. - * @param bucketsToRelease The specific tableBuckets to release. - */ - CompletableFuture releaseKvSnapshotLease( - String leaseId, Set bucketsToRelease); - - /** - * Releases the entire lease asynchronously for all leased snapshots for all table buckets of - * specified leaseId. - * - *

All snapshots locked under this {@code leaseId} will be released immediately. This is - * equivalent to calling {@link #releaseKvSnapshotLease} with all held buckets. - * - * @param leaseId The lease id to release. + * @param leaseId the unique identifier for the lease + * @param leaseDurationMs the lease duration in milliseconds + * @return a {@link KvSnapshotLease} instance representing the created lease */ - CompletableFuture releaseAllKvSnapshotLease(String leaseId); + KvSnapshotLease createKvSnapshotLease(String leaseId, long leaseDurationMs); /** * Get table lake snapshot info of the given table asynchronously. diff --git a/fluss-client/src/main/java/org/apache/fluss/client/admin/FlussAdmin.java b/fluss-client/src/main/java/org/apache/fluss/client/admin/FlussAdmin.java index 79035f14f9..a905823c4a 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/admin/FlussAdmin.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/admin/FlussAdmin.java @@ -18,7 +18,6 @@ package org.apache.fluss.client.admin; import org.apache.fluss.annotation.VisibleForTesting; -import org.apache.fluss.client.metadata.AcquireKvSnapshotLeaseResult; import org.apache.fluss.client.metadata.KvSnapshotMetadata; import org.apache.fluss.client.metadata.KvSnapshots; import org.apache.fluss.client.metadata.LakeSnapshot; @@ -85,7 +84,6 @@ import org.apache.fluss.rpc.messages.PbTablePath; import org.apache.fluss.rpc.messages.RebalanceRequest; import org.apache.fluss.rpc.messages.RebalanceResponse; -import org.apache.fluss.rpc.messages.ReleaseKvSnapshotLeaseRequest; import org.apache.fluss.rpc.messages.RemoveServerTagRequest; import org.apache.fluss.rpc.messages.TableExistsRequest; import org.apache.fluss.rpc.messages.TableExistsResponse; @@ -103,17 +101,14 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.Set; import java.util.concurrent.CompletableFuture; -import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makeAcquireKvSnapshotLeaseRequest; import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makeAlterTableRequest; import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makeCreatePartitionRequest; import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makeDropPartitionRequest; import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makeListOffsetsRequest; import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makePbPartitionSpec; import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makeRegisterProducerOffsetsRequest; -import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makeReleaseKvSnapshotLeaseRequest; import static org.apache.fluss.client.utils.ClientRpcMessageUtils.toConfigEntries; import static org.apache.fluss.client.utils.MetadataUtils.sendMetadataRequestAndRebuildCluster; import static org.apache.fluss.rpc.util.CommonRpcMessageUtils.toAclBindings; @@ -406,31 +401,8 @@ public CompletableFuture getKvSnapshotMetadata( } @Override - public CompletableFuture acquireKvSnapshotLease( - String leaseId, Map snapshotIds, long leaseDuration) { - if (snapshotIds.isEmpty()) { - throw new IllegalArgumentException( - "The snapshotIds to acquire kv snapshot lease is empty"); - } - - return gateway.acquireKvSnapshotLease( - makeAcquireKvSnapshotLeaseRequest(leaseId, snapshotIds, leaseDuration)) - .thenApply(ClientRpcMessageUtils::toAcquireKvSnapshotLeaseResult); - } - - @Override - public CompletableFuture releaseKvSnapshotLease( - String leaseId, Set bucketsToRelease) { - return gateway.releaseKvSnapshotLease( - makeReleaseKvSnapshotLeaseRequest(leaseId, bucketsToRelease)) - .thenApply(r -> null); - } - - @Override - public CompletableFuture releaseAllKvSnapshotLease(String leaseId) { - ReleaseKvSnapshotLeaseRequest request = - new ReleaseKvSnapshotLeaseRequest().setLeaseId(leaseId); - return gateway.releaseKvSnapshotLease(request).thenApply(r -> null); + public KvSnapshotLease createKvSnapshotLease(String leaseId, long leaseDurationMs) { + return new KvSnapshotLeaseImpl(leaseId, leaseDurationMs, gateway); } @Override diff --git a/fluss-client/src/main/java/org/apache/fluss/client/admin/KvSnapshotLease.java b/fluss-client/src/main/java/org/apache/fluss/client/admin/KvSnapshotLease.java new file mode 100644 index 0000000000..e8a654bc0b --- /dev/null +++ b/fluss-client/src/main/java/org/apache/fluss/client/admin/KvSnapshotLease.java @@ -0,0 +1,73 @@ +/* + * 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.client.admin; + +import org.apache.fluss.annotation.PublicEvolving; +import org.apache.fluss.client.metadata.AcquireKvSnapshotLeaseResult; +import org.apache.fluss.metadata.TableBucket; + +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; + +/** + * Represents a lease for managing KV snapshots. A lease allows acquiring, renewing, releasing, and + * dropping snapshot references under a time-bound agreement. + * + * @since 0.9 + */ +@PublicEvolving +public interface KvSnapshotLease { + + /** Returns the unique identifier of this lease. */ + String leaseId(); + + /** Returns the lease duration in milliseconds. */ + long leaseDurationMs(); + + /** + * Acquires snapshots for the specified table buckets under this lease. + * + * @param snapshotIds mapping from table buckets to desired snapshot IDs + * @return a future containing the result of the acquisition + */ + CompletableFuture acquireSnapshots( + Map snapshotIds); + + /** + * Renews the lease, extending its validity period. + * + * @return a future that completes when the renewal is acknowledged + */ + CompletableFuture renew(); + + /** + * Releases snapshots for the specified table buckets, freeing associated resources. + * + * @param bucketsToRelease the set of table buckets whose snapshots should be released + * @return a future that completes when the release is processed + */ + CompletableFuture releaseSnapshots(Set bucketsToRelease); + + /** + * Drops the entire lease, releasing all held snapshots and invalidating the lease. + * + * @return a future that completes when the lease is fully dropped + */ + CompletableFuture dropLease(); +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/admin/KvSnapshotLeaseImpl.java b/fluss-client/src/main/java/org/apache/fluss/client/admin/KvSnapshotLeaseImpl.java new file mode 100644 index 0000000000..d38cf22643 --- /dev/null +++ b/fluss-client/src/main/java/org/apache/fluss/client/admin/KvSnapshotLeaseImpl.java @@ -0,0 +1,90 @@ +/* + * 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.client.admin; + +import org.apache.fluss.client.metadata.AcquireKvSnapshotLeaseResult; +import org.apache.fluss.client.utils.ClientRpcMessageUtils; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.rpc.gateway.AdminGateway; +import org.apache.fluss.rpc.messages.AcquireKvSnapshotLeaseRequest; +import org.apache.fluss.rpc.messages.DropKvSnapshotLeaseRequest; +import org.apache.fluss.rpc.messages.ReleaseKvSnapshotLeaseRequest; + +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; + +import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makeAcquireKvSnapshotLeaseRequest; + +/** The default implementation of KvSnapshotLease. */ +public class KvSnapshotLeaseImpl implements KvSnapshotLease { + private final String leaseId; + private final long leaseDurationMs; + private final AdminGateway gateway; + + public KvSnapshotLeaseImpl(String leaseId, long leaseDurationMs, AdminGateway gateway) { + this.leaseId = leaseId; + this.leaseDurationMs = leaseDurationMs; + this.gateway = gateway; + } + + @Override + public String leaseId() { + return leaseId; + } + + @Override + public long leaseDurationMs() { + return leaseDurationMs; + } + + @Override + public CompletableFuture acquireSnapshots( + Map snapshotIds) { + if (snapshotIds.isEmpty()) { + throw new IllegalArgumentException( + "The snapshotIds to acquire kv snapshot lease is empty"); + } + + return gateway.acquireKvSnapshotLease( + makeAcquireKvSnapshotLeaseRequest(leaseId, snapshotIds, leaseDurationMs)) + .thenApply(ClientRpcMessageUtils::toAcquireKvSnapshotLeaseResult); + } + + @Override + public CompletableFuture renew() { + AcquireKvSnapshotLeaseRequest request = + new AcquireKvSnapshotLeaseRequest() + .setLeaseId(leaseId) + .setLeaseDurationMs(leaseDurationMs); + return gateway.acquireKvSnapshotLease(request).thenApply(r -> null); + } + + @Override + public CompletableFuture releaseSnapshots(Set bucketsToRelease) { + ReleaseKvSnapshotLeaseRequest request = + new ReleaseKvSnapshotLeaseRequest().setLeaseId(leaseId); + return gateway.releaseKvSnapshotLease(request).thenApply(r -> null); + } + + @Override + public CompletableFuture dropLease() { + DropKvSnapshotLeaseRequest request = new DropKvSnapshotLeaseRequest().setLeaseId(leaseId); + return gateway.dropKvSnapshotLease(request).thenApply(r -> null); + } +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java b/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java index 34de4e9782..79f8ef4fbe 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java @@ -62,7 +62,6 @@ import org.apache.fluss.rpc.messages.MetadataRequest; import org.apache.fluss.rpc.messages.PbAddColumn; import org.apache.fluss.rpc.messages.PbAlterConfig; -import org.apache.fluss.rpc.messages.PbBucket; import org.apache.fluss.rpc.messages.PbBucketOffset; import org.apache.fluss.rpc.messages.PbDatabaseSummary; import org.apache.fluss.rpc.messages.PbDescribeConfig; @@ -84,6 +83,7 @@ import org.apache.fluss.rpc.messages.PbRebalanceProgressForTable; import org.apache.fluss.rpc.messages.PbRemotePathAndLocalFile; import org.apache.fluss.rpc.messages.PbRenameColumn; +import org.apache.fluss.rpc.messages.PbTableBucket; import org.apache.fluss.rpc.messages.PrefixLookupRequest; import org.apache.fluss.rpc.messages.ProduceLogRequest; import org.apache.fluss.rpc.messages.PutKvRequest; @@ -421,7 +421,7 @@ public static AlterTableRequest makeAlterTableRequest( public static AcquireKvSnapshotLeaseRequest makeAcquireKvSnapshotLeaseRequest( String leaseId, Map snapshotIds, long leaseDuration) { AcquireKvSnapshotLeaseRequest request = new AcquireKvSnapshotLeaseRequest(); - request.setLeaseId(leaseId).setLeaseDuration(leaseDuration); + request.setLeaseId(leaseId).setLeaseDurationMs(leaseDuration); Map> pbLeaseForTables = new HashMap<>(); for (Map.Entry entry : snapshotIds.entrySet()) { @@ -441,9 +441,9 @@ public static AcquireKvSnapshotLeaseRequest makeAcquireKvSnapshotLeaseRequest( for (Map.Entry> entry : pbLeaseForTables.entrySet()) { - request.addTableLeaseReq() + request.addSnapshotsToLease() .setTableId(entry.getKey()) - .addAllBucketsReqs(entry.getValue()); + .addAllBucketSnapshots(entry.getValue()); } return request; } @@ -451,9 +451,11 @@ public static AcquireKvSnapshotLeaseRequest makeAcquireKvSnapshotLeaseRequest( public static AcquireKvSnapshotLeaseResult toAcquireKvSnapshotLeaseResult( AcquireKvSnapshotLeaseResponse response) { Map unavailableSnapshots = new HashMap<>(); - for (PbKvSnapshotLeaseForTable leaseForTable : response.getTablesLeaseResList()) { - long tableId = leaseForTable.getTableId(); - for (PbKvSnapshotLeaseForBucket leaseForBucket : leaseForTable.getBucketsReqsList()) { + for (PbKvSnapshotLeaseForTable unavailableSnapshot : + response.getUnavailableSnapshotsList()) { + long tableId = unavailableSnapshot.getTableId(); + for (PbKvSnapshotLeaseForBucket leaseForBucket : + unavailableSnapshot.getBucketSnapshotsList()) { TableBucket tableBucket = new TableBucket( tableId, @@ -472,18 +474,17 @@ public static ReleaseKvSnapshotLeaseRequest makeReleaseKvSnapshotLeaseRequest( ReleaseKvSnapshotLeaseRequest request = new ReleaseKvSnapshotLeaseRequest(); request.setLeaseId(leaseId); - Map> pbLeasedTable = new HashMap<>(); + List pbTableBuckets = new ArrayList<>(); for (TableBucket tb : bucketsToRelease) { - PbBucket pbBucket = new PbBucket().setBucketId(tb.getBucket()); + PbTableBucket pbBucket = + new PbTableBucket().setTableId(tb.getTableId()).setBucketId(tb.getBucket()); if (tb.getPartitionId() != null) { pbBucket.setPartitionId(tb.getPartitionId()); } - pbLeasedTable.computeIfAbsent(tb.getTableId(), k -> new ArrayList<>()).add(pbBucket); + pbTableBuckets.add(pbBucket); } - for (Map.Entry> entry : pbLeasedTable.entrySet()) { - request.addReleaseTable().setTableId(entry.getKey()).addAllBuckets(entry.getValue()); - } + request.addAllBucketsToReleases(pbTableBuckets); return request; } diff --git a/fluss-client/src/test/java/org/apache/fluss/client/admin/CustomFlussClusterITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/admin/CustomFlussClusterITCase.java index 28b725595b..c5595c5d43 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/admin/CustomFlussClusterITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/admin/CustomFlussClusterITCase.java @@ -32,23 +32,34 @@ import org.apache.fluss.metadata.DataLakeFormat; import org.apache.fluss.metadata.DatabaseDescriptor; import org.apache.fluss.metadata.MergeEngineType; +import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableDescriptor; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.record.ChangeType; import org.apache.fluss.row.InternalRow; import org.apache.fluss.server.testutils.FlussClusterExtension; +import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.types.RowType; import org.junit.jupiter.api.Test; import java.time.Duration; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import static org.apache.fluss.record.LogRecordBatchFormat.V0_RECORD_BATCH_HEADER_SIZE; import static org.apache.fluss.record.TestData.DATA1_SCHEMA_PK; +import static org.apache.fluss.record.TestData.DATA1_TABLE_DESCRIPTOR_PK; import static org.apache.fluss.testutils.DataTestUtils.row; import static org.apache.fluss.testutils.InternalRowAssert.assertThatRow; +import static org.apache.fluss.testutils.common.CommonTestUtils.retry; import static org.assertj.core.api.Assertions.assertThat; /** IT case for tests that require manual cluster management. */ @@ -182,6 +193,120 @@ void testProjectionPushdownWithEmptyBatches() throws Exception { } } + @Test + void testConcurrentKvSnapshotLeaseOperations() throws Exception { + Configuration conf = initConfig(); + // Use a short snapshot interval for stress testing + conf.set(ConfigOptions.KV_SNAPSHOT_INTERVAL, Duration.ofSeconds(1)); + final FlussClusterExtension flussClusterExtension = + FlussClusterExtension.builder() + .setNumOfTabletServers(3) + .setClusterConf(conf) + .build(); + flussClusterExtension.start(); + + TablePath tablePath = TablePath.of("test_stress_db", "test_concurrent_kv_snapshot_lease"); + + try (Connection connection = + ConnectionFactory.createConnection( + flussClusterExtension.getClientConfig()); + Admin admin = connection.getAdmin()) { + admin.createDatabase(tablePath.getDatabaseName(), DatabaseDescriptor.EMPTY, false) + .get(); + admin.createTable(tablePath, DATA1_TABLE_DESCRIPTOR_PK, false).get(); + + long tableId = admin.getTableInfo(tablePath).get().getTableId(); + int bucketNum = 3; + + // Write initial data + try (Table table = connection.getTable(tablePath)) { + UpsertWriter upsertWriter = table.newUpsert().createWriter(); + for (int i = 0; i < 20; i++) { + upsertWriter.upsert(row(i, "value_" + i)); + } + upsertWriter.flush(); + } + + // Trigger and wait for snapshot + flussClusterExtension.triggerAndWaitSnapshot(tablePath); + + // Prepare snapshot IDs for lease operations + Map bucketSnapshotIds = new HashMap<>(); + for (int bucket = 0; bucket < bucketNum; bucket++) { + bucketSnapshotIds.put(new TableBucket(tableId, bucket), 0L); + } + + int numLeases = 5; + int numConcurrentOps = 3; + ExecutorService executor = Executors.newFixedThreadPool(numLeases * numConcurrentOps); + AtomicInteger successCount = new AtomicInteger(0); + AtomicInteger errorCount = new AtomicInteger(0); + List> futures = new ArrayList<>(); + + // Concurrently acquire, renew, and drop leases + for (int i = 0; i < numLeases; i++) { + final String leaseId = "stress-lease-" + i; + final long leaseDuration = Duration.ofDays(1).toMillis(); + + CompletableFuture future = + CompletableFuture.runAsync( + () -> { + try { + KvSnapshotLease lease = + admin.createKvSnapshotLease(leaseId, leaseDuration); + + // Acquire snapshots + lease.acquireSnapshots(bucketSnapshotIds).get(); + + // Renew the lease multiple times + for (int r = 0; r < 3; r++) { + lease.renew().get(); + } + + // Drop the lease + lease.dropLease().get(); + + successCount.incrementAndGet(); + } catch (Exception e) { + errorCount.incrementAndGet(); + } + }, + executor); + futures.add(future); + } + + // Wait for all concurrent operations to complete + CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])) + .get(2, TimeUnit.MINUTES); + + executor.shutdown(); + assertThat(executor.awaitTermination(30, TimeUnit.SECONDS)).isTrue(); + + // Verify all operations succeeded + assertThat(successCount.get()).isEqualTo(numLeases); + assertThat(errorCount.get()).isEqualTo(0); + + // Verify all leases are cleaned up + ZooKeeperClient zkClient = flussClusterExtension.getZooKeeperClient(); + retry( + Duration.ofMinutes(1), + () -> assertThat(zkClient.getKvSnapshotLeasesList()).isEmpty()); + + // Verify data is still consistent after concurrent operations + try (Table table = connection.getTable(tablePath)) { + Lookuper lookuper = table.newLookup().createLookuper(); + for (int i = 0; i < 20; i++) { + InternalRow gotRow = lookuper.lookup(row(i)).get().getSingletonRow(); + assertThatRow(gotRow) + .withSchema(DATA1_SCHEMA_PK.getRowType()) + .isEqualTo(row(i, "value_" + i)); + } + } + } finally { + flussClusterExtension.close(); + } + } + protected static Configuration initConfig() { Configuration conf = new Configuration(); conf.setInt(ConfigOptions.DEFAULT_REPLICATION_FACTOR, 3); diff --git a/fluss-client/src/test/java/org/apache/fluss/client/security/acl/FlussAuthorizationITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/security/acl/FlussAuthorizationITCase.java index fdb5e3015a..0f23f58b27 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/security/acl/FlussAuthorizationITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/security/acl/FlussAuthorizationITCase.java @@ -22,6 +22,7 @@ import org.apache.fluss.client.FlussConnection; import org.apache.fluss.client.admin.Admin; import org.apache.fluss.client.admin.FlussAdmin; +import org.apache.fluss.client.admin.KvSnapshotLease; import org.apache.fluss.client.table.Table; import org.apache.fluss.client.table.scanner.batch.BatchScanner; import org.apache.fluss.client.table.writer.AppendWriter; @@ -55,6 +56,7 @@ import org.apache.fluss.rpc.messages.InitWriterRequest; import org.apache.fluss.rpc.messages.InitWriterResponse; import org.apache.fluss.rpc.messages.MetadataRequest; +import org.apache.fluss.rpc.messages.ReleaseKvSnapshotLeaseRequest; import org.apache.fluss.rpc.metrics.TestingClientMetricGroup; import org.apache.fluss.security.acl.AccessControlEntry; import org.apache.fluss.security.acl.AccessControlEntryFilter; @@ -1060,6 +1062,119 @@ void testRebalanceDuringConcurrentTableCreation() throws Exception { } } + // ------------------------------------------------------------------------ + // KV Snapshot Lease Authorization Tests + // ------------------------------------------------------------------------ + + @Test + void testAcquireKvSnapshotLease() throws Exception { + TableInfo tableInfo = rootAdmin.getTableInfo(DATA1_TABLE_PATH_PK).get(); + long tableId = tableInfo.getTableId(); + FLUSS_CLUSTER_EXTENSION.waitUntilTableReady(tableId); + + KvSnapshotLease kvSnapshotLease = + guestAdmin.createKvSnapshotLease( + "test-acquire-lease", Duration.ofDays(1).toMillis()); + Map snapshotIds = new HashMap<>(); + snapshotIds.put(new TableBucket(tableId, 0), 0L); + + // test acquireKvSnapshotLease without READ permission on table resource + assertThatThrownBy(() -> kvSnapshotLease.acquireSnapshots(snapshotIds).get()) + .rootCause() + .isInstanceOf(AuthorizationException.class) + .hasMessageContaining( + String.format( + "Principal %s have no authorization to operate READ on resource Resource{type=TABLE, name='%s'}", + guestPrincipal, DATA1_TABLE_PATH_PK)); + + // add READ permission to guest user on table resource + List aclBindings = + Collections.singletonList( + new AclBinding( + Resource.table(DATA1_TABLE_PATH_PK), + new AccessControlEntry( + guestPrincipal, "*", READ, PermissionType.ALLOW))); + rootAdmin.createAcls(aclBindings).all().get(); + FLUSS_CLUSTER_EXTENSION.waitUntilAuthenticationSync(aclBindings, true); + + // test acquireKvSnapshotLease with READ permission should succeed + // (no AuthorizationException should be thrown) + kvSnapshotLease.acquireSnapshots(snapshotIds).get(); + + // cleanup: drop the lease using root admin + rootAdmin + .createKvSnapshotLease("test-acquire-lease", Duration.ofDays(1).toMillis()) + .dropLease() + .get(); + } + + @Test + void testReleaseKvSnapshotLease() throws Exception { + TableInfo tableInfo = rootAdmin.getTableInfo(DATA1_TABLE_PATH_PK).get(); + long tableId = tableInfo.getTableId(); + FLUSS_CLUSTER_EXTENSION.waitUntilTableReady(tableId); + + AdminGateway guestGateway = ((FlussAdmin) guestAdmin).getAdminGateway(); + ReleaseKvSnapshotLeaseRequest request = + ClientRpcMessageUtils.makeReleaseKvSnapshotLeaseRequest( + "test-release-lease", Collections.singleton(new TableBucket(tableId, 0))); + + // test releaseKvSnapshotLease without READ permission on table resource + assertThatThrownBy(() -> guestGateway.releaseKvSnapshotLease(request).get()) + .rootCause() + .isInstanceOf(AuthorizationException.class) + .hasMessageContaining( + String.format( + "Principal %s have no authorization to operate READ on resource Resource{type=TABLE, name='%s'}", + guestPrincipal, DATA1_TABLE_PATH_PK)); + + // add READ permission to guest user on table resource + List aclBindings = + Collections.singletonList( + new AclBinding( + Resource.table(DATA1_TABLE_PATH_PK), + new AccessControlEntry( + guestPrincipal, "*", READ, PermissionType.ALLOW))); + rootAdmin.createAcls(aclBindings).all().get(); + FLUSS_CLUSTER_EXTENSION.waitUntilAuthenticationSync(aclBindings, true); + + // test releaseKvSnapshotLease with READ permission should succeed + // (the lease doesn't exist, but no AuthorizationException should be thrown) + guestGateway.releaseKvSnapshotLease(request).get(); + } + + @Test + void testDropKvSnapshotLease() throws Exception { + KvSnapshotLease kvSnapshotLease = + guestAdmin.createKvSnapshotLease("test-drop-lease", Duration.ofDays(1).toMillis()); + + // test dropKvSnapshotLease without WRITE permission on cluster resource + assertThatThrownBy(() -> kvSnapshotLease.dropLease().get()) + .rootCause() + .isInstanceOf(AuthorizationException.class) + .hasMessageContaining( + String.format( + "Principal %s have no authorization to operate WRITE on resource Resource{type=CLUSTER, name='fluss-cluster'}", + guestPrincipal)); + + // add WRITE permission to guest user on cluster resource + rootAdmin + .createAcls( + Collections.singletonList( + new AclBinding( + Resource.cluster(), + new AccessControlEntry( + guestPrincipal, + "*", + OperationType.WRITE, + PermissionType.ALLOW)))) + .all() + .get(); + + // test dropKvSnapshotLease with WRITE permission should succeed + kvSnapshotLease.dropLease().get(); + } + // ------------------------------------------------------------------------ // Producer Offsets Authorization Tests // ------------------------------------------------------------------------ diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvSnapshotBatchScannerITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvSnapshotBatchScannerITCase.java index 19cdc28ab7..24415f075c 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvSnapshotBatchScannerITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvSnapshotBatchScannerITCase.java @@ -19,12 +19,12 @@ import org.apache.fluss.bucketing.BucketingFunction; import org.apache.fluss.client.admin.ClientToServerITCaseBase; +import org.apache.fluss.client.admin.KvSnapshotLease; import org.apache.fluss.client.metadata.KvSnapshots; import org.apache.fluss.client.table.Table; import org.apache.fluss.client.table.scanner.RemoteFileDownloader; import org.apache.fluss.client.table.writer.UpsertWriter; import org.apache.fluss.client.write.HashBucketAssigner; -import org.apache.fluss.exception.KvSnapshotLeaseNotExistException; import org.apache.fluss.metadata.DataLakeFormat; import org.apache.fluss.metadata.Schema; import org.apache.fluss.metadata.TableBucket; @@ -35,9 +35,9 @@ import org.apache.fluss.row.ProjectedRow; import org.apache.fluss.row.encode.CompactedKeyEncoder; import org.apache.fluss.row.encode.KeyEncoder; +import org.apache.fluss.server.coordinator.lease.KvSnapshotLeaseHandler; +import org.apache.fluss.server.coordinator.lease.KvSnapshotLeaseMetadataManager; import org.apache.fluss.server.zk.ZooKeeperClient; -import org.apache.fluss.server.zk.data.lease.KvSnapshotLease; -import org.apache.fluss.server.zk.data.lease.KvSnapshotLeaseMetadataManager; import org.apache.fluss.server.zk.data.lease.KvSnapshotTableLease; import org.apache.fluss.types.DataTypes; @@ -56,7 +56,6 @@ import static org.apache.fluss.record.TestData.DATA1_ROW_TYPE; import static org.apache.fluss.testutils.DataTestUtils.compactedRow; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; /** IT Case for {@link KvSnapshotBatchScanner}. */ class KvSnapshotBatchScannerITCase extends ClientToServerITCaseBase { @@ -193,8 +192,8 @@ public void testKvSnapshotLease() throws Exception { TablePath tablePath = TablePath.of(DEFAULT_DB, "test-kv-snapshot-lease"); long tableId = createTable(tablePath, DEFAULT_TABLE_DESCRIPTOR, true); - String kvSnapshotLease1 = "test-lease"; - String kvSnapshotLease2 = "test-lease2"; + String kvSnapshotLeaseId1 = "test-lease"; + String kvSnapshotLeaseId2 = "test-lease2"; // scan the snapshot Map> expectedRowByBuckets = putRows(tableId, tablePath, 10); @@ -216,11 +215,12 @@ public void testKvSnapshotLease() throws Exception { TableBucket tableBucket = new TableBucket(kvSnapshots.getTableId(), bucketId); consumeBuckets.put(tableBucket, kvSnapshots.getSnapshotId(bucketId).getAsLong()); } - admin.acquireKvSnapshotLease( - kvSnapshotLease1, consumeBuckets, Duration.ofDays(1).toMillis()) - .get(); + + KvSnapshotLease kvSnapshotLease1 = + admin.createKvSnapshotLease(kvSnapshotLeaseId1, Duration.ofDays(1).toMillis()); + kvSnapshotLease1.acquireSnapshots(consumeBuckets).get(); checkKvSnapshotLeaseEquals( - metadataManager, kvSnapshotLease1, tableId, new Long[] {0L, 0L, 0L}); + metadataManager, kvSnapshotLeaseId1, tableId, new Long[] {0L, 0L, 0L}); expectedRowByBuckets = putRows(tableId, tablePath, 10); // wait snapshot2 finish @@ -233,11 +233,12 @@ public void testKvSnapshotLease() throws Exception { TableBucket tableBucket = new TableBucket(kvSnapshots.getTableId(), bucketId); consumeBuckets.put(tableBucket, kvSnapshots.getSnapshotId(bucketId).getAsLong()); } - admin.acquireKvSnapshotLease( - kvSnapshotLease2, consumeBuckets, Duration.ofDays(1).toMillis()) - .get(); + + KvSnapshotLease kvSnapshotLease2 = + admin.createKvSnapshotLease(kvSnapshotLeaseId2, Duration.ofDays(1).toMillis()); + kvSnapshotLease2.acquireSnapshots(consumeBuckets).get(); checkKvSnapshotLeaseEquals( - metadataManager, kvSnapshotLease2, tableId, new Long[] {1L, 1L, 1L}); + metadataManager, kvSnapshotLeaseId2, tableId, new Long[] {1L, 1L, 1L}); // check even snapshot1 is generated, snapshot0 also retained as lease exists. for (TableBucket tb : expectedRowByBuckets.keySet()) { assertThat(zkClient.getTableBucketSnapshot(tb, 0L).isPresent()).isTrue(); @@ -249,18 +250,16 @@ public void testKvSnapshotLease() throws Exception { FLUSS_CLUSTER_EXTENSION.triggerAndWaitSnapshots(expectedRowByBuckets.keySet()); // release lease1. - admin.releaseKvSnapshotLease( - kvSnapshotLease1, Collections.singleton(new TableBucket(tableId, 0))) - .get(); + kvSnapshotLease1.releaseSnapshots(Collections.singleton(new TableBucket(tableId, 0))).get(); checkKvSnapshotLeaseEquals( - metadataManager, kvSnapshotLease1, tableId, new Long[] {-1L, 0L, 0L}); + metadataManager, kvSnapshotLeaseId1, tableId, new Long[] {-1L, 0L, 0L}); // release lease2. - admin.releaseKvSnapshotLease(kvSnapshotLease2, consumeBuckets.keySet()).get(); - assertThat(zkClient.getKvSnapshotLeasesList()).doesNotContain(kvSnapshotLease2); + kvSnapshotLease2.releaseSnapshots(consumeBuckets.keySet()).get(); + assertThat(zkClient.getKvSnapshotLeasesList()).doesNotContain(kvSnapshotLeaseId2); // release all kv snapshot lease of lease1 - admin.releaseAllKvSnapshotLease(kvSnapshotLease1).get(); + kvSnapshotLease1.dropLease().get(); assertThat(zkClient.getKvSnapshotLeasesList()).isEmpty(); expectedRowByBuckets = putRows(tableId, tablePath, 10); @@ -273,10 +272,10 @@ public void testKvSnapshotLease() throws Exception { assertThat(zkClient.getTableBucketSnapshot(tb, 1L).isPresent()).isFalse(); } - assertThatThrownBy(() -> admin.releaseAllKvSnapshotLease("no-exist-lease").get()) - .rootCause() - .isInstanceOf(KvSnapshotLeaseNotExistException.class) - .hasMessageContaining("kv snapshot lease 'no-exist-lease' not exits"); + // drop no exist lease, no exception. + admin.createKvSnapshotLease("no-exist-lease", Duration.ofDays(1).toMillis()) + .dropLease() + .get(); } private Map> putRows( @@ -345,9 +344,9 @@ private void checkKvSnapshotLeaseEquals( Long[] expectedBucketIndex) throws Exception { assertThat(metadataManager.getLeasesList()).contains(leaseId); - Optional leaseOpt = metadataManager.getLease(leaseId); + Optional leaseOpt = metadataManager.getLease(leaseId); assertThat(leaseOpt).isPresent(); - KvSnapshotLease actualLease = leaseOpt.get(); + KvSnapshotLeaseHandler actualLease = leaseOpt.get(); Map tableIdToTableLease = actualLease.getTableIdToTableLease(); KvSnapshotTableLease tableLease = tableIdToTableLease.get(tableId); assertThat(tableLease).isNotNull(); diff --git a/fluss-common/src/main/java/org/apache/fluss/exception/KvSnapshotLeaseNotExistException.java b/fluss-common/src/main/java/org/apache/fluss/exception/KvSnapshotLeaseNotExistException.java deleted file mode 100644 index b77660c67a..0000000000 --- a/fluss-common/src/main/java/org/apache/fluss/exception/KvSnapshotLeaseNotExistException.java +++ /dev/null @@ -1,31 +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.exception; - -/** - * Kv snapshot lease not exist exception. - * - * @since 0.9 - */ -public class KvSnapshotLeaseNotExistException extends ApiException { - private static final long serialVersionUID = 1L; - - public KvSnapshotLeaseNotExistException(String message) { - super(message); - } -} diff --git a/fluss-common/src/main/java/org/apache/fluss/metadata/KvSnapshotLeaseForBucket.java b/fluss-common/src/main/java/org/apache/fluss/metadata/TableBucketSnapshot.java similarity index 68% rename from fluss-common/src/main/java/org/apache/fluss/metadata/KvSnapshotLeaseForBucket.java rename to fluss-common/src/main/java/org/apache/fluss/metadata/TableBucketSnapshot.java index fc3840bd27..2b9b470960 100644 --- a/fluss-common/src/main/java/org/apache/fluss/metadata/KvSnapshotLeaseForBucket.java +++ b/fluss-common/src/main/java/org/apache/fluss/metadata/TableBucketSnapshot.java @@ -19,31 +19,31 @@ import java.util.Objects; -/** An entity for kv snapshot lease for bucket. */ -public class KvSnapshotLeaseForBucket { +/** An entity for kv snapshot of table bucket. */ +public class TableBucketSnapshot { private final TableBucket tableBucket; - private final long kvSnapshotId; + private final long snapshotId; - public KvSnapshotLeaseForBucket(TableBucket tableBucket, long kvSnapshotId) { + public TableBucketSnapshot(TableBucket tableBucket, long snapshotId) { this.tableBucket = tableBucket; - this.kvSnapshotId = kvSnapshotId; + this.snapshotId = snapshotId; } public TableBucket getTableBucket() { return tableBucket; } - public long getKvSnapshotId() { - return kvSnapshotId; + public long getSnapshotId() { + return snapshotId; } @Override public String toString() { - return "KvSnapshotLeaseForBucket{" + return "TableBucketSnapshot{" + "tableBucket=" + tableBucket - + ", kvSnapshotId=" - + kvSnapshotId + + ", snapshotId=" + + snapshotId + '}'; } @@ -55,12 +55,12 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } - KvSnapshotLeaseForBucket that = (KvSnapshotLeaseForBucket) o; - return kvSnapshotId == that.kvSnapshotId && Objects.equals(tableBucket, that.tableBucket); + TableBucketSnapshot that = (TableBucketSnapshot) o; + return snapshotId == that.snapshotId && Objects.equals(tableBucket, that.tableBucket); } @Override public int hashCode() { - return Objects.hash(tableBucket, kvSnapshotId); + return Objects.hash(tableBucket, snapshotId); } } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/FlinkConnectorOptions.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/FlinkConnectorOptions.java index a00e0ff5e4..03f24dec44 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/FlinkConnectorOptions.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/FlinkConnectorOptions.java @@ -82,9 +82,10 @@ public class FlinkConnectorOptions { .stringType() .defaultValue(String.valueOf(UUID.randomUUID())) .withDescription( - "The lease id to ping kv snapshots. If set, the acquired kv snapshots will not be deleted " - + "until the consumer finished consuming all the snapshots or the lease duration time " - + "is reached. If not set, an UUID will be set."); + "The lease ID used to protect acquired KV snapshots from deletion. If specified, " + + "the snapshots will be retained until either the consumer finishes " + + "processing all of them or the lease duration expires. By default, " + + "this value is set to a randomly generated UUID string if not explicitly provided."); public static final ConfigOption SCAN_KV_SNAPSHOT_LEASE_DURATION = ConfigOptions.key("scan.kv.snapshot.lease.duration") diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/catalog/FlinkTableFactory.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/catalog/FlinkTableFactory.java index ab3b70119a..b839a94a41 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/catalog/FlinkTableFactory.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/catalog/FlinkTableFactory.java @@ -143,15 +143,7 @@ public DynamicTableSource createDynamicTableSource(Context context) { .get(FlinkConnectorOptions.SCAN_PARTITION_DISCOVERY_INTERVAL) .toMillis(); - LeaseContext leaseContext = - primaryKeyIndexes.length > 0 - ? new LeaseContext( - tableOptions.get(FlinkConnectorOptions.SCAN_KV_SNAPSHOT_LEASE_ID), - tableOptions - .get(FlinkConnectorOptions.SCAN_KV_SNAPSHOT_LEASE_DURATION) - .toMillis()) - : new LeaseContext(null, null); - + LeaseContext leaseContext = LeaseContext.fromConf(tableOptions); return new FlinkTableSource( toFlussTablePath(context.getObjectIdentifier()), toFlussClientConfig( diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/ReleaseAllKvSnapshotLeaseProcedure.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/DropKvSnapshotLeaseProcedure.java similarity index 74% rename from fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/ReleaseAllKvSnapshotLeaseProcedure.java rename to fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/DropKvSnapshotLeaseProcedure.java index ba6e9c76ff..a55255cf47 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/ReleaseAllKvSnapshotLeaseProcedure.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/DropKvSnapshotLeaseProcedure.java @@ -17,30 +17,34 @@ package org.apache.fluss.flink.procedure; +import org.apache.fluss.client.admin.KvSnapshotLease; + import org.apache.flink.table.annotation.ArgumentHint; import org.apache.flink.table.annotation.DataTypeHint; import org.apache.flink.table.annotation.ProcedureHint; import org.apache.flink.table.procedure.ProcedureContext; +import java.time.Duration; + /** - * Procedure to release all kv snapshots leased of specified leaseId. See {@link - * org.apache.fluss.client.admin.Admin#releaseAllKvSnapshotLease(String)} for more details. + * Procedure to drop all kv snapshots leased of specified leaseId. See {@link + * KvSnapshotLease#dropLease()} for more details. * *

Usage examples: * *

- * -- Release all kv snapshots leased of specified leaseId
- * CALL sys.release_all_kv_snapshot_lease('test-lease-id');
+ * -- Drop kv snapshots leased of specified leaseId
+ * CALL sys.drop_kv_snapshot_lease('test-lease-id');
  * 
*/ -public class ReleaseAllKvSnapshotLeaseProcedure extends ProcedureBase { +public class DropKvSnapshotLeaseProcedure extends ProcedureBase { @ProcedureHint( argument = { @ArgumentHint(name = "leaseId", type = @DataTypeHint("STRING")), }) public String[] call(ProcedureContext context, String leaseId) throws Exception { - admin.releaseAllKvSnapshotLease(leaseId).get(); + admin.createKvSnapshotLease(leaseId, Duration.ofDays(1).toMillis()).dropLease().get(); return new String[] {"success"}; } } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/ProcedureManager.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/ProcedureManager.java index a0d0d54e0d..fc2632e856 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/ProcedureManager.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/ProcedureManager.java @@ -78,8 +78,7 @@ private enum ProcedureEnum { REBALANCE("sys.rebalance", RebalanceProcedure.class), CANCEL_REBALANCE("sys.cancel_rebalance", CancelRebalanceProcedure.class), LIST_REBALANCE_PROGRESS("sys.list_rebalance", ListRebalanceProcessProcedure.class), - RELEASE_ALL_KV_SNAPSHOT_LEASE( - "sys.release_all_kv_snapshot_lease", ReleaseAllKvSnapshotLeaseProcedure.class); + DROP_KV_SNAPSHOT_LEASE("sys.drop_kv_snapshot_lease", DropKvSnapshotLeaseProcedure.class); private final String path; private final Class procedureClass; diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkSource.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkSource.java index d8072559dd..fba394edfd 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkSource.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkSource.java @@ -171,7 +171,9 @@ public SplitEnumerator restoreEnumerator streaming, partitionFilters, lakeSource, - sourceEnumeratorState.getLeaseContext()); + new LeaseContext( + sourceEnumeratorState.getLeaseId(), + leaseContext.getKvSnapshotLeaseDurationMs())); } @Override diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlussSource.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlussSource.java index 1c181aafbc..d00a74a26b 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlussSource.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlussSource.java @@ -68,8 +68,7 @@ public class FlussSource extends FlinkSource { OffsetsInitializer offsetsInitializer, long scanPartitionDiscoveryIntervalMs, FlussDeserializationSchema deserializationSchema, - boolean streaming, - LeaseContext leaseContext) { + boolean streaming) { // TODO: Support partition pushDown in datastream super( flussConf, @@ -83,7 +82,7 @@ public class FlussSource extends FlinkSource { deserializationSchema, streaming, null, - leaseContext); + LeaseContext.DEFAULT); } /** diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlussSourceBuilder.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlussSourceBuilder.java index da1d8f7b7b..afd955c01f 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlussSourceBuilder.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlussSourceBuilder.java @@ -25,7 +25,6 @@ import org.apache.fluss.config.Configuration; import org.apache.fluss.flink.FlinkConnectorOptions; import org.apache.fluss.flink.source.deserializer.FlussDeserializationSchema; -import org.apache.fluss.flink.source.reader.LeaseContext; import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.types.RowType; @@ -72,8 +71,6 @@ public class FlussSourceBuilder { private Long scanPartitionDiscoveryIntervalMs; private OffsetsInitializer offsetsInitializer; private FlussDeserializationSchema deserializationSchema; - private String kvSnapshotLeaseId; - private long kvSnapshotLeaseDurationMs; private String bootstrapServers; @@ -177,16 +174,6 @@ public FlussSourceBuilder setProjectedFields(String... projectedFieldNames) return this; } - public FlussSourceBuilder setKvSnapshotLeaseId(String kvSnapshotLeaseId) { - this.kvSnapshotLeaseId = kvSnapshotLeaseId; - return this; - } - - public FlussSourceBuilder setKvSnapshotLeaseDurationMs(long kvSnapshotLeaseDurationMs) { - this.kvSnapshotLeaseDurationMs = kvSnapshotLeaseDurationMs; - return this; - } - /** * Sets custom Fluss configuration properties for the source connector. * @@ -313,9 +300,6 @@ public FlussSource build() { offsetsInitializer, scanPartitionDiscoveryIntervalMs, deserializationSchema, - true, - hasPrimaryKey - ? new LeaseContext(kvSnapshotLeaseId, kvSnapshotLeaseDurationMs) - : new LeaseContext(null, null)); + true); } } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumerator.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumerator.java index 0ebb808e81..1e3eac0189 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumerator.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumerator.java @@ -42,6 +42,7 @@ import org.apache.fluss.lake.source.LakeSource; import org.apache.fluss.lake.source.LakeSplit; import org.apache.fluss.metadata.PartitionInfo; +import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePath; @@ -106,6 +107,7 @@ public class FlinkSourceEnumerator private final boolean hasPrimaryKey; private final boolean isPartitioned; private final Configuration flussConf; + private final boolean isStreaming; private final SplitEnumeratorContext context; @@ -258,6 +260,7 @@ public FlinkSourceEnumerator( this.lakeSource = lakeSource; this.workerExecutor = workerExecutor; this.leaseContext = leaseContext; + this.isStreaming = streaming; } @Override @@ -550,24 +553,20 @@ private KvSnapshots getLatestKvSnapshotsAndRegister(@Nullable String partitionNa Map snapshotIds = new HashMap<>(); Map logOffsets = new HashMap<>(); - // retry to get the latest kv snapshots and acquire kvSnapshot lease. + // Get the latest kv snapshots and acquire kvSnapshot lease. try { KvSnapshots kvSnapshots = getLatestKvSnapshots(partitionName); - Set remainingTableBuckets = new HashSet<>(kvSnapshots.getTableBuckets()); tableId = kvSnapshots.getTableId(); partitionId = kvSnapshots.getPartitionId(); - Set ignoreBuckets = new HashSet<>(); Map bucketsToLease = new HashMap<>(); - for (TableBucket tb : remainingTableBuckets) { + for (TableBucket tb : kvSnapshots.getTableBuckets()) { int bucket = tb.getBucket(); OptionalLong snapshotIdOpt = kvSnapshots.getSnapshotId(bucket); OptionalLong logOffsetOpt = kvSnapshots.getLogOffset(bucket); if (snapshotIdOpt.isPresent() && !ignoreTableBucket(tb)) { bucketsToLease.put(tb, snapshotIdOpt.getAsLong()); - } else { - ignoreBuckets.add(tb); } snapshotIds.put( @@ -575,31 +574,25 @@ private KvSnapshots getLatestKvSnapshotsAndRegister(@Nullable String partitionNa logOffsets.put(bucket, logOffsetOpt.isPresent() ? logOffsetOpt.getAsLong() : null); } - if (!ignoreBuckets.isEmpty()) { - remainingTableBuckets.removeAll(ignoreBuckets); - } - if (!bucketsToLease.isEmpty()) { String kvSnapshotLeaseId = leaseContext.getKvSnapshotLeaseId(); LOG.info( "Try to acquire kv snapshot lease {} for table {}", kvSnapshotLeaseId, - tablePath); - Long kvSnapshotLeaseDurationMs = leaseContext.getKvSnapshotLeaseDurationMs(); + PhysicalTablePath.of(tablePath, partitionName)); + long kvSnapshotLeaseDurationMs = leaseContext.getKvSnapshotLeaseDurationMs(); checkNotNull(kvSnapshotLeaseDurationMs, "kv snapshot lease duration is null."); - remainingTableBuckets = + Set unavailableTableBucketSet = flussAdmin - .acquireKvSnapshotLease( - kvSnapshotLeaseId, - bucketsToLease, - kvSnapshotLeaseDurationMs) + .createKvSnapshotLease(kvSnapshotLeaseId, kvSnapshotLeaseDurationMs) + .acquireSnapshots(bucketsToLease) .get() .getUnavailableTableBucketSet(); - if (!remainingTableBuckets.isEmpty()) { + if (!unavailableTableBucketSet.isEmpty()) { LOG.info( - "Failed to acquire kv snapshot lease for table {}: {}. Retry to re-acquire", + "Failed to acquire kv snapshot lease for table {}: {}.", tablePath, - remainingTableBuckets); + unavailableTableBucketSet); } } } catch (Exception e) { @@ -1010,7 +1003,7 @@ public SourceEnumeratorState snapshotState(long checkpointId) { assignedTableBuckets, assignedPartitions, pendingHybridLakeFlussSplits, - leaseContext); + leaseContext.getKvSnapshotLeaseId()); LOG.debug("Source Checkpoint is {}", enumeratorState); return enumeratorState; } @@ -1028,8 +1021,10 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception { // send request to fluss to unregister the kv snapshot lease. try { flussAdmin - .releaseKvSnapshotLease( - leaseContext.getKvSnapshotLeaseId(), consumedKvSnapshots) + .createKvSnapshotLease( + leaseContext.getKvSnapshotLeaseId(), + leaseContext.getKvSnapshotLeaseDurationMs()) + .releaseSnapshots(consumedKvSnapshots) .get(); } catch (Exception e) { LOG.error("Failed to release kv snapshot lease. These snapshot need to re-enqueue", e); @@ -1060,6 +1055,17 @@ public Set getAndRemoveConsumedBucketsUpTo(long checkpointId) { public void close() throws IOException { try { closed = true; + + if (!isStreaming) { + // drop the kv snapshot lease for the batch mode. + flussAdmin + .createKvSnapshotLease( + leaseContext.getKvSnapshotLeaseId(), + leaseContext.getKvSnapshotLeaseDurationMs()) + .dropLease() + .get(); + } + if (workerExecutor != null) { workerExecutor.close(); } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/reader/FlinkSourceReader.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/reader/FlinkSourceReader.java index 01eadbcea3..9a6c4e8dad 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/reader/FlinkSourceReader.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/reader/FlinkSourceReader.java @@ -124,6 +124,8 @@ public List snapshotState(long checkpointId) { bucketsFinishedConsumeKvSnapshot, checkpointId); + // TODO Reduce the external IO operation, trace by + // https://github.com/apache/fluss/issues/2597. context.sendSourceEventToCoordinator( new FinishedKvSnapshotConsumeEvent( checkpointId, bucketsFinishedConsumeKvSnapshot)); diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/reader/LeaseContext.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/reader/LeaseContext.java index 5ee12bdfa9..c17fb15b77 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/reader/LeaseContext.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/reader/LeaseContext.java @@ -17,33 +17,45 @@ package org.apache.fluss.flink.source.reader; -import javax.annotation.Nullable; +import org.apache.fluss.flink.FlinkConnectorOptions; + +import org.apache.flink.configuration.ReadableConfig; import java.io.Serializable; +import java.time.Duration; import java.util.Objects; +import java.util.UUID; /** Context for lease. */ public class LeaseContext implements Serializable { private static final long serialVersionUID = 1L; + public static final LeaseContext DEFAULT = + new LeaseContext(UUID.randomUUID().toString(), Duration.ofDays(1).toMillis()); + // kv snapshot lease id. null for log table. - private final @Nullable String kvSnapshotLeaseId; + private final String kvSnapshotLeaseId; // kv snapshot lease duration. null for log table. - private final @Nullable Long kvSnapshotLeaseDurationMs; + private final long kvSnapshotLeaseDurationMs; - public LeaseContext( - @Nullable String kvSnapshotLeaseId, @Nullable Long kvSnapshotLeaseDurationMs) { + public LeaseContext(String kvSnapshotLeaseId, long kvSnapshotLeaseDurationMs) { this.kvSnapshotLeaseId = kvSnapshotLeaseId; this.kvSnapshotLeaseDurationMs = kvSnapshotLeaseDurationMs; } - public @Nullable String getKvSnapshotLeaseId() { + public static LeaseContext fromConf(ReadableConfig tableOptions) { + return new LeaseContext( + tableOptions.get(FlinkConnectorOptions.SCAN_KV_SNAPSHOT_LEASE_ID), + tableOptions.get(FlinkConnectorOptions.SCAN_KV_SNAPSHOT_LEASE_DURATION).toMillis()); + } + + public String getKvSnapshotLeaseId() { return kvSnapshotLeaseId; } - public @Nullable Long getKvSnapshotLeaseDurationMs() { + public long getKvSnapshotLeaseDurationMs() { return kvSnapshotLeaseDurationMs; } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/state/FlussSourceEnumeratorStateSerializer.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/state/FlussSourceEnumeratorStateSerializer.java index 6907b0e271..0ca309f6f8 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/state/FlussSourceEnumeratorStateSerializer.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/state/FlussSourceEnumeratorStateSerializer.java @@ -199,7 +199,7 @@ private SourceEnumeratorState deserializeV0(byte[] serialized) throws IOExceptio assignBucketAndPartitions.f0, assignBucketAndPartitions.f1, remainingHybridLakeFlussSplits, - new LeaseContext(null, null)); + LeaseContext.DEFAULT.getKvSnapshotLeaseId()); } private SourceEnumeratorState deserializeV1(byte[] serialized) throws IOException { @@ -217,7 +217,7 @@ private SourceEnumeratorState deserializeV1(byte[] serialized) throws IOExceptio assignBucketAndPartitions.f0, assignBucketAndPartitions.f1, remainingHybridLakeFlussSplits, - new LeaseContext(null, null)); + LeaseContext.DEFAULT.getKvSnapshotLeaseId()); } private SourceEnumeratorState deserializeV2(byte[] serialized) throws IOException { @@ -233,7 +233,7 @@ private SourceEnumeratorState deserializeV2(byte[] serialized) throws IOExceptio assignBucketAndPartitions.f0, assignBucketAndPartitions.f1, remainingHybridLakeFlussSplits, - leaseContext); + leaseContext.getKvSnapshotLeaseId()); } private Tuple2, Map> deserializeAssignBucketAndPartitions( @@ -290,36 +290,14 @@ private List deserializeRemainingHybridLakeFlussSplits( private void serializeLeaseContext(final DataOutputSerializer out, SourceEnumeratorState state) throws IOException { - LeaseContext leaseContext = state.getLeaseContext(); - String kvSnapshotLeaseId = leaseContext.getKvSnapshotLeaseId(); - if (kvSnapshotLeaseId != null) { - out.writeBoolean(true); - out.writeUTF(kvSnapshotLeaseId); - } else { - out.writeBoolean(false); - } - - Long kvSnapshotLeaseDurationMs = leaseContext.getKvSnapshotLeaseDurationMs(); - if (kvSnapshotLeaseDurationMs != null) { - out.writeBoolean(true); - out.writeLong(kvSnapshotLeaseDurationMs); - } else { - out.writeBoolean(false); - } + String leaseId = state.getLeaseId(); + out.writeUTF(leaseId); } private LeaseContext deserializeLeaseContext(final DataInputDeserializer in) throws IOException { - if (in.readBoolean()) { - String kvSnapshotLeaseId = in.readUTF(); - if (in.readBoolean()) { - Long kvSnapshotLeaseDurationMs = in.readLong(); - return new LeaseContext(kvSnapshotLeaseId, kvSnapshotLeaseDurationMs); - } else { - return new LeaseContext(kvSnapshotLeaseId, null); - } - } else { - return new LeaseContext(null, null); - } + String kvSnapshotLeaseId = in.readUTF(); + return new LeaseContext( + kvSnapshotLeaseId, LeaseContext.DEFAULT.getKvSnapshotLeaseDurationMs()); } } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/state/SourceEnumeratorState.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/state/SourceEnumeratorState.java index a97736f47d..7ee021ff58 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/state/SourceEnumeratorState.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/state/SourceEnumeratorState.java @@ -17,7 +17,6 @@ package org.apache.fluss.flink.source.state; -import org.apache.fluss.flink.source.reader.LeaseContext; import org.apache.fluss.flink.source.split.SourceSplitBase; import org.apache.fluss.metadata.TableBucket; @@ -43,17 +42,17 @@ public class SourceEnumeratorState { @Nullable private final List remainingHybridLakeFlussSplits; // lease context for restore. - private final LeaseContext leaseContext; + private final String leaseId; public SourceEnumeratorState( Set assignedBuckets, Map assignedPartitions, @Nullable List remainingHybridLakeFlussSplits, - @Nullable LeaseContext leaseContext) { + String leaseId) { this.assignedBuckets = assignedBuckets; this.assignedPartitions = assignedPartitions; this.remainingHybridLakeFlussSplits = remainingHybridLakeFlussSplits; - this.leaseContext = leaseContext; + this.leaseId = leaseId; } public Set getAssignedBuckets() { @@ -69,8 +68,8 @@ public List getRemainingHybridLakeFlussSplits() { return remainingHybridLakeFlussSplits; } - public LeaseContext getLeaseContext() { - return leaseContext; + public String getLeaseId() { + return leaseId; } @Override @@ -86,7 +85,7 @@ public boolean equals(Object o) { && Objects.equals(assignedPartitions, that.assignedPartitions) && Objects.equals( remainingHybridLakeFlussSplits, that.remainingHybridLakeFlussSplits) - && Objects.equals(leaseContext, that.leaseContext); + && Objects.equals(leaseId, that.leaseId); } @Override @@ -103,8 +102,8 @@ public String toString() { + assignedPartitions + ", remainingHybridLakeFlussSplits=" + remainingHybridLakeFlussSplits - + ", leaseContext=" - + leaseContext + + ", leaseId=" + + leaseId + '}'; } } diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/procedure/FlinkProcedureITCase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/procedure/FlinkProcedureITCase.java index 1cd87da56f..5e22c3ff0d 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/procedure/FlinkProcedureITCase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/procedure/FlinkProcedureITCase.java @@ -144,7 +144,7 @@ void testShowProcedures() throws Exception { "+I[sys.rebalance]", "+I[sys.cancel_rebalance]", "+I[sys.list_rebalance]", - "+I[sys.release_all_kv_snapshot_lease]"); + "+I[sys.drop_kv_snapshot_lease]"); // make sure no more results is unread. assertResultsIgnoreOrder(showProceduresIterator, expectedShowProceduresResult, true); } @@ -784,7 +784,7 @@ void testListRebalanceProgress() throws Exception { } @Test - void testReleaseAllKvSnapshotLeaseProcedure() throws Exception { + void testDropKvSnapshotLeaseProcedure() throws Exception { tEnv.executeSql( "create table testcatalog.fluss.pk_table_test_kv_snapshot_lease (" + "a int not null primary key not enforced, b varchar)"); @@ -814,7 +814,7 @@ void testReleaseAllKvSnapshotLeaseProcedure() throws Exception { assertThat(zkClient.getKvSnapshotLeaseMetadata(leaseId)).isPresent(); tEnv.executeSql( String.format( - "Call %s.sys.release_all_kv_snapshot_lease('" + leaseId + "' )", + "Call %s.sys.drop_kv_snapshot_lease('" + leaseId + "' )", CATALOG_NAME)) .await(); assertThat(zkClient.getKvSnapshotLeaseMetadata(leaseId)).isNotPresent(); diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceBatchITCase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceBatchITCase.java index 03f853f61a..31e6729812 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceBatchITCase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceBatchITCase.java @@ -22,6 +22,7 @@ import org.apache.fluss.client.table.writer.UpsertWriter; import org.apache.fluss.flink.utils.FlinkTestBase; import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.commons.lang3.RandomUtils; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -36,6 +37,7 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; +import java.time.Duration; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -48,6 +50,7 @@ import static org.apache.fluss.flink.source.testutils.FlinkRowAssertionsUtils.collectRowsWithTimeout; import static org.apache.fluss.server.testutils.FlussClusterExtension.BUILTIN_DATABASE; import static org.apache.fluss.testutils.DataTestUtils.row; +import static org.apache.fluss.testutils.common.CommonTestUtils.retry; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -82,6 +85,55 @@ void after() { tEnv.executeSql(String.format("drop database %s cascade", DEFAULT_DB)); } + @Test + void testPkTableBatchReadWithKvSnapshotLease() throws Exception { + // Create a PK table, write data, trigger snapshot, and verify batch read + // works correctly with KV snapshot lease. + String tableName = "pk_table_batch_kv_snapshot_lease"; + tEnv.executeSql( + String.format( + "create table %s (" + + " id int not null," + + " address varchar," + + " name varchar," + + " primary key (id) NOT ENFORCED)" + + " with ('bucket.num' = '4')", + tableName)); + + TablePath tablePath = TablePath.of(DEFAULT_DB, tableName); + + // Write data via Fluss client + try (Table table = conn.getTable(tablePath)) { + UpsertWriter upsertWriter = table.newUpsert().createWriter(); + for (int i = 1; i <= 5; i++) { + upsertWriter.upsert(row(i, "address" + i, "name" + i)); + } + upsertWriter.flush(); + } + + // Trigger and wait for snapshot to ensure data is available for batch read + FLUSS_CLUSTER_EXTENSION.triggerAndWaitSnapshot(tablePath); + + // Execute batch read via LIMIT query + String query = String.format("SELECT * FROM %s limit 5", tableName); + CloseableIterator iterRows = tEnv.executeSql(query).collect(); + List collected = collectRowsWithTimeout(iterRows, 5); + List expected = + Arrays.asList( + "+I[1, address1, name1]", + "+I[2, address2, name2]", + "+I[3, address3, name3]", + "+I[4, address4, name4]", + "+I[5, address5, name5]"); + assertThat(collected).containsExactlyInAnyOrderElementsOf(expected); + + // Verify KV snapshot lease is cleaned up after batch job finishes + ZooKeeperClient zkClient = FLUSS_CLUSTER_EXTENSION.getZooKeeperClient(); + retry( + Duration.ofMinutes(1), + () -> assertThat(zkClient.getKvSnapshotLeasesList().isEmpty()).isTrue()); + } + @Test void testScanSingleRowFilter() throws Exception { String tableName = prepareSourceTable(new String[] {"name", "id"}, null); diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java index 57beca3a8b..e1d6d9a86b 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java @@ -372,10 +372,7 @@ void testPkTableReadWithKvSnapshotLease() throws Exception { List expectedRows = Arrays.asList("+I[1, v1]", "+I[2, v2]", "+I[3, v3]"); org.apache.flink.util.CloseableIterator rowIter = - tEnv.executeSql( - "select * from pk_table_with_kv_snapshot_lease " - + "/*+ OPTIONS('scan.kv.snapshot.lease.id' = 'test-lease-10001') */") - .collect(); + tEnv.executeSql("select * from pk_table_with_kv_snapshot_lease").collect(); assertResultsIgnoreOrder(rowIter, expectedRows, false); // now, we put rows to the table again, should read the log @@ -394,9 +391,7 @@ void testPkTableReadWithKvSnapshotLease() throws Exception { ZooKeeperClient zkClient = FLUSS_CLUSTER_EXTENSION.getZooKeeperClient(); retry( Duration.ofMinutes(1), - () -> - assertThat(zkClient.getKvSnapshotLeaseMetadata("test-lease-10001")) - .isNotPresent()); + () -> assertThat(zkClient.getKvSnapshotLeasesList().isEmpty()).isTrue()); } // ------------------------------------------------------------------------------------- diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlussSourceITCase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlussSourceITCase.java index 05ded07232..9938fcce5e 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlussSourceITCase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlussSourceITCase.java @@ -45,7 +45,6 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import java.time.Duration; import java.util.Arrays; import java.util.List; import java.util.stream.Collectors; @@ -92,8 +91,6 @@ public void testTablePKSource() throws Exception { .setTable(pkTableName) .setStartingOffsets(OffsetsInitializer.earliest()) .setScanPartitionDiscoveryIntervalMs(1000L) - .setKvSnapshotLeaseId("test-lease-1vd7j9") - .setKvSnapshotLeaseDurationMs(Duration.ofDays(1).toMillis()) .setDeserializationSchema(new MockDataUtils.OrderDeserializationSchema()) .build(); @@ -126,8 +123,6 @@ public void testTablePKSourceWithProjectionPushdown() throws Exception { .setTable(pkTableName) .setStartingOffsets(OffsetsInitializer.earliest()) .setScanPartitionDiscoveryIntervalMs(1000L) - .setKvSnapshotLeaseId("test-lease-kj232df") - .setKvSnapshotLeaseDurationMs(Duration.ofDays(1).toMillis()) .setDeserializationSchema(new OrderPartialDeserializationSchema()) .setProjectedFields("orderId", "amount") .build(); @@ -156,8 +151,6 @@ public void testRowDataPKTableSource() throws Exception { .setTable(pkTableName) .setStartingOffsets(OffsetsInitializer.earliest()) .setScanPartitionDiscoveryIntervalMs(1000L) - .setKvSnapshotLeaseId("test-lease-sdafa3") - .setKvSnapshotLeaseDurationMs(Duration.ofDays(1).toMillis()) .setDeserializationSchema(new RowDataDeserializationSchema()) .build(); 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 ac0393307f..d438155782 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 @@ -237,7 +237,7 @@ void testNonPkTable() throws Throwable { streaming, null, null, - new LeaseContext(null, null)); + LeaseContext.DEFAULT); enumerator.start(); @@ -438,7 +438,7 @@ void testDiscoverPartitionsPeriodically(boolean isPrimaryKeyTable) throws Throwa isPrimaryKeyTable ? new LeaseContext( "kv_snapshot_lease1", Duration.ofDays(1).toMillis()) - : new LeaseContext(null, null))) { + : LeaseContext.DEFAULT)) { Map partitionNameByIds = waitUntilPartitions(zooKeeperClient, DEFAULT_TABLE_PATH); diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/state/SourceEnumeratorStateSerializerTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/state/SourceEnumeratorStateSerializerTest.java index 9ce6cade2c..5273d12d16 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/state/SourceEnumeratorStateSerializerTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/state/SourceEnumeratorStateSerializerTest.java @@ -28,8 +28,6 @@ import org.apache.fluss.metadata.TableBucket; import org.junit.jupiter.api.Test; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.ValueSource; import java.util.ArrayList; import java.util.Arrays; @@ -47,9 +45,8 @@ */ class SourceEnumeratorStateSerializerTest { - @ParameterizedTest - @ValueSource(booleans = {true, false}) - void testPendingSplitsCheckpointSerde(boolean isLogTable) throws Exception { + @Test + void testPendingSplitsCheckpointSerde() throws Exception { FlussSourceEnumeratorStateSerializer serializer = new FlussSourceEnumeratorStateSerializer(new TestingLakeSource()); @@ -83,16 +80,12 @@ void testPendingSplitsCheckpointSerde(boolean isLogTable) throws Exception { lakeHybridSplitBucket, "2024-01-01", lakeSplits, 300L, Long.MIN_VALUE); remainingHybridLakeFlussSplits.add(lakeHybridSplit); - // Add a LeaseContext - LeaseContext leaseContext = - isLogTable ? new LeaseContext(null, null) : new LeaseContext("leaseId", 1000L); - SourceEnumeratorState sourceEnumeratorState = new SourceEnumeratorState( assignedBuckets, assignedPartitions, remainingHybridLakeFlussSplits, - leaseContext); + "leaseId"); // serialize state with remaining hybrid lake fluss splits byte[] serialized = serializer.serialize(sourceEnumeratorState); @@ -119,7 +112,10 @@ void testV0Compatibility() throws Exception { assignedPartitions.put(2L, "partition2"); SourceEnumeratorState sourceEnumeratorState = new SourceEnumeratorState( - assignedBuckets, assignedPartitions, null, new LeaseContext(null, null)); + assignedBuckets, + assignedPartitions, + null, + LeaseContext.DEFAULT.getKvSnapshotLeaseId()); byte[] serialized = serializer.serializeV0(sourceEnumeratorState); // then deserialize @@ -139,7 +135,7 @@ void testV0Compatibility() throws Exception { assignedBuckets, assignedPartitions, remainingHybridLakeFlussSplits, - new LeaseContext(null, null)); + LeaseContext.DEFAULT.getKvSnapshotLeaseId()); serialized = serializer.serializeV0(sourceEnumeratorState); @@ -161,7 +157,10 @@ void testInconsistentLakeSourceSerde() throws Exception { assignedPartitions.put(2L, "partition2"); SourceEnumeratorState sourceEnumeratorState = new SourceEnumeratorState( - assignedBuckets, assignedPartitions, null, new LeaseContext(null, null)); + assignedBuckets, + assignedPartitions, + null, + LeaseContext.DEFAULT.getKvSnapshotLeaseId()); byte[] serialized = serializer.serialize(sourceEnumeratorState); // test deserialize with nonnull lake source diff --git a/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/AdminGateway.java b/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/AdminGateway.java index e2ec83ff71..d8ef3fde6f 100644 --- a/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/AdminGateway.java +++ b/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/AdminGateway.java @@ -41,6 +41,8 @@ import org.apache.fluss.rpc.messages.DropAclsResponse; import org.apache.fluss.rpc.messages.DropDatabaseRequest; import org.apache.fluss.rpc.messages.DropDatabaseResponse; +import org.apache.fluss.rpc.messages.DropKvSnapshotLeaseRequest; +import org.apache.fluss.rpc.messages.DropKvSnapshotLeaseResponse; import org.apache.fluss.rpc.messages.DropPartitionRequest; import org.apache.fluss.rpc.messages.DropPartitionResponse; import org.apache.fluss.rpc.messages.DropTableRequest; @@ -198,6 +200,10 @@ CompletableFuture acquireKvSnapshotLease( CompletableFuture releaseKvSnapshotLease( ReleaseKvSnapshotLeaseRequest request); + @RPC(api = ApiKeys.DROP_KV_SNAPSHOT_LEASE) + CompletableFuture dropKvSnapshotLease( + DropKvSnapshotLeaseRequest request); + // todo: rename table & alter table } diff --git a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java index c9bce10ae8..5084d73fda 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 @@ -99,7 +99,8 @@ public enum ApiKeys { GET_PRODUCER_OFFSETS(1054, 0, 0, PUBLIC), DELETE_PRODUCER_OFFSETS(1055, 0, 0, PUBLIC), ACQUIRE_KV_SNAPSHOT_LEASE(1056, 0, 0, PUBLIC), - RELEASE_KV_SNAPSHOT_LEASE(1057, 0, 0, PUBLIC); + RELEASE_KV_SNAPSHOT_LEASE(1057, 0, 0, PUBLIC), + DROP_KV_SNAPSHOT_LEASE(1058, 0, 0, PUBLIC); private static final Map ID_TO_TYPE = Arrays.stream(ApiKeys.values()) diff --git a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java index ffe99ad146..36ecd819ff 100644 --- a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java +++ b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java @@ -44,7 +44,6 @@ import org.apache.fluss.exception.InvalidTargetColumnException; import org.apache.fluss.exception.InvalidTimestampException; import org.apache.fluss.exception.InvalidUpdateVersionException; -import org.apache.fluss.exception.KvSnapshotLeaseNotExistException; import org.apache.fluss.exception.KvSnapshotNotExistException; import org.apache.fluss.exception.KvStorageException; import org.apache.fluss.exception.LakeStorageNotConfiguredException; @@ -246,9 +245,7 @@ public enum Errors { INVALID_PRODUCER_ID_EXCEPTION( 63, "The client has attempted to perform an operation with an invalid producer ID.", - InvalidProducerIdException::new), - KV_SNAPSHOT_LEASE_NOT_EXIST( - 64, "The kv snapshot lease is not exist.", KvSnapshotLeaseNotExistException::new); + InvalidProducerIdException::new); private static final Logger LOG = LoggerFactory.getLogger(Errors.class); diff --git a/fluss-rpc/src/main/proto/FlussApi.proto b/fluss-rpc/src/main/proto/FlussApi.proto index c3ad9840f6..405744eb92 100644 --- a/fluss-rpc/src/main/proto/FlussApi.proto +++ b/fluss-rpc/src/main/proto/FlussApi.proto @@ -380,17 +380,17 @@ message GetKvSnapshotMetadataResponse { message AcquireKvSnapshotLeaseRequest { required string lease_id = 1; - required int64 lease_duration = 2; - repeated PbKvSnapshotLeaseForTable table_lease_req = 3; + required int64 lease_duration_ms = 2; + repeated PbKvSnapshotLeaseForTable snapshots_to_lease = 3; } message AcquireKvSnapshotLeaseResponse { - repeated PbKvSnapshotLeaseForTable tables_lease_res = 3; + repeated PbKvSnapshotLeaseForTable unavailable_snapshots = 3; } message ReleaseKvSnapshotLeaseRequest { required string lease_id = 1; - repeated PbTable release_tables = 2; + repeated PbTableBucket buckets_to_release = 2; } message ReleaseKvSnapshotLeaseResponse {} @@ -864,16 +864,6 @@ message PbTableBucket { required int32 bucket_id = 3; } -message PbTable { - required int64 table_id = 1; - repeated PbBucket buckets = 2; -} - -message PbBucket { - optional int64 partition_id = 1; - required int32 bucket_id = 2; -} - message PbAdjustIsrReqForTable { required int64 table_id = 1; repeated PbAdjustIsrReqForBucket buckets_req = 2; @@ -1175,7 +1165,7 @@ message PbDatabaseSummary { message PbKvSnapshotLeaseForTable { required int64 table_id = 1; - repeated PbKvSnapshotLeaseForBucket buckets_req = 2; + repeated PbKvSnapshotLeaseForBucket bucket_snapshots = 2; } message PbKvSnapshotLeaseForBucket { 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 892cf845ae..0d6f9959d8 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 @@ -32,7 +32,6 @@ import org.apache.fluss.exception.IneligibleReplicaException; import org.apache.fluss.exception.InvalidCoordinatorException; import org.apache.fluss.exception.InvalidUpdateVersionException; -import org.apache.fluss.exception.KvSnapshotLeaseNotExistException; import org.apache.fluss.exception.RebalanceFailureException; import org.apache.fluss.exception.ServerNotExistException; import org.apache.fluss.exception.ServerTagAlreadyExistException; @@ -48,7 +47,6 @@ import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePartition; import org.apache.fluss.metadata.TablePath; -import org.apache.fluss.rpc.messages.AcquireKvSnapshotLeaseResponse; import org.apache.fluss.rpc.messages.AddServerTagResponse; import org.apache.fluss.rpc.messages.AdjustIsrResponse; import org.apache.fluss.rpc.messages.CancelRebalanceResponse; @@ -58,13 +56,10 @@ import org.apache.fluss.rpc.messages.ControlledShutdownResponse; import org.apache.fluss.rpc.messages.ListRebalanceProgressResponse; import org.apache.fluss.rpc.messages.PbCommitLakeTableSnapshotRespForTable; -import org.apache.fluss.rpc.messages.PbKvSnapshotLeaseForBucket; import org.apache.fluss.rpc.messages.RebalanceResponse; -import org.apache.fluss.rpc.messages.ReleaseKvSnapshotLeaseResponse; import org.apache.fluss.rpc.messages.RemoveServerTagResponse; import org.apache.fluss.rpc.protocol.ApiError; import org.apache.fluss.server.coordinator.event.AccessContextEvent; -import org.apache.fluss.server.coordinator.event.AcquireKvSnapshotLeaseEvent; import org.apache.fluss.server.coordinator.event.AddServerTagEvent; import org.apache.fluss.server.coordinator.event.AdjustIsrReceivedEvent; import org.apache.fluss.server.coordinator.event.CancelRebalanceEvent; @@ -88,12 +83,12 @@ import org.apache.fluss.server.coordinator.event.NotifyLakeTableOffsetEvent; import org.apache.fluss.server.coordinator.event.NotifyLeaderAndIsrResponseReceivedEvent; import org.apache.fluss.server.coordinator.event.RebalanceEvent; -import org.apache.fluss.server.coordinator.event.ReleaseKvSnapshotLeaseEvent; import org.apache.fluss.server.coordinator.event.RemoveServerTagEvent; import org.apache.fluss.server.coordinator.event.SchemaChangeEvent; import org.apache.fluss.server.coordinator.event.TableRegistrationChangeEvent; import org.apache.fluss.server.coordinator.event.watcher.TableChangeWatcher; import org.apache.fluss.server.coordinator.event.watcher.TabletServerChangeWatcher; +import org.apache.fluss.server.coordinator.lease.KvSnapshotLeaseManager; import org.apache.fluss.server.coordinator.rebalance.RebalanceManager; import org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElection.ControlledShutdownLeaderElection; import org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElection.ReassignmentLeaderElection; @@ -124,8 +119,6 @@ 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.server.zk.data.lease.KvSnapshotLeaseMetadataManager; -import org.apache.fluss.utils.clock.Clock; import org.apache.fluss.utils.types.Tuple2; import org.slf4j.Logger; @@ -186,8 +179,6 @@ public class CoordinatorEventProcessor implements EventProcessor { private final String internalListenerName; private final CoordinatorMetricGroup coordinatorMetricGroup; private final RebalanceManager rebalanceManager; - - private final KvSnapshotLeaseManager kvSnapshotLeaseManager; private final CompletedSnapshotStoreManager completedSnapshotStoreManager; private final LakeTableHelper lakeTableHelper; @@ -202,7 +193,7 @@ public CoordinatorEventProcessor( Configuration conf, ExecutorService ioExecutor, MetadataManager metadataManager, - Clock clock) { + KvSnapshotLeaseManager kvSnapshotLeaseManager) { this.zooKeeperClient = zooKeeperClient; this.serverMetadataCache = serverMetadataCache; this.coordinatorChannelManager = coordinatorChannelManager; @@ -241,15 +232,6 @@ public CoordinatorEventProcessor( new CoordinatorRequestBatch( coordinatorChannelManager, coordinatorEventManager, coordinatorContext); - String remoteDataDir = conf.getString(ConfigOptions.REMOTE_DATA_DIR); - this.kvSnapshotLeaseManager = - new KvSnapshotLeaseManager( - conf, - new KvSnapshotLeaseMetadataManager(zooKeeperClient, remoteDataDir), - coordinatorContext, - clock, - coordinatorMetricGroup); - this.completedSnapshotStoreManager = new CompletedSnapshotStoreManager( conf.getInt(ConfigOptions.KV_MAX_RETAINED_SNAPSHOTS), @@ -263,7 +245,8 @@ public CoordinatorEventProcessor( this.internalListenerName = conf.getString(ConfigOptions.INTERNAL_LISTENER_NAME); this.rebalanceManager = new RebalanceManager(this, zooKeeperClient); this.ioExecutor = ioExecutor; - this.lakeTableHelper = new LakeTableHelper(zooKeeperClient, remoteDataDir); + this.lakeTableHelper = + new LakeTableHelper(zooKeeperClient, conf.getString(ConfigOptions.REMOTE_DATA_DIR)); } public CoordinatorEventManager getCoordinatorEventManager() { @@ -312,9 +295,6 @@ public void startup() { // start rebalance manager. rebalanceManager.startup(); - - // start kv snapshot lease manager - kvSnapshotLeaseManager.start(); } public void shutdown() { @@ -463,9 +443,6 @@ private void initCoordinatorContext() throws Exception { "Load table and partition assignment success in {}ms when initializing coordinator context.", System.currentTimeMillis() - start4loadAssignment); - // load all kv snapshot lease from zookeeper when starting. - kvSnapshotLeaseManager.initialize(); - long end = System.currentTimeMillis(); LOG.info("Current total {} tables in the cluster.", coordinatorContext.allTables().size()); LOG.info( @@ -659,18 +636,6 @@ public void process(CoordinatorEvent event) { completeFromCallable( listRebalanceProgressEvent.getRespCallback(), () -> processListRebalanceProgress(listRebalanceProgressEvent)); - } else if (event instanceof AcquireKvSnapshotLeaseEvent) { - AcquireKvSnapshotLeaseEvent acquireKvSnapshotLeaseEvent = - (AcquireKvSnapshotLeaseEvent) event; - completeFromCallable( - acquireKvSnapshotLeaseEvent.getRespCallback(), - () -> tryProcessAcquireKvSnapshotLease(acquireKvSnapshotLeaseEvent)); - } else if (event instanceof ReleaseKvSnapshotLeaseEvent) { - ReleaseKvSnapshotLeaseEvent releaseKvSnapshotLeaseEvent = - (ReleaseKvSnapshotLeaseEvent) event; - completeFromCallable( - releaseKvSnapshotLeaseEvent.getRespCallback(), - () -> tryProcessReleaseKvSnapshotLease(releaseKvSnapshotLeaseEvent)); } else if (event instanceof AccessContextEvent) { AccessContextEvent accessContextEvent = (AccessContextEvent) event; processAccessContext(accessContextEvent); @@ -2073,53 +2038,6 @@ private ControlledShutdownResponse tryProcessControlledShutdown( return response; } - private AcquireKvSnapshotLeaseResponse tryProcessAcquireKvSnapshotLease( - AcquireKvSnapshotLeaseEvent event) throws Exception { - AcquireKvSnapshotLeaseResponse response = new AcquireKvSnapshotLeaseResponse(); - Map unavailableSnapshots = - kvSnapshotLeaseManager.acquireLease( - event.getLeaseId(), - event.getLeaseDuration(), - event.getTableIdToLeasedBucket()); - - Map> pbFailedTables = new HashMap<>(); - for (Map.Entry entry : unavailableSnapshots.entrySet()) { - TableBucket tb = entry.getKey(); - Long snapshotId = entry.getValue(); - PbKvSnapshotLeaseForBucket pbBucket = - new PbKvSnapshotLeaseForBucket().setBucketId(tb.getBucket()); - if (tb.getPartitionId() != null) { - pbBucket.setPartitionId(tb.getPartitionId()); - } - pbBucket.setSnapshotId(snapshotId); - pbFailedTables.computeIfAbsent(tb.getTableId(), k -> new ArrayList<>()).add(pbBucket); - } - - for (Map.Entry> entry : pbFailedTables.entrySet()) { - response.addTablesLeaseRe() - .setTableId(entry.getKey()) - .addAllBucketsReqs(entry.getValue()); - } - return response; - } - - private ReleaseKvSnapshotLeaseResponse tryProcessReleaseKvSnapshotLease( - ReleaseKvSnapshotLeaseEvent event) throws Exception { - ReleaseKvSnapshotLeaseResponse response = new ReleaseKvSnapshotLeaseResponse(); - Map> tableIdToReleasedBucket = event.getTableIdToReleasedBucket(); - if (tableIdToReleasedBucket.isEmpty()) { - // release all - boolean exist = kvSnapshotLeaseManager.releaseAll(event.getLeaseId()); - if (!exist) { - throw new KvSnapshotLeaseNotExistException( - "kv snapshot lease '" + event.getLeaseId() + "' not exits."); - } - } else { - kvSnapshotLeaseManager.release(event.getLeaseId(), tableIdToReleasedBucket); - } - return response; - } - private void validateFencedEvent(FencedCoordinatorEvent event) { TableBucket tb = event.getTableBucket(); if (coordinatorContext.getTablePathById(tb.getTableId()) == null) { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java index 9ae3d6139b..fa927be836 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java @@ -33,6 +33,7 @@ import org.apache.fluss.server.ServerBase; import org.apache.fluss.server.authorizer.Authorizer; import org.apache.fluss.server.authorizer.AuthorizerLoader; +import org.apache.fluss.server.coordinator.lease.KvSnapshotLeaseManager; import org.apache.fluss.server.coordinator.rebalance.RebalanceManager; import org.apache.fluss.server.metadata.CoordinatorMetadataCache; import org.apache.fluss.server.metadata.ServerMetadataCache; @@ -144,6 +145,9 @@ public class CoordinatorServer extends ServerBase { @GuardedBy("lock") private LakeCatalogDynamicLoader lakeCatalogDynamicLoader; + @GuardedBy("lock") + private KvSnapshotLeaseManager kvSnapshotLeaseManager; + public CoordinatorServer(Configuration conf) { this(conf, SystemClock.getInstance()); } @@ -204,6 +208,18 @@ protected void startServices() throws Exception { Executors.newFixedThreadPool( conf.get(ConfigOptions.SERVER_IO_POOL_SIZE), new ExecutorThreadFactory("coordinator-io")); + + // Initialize and start the kv snapshot lease manager + this.kvSnapshotLeaseManager = + new KvSnapshotLeaseManager( + conf.get(ConfigOptions.KV_SNAPSHOT_LEASE_EXPIRATION_CHECK_INTERVAL) + .toMillis(), + zkClient, + conf.getString(ConfigOptions.REMOTE_DATA_DIR), + clock, + serverMetricGroup); + kvSnapshotLeaseManager.start(); + this.coordinatorService = new CoordinatorService( conf, @@ -216,7 +232,8 @@ protected void startServices() throws Exception { lakeCatalogDynamicLoader, lakeTableTieringManager, dynamicConfigManager, - ioExecutor); + ioExecutor, + kvSnapshotLeaseManager); this.rpcServer = RpcServer.create( @@ -259,7 +276,7 @@ protected void startServices() throws Exception { conf, ioExecutor, metadataManager, - clock); + kvSnapshotLeaseManager); coordinatorEventProcessor.startup(); createDefaultDatabase(); 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 631befb8c3..6efa3672cc 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 @@ -86,6 +86,8 @@ import org.apache.fluss.rpc.messages.DropAclsResponse; import org.apache.fluss.rpc.messages.DropDatabaseRequest; import org.apache.fluss.rpc.messages.DropDatabaseResponse; +import org.apache.fluss.rpc.messages.DropKvSnapshotLeaseRequest; +import org.apache.fluss.rpc.messages.DropKvSnapshotLeaseResponse; import org.apache.fluss.rpc.messages.DropPartitionRequest; import org.apache.fluss.rpc.messages.DropPartitionResponse; import org.apache.fluss.rpc.messages.DropTableRequest; @@ -101,8 +103,10 @@ import org.apache.fluss.rpc.messages.PbAlterConfig; import org.apache.fluss.rpc.messages.PbHeartbeatReqForTable; import org.apache.fluss.rpc.messages.PbHeartbeatRespForTable; +import org.apache.fluss.rpc.messages.PbKvSnapshotLeaseForTable; import org.apache.fluss.rpc.messages.PbPrepareLakeTableRespForTable; import org.apache.fluss.rpc.messages.PbProducerTableOffsets; +import org.apache.fluss.rpc.messages.PbTableBucket; import org.apache.fluss.rpc.messages.PbTableOffsets; import org.apache.fluss.rpc.messages.PrepareLakeTableSnapshotRequest; import org.apache.fluss.rpc.messages.PrepareLakeTableSnapshotResponse; @@ -128,7 +132,6 @@ import org.apache.fluss.server.authorizer.AclDeleteResult; import org.apache.fluss.server.authorizer.Authorizer; import org.apache.fluss.server.coordinator.event.AccessContextEvent; -import org.apache.fluss.server.coordinator.event.AcquireKvSnapshotLeaseEvent; import org.apache.fluss.server.coordinator.event.AddServerTagEvent; import org.apache.fluss.server.coordinator.event.AdjustIsrReceivedEvent; import org.apache.fluss.server.coordinator.event.CancelRebalanceEvent; @@ -139,8 +142,8 @@ import org.apache.fluss.server.coordinator.event.EventManager; import org.apache.fluss.server.coordinator.event.ListRebalanceProgressEvent; import org.apache.fluss.server.coordinator.event.RebalanceEvent; -import org.apache.fluss.server.coordinator.event.ReleaseKvSnapshotLeaseEvent; import org.apache.fluss.server.coordinator.event.RemoveServerTagEvent; +import org.apache.fluss.server.coordinator.lease.KvSnapshotLeaseManager; import org.apache.fluss.server.coordinator.producer.ProducerOffsetsManager; import org.apache.fluss.server.coordinator.rebalance.goal.Goal; import org.apache.fluss.server.entity.CommitKvSnapshotData; @@ -196,6 +199,7 @@ import static org.apache.fluss.server.utils.ServerRpcMessageUtils.getPartitionSpec; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.getReleaseKvSnapshotLeaseData; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.groupOffsetsByTableId; +import static org.apache.fluss.server.utils.ServerRpcMessageUtils.makeAcquireKvSnapshotLeaseResponse; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.makeCreateAclsResponse; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.makeDropAclsResponse; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.toAlterTableConfigChanges; @@ -224,6 +228,7 @@ public final class CoordinatorService extends RpcServiceBase implements Coordina private final ExecutorService ioExecutor; private final LakeTableHelper lakeTableHelper; private final ProducerOffsetsManager producerOffsetsManager; + private final KvSnapshotLeaseManager kvSnapshotLeaseManager; public CoordinatorService( Configuration conf, @@ -236,7 +241,8 @@ public CoordinatorService( LakeCatalogDynamicLoader lakeCatalogDynamicLoader, LakeTableTieringManager lakeTableTieringManager, DynamicConfigManager dynamicConfigManager, - ExecutorService ioExecutor) { + ExecutorService ioExecutor, + KvSnapshotLeaseManager kvSnapshotLeaseManager) { super( remoteFileSystem, ServerType.COORDINATOR, @@ -263,6 +269,8 @@ public CoordinatorService( // Initialize and start the producer snapshot manager this.producerOffsetsManager = new ProducerOffsetsManager(conf, zkClient); this.producerOffsetsManager.start(); + + this.kvSnapshotLeaseManager = kvSnapshotLeaseManager; } @Override @@ -867,30 +875,90 @@ public CompletableFuture controlledShutdown( @Override public CompletableFuture acquireKvSnapshotLease( AcquireKvSnapshotLeaseRequest request) { - CompletableFuture response = new CompletableFuture<>(); - eventManagerSupplier - .get() - .put( - new AcquireKvSnapshotLeaseEvent( - request.getLeaseId(), - request.getLeaseDuration(), - getAcquireKvSnapshotLeaseData(request), - response)); - return response; + // Authorization: require WRITE permission on all tables in the request + if (authorizer != null) { + for (PbKvSnapshotLeaseForTable kvSnapshotLeaseForTable : + request.getSnapshotsToLeasesList()) { + long tableId = kvSnapshotLeaseForTable.getTableId(); + authorizeTable(OperationType.READ, tableId); + } + } + + String leaseId = request.getLeaseId(); + return CompletableFuture.supplyAsync( + () -> { + try { + return makeAcquireKvSnapshotLeaseResponse( + kvSnapshotLeaseManager.acquireLease( + leaseId, + request.getLeaseDurationMs(), + getAcquireKvSnapshotLeaseData(request))); + } catch (ApiException e) { + // Re-throw ApiExceptions as-is to preserve exception type for client + throw e; + } catch (Exception e) { + throw new UnknownServerException( + "Failed to acquire kv snapshot lease for" + leaseId, e); + } + }, + ioExecutor); } @Override public CompletableFuture releaseKvSnapshotLease( ReleaseKvSnapshotLeaseRequest request) { - CompletableFuture response = new CompletableFuture<>(); - eventManagerSupplier - .get() - .put( - new ReleaseKvSnapshotLeaseEvent( - request.getLeaseId(), - getReleaseKvSnapshotLeaseData(request), - response)); - return response; + // Authorization: require WRITE permission on all tables in the request + if (authorizer != null) { + for (PbTableBucket tableBucket : request.getBucketsToReleasesList()) { + long tableId = tableBucket.getTableId(); + authorizeTable(OperationType.READ, tableId); + } + } + + String leaseId = request.getLeaseId(); + return CompletableFuture.supplyAsync( + () -> { + try { + ReleaseKvSnapshotLeaseResponse response = + new ReleaseKvSnapshotLeaseResponse(); + kvSnapshotLeaseManager.release( + leaseId, getReleaseKvSnapshotLeaseData(request)); + return response; + } catch (ApiException e) { + // Re-throw ApiExceptions as-is to preserve exception type for client + throw e; + } catch (Exception e) { + throw new UnknownServerException( + "Failed to release kv snapshot lease for" + leaseId, e); + } + }, + ioExecutor); + } + + @Override + public CompletableFuture dropKvSnapshotLease( + DropKvSnapshotLeaseRequest request) { + // Authorization: require WRITE permission on the cluster + if (authorizer != null) { + authorizer.authorize(currentSession(), OperationType.WRITE, Resource.cluster()); + } + + String leaseId = request.getLeaseId(); + return CompletableFuture.supplyAsync( + () -> { + try { + DropKvSnapshotLeaseResponse response = new DropKvSnapshotLeaseResponse(); + kvSnapshotLeaseManager.dropLease(leaseId); + return response; + } catch (ApiException e) { + // Re-throw ApiExceptions as-is to preserve exception type for client + throw e; + } catch (Exception e) { + throw new UnknownServerException( + "Failed to drop kv snapshot lease for" + leaseId, e); + } + }, + ioExecutor); } @Override diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/AcquireKvSnapshotLeaseEvent.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/AcquireKvSnapshotLeaseEvent.java deleted file mode 100644 index 36ef0f56d2..0000000000 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/AcquireKvSnapshotLeaseEvent.java +++ /dev/null @@ -1,60 +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.coordinator.event; - -import org.apache.fluss.metadata.KvSnapshotLeaseForBucket; -import org.apache.fluss.rpc.messages.AcquireKvSnapshotLeaseResponse; - -import java.util.List; -import java.util.Map; -import java.util.concurrent.CompletableFuture; - -/** An event for acquire a kv snapshot lease. */ -public class AcquireKvSnapshotLeaseEvent implements CoordinatorEvent { - private final String leaseId; - private final long leaseDuration; - private final Map> tableIdToLeasedBucket; - private final CompletableFuture respCallback; - - public AcquireKvSnapshotLeaseEvent( - String leaseId, - long leaseDuration, - Map> tableIdToLeasedBucket, - CompletableFuture respCallback) { - this.leaseId = leaseId; - this.leaseDuration = leaseDuration; - this.tableIdToLeasedBucket = tableIdToLeasedBucket; - this.respCallback = respCallback; - } - - public String getLeaseId() { - return leaseId; - } - - public long getLeaseDuration() { - return leaseDuration; - } - - public Map> getTableIdToLeasedBucket() { - return tableIdToLeasedBucket; - } - - public CompletableFuture getRespCallback() { - return respCallback; - } -} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/ReleaseKvSnapshotLeaseEvent.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/ReleaseKvSnapshotLeaseEvent.java deleted file mode 100644 index 9165cd419e..0000000000 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/ReleaseKvSnapshotLeaseEvent.java +++ /dev/null @@ -1,53 +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.coordinator.event; - -import org.apache.fluss.metadata.TableBucket; -import org.apache.fluss.rpc.messages.ReleaseKvSnapshotLeaseResponse; - -import java.util.List; -import java.util.Map; -import java.util.concurrent.CompletableFuture; - -/** An event for release a kv snapshot lease for table buckets. */ -public class ReleaseKvSnapshotLeaseEvent implements CoordinatorEvent { - private final String leaseId; - private final Map> tableIdToReleasedBucket; - private final CompletableFuture respCallback; - - public ReleaseKvSnapshotLeaseEvent( - String leaseId, - Map> tableIdToReleasedBucket, - CompletableFuture respCallback) { - this.leaseId = leaseId; - this.tableIdToReleasedBucket = tableIdToReleasedBucket; - this.respCallback = respCallback; - } - - public String getLeaseId() { - return leaseId; - } - - public Map> getTableIdToReleasedBucket() { - return tableIdToReleasedBucket; - } - - public CompletableFuture getRespCallback() { - return respCallback; - } -} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLease.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/lease/KvSnapshotLeaseHandler.java similarity index 70% rename from fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLease.java rename to fluss-server/src/main/java/org/apache/fluss/server/coordinator/lease/KvSnapshotLeaseHandler.java index 893fe8b530..c77666a615 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLease.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/lease/KvSnapshotLeaseHandler.java @@ -15,9 +15,10 @@ * limitations under the License. */ -package org.apache.fluss.server.zk.data.lease; +package org.apache.fluss.server.coordinator.lease; import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.server.zk.data.lease.KvSnapshotTableLease; import org.apache.fluss.utils.MapUtils; import javax.annotation.concurrent.NotThreadSafe; @@ -26,19 +27,19 @@ import java.util.Map; import java.util.Objects; -/** The entity of kv snapshot lease. */ +/** handler of kv snapshot lease. */ @NotThreadSafe -public class KvSnapshotLease { +public class KvSnapshotLeaseHandler { private long expirationTime; /** A map from table id to kv snapshot lease for one table. */ private final Map tableIdToTableLease; - public KvSnapshotLease(long expirationTime) { + public KvSnapshotLeaseHandler(long expirationTime) { this(expirationTime, MapUtils.newConcurrentHashMap()); } - public KvSnapshotLease( + public KvSnapshotLeaseHandler( long expirationTime, Map tableIdToTableLease) { this.expirationTime = expirationTime; this.tableIdToTableLease = tableIdToTableLease; @@ -57,14 +58,15 @@ public Map getTableIdToTableLease() { } /** - * Acquire a bucket to the lease id. + * Acquire a bucket to the lease id. If the bucket array already exists but is too small to + * accommodate the given bucket id, the array will be dynamically expanded to {@code bucketId + + * 1}. * * @param tableBucket table bucket * @param snapshotId snapshot id - * @param bucketNum bucket number of this table or partition * @return the original registered snapshotId. if -1 means the bucket is new registered */ - public long acquireBucket(TableBucket tableBucket, long snapshotId, int bucketNum) { + public long acquireBucket(TableBucket tableBucket, long snapshotId) { Long[] bucketSnapshot; Long partitionId = tableBucket.getPartitionId(); long tableId = tableBucket.getTableId(); @@ -75,11 +77,17 @@ public long acquireBucket(TableBucket tableBucket, long snapshotId, int bucketNu tableIdToTableLease.computeIfAbsent( tableId, k -> { - Long[] array = new Long[bucketNum]; + Long[] array = new Long[bucketId + 1]; Arrays.fill(array, -1L); return new KvSnapshotTableLease(tableId, array); }); bucketSnapshot = tableLease.getBucketSnapshots(); + // Dynamically expand the array if the bucket id exceeds the current array size. + // This can happen when new buckets are added to an existing table. + if (bucketSnapshot != null && bucketId >= bucketSnapshot.length) { + bucketSnapshot = expandArray(bucketSnapshot, bucketId + 1); + tableLease.setBucketSnapshots(bucketSnapshot); + } } else { // For partitioned table. @@ -93,16 +101,23 @@ public long acquireBucket(TableBucket tableBucket, long snapshotId, int bucketNu partitionSnapshots.computeIfAbsent( partitionId, k -> { - Long[] array = new Long[bucketNum]; + Long[] array = new Long[bucketId + 1]; Arrays.fill(array, -1L); return array; }); + // Dynamically expand the array if the bucket id exceeds the current array size. + if (bucketId >= bucketSnapshot.length) { + bucketSnapshot = expandArray(bucketSnapshot, bucketId + 1); + partitionSnapshots.put(partitionId, bucketSnapshot); + } } - if (bucketSnapshot == null || bucketSnapshot.length != bucketNum) { + if (bucketSnapshot == null) { throw new IllegalArgumentException( - "Bucket index is null, or input bucket number is not equal to the bucket number of the table."); + "Bucket snapshot array is null. This may indicate a conflict between " + + "partitioned and non-partitioned usage for the same table ID."); } + long originalSnapshotId = bucketSnapshot[bucketId]; bucketSnapshot[bucketId] = snapshotId; return originalSnapshotId; @@ -112,7 +127,8 @@ public long acquireBucket(TableBucket tableBucket, long snapshotId, int bucketNu * Release a bucket from the lease id. * * @param tableBucket table bucket - * @return the snapshot id of the unregistered bucket + * @return the snapshot id of the unregistered bucket, or -1 if the bucket was never registered + * or the bucket id exceeds the current array size */ public long releaseBucket(TableBucket tableBucket) { Long[] bucketIndex; @@ -130,6 +146,12 @@ public long releaseBucket(TableBucket tableBucket) { Long snapshotId = -1L; if (bucketIndex != null) { + // The bucket id exceeds the current array size, meaning it was never registered + // under this lease. Return -1 directly. + if (bucketId >= bucketIndex.length) { + return -1L; + } + snapshotId = bucketIndex[bucketId]; bucketIndex[bucketId] = -1L; @@ -160,6 +182,19 @@ public boolean isEmpty() { return tableIdToTableLease.isEmpty(); } + /** + * Expand the given array to the specified new size, filling new slots with -1L. + * + * @param original the original array + * @param newSize the desired new size (must be greater than original.length) + * @return a new expanded array with original values preserved + */ + private Long[] expandArray(Long[] original, int newSize) { + Long[] expanded = Arrays.copyOf(original, newSize); + Arrays.fill(expanded, original.length, newSize, -1L); + return expanded; + } + public int getLeasedSnapshotCount() { int count = 0; for (KvSnapshotTableLease tableLease : tableIdToTableLease.values()) { @@ -183,10 +218,10 @@ public boolean equals(Object o) { if (this == o) { return true; } - if (!(o instanceof KvSnapshotLease)) { + if (!(o instanceof KvSnapshotLeaseHandler)) { return false; } - KvSnapshotLease that = (KvSnapshotLease) o; + KvSnapshotLeaseHandler that = (KvSnapshotLeaseHandler) o; return expirationTime == that.expirationTime && Objects.equals(tableIdToTableLease, that.tableIdToTableLease); } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/KvSnapshotLeaseManager.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/lease/KvSnapshotLeaseManager.java similarity index 56% rename from fluss-server/src/main/java/org/apache/fluss/server/coordinator/KvSnapshotLeaseManager.java rename to fluss-server/src/main/java/org/apache/fluss/server/coordinator/lease/KvSnapshotLeaseManager.java index 60e47d5337..73dd08086f 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/KvSnapshotLeaseManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/lease/KvSnapshotLeaseManager.java @@ -15,18 +15,14 @@ * limitations under the License. */ -package org.apache.fluss.server.coordinator; +package org.apache.fluss.server.coordinator.lease; import org.apache.fluss.annotation.VisibleForTesting; -import org.apache.fluss.config.ConfigOptions; -import org.apache.fluss.config.Configuration; -import org.apache.fluss.metadata.KvSnapshotLeaseForBucket; import org.apache.fluss.metadata.TableBucket; -import org.apache.fluss.metadata.TableInfo; +import org.apache.fluss.metadata.TableBucketSnapshot; import org.apache.fluss.metrics.MetricNames; import org.apache.fluss.server.metrics.group.CoordinatorMetricGroup; -import org.apache.fluss.server.zk.data.lease.KvSnapshotLease; -import org.apache.fluss.server.zk.data.lease.KvSnapshotLeaseMetadataManager; +import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.server.zk.data.lease.KvSnapshotTableLease; import org.apache.fluss.utils.MapUtils; import org.apache.fluss.utils.clock.Clock; @@ -38,10 +34,12 @@ import javax.annotation.concurrent.GuardedBy; import javax.annotation.concurrent.ThreadSafe; +import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -59,38 +57,38 @@ public class KvSnapshotLeaseManager { private static final Logger LOG = LoggerFactory.getLogger(KvSnapshotLeaseManager.class); private final KvSnapshotLeaseMetadataManager metadataManager; - private final CoordinatorContext coordinatorContext; private final Clock clock; private final ScheduledExecutorService scheduledExecutor; - private final Configuration conf; + private final long leaseExpirationCheckInterval; - private final Map leaseLocks = MapUtils.newConcurrentHashMap(); - /** lease id to kv snapshot lease. */ - @GuardedBy("leaseLocks") - private final Map kvSnapshotLeaseMap; + private final ReadWriteLock managerLock = new ReentrantReadWriteLock(); - private final ReadWriteLock refCountLock = new ReentrantReadWriteLock(); + /** lease id to kv snapshot lease. */ + @GuardedBy("managerLock") + private final ConcurrentHashMap kvSnapshotLeaseMap = + MapUtils.newConcurrentHashMap(); /** * KvSnapshotLeaseForBucket to the ref count, which means this table bucket + snapshotId has * been leased by how many lease id. */ - private final Map refCount = + @GuardedBy("managerLock") + private final Map refCount = MapUtils.newConcurrentHashMap(); /** For metrics. */ private final AtomicInteger leasedBucketCount = new AtomicInteger(0); public KvSnapshotLeaseManager( - Configuration conf, - KvSnapshotLeaseMetadataManager metadataManager, - CoordinatorContext coordinatorContext, + long leaseExpirationCheckInterval, + ZooKeeperClient zkClient, + String remoteDataDir, Clock clock, CoordinatorMetricGroup coordinatorMetricGroup) { this( - conf, - metadataManager, - coordinatorContext, + leaseExpirationCheckInterval, + zkClient, + remoteDataDir, Executors.newScheduledThreadPool( 1, new ExecutorThreadFactory("kv-snapshot-lease-cleaner")), clock, @@ -99,51 +97,57 @@ public KvSnapshotLeaseManager( @VisibleForTesting public KvSnapshotLeaseManager( - Configuration conf, - KvSnapshotLeaseMetadataManager metadataManager, - CoordinatorContext coordinatorContext, + long leaseExpirationCheckInterval, + ZooKeeperClient zkClient, + String remoteDataDir, ScheduledExecutorService scheduledExecutor, Clock clock, CoordinatorMetricGroup coordinatorMetricGroup) { - this.metadataManager = metadataManager; - this.conf = conf; + this.metadataManager = new KvSnapshotLeaseMetadataManager(zkClient, remoteDataDir); + this.leaseExpirationCheckInterval = leaseExpirationCheckInterval; this.scheduledExecutor = scheduledExecutor; - this.coordinatorContext = coordinatorContext; this.clock = clock; - this.kvSnapshotLeaseMap = MapUtils.newConcurrentHashMap(); registerMetrics(coordinatorMetricGroup); } public void start() { LOG.info("kv snapshot lease manager has been started."); - scheduledExecutor.scheduleWithFixedDelay( - this::expireLeases, - 0L, - conf.get(ConfigOptions.KV_SNAPSHOT_LEASE_EXPIRATION_CHECK_INTERVAL).toMillis(), - TimeUnit.MILLISECONDS); - } - public void initialize() throws Exception { - for (String leaseId : metadataManager.getLeasesList()) { - Optional kvSnapshotLeaseOpt = metadataManager.getLease(leaseId); + List leasesList = new ArrayList<>(); + try { + leasesList = metadataManager.getLeasesList(); + } catch (Exception e) { + LOG.error("Failed to get leases list from zookeeper.", e); + } + + for (String leaseId : leasesList) { + Optional kvSnapshotLeaseOpt = Optional.empty(); + try { + kvSnapshotLeaseOpt = metadataManager.getLease(leaseId); + } catch (Exception e) { + LOG.error("Failed to get kv snapshot lease from zookeeper.", e); + } + if (kvSnapshotLeaseOpt.isPresent()) { - KvSnapshotLease kvSnapshotLease = kvSnapshotLeaseOpt.get(); - this.leaseLocks.put(leaseId, new ReentrantReadWriteLock()); - this.kvSnapshotLeaseMap.put(leaseId, kvSnapshotLease); + KvSnapshotLeaseHandler kvSnapshotLeasehandle = kvSnapshotLeaseOpt.get(); + this.kvSnapshotLeaseMap.put(leaseId, kvSnapshotLeasehandle); - initializeRefCount(kvSnapshotLease); + initializeRefCount(kvSnapshotLeasehandle); - leasedBucketCount.addAndGet(kvSnapshotLease.getLeasedSnapshotCount()); + leasedBucketCount.addAndGet(kvSnapshotLeasehandle.getLeasedSnapshotCount()); } } + + scheduledExecutor.scheduleWithFixedDelay( + this::expireLeases, 0L, leaseExpirationCheckInterval, TimeUnit.MILLISECONDS); } - public boolean snapshotLeaseNotExist(KvSnapshotLeaseForBucket kvSnapshotLeaseForBucket) { + public boolean snapshotLeaseNotExist(TableBucketSnapshot tableBucketSnapshot) { return inReadLock( - refCountLock, + managerLock, () -> { - AtomicInteger count = refCount.get(kvSnapshotLeaseForBucket); + AtomicInteger count = refCount.get(tableBucketSnapshot); return count == null || count.get() <= 0; }); } @@ -159,11 +163,10 @@ public boolean snapshotLeaseNotExist(KvSnapshotLeaseForBucket kvSnapshotLeaseFor public Map acquireLease( String leaseId, long leaseDuration, - Map> tableIdToLeaseBucket) + Map> tableIdToLeaseBucket) throws Exception { - ReadWriteLock lock = leaseLocks.computeIfAbsent(leaseId, k -> new ReentrantReadWriteLock()); return inWriteLock( - lock, + managerLock, () -> { // To record the unavailable snapshots such as the kv snapshotId to lease not // exists. @@ -172,7 +175,7 @@ public Map acquireLease( boolean update = kvSnapshotLeaseMap.containsKey(leaseId); // set the expiration time as: current time + leaseDuration long newExpirationTime = clock.milliseconds() + leaseDuration; - KvSnapshotLease kvSnapshotLease = + KvSnapshotLeaseHandler kvSnapshotLeaseHandle = kvSnapshotLeaseMap.compute( leaseId, (key, existingLease) -> { @@ -182,94 +185,71 @@ public Map acquireLease( + "time is {}", leaseId, newExpirationTime); - return new KvSnapshotLease(newExpirationTime); + return new KvSnapshotLeaseHandler(newExpirationTime); } else { existingLease.setExpirationTime(newExpirationTime); return existingLease; } }); - for (Map.Entry> entry : + for (Map.Entry> entry : tableIdToLeaseBucket.entrySet()) { - Long tableId = entry.getKey(); - TableInfo tableInfo = coordinatorContext.getTableInfoById(tableId); - int numBuckets = tableInfo.getNumBuckets(); - List buckets = entry.getValue(); - for (KvSnapshotLeaseForBucket bucket : buckets) { - + List buckets = entry.getValue(); + for (TableBucketSnapshot bucket : buckets) { TableBucket tableBucket = bucket.getTableBucket(); - long kvSnapshotId = bucket.getKvSnapshotId(); - try { - boolean snapshotExists = - metadataManager.isSnapshotExists(tableBucket, kvSnapshotId); - if (!snapshotExists) { - unavailableSnapshots.put(tableBucket, kvSnapshotId); - continue; - } - } catch (Exception e) { - LOG.error( - "Failed to check snapshotExists for tableBucket when acquire kv " - + "snapshot kvSnapshotLease {}.", - tableBucket, - e); - unavailableSnapshots.put(tableBucket, kvSnapshotId); - continue; - } + long kvSnapshotId = bucket.getSnapshotId(); + // TODO Check whether this snapshot exists, if not add it to the + // unavailable snapshots. Trace by: + // https://github.com/apache/fluss/issues/2600 long originalSnapshotId = - kvSnapshotLease.acquireBucket( - tableBucket, kvSnapshotId, numBuckets); + kvSnapshotLeaseHandle.acquireBucket(tableBucket, kvSnapshotId); if (originalSnapshotId == -1L) { leasedBucketCount.incrementAndGet(); } else { // clear the original ref. decrementRefCount( - new KvSnapshotLeaseForBucket( - tableBucket, originalSnapshotId)); + new TableBucketSnapshot(tableBucket, originalSnapshotId)); } incrementRefCount(bucket); } } if (update) { - metadataManager.updateLease(leaseId, kvSnapshotLease); + metadataManager.updateLease(leaseId, kvSnapshotLeaseHandle); } else { - metadataManager.registerLease(leaseId, kvSnapshotLease); + metadataManager.registerLease(leaseId, kvSnapshotLeaseHandle); } return unavailableSnapshots; }); } - public void release(String leaseId, Map> tableIdToUnregisterBucket) - throws Exception { - ReadWriteLock lock = leaseLocks.get(leaseId); - if (lock == null) { - return; - } - + /** + * Release kv snapshot lease. + * + * @param leaseId the lease id + * @param tableBucketsToRelease the table buckets to release + */ + public void release(String leaseId, List tableBucketsToRelease) throws Exception { inWriteLock( - lock, + managerLock, () -> { - KvSnapshotLease lease = kvSnapshotLeaseMap.get(leaseId); + KvSnapshotLeaseHandler lease = kvSnapshotLeaseMap.get(leaseId); if (lease == null) { return; } - for (Map.Entry> entry : - tableIdToUnregisterBucket.entrySet()) { - List buckets = entry.getValue(); - for (TableBucket bucket : buckets) { - long snapshotId = lease.releaseBucket(bucket); - if (snapshotId != -1L) { - leasedBucketCount.decrementAndGet(); - decrementRefCount(new KvSnapshotLeaseForBucket(bucket, snapshotId)); - } + for (TableBucket bucket : tableBucketsToRelease) { + long snapshotId = lease.releaseBucket(bucket); + if (snapshotId != -1L) { + leasedBucketCount.decrementAndGet(); + decrementRefCount(new TableBucketSnapshot(bucket, snapshotId)); } } if (lease.isEmpty()) { - releaseAll(leaseId); + dropLease(leaseId); } else { metadataManager.updateLease(leaseId, lease); } @@ -277,41 +257,30 @@ public void release(String leaseId, Map> tableIdToUnregi } /** - * Release kv snapshot lease. + * Drop kv snapshot lease. * * @param leaseId the lease id * @return true if clear success, false if lease not exist */ - public boolean releaseAll(String leaseId) throws Exception { - ReadWriteLock lock = leaseLocks.get(leaseId); - if (lock == null) { - return false; - } - - boolean exist = - inWriteLock( - lock, - () -> { - KvSnapshotLease kvSnapshotLease = kvSnapshotLeaseMap.remove(leaseId); - if (kvSnapshotLease == null) { - return false; - } - - clearRefCount(kvSnapshotLease); - metadataManager.deleteLease(leaseId); + public boolean dropLease(String leaseId) throws Exception { + return inWriteLock( + managerLock, + () -> { + KvSnapshotLeaseHandler kvSnapshotLeasehandle = + kvSnapshotLeaseMap.remove(leaseId); + if (kvSnapshotLeasehandle == null) { + return false; + } - LOG.info( - "kv snapshots of lease '" - + leaseId - + "' has been all released."); - return true; - }); + clearRefCount(kvSnapshotLeasehandle); + metadataManager.deleteLease(leaseId); - leaseLocks.remove(leaseId); - return exist; + LOG.info("kv snapshots of lease '{}' has been all released.", leaseId); + return true; + }); } - private void initializeRefCount(KvSnapshotLease lease) { + private void initializeRefCount(KvSnapshotLeaseHandler lease) { for (Map.Entry tableEntry : lease.getTableIdToTableLease().entrySet()) { long tableId = tableEntry.getKey(); @@ -324,8 +293,7 @@ private void initializeRefCount(KvSnapshotLease lease) { } incrementRefCount( - new KvSnapshotLeaseForBucket( - new TableBucket(tableId, i), snapshots[i])); + new TableBucketSnapshot(new TableBucket(tableId, i), snapshots[i])); } } else { Map partitionSnapshots = tableLease.getPartitionSnapshots(); @@ -338,7 +306,7 @@ private void initializeRefCount(KvSnapshotLease lease) { } incrementRefCount( - new KvSnapshotLeaseForBucket( + new TableBucketSnapshot( new TableBucket(tableId, partitionId, i), snapshots[i])); } } @@ -346,7 +314,7 @@ private void initializeRefCount(KvSnapshotLease lease) { } } - private void clearRefCount(KvSnapshotLease lease) { + private void clearRefCount(KvSnapshotLeaseHandler lease) { for (Map.Entry tableEntry : lease.getTableIdToTableLease().entrySet()) { long tableId = tableEntry.getKey(); @@ -358,8 +326,7 @@ private void clearRefCount(KvSnapshotLease lease) { continue; } decrementRefCount( - new KvSnapshotLeaseForBucket( - new TableBucket(tableId, i), snapshots[i])); + new TableBucketSnapshot(new TableBucket(tableId, i), snapshots[i])); leasedBucketCount.decrementAndGet(); } } else { @@ -373,7 +340,7 @@ private void clearRefCount(KvSnapshotLease lease) { } decrementRefCount( - new KvSnapshotLeaseForBucket( + new TableBucketSnapshot( new TableBucket(tableId, partitionId, i), snapshots[i])); leasedBucketCount.decrementAndGet(); } @@ -382,43 +349,39 @@ private void clearRefCount(KvSnapshotLease lease) { } } - private void incrementRefCount(KvSnapshotLeaseForBucket kvSnapshotLeaseForBucket) { - inWriteLock( - refCountLock, - () -> - refCount.computeIfAbsent( - kvSnapshotLeaseForBucket, k -> new AtomicInteger(0)) - .incrementAndGet()); + private void incrementRefCount(TableBucketSnapshot tableBucketSnapshot) { + refCount.computeIfAbsent(tableBucketSnapshot, k -> new AtomicInteger(0)).incrementAndGet(); } - private void decrementRefCount(KvSnapshotLeaseForBucket kvSnapshotLeaseForBucket) { - inWriteLock( - refCountLock, - () -> { - AtomicInteger atomicInteger = refCount.get(kvSnapshotLeaseForBucket); - if (atomicInteger != null) { - int decrementAndGet = atomicInteger.decrementAndGet(); - if (decrementAndGet <= 0) { - refCount.remove(kvSnapshotLeaseForBucket); - } - } + private void decrementRefCount(TableBucketSnapshot tableBucketSnapshot) { + refCount.computeIfPresent( + tableBucketSnapshot, + (k, v) -> { + int newCount = v.decrementAndGet(); + return newCount <= 0 ? null : v; }); } private void expireLeases() { long currentTime = clock.milliseconds(); - // 1. First collect all expired lease IDs + // 1. First collect all expired lease IDs under read lock List expiredLeaseIds = - kvSnapshotLeaseMap.entrySet().stream() - .filter(entry -> entry.getValue().getExpirationTime() < currentTime) - .map(Map.Entry::getKey) - .collect(Collectors.toList()); - - // 2. Then process each collected ID + inReadLock( + managerLock, + () -> + kvSnapshotLeaseMap.entrySet().stream() + .filter( + entry -> + entry.getValue().getExpirationTime() + < currentTime) + .map(Map.Entry::getKey) + .collect(Collectors.toList())); + + // 2. Then process each collected ID (dropLease acquires write lock) expiredLeaseIds.forEach( leaseId -> { try { - releaseAll(leaseId); + dropLease(leaseId); } catch (Exception e) { LOG.error("Failed to clear kv snapshot lease {}", leaseId, e); } @@ -434,7 +397,7 @@ private void registerMetrics(CoordinatorMetricGroup coordinatorMetricGroup) { @VisibleForTesting int getLeaseCount() { - return kvSnapshotLeaseMap.size(); + return inReadLock(managerLock, kvSnapshotLeaseMap::size); } @VisibleForTesting @@ -443,17 +406,22 @@ int getLeasedBucketCount() { } @VisibleForTesting - int getRefCount(KvSnapshotLeaseForBucket kvSnapshotLeaseForBucket) { + int getRefCount(TableBucketSnapshot tableBucketSnapshot) { return inReadLock( - refCountLock, + managerLock, () -> { - AtomicInteger count = refCount.get(kvSnapshotLeaseForBucket); + AtomicInteger count = refCount.get(tableBucketSnapshot); return count == null ? 0 : count.get(); }); } @VisibleForTesting - KvSnapshotLease getKvSnapshotLease(String leaseId) { - return kvSnapshotLeaseMap.get(leaseId); + KvSnapshotLeaseHandler getKvSnapshotLeaseData(String leaseId) { + return inReadLock(managerLock, () -> kvSnapshotLeaseMap.get(leaseId)); + } + + @VisibleForTesting + KvSnapshotLeaseMetadataManager getMetadataManager() { + return metadataManager; } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataManager.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/lease/KvSnapshotLeaseMetadataManager.java similarity index 51% rename from fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataManager.java rename to fluss-server/src/main/java/org/apache/fluss/server/coordinator/lease/KvSnapshotLeaseMetadataManager.java index d269e8d763..d5333885f4 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/lease/KvSnapshotLeaseMetadataManager.java @@ -16,23 +16,24 @@ * limitations under the License. */ -package org.apache.fluss.server.zk.data.lease; +package org.apache.fluss.server.coordinator.lease; import org.apache.fluss.fs.FSDataInputStream; import org.apache.fluss.fs.FSDataOutputStream; import org.apache.fluss.fs.FileSystem; import org.apache.fluss.fs.FsPath; -import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.server.zk.ZooKeeperClient; -import org.apache.fluss.server.zk.data.BucketSnapshot; +import org.apache.fluss.server.zk.data.lease.KvSnapshotLeaseMetadata; +import org.apache.fluss.server.zk.data.lease.KvSnapshotTableLease; +import org.apache.fluss.server.zk.data.lease.KvSnapshotTableLeaseJsonSerde; import org.apache.fluss.utils.FlussPaths; import org.apache.fluss.utils.IOUtils; -import org.apache.fluss.utils.types.Tuple2; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.ByteArrayOutputStream; +import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -41,8 +42,8 @@ import static org.apache.fluss.utils.Preconditions.checkNotNull; /** - * The manager to handle {@link KvSnapshotLease} to register/update/delete metadata from zk and - * remote fs. + * The manager to handle {@link KvSnapshotLeaseHandler} to register/update/delete metadata from zk + * and remote fs. */ public class KvSnapshotLeaseMetadataManager { private static final Logger LOG = LoggerFactory.getLogger(KvSnapshotLeaseMetadataManager.class); @@ -62,23 +63,34 @@ public List getLeasesList() throws Exception { /** * Register a new kv snapshot lease to zk and remote fs. * + *

Follows the createdFiles pattern (similar to {@code ProducerOffsetsStore.tryStoreOffsets}) + * to track files created in this operation. On failure, only the newly created files are + * cleaned up, avoiding accidental removal of previously successful metadata files. + * * @param leaseId the lease id. * @param lease the kv snapshot lease. */ - public void registerLease(String leaseId, KvSnapshotLease lease) throws Exception { - Map tableIdToRemoteMetadataFsPath = generateMetadataFile(leaseId, lease); + public void registerLease(String leaseId, KvSnapshotLeaseHandler lease) throws Exception { + List createdFiles = new ArrayList<>(); + try { + Map tableIdToRemoteMetadataFsPath = + generateMetadataFile(leaseId, lease, createdFiles); - // generate remote fsPath of metadata. - KvSnapshotLeaseMetadata leaseMetadata = - new KvSnapshotLeaseMetadata( - lease.getExpirationTime(), tableIdToRemoteMetadataFsPath); + // generate remote fsPath of metadata. + KvSnapshotLeaseMetadata leaseMetadata = + new KvSnapshotLeaseMetadata( + lease.getExpirationTime(), tableIdToRemoteMetadataFsPath); - // register kv snapshot metadata to zk. - try { + // register kv snapshot metadata to zk. zkClient.registerKvSnapshotLeaseMetadata(leaseId, leaseMetadata); } catch (Exception e) { - LOG.warn("Failed to register kv snapshot lease metadata to zk.", e); - leaseMetadata.discard(); + LOG.warn( + "Failed to register kv snapshot lease metadata for lease {}, " + + "cleaning up {} created files.", + leaseId, + createdFiles.size(), + e); + cleanupFilesSafely(createdFiles); throw e; } } @@ -86,33 +98,53 @@ public void registerLease(String leaseId, KvSnapshotLease lease) throws Exceptio /** * Update a kv snapshot lease to zk and remote fs. * + *

Follows the createdFiles pattern (similar to {@code ProducerOffsetsStore.tryStoreOffsets}) + * to track files created in this operation. On failure, only the newly created files are + * cleaned up, avoiding accidental removal of previously successful metadata files. + * * @param leaseId the lease id. - * @param kvSnapshotLease the kv snapshot lease. + * @param kvSnapshotLeaseHandle the kv snapshot lease. */ - public void updateLease(String leaseId, KvSnapshotLease kvSnapshotLease) throws Exception { + public void updateLease(String leaseId, KvSnapshotLeaseHandler kvSnapshotLeaseHandle) + throws Exception { // TODO change this to incremental update to avoid create too many remote metadata files. + Optional originalLeaseMetadata; + List createdFiles = new ArrayList<>(); + try { + originalLeaseMetadata = zkClient.getKvSnapshotLeaseMetadata(leaseId); - Optional originalLeaseMetadata = - zkClient.getKvSnapshotLeaseMetadata(leaseId); + Map tableIdToNewRemoteMetadataFsPath = + generateMetadataFile(leaseId, kvSnapshotLeaseHandle, createdFiles); - Map tableIdToNewRemoteMetadataFsPath = - generateMetadataFile(leaseId, kvSnapshotLease); + // generate new kv snapshot lease metadata. + KvSnapshotLeaseMetadata newLeaseMetadata = + new KvSnapshotLeaseMetadata( + kvSnapshotLeaseHandle.getExpirationTime(), + tableIdToNewRemoteMetadataFsPath); - // generate new kv snapshot lease metadata. - KvSnapshotLeaseMetadata newLeaseMetadata = - new KvSnapshotLeaseMetadata( - kvSnapshotLease.getExpirationTime(), tableIdToNewRemoteMetadataFsPath); - // register new snapshot metadata to zk. - try { zkClient.updateKvSnapshotLeaseMetadata(leaseId, newLeaseMetadata); } catch (Exception e) { - LOG.warn("Failed to update kv snapshot lease metadata to zk.", e); - newLeaseMetadata.discard(); + LOG.warn( + "Failed to update kv snapshot lease metadata for lease {}, " + + "cleaning up {} created files.", + leaseId, + createdFiles.size(), + e); + cleanupFilesSafely(createdFiles); throw e; } - // discard original snapshot metadata. - originalLeaseMetadata.ifPresent(KvSnapshotLeaseMetadata::discard); + // Best-effort cleanup of old metadata files. Failures are logged but not propagated, + // as the ZK metadata has already been updated to point to the new files. + try { + originalLeaseMetadata.ifPresent(this::discardLeaseMetadata); + } catch (Exception e) { + LOG.warn( + "Failed to discard original lease metadata for lease {}, " + + "orphaned files may exist.", + leaseId, + e); + } } /** @@ -121,7 +153,7 @@ public void updateLease(String leaseId, KvSnapshotLease kvSnapshotLease) throws * @param leaseId the lease id. * @return the kv snapshot lease. */ - public Optional getLease(String leaseId) throws Exception { + public Optional getLease(String leaseId) throws Exception { Optional kvSnapshotLeaseMetadataOpt = zkClient.getKvSnapshotLeaseMetadata(leaseId); if (!kvSnapshotLeaseMetadataOpt.isPresent()) { @@ -129,8 +161,9 @@ public Optional getLease(String leaseId) throws Exception { } KvSnapshotLeaseMetadata kvSnapshotLeaseMetadata = kvSnapshotLeaseMetadataOpt.get(); - KvSnapshotLease kvSnapshotLease = buildKvSnapshotLease(kvSnapshotLeaseMetadata); - return Optional.of(kvSnapshotLease); + KvSnapshotLeaseHandler kvSnapshotLeasehandle = + buildKvSnapshotLease(kvSnapshotLeaseMetadata); + return Optional.of(kvSnapshotLeasehandle); } /** @@ -145,36 +178,32 @@ public void deleteLease(String leaseId) throws Exception { // delete zk metadata. zkClient.deleteKvSnapshotLease(leaseId); - // delete remote metadata file. - leaseMetadataOpt.ifPresent(KvSnapshotLeaseMetadata::discard); + // Best-effort cleanup of remote metadata files. Failures are logged but not + // propagated, as the ZK metadata has already been deleted. + try { + leaseMetadataOpt.ifPresent(this::discardLeaseMetadata); + } catch (Exception e) { + LOG.warn( + "Failed to discard lease metadata for lease {}, " + "orphaned files may exist.", + leaseId, + e); + } } /** - * Check whether the snapshot exists for the bucket in zookeeper. - * - * @param tableBucket the table bucket. - * @param snapshotId the snapshot id. - * @return true if the snapshot exists in the bucket. + * Generate metadata files for all tables in the lease. Each created file is tracked in the + * provided {@code createdFiles} list for safe cleanup on failure. */ - public boolean isSnapshotExists(TableBucket tableBucket, long snapshotId) throws Exception { - List> allSnapshotAndIds = - zkClient.getTableBucketAllSnapshotAndIds(tableBucket); - for (Tuple2 snapshotAndId : allSnapshotAndIds) { - if (snapshotAndId.f1 == snapshotId) { - return true; - } - } - return false; - } - - private Map generateMetadataFile(String leaseId, KvSnapshotLease lease) + private Map generateMetadataFile( + String leaseId, KvSnapshotLeaseHandler lease, List createdFiles) throws Exception { Map tableIdToMetadataFile = new HashMap<>(); for (Map.Entry entry : lease.getTableIdToTableLease().entrySet()) { long tableId = entry.getKey(); - tableIdToMetadataFile.put( - tableId, generateMetadataFile(tableId, leaseId, entry.getValue())); + FsPath path = generateMetadataFile(tableId, leaseId, entry.getValue()); + createdFiles.add(path); + tableIdToMetadataFile.put(tableId, path); } return tableIdToMetadataFile; } @@ -199,8 +228,8 @@ private FsPath generateMetadataFile( return remoteKvSnapshotLeaseFile; } - private KvSnapshotLease buildKvSnapshotLease(KvSnapshotLeaseMetadata kvSnapshotLeaseMetadata) - throws Exception { + private KvSnapshotLeaseHandler buildKvSnapshotLease( + KvSnapshotLeaseMetadata kvSnapshotLeaseMetadata) throws Exception { Map tableIdToRemoteMetadataFilePath = kvSnapshotLeaseMetadata.getTableIdToRemoteMetadataFilePath(); Map tableIdToTableLease = new HashMap<>(); @@ -209,7 +238,7 @@ private KvSnapshotLease buildKvSnapshotLease(KvSnapshotLeaseMetadata kvSnapshotL FsPath remoteMetadataFilePath = entry.getValue(); tableIdToTableLease.put(tableId, buildKvSnapshotTableLease(remoteMetadataFilePath)); } - return new KvSnapshotLease( + return new KvSnapshotLeaseHandler( kvSnapshotLeaseMetadata.getExpirationTime(), tableIdToTableLease); } @@ -223,4 +252,30 @@ private KvSnapshotTableLease buildKvSnapshotTableLease(FsPath remoteMetadataFile return KvSnapshotTableLeaseJsonSerde.fromJson(outputStream.toByteArray()); } } + + private void discardLeaseMetadata(KvSnapshotLeaseMetadata kvSnapshotLeaseMetadata) { + cleanupFilesSafely( + new ArrayList<>( + kvSnapshotLeaseMetadata.getTableIdToRemoteMetadataFilePath().values())); + } + + /** + * Safely clean up a list of remote files. Each file deletion is independent - failures for + * individual files are logged but do not prevent cleanup of remaining files. + */ + private void cleanupFilesSafely(List files) { + for (FsPath file : files) { + try { + FileSystem fileSystem = file.getFileSystem(); + if (fileSystem.exists(file)) { + fileSystem.delete(file, false); + } + } catch (Exception e) { + LOG.warn( + "Error deleting remote file path of kv snapshot lease metadata at {}", + file, + e); + } + } + } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStore.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStore.java index dc2d9c8fee..e1a431c3d4 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStore.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStore.java @@ -21,8 +21,8 @@ import org.apache.fluss.fs.FSDataOutputStream; import org.apache.fluss.fs.FileSystem; import org.apache.fluss.fs.FsPath; -import org.apache.fluss.metadata.KvSnapshotLeaseForBucket; import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TableBucketSnapshot; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -211,7 +211,7 @@ interface SubsumeAction { /** A function to check whether a snapshot can be subsumed. */ @FunctionalInterface public interface SubsumptionChecker { - boolean canSubsume(KvSnapshotLeaseForBucket bucket); + boolean canSubsume(TableBucketSnapshot bucket); } private static boolean canSubsume( @@ -224,7 +224,7 @@ private static boolean canSubsume( } return subsumptionChecker.canSubsume( - new KvSnapshotLeaseForBucket(next.getTableBucket(), next.getSnapshotID())); + new TableBucketSnapshot(next.getTableBucket(), next.getSnapshotID())); } /** diff --git a/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java b/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java index 968a8b0e55..0ecc719f1d 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 @@ -30,11 +30,11 @@ import org.apache.fluss.fs.FsPath; import org.apache.fluss.fs.token.ObtainedSecurityToken; import org.apache.fluss.metadata.DatabaseSummary; -import org.apache.fluss.metadata.KvSnapshotLeaseForBucket; import org.apache.fluss.metadata.PartitionSpec; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.ResolvedPartitionSpec; import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TableBucketSnapshot; import org.apache.fluss.metadata.TableChange; import org.apache.fluss.metadata.TableDescriptor; import org.apache.fluss.metadata.TableInfo; @@ -57,6 +57,7 @@ import org.apache.fluss.rpc.entity.ProduceLogResultForBucket; import org.apache.fluss.rpc.entity.PutKvResultForBucket; import org.apache.fluss.rpc.messages.AcquireKvSnapshotLeaseRequest; +import org.apache.fluss.rpc.messages.AcquireKvSnapshotLeaseResponse; import org.apache.fluss.rpc.messages.AdjustIsrRequest; import org.apache.fluss.rpc.messages.AdjustIsrResponse; import org.apache.fluss.rpc.messages.AlterTableRequest; @@ -95,7 +96,6 @@ import org.apache.fluss.rpc.messages.PbAdjustIsrRespForBucket; import org.apache.fluss.rpc.messages.PbAdjustIsrRespForTable; import org.apache.fluss.rpc.messages.PbAlterConfig; -import org.apache.fluss.rpc.messages.PbBucket; import org.apache.fluss.rpc.messages.PbBucketMetadata; import org.apache.fluss.rpc.messages.PbBucketOffset; import org.apache.fluss.rpc.messages.PbCreateAclRespInfo; @@ -141,7 +141,6 @@ import org.apache.fluss.rpc.messages.PbServerNode; import org.apache.fluss.rpc.messages.PbStopReplicaReqForBucket; import org.apache.fluss.rpc.messages.PbStopReplicaRespForBucket; -import org.apache.fluss.rpc.messages.PbTable; import org.apache.fluss.rpc.messages.PbTableBucket; import org.apache.fluss.rpc.messages.PbTableMetadata; import org.apache.fluss.rpc.messages.PbTableOffsets; @@ -1975,13 +1974,14 @@ private static PbRebalancePlanForBucket toPbRebalancePlanForBucket( return pbRebalancePlanForBucket; } - public static Map> getAcquireKvSnapshotLeaseData( + public static Map> getAcquireKvSnapshotLeaseData( AcquireKvSnapshotLeaseRequest request) { - Map> tableIdToLeasedBucket = new HashMap<>(); - for (PbKvSnapshotLeaseForTable leaseForTable : request.getTableLeaseReqsList()) { - long tableId = leaseForTable.getTableId(); - List bucketList = new ArrayList<>(); - for (PbKvSnapshotLeaseForBucket leaseForBucket : leaseForTable.getBucketsReqsList()) { + Map> tableIdToLeasedBucket = new HashMap<>(); + for (PbKvSnapshotLeaseForTable snapshotToLease : request.getSnapshotsToLeasesList()) { + long tableId = snapshotToLease.getTableId(); + List bucketList = new ArrayList<>(); + for (PbKvSnapshotLeaseForBucket leaseForBucket : + snapshotToLease.getBucketSnapshotsList()) { bucketList.add(getKvSnapshotLeaseForBucket(tableId, leaseForBucket)); } tableIdToLeasedBucket.put(tableId, bucketList); @@ -1989,27 +1989,46 @@ public static Map> getAcquireKvSnapshotLeas return tableIdToLeasedBucket; } - public static Map> getReleaseKvSnapshotLeaseData( + public static List getReleaseKvSnapshotLeaseData( ReleaseKvSnapshotLeaseRequest request) { - Map> tableIdToReleasedBucket = new HashMap<>(); - for (PbTable pbTable : request.getReleaseTablesList()) { - long tableId = pbTable.getTableId(); - List bucketList = new ArrayList<>(); - for (PbBucket pbBucket : pbTable.getBucketsList()) { - bucketList.add( - new TableBucket( - tableId, - pbBucket.hasPartitionId() ? pbBucket.getPartitionId() : null, - pbBucket.getBucketId())); + List bucketList = new ArrayList<>(); + for (PbTableBucket pbTableBucket : request.getBucketsToReleasesList()) { + bucketList.add( + new TableBucket( + pbTableBucket.getTableId(), + pbTableBucket.hasPartitionId() ? pbTableBucket.getPartitionId() : null, + pbTableBucket.getBucketId())); + } + return bucketList; + } + + public static AcquireKvSnapshotLeaseResponse makeAcquireKvSnapshotLeaseResponse( + Map unavailableSnapshots) { + AcquireKvSnapshotLeaseResponse response = new AcquireKvSnapshotLeaseResponse(); + Map> pbFailedTables = new HashMap<>(); + for (Map.Entry entry : unavailableSnapshots.entrySet()) { + TableBucket tb = entry.getKey(); + Long snapshotId = entry.getValue(); + PbKvSnapshotLeaseForBucket pbBucket = + new PbKvSnapshotLeaseForBucket().setBucketId(tb.getBucket()); + if (tb.getPartitionId() != null) { + pbBucket.setPartitionId(tb.getPartitionId()); } - tableIdToReleasedBucket.put(tableId, bucketList); + pbBucket.setSnapshotId(snapshotId); + pbFailedTables.computeIfAbsent(tb.getTableId(), k -> new ArrayList<>()).add(pbBucket); + } + + for (Map.Entry> entry : pbFailedTables.entrySet()) { + response.addUnavailableSnapshot() + .setTableId(entry.getKey()) + .addAllBucketSnapshots(entry.getValue()); } - return tableIdToReleasedBucket; + return response; } - private static KvSnapshotLeaseForBucket getKvSnapshotLeaseForBucket( + private static TableBucketSnapshot getKvSnapshotLeaseForBucket( long tableId, PbKvSnapshotLeaseForBucket leaseForBucket) { - return new KvSnapshotLeaseForBucket( + return new TableBucketSnapshot( new TableBucket( tableId, leaseForBucket.hasPartitionId() ? leaseForBucket.getPartitionId() : null, diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadata.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadata.java index 2c9a40a9c5..b6adff1cbc 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadata.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadata.java @@ -17,15 +17,11 @@ package org.apache.fluss.server.zk.data.lease; -import org.apache.fluss.fs.FileSystem; import org.apache.fluss.fs.FsPath; -import java.io.IOException; import java.util.Map; import java.util.Objects; -import static org.apache.fluss.metrics.registry.MetricRegistry.LOG; - /** The zkNode data of kv snapshot lease. */ public class KvSnapshotLeaseMetadata { private final long expirationTime; @@ -45,25 +41,6 @@ public Map getTableIdToRemoteMetadataFilePath() { return tableIdToRemoteMetadataFilePath; } - public void discard() { - // delete all remote metadata file. - tableIdToRemoteMetadataFilePath.values().forEach(this::delete); - } - - private void delete(FsPath fsPath) { - try { - FileSystem fileSystem = fsPath.getFileSystem(); - if (fileSystem.exists(fsPath)) { - fileSystem.delete(fsPath, false); - } - } catch (IOException e) { - LOG.warn( - "Error deleting remote file path of kv snapshot lease metadata at {}", - fsPath, - e); - } - } - @Override public String toString() { return "KvSnapshotLeaseMetadata{" diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLease.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLease.java index 4cebd27405..df6a1bfd14 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLease.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLease.java @@ -23,31 +23,33 @@ import javax.annotation.concurrent.NotThreadSafe; import java.util.Arrays; -import java.util.Collections; import java.util.Map; import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; /** The lease of kv snapshot for a table. */ @NotThreadSafe public class KvSnapshotTableLease { private final long tableId; - private final @Nullable Long[] bucketSnapshots; - private final Map partitionSnapshots; + private @Nullable Long[] bucketSnapshots; + private final ConcurrentHashMap partitionSnapshots; public KvSnapshotTableLease(long tableId) { this(tableId, null, MapUtils.newConcurrentHashMap()); } public KvSnapshotTableLease(long tableId, Long[] bucketSnapshots) { - this(tableId, bucketSnapshots, Collections.emptyMap()); + this(tableId, bucketSnapshots, MapUtils.newConcurrentHashMap()); } - public KvSnapshotTableLease(long tableId, Map partitionSnapshots) { + public KvSnapshotTableLease(long tableId, ConcurrentHashMap partitionSnapshots) { this(tableId, null, partitionSnapshots); } public KvSnapshotTableLease( - long tableId, @Nullable Long[] bucketSnapshots, Map partitionSnapshots) { + long tableId, + @Nullable Long[] bucketSnapshots, + ConcurrentHashMap partitionSnapshots) { this.tableId = tableId; this.bucketSnapshots = bucketSnapshots; this.partitionSnapshots = partitionSnapshots; @@ -61,6 +63,10 @@ public long getTableId() { return bucketSnapshots; } + public void setBucketSnapshots(@Nullable Long[] bucketSnapshots) { + this.bucketSnapshots = bucketSnapshots; + } + public @Nullable Long[] getBucketSnapshots(long partitionId) { return partitionSnapshots.get(partitionId); } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLeaseJsonSerde.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLeaseJsonSerde.java index 09dfbb3f32..6aa33a3cc8 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLeaseJsonSerde.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLeaseJsonSerde.java @@ -19,13 +19,14 @@ 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.MapUtils; 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.HashMap; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; /** Json serializer and deserializer for {@link KvSnapshotTableLease}. */ public class KvSnapshotTableLeaseJsonSerde @@ -88,7 +89,7 @@ public KvSnapshotTableLease deserialize(JsonNode node) { return new KvSnapshotTableLease(tableId, bucketSnapshots); } else { // for partition table. - Map partitionSnapshots = new HashMap<>(); + ConcurrentHashMap partitionSnapshots = MapUtils.newConcurrentHashMap(); JsonNode partitionSnapshotsNode = node.get(PARTITION_SNAPSHOTS); for (JsonNode partitionSnapshotNode : partitionSnapshotsNode) { long partitionId = partitionSnapshotNode.get(PARTITION_ID).asLong(); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java index 0c1c11c0e6..045d40ed56 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java @@ -48,6 +48,7 @@ import org.apache.fluss.server.coordinator.event.CommitKvSnapshotEvent; import org.apache.fluss.server.coordinator.event.CommitRemoteLogManifestEvent; import org.apache.fluss.server.coordinator.event.CoordinatorEventManager; +import org.apache.fluss.server.coordinator.lease.KvSnapshotLeaseManager; import org.apache.fluss.server.coordinator.statemachine.BucketState; import org.apache.fluss.server.coordinator.statemachine.ReplicaState; import org.apache.fluss.server.entity.AdjustIsrResultForBucket; @@ -155,6 +156,7 @@ class CoordinatorEventProcessorTest { private LakeTableTieringManager lakeTableTieringManager; private CompletedSnapshotStoreManager completedSnapshotStoreManager; private CoordinatorMetadataCache serverMetadataCache; + private KvSnapshotLeaseManager kvSnapshotLeaseManager; @BeforeAll static void baseBeforeAll() throws Exception { @@ -194,12 +196,22 @@ void beforeEach() throws IOException { new AutoPartitionManager(serverMetadataCache, metadataManager, new Configuration()); lakeTableTieringManager = new LakeTableTieringManager(); Configuration conf = new Configuration(); - conf.setString(ConfigOptions.REMOTE_DATA_DIR, "/tmp/fluss/remote-data"); + String remoteDataDir = "/tmp/fluss/remote-data"; + conf.setString(ConfigOptions.REMOTE_DATA_DIR, remoteDataDir); eventProcessor = buildCoordinatorEventProcessor(); eventProcessor.startup(); metadataManager.createDatabase( defaultDatabase, DatabaseDescriptor.builder().build(), false); completedSnapshotStoreManager = eventProcessor.completedSnapshotStoreManager(); + + kvSnapshotLeaseManager = + new KvSnapshotLeaseManager( + Duration.ofMinutes(10).toMillis(), + zookeeperClient, + remoteDataDir, + SystemClock.getInstance(), + TestingMetricGroups.COORDINATOR_METRICS); + kvSnapshotLeaseManager.start(); } @AfterEach @@ -1053,7 +1065,7 @@ private CoordinatorEventProcessor buildCoordinatorEventProcessor() { new Configuration(), Executors.newFixedThreadPool(1, new ExecutorThreadFactory("test-coordinator-io")), metadataManager, - SystemClock.getInstance()); + kvSnapshotLeaseManager); } private void initCoordinatorChannel() throws Exception { diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TestCoordinatorGateway.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TestCoordinatorGateway.java index b6d567b8cd..38e9730ca1 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 @@ -62,6 +62,8 @@ import org.apache.fluss.rpc.messages.DropAclsResponse; import org.apache.fluss.rpc.messages.DropDatabaseRequest; import org.apache.fluss.rpc.messages.DropDatabaseResponse; +import org.apache.fluss.rpc.messages.DropKvSnapshotLeaseRequest; +import org.apache.fluss.rpc.messages.DropKvSnapshotLeaseResponse; import org.apache.fluss.rpc.messages.DropPartitionRequest; import org.apache.fluss.rpc.messages.DropPartitionResponse; import org.apache.fluss.rpc.messages.DropTableRequest; @@ -413,6 +415,12 @@ public CompletableFuture releaseKvSnapshotLease( throw new UnsupportedOperationException(); } + @Override + public CompletableFuture dropKvSnapshotLease( + DropKvSnapshotLeaseRequest request) { + return null; + } + @Override public CompletableFuture listAcls(ListAclsRequest request) { throw new UnsupportedOperationException(); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/lease/KvSnapshotLeaseHandlerTest.java similarity index 50% rename from fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseTest.java rename to fluss-server/src/test/java/org/apache/fluss/server/coordinator/lease/KvSnapshotLeaseHandlerTest.java index 1dcf26054b..f6ad5c0e11 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/lease/KvSnapshotLeaseHandlerTest.java @@ -15,36 +15,36 @@ * limitations under the License. */ -package org.apache.fluss.server.zk.data.lease; +package org.apache.fluss.server.coordinator.lease; import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.server.zk.data.lease.KvSnapshotTableLease; +import org.apache.fluss.utils.MapUtils; import org.junit.jupiter.api.Test; import java.util.HashMap; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; -/** Test for {@link KvSnapshotLease}. */ -public class KvSnapshotLeaseTest { - - private static final int NUM_BUCKET = 2; +/** Test for {@link KvSnapshotLeaseHandler}. */ +public class KvSnapshotLeaseHandlerTest { @Test void testConstructorAndGetters() { long expirationTime = 1000L; - KvSnapshotLease kvSnapshotLease = new KvSnapshotLease(expirationTime); + KvSnapshotLeaseHandler kvSnapshotLeasehandle = new KvSnapshotLeaseHandler(expirationTime); - assertThat(kvSnapshotLease.getExpirationTime()).isEqualTo(expirationTime); - assertThat(kvSnapshotLease.getTableIdToTableLease()).isEmpty(); - assertThat(kvSnapshotLease.getLeasedSnapshotCount()).isEqualTo(0); + assertThat(kvSnapshotLeasehandle.getExpirationTime()).isEqualTo(expirationTime); + assertThat(kvSnapshotLeasehandle.getTableIdToTableLease()).isEmpty(); + assertThat(kvSnapshotLeasehandle.getLeasedSnapshotCount()).isEqualTo(0); } @Test void testRegisterBucketForNonPartitionedTable() { - KvSnapshotLease lease = new KvSnapshotLease(1000L); + KvSnapshotLeaseHandler lease = new KvSnapshotLeaseHandler(1000L); long tableId = 1L; int bucketId = 0; @@ -55,9 +55,8 @@ void testRegisterBucketForNonPartitionedTable() { KvSnapshotTableLease tableLease = lease.getTableIdToTableLease().get(tableId); Long[] bucketSnapshots = tableLease.getBucketSnapshots(); assertThat(bucketSnapshots).isNotNull(); - assertThat(bucketSnapshots).hasSize(NUM_BUCKET); + assertThat(bucketSnapshots).hasSize(1); assertThat(bucketSnapshots[bucketId]).isEqualTo(123L); - assertThat(bucketSnapshots[1]).isEqualTo(-1L); // Register again same bucket → should be update originalSnapshot = acquireBucket(lease, new TableBucket(tableId, bucketId), 456L); @@ -69,27 +68,72 @@ void testRegisterBucketForNonPartitionedTable() { } @Test - void testIllegalBucketNum() { - // Currently, for the same table, the bucket num should be the same. - KvSnapshotLease lease = new KvSnapshotLease(1000L); + void testAcquireBucketDynamicExpansionForNonPartitionedTable() { + KvSnapshotLeaseHandler lease = new KvSnapshotLeaseHandler(1000L); long tableId = 1L; - int bucketId = 0; - lease.acquireBucket(new TableBucket(tableId, bucketId), 123L, 10); - assertThatThrownBy(() -> lease.acquireBucket(new TableBucket(tableId, bucketId), 456L, 20)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining( - "Bucket index is null, or input bucket number is not equal to the bucket " - + "number of the table."); + // Acquire bucket 0 → creates array of size 1 + acquireBucket(lease, new TableBucket(tableId, 0), 100L); + Long[] bucketSnapshots = lease.getTableIdToTableLease().get(tableId).getBucketSnapshots(); + assertThat(bucketSnapshots).hasSize(1); + assertThat(bucketSnapshots[0]).isEqualTo(100L); + + // Acquire bucket 2 → array should expand from size 1 to size 3 + long originalSnapshot = acquireBucket(lease, new TableBucket(tableId, 2), 200L); + assertThat(originalSnapshot).isEqualTo(-1L); + bucketSnapshots = lease.getTableIdToTableLease().get(tableId).getBucketSnapshots(); + assertThat(bucketSnapshots).hasSize(3); + assertThat(bucketSnapshots[0]).isEqualTo(100L); + assertThat(bucketSnapshots[1]).isEqualTo(-1L); + assertThat(bucketSnapshots[2]).isEqualTo(200L); + + // Acquire bucket 1 → no expansion needed, just fill in + originalSnapshot = acquireBucket(lease, new TableBucket(tableId, 1), 150L); + assertThat(originalSnapshot).isEqualTo(-1L); + bucketSnapshots = lease.getTableIdToTableLease().get(tableId).getBucketSnapshots(); + assertThat(bucketSnapshots).hasSize(3); + assertThat(bucketSnapshots[1]).isEqualTo(150L); + } + + @Test + void testAcquireBucketDynamicExpansionForPartitionedTable() { + KvSnapshotLeaseHandler lease = new KvSnapshotLeaseHandler(1000L); + long tableId = 1L; + long partitionId = 1000L; + + // Acquire partition 1000, bucket 0 → creates array of size 1 + acquireBucket(lease, new TableBucket(tableId, partitionId, 0), 100L); + Long[] partitionBuckets = + lease.getTableIdToTableLease() + .get(tableId) + .getPartitionSnapshots() + .get(partitionId); + assertThat(partitionBuckets).hasSize(1); + assertThat(partitionBuckets[0]).isEqualTo(100L); + + // Acquire partition 1000, bucket 2 → array should expand from size 1 to size 3 + long originalSnapshot = + acquireBucket(lease, new TableBucket(tableId, partitionId, 2), 200L); + assertThat(originalSnapshot).isEqualTo(-1L); + partitionBuckets = + lease.getTableIdToTableLease() + .get(tableId) + .getPartitionSnapshots() + .get(partitionId); + assertThat(partitionBuckets).hasSize(3); + assertThat(partitionBuckets[0]).isEqualTo(100L); + assertThat(partitionBuckets[1]).isEqualTo(-1L); + assertThat(partitionBuckets[2]).isEqualTo(200L); } @Test void testRegisterBucketForPartitionedTable() { - KvSnapshotLease lease = new KvSnapshotLease(1000L); + KvSnapshotLeaseHandler lease = new KvSnapshotLeaseHandler(1000L); long tableId = 1L; long originalSnapshot = acquireBucket(lease, new TableBucket(tableId, 1000L, 0), 111L); assertThat(originalSnapshot).isEqualTo(-1L); + // Acquire bucket 1 for partition 1000 → array expands from size 1 to size 2 originalSnapshot = acquireBucket(lease, new TableBucket(tableId, 1000L, 1), 122L); assertThat(originalSnapshot).isEqualTo(-1L); originalSnapshot = acquireBucket(lease, new TableBucket(tableId, 1001L, 0), 122L); @@ -102,8 +146,9 @@ void testRegisterBucketForPartitionedTable() { assertThat(partitionSnapshots).containsKeys(1000L, 1001L); assertThat(partitionSnapshots.get(1000L)[0]).isEqualTo(111L); assertThat(partitionSnapshots.get(1000L)[1]).isEqualTo(122L); + // Partition 1001 only has bucket 0, so its array size is 1 + assertThat(partitionSnapshots.get(1001L)).hasSize(1); assertThat(partitionSnapshots.get(1001L)[0]).isEqualTo(122L); - assertThat(partitionSnapshots.get(1001L)[1]).isEqualTo(-1L); // test update. originalSnapshot = acquireBucket(lease, new TableBucket(tableId, 1000L, 0), 222L); @@ -113,7 +158,7 @@ void testRegisterBucketForPartitionedTable() { @Test void testReleaseBucket() { - KvSnapshotLease lease = new KvSnapshotLease(1000L); + KvSnapshotLeaseHandler lease = new KvSnapshotLeaseHandler(1000L); long tableId = 1L; // Register @@ -127,9 +172,44 @@ void testReleaseBucket() { assertThat(lease.isEmpty()).isTrue(); } + @Test + void testReleaseBucketExceedingArraySize() { + KvSnapshotLeaseHandler lease = new KvSnapshotLeaseHandler(1000L); + long tableId = 1L; + + // Register bucket 0 only → array size is 1 + acquireBucket(lease, new TableBucket(tableId, 0), 100L); + assertThat(lease.getLeasedSnapshotCount()).isEqualTo(1); + + // Release bucket 5 which exceeds the array size → should return -1 + long snapshotId = releaseBucket(lease, new TableBucket(tableId, 5)); + assertThat(snapshotId).isEqualTo(-1L); + // The original lease should remain unchanged + assertThat(lease.getLeasedSnapshotCount()).isEqualTo(1); + assertThat(lease.isEmpty()).isFalse(); + } + + @Test + void testReleaseBucketExceedingArraySizeForPartitionedTable() { + KvSnapshotLeaseHandler lease = new KvSnapshotLeaseHandler(1000L); + long tableId = 1L; + long partitionId = 1000L; + + // Register partition 1000, bucket 0 only → array size is 1 + acquireBucket(lease, new TableBucket(tableId, partitionId, 0), 100L); + assertThat(lease.getLeasedSnapshotCount()).isEqualTo(1); + + // Release partition 1000, bucket 3 which exceeds the array size → should return -1 + long snapshotId = releaseBucket(lease, new TableBucket(tableId, partitionId, 3)); + assertThat(snapshotId).isEqualTo(-1L); + // The original lease should remain unchanged + assertThat(lease.getLeasedSnapshotCount()).isEqualTo(1); + assertThat(lease.isEmpty()).isFalse(); + } + @Test void testGetLeasedSnapshotCount() { - KvSnapshotLease lease = new KvSnapshotLease(1000L); + KvSnapshotLeaseHandler lease = new KvSnapshotLeaseHandler(1000L); // Non-partitioned acquireBucket(lease, new TableBucket(1L, 0), 100L); @@ -148,27 +228,27 @@ void testGetLeasedSnapshotCount() { @Test void testEqualsAndHashCode() { - KvSnapshotLease lease = new KvSnapshotLease(1000L); + KvSnapshotLeaseHandler lease = new KvSnapshotLeaseHandler(1000L); assertThat(lease).isEqualTo(lease); assertThat(lease.hashCode()).isEqualTo(lease.hashCode()); - KvSnapshotLease c1 = new KvSnapshotLease(1000L); - KvSnapshotLease c2 = new KvSnapshotLease(2000L); + KvSnapshotLeaseHandler c1 = new KvSnapshotLeaseHandler(1000L); + KvSnapshotLeaseHandler c2 = new KvSnapshotLeaseHandler(2000L); assertThat(c1).isNotEqualTo(c2); // Create two leases with same logical content but different array objects Map map1 = new HashMap<>(); - Map partitionSnapshots1 = new HashMap<>(); + ConcurrentHashMap partitionSnapshots1 = MapUtils.newConcurrentHashMap(); partitionSnapshots1.put(2001L, new Long[] {100L, -1L}); partitionSnapshots1.put(2002L, new Long[] {-1L, 101L}); map1.put(1L, new KvSnapshotTableLease(1L, new Long[] {100L, -1L}, partitionSnapshots1)); Map map2 = new HashMap<>(); - Map partitionSnapshots2 = new HashMap<>(); + ConcurrentHashMap partitionSnapshots2 = MapUtils.newConcurrentHashMap(); partitionSnapshots2.put(2001L, new Long[] {100L, -1L}); partitionSnapshots2.put(2002L, new Long[] {-1L, 101L}); map2.put(1L, new KvSnapshotTableLease(1L, new Long[] {100L, -1L}, partitionSnapshots2)); - c1 = new KvSnapshotLease(1000L, map1); - c2 = new KvSnapshotLease(1000L, map2); + c1 = new KvSnapshotLeaseHandler(1000L, map1); + c2 = new KvSnapshotLeaseHandler(1000L, map2); assertThat(c1).isEqualTo(c2); assertThat(c1.hashCode()).isEqualTo(c2.hashCode()); @@ -177,14 +257,14 @@ void testEqualsAndHashCode() { map1.put(1L, new KvSnapshotTableLease(1L, new Long[] {100L, -1L})); map2 = new HashMap<>(); map2.put(1L, new KvSnapshotTableLease(1L, new Long[] {200L, -1L})); - c1 = new KvSnapshotLease(1000L, map1); - c2 = new KvSnapshotLease(1000L, map2); + c1 = new KvSnapshotLeaseHandler(1000L, map1); + c2 = new KvSnapshotLeaseHandler(1000L, map2); assertThat(c1).isNotEqualTo(c2); } @Test void testToString() { - KvSnapshotLease lease = new KvSnapshotLease(1000L); + KvSnapshotLeaseHandler lease = new KvSnapshotLeaseHandler(1000L); acquireBucket(lease, new TableBucket(1L, 0), 100L); acquireBucket(lease, new TableBucket(1L, 1), 101L); acquireBucket(lease, new TableBucket(2L, 0L, 0), 200L); @@ -194,14 +274,14 @@ void testToString() { "KvSnapshotLease{expirationTime=1000, tableIdToTableLease={" + "1=KvSnapshotTableLease{tableId=1, bucketSnapshots=[100, 101], partitionSnapshots={}}, " + "2=KvSnapshotTableLease{tableId=2, bucketSnapshots=null, partitionSnapshots={" - + "0=[200, -1], 1=[-1, 201]}}}}"); + + "0=[200], 1=[-1, 201]}}}}"); } - private long acquireBucket(KvSnapshotLease lease, TableBucket tb, long kvSnapshotId) { - return lease.acquireBucket(tb, kvSnapshotId, NUM_BUCKET); + private long acquireBucket(KvSnapshotLeaseHandler lease, TableBucket tb, long kvSnapshotId) { + return lease.acquireBucket(tb, kvSnapshotId); } - private long releaseBucket(KvSnapshotLease lease, TableBucket tb) { + private long releaseBucket(KvSnapshotLeaseHandler lease, TableBucket tb) { return lease.releaseBucket(tb); } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/KvSnapshotLeaseManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/lease/KvSnapshotLeaseManagerTest.java similarity index 52% rename from fluss-server/src/test/java/org/apache/fluss/server/coordinator/KvSnapshotLeaseManagerTest.java rename to fluss-server/src/test/java/org/apache/fluss/server/coordinator/lease/KvSnapshotLeaseManagerTest.java index 74f62456b1..eb1cc90c25 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/KvSnapshotLeaseManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/lease/KvSnapshotLeaseManagerTest.java @@ -15,20 +15,15 @@ * limitations under the License. */ -package org.apache.fluss.server.coordinator; +package org.apache.fluss.server.coordinator.lease; -import org.apache.fluss.config.ConfigOptions; -import org.apache.fluss.config.Configuration; -import org.apache.fluss.metadata.KvSnapshotLeaseForBucket; -import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TableBucketSnapshot; import org.apache.fluss.server.metrics.group.TestingMetricGroups; import org.apache.fluss.server.zk.NOPErrorHandler; import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.server.zk.ZooKeeperExtension; import org.apache.fluss.server.zk.data.BucketSnapshot; -import org.apache.fluss.server.zk.data.lease.KvSnapshotLease; -import org.apache.fluss.server.zk.data.lease.KvSnapshotLeaseMetadataManager; import org.apache.fluss.server.zk.data.lease.KvSnapshotTableLease; import org.apache.fluss.testutils.common.AllCallbackWrapper; import org.apache.fluss.testutils.common.ManuallyTriggeredScheduledExecutorService; @@ -51,12 +46,7 @@ import java.util.concurrent.TimeUnit; import static org.apache.fluss.record.TestData.DATA1_TABLE_ID_PK; -import static org.apache.fluss.record.TestData.DATA1_TABLE_INFO_PK; -import static org.apache.fluss.record.TestData.DATA1_TABLE_PATH_PK; import static org.apache.fluss.record.TestData.PARTITION_TABLE_ID; -import static org.apache.fluss.record.TestData.PARTITION_TABLE_INFO; -import static org.apache.fluss.record.TestData.PARTITION_TABLE_PATH; -import static org.apache.fluss.server.coordinator.CoordinatorTestUtils.createServers; import static org.assertj.core.api.Assertions.assertThat; /** Test for {@link KvSnapshotLeaseManager}. */ @@ -67,14 +57,8 @@ public class KvSnapshotLeaseManagerTest { new AllCallbackWrapper<>(new ZooKeeperExtension()); private static final long PARTITION_ID_1 = 19001L; - private static final PhysicalTablePath PARTITION_TABLE_PATH_1 = - PhysicalTablePath.of(PARTITION_TABLE_PATH, "2024"); - private static final long PARTITION_ID_2 = 19002L; - private static final PhysicalTablePath PARTITION_TABLE_PATH_2 = - PhysicalTablePath.of(PARTITION_TABLE_PATH, "2025"); - private static final int NUM_BUCKETS = DATA1_TABLE_INFO_PK.getNumBuckets(); private static final TableBucket t0b0 = new TableBucket(DATA1_TABLE_ID_PK, 0); private static final TableBucket t0b1 = new TableBucket(DATA1_TABLE_ID_PK, 1); private static final TableBucket t1p0b0 = @@ -86,11 +70,9 @@ public class KvSnapshotLeaseManagerTest { protected static ZooKeeperClient zookeeperClient; - private CoordinatorContext coordinatorContext; private ManualClock manualClock; private ManuallyTriggeredScheduledExecutorService clearLeaseScheduler; private KvSnapshotLeaseManager kvSnapshotLeaseManager; - private KvSnapshotLeaseMetadataManager metadataManager; private @TempDir Path tempDir; @@ -104,18 +86,13 @@ static void beforeAll() { @BeforeEach void beforeEach() throws Exception { - initCoordinatorContext(); - Configuration conf = new Configuration(); - // set a huge expiration check interval to avoid expiration check. - conf.set(ConfigOptions.KV_SNAPSHOT_LEASE_EXPIRATION_CHECK_INTERVAL, Duration.ofDays(7)); manualClock = new ManualClock(System.currentTimeMillis()); clearLeaseScheduler = new ManuallyTriggeredScheduledExecutorService(); - metadataManager = new KvSnapshotLeaseMetadataManager(zookeeperClient, tempDir.toString()); kvSnapshotLeaseManager = new KvSnapshotLeaseManager( - conf, - metadataManager, - coordinatorContext, + Duration.ofDays(7).toMillis(), + zookeeperClient, + tempDir.toString(), clearLeaseScheduler, manualClock, TestingMetricGroups.COORDINATOR_METRICS); @@ -141,53 +118,60 @@ void testInitialize() throws Exception { assertThat( snapshotLeaseNotExists( Arrays.asList( - new KvSnapshotLeaseForBucket(t0b0, 0L), - new KvSnapshotLeaseForBucket(t0b1, 0L), - new KvSnapshotLeaseForBucket(t1p0b0, 0L), - new KvSnapshotLeaseForBucket(t1p0b1, 0L), - new KvSnapshotLeaseForBucket(t1p1b0, 0L)))) + new TableBucketSnapshot(t0b0, 0L), + new TableBucketSnapshot(t0b1, 0L), + new TableBucketSnapshot(t1p0b0, 0L), + new TableBucketSnapshot(t1p0b1, 0L), + new TableBucketSnapshot(t1p1b0, 0L)))) .isTrue(); assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(0); // test initialize from zookeeper when coordinator is started. - KvSnapshotLease kvSnapshotLease = new KvSnapshotLease(1000L); - acquire(kvSnapshotLease, new KvSnapshotLeaseForBucket(t0b0, 0L)); - acquire(kvSnapshotLease, new KvSnapshotLeaseForBucket(t0b1, 0L)); - acquire(kvSnapshotLease, new KvSnapshotLeaseForBucket(t1p0b0, 0L)); - acquire(kvSnapshotLease, new KvSnapshotLeaseForBucket(t1p0b1, 0L)); - acquire(kvSnapshotLease, new KvSnapshotLeaseForBucket(t1p1b0, 0L)); - metadataManager.registerLease("lease1", kvSnapshotLease); + KvSnapshotLeaseHandler kvSnapshotLeasehandle = new KvSnapshotLeaseHandler(1000L); + acquire(kvSnapshotLeasehandle, new TableBucketSnapshot(t0b0, 0L)); + acquire(kvSnapshotLeasehandle, new TableBucketSnapshot(t0b1, 0L)); + acquire(kvSnapshotLeasehandle, new TableBucketSnapshot(t1p0b0, 0L)); + acquire(kvSnapshotLeasehandle, new TableBucketSnapshot(t1p0b1, 0L)); + acquire(kvSnapshotLeasehandle, new TableBucketSnapshot(t1p1b0, 0L)); + kvSnapshotLeaseManager.getMetadataManager().registerLease("lease1", kvSnapshotLeasehandle); - kvSnapshotLeaseManager.initialize(); + kvSnapshotLeaseManager.start(); assertThat( snapshotLeaseExists( Arrays.asList( - new KvSnapshotLeaseForBucket(t0b0, 0L), - new KvSnapshotLeaseForBucket(t0b1, 0L), - new KvSnapshotLeaseForBucket(t1p0b0, 0L), - new KvSnapshotLeaseForBucket(t1p0b1, 0L), - new KvSnapshotLeaseForBucket(t1p1b0, 0L)))) + new TableBucketSnapshot(t0b0, 0L), + new TableBucketSnapshot(t0b1, 0L), + new TableBucketSnapshot(t1p0b0, 0L), + new TableBucketSnapshot(t1p0b1, 0L), + new TableBucketSnapshot(t1p1b0, 0L)))) .isTrue(); assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(5); // check detail content. + // Array sizes are dynamically determined by the maximum bucket id: + // - DATA1_TABLE_ID_PK: bucket 0 and 1 → array size 2 + // - PARTITION_ID_1: bucket 0 and 1 → array size 2 + // - PARTITION_ID_2: bucket 0 only → array size 1 Map tableIdToTableLease = new HashMap<>(); tableIdToTableLease.put( DATA1_TABLE_ID_PK, - new KvSnapshotTableLease(DATA1_TABLE_ID_PK, new Long[] {0L, 0L, -1L})); + new KvSnapshotTableLease(DATA1_TABLE_ID_PK, new Long[] {0L, 0L})); KvSnapshotTableLease leaseForPartitionTable = new KvSnapshotTableLease(PARTITION_TABLE_ID); - leaseForPartitionTable.addPartitionSnapshots(PARTITION_ID_1, new Long[] {0L, 0L, -1L}); - leaseForPartitionTable.addPartitionSnapshots(PARTITION_ID_2, new Long[] {0L, -1L, -1L}); + leaseForPartitionTable.addPartitionSnapshots(PARTITION_ID_1, new Long[] {0L, 0L}); + leaseForPartitionTable.addPartitionSnapshots(PARTITION_ID_2, new Long[] {0L}); tableIdToTableLease.put(PARTITION_TABLE_ID, leaseForPartitionTable); - KvSnapshotLease expectedLease = new KvSnapshotLease(1000L, tableIdToTableLease); - assertThat(kvSnapshotLeaseManager.getKvSnapshotLease("lease1")).isEqualTo(expectedLease); - assertThat(metadataManager.getLease("lease1")).hasValue(expectedLease); + KvSnapshotLeaseHandler expectedLease = + new KvSnapshotLeaseHandler(1000L, tableIdToTableLease); + assertThat(kvSnapshotLeaseManager.getKvSnapshotLeaseData("lease1")) + .isEqualTo(expectedLease); + assertThat(kvSnapshotLeaseManager.getMetadataManager().getLease("lease1")) + .hasValue(expectedLease); } @Test void testAcquireAndRelease() throws Exception { - Map> tableIdToRegisterBucket = initRegisterBuckets(); + Map> tableIdToRegisterBucket = initRegisterBuckets(); acquire("lease1", tableIdToRegisterBucket); // first register snapshot to zk. @@ -200,20 +184,19 @@ void testAcquireAndRelease() throws Exception { tableIdToRegisterBucket.put( PARTITION_TABLE_ID, Arrays.asList( - new KvSnapshotLeaseForBucket(t1p0b0, 1L), - new KvSnapshotLeaseForBucket(t1p0b1, 1L))); + new TableBucketSnapshot(t1p0b0, 1L), new TableBucketSnapshot(t1p0b1, 1L))); acquire("lease2", tableIdToRegisterBucket); assertThat( snapshotLeaseExists( Arrays.asList( - new KvSnapshotLeaseForBucket(t0b0, 0L), - new KvSnapshotLeaseForBucket(t0b1, 0L), - new KvSnapshotLeaseForBucket(t1p0b0, 0), - new KvSnapshotLeaseForBucket(t1p0b1, 0L), - new KvSnapshotLeaseForBucket(t1p1b0, 0L), - new KvSnapshotLeaseForBucket(t1p0b0, 1L), - new KvSnapshotLeaseForBucket(t1p0b1, 1L)))) + new TableBucketSnapshot(t0b0, 0L), + new TableBucketSnapshot(t0b1, 0L), + new TableBucketSnapshot(t1p0b0, 0), + new TableBucketSnapshot(t1p0b1, 0L), + new TableBucketSnapshot(t1p1b0, 0L), + new TableBucketSnapshot(t1p0b0, 1L), + new TableBucketSnapshot(t1p0b1, 1L)))) .isTrue(); assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(7); assertThat(kvSnapshotLeaseManager.getLeaseCount()).isEqualTo(2); @@ -222,8 +205,7 @@ void testAcquireAndRelease() throws Exception { tableIdToRegisterBucket = new HashMap<>(); zookeeperClient.registerTableBucketSnapshot(t0b0, new BucketSnapshot(1L, 10L, "test-path")); tableIdToRegisterBucket.put( - DATA1_TABLE_ID_PK, - Collections.singletonList(new KvSnapshotLeaseForBucket(t0b0, 1L))); + DATA1_TABLE_ID_PK, Collections.singletonList(new TableBucketSnapshot(t0b0, 1L))); acquire("lease1", tableIdToRegisterBucket); assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(7); @@ -235,63 +217,64 @@ void testAcquireAndRelease() throws Exception { newTableBucket, new BucketSnapshot(1L, 10L, "test-path")); tableIdToRegisterBucket.put( DATA1_TABLE_ID_PK, - Collections.singletonList(new KvSnapshotLeaseForBucket(newTableBucket, 1L))); + Collections.singletonList(new TableBucketSnapshot(newTableBucket, 1L))); acquire("lease1", tableIdToRegisterBucket); assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(8); // release - Map> tableIdToUnregisterBucket = new HashMap<>(); - tableIdToUnregisterBucket.put(DATA1_TABLE_ID_PK, Collections.singletonList(newTableBucket)); - release("lease1", tableIdToUnregisterBucket); + release("lease1", Collections.singletonList(newTableBucket)); assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(7); // release a non-exist bucket. - tableIdToUnregisterBucket = new HashMap<>(); - tableIdToUnregisterBucket.put( - DATA1_TABLE_ID_PK, + release( + "lease1", Collections.singletonList(new TableBucket(DATA1_TABLE_ID_PK, PARTITION_ID_1, 2))); - release("lease1", tableIdToUnregisterBucket); assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(7); // check detail content for lease1. + // - DATA1_TABLE_ID_PK: bucket 0,1 initially (size 2), then bucket 2 acquired and + // released → expanded to size 3: {1L, 0L, -1L} + // - PARTITION_ID_1: bucket 0,1 → array size 2 + // - PARTITION_ID_2: bucket 0 only → array size 1 Map tableIdToTableLease = new HashMap<>(); tableIdToTableLease.put( DATA1_TABLE_ID_PK, new KvSnapshotTableLease(DATA1_TABLE_ID_PK, new Long[] {1L, 0L, -1L})); KvSnapshotTableLease leaseForPartitionTable = new KvSnapshotTableLease(PARTITION_TABLE_ID); - leaseForPartitionTable.addPartitionSnapshots(PARTITION_ID_1, new Long[] {0L, 0L, -1L}); - leaseForPartitionTable.addPartitionSnapshots(PARTITION_ID_2, new Long[] {0L, -1L, -1L}); + leaseForPartitionTable.addPartitionSnapshots(PARTITION_ID_1, new Long[] {0L, 0L}); + leaseForPartitionTable.addPartitionSnapshots(PARTITION_ID_2, new Long[] {0L}); tableIdToTableLease.put(PARTITION_TABLE_ID, leaseForPartitionTable); - KvSnapshotLease expectedLease = - new KvSnapshotLease(manualClock.milliseconds() + 1000L, tableIdToTableLease); - assertThat(kvSnapshotLeaseManager.getKvSnapshotLease("lease1")).isEqualTo(expectedLease); - assertThat(metadataManager.getLease("lease1")).hasValue(expectedLease); + KvSnapshotLeaseHandler expectedLease = + new KvSnapshotLeaseHandler(manualClock.milliseconds() + 1000L, tableIdToTableLease); + assertThat(kvSnapshotLeaseManager.getKvSnapshotLeaseData("lease1")) + .isEqualTo(expectedLease); + assertThat(kvSnapshotLeaseManager.getMetadataManager().getLease("lease1")) + .hasValue(expectedLease); // check detail content for lease2. + // - PARTITION_ID_1: bucket 0,1 → array size 2 tableIdToTableLease = new HashMap<>(); leaseForPartitionTable = new KvSnapshotTableLease(PARTITION_TABLE_ID); - leaseForPartitionTable.addPartitionSnapshots(PARTITION_ID_1, new Long[] {1L, 1L, -1L}); + leaseForPartitionTable.addPartitionSnapshots(PARTITION_ID_1, new Long[] {1L, 1L}); tableIdToTableLease.put(PARTITION_TABLE_ID, leaseForPartitionTable); - KvSnapshotLease expectedLease2 = - new KvSnapshotLease(manualClock.milliseconds() + 1000L, tableIdToTableLease); - assertThat(kvSnapshotLeaseManager.getKvSnapshotLease("lease2")).isEqualTo(expectedLease2); - assertThat(metadataManager.getLease("lease2")).hasValue(expectedLease2); + KvSnapshotLeaseHandler expectedLease2 = + new KvSnapshotLeaseHandler(manualClock.milliseconds() + 1000L, tableIdToTableLease); + assertThat(kvSnapshotLeaseManager.getKvSnapshotLeaseData("lease2")) + .isEqualTo(expectedLease2); + assertThat(kvSnapshotLeaseManager.getMetadataManager().getLease("lease2")) + .hasValue(expectedLease2); } @Test - void testUnregisterAll() throws Exception { - Map> tableIdToRegisterBucket = initRegisterBuckets(); + void testDropLease() throws Exception { + Map> tableIdToRegisterBucket = initRegisterBuckets(); acquire("lease1", tableIdToRegisterBucket); assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(5); assertThat(kvSnapshotLeaseManager.getLeaseCount()).isEqualTo(1); assertThat(zookeeperClient.getKvSnapshotLeaseMetadata("lease1")).isPresent(); - Map> tableIdToUnregisterBucket = new HashMap<>(); - tableIdToUnregisterBucket.put(DATA1_TABLE_ID_PK, Arrays.asList(t0b0, t0b1)); - tableIdToUnregisterBucket.put(PARTITION_TABLE_ID, Arrays.asList(t1p0b0, t1p0b1, t1p1b0)); - - // unregister all will clear this lease. - release("lease1", tableIdToUnregisterBucket); + // release all will clear this lease. + release("lease1", Arrays.asList(t0b0, t0b1, t1p0b0, t1p0b1, t1p1b0)); assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(0); assertThat(kvSnapshotLeaseManager.getLeaseCount()).isEqualTo(0); assertThat(zookeeperClient.getKvSnapshotLeaseMetadata("lease1")).isNotPresent(); @@ -299,7 +282,7 @@ void testUnregisterAll() throws Exception { @Test void testClear() throws Exception { - Map> tableIdToRegisterBucket = initRegisterBuckets(); + Map> tableIdToRegisterBucket = initRegisterBuckets(); acquire("lease1", tableIdToRegisterBucket); // first register snapshot to zk. @@ -311,48 +294,48 @@ void testClear() throws Exception { tableIdToRegisterBucket.put( PARTITION_TABLE_ID, Arrays.asList( - new KvSnapshotLeaseForBucket(t0b0, 0L), // same ref. - new KvSnapshotLeaseForBucket(t1p0b0, 1L), - new KvSnapshotLeaseForBucket(t1p0b1, 1L))); + new TableBucketSnapshot(t0b0, 0L), // same ref. + new TableBucketSnapshot(t1p0b0, 1L), + new TableBucketSnapshot(t1p0b1, 1L))); acquire("lease2", tableIdToRegisterBucket); assertThat( snapshotLeaseExists( Arrays.asList( - new KvSnapshotLeaseForBucket(t0b0, 0L), - new KvSnapshotLeaseForBucket(t0b1, 0L), - new KvSnapshotLeaseForBucket(t1p0b0, 0L), - new KvSnapshotLeaseForBucket(t1p0b1, 0L), - new KvSnapshotLeaseForBucket(t1p1b0, 0L), - new KvSnapshotLeaseForBucket(t1p0b0, 1L), - new KvSnapshotLeaseForBucket(t1p0b1, 1L)))) + new TableBucketSnapshot(t0b0, 0L), + new TableBucketSnapshot(t0b1, 0L), + new TableBucketSnapshot(t1p0b0, 0L), + new TableBucketSnapshot(t1p0b1, 0L), + new TableBucketSnapshot(t1p1b0, 0L), + new TableBucketSnapshot(t1p0b0, 1L), + new TableBucketSnapshot(t1p0b1, 1L)))) .isTrue(); - assertThat(kvSnapshotLeaseManager.getRefCount(new KvSnapshotLeaseForBucket(t0b0, 0L))) + assertThat(kvSnapshotLeaseManager.getRefCount(new TableBucketSnapshot(t0b0, 0L))) .isEqualTo(2); assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(8); assertThat(kvSnapshotLeaseManager.getLeaseCount()).isEqualTo(2); - kvSnapshotLeaseManager.releaseAll("lease1"); - assertThat(kvSnapshotLeaseManager.getRefCount(new KvSnapshotLeaseForBucket(t0b0, 0L))) + kvSnapshotLeaseManager.dropLease("lease1"); + assertThat(kvSnapshotLeaseManager.getRefCount(new TableBucketSnapshot(t0b0, 0L))) .isEqualTo(1); assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(3); assertThat(kvSnapshotLeaseManager.getLeaseCount()).isEqualTo(1); assertThat(zookeeperClient.getKvSnapshotLeaseMetadata("lease1")).isEmpty(); - kvSnapshotLeaseManager.releaseAll("lease2"); - assertThat(kvSnapshotLeaseManager.getRefCount(new KvSnapshotLeaseForBucket(t0b0, 0L))) + kvSnapshotLeaseManager.dropLease("lease2"); + assertThat(kvSnapshotLeaseManager.getRefCount(new TableBucketSnapshot(t0b0, 0L))) .isEqualTo(0); assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(0); assertThat(kvSnapshotLeaseManager.getLeaseCount()).isEqualTo(0); assertThat(zookeeperClient.getKvSnapshotLeaseMetadata("lease2")).isEmpty(); - assertThat(kvSnapshotLeaseManager.releaseAll("non-exist")).isFalse(); + assertThat(kvSnapshotLeaseManager.dropLease("non-exist")).isFalse(); } @Test void testExpireLeases() throws Exception { // test lease expire by expire thread. - Map> tableIdToLeaseBucket = initRegisterBuckets(); + Map> tableIdToLeaseBucket = initRegisterBuckets(); // expire after 1000ms. kvSnapshotLeaseManager.acquireLease("lease1", 1000L, tableIdToLeaseBucket); @@ -365,9 +348,9 @@ void testExpireLeases() throws Exception { tableIdToLeaseBucket.put( PARTITION_TABLE_ID, Arrays.asList( - new KvSnapshotLeaseForBucket(t0b0, 0L), // same ref. - new KvSnapshotLeaseForBucket(t1p0b0, 1L), - new KvSnapshotLeaseForBucket(t1p0b1, 1L))); + new TableBucketSnapshot(t0b0, 0L), // same ref. + new TableBucketSnapshot(t1p0b0, 1L), + new TableBucketSnapshot(t1p0b1, 1L))); // expire after 2000ms. kvSnapshotLeaseManager.acquireLease("lease2", 2000L, tableIdToLeaseBucket); @@ -397,74 +380,149 @@ void testExpireLeases() throws Exception { @Test void registerWithNotExistSnapshotId() throws Exception { - Map> tableIdToRegisterBucket = new HashMap<>(); + Map> tableIdToRegisterBucket = new HashMap<>(); tableIdToRegisterBucket.put( DATA1_TABLE_ID_PK, Arrays.asList( - new KvSnapshotLeaseForBucket(t0b0, 1000L), - new KvSnapshotLeaseForBucket(t0b1, 1000L))); + new TableBucketSnapshot(t0b0, 1000L), + new TableBucketSnapshot(t0b1, 1000L))); + + // TODO this case will return t0b0, t0b1, after we add the checker the check whether + // snapshot exists. assertThat( kvSnapshotLeaseManager .acquireLease("lease1", 1000L, tableIdToRegisterBucket) .keySet()) - .contains(t0b0); - } - - private void initCoordinatorContext() { - coordinatorContext = new CoordinatorContext(); - coordinatorContext.setLiveTabletServers(createServers(Arrays.asList(0, 1, 2))); - - // register an non-partitioned table. - coordinatorContext.putTableInfo(DATA1_TABLE_INFO_PK); - coordinatorContext.putTablePath(DATA1_TABLE_ID_PK, DATA1_TABLE_PATH_PK); - - // register a partitioned table. - coordinatorContext.putTableInfo(PARTITION_TABLE_INFO); - coordinatorContext.putTablePath( - PARTITION_TABLE_INFO.getTableId(), PARTITION_TABLE_INFO.getTablePath()); - coordinatorContext.putPartition(PARTITION_ID_1, PARTITION_TABLE_PATH_1); - coordinatorContext.putPartition(PARTITION_ID_2, PARTITION_TABLE_PATH_2); + .isEmpty(); } - private Map> initRegisterBuckets() { - Map> tableIdToRegisterBucket = new HashMap<>(); + private Map> initRegisterBuckets() { + Map> tableIdToRegisterBucket = new HashMap<>(); tableIdToRegisterBucket.put( DATA1_TABLE_ID_PK, Arrays.asList( - new KvSnapshotLeaseForBucket(t0b0, 0L), - new KvSnapshotLeaseForBucket(t0b1, 0L))); + new TableBucketSnapshot(t0b0, 0L), new TableBucketSnapshot(t0b1, 0L))); tableIdToRegisterBucket.put( PARTITION_TABLE_ID, Arrays.asList( - new KvSnapshotLeaseForBucket(t1p0b0, 0L), - new KvSnapshotLeaseForBucket(t1p0b1, 0L), - new KvSnapshotLeaseForBucket(t1p1b0, 0L))); + new TableBucketSnapshot(t1p0b0, 0L), + new TableBucketSnapshot(t1p0b1, 0L), + new TableBucketSnapshot(t1p1b0, 0L))); return tableIdToRegisterBucket; } - private boolean snapshotLeaseNotExists(List bucketList) { + private boolean snapshotLeaseNotExists(List bucketList) { return bucketList.stream() .allMatch(bucket -> kvSnapshotLeaseManager.snapshotLeaseNotExist(bucket)); } - private boolean snapshotLeaseExists(List bucketList) { + private boolean snapshotLeaseExists(List bucketList) { return bucketList.stream() .noneMatch(bucket -> kvSnapshotLeaseManager.snapshotLeaseNotExist(bucket)); } - private void acquire( - String leaseId, Map> tableIdToLeaseBucket) + private void acquire(String leaseId, Map> tableIdToLeaseBucket) throws Exception { kvSnapshotLeaseManager.acquireLease(leaseId, 1000L, tableIdToLeaseBucket); } - private void release(String leaseId, Map> tableIdToReleaseBucket) - throws Exception { - kvSnapshotLeaseManager.release(leaseId, tableIdToReleaseBucket); + private void release(String leaseId, List tableBucketsToRelease) throws Exception { + kvSnapshotLeaseManager.release(leaseId, tableBucketsToRelease); + } + + private long acquire( + KvSnapshotLeaseHandler kvSnapshotLeasehandle, TableBucketSnapshot leaseForBucket) { + return kvSnapshotLeasehandle.acquireBucket( + leaseForBucket.getTableBucket(), leaseForBucket.getSnapshotId()); + } + + // ------------------------------------------------------------------------ + // Bucket Expansion Tests + // ------------------------------------------------------------------------ + + @Test + void testAcquireBucketExpansionForNonPartitionedTable() throws Exception { + // Acquire bucket 0 for non-partitioned table → creates array of size 1 + Map> buckets = new HashMap<>(); + buckets.put( + DATA1_TABLE_ID_PK, Collections.singletonList(new TableBucketSnapshot(t0b0, 0L))); + acquire("lease1", buckets); + assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(1); + + // Acquire bucket 1 → should expand array from size 1 to size 2 + buckets = new HashMap<>(); + buckets.put( + DATA1_TABLE_ID_PK, Collections.singletonList(new TableBucketSnapshot(t0b1, 0L))); + acquire("lease1", buckets); + assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(2); + + // Acquire bucket 5 (skipping buckets 2,3,4) → should expand array from size 2 to size 6 + TableBucket t0b5 = new TableBucket(DATA1_TABLE_ID_PK, 5); + zookeeperClient.registerTableBucketSnapshot(t0b5, new BucketSnapshot(0L, 0L, "test-path")); + buckets = new HashMap<>(); + buckets.put( + DATA1_TABLE_ID_PK, Collections.singletonList(new TableBucketSnapshot(t0b5, 0L))); + acquire("lease1", buckets); + assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(3); + + // Verify the array content: bucket 0,1 = 0L, bucket 2-4 = -1L, bucket 5 = 0L + Map tableIdToTableLease = new HashMap<>(); + tableIdToTableLease.put( + DATA1_TABLE_ID_PK, + new KvSnapshotTableLease( + DATA1_TABLE_ID_PK, new Long[] {0L, 0L, -1L, -1L, -1L, 0L})); + KvSnapshotLeaseHandler expectedLease = + new KvSnapshotLeaseHandler(manualClock.milliseconds() + 1000L, tableIdToTableLease); + assertThat(kvSnapshotLeaseManager.getKvSnapshotLeaseData("lease1")) + .isEqualTo(expectedLease); + } + + @Test + void testAcquireBucketExpansionForPartitionedTable() throws Exception { + // Acquire partition bucket 0 → creates array of size 1 + Map> buckets = new HashMap<>(); + buckets.put( + PARTITION_TABLE_ID, Collections.singletonList(new TableBucketSnapshot(t1p0b0, 0L))); + acquire("lease1", buckets); + assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(1); + + // Acquire partition bucket 3 (skipping bucket 1,2) → should expand array to size 4 + TableBucket t1p0b3 = new TableBucket(PARTITION_TABLE_ID, PARTITION_ID_1, 3); + zookeeperClient.registerTableBucketSnapshot( + t1p0b3, new BucketSnapshot(0L, 0L, "test-path")); + buckets = new HashMap<>(); + buckets.put( + PARTITION_TABLE_ID, Collections.singletonList(new TableBucketSnapshot(t1p0b3, 0L))); + acquire("lease1", buckets); + assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(2); + + // Verify the array content: bucket 0 = 0L, bucket 1,2 = -1L, bucket 3 = 0L + Map tableIdToTableLease = new HashMap<>(); + KvSnapshotTableLease leaseForPartitionTable = new KvSnapshotTableLease(PARTITION_TABLE_ID); + leaseForPartitionTable.addPartitionSnapshots(PARTITION_ID_1, new Long[] {0L, -1L, -1L, 0L}); + tableIdToTableLease.put(PARTITION_TABLE_ID, leaseForPartitionTable); + KvSnapshotLeaseHandler expectedLease = + new KvSnapshotLeaseHandler(manualClock.milliseconds() + 1000L, tableIdToTableLease); + assertThat(kvSnapshotLeaseManager.getKvSnapshotLeaseData("lease1")) + .isEqualTo(expectedLease); } - private long acquire(KvSnapshotLease kvSnapshotLease, KvSnapshotLeaseForBucket leaseForBucket) { - return kvSnapshotLease.acquireBucket( - leaseForBucket.getTableBucket(), leaseForBucket.getKvSnapshotId(), NUM_BUCKETS); + @Test + void testReleaseBucketExceedingArraySize() throws Exception { + // Acquire bucket 0 only → array size 1 + Map> buckets = new HashMap<>(); + buckets.put( + DATA1_TABLE_ID_PK, Collections.singletonList(new TableBucketSnapshot(t0b0, 0L))); + acquire("lease1", buckets); + assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(1); + + // Release bucket 1 which exceeds array size → should not affect existing leases + release("lease1", Collections.singletonList(t0b1)); + assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(1); + + // Release bucket 0 → should remove the lease entry + release("lease1", Collections.singletonList(t0b0)); + assertThat(kvSnapshotLeaseManager.getLeasedBucketCount()).isEqualTo(0); + assertThat(kvSnapshotLeaseManager.getLeaseCount()).isEqualTo(0); } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/lease/KvSnapshotLeaseMetadataManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/lease/KvSnapshotLeaseMetadataManagerTest.java new file mode 100644 index 0000000000..6f10400f6b --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/lease/KvSnapshotLeaseMetadataManagerTest.java @@ -0,0 +1,281 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.coordinator.lease; + +import org.apache.fluss.fs.FSDataInputStream; +import org.apache.fluss.fs.FsPath; +import org.apache.fluss.server.zk.NOPErrorHandler; +import org.apache.fluss.server.zk.ZooKeeperClient; +import org.apache.fluss.server.zk.ZooKeeperExtension; +import org.apache.fluss.server.zk.data.lease.KvSnapshotLeaseMetadata; +import org.apache.fluss.server.zk.data.lease.KvSnapshotTableLease; +import org.apache.fluss.server.zk.data.lease.KvSnapshotTableLeaseJsonSerde; +import org.apache.fluss.testutils.common.AllCallbackWrapper; +import org.apache.fluss.utils.IOUtils; +import org.apache.fluss.utils.MapUtils; + +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +import java.io.ByteArrayOutputStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Stream; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Test for {@link KvSnapshotLeaseMetadataManager}. */ +public class KvSnapshotLeaseMetadataManagerTest { + + @RegisterExtension + public static final AllCallbackWrapper ZOO_KEEPER_EXTENSION_WRAPPER = + new AllCallbackWrapper<>(new ZooKeeperExtension()); + + protected static ZooKeeperClient zookeeperClient; + private @TempDir Path tempDir; + private KvSnapshotLeaseMetadataManager metadataManager; + + @BeforeAll + static void beforeAll() { + zookeeperClient = + ZOO_KEEPER_EXTENSION_WRAPPER + .getCustomExtension() + .getZooKeeperClient(NOPErrorHandler.INSTANCE); + } + + @BeforeEach + void beforeEach() { + metadataManager = new KvSnapshotLeaseMetadataManager(zookeeperClient, tempDir.toString()); + } + + @AfterEach + void afterEach() { + ZOO_KEEPER_EXTENSION_WRAPPER.getCustomExtension().cleanupRoot(); + } + + @Test + void testGetLeasesList() throws Exception { + List leasesList = metadataManager.getLeasesList(); + assertThat(leasesList).isEmpty(); + + metadataManager.registerLease("leaseId1", new KvSnapshotLeaseHandler(1000L)); + + Map tableIdToTableLease = new HashMap<>(); + tableIdToTableLease.put(1L, new KvSnapshotTableLease(1L, new Long[] {100L, -1L})); + metadataManager.registerLease( + "leaseId2", new KvSnapshotLeaseHandler(2000L, tableIdToTableLease)); + leasesList = metadataManager.getLeasesList(); + assertThat(leasesList).containsExactlyInAnyOrder("leaseId1", "leaseId2"); + } + + @Test + void testRegisterAndUpdateLease() throws Exception { + Map tableIdToTableLease = new HashMap<>(); + tableIdToTableLease.put(1L, new KvSnapshotTableLease(1L, new Long[] {100L, -1L})); + + ConcurrentHashMap partitionSnapshots = MapUtils.newConcurrentHashMap(); + partitionSnapshots.put(1000L, new Long[] {111L, 122L}); + partitionSnapshots.put(1001L, new Long[] {122L, -1L}); + tableIdToTableLease.put(2L, new KvSnapshotTableLease(2L, partitionSnapshots)); + + KvSnapshotLeaseHandler expectedLease = + new KvSnapshotLeaseHandler(1000L, tableIdToTableLease); + metadataManager.registerLease("leaseId1", expectedLease); + + Optional lease = metadataManager.getLease("leaseId1"); + assertThat(lease).hasValue(expectedLease); + // assert zk and remote fs. + assertRemoteFsAndZkEquals("leaseId1", expectedLease); + + // test update lease. + tableIdToTableLease.remove(1L); + expectedLease = new KvSnapshotLeaseHandler(2000L, tableIdToTableLease); + metadataManager.updateLease("leaseId1", expectedLease); + lease = metadataManager.getLease("leaseId1"); + assertThat(lease).hasValue(expectedLease); + // assert zk and remote fs. + assertRemoteFsAndZkEquals("leaseId1", expectedLease); + + // test delete lease. + metadataManager.deleteLease("leaseId1"); + lease = metadataManager.getLease("leaseId1"); + assertThat(lease).isEmpty(); + } + + @Test + void testRegisterLeaseFailureOnlyCleanupCreatedFiles() throws Exception { + // Step 1: Register a lease successfully with table 1. + Map tableLeases1 = new HashMap<>(); + tableLeases1.put(1L, new KvSnapshotTableLease(1L, new Long[] {100L})); + KvSnapshotLeaseHandler lease1 = new KvSnapshotLeaseHandler(1000L, tableLeases1); + metadataManager.registerLease("leaseId1", lease1); + + // Record the original file path from ZK metadata. + Optional originalMeta = + zookeeperClient.getKvSnapshotLeaseMetadata("leaseId1"); + assertThat(originalMeta).isPresent(); + FsPath originalFile = originalMeta.get().getTableIdToRemoteMetadataFilePath().get(1L); + assertThat(originalFile.getFileSystem().exists(originalFile)).isTrue(); + + // Step 2: Try to register again with the same leaseId but a different table. + // This should fail because the ZK node already exists. + Map tableLeases2 = new HashMap<>(); + tableLeases2.put(2L, new KvSnapshotTableLease(2L, new Long[] {200L})); + KvSnapshotLeaseHandler lease2 = new KvSnapshotLeaseHandler(2000L, tableLeases2); + + assertThatThrownBy(() -> metadataManager.registerLease("leaseId1", lease2)) + .isInstanceOf(Exception.class); + + // Step 3: Verify the original file (from the first successful register) still exists. + assertThat(originalFile.getFileSystem().exists(originalFile)).isTrue(); + + // Step 4: Verify the file created for table 2 was cleaned up. + Path table2Dir = tempDir.resolve("lease/kv-snapshot/leaseId1/2"); + if (Files.exists(table2Dir)) { + try (Stream files = Files.list(table2Dir)) { + assertThat(files.filter(p -> p.toString().endsWith(".metadata")).count()) + .isEqualTo(0); + } + } + } + + @Test + void testUpdateLeaseFailureOnlyCleanupCreatedFiles() throws Exception { + // Step 1: Register a lease successfully with table 1. + Map tableLeases = new HashMap<>(); + tableLeases.put(1L, new KvSnapshotTableLease(1L, new Long[] {100L})); + KvSnapshotLeaseHandler originalLease = new KvSnapshotLeaseHandler(1000L, tableLeases); + metadataManager.registerLease("leaseId1", originalLease); + + // Record the original file path. + Optional originalMeta = + zookeeperClient.getKvSnapshotLeaseMetadata("leaseId1"); + assertThat(originalMeta).isPresent(); + FsPath originalFile = originalMeta.get().getTableIdToRemoteMetadataFilePath().get(1L); + assertThat(originalFile.getFileSystem().exists(originalFile)).isTrue(); + + // Step 2: Delete the ZK node to simulate a failure scenario. + // updateKvSnapshotLeaseMetadata will throw NoNodeException. + zookeeperClient.deleteKvSnapshotLease("leaseId1"); + + // Step 3: Try to update the lease (will fail at ZK update). + // generateMetadataFile creates new files, but they should be cleaned up on failure. + Map newTableLeases = new HashMap<>(); + newTableLeases.put(1L, new KvSnapshotTableLease(1L, new Long[] {200L})); + newTableLeases.put(2L, new KvSnapshotTableLease(2L, new Long[] {300L})); + KvSnapshotLeaseHandler updatedLease = new KvSnapshotLeaseHandler(2000L, newTableLeases); + + assertThatThrownBy(() -> metadataManager.updateLease("leaseId1", updatedLease)) + .isInstanceOf(Exception.class); + + // Step 4: Verify the original file (from the first register) is preserved on disk. + // It was NOT in createdFiles, so it should NOT be cleaned up. + assertThat(originalFile.getFileSystem().exists(originalFile)).isTrue(); + + // Step 5: Verify the newly created files for table 2 were cleaned up. + Path table2Dir = tempDir.resolve("lease/kv-snapshot/leaseId1/2"); + if (Files.exists(table2Dir)) { + try (Stream files = Files.list(table2Dir)) { + assertThat(files.filter(p -> p.toString().endsWith(".metadata")).count()) + .isEqualTo(0); + } + } + + // Verify the newly created file for table 1 (with new UUID) was also cleaned up. + // Only the original file should remain under table 1 directory. + Path table1Dir = tempDir.resolve("lease/kv-snapshot/leaseId1/1"); + try (Stream files = Files.list(table1Dir)) { + long metadataFileCount = files.filter(p -> p.toString().endsWith(".metadata")).count(); + // Only the original file should remain. + assertThat(metadataFileCount).isEqualTo(1); + } + } + + @Test + void testUpdateLeaseSuccessCleansUpOldFiles() throws Exception { + // Step 1: Register a lease with table 1. + Map tableLeases = new HashMap<>(); + tableLeases.put(1L, new KvSnapshotTableLease(1L, new Long[] {100L})); + KvSnapshotLeaseHandler originalLease = new KvSnapshotLeaseHandler(1000L, tableLeases); + metadataManager.registerLease("leaseId1", originalLease); + + // Record the original file path. + Optional originalMeta = + zookeeperClient.getKvSnapshotLeaseMetadata("leaseId1"); + assertThat(originalMeta).isPresent(); + FsPath originalFile = originalMeta.get().getTableIdToRemoteMetadataFilePath().get(1L); + assertThat(originalFile.getFileSystem().exists(originalFile)).isTrue(); + + // Step 2: Update the lease successfully with different data. + Map newTableLeases = new HashMap<>(); + newTableLeases.put(1L, new KvSnapshotTableLease(1L, new Long[] {200L})); + KvSnapshotLeaseHandler updatedLease = new KvSnapshotLeaseHandler(2000L, newTableLeases); + metadataManager.updateLease("leaseId1", updatedLease); + + // Step 3: Verify the old file was cleaned up after successful update. + assertThat(originalFile.getFileSystem().exists(originalFile)).isFalse(); + + // Step 4: Verify the new file exists and ZK points to it. + Optional newMeta = + zookeeperClient.getKvSnapshotLeaseMetadata("leaseId1"); + assertThat(newMeta).isPresent(); + FsPath newFile = newMeta.get().getTableIdToRemoteMetadataFilePath().get(1L); + assertThat(newFile.getFileSystem().exists(newFile)).isTrue(); + // The new file should be different from the old one (different UUID). + assertThat(newFile).isNotEqualTo(originalFile); + } + + private void assertRemoteFsAndZkEquals(String leaseId, KvSnapshotLeaseHandler expectedLease) + throws Exception { + Optional leaseMetadataOpt = + zookeeperClient.getKvSnapshotLeaseMetadata(leaseId); + assertThat(leaseMetadataOpt).isPresent(); + KvSnapshotLeaseMetadata leaseMetadata = leaseMetadataOpt.get(); + assertThat(leaseMetadata.getExpirationTime()).isEqualTo(expectedLease.getExpirationTime()); + Map actualFsPathSet = leaseMetadata.getTableIdToRemoteMetadataFilePath(); + Map expectedTableLeases = + expectedLease.getTableIdToTableLease(); + assertThat(actualFsPathSet).hasSize(expectedTableLeases.size()); + for (Map.Entry actualEntry : actualFsPathSet.entrySet()) { + long tableId = actualEntry.getKey(); + FsPath actualMetadataPath = actualEntry.getValue(); + assertThat(actualMetadataPath).isNotNull(); + KvSnapshotTableLease expectedTableLease = expectedTableLeases.get(tableId); + assertThat(expectedTableLease).isNotNull(); + FSDataInputStream inputStream = + actualMetadataPath.getFileSystem().open(actualMetadataPath); + try (ByteArrayOutputStream outputStream = new ByteArrayOutputStream()) { + IOUtils.copyBytes(inputStream, outputStream, true); + Assertions.assertThat( + KvSnapshotTableLeaseJsonSerde.fromJson(outputStream.toByteArray())) + .isEqualTo(expectedTableLease); + } + } + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/RebalanceManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/RebalanceManagerTest.java index 0c8a0e4415..b096a4c8b7 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/RebalanceManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/RebalanceManagerTest.java @@ -26,6 +26,7 @@ import org.apache.fluss.server.coordinator.LakeTableTieringManager; import org.apache.fluss.server.coordinator.MetadataManager; import org.apache.fluss.server.coordinator.TestCoordinatorChannelManager; +import org.apache.fluss.server.coordinator.lease.KvSnapshotLeaseManager; import org.apache.fluss.server.metadata.CoordinatorMetadataCache; import org.apache.fluss.server.metrics.group.TestingMetricGroups; import org.apache.fluss.server.zk.NOPErrorHandler; @@ -42,6 +43,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; +import java.time.Duration; import java.util.HashMap; import java.util.concurrent.Executors; @@ -64,6 +66,7 @@ public class RebalanceManagerTest { private AutoPartitionManager autoPartitionManager; private LakeTableTieringManager lakeTableTieringManager; private RebalanceManager rebalanceManager; + private KvSnapshotLeaseManager kvSnapshotLeaseManager; @BeforeAll static void baseBeforeAll() { @@ -83,6 +86,16 @@ void beforeEach() { CoordinatorEventProcessor eventProcessor = buildCoordinatorEventProcessor(); rebalanceManager = new RebalanceManager(eventProcessor, zookeeperClient); rebalanceManager.startup(); + + String remoteDataDir = "/tmp/fluss/remote-data"; + kvSnapshotLeaseManager = + new KvSnapshotLeaseManager( + Duration.ofMinutes(10).toMillis(), + zookeeperClient, + remoteDataDir, + SystemClock.getInstance(), + TestingMetricGroups.COORDINATOR_METRICS); + kvSnapshotLeaseManager.start(); } @AfterEach @@ -128,6 +141,6 @@ private CoordinatorEventProcessor buildCoordinatorEventProcessor() { new Configuration(), Executors.newFixedThreadPool(1, new ExecutorThreadFactory("test-coordinator-io")), metadataManager, - SystemClock.getInstance()); + kvSnapshotLeaseManager); } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachineTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachineTest.java index a34ff1e238..26dfeb8dc9 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachineTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachineTest.java @@ -35,6 +35,7 @@ import org.apache.fluss.server.coordinator.MetadataManager; import org.apache.fluss.server.coordinator.TestCoordinatorChannelManager; import org.apache.fluss.server.coordinator.event.CoordinatorEventManager; +import org.apache.fluss.server.coordinator.lease.KvSnapshotLeaseManager; import org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElection.ControlledShutdownLeaderElection; import org.apache.fluss.server.coordinator.statemachine.TableBucketStateMachine.ElectionResult; import org.apache.fluss.server.metadata.CoordinatorMetadataCache; @@ -87,6 +88,7 @@ class TableBucketStateMachineTest { private AutoPartitionManager autoPartitionManager; private LakeTableTieringManager lakeTableTieringManager; private CoordinatorMetadataCache serverMetadataCache; + private KvSnapshotLeaseManager kvSnapshotLeaseManager; @BeforeAll static void baseBeforeAll() { @@ -100,7 +102,8 @@ static void baseBeforeAll() { void beforeEach() throws IOException { Configuration conf = new Configuration(); conf.setString(ConfigOptions.COORDINATOR_HOST, "localhost"); - conf.setString(ConfigOptions.REMOTE_DATA_DIR, "/tmp/fluss/remote-data"); + String remoteDir = "/tmp/fluss/remote-data"; + conf.setString(ConfigOptions.REMOTE_DATA_DIR, remoteDir); coordinatorContext = new CoordinatorContext(); testCoordinatorChannelManager = new TestCoordinatorChannelManager(); coordinatorRequestBatch = @@ -120,6 +123,15 @@ void beforeEach() throws IOException { new LakeCatalogDynamicLoader(new Configuration(), null, true)), new Configuration()); lakeTableTieringManager = new LakeTableTieringManager(); + + kvSnapshotLeaseManager = + new KvSnapshotLeaseManager( + Duration.ofMinutes(10).toMillis(), + zookeeperClient, + remoteDir, + SystemClock.getInstance(), + TestingMetricGroups.COORDINATOR_METRICS); + kvSnapshotLeaseManager.start(); } @Test @@ -270,7 +282,7 @@ void testStateChangeToOnline() throws Exception { zookeeperClient, new Configuration(), new LakeCatalogDynamicLoader(new Configuration(), null, true)), - SystemClock.getInstance()); + kvSnapshotLeaseManager); CoordinatorEventManager eventManager = new CoordinatorEventManager( coordinatorEventProcessor, TestingMetricGroups.COORDINATOR_METRICS); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java index da4e8ef994..4cf0a511e7 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java @@ -42,7 +42,6 @@ import org.apache.fluss.server.zk.data.TableAssignment; import org.apache.fluss.server.zk.data.TableRegistration; import org.apache.fluss.server.zk.data.TabletServerRegistration; -import org.apache.fluss.server.zk.data.lease.KvSnapshotLease; import org.apache.fluss.server.zk.data.lease.KvSnapshotLeaseMetadata; import org.apache.fluss.shaded.curator5.org.apache.curator.CuratorZookeeperClient; import org.apache.fluss.shaded.curator5.org.apache.curator.framework.CuratorFramework; @@ -737,13 +736,4 @@ void testGetDatabaseSummary() throws Exception { .isGreaterThanOrEqualTo(beforeCreateTime) .isLessThanOrEqualTo(afterCreateTime); } - - private void registerBucket( - KvSnapshotLease consumer, TableBucket tb, long kvSnapshotId, int bucketNum) { - consumer.acquireBucket(tb, kvSnapshotId, bucketNum); - } - - private void unregisterBucket(KvSnapshotLease consumer, TableBucket tb) { - consumer.releaseBucket(tb); - } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataManagerTest.java deleted file mode 100644 index 1e1ad7fb23..0000000000 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataManagerTest.java +++ /dev/null @@ -1,146 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.fluss.server.zk.data.lease; - -import org.apache.fluss.fs.FSDataInputStream; -import org.apache.fluss.fs.FsPath; -import org.apache.fluss.server.zk.NOPErrorHandler; -import org.apache.fluss.server.zk.ZooKeeperClient; -import org.apache.fluss.server.zk.ZooKeeperExtension; -import org.apache.fluss.testutils.common.AllCallbackWrapper; -import org.apache.fluss.utils.IOUtils; - -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.RegisterExtension; -import org.junit.jupiter.api.io.TempDir; - -import java.io.ByteArrayOutputStream; -import java.nio.file.Path; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; - -import static org.assertj.core.api.Assertions.assertThat; - -/** Test for {@link KvSnapshotLeaseMetadataManager}. */ -public class KvSnapshotLeaseMetadataManagerTest { - - @RegisterExtension - public static final AllCallbackWrapper ZOO_KEEPER_EXTENSION_WRAPPER = - new AllCallbackWrapper<>(new ZooKeeperExtension()); - - protected static ZooKeeperClient zookeeperClient; - private @TempDir Path tempDir; - private KvSnapshotLeaseMetadataManager metadataManager; - - @BeforeAll - static void beforeAll() { - zookeeperClient = - ZOO_KEEPER_EXTENSION_WRAPPER - .getCustomExtension() - .getZooKeeperClient(NOPErrorHandler.INSTANCE); - } - - @BeforeEach - void beforeEach() { - metadataManager = new KvSnapshotLeaseMetadataManager(zookeeperClient, tempDir.toString()); - } - - @AfterEach - void afterEach() { - ZOO_KEEPER_EXTENSION_WRAPPER.getCustomExtension().cleanupRoot(); - } - - @Test - void testGetLeasesList() throws Exception { - List leasesList = metadataManager.getLeasesList(); - assertThat(leasesList).isEmpty(); - - metadataManager.registerLease("leaseId1", new KvSnapshotLease(1000L)); - - Map tableIdToTableLease = new HashMap<>(); - tableIdToTableLease.put(1L, new KvSnapshotTableLease(1L, new Long[] {100L, -1L})); - metadataManager.registerLease("leaseId2", new KvSnapshotLease(2000L, tableIdToTableLease)); - leasesList = metadataManager.getLeasesList(); - assertThat(leasesList).containsExactlyInAnyOrder("leaseId1", "leaseId2"); - } - - @Test - void testRegisterAndUpdateLease() throws Exception { - Map tableIdToTableLease = new HashMap<>(); - tableIdToTableLease.put(1L, new KvSnapshotTableLease(1L, new Long[] {100L, -1L})); - - Map partitionSnapshots = new HashMap<>(); - partitionSnapshots.put(1000L, new Long[] {111L, 122L}); - partitionSnapshots.put(1001L, new Long[] {122L, -1L}); - tableIdToTableLease.put(2L, new KvSnapshotTableLease(2L, partitionSnapshots)); - - KvSnapshotLease expectedLease = new KvSnapshotLease(1000L, tableIdToTableLease); - metadataManager.registerLease("leaseId1", expectedLease); - - Optional lease = metadataManager.getLease("leaseId1"); - assertThat(lease).hasValue(expectedLease); - // assert zk and remote fs. - assertRemoteFsAndZkEquals("leaseId1", expectedLease); - - // test update lease. - tableIdToTableLease.remove(1L); - expectedLease = new KvSnapshotLease(2000L, tableIdToTableLease); - metadataManager.updateLease("leaseId1", expectedLease); - lease = metadataManager.getLease("leaseId1"); - assertThat(lease).hasValue(expectedLease); - // assert zk and remote fs. - assertRemoteFsAndZkEquals("leaseId1", expectedLease); - - // test delete lease. - metadataManager.deleteLease("leaseId1"); - lease = metadataManager.getLease("leaseId1"); - assertThat(lease).isEmpty(); - } - - private void assertRemoteFsAndZkEquals(String leaseId, KvSnapshotLease expectedLease) - throws Exception { - Optional leaseMetadataOpt = - zookeeperClient.getKvSnapshotLeaseMetadata(leaseId); - assertThat(leaseMetadataOpt).isPresent(); - KvSnapshotLeaseMetadata leaseMetadata = leaseMetadataOpt.get(); - assertThat(leaseMetadata.getExpirationTime()).isEqualTo(expectedLease.getExpirationTime()); - Map actualFsPathSet = leaseMetadata.getTableIdToRemoteMetadataFilePath(); - Map expectedTableLeases = - expectedLease.getTableIdToTableLease(); - assertThat(actualFsPathSet).hasSize(expectedTableLeases.size()); - for (Map.Entry actualEntry : actualFsPathSet.entrySet()) { - long tableId = actualEntry.getKey(); - FsPath actualMetadataPath = actualEntry.getValue(); - assertThat(actualMetadataPath).isNotNull(); - KvSnapshotTableLease expectedTableLease = expectedTableLeases.get(tableId); - assertThat(expectedTableLease).isNotNull(); - FSDataInputStream inputStream = - actualMetadataPath.getFileSystem().open(actualMetadataPath); - try (ByteArrayOutputStream outputStream = new ByteArrayOutputStream()) { - IOUtils.copyBytes(inputStream, outputStream, true); - assertThat(KvSnapshotTableLeaseJsonSerde.fromJson(outputStream.toByteArray())) - .isEqualTo(expectedTableLease); - } - } - } -} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLeaseJsonSerdeTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLeaseJsonSerdeTest.java index de77bb7779..e98c3b3e97 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLeaseJsonSerdeTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLeaseJsonSerdeTest.java @@ -17,10 +17,10 @@ package org.apache.fluss.server.zk.data.lease; +import org.apache.fluss.utils.MapUtils; import org.apache.fluss.utils.json.JsonSerdeTestBase; -import java.util.HashMap; -import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; /** Test for {@link KvSnapshotTableLeaseJsonSerde}. */ public class KvSnapshotTableLeaseJsonSerdeTest extends JsonSerdeTestBase { @@ -34,7 +34,7 @@ protected KvSnapshotTableLease[] createObjects() { KvSnapshotTableLease[] kvSnapshotTableLeases = new KvSnapshotTableLease[2]; kvSnapshotTableLeases[0] = new KvSnapshotTableLease(1L, new Long[] {1L, -1L, 1L, 2L}); - Map partitionSnapshots = new HashMap<>(); + ConcurrentHashMap partitionSnapshots = MapUtils.newConcurrentHashMap(); partitionSnapshots.put(2001L, new Long[] {10L, -1L, 20L, 30L}); partitionSnapshots.put(2002L, new Long[] {15L, -1L, 25L, 35L}); kvSnapshotTableLeases[1] = new KvSnapshotTableLease(2L, partitionSnapshots); diff --git a/website/docs/_configs/_partial_config.mdx b/website/docs/_configs/_partial_config.mdx index d410834f64..74bce1ad6e 100644 --- a/website/docs/_configs/_partial_config.mdx +++ b/website/docs/_configs/_partial_config.mdx @@ -78,7 +78,7 @@ | `client.lookup.batch-timeout` | `0 s` | Duration | The maximum time to wait for the lookup batch to full, if this timeout is reached, the lookup batch will be closed to send. | | `client.lookup.max-retries` | `2147483647` | Integer | Setting a value greater than zero will cause the client to resend any lookup request that fails with a potentially transient error. | | `client.scanner.remote-log.prefetch-num` | `4` | Integer | The number of remote log segments to keep in local temp file for LogScanner, which download from remote storage. The default setting is 4. | -| `client.scanner.io.tmpdir` | `/tmp/fluss` | String | Local directory that is used by client for storing the data files (like kv snapshot, log segment files) to read temporarily | +| `client.scanner.io.tmpdir` | `/var/folders/ck/q6rhkhq9759b3nwc7501p6780000gp/T//fluss` | String | Local directory that is used by client for storing the data files (like kv snapshot, log segment files) to read temporarily | | `client.remote-file.download-thread-num` | `3` | Integer | The number of threads the client uses to download remote files. | | `client.filesystem.security.token.renewal.backoff` | `1 hours` | Duration | The time period how long to wait before retrying to obtain new security tokens for filesystem after a failure. | | `client.filesystem.security.token.renewal.time-ratio` | `0.75` | Double | Ratio of the token's expiration time when new credentials for access filesystem should be re-obtained. | @@ -95,6 +95,8 @@ | `coordinator.host` | `none` | String | The config parameter defining the network address to connect to for communication with the coordinator server. If the coordinator server is used as a bootstrap server (discover all the servers in the cluster), the value of this config option should be a static hostname or address.This option is deprecated. Please use bind.listeners instead, which provides a more flexible configuration for multiple ports | | `coordinator.port` | `9123` | String | The config parameter defining the network port to connect to for communication with the coordinator server. Like coordinator.host, if the coordinator server is used as a bootstrap server (discover all the servers in the cluster), the value of this config option should be a static port. Otherwise, the value can be set to "0" for a dynamic service name resolution. The value accepts a list of ports (“50100,50101”), ranges (“50100-50200”) or a combination of both.This option is deprecated. Please use bind.listeners instead, which provides a more flexible configuration for multiple ports | | `coordinator.io-pool.size` | `10` | Integer | The size of the IO thread pool to run blocking operations for coordinator server. This includes discard unnecessary snapshot files. Increase this value if you experience slow unnecessary snapshot files clean. The default value is 10. This option is deprecated. Please use server.io-pool.size instead. | +| `coordinator.producer-offsets.ttl` | `24 hours` | Duration | The TTL (time-to-live) for producer offsets. Producer offsets older than this TTL will be automatically cleaned up by the coordinator server. Default is 24 hours. | +| `coordinator.producer-offsets.cleanup-interval` | `1 hours` | Duration | The interval for cleaning up expired producer offsets and orphan files in remote storage. Default is 1 hour. | ## Data Configurations @@ -137,7 +139,8 @@ | `kv.snapshot.interval` | `10 min` | Duration | The interval to perform periodic snapshot for kv data. The default setting is 10 minutes. | | `kv.snapshot.scheduler-thread-num` | `1` | Integer | The number of threads that the server uses to schedule snapshot kv data for all the replicas in the server. | | `kv.snapshot.transfer-thread-num` | `4` | Integer | The number of threads the server uses to transfer (download and upload) kv snapshot files. This option is deprecated. Please use server.io-pool.size instead. | -| `kv.snapshot.num-retained` | `1` | Integer | The maximum number of completed snapshots to retain. | +| `kv.snapshot.num-retained` | `2` | Integer | The maximum number of completed snapshots to retain. | +| `kv.snapshot.lease.expiration-check-interval` | `10 min` | Duration | The interval to check the expiration of kv snapshot lease. The default setting is 10 minutes. | | `kv.rocksdb.thread.num` | `2` | Integer | The maximum number of concurrent background flush and compaction jobs (per bucket of table). The default value is `2`. | | `kv.rocksdb.files.open` | `-1` | Integer | The maximum number of open files (per bucket of table) that can be used by the DB, `-1` means no limit. The default value is `-1`. | | `kv.rocksdb.log.max-file-size` | `25 mb` | MemorySize | The maximum size of RocksDB's file used for information logging. If the log files becomes larger than this, a new file will be created. If 0, all logs will be written to one log file. The default maximum file size is `25MB`. | @@ -206,6 +209,12 @@ | :--- | :--- | :--- | :--- | | `metrics.reporters` | `none` | String | An optional list of reporter names. If configured, only reporters whose name matches in the list will be started | | `metrics.reporter.prometheus.port` | `9249` | String | The port the Prometheus reporter listens on.In order to be able to run several instances of the reporter on one host (e.g. when one TabletServer is colocated with the CoordinatorServer) it is advisable to use a port range like 9250-9260. | +| `metrics.reporter.prometheus-push.host-url` | `none` | String | The PushGateway server host URL including scheme, host name, and port. | +| `metrics.reporter.prometheus-push.job-name` | `none` | String | The job name under which metrics will be pushed | +| `metrics.reporter.prometheus-push.random-job-name-suffix` | `true` | Boolean | Specifies whether a random suffix should be appended to the job name. This is useful when multiple instances of the reporter are running on the same host. | +| `metrics.reporter.prometheus-push.delete-on-shutdown` | `true` | Boolean | Specifies whether to delete metrics from the PushGateway on shutdown. Fluss will try its best to delete the metrics but this is not guaranteed. | +| `metrics.reporter.prometheus-push.grouping-key` | `none` | String | Specifies the grouping key which is the group and global labels of all metrics. The label name and value are separated by '=', and labels are separated by ';', e.g., k1=v1;k2=v2. | +| `metrics.reporter.prometheus-push.push-interval` | `10 s` | Duration | The interval of pushing metrics to Prometheus PushGateway. | | `metrics.reporter.jmx.port` | `none` | String | The port for the JMXServer that JMX clients can connect to. If not set, the JMXServer won't start. In order to be able to run several instances of the reporter on one host (e.g. when one TabletServer is colocated with the CoordinatorServer) it is advisable to use a port range like 9990-9999. | ## Netty Configurations @@ -271,10 +280,11 @@ | `table.log.arrow.compression.type` | `ZSTD` | ArrowCompressionType | The compression type of the log records if the log format is set to `ARROW`. The candidate compression type is [NONE, LZ4_FRAME, ZSTD] | | `table.log.arrow.compression.zstd.level` | `3` | Integer | The compression level of ZSTD for the log records if the log format is set to `ARROW` and the compression type is set to `ZSTD`. The valid range is 1 to 22. | | `table.kv.format` | `COMPACTED` | KvFormat | The format of the kv records in kv store. The default value is `compacted`. The supported formats are `compacted` and `indexed`. | +| `table.kv.format-version` | `none` | String | The version of the kv format. Automatically set by the coordinator during table creation if not configured by users. Note: The datalake encoding and bucketing strategy mentioned below only takes effect when 'datalake.format' is configured at cluster level. Version Behaviors: (1) Version 1: Tables created before 'table.kv.format-version' was introduced are treated as version 1. Uses datalake's encoder (e.g., Paimon/Iceberg) for both primary key and bucket key encoding. This may not support prefix lookup properly because some datalake encoders (like Paimon) don't guarantee that encoded bucket key bytes are a prefix of encoded primary key bytes. (2) Version 2 (current): New tables use Fluss's default encoder for primary key encoding when bucket key differs from primary key, which ensures proper prefix lookup support. When bucket key equals primary key (default bucket key), it still uses datalake's encoder for optimization (encoded bytes can be reused for bucket calculation). Bucket key encoding always uses datalake's encoder to align with datalake bucket calculation. | | `table.auto-partition.enabled` | `false` | Boolean | Whether enable auto partition for the table. Disable by default. When auto partition is enabled, the partitions of the table will be created automatically. | | `table.auto-partition.key` | `none` | String | This configuration defines the time-based partition key to be used for auto-partitioning when a table is partitioned with multiple keys. Auto-partitioning utilizes a time-based partition key to handle partitions automatically, including creating new ones and removing outdated ones, by comparing the time value of the partition with the current system time. In the case of a table using multiple partition keys (such as a composite partitioning strategy), this feature determines which key should serve as the primary time dimension for making auto-partitioning decisions.And If the table has only one partition key, this config is not necessary. Otherwise, it must be specified. | | `table.auto-partition.time-unit` | `DAY` | AutoPartitionTimeUnit | The time granularity for auto created partitions. The default value is `DAY`. Valid values are `HOUR`, `DAY`, `MONTH`, `QUARTER`, `YEAR`. If the value is `HOUR`, the partition format for auto created is yyyyMMddHH. If the value is `DAY`, the partition format for auto created is yyyyMMdd. If the value is `MONTH`, the partition format for auto created is yyyyMM. If the value is `QUARTER`, the partition format for auto created is yyyyQ. If the value is `YEAR`, the partition format for auto created is yyyy. | -| `table.auto-partition.time-zone` | `Etc/UTC` | String | The time zone for auto partitions, which is by default the same as the system time zone. | +| `table.auto-partition.time-zone` | `Asia/Shanghai` | String | The time zone for auto partitions, which is by default the same as the system time zone. | | `table.auto-partition.num-precreate` | `2` | Integer | The number of partitions to pre-create for auto created partitions in each check for auto partition. For example, if the current check time is 2024-11-11 and the value is configured as 3, then partitions 20241111, 20241112, 20241113 will be pre-created. If any one partition exists, it'll skip creating the partition. The default value is 2, which means 2 partitions will be pre-created. If the `table.auto-partition.time-unit` is `DAY`(default), one precreated partition is for today and another one is for tomorrow.For a partition table with multiple partition keys, pre-create is unsupported and will be set to 0 automatically when creating table if it is not explicitly specified. | | `table.auto-partition.num-retention` | `7` | Integer | The number of history partitions to retain for auto created partitions in each check for auto partition. For example, if the current check time is 2024-11-11, time-unit is DAY, and the value is configured as 3, then the history partitions 20241108, 20241109, 20241110 will be retained. The partitions earlier than 20241108 will be deleted. The default value is 7. | | `table.log.ttl` | `168 hours` | Duration | The time to live for log segments. The configuration controls the maximum time we will retain a log before we will delete old segments to free up space. If set to -1, the log will not be deleted. | diff --git a/website/docs/engine-flink/options.md b/website/docs/engine-flink/options.md index d70c71f416..6060fde451 100644 --- a/website/docs/engine-flink/options.md +++ b/website/docs/engine-flink/options.md @@ -98,7 +98,7 @@ See more details about [ALTER TABLE ... SET](engine-flink/ddl.md#set-properties) | scan.startup.mode | Enum | full | The scan startup mode enables you to specify the starting point for data consumption. Fluss currently supports the following `scan.startup.mode` options: `full` (default), earliest, latest, timestamp. See the [Start Reading Position](engine-flink/reads.md#start-reading-position) for more details. | | scan.startup.timestamp | Long | (None) | The timestamp to start reading the data from. This option is only valid when `scan.startup.mode` is set to `timestamp`. The format is 'milli-second-since-epoch' or `yyyy-MM-dd HH:mm:ss`, like `1678883047356` or `2023-12-09 23:09:12`. | | scan.partition.discovery.interval | Duration | 1min | The time interval for the Fluss source to discover the new partitions for partitioned table while scanning. A non-positive value disables the partition discovery. The default value is 1 minute. Currently, since Fluss Admin#listPartitions(TablePath tablePath) requires a large number of requests to ZooKeeper in server, this option cannot be set too small, as a small value would cause frequent requests and increase server load. In the future, once list partitions is optimized, the default value of this parameter can be reduced. | -| scan.kv.snapshot.lease.id | String | UUID | The lease id to lease kv snapshots. If set, the acquired kv snapshots will not be deleted until the consumer finished consuming all the snapshots or the lease duration time is reached. If not set, an UUID will be set. | +| scan.kv.snapshot.lease.id | String | UUID | The lease ID used to protect acquired KV snapshots from deletion. If specified, the snapshots will be retained until either the consumer finishes processing all of them or the lease duration expires. By default, this value is set to a randomly generated UUID string if not explicitly provided. | | scan.kv.snapshot.lease.duration | Duration | 1day | The time period how long to wait before expiring the kv snapshot lease to avoid kv snapshot blocking to delete. | | client.scanner.log.check-crc | Boolean | true | Automatically check the CRC3 of the read records for LogScanner. This ensures no on-the-wire or on-disk corruption to the messages occurred. This check adds some overhead, so it may be disabled in cases seeking extreme performance. | | client.scanner.log.max-poll-records | Integer | 500 | The maximum number of records returned in a single call to poll() for LogScanner. Note that this config doesn't impact the underlying fetching behavior. The Scanner will cache the records from each fetch request and returns them incrementally from each poll. | diff --git a/website/docs/engine-flink/procedures.md b/website/docs/engine-flink/procedures.md index 63d64f4866..9cd9cb1d64 100644 --- a/website/docs/engine-flink/procedures.md +++ b/website/docs/engine-flink/procedures.md @@ -491,18 +491,18 @@ CALL sys.cancel_rebalance('rebalance-12345'); ## kv snapshot lease -Fluss provides procedures to manage KV snapshot leases, allowing you to release leased kv snapshots. +Fluss provides procedures to manage KV snapshot leases, allowing you to drop leased kv snapshots. -### release_all_kv_snapshot_lease +### drop_kv_snapshot_lease -Release all KV snapshots leased under a specified leaseId. This is typically used for handle the scenario of lease -remnants. After a normal job completion, the registered lease is not released, requiring a manual trigger of the -procedure for cleanup. +Drop KV snapshots leased under a specified leaseId. This is typically used for handle the scenario of lease +remnants. After an abnormal job termination (e.g., crash or forced cancellation), the registered lease may not +be released automatically and could require manual cleanup. **Syntax:** ```sql -CALL [catalog_name.]sys.release_all_kv_snapshot_lease( +CALL [catalog_name.]sys.drop_kv_snapshot_lease( leaseId => 'STRING' ) ``` @@ -519,6 +519,6 @@ CALL [catalog_name.]sys.release_all_kv_snapshot_lease( -- Use the Fluss catalog (replace 'fluss_catalog' with your catalog name if different) USE fluss_catalog; --- Release all KV snapshots leased under the given leaseId -CALL sys.release_all_kv_snapshot_lease('test-lease-id'); +-- Drop KV snapshots leased under the given leaseId +CALL sys.drop_kv_snapshot_lease('test-lease-id'); ``` \ No newline at end of file From d6b98d03eb6c90fe0b63fd45a09dc6d4fb56069a Mon Sep 17 00:00:00 2001 From: yunhong <337361684@qq.com> Date: Sun, 8 Feb 2026 18:35:22 +0800 Subject: [PATCH 3/6] refactor kv snapshot store --- .../kv/snapshot/CompletedSnapshotStore.java | 124 ++++++--- .../kv/snapshot/SharedKvFileRegistry.java | 57 +++- .../CoordinatorEventProcessorTest.java | 12 +- .../rebalance/RebalanceManagerTest.java | 13 +- .../snapshot/CompletedSnapshotStoreTest.java | 244 ++++++++++++++++++ .../kv/snapshot/SharedKvFileRegistryTest.java | 90 +++++++ 6 files changed, 491 insertions(+), 49 deletions(-) diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStore.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStore.java index e1a431c3d4..cd008da8dc 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStore.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStore.java @@ -32,11 +32,14 @@ import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.Deque; +import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.concurrent.Executor; import java.util.concurrent.locks.ReentrantLock; @@ -77,6 +80,18 @@ public class CompletedSnapshotStore { */ private final ArrayDeque completedSnapshots; + /** + * Snapshots that are still in use by a lease but have been moved out of the standard retention + * window. These snapshots are protected from deletion (both metadata and SST files) until their + * lease expires. + * + *

When a snapshot should be subsumed (beyond the retention window) but cannot be (because + * it's leased), it is moved here instead of staying in {@link #completedSnapshots}. This + * ensures the effective {@code lowestSnapshotID} is computed from retained snapshots only, + * allowing SST files from non-leased subsumed snapshots to be cleaned up properly. + */ + @VisibleForTesting final Map stillInUseSnapshots = new HashMap<>(); + public CompletedSnapshotStore( int maxNumberOfSnapshotsToRetain, SharedKvFileRegistry sharedKvFileRegistry, @@ -139,32 +154,80 @@ CompletedSnapshot addSnapshotAndSubsumeOldestOne( // Remove completed snapshot from queue and snapshotStateHandleStore, not // discard. - Optional subsume = - subsume( - completedSnapshots, - maxNumberOfSnapshotsToRetain, - completedSnapshot -> { - remove( - completedSnapshot.getTableBucket(), - completedSnapshot.getSnapshotID()); - snapshotsCleaner.addSubsumedSnapshot(completedSnapshot); - }, - subsumptionChecker); - + subsume( + completedSnapshots, + maxNumberOfSnapshotsToRetain, + completedSnapshot -> { + remove( + completedSnapshot.getTableBucket(), + completedSnapshot.getSnapshotID()); + snapshotsCleaner.addSubsumedSnapshot(completedSnapshot); + }, + subsumptionChecker); + + // Move leased snapshots that should have been subsumed but couldn't + // (protected by a lease) from completedSnapshots to stillInUseSnapshots. + // This ensures the effective lowestSnapshotID is computed from retained + // (non-leased) snapshots only, allowing SST files from non-leased + // subsumed snapshots to be cleaned up properly. + CompletedSnapshot latest = completedSnapshots.peekLast(); + Iterator leaseIt = completedSnapshots.iterator(); + while (leaseIt.hasNext()) { + CompletedSnapshot next = leaseIt.next(); + if (next != latest + && !subsumptionChecker.canSubsume( + new TableBucketSnapshot( + next.getTableBucket(), next.getSnapshotID()))) { + leaseIt.remove(); + stillInUseSnapshots.put(next.getSnapshotID(), next); + LOG.debug( + "Moved leased snapshot {} to stillInUseSnapshots", + next.getSnapshotID()); + } + } + + // Check if any previously still-in-use snapshots can now be released + // (lease expired). + Iterator> stillInUseIter = + stillInUseSnapshots.entrySet().iterator(); + while (stillInUseIter.hasNext()) { + Map.Entry entry = stillInUseIter.next(); + CompletedSnapshot s = entry.getValue(); + if (subsumptionChecker.canSubsume( + new TableBucketSnapshot(s.getTableBucket(), s.getSnapshotID()))) { + stillInUseIter.remove(); + try { + remove(s.getTableBucket(), s.getSnapshotID()); + } catch (Exception e) { + LOG.warn( + "Failed to remove released snapshot {} from store", + s.getSnapshotID(), + e); + } + snapshotsCleaner.addSubsumedSnapshot(s); + LOG.debug( + "Released snapshot {} from stillInUseSnapshots (lease expired)", + s.getSnapshotID()); + } + } + + // SST file cleanup: compute effective lowest from retained (non-leased) + // snapshots only, and protect files referenced by still-in-use snapshots. + Set stillInUseIds = new HashSet<>(stillInUseSnapshots.keySet()); findLowest(completedSnapshots) .ifPresent( - id -> { - // unregister the unused kv file, which will then cause the - // kv file - // deletion - sharedKvFileRegistry.unregisterUnusedKvFile(id); - snapshotsCleaner.cleanSubsumedSnapshots( - id, - Collections.emptySet(), - postCleanup, - ioExecutor); - }); - return subsume.orElse(null); + id -> + sharedKvFileRegistry.unregisterUnusedKvFile( + id, stillInUseIds)); + + // Snapshot metadata/private files cleanup: use the latest snapshot + // ID + 1 so subsumed snapshots can be cleaned even when a lower + // snapshot has a lease. This is safe because + // KvSnapshotHandle.discard() only deletes private files and + // metadata, not shared SST files registered in SharedKvFileRegistry. + snapshotsCleaner.cleanSubsumedSnapshots( + snapshot.getSnapshotID() + 1, stillInUseIds, postCleanup, ioExecutor); + return null; }); } @@ -172,26 +235,20 @@ public List getAllSnapshots() { return inLock(lock, () -> new ArrayList<>(completedSnapshots)); } - private static Optional subsume( + private static void subsume( Deque snapshots, int numRetain, SubsumeAction subsumeAction, SubsumptionChecker subsumptionChecker) { if (snapshots.isEmpty()) { - return Optional.empty(); + return; } CompletedSnapshot latest = snapshots.peekLast(); - Optional lastSubsumedSnapshot = Optional.empty(); Iterator iterator = snapshots.iterator(); while (snapshots.size() > numRetain && iterator.hasNext()) { CompletedSnapshot next = iterator.next(); if (canSubsume(next, latest, subsumptionChecker)) { - // always return the subsumed snapshot with larger snapshot id. - if (!lastSubsumedSnapshot.isPresent() - || next.getSnapshotID() > lastSubsumedSnapshot.get().getSnapshotID()) { - lastSubsumedSnapshot = Optional.of(next); - } iterator.remove(); try { subsumeAction.subsume(next); @@ -200,7 +257,6 @@ private static Optional subsume( } } } - return lastSubsumedSnapshot; } @FunctionalInterface diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/SharedKvFileRegistry.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/SharedKvFileRegistry.java index 9d13d3e0e9..558e847d96 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/SharedKvFileRegistry.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/SharedKvFileRegistry.java @@ -23,11 +23,13 @@ import org.slf4j.LoggerFactory; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Set; import java.util.concurrent.Executor; import static org.apache.fluss.utils.Preconditions.checkNotNull; @@ -127,10 +129,31 @@ public KvFileHandle registerReference( } public void unregisterUnusedKvFile(long lowestSnapshotID) { + unregisterUnusedKvFile(lowestSnapshotID, Collections.emptySet()); + } + + /** + * Unregister and delete KV files that are no longer used by any retained or still-in-use + * (leased) snapshot. + * + *

A KV file is safe to delete only if: + * + *

    + *
  1. Its {@code lastUsedSnapshotID < lowestSnapshotID} (not used by any retained snapshot) + *
  2. No snapshot in {@code stillInUseSnapshotIds} falls within the file's referencing range + * [{@code createdBySnapshotID}, {@code lastUsedSnapshotID}] + *
+ * + * @param lowestSnapshotID the effective lowest snapshot ID from retained (non-leased) snapshots + * @param stillInUseSnapshotIds snapshot IDs that are protected by leases + */ + public void unregisterUnusedKvFile(long lowestSnapshotID, Set stillInUseSnapshotIds) { // delete kv files that aren't used LOG.debug( - "Discard kv files created before snapshot {} and not used afterwards", - lowestSnapshotID); + "Discard kv files created before snapshot {} and not used afterwards, " + + "stillInUse snapshots: {}", + lowestSnapshotID, + stillInUseSnapshotIds); List subsumed = new ArrayList<>(); // Iterate over all the registered kv file handles. // Using a simple loop and NOT index by snapshotID because: @@ -140,7 +163,8 @@ public void unregisterUnusedKvFile(long lowestSnapshotID) { Iterator it = registeredKvEntries.values().iterator(); while (it.hasNext()) { SharedKvEntry entry = it.next(); - if (entry.lastUsedSnapshotID < lowestSnapshotID) { + if (entry.lastUsedSnapshotID < lowestSnapshotID + && !isReferencedByStillInUse(entry, stillInUseSnapshotIds)) { subsumed.add(entry.kvFileHandle); it.remove(); fileSize -= entry.kvFileHandle.getSize(); @@ -153,6 +177,23 @@ public void unregisterUnusedKvFile(long lowestSnapshotID) { } } + /** + * Check if a KV file entry is referenced by any still-in-use (leased) snapshot. A file is + * referenced by a snapshot if the snapshot ID falls within the file's usage range [{@code + * createdBySnapshotID}, {@code lastUsedSnapshotID}]. + * + *

Note: {@code createdBySnapshotID} serves as the first snapshot ID that uses this file + * (i.e., firstUsedSnapshotID), while {@code lastUsedSnapshotID} is the last. + */ + private boolean isReferencedByStillInUse(SharedKvEntry entry, Set stillInUseSnapshotIds) { + for (Long snapshotId : stillInUseSnapshotIds) { + if (snapshotId >= entry.createdBySnapshotID && snapshotId <= entry.lastUsedSnapshotID) { + return true; + } + } + return false; + } + public void registerAll(KvSnapshotHandle kvSnapshotHandle, long snapshotID) { if (kvSnapshotHandle == null) { return; @@ -211,7 +252,17 @@ public void run() { private static final class SharedKvEntry { + /** + * The snapshot ID that first created/registered this KV file. This effectively serves as + * {@code firstUsedSnapshotID} — the lower bound of the snapshot range referencing this + * file. + */ private final long createdBySnapshotID; + + /** + * The snapshot ID that last referenced this KV file — the upper bound of the snapshot range + * referencing this file. + */ private long lastUsedSnapshotID; /** The shared kv file handle. */ KvFileHandle kvFileHandle; diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java index 045d40ed56..c9e8bf5a05 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java @@ -198,12 +198,6 @@ void beforeEach() throws IOException { Configuration conf = new Configuration(); String remoteDataDir = "/tmp/fluss/remote-data"; conf.setString(ConfigOptions.REMOTE_DATA_DIR, remoteDataDir); - eventProcessor = buildCoordinatorEventProcessor(); - eventProcessor.startup(); - metadataManager.createDatabase( - defaultDatabase, DatabaseDescriptor.builder().build(), false); - completedSnapshotStoreManager = eventProcessor.completedSnapshotStoreManager(); - kvSnapshotLeaseManager = new KvSnapshotLeaseManager( Duration.ofMinutes(10).toMillis(), @@ -212,6 +206,12 @@ void beforeEach() throws IOException { SystemClock.getInstance(), TestingMetricGroups.COORDINATOR_METRICS); kvSnapshotLeaseManager.start(); + + eventProcessor = buildCoordinatorEventProcessor(); + eventProcessor.startup(); + metadataManager.createDatabase( + defaultDatabase, DatabaseDescriptor.builder().build(), false); + completedSnapshotStoreManager = eventProcessor.completedSnapshotStoreManager(); } @AfterEach diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/RebalanceManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/RebalanceManagerTest.java index b096a4c8b7..43ed3ae78a 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/RebalanceManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/RebalanceManagerTest.java @@ -80,12 +80,6 @@ static void baseBeforeAll() { void beforeEach() { serverMetadataCache = new CoordinatorMetadataCache(); testCoordinatorChannelManager = new TestCoordinatorChannelManager(); - autoPartitionManager = - new AutoPartitionManager(serverMetadataCache, metadataManager, new Configuration()); - lakeTableTieringManager = new LakeTableTieringManager(); - CoordinatorEventProcessor eventProcessor = buildCoordinatorEventProcessor(); - rebalanceManager = new RebalanceManager(eventProcessor, zookeeperClient); - rebalanceManager.startup(); String remoteDataDir = "/tmp/fluss/remote-data"; kvSnapshotLeaseManager = @@ -96,6 +90,13 @@ void beforeEach() { SystemClock.getInstance(), TestingMetricGroups.COORDINATOR_METRICS); kvSnapshotLeaseManager.start(); + + autoPartitionManager = + new AutoPartitionManager(serverMetadataCache, metadataManager, new Configuration()); + lakeTableTieringManager = new LakeTableTieringManager(); + CoordinatorEventProcessor eventProcessor = buildCoordinatorEventProcessor(); + rebalanceManager = new RebalanceManager(eventProcessor, zookeeperClient); + rebalanceManager.startup(); } @AfterEach diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStoreTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStoreTest.java index 78eaa97506..302589e2ec 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStoreTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStoreTest.java @@ -505,6 +505,223 @@ void testConcurrentGetNumSnapshotsAccuracy() throws Exception { } } + @Test + void testLeasedSnapshotDoesNotBlockSubsumedSnapshotCleanup() throws Exception { + // Simulate that snapshot 1 has a lease and cannot be subsumed. + // The subsumption checker returns false for snapshot 1, meaning it is in-use. + CompletedSnapshotStore store = + createCompletedSnapshotStore( + 1, + defaultHandleStore, + Collections.emptyList(), + bucket -> bucket.getSnapshotId() != 1L); + + CompletedSnapshot cp1 = getSnapshot(1L); + CompletedSnapshot cp2 = getSnapshot(2L); + CompletedSnapshot cp3 = getSnapshot(3L); + + store.add(cp1); + store.add(cp2); + + // After adding cp1 and cp2 with maxRetain=1: + // cp1 can't be subsumed (leased) → moved to stillInUseSnapshots + // cp2 is latest → retained in completedSnapshots + assertThat(store.getAllSnapshots()).hasSize(1); + assertThat(store.getAllSnapshots().get(0).getSnapshotID()).isEqualTo(2L); + assertThat(store.stillInUseSnapshots).containsKey(1L); + + store.add(cp3); + + // After adding cp3 with maxRetain=1: + // cp1 is still in stillInUseSnapshots (lease not expired) + // cp2 CAN be subsumed → subsumed and cleaned. + // With the fix, cleanSubsumedSnapshots uses upTo = snapshot.getSnapshotID() + 1 = 4, + // and stillInUseIds = {1L}, so cp2 (ID=2, not in stillInUse) will be properly cleaned. + // cp3 is latest → retained in completedSnapshots + List remaining = store.getAllSnapshots(); + assertThat(remaining).hasSize(1); + assertThat(remaining.get(0).getSnapshotID()).isEqualTo(3L); + assertThat(store.stillInUseSnapshots).containsKey(1L); + } + + @Test + void testLeasedSnapshotSSTFileCleanupWithStillInUse() throws Exception { + // Lease protects snapshot 1. + SharedKvFileRegistry registry = new SharedKvFileRegistry(); + + TestKvHandle fileA = new TestKvHandle("fileA"); + TestKvHandle fileB = new TestKvHandle("fileB"); + TestKvHandle fileC = new TestKvHandle("fileC"); + TestKvHandle fileD = new TestKvHandle("fileD"); + TestKvHandle fileE = new TestKvHandle("fileE"); + TestKvHandle fileF = new TestKvHandle("fileF"); + + // Snapshot 1: fileA (new), fileB (new) + CompletedSnapshot cp1 = + getSnapshotWithSharedFiles( + 1L, + Arrays.asList( + KvFileHandleAndLocalPath.of(fileA, "a.sst"), + KvFileHandleAndLocalPath.of(fileB, "b.sst"))); + // Snapshot 2: fileA (placeholder), fileC (new) + CompletedSnapshot cp2 = + getSnapshotWithSharedFiles( + 2L, + Arrays.asList( + KvFileHandleAndLocalPath.of( + new PlaceholderKvFileHandler(fileA), "a.sst"), + KvFileHandleAndLocalPath.of(fileC, "c.sst"))); + // Snapshot 3: fileA (placeholder), fileD (new) + CompletedSnapshot cp3 = + getSnapshotWithSharedFiles( + 3L, + Arrays.asList( + KvFileHandleAndLocalPath.of( + new PlaceholderKvFileHandler(fileA), "a.sst"), + KvFileHandleAndLocalPath.of(fileD, "d.sst"))); + // Snapshot 4: fileA (placeholder), fileE (new) + CompletedSnapshot cp4 = + getSnapshotWithSharedFiles( + 4L, + Arrays.asList( + KvFileHandleAndLocalPath.of( + new PlaceholderKvFileHandler(fileA), "a.sst"), + KvFileHandleAndLocalPath.of(fileE, "e.sst"))); + // Snapshot 5: fileA (placeholder), fileF (new) + CompletedSnapshot cp5 = + getSnapshotWithSharedFiles( + 5L, + Arrays.asList( + KvFileHandleAndLocalPath.of( + new PlaceholderKvFileHandler(fileA), "a.sst"), + KvFileHandleAndLocalPath.of(fileF, "f.sst"))); + + CompletedSnapshotStore store = + new CompletedSnapshotStore( + 2, + registry, + Collections.emptyList(), + defaultHandleStore, + executorService, + bucket -> bucket.getSnapshotId() != 1L); + + store.add(cp1); + store.add(cp2); + store.add(cp3); + store.add(cp4); + store.add(cp5); + + // Wait for async cleanup + Thread.sleep(200); + + // Verify snapshot state: completedSnapshots = [4, 5], stillInUse = {1} + assertThat(store.getAllSnapshots()).hasSize(2); + assertThat( + store.getAllSnapshots().stream() + .map(CompletedSnapshot::getSnapshotID) + .collect(Collectors.toList())) + .containsExactly(4L, 5L); + assertThat(store.stillInUseSnapshots).hasSize(1); + assertThat(store.stillInUseSnapshots).containsKey(1L); + + // Verify SST file cleanup: + // fileA: shared across all, lastUsed=5, NOT deleted + assertThat(fileA.discarded).isFalse(); + // fileB: exclusive to snapshot 1, protected by lease + assertThat(fileB.discarded).isFalse(); + // fileC: exclusive to snapshot 2, not protected, DELETED + assertThat(fileC.discarded).isTrue(); + // fileD: exclusive to snapshot 3, not protected, DELETED + assertThat(fileD.discarded).isTrue(); + // fileE: exclusive to snapshot 4, still in retention window, NOT deleted + assertThat(fileE.discarded).isFalse(); + // fileF: exclusive to snapshot 5, still in retention window, NOT deleted + assertThat(fileF.discarded).isFalse(); + } + + @Test + void testLeaseExpirySSTCleanup() throws Exception { + // Use a mutable set to control which snapshots are leased. + Set leasedSnapshotIds = new HashSet<>(); + leasedSnapshotIds.add(1L); + + SharedKvFileRegistry registry = new SharedKvFileRegistry(); + + TestKvHandle fileA = new TestKvHandle("fileA"); + TestKvHandle fileB = new TestKvHandle("fileB"); // exclusive to snapshot 1 + + CompletedSnapshot cp1 = + getSnapshotWithSharedFiles( + 1L, + Arrays.asList( + KvFileHandleAndLocalPath.of(fileA, "a.sst"), + KvFileHandleAndLocalPath.of(fileB, "b.sst"))); + CompletedSnapshot cp2 = + getSnapshotWithSharedFiles( + 2L, + Collections.singletonList( + KvFileHandleAndLocalPath.of( + new PlaceholderKvFileHandler(fileA), "a.sst"))); + + CompletedSnapshotStore store = + new CompletedSnapshotStore( + 1, + registry, + Collections.emptyList(), + defaultHandleStore, + executorService, + bucket -> !leasedSnapshotIds.contains(bucket.getSnapshotId())); + + store.add(cp1); + store.add(cp2); + + // cp1 leased → moved to stillInUseSnapshots, completedSnapshots = [2] + assertThat(store.stillInUseSnapshots).containsKey(1L); + assertThat(store.getAllSnapshots()).hasSize(1); + // fileB is protected by lease + assertThat(fileB.discarded).isFalse(); + + // Simulate lease expiry. + leasedSnapshotIds.remove(1L); + + // Add a new snapshot to trigger the release check. + TestKvHandle fileC = new TestKvHandle("fileC"); + CompletedSnapshot cp3 = + getSnapshotWithSharedFiles( + 3L, + Collections.singletonList( + KvFileHandleAndLocalPath.of( + new PlaceholderKvFileHandler(fileA), "a.sst"))); + store.add(cp3); + Thread.sleep(200); + + // After lease expiry: stillInUseSnapshots should be empty + assertThat(store.stillInUseSnapshots).isEmpty(); + assertThat(store.getAllSnapshots()).hasSize(1); + assertThat(store.getAllSnapshots().get(0).getSnapshotID()).isEqualTo(3L); + + // fileB should now be cleaned up (no longer protected) + assertThat(fileB.discarded).isTrue(); + // fileA is still used by the latest snapshot + assertThat(fileA.discarded).isFalse(); + } + + private CompletedSnapshotStore createCompletedSnapshotStore( + int numToRetain, + CompletedSnapshotHandleStore snapshotHandleStore, + Collection completedSnapshots, + CompletedSnapshotStore.SubsumptionChecker subsumptionChecker) { + + SharedKvFileRegistry sharedKvFileRegistry = new SharedKvFileRegistry(); + return new CompletedSnapshotStore( + numToRetain, + sharedKvFileRegistry, + completedSnapshots, + snapshotHandleStore, + executorService, + subsumptionChecker); + } + private List mapToCompletedSnapshot( List> snapshotHandles) { return snapshotHandles.stream() @@ -529,6 +746,16 @@ private CompletedSnapshot getSnapshot(long id) { new KvSnapshotHandle(Collections.emptyList(), Collections.emptyList(), 0)); } + private CompletedSnapshot getSnapshotWithSharedFiles( + long id, List sharedFiles) { + TableBucket tableBucket = new TableBucket(1, 1); + return new CompletedSnapshot( + tableBucket, + id, + new FsPath(tempDir.toString(), "snapshot_" + id), + new KvSnapshotHandle(sharedFiles, Collections.emptyList(), 0)); + } + private void testSnapshotRetention( int numToRetain, List completed, @@ -583,4 +810,21 @@ private List> createSnapshotHandles( } return stateHandles; } + + /** A test KvFileHandle that tracks whether it has been discarded. */ + private static class TestKvHandle extends KvFileHandle { + + private static final long serialVersionUID = 1L; + + boolean discarded; + + TestKvHandle(String path) { + super(path, 10); + } + + @Override + public void discard() throws Exception { + this.discarded = true; + } + } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/snapshot/SharedKvFileRegistryTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/snapshot/SharedKvFileRegistryTest.java index 6a178b40c2..a8082cb926 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/kv/snapshot/SharedKvFileRegistryTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/snapshot/SharedKvFileRegistryTest.java @@ -19,6 +19,10 @@ import org.junit.jupiter.api.Test; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; + import static org.assertj.core.api.Assertions.assertThat; /** Test for {@link org.apache.fluss.server.kv.snapshot.SharedKvFileRegistry}. */ @@ -77,6 +81,92 @@ void testUnregisterWithUnexistedKey() { sharedStateRegistry.unregisterUnusedKvFile(Long.MAX_VALUE); } + /** + * Test that stillInUse snapshot IDs protect KV files from being deleted during unregister. + * + *

Scenario: file created by snapshot 1, used by snapshot 1 through 3. If snapshot 1 is + * leased (in stillInUse), the file should NOT be deleted even if lowestSnapshotID > 3. + */ + @Test + void testUnregisterWithStillInUseProtection() { + SharedKvFileRegistry registry = new SharedKvFileRegistry(); + + // Register fileA: created by snapshot 1 + TestKvHandle fileA = new TestKvHandle("fileA"); + registry.registerReference(SharedKvFileRegistryKey.fromKvFileHandle(fileA), fileA, 1L); + + // fileA used by snapshot 2 and 3 (placeholder references) + PlaceholderKvFileHandler placeholder2 = new PlaceholderKvFileHandler(fileA); + registry.registerReference( + SharedKvFileRegistryKey.fromKvFileHandle(placeholder2), placeholder2, 2L); + PlaceholderKvFileHandler placeholder3 = new PlaceholderKvFileHandler(fileA); + registry.registerReference( + SharedKvFileRegistryKey.fromKvFileHandle(placeholder3), placeholder3, 3L); + + // Register fileB: created by snapshot 2, only used by snapshot 2 + TestKvHandle fileB = new TestKvHandle("fileB"); + registry.registerReference(SharedKvFileRegistryKey.fromKvFileHandle(fileB), fileB, 2L); + + // Register fileC: created by snapshot 4, used by snapshot 4 + TestKvHandle fileC = new TestKvHandle("fileC"); + registry.registerReference(SharedKvFileRegistryKey.fromKvFileHandle(fileC), fileC, 4L); + + // lowestSnapshotID = 4, stillInUse = {1} + // fileA: createdBy=1, lastUsed=3 → lastUsed(3) < 4, but snapshot 1 ∈ [1,3] → NOT deleted + // fileB: createdBy=2, lastUsed=2 → lastUsed(2) < 4, snapshot 1 NOT ∈ [2,2] → DELETED + // fileC: createdBy=4, lastUsed=4 → lastUsed(4) >= 4 → NOT deleted + Set stillInUse = new HashSet<>(); + stillInUse.add(1L); + registry.unregisterUnusedKvFile(4L, stillInUse); + + assertThat(fileA.discarded).isFalse(); + assertThat(fileB.discarded).isTrue(); + assertThat(fileC.discarded).isFalse(); + + // Now remove the lease: stillInUse = empty + // fileA: lastUsed(3) < 4, no stillInUse protection → DELETED + registry.unregisterUnusedKvFile(4L, Collections.emptySet()); + assertThat(fileA.discarded).isTrue(); + // fileC still NOT deleted + assertThat(fileC.discarded).isFalse(); + } + + /** Test with multiple leased snapshots protecting different file ranges. */ + @Test + void testUnregisterWithMultipleStillInUseSnapshots() { + SharedKvFileRegistry registry = new SharedKvFileRegistry(); + + // fileA: createdBy=1, lastUsed=3 + TestKvHandle fileA = new TestKvHandle("fileA"); + registry.registerReference(SharedKvFileRegistryKey.fromKvFileHandle(fileA), fileA, 1L); + PlaceholderKvFileHandler pA3 = new PlaceholderKvFileHandler(fileA); + registry.registerReference(SharedKvFileRegistryKey.fromKvFileHandle(pA3), pA3, 3L); + + // fileB: createdBy=2, lastUsed=4 + TestKvHandle fileB = new TestKvHandle("fileB"); + registry.registerReference(SharedKvFileRegistryKey.fromKvFileHandle(fileB), fileB, 2L); + PlaceholderKvFileHandler pB4 = new PlaceholderKvFileHandler(fileB); + registry.registerReference(SharedKvFileRegistryKey.fromKvFileHandle(pB4), pB4, 4L); + + // fileC: createdBy=5, lastUsed=5 + TestKvHandle fileC = new TestKvHandle("fileC"); + registry.registerReference(SharedKvFileRegistryKey.fromKvFileHandle(fileC), fileC, 5L); + + // lowestSnapshotID = 6, stillInUse = {1, 3} + // fileA: lastUsed(3) < 6, snapshot 1 ∈ [1,3] → protected + // fileA: lastUsed(3) < 6, snapshot 3 ∈ [1,3] → protected + // fileB: lastUsed(4) < 6, snapshot 1 NOT ∈ [2,4], snapshot 3 ∈ [2,4] → protected + // fileC: lastUsed(5) < 6, snapshot 1 NOT ∈ [5,5], snapshot 3 NOT ∈ [5,5] → DELETED + Set stillInUse = new HashSet<>(); + stillInUse.add(1L); + stillInUse.add(3L); + registry.unregisterUnusedKvFile(6L, stillInUse); + + assertThat(fileA.discarded).isFalse(); + assertThat(fileB.discarded).isFalse(); + assertThat(fileC.discarded).isTrue(); + } + private static class TestKvHandle extends KvFileHandle { private static final long serialVersionUID = 1; From bab57ae41d564dcd71cab0319c3e6676f0aa0250 Mon Sep 17 00:00:00 2001 From: yunhong <337361684@qq.com> Date: Sun, 8 Feb 2026 20:59:12 +0800 Subject: [PATCH 4/6] fix failed cases --- .../apache/fluss/client/admin/KvSnapshotLeaseImpl.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/fluss-client/src/main/java/org/apache/fluss/client/admin/KvSnapshotLeaseImpl.java b/fluss-client/src/main/java/org/apache/fluss/client/admin/KvSnapshotLeaseImpl.java index d38cf22643..9d8900c064 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/admin/KvSnapshotLeaseImpl.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/admin/KvSnapshotLeaseImpl.java @@ -23,13 +23,13 @@ import org.apache.fluss.rpc.gateway.AdminGateway; import org.apache.fluss.rpc.messages.AcquireKvSnapshotLeaseRequest; import org.apache.fluss.rpc.messages.DropKvSnapshotLeaseRequest; -import org.apache.fluss.rpc.messages.ReleaseKvSnapshotLeaseRequest; import java.util.Map; import java.util.Set; import java.util.concurrent.CompletableFuture; import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makeAcquireKvSnapshotLeaseRequest; +import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makeReleaseKvSnapshotLeaseRequest; /** The default implementation of KvSnapshotLease. */ public class KvSnapshotLeaseImpl implements KvSnapshotLease { @@ -77,9 +77,9 @@ public CompletableFuture renew() { @Override public CompletableFuture releaseSnapshots(Set bucketsToRelease) { - ReleaseKvSnapshotLeaseRequest request = - new ReleaseKvSnapshotLeaseRequest().setLeaseId(leaseId); - return gateway.releaseKvSnapshotLease(request).thenApply(r -> null); + return gateway.releaseKvSnapshotLease( + makeReleaseKvSnapshotLeaseRequest(leaseId, bucketsToRelease)) + .thenApply(r -> null); } @Override From 4b2516c2c795f9f5a3db65b693476a47392ee9e8 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Sun, 8 Feb 2026 21:33:20 +0800 Subject: [PATCH 5/6] code improvement --- .../CompletedSnapshotStoreManager.java | 14 +- .../CoordinatorEventProcessor.java | 2 +- .../lease/KvSnapshotLeaseManager.java | 13 +- .../kv/snapshot/CompletedSnapshotStore.java | 124 ++++++++---------- .../zk/data/lease/KvSnapshotTableLease.java | 16 +-- .../lease/KvSnapshotTableLeaseJsonSerde.java | 5 +- .../lease/KvSnapshotLeaseManagerTest.java | 4 +- .../snapshot/CompletedSnapshotStoreTest.java | 17 ++- .../KvSnapshotTableLeaseJsonSerdeTest.java | 6 +- 9 files changed, 93 insertions(+), 108 deletions(-) diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CompletedSnapshotStoreManager.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CompletedSnapshotStoreManager.java index 78c719f1e7..1b739f9a2a 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CompletedSnapshotStoreManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CompletedSnapshotStoreManager.java @@ -26,7 +26,7 @@ import org.apache.fluss.server.kv.snapshot.CompletedSnapshotHandle; import org.apache.fluss.server.kv.snapshot.CompletedSnapshotHandleStore; import org.apache.fluss.server.kv.snapshot.CompletedSnapshotStore; -import org.apache.fluss.server.kv.snapshot.CompletedSnapshotStore.SubsumptionChecker; +import org.apache.fluss.server.kv.snapshot.CompletedSnapshotStore.SnapshotInUseChecker; import org.apache.fluss.server.kv.snapshot.SharedKvFileRegistry; import org.apache.fluss.server.kv.snapshot.ZooKeeperCompletedSnapshotHandleStore; import org.apache.fluss.server.metrics.group.CoordinatorMetricGroup; @@ -65,7 +65,7 @@ public class CompletedSnapshotStoreManager { private final Executor ioExecutor; private final Function makeZookeeperCompletedSnapshotHandleStore; - private final SubsumptionChecker subsumptionChecker; + private final SnapshotInUseChecker snapshotInUseChecker; private final CoordinatorMetricGroup coordinatorMetricGroup; public CompletedSnapshotStoreManager( @@ -73,14 +73,14 @@ public CompletedSnapshotStoreManager( Executor ioExecutor, ZooKeeperClient zooKeeperClient, CoordinatorMetricGroup coordinatorMetricGroup, - SubsumptionChecker subsumptionChecker) { + SnapshotInUseChecker snapshotInUseChecker) { this( maxNumberOfSnapshotsToRetain, ioExecutor, zooKeeperClient, ZooKeeperCompletedSnapshotHandleStore::new, coordinatorMetricGroup, - subsumptionChecker); + snapshotInUseChecker); } @VisibleForTesting @@ -91,14 +91,14 @@ public CompletedSnapshotStoreManager( Function makeZookeeperCompletedSnapshotHandleStore, CoordinatorMetricGroup coordinatorMetricGroup, - SubsumptionChecker subsumptionChecker) { + SnapshotInUseChecker snapshotInUseChecker) { checkArgument( maxNumberOfSnapshotsToRetain > 0, "maxNumberOfSnapshotsToRetain must be positive"); this.maxNumberOfSnapshotsToRetain = maxNumberOfSnapshotsToRetain; this.zooKeeperClient = zooKeeperClient; this.bucketCompletedSnapshotStores = MapUtils.newConcurrentHashMap(); this.ioExecutor = ioExecutor; - this.subsumptionChecker = subsumptionChecker; + this.snapshotInUseChecker = snapshotInUseChecker; this.makeZookeeperCompletedSnapshotHandleStore = makeZookeeperCompletedSnapshotHandleStore; this.coordinatorMetricGroup = coordinatorMetricGroup; @@ -244,7 +244,7 @@ private CompletedSnapshotStore createCompletedSnapshotStore( retrievedSnapshots, completedSnapshotHandleStore, ioExecutor, - subsumptionChecker); + snapshotInUseChecker); } @VisibleForTesting 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 0d6f9959d8..b7ef915d1d 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 @@ -238,7 +238,7 @@ public CoordinatorEventProcessor( ioExecutor, zooKeeperClient, coordinatorMetricGroup, - kvSnapshotLeaseManager::snapshotLeaseNotExist); + kvSnapshotLeaseManager::snapshotLeaseExist); this.autoPartitionManager = autoPartitionManager; this.lakeTableTieringManager = lakeTableTieringManager; this.coordinatorMetricGroup = coordinatorMetricGroup; diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/lease/KvSnapshotLeaseManager.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/lease/KvSnapshotLeaseManager.java index 73dd08086f..88f5d923c8 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/lease/KvSnapshotLeaseManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/lease/KvSnapshotLeaseManager.java @@ -21,6 +21,7 @@ import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableBucketSnapshot; import org.apache.fluss.metrics.MetricNames; +import org.apache.fluss.server.kv.snapshot.CompletedSnapshot; import org.apache.fluss.server.metrics.group.CoordinatorMetricGroup; import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.server.zk.data.lease.KvSnapshotTableLease; @@ -143,12 +144,18 @@ public void start() { this::expireLeases, 0L, leaseExpirationCheckInterval, TimeUnit.MILLISECONDS); } - public boolean snapshotLeaseNotExist(TableBucketSnapshot tableBucketSnapshot) { + public boolean snapshotLeaseExist(CompletedSnapshot snapshot) { + TableBucketSnapshot tbs = + new TableBucketSnapshot(snapshot.getTableBucket(), snapshot.getSnapshotID()); + return snapshotLeaseExist(tbs); + } + + public boolean snapshotLeaseExist(TableBucketSnapshot tbs) { return inReadLock( managerLock, () -> { - AtomicInteger count = refCount.get(tableBucketSnapshot); - return count == null || count.get() <= 0; + AtomicInteger count = refCount.get(tbs); + return count != null && count.get() > 0; }); } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStore.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStore.java index cd008da8dc..ccf2fb28ce 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStore.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStore.java @@ -22,7 +22,6 @@ import org.apache.fluss.fs.FileSystem; import org.apache.fluss.fs.FsPath; import org.apache.fluss.metadata.TableBucket; -import org.apache.fluss.metadata.TableBucketSnapshot; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -70,7 +69,7 @@ public class CompletedSnapshotStore { private final Executor ioExecutor; private final SnapshotsCleaner snapshotsCleaner; - private final SubsumptionChecker subsumptionChecker; + private final SnapshotInUseChecker snapshotInUseChecker; private final ReentrantLock lock = new ReentrantLock(); @@ -98,13 +97,13 @@ public CompletedSnapshotStore( Collection completedSnapshots, CompletedSnapshotHandleStore completedSnapshotHandleStore, Executor executor, - SubsumptionChecker subsumptionChecker) { + SnapshotInUseChecker snapshotInUseChecker) { this.maxNumberOfSnapshotsToRetain = maxNumberOfSnapshotsToRetain; this.sharedKvFileRegistry = sharedKvFileRegistry; this.completedSnapshots = new ArrayDeque<>(); this.completedSnapshots.addAll(completedSnapshots); this.completedSnapshotHandleStore = completedSnapshotHandleStore; - this.subsumptionChecker = subsumptionChecker; + this.snapshotInUseChecker = snapshotInUseChecker; this.ioExecutor = executor; this.snapshotsCleaner = new SnapshotsCleaner(); } @@ -122,7 +121,7 @@ public long getPhysicalStorageRemoteKvSize() { } public long getNumSnapshots() { - return inLock(lock, () -> completedSnapshots.size()); + return inLock(lock, completedSnapshots::size); } /** @@ -132,7 +131,7 @@ public long getNumSnapshots() { * @param snapshot Completed snapshot to add. */ @VisibleForTesting - CompletedSnapshot addSnapshotAndSubsumeOldestOne( + void addSnapshotAndSubsumeOldestOne( final CompletedSnapshot snapshot, SnapshotsCleaner snapshotsCleaner, Runnable postCleanup) @@ -147,7 +146,7 @@ CompletedSnapshot addSnapshotAndSubsumeOldestOne( snapshot.getTableBucket(), snapshot.getSnapshotID(), completedSnapshotHandle); // Now add the new one. If it fails, we don't want to lose existing data. - return inLock( + inLock( lock, () -> { completedSnapshots.addLast(snapshot); @@ -158,58 +157,23 @@ CompletedSnapshot addSnapshotAndSubsumeOldestOne( completedSnapshots, maxNumberOfSnapshotsToRetain, completedSnapshot -> { - remove( - completedSnapshot.getTableBucket(), - completedSnapshot.getSnapshotID()); - snapshotsCleaner.addSubsumedSnapshot(completedSnapshot); - }, - subsumptionChecker); - - // Move leased snapshots that should have been subsumed but couldn't - // (protected by a lease) from completedSnapshots to stillInUseSnapshots. - // This ensures the effective lowestSnapshotID is computed from retained - // (non-leased) snapshots only, allowing SST files from non-leased - // subsumed snapshots to be cleaned up properly. - CompletedSnapshot latest = completedSnapshots.peekLast(); - Iterator leaseIt = completedSnapshots.iterator(); - while (leaseIt.hasNext()) { - CompletedSnapshot next = leaseIt.next(); - if (next != latest - && !subsumptionChecker.canSubsume( - new TableBucketSnapshot( - next.getTableBucket(), next.getSnapshotID()))) { - leaseIt.remove(); - stillInUseSnapshots.put(next.getSnapshotID(), next); - LOG.debug( - "Moved leased snapshot {} to stillInUseSnapshots", - next.getSnapshotID()); - } - } + if (snapshotInUseChecker.isInUse(completedSnapshot)) { + LOG.debug( + "Snapshot {} is still in use, move it to stillInUseSnapshots", + completedSnapshot.getSnapshotID()); + stillInUseSnapshots.put( + completedSnapshot.getSnapshotID(), completedSnapshot); + } else { + remove( + completedSnapshot.getTableBucket(), + completedSnapshot.getSnapshotID()); + snapshotsCleaner.addSubsumedSnapshot(completedSnapshot); + } + }); // Check if any previously still-in-use snapshots can now be released // (lease expired). - Iterator> stillInUseIter = - stillInUseSnapshots.entrySet().iterator(); - while (stillInUseIter.hasNext()) { - Map.Entry entry = stillInUseIter.next(); - CompletedSnapshot s = entry.getValue(); - if (subsumptionChecker.canSubsume( - new TableBucketSnapshot(s.getTableBucket(), s.getSnapshotID()))) { - stillInUseIter.remove(); - try { - remove(s.getTableBucket(), s.getSnapshotID()); - } catch (Exception e) { - LOG.warn( - "Failed to remove released snapshot {} from store", - s.getSnapshotID(), - e); - } - snapshotsCleaner.addSubsumedSnapshot(s); - LOG.debug( - "Released snapshot {} from stillInUseSnapshots (lease expired)", - s.getSnapshotID()); - } - } + removeUnusedSnapshots(snapshotsCleaner); // SST file cleanup: compute effective lowest from retained (non-leased) // snapshots only, and protect files referenced by still-in-use snapshots. @@ -227,19 +191,39 @@ CompletedSnapshot addSnapshotAndSubsumeOldestOne( // metadata, not shared SST files registered in SharedKvFileRegistry. snapshotsCleaner.cleanSubsumedSnapshots( snapshot.getSnapshotID() + 1, stillInUseIds, postCleanup, ioExecutor); - return null; }); } + private void removeUnusedSnapshots(SnapshotsCleaner snapshotsCleaner) throws Exception { + Iterator> stillInUseIter = + stillInUseSnapshots.entrySet().iterator(); + while (stillInUseIter.hasNext()) { + Map.Entry entry = stillInUseIter.next(); + CompletedSnapshot snapshot = entry.getValue(); + if (!snapshotInUseChecker.isInUse(snapshot)) { + stillInUseIter.remove(); + + remove(snapshot.getTableBucket(), snapshot.getSnapshotID()); + snapshotsCleaner.addSubsumedSnapshot(snapshot); + LOG.debug( + "Released snapshot {} from stillInUseSnapshots (lease expired)", + snapshot.getSnapshotID()); + } + } + } + + /** + * Returns a list of all the completed snapshots in this store. Note this doesn't include the + * snapshots that are still in use but have been moved out of the standard retention window, + * which can be obtained from {@link #stillInUseSnapshots}. + */ + @VisibleForTesting public List getAllSnapshots() { return inLock(lock, () -> new ArrayList<>(completedSnapshots)); } private static void subsume( - Deque snapshots, - int numRetain, - SubsumeAction subsumeAction, - SubsumptionChecker subsumptionChecker) { + Deque snapshots, int numRetain, SubsumeAction subsumeAction) { if (snapshots.isEmpty()) { return; } @@ -248,7 +232,7 @@ private static void subsume( Iterator iterator = snapshots.iterator(); while (snapshots.size() > numRetain && iterator.hasNext()) { CompletedSnapshot next = iterator.next(); - if (canSubsume(next, latest, subsumptionChecker)) { + if (canSubsume(next, latest)) { iterator.remove(); try { subsumeAction.subsume(next); @@ -264,23 +248,19 @@ interface SubsumeAction { void subsume(CompletedSnapshot snapshot) throws Exception; } - /** A function to check whether a snapshot can be subsumed. */ @FunctionalInterface - public interface SubsumptionChecker { - boolean canSubsume(TableBucketSnapshot bucket); + public interface SnapshotInUseChecker { + boolean isInUse(CompletedSnapshot snapshot); } - private static boolean canSubsume( - CompletedSnapshot next, - CompletedSnapshot latest, - SubsumptionChecker subsumptionChecker) { + private static boolean canSubsume(CompletedSnapshot next, CompletedSnapshot latest) { // if the snapshot is equal to the latest snapshot, it means it can't be subsumed if (next == latest) { return false; } - - return subsumptionChecker.canSubsume( - new TableBucketSnapshot(next.getTableBucket(), next.getSnapshotID())); + // else, we always subsume it as we will only keep single one snapshot currently + // todo: consider some client are pining this snapshot in FLUSS-54730210 + return true; } /** diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLease.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLease.java index df6a1bfd14..8ff2e8c9c6 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLease.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLease.java @@ -17,39 +17,35 @@ package org.apache.fluss.server.zk.data.lease; -import org.apache.fluss.utils.MapUtils; - import javax.annotation.Nullable; import javax.annotation.concurrent.NotThreadSafe; import java.util.Arrays; +import java.util.HashMap; import java.util.Map; import java.util.Objects; -import java.util.concurrent.ConcurrentHashMap; /** The lease of kv snapshot for a table. */ @NotThreadSafe public class KvSnapshotTableLease { private final long tableId; private @Nullable Long[] bucketSnapshots; - private final ConcurrentHashMap partitionSnapshots; + private final Map partitionSnapshots; public KvSnapshotTableLease(long tableId) { - this(tableId, null, MapUtils.newConcurrentHashMap()); + this(tableId, null, new HashMap<>()); } public KvSnapshotTableLease(long tableId, Long[] bucketSnapshots) { - this(tableId, bucketSnapshots, MapUtils.newConcurrentHashMap()); + this(tableId, bucketSnapshots, new HashMap<>()); } - public KvSnapshotTableLease(long tableId, ConcurrentHashMap partitionSnapshots) { + public KvSnapshotTableLease(long tableId, Map partitionSnapshots) { this(tableId, null, partitionSnapshots); } public KvSnapshotTableLease( - long tableId, - @Nullable Long[] bucketSnapshots, - ConcurrentHashMap partitionSnapshots) { + long tableId, @Nullable Long[] bucketSnapshots, Map partitionSnapshots) { this.tableId = tableId; this.bucketSnapshots = bucketSnapshots; this.partitionSnapshots = partitionSnapshots; diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLeaseJsonSerde.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLeaseJsonSerde.java index 6aa33a3cc8..09dfbb3f32 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLeaseJsonSerde.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLeaseJsonSerde.java @@ -19,14 +19,13 @@ 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.MapUtils; 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.HashMap; import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; /** Json serializer and deserializer for {@link KvSnapshotTableLease}. */ public class KvSnapshotTableLeaseJsonSerde @@ -89,7 +88,7 @@ public KvSnapshotTableLease deserialize(JsonNode node) { return new KvSnapshotTableLease(tableId, bucketSnapshots); } else { // for partition table. - ConcurrentHashMap partitionSnapshots = MapUtils.newConcurrentHashMap(); + Map partitionSnapshots = new HashMap<>(); JsonNode partitionSnapshotsNode = node.get(PARTITION_SNAPSHOTS); for (JsonNode partitionSnapshotNode : partitionSnapshotsNode) { long partitionId = partitionSnapshotNode.get(PARTITION_ID).asLong(); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/lease/KvSnapshotLeaseManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/lease/KvSnapshotLeaseManagerTest.java index eb1cc90c25..8851e0b688 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/lease/KvSnapshotLeaseManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/lease/KvSnapshotLeaseManagerTest.java @@ -413,12 +413,12 @@ private Map> initRegisterBuckets() { private boolean snapshotLeaseNotExists(List bucketList) { return bucketList.stream() - .allMatch(bucket -> kvSnapshotLeaseManager.snapshotLeaseNotExist(bucket)); + .noneMatch(bucket -> kvSnapshotLeaseManager.snapshotLeaseExist(bucket)); } private boolean snapshotLeaseExists(List bucketList) { return bucketList.stream() - .noneMatch(bucket -> kvSnapshotLeaseManager.snapshotLeaseNotExist(bucket)); + .allMatch(bucket -> kvSnapshotLeaseManager.snapshotLeaseExist(bucket)); } private void acquire(String leaseId, Map> tableIdToLeaseBucket) diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStoreTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStoreTest.java index 302589e2ec..06521e0faf 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStoreTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStoreTest.java @@ -508,19 +508,23 @@ void testConcurrentGetNumSnapshotsAccuracy() throws Exception { @Test void testLeasedSnapshotDoesNotBlockSubsumedSnapshotCleanup() throws Exception { // Simulate that snapshot 1 has a lease and cannot be subsumed. - // The subsumption checker returns false for snapshot 1, meaning it is in-use. + // The snapshotInUse checker returns true for snapshot 1, meaning it is in-use. CompletedSnapshotStore store = createCompletedSnapshotStore( 1, defaultHandleStore, Collections.emptyList(), - bucket -> bucket.getSnapshotId() != 1L); + bucket -> bucket.getSnapshotID() == 1L); CompletedSnapshot cp1 = getSnapshot(1L); CompletedSnapshot cp2 = getSnapshot(2L); CompletedSnapshot cp3 = getSnapshot(3L); store.add(cp1); + assertThat(store.getAllSnapshots()).hasSize(1); + assertThat(store.getAllSnapshots().get(0).getSnapshotID()).isEqualTo(1L); + assertThat(store.stillInUseSnapshots).isEmpty(); + store.add(cp2); // After adding cp1 and cp2 with maxRetain=1: @@ -603,7 +607,7 @@ void testLeasedSnapshotSSTFileCleanupWithStillInUse() throws Exception { Collections.emptyList(), defaultHandleStore, executorService, - bucket -> bucket.getSnapshotId() != 1L); + bucket -> bucket.getSnapshotID() == 1L); store.add(cp1); store.add(cp2); @@ -670,7 +674,7 @@ void testLeaseExpirySSTCleanup() throws Exception { Collections.emptyList(), defaultHandleStore, executorService, - bucket -> !leasedSnapshotIds.contains(bucket.getSnapshotId())); + bucket -> leasedSnapshotIds.contains(bucket.getSnapshotID())); store.add(cp1); store.add(cp2); @@ -685,7 +689,6 @@ void testLeaseExpirySSTCleanup() throws Exception { leasedSnapshotIds.remove(1L); // Add a new snapshot to trigger the release check. - TestKvHandle fileC = new TestKvHandle("fileC"); CompletedSnapshot cp3 = getSnapshotWithSharedFiles( 3L, @@ -710,7 +713,7 @@ private CompletedSnapshotStore createCompletedSnapshotStore( int numToRetain, CompletedSnapshotHandleStore snapshotHandleStore, Collection completedSnapshots, - CompletedSnapshotStore.SubsumptionChecker subsumptionChecker) { + CompletedSnapshotStore.SnapshotInUseChecker snapshotInUseChecker) { SharedKvFileRegistry sharedKvFileRegistry = new SharedKvFileRegistry(); return new CompletedSnapshotStore( @@ -719,7 +722,7 @@ private CompletedSnapshotStore createCompletedSnapshotStore( completedSnapshots, snapshotHandleStore, executorService, - subsumptionChecker); + snapshotInUseChecker); } private List mapToCompletedSnapshot( diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLeaseJsonSerdeTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLeaseJsonSerdeTest.java index e98c3b3e97..de77bb7779 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLeaseJsonSerdeTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLeaseJsonSerdeTest.java @@ -17,10 +17,10 @@ package org.apache.fluss.server.zk.data.lease; -import org.apache.fluss.utils.MapUtils; import org.apache.fluss.utils.json.JsonSerdeTestBase; -import java.util.concurrent.ConcurrentHashMap; +import java.util.HashMap; +import java.util.Map; /** Test for {@link KvSnapshotTableLeaseJsonSerde}. */ public class KvSnapshotTableLeaseJsonSerdeTest extends JsonSerdeTestBase { @@ -34,7 +34,7 @@ protected KvSnapshotTableLease[] createObjects() { KvSnapshotTableLease[] kvSnapshotTableLeases = new KvSnapshotTableLease[2]; kvSnapshotTableLeases[0] = new KvSnapshotTableLease(1L, new Long[] {1L, -1L, 1L, 2L}); - ConcurrentHashMap partitionSnapshots = MapUtils.newConcurrentHashMap(); + Map partitionSnapshots = new HashMap<>(); partitionSnapshots.put(2001L, new Long[] {10L, -1L, 20L, 30L}); partitionSnapshots.put(2002L, new Long[] {15L, -1L, 25L, 35L}); kvSnapshotTableLeases[1] = new KvSnapshotTableLease(2L, partitionSnapshots); From 385950d155c3ae72db410e977ee1ac629e7785da Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Sun, 8 Feb 2026 22:46:50 +0800 Subject: [PATCH 6/6] add javadoc --- .../apache/fluss/server/kv/snapshot/CompletedSnapshotStore.java | 1 + 1 file changed, 1 insertion(+) diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStore.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStore.java index ccf2fb28ce..cf398636b8 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStore.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotStore.java @@ -248,6 +248,7 @@ interface SubsumeAction { void subsume(CompletedSnapshot snapshot) throws Exception; } + /** A function to check whether a snapshot still in use. */ @FunctionalInterface public interface SnapshotInUseChecker { boolean isInUse(CompletedSnapshot snapshot);