From c0255f5fbd63f8754a8b11fdc3e1660553b7029d Mon Sep 17 00:00:00 2001 From: yunhong <337361684@qq.com> Date: Mon, 15 Dec 2025 13:57:27 +0800 Subject: [PATCH 1/2] [kv] Support kv snapshot lease --- .../org/apache/fluss/client/admin/Admin.java | 53 ++ .../apache/fluss/client/admin/FlussAdmin.java | 32 ++ .../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 | 4 + .../org/apache/fluss/utils/FlussPaths.java | 37 ++ .../fluss/flink/FlinkConnectorOptions.java | 18 + .../flink/catalog/FlinkTableFactory.java | 15 +- .../DropKvSnapshotLeaseProcedure.java | 36 ++ .../flink/procedure/ProcedureManager.java | 3 +- .../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 | 208 +++++++- .../event/FinishedKvSnapshotConsumeEvent.java | 74 +++ .../source/reader/FlinkSourceReader.java | 51 ++ .../flink/source/reader/LeaseContext.java | 79 +++ .../FlussSourceEnumeratorStateSerializer.java | 44 +- .../source/state/SourceEnumeratorState.java | 19 +- .../flink/procedure/FlinkProcedureITCase.java | 75 ++- .../flink/source/FlinkTableSourceITCase.java | 56 +- .../fluss/flink/source/FlussSourceITCase.java | 7 + .../enumerator/FlinkSourceEnumeratorTest.java | 34 +- .../SourceEnumeratorStateSerializerTest.java | 18 +- .../fluss/rpc/gateway/AdminGateway.java | 18 + .../apache/fluss/rpc/protocol/ApiKeys.java | 5 +- .../org/apache/fluss/rpc/protocol/Errors.java | 5 +- fluss-rpc/src/main/proto/FlussApi.proto | 44 ++ .../CompletedSnapshotStoreManager.java | 15 +- .../CoordinatorEventProcessor.java | 101 +++- .../server/coordinator/CoordinatorServer.java | 11 +- .../coordinator/CoordinatorService.java | 50 ++ .../coordinator/KvSnapshotLeaseManager.java | 453 +++++++++++++++++ .../event/AcquireKvSnapshotLeaseEvent.java | 60 +++ .../event/DropKvSnapshotLeaseEvent.java | 42 ++ .../event/ReleaseKvSnapshotLeaseEvent.java | 53 ++ .../kv/snapshot/CompletedSnapshotStore.java | 31 +- .../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 ++++ .../lease/KvSnapshotLeaseMetadataHelper.java | 226 ++++++++ .../KvSnapshotLeaseMetadataJsonSerde.java | 84 +++ .../zk/data/lease/KvSnapshotTableLease.java | 188 +++++++ .../lease/KvSnapshotTableLeaseJsonSerde.java | 116 +++++ .../CompletedSnapshotStoreManagerTest.java | 6 +- .../CoordinatorEventProcessorTest.java | 4 +- .../KvSnapshotLeaseManagerTest.java | 481 ++++++++++++++++++ .../coordinator/TestCoordinatorGateway.java | 24 + .../TableBucketStateMachineTest.java | 4 +- .../snapshot/CompletedSnapshotStoreTest.java | 3 +- .../snapshot/KvTabletSnapshotTargetTest.java | 3 +- .../ZooKeeperCompletedSnapshotStoreTest.java | 3 +- .../testutils/FlussClusterExtension.java | 2 +- .../fluss/server/zk/ZooKeeperClientTest.java | 42 ++ .../KvSnapshotLeaseMetadataHelperTest.java | 146 ++++++ .../KvSnapshotLeaseMetadataJsonSerdeTest.java | 57 +++ .../zk/data/lease/KvSnapshotLeaseTest.java | 207 ++++++++ .../KvSnapshotTableLeaseJsonSerdeTest.java | 54 ++ fluss-test-coverage/pom.xml | 3 + website/docs/engine-flink/options.md | 30 +- website/docs/maintenance/configuration.md | 3 +- 71 files changed, 4162 insertions(+), 113 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/DropKvSnapshotLeaseProcedure.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/DropKvSnapshotLeaseEvent.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/KvSnapshotLeaseMetadataHelper.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/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/KvSnapshotLeaseMetadataHelperTest.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/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 b5ae7b06b9..0e43bc603d 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; @@ -68,6 +69,8 @@ import java.util.Collection; import java.util.List; +import java.util.Map; +import java.util.Set; import java.util.concurrent.CompletableFuture; /** @@ -408,6 +411,56 @@ 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); + + /** + * Drops the entire lease asynchronously. + * + *

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 drop. + */ + CompletableFuture dropKvSnapshotLease(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 ac34caed87..e294a05b98 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 @@ -17,6 +17,7 @@ package org.apache.fluss.client.admin; +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; @@ -57,6 +58,7 @@ import org.apache.fluss.rpc.messages.DescribeClusterConfigsRequest; import org.apache.fluss.rpc.messages.DropAclsRequest; import org.apache.fluss.rpc.messages.DropDatabaseRequest; +import org.apache.fluss.rpc.messages.DropKvSnapshotLeaseRequest; import org.apache.fluss.rpc.messages.DropTableRequest; import org.apache.fluss.rpc.messages.GetDatabaseInfoRequest; import org.apache.fluss.rpc.messages.GetKvSnapshotMetadataRequest; @@ -91,13 +93,16 @@ import java.util.HashMap; import java.util.List; 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.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.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; @@ -381,6 +386,33 @@ 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 dropKvSnapshotLease(String leaseId) { + DropKvSnapshotLeaseRequest request = new DropKvSnapshotLeaseRequest().setLeaseId(leaseId); + return gateway.dropKvSnapshotLease(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 3e349cca3a..00a0a67aff 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 @@ -20,6 +20,7 @@ import org.apache.fluss.client.admin.OffsetSpec; 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; @@ -37,6 +38,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; @@ -50,10 +53,13 @@ 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.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; @@ -66,6 +72,7 @@ import org.apache.fluss.rpc.messages.PrefixLookupRequest; import org.apache.fluss.rpc.messages.ProduceLogRequest; import org.apache.fluss.rpc.messages.PutKvRequest; +import org.apache.fluss.rpc.messages.ReleaseKvSnapshotLeaseRequest; import org.apache.fluss.utils.json.DataTypeJsonSerde; import org.apache.fluss.utils.json.JsonSerdeUtils; @@ -370,6 +377,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 List toPartitionInfos(ListPartitionInfosResponse response) { return response.getPartitionsInfosList().stream() .map( 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 ea1aaf2378..b0d2768baa 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,22 +35,29 @@ 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.KvSnapshotLeaseMetadataHelper; +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 java.util.Set; 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 { @@ -181,6 +189,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 + waitUntilAllSnapshotFinished(expectedRowByBuckets.keySet(), 0); + + ZooKeeperClient zkClient = FLUSS_CLUSTER_EXTENSION.getZooKeeperClient(); + String remoteDataDir = FLUSS_CLUSTER_EXTENSION.getRemoteDataDir(); + KvSnapshotLeaseMetadataHelper metadataHelper = + new KvSnapshotLeaseMetadataHelper(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( + metadataHelper, kvSnapshotLease1, tableId, new Long[] {0L, 0L, 0L}); + + expectedRowByBuckets = putRows(tableId, tablePath, 10); + // wait snapshot2 finish + waitUntilAllSnapshotFinished(expectedRowByBuckets.keySet(), 1); + + // 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( + metadataHelper, 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 + waitUntilAllSnapshotFinished(expectedRowByBuckets.keySet(), 2); + + // release lease1. + admin.releaseKvSnapshotLease( + kvSnapshotLease1, Collections.singleton(new TableBucket(tableId, 0))) + .get(); + checkKvSnapshotLeaseEquals( + metadataHelper, kvSnapshotLease1, tableId, new Long[] {-1L, 0L, 0L}); + + // release lease2. + admin.releaseKvSnapshotLease(kvSnapshotLease2, consumeBuckets.keySet()).get(); + assertThat(zkClient.getKvSnapshotLeasesList()).doesNotContain(kvSnapshotLease2); + + // drop lease1 + admin.dropKvSnapshotLease(kvSnapshotLease1).get(); + assertThat(zkClient.getKvSnapshotLeasesList()).isEmpty(); + + expectedRowByBuckets = putRows(tableId, tablePath, 10); + // wait snapshot2 finish + waitUntilAllSnapshotFinished(expectedRowByBuckets.keySet(), 3); + // 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.dropKvSnapshotLease("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<>(); @@ -245,4 +344,20 @@ private void waitUntilAllSnapshotFinished(Set tableBuckets, long sn FLUSS_CLUSTER_EXTENSION.waitUntilSnapshotFinished(tableBucket, snapshotId); } } + + private void checkKvSnapshotLeaseEquals( + KvSnapshotLeaseMetadataHelper metadataHelper, + String leaseId, + long tableId, + Long[] expectedBucketIndex) + throws Exception { + assertThat(metadataHelper.getLeasesList()).contains(leaseId); + Optional leaseOpt = metadataHelper.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 9f4b603e98..bec8fef97f 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 @@ -1507,9 +1507,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 b1326d46bc..8ff19d0130 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 @@ -53,6 +53,10 @@ 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. + 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 d04268fbaa..ab1c36fccf 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 @@ -93,6 +93,8 @@ public class FlussPaths { private static final String REMOTE_LAKE_DIR_NAME = "lake"; + private static final String REMOTE_LEASE_DIR_NAME = "lease"; + // ---------------------------------------------------------------------------------------- // LOG/KV Tablet Paths // ---------------------------------------------------------------------------------------- @@ -722,6 +724,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-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 9c2f7aafa7..e194066c81 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 @@ -30,6 +30,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; @@ -62,6 +63,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 aba27735cf..fc53628eda 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 @@ -24,6 +24,7 @@ import org.apache.fluss.flink.lake.LakeTableFactory; import org.apache.fluss.flink.sink.FlinkTableSink; 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.DataLakeFormat; import org.apache.fluss.metadata.TablePath; @@ -136,6 +137,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( @@ -151,7 +161,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 @@ -211,6 +222,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/DropKvSnapshotLeaseProcedure.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/DropKvSnapshotLeaseProcedure.java new file mode 100644 index 0000000000..9d932c83a1 --- /dev/null +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/DropKvSnapshotLeaseProcedure.java @@ -0,0 +1,36 @@ +/* + * 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 drop kv snapshot lease. */ +public class DropKvSnapshotLeaseProcedure extends ProcedureBase { + + @ProcedureHint( + argument = { + @ArgumentHint(name = "leaseId", type = @DataTypeHint("STRING")), + }) + public String[] call(ProcedureContext context, String leaseId) throws Exception { + admin.dropKvSnapshotLease(leaseId).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 12d4cf3675..aa3deff77d 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 @@ -73,7 +73,8 @@ private enum ProcedureEnum { SET_CLUSTER_CONFIG("sys.set_cluster_config", SetClusterConfigProcedure.class), GET_CLUSTER_CONFIG("sys.get_cluster_config", GetClusterConfigProcedure.class), ADD_SERVER_TAG("sys.add_server_tag", AddServerTagProcedure.class), - REMOVE_SERVER_TAG("sys.remove_server_tag", RemoveServerTagProcedure.class); + REMOVE_SERVER_TAG("sys.remove_server_tag", RemoveServerTagProcedure.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 ba880ff916..84bea8e743 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.initializer.OffsetsInitializer; 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 6acdbf8406..3df0e36b33 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; @@ -133,6 +134,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 @@ -171,7 +174,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; @@ -187,6 +191,7 @@ public FlinkTableSource( this.scanPartitionDiscoveryIntervalMs = scanPartitionDiscoveryIntervalMs; this.isDataLakeEnabled = isDataLakeEnabled; + this.leaseContext = leaseContext; this.mergeEngineType = mergeEngineType; this.tableOptions = tableOptions; if (isDataLakeEnabled) { @@ -354,7 +359,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, @@ -471,7 +477,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 4277418349..a7832c647a 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.config.Configuration; import org.apache.fluss.flink.source.deserializer.FlussDeserializationSchema; import org.apache.fluss.flink.source.enumerator.initializer.OffsetsInitializer; +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 f166397531..8fe40e03b4 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.flink.FlinkConnectorOptions; import org.apache.fluss.flink.source.deserializer.FlussDeserializationSchema; import org.apache.fluss.flink.source.enumerator.initializer.OffsetsInitializer; +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 f8db0aff14..b4091a02fa 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.source.enumerator.initializer.OffsetsInitializer; import org.apache.fluss.flink.source.enumerator.initializer.OffsetsInitializer.BucketOffsetsRetriever; import org.apache.fluss.flink.source.enumerator.initializer.SnapshotOffsetsInitializer; +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; @@ -60,6 +62,7 @@ import org.slf4j.LoggerFactory; import javax.annotation.Nullable; +import javax.annotation.concurrent.GuardedBy; import java.io.IOException; import java.util.ArrayList; @@ -71,13 +74,18 @@ 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.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.stream.Collectors; import static org.apache.fluss.utils.Preconditions.checkNotNull; import static org.apache.fluss.utils.Preconditions.checkState; +import static org.apache.fluss.utils.concurrent.LockUtils.inWriteLock; /** * An implementation of {@link SplitEnumerator} for the data of Fluss. @@ -132,6 +140,13 @@ public class FlinkSourceEnumerator private final OffsetsInitializer startingOffsetsInitializer; private final OffsetsInitializer stoppingOffsetsInitializer; + private final LeaseContext leaseContext; + + private final ReadWriteLock lock = new ReentrantReadWriteLock(); + /** checkpointId -> tableBuckets who finished consume kv snapshots. */ + @GuardedBy("lock") + private final TreeMap> consumedKvSnapshotMap = new TreeMap<>(); + // Lazily instantiated or mutable fields. private Connection connection; private Admin flussAdmin; @@ -160,7 +175,8 @@ public FlinkSourceEnumerator( long scanPartitionDiscoveryIntervalMs, boolean streaming, @Nullable Predicate partitionFilters, - @Nullable LakeSource lakeSource) { + @Nullable LakeSource lakeSource, + LeaseContext leaseContext) { this( tablePath, flussConf, @@ -174,7 +190,8 @@ public FlinkSourceEnumerator( scanPartitionDiscoveryIntervalMs, streaming, partitionFilters, - lakeSource); + lakeSource, + leaseContext); } public FlinkSourceEnumerator( @@ -190,7 +207,8 @@ public FlinkSourceEnumerator( long scanPartitionDiscoveryIntervalMs, boolean streaming, @Nullable Predicate partitionFilters, - @Nullable LakeSource lakeSource) { + @Nullable LakeSource lakeSource, + LeaseContext leaseContext) { this( tablePath, flussConf, @@ -205,7 +223,8 @@ public FlinkSourceEnumerator( streaming, partitionFilters, lakeSource, - new WorkerExecutor(context)); + new WorkerExecutor(context), + leaseContext); } FlinkSourceEnumerator( @@ -222,7 +241,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 +263,7 @@ public FlinkSourceEnumerator( streaming ? new NoStoppingOffsetsInitializer() : OffsetsInitializer.latest(); this.lakeSource = lakeSource; this.workerExecutor = workerExecutor; + this.leaseContext = leaseContext; } @Override @@ -349,16 +370,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 +543,96 @@ 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 util all buckets + // acquire success. The reason is that getLatestKvSnapshots and acquireKvSnapshotLease + // are not atomic operations, the latest kv snapshot obtained via get may become outdated by + // the time it is passed to acquire. Therefore, this logic must implement a retry + // mechanism: the unavailable tableBuckets in the AcquiredKvSnapshotLeaseResult returned by + // acquireKvSnapshotLease must be retried repeatedly until all buckets are successfully + // acquired. + try { + Set remainingTableBuckets; + do { + KvSnapshots kvSnapshots = getLatestKvSnapshots(partitionName); + 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); + remainingTableBuckets = + flussAdmin + .acquireKvSnapshotLease( + kvSnapshotLeaseId, + bucketsToLease, + leaseContext.getKvSnapshotLeaseDurationMs()) + .get() + .getUnavailableTableBucketSet(); + if (!remainingTableBuckets.isEmpty()) { + LOG.info( + "Failed to acquire kv snapshot lease for table {}: {}. Retry to re-acquire", + tablePath, + remainingTableBuckets); + } + } + } while (!remainingTableBuckets.isEmpty()); + } 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(); @@ -888,6 +974,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)); } } @@ -922,11 +1020,65 @@ 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) { + inWriteLock( + lock, + () -> { + 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) { + return inWriteLock( + lock, + () -> { + 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..9332bbd2e6 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 ignoreBuckets; + public FlinkSourceReader( FutureCompletingBlockingQueue> elementsQueue, Configuration flussConfig, @@ -77,6 +88,7 @@ public FlinkSourceReader( recordEmitter, context.getConfiguration(), context); + this.ignoreBuckets = 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 (ignoreBuckets.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. + ignoreBuckets.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 c355109a2d..42ba7640cb 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 @@ -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.flink.source.split.SourceSplitSerializer; import org.apache.fluss.lake.source.LakeSource; @@ -88,6 +89,9 @@ public byte[] serialize(SourceEnumeratorState state) throws IOException { serializeRemainingHybridLakeFlussSplits(out, state); } + // write lease context + serializeLeaseContext(out, state); + final byte[] result = out.getCopyOfBuffer(); out.clear(); return result; @@ -129,8 +133,11 @@ public SourceEnumeratorState deserialize(int version, byte[] serialized) throws remainingHybridLakeFlussSplits = deserializeRemainingHybridLakeFlussSplits(in); } + // deserialize lease context + LeaseContext leaseContext = deserializeLeaseContext(in); + return new SourceEnumeratorState( - assignedBuckets, assignedPartitions, remainingHybridLakeFlussSplits); + assignedBuckets, assignedPartitions, remainingHybridLakeFlussSplits, leaseContext); } private void serializeRemainingHybridLakeFlussSplits( @@ -173,4 +180,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 7a677df20f..7f35da3fa3 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) { @@ -73,7 +83,10 @@ public boolean equals(Object o) { } SourceEnumeratorState that = (SourceEnumeratorState) o; return Objects.equals(assignedBuckets, that.assignedBuckets) - && Objects.equals(assignedPartitions, that.assignedPartitions); + && Objects.equals(assignedPartitions, that.assignedPartitions) + && Objects.equals( + remainingHybridLakeFlussSplits, that.remainingHybridLakeFlussSplits) + && Objects.equals(leaseContext, that.leaseContext); } @Override @@ -90,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 b764923050..0369b48552 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 @@ -17,12 +17,18 @@ package org.apache.fluss.flink.procedure; +import org.apache.fluss.client.Connection; +import org.apache.fluss.client.ConnectionFactory; +import org.apache.fluss.client.admin.Admin; +import org.apache.fluss.client.metadata.KvSnapshots; import org.apache.fluss.cluster.rebalance.ServerTag; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; import org.apache.fluss.config.MemorySize; 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; @@ -32,6 +38,7 @@ import org.apache.flink.types.Row; import org.apache.flink.util.CloseableIterator; import org.apache.flink.util.CollectionUtil; +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; @@ -48,6 +55,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.testutils.DataTestUtils.row; +import static org.apache.fluss.testutils.common.CommonTestUtils.waitUntil; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -64,9 +74,19 @@ public abstract class FlinkProcedureITCase { .build(); static final String CATALOG_NAME = "testcatalog"; + static Configuration clientConf; + static Connection conn; + static Admin admin; TableEnvironment tEnv; + @BeforeAll + protected static void beforeAll() { + clientConf = FLUSS_CLUSTER_EXTENSION.getClientConfig(); + conn = ConnectionFactory.createConnection(clientConf); + admin = conn.getAdmin(); + } + @BeforeEach void before() throws ExecutionException, InterruptedException { String bootstrapServers = @@ -104,7 +124,8 @@ void testShowProcedures() throws Exception { "+I[sys.list_acl]", "+I[sys.set_cluster_config]", "+I[sys.add_server_tag]", - "+I[sys.remove_server_tag]"); + "+I[sys.remove_server_tag]", + "+I[sys.drop_kv_snapshot_lease]"); // make sure no more results is unread. assertResultsIgnoreOrder(showProceduresIterator, expectedShowProceduresResult, true); } @@ -460,6 +481,43 @@ void testAddAndRemoveServerTag(boolean upperCase) throws Exception { } } + @Test + 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)"); + 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); + + waitUntilAllBucketFinishSnapshot(admin, 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.drop_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); @@ -500,4 +558,19 @@ private static void assertCallResult(CloseableIterator rows, String[] expec assertThat(actual).containsExactlyInAnyOrder(expected); } } + + private void waitUntilAllBucketFinishSnapshot(Admin admin, TablePath tablePath) { + waitUntil( + () -> { + KvSnapshots snapshots = admin.getLatestKvSnapshots(tablePath).get(); + for (int bucketId : snapshots.getBucketIds()) { + if (!snapshots.getSnapshotId(bucketId).isPresent()) { + return false; + } + } + return true; + }, + Duration.ofMinutes(1), + "Fail to wait until all bucket finish snapshot"); + } } 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 4d056a12df..158ba51c7a 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 @@ -29,6 +29,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; @@ -78,6 +79,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.apache.fluss.testutils.common.CommonTestUtils.waitUntil; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -357,6 +359,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); + + waitUntilAllBucketFinishSnapshot(admin, 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 // ------------------------------------------------------------------------------------- @@ -607,15 +654,8 @@ void testReadPrimaryKeyPartitionedTable(boolean isAutoPartition) throws Exceptio writeRowsToPartition(conn, tablePath, partitionNameById.values()); waitUntilAllBucketFinishSnapshot(admin, tablePath, partitionNameById.values()); - // This test requires dynamically discovering newly created partitions, so - // 'scan.partition.discovery.interval' needs to be set to 2s (default is 1 minute), - // otherwise the test may hang for 1 minute. org.apache.flink.util.CloseableIterator rowIter = - tEnv.executeSql( - String.format( - "select * from %s /*+ OPTIONS('scan.partition.discovery.interval' = '2s') */", - tableName)) - .collect(); + tEnv.executeSql(String.format("select * from %s", tableName)).collect(); assertResultsIgnoreOrder(rowIter, expectedRowValues, false); // then create some new partitions, and write rows to the new partitions 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 0098455006..fcc0706b88 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 0e9260a54c..f755f8fd70 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.source.enumerator.initializer.OffsetsInitializer; 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 26b4a024c0..b6e35bc1f5 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; @@ -26,7 +27,8 @@ import org.apache.fluss.lake.source.TestingLakeSplit; 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 +46,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 +82,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); 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 072a5954f1..0393a65b6f 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; @@ -37,6 +39,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; @@ -45,6 +49,8 @@ import org.apache.fluss.rpc.messages.ListRebalanceProgressResponse; 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.ReleaseKvSnapshotLeaseResponse; import org.apache.fluss.rpc.messages.RemoveServerTagRequest; import org.apache.fluss.rpc.messages.RemoveServerTagResponse; import org.apache.fluss.rpc.protocol.ApiKeys; @@ -146,6 +152,18 @@ CompletableFuture listRebalanceProgress( @RPC(api = ApiKeys.CANCEL_REBALANCE) CompletableFuture cancelRebalance(CancelRebalanceRequest request); + @RPC(api = ApiKeys.ACQUIRE_KV_SNAPSHOT_LEASE) + CompletableFuture acquireKvSnapshotLease( + AcquireKvSnapshotLeaseRequest request); + + @RPC(api = ApiKeys.RELEASE_KV_SNAPSHOT_LEASE) + 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 cc033ba8a9..5d7613b5be 100644 --- a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java +++ b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java @@ -80,7 +80,10 @@ public enum ApiKeys { REBALANCE(1049, 0, 0, PUBLIC), LIST_REBALANCE_PROGRESS(1050, 0, 0, PUBLIC), CANCEL_REBALANCE(1051, 0, 0, PUBLIC), - PREPARE_LAKE_TABLE_SNAPSHOT(1052, 0, 0, PRIVATE); + PREPARE_LAKE_TABLE_SNAPSHOT(1052, 0, 0, PRIVATE), + ACQUIRE_KV_SNAPSHOT_LEASE(1053, 0, 0, PUBLIC), + RELEASE_KV_SNAPSHOT_LEASE(1054, 0, 0, PUBLIC), + DROP_KV_SNAPSHOT_LEASE(1055, 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 5ee652cce2..cc0ddb68b5 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 @@ -43,6 +43,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; @@ -240,7 +241,9 @@ public enum Errors { SEVER_TAG_NOT_EXIST_EXCEPTION(60, "The server tag not exist.", ServerTagNotExistException::new), REBALANCE_FAILURE_EXCEPTION(61, "The rebalance task failure.", RebalanceFailureException::new), NO_REBALANCE_IN_PROGRESS_EXCEPTION( - 62, "No rebalance task in progress.", NoRebalanceInProgressException::new); + 62, "No rebalance task in progress.", NoRebalanceInProgressException::new), + KV_SNAPSHOT_LEASE_NOT_EXIST( + 63, "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 6698d98d58..77258c4a09 100644 --- a/fluss-rpc/src/main/proto/FlussApi.proto +++ b/fluss-rpc/src/main/proto/FlussApi.proto @@ -364,6 +364,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; } @@ -780,6 +803,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; @@ -1059,4 +1092,15 @@ message PbBucketOffset { optional int64 partition_id = 1; required int32 bucket_id = 2; optional int64 log_end_offset = 4; +} + +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..ba96aee1cd 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.CanSubsume; 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 CanSubsume canSubsume; private final CoordinatorMetricGroup coordinatorMetricGroup; public CompletedSnapshotStoreManager( int maxNumberOfSnapshotsToRetain, Executor ioExecutor, ZooKeeperClient zooKeeperClient, - CoordinatorMetricGroup coordinatorMetricGroup) { + CoordinatorMetricGroup coordinatorMetricGroup, + CanSubsume canSubsume) { this( maxNumberOfSnapshotsToRetain, ioExecutor, zooKeeperClient, ZooKeeperCompletedSnapshotHandleStore::new, - coordinatorMetricGroup); + coordinatorMetricGroup, + canSubsume); } @VisibleForTesting @@ -86,13 +90,15 @@ public CompletedSnapshotStoreManager( ZooKeeperClient zooKeeperClient, Function makeZookeeperCompletedSnapshotHandleStore, - CoordinatorMetricGroup coordinatorMetricGroup) { + CoordinatorMetricGroup coordinatorMetricGroup, + CanSubsume canSubsume) { checkArgument( maxNumberOfSnapshotsToRetain > 0, "maxNumberOfSnapshotsToRetain must be positive"); this.maxNumberOfSnapshotsToRetain = maxNumberOfSnapshotsToRetain; this.zooKeeperClient = zooKeeperClient; this.bucketCompletedSnapshotStores = MapUtils.newConcurrentHashMap(); this.ioExecutor = ioExecutor; + this.canSubsume = canSubsume; this.makeZookeeperCompletedSnapshotHandleStore = makeZookeeperCompletedSnapshotHandleStore; this.coordinatorMetricGroup = coordinatorMetricGroup; @@ -237,7 +243,8 @@ private CompletedSnapshotStore createCompletedSnapshotStore( sharedKvFileRegistry, retrievedSnapshots, completedSnapshotHandleStore, - ioExecutor); + ioExecutor, + canSubsume); } @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 471c62f7dc..abd40698a9 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 @@ -29,6 +29,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.ServerNotExistException; import org.apache.fluss.exception.ServerTagAlreadyExistException; import org.apache.fluss.exception.ServerTagNotExistException; @@ -43,16 +44,21 @@ 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.CommitKvSnapshotResponse; import org.apache.fluss.rpc.messages.CommitLakeTableSnapshotResponse; import org.apache.fluss.rpc.messages.CommitRemoteLogManifestResponse; import org.apache.fluss.rpc.messages.ControlledShutdownResponse; +import org.apache.fluss.rpc.messages.DropKvSnapshotLeaseResponse; import org.apache.fluss.rpc.messages.PbCommitLakeTableSnapshotRespForTable; +import org.apache.fluss.rpc.messages.PbKvSnapshotLeaseForBucket; +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.CommitKvSnapshotEvent; @@ -65,6 +71,7 @@ import org.apache.fluss.server.coordinator.event.CreateTableEvent; import org.apache.fluss.server.coordinator.event.DeadTabletServerEvent; import org.apache.fluss.server.coordinator.event.DeleteReplicaResponseReceivedEvent; +import org.apache.fluss.server.coordinator.event.DropKvSnapshotLeaseEvent; import org.apache.fluss.server.coordinator.event.DropPartitionEvent; import org.apache.fluss.server.coordinator.event.DropTableEvent; import org.apache.fluss.server.coordinator.event.EventProcessor; @@ -73,6 +80,7 @@ import org.apache.fluss.server.coordinator.event.NotifyKvSnapshotOffsetEvent; import org.apache.fluss.server.coordinator.event.NotifyLakeTableOffsetEvent; import org.apache.fluss.server.coordinator.event.NotifyLeaderAndIsrResponseReceivedEvent; +import org.apache.fluss.server.coordinator.event.ReleaseKvSnapshotLeaseEvent; import org.apache.fluss.server.coordinator.event.RemoveServerTagEvent; import org.apache.fluss.server.coordinator.event.SchemaChangeEvent; import org.apache.fluss.server.coordinator.event.watcher.TableChangeWatcher; @@ -103,6 +111,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.KvSnapshotLeaseMetadataHelper; +import org.apache.fluss.utils.clock.Clock; import org.apache.fluss.utils.types.Tuple2; import org.slf4j.Logger; @@ -157,6 +167,7 @@ public class CoordinatorEventProcessor implements EventProcessor { private final CoordinatorMetricGroup coordinatorMetricGroup; private final String internalListenerName; + private final KvSnapshotLeaseManager kvSnapshotLeaseManager; private final CompletedSnapshotStoreManager completedSnapshotStoreManager; private final LakeTableHelper lakeTableHelper; @@ -170,7 +181,8 @@ public CoordinatorEventProcessor( CoordinatorMetricGroup coordinatorMetricGroup, Configuration conf, ExecutorService ioExecutor, - MetadataManager metadataManager) { + MetadataManager metadataManager, + Clock clock) { this.zooKeeperClient = zooKeeperClient; this.serverMetadataCache = serverMetadataCache; this.coordinatorChannelManager = coordinatorChannelManager; @@ -208,19 +220,29 @@ public CoordinatorEventProcessor( this.coordinatorRequestBatch = new CoordinatorRequestBatch( coordinatorChannelManager, coordinatorEventManager, coordinatorContext); + + String remoteDataDir = conf.getString(ConfigOptions.REMOTE_DATA_DIR); + this.kvSnapshotLeaseManager = + new KvSnapshotLeaseManager( + conf, + new KvSnapshotLeaseMetadataHelper(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.ioExecutor = ioExecutor; - this.lakeTableHelper = - new LakeTableHelper(zooKeeperClient, conf.getString(ConfigOptions.REMOTE_DATA_DIR)); + this.lakeTableHelper = new LakeTableHelper(zooKeeperClient, remoteDataDir); } public CoordinatorEventManager getCoordinatorEventManager() { @@ -256,6 +278,9 @@ public void startup() { // start the event manager which will then process the event coordinatorEventManager.start(); + + // start kv snapshot lease manager + kvSnapshotLeaseManager.start(); } public void shutdown() { @@ -403,6 +428,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( @@ -579,6 +607,23 @@ public void process(CoordinatorEvent event) { completeFromCallable( removeServerTagEvent.getRespCallback(), () -> processRemoveServerTag(removeServerTagEvent)); + } 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 DropKvSnapshotLeaseEvent) { + DropKvSnapshotLeaseEvent dropKvSnapshotLeaseEvent = (DropKvSnapshotLeaseEvent) event; + completeFromCallable( + dropKvSnapshotLeaseEvent.getRespCallback(), + () -> tryProcessDropKvSnapshotLease(dropKvSnapshotLeaseEvent)); } else if (event instanceof AccessContextEvent) { AccessContextEvent accessContextEvent = (AccessContextEvent) event; processAccessContext(accessContextEvent); @@ -1542,6 +1587,54 @@ 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(); + kvSnapshotLeaseManager.release(event.getLeaseId(), event.getTableIdToReleasedBucket()); + return response; + } + + private DropKvSnapshotLeaseResponse tryProcessDropKvSnapshotLease( + DropKvSnapshotLeaseEvent event) throws Exception { + DropKvSnapshotLeaseResponse response = new DropKvSnapshotLeaseResponse(); + boolean exist = kvSnapshotLeaseManager.drop(event.getLeaseId()); + if (!exist) { + throw new KvSnapshotLeaseNotExistException( + "kv snapshot lease '" + event.getLeaseId() + "' not exits."); + } + 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 cd546f4479..52b42755fe 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 @@ -43,6 +43,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; @@ -84,6 +86,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; @@ -141,9 +144,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) { @@ -249,7 +257,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 7fc53b96f8..378e0db32d 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 @@ -46,6 +46,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; @@ -76,6 +78,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; @@ -95,6 +99,8 @@ import org.apache.fluss.rpc.messages.PrepareLakeTableSnapshotResponse; import org.apache.fluss.rpc.messages.RebalanceRequest; import org.apache.fluss.rpc.messages.RebalanceResponse; +import org.apache.fluss.rpc.messages.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; @@ -111,13 +117,16 @@ 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.CommitKvSnapshotEvent; import org.apache.fluss.server.coordinator.event.CommitLakeTableSnapshotEvent; import org.apache.fluss.server.coordinator.event.CommitRemoteLogManifestEvent; import org.apache.fluss.server.coordinator.event.ControlledShutdownEvent; +import org.apache.fluss.server.coordinator.event.DropKvSnapshotLeaseEvent; import org.apache.fluss.server.coordinator.event.EventManager; +import org.apache.fluss.server.coordinator.event.ReleaseKvSnapshotLeaseEvent; import org.apache.fluss.server.coordinator.event.RemoveServerTagEvent; import org.apache.fluss.server.entity.CommitKvSnapshotData; import org.apache.fluss.server.entity.LakeTieringTableInfo; @@ -154,10 +163,12 @@ import static org.apache.fluss.rpc.util.CommonRpcMessageUtils.toAclBindingFilters; import static org.apache.fluss.rpc.util.CommonRpcMessageUtils.toAclBindings; 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.makeCreateAclsResponse; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.makeDropAclsResponse; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.toAlterTableConfigChanges; @@ -779,6 +790,45 @@ 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 dropKvSnapshotLease( + DropKvSnapshotLeaseRequest request) { + CompletableFuture response = new CompletableFuture<>(); + eventManagerSupplier + .get() + .put(new DropKvSnapshotLeaseEvent(request.getLeaseId(), 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..853e62a426 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/KvSnapshotLeaseManager.java @@ -0,0 +1,453 @@ +/* + * 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.KvSnapshotLeaseMetadataHelper; +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 KvSnapshotLeaseMetadataHelper metadataHelper; + 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. + */ + @GuardedBy("refCountLock") + private final Map refCount = + MapUtils.newConcurrentHashMap(); + + /** For metrics. */ + private final AtomicInteger leasedBucketCount = new AtomicInteger(0); + + public KvSnapshotLeaseManager( + Configuration conf, + KvSnapshotLeaseMetadataHelper metadataHelper, + CoordinatorContext coordinatorContext, + Clock clock, + CoordinatorMetricGroup coordinatorMetricGroup) { + this( + conf, + metadataHelper, + coordinatorContext, + Executors.newScheduledThreadPool( + 1, new ExecutorThreadFactory("kv-snapshot-lease-cleaner")), + clock, + coordinatorMetricGroup); + } + + @VisibleForTesting + public KvSnapshotLeaseManager( + Configuration conf, + KvSnapshotLeaseMetadataHelper metadataHelper, + CoordinatorContext coordinatorContext, + ScheduledExecutorService scheduledExecutor, + Clock clock, + CoordinatorMetricGroup coordinatorMetricGroup) { + this.metadataHelper = metadataHelper; + 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 { + List leaseIds = metadataHelper.getLeasesList(); + for (String leaseId : leaseIds) { + Optional kvSnapshotLeaseOpt = metadataHelper.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, + () -> + !refCount.containsKey(kvSnapshotLeaseForBucket) + || refCount.get(kvSnapshotLeaseForBucket).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); + KvSnapshotLease kvSnapshotLease; + long newLeaseDuration = clock.milliseconds() + leaseDuration; + if (!update) { + // set the expiration time as: current time + leaseDuration + kvSnapshotLease = new KvSnapshotLease(newLeaseDuration); + kvSnapshotLeaseMap.put(leaseId, kvSnapshotLease); + LOG.info( + "kv snapshot lease '" + + leaseId + + "' has been acquired. The lease expiration time is " + + kvSnapshotLease.getExpirationTime()); + } else { + kvSnapshotLease = kvSnapshotLeaseMap.get(leaseId); + kvSnapshotLease.setExpirationTime(newLeaseDuration); + } + + 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 = + metadataHelper.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) { + metadataHelper.updateLease(leaseId, kvSnapshotLease); + } else { + metadataHelper.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()) { + drop(leaseId); + } else { + metadataHelper.updateLease(leaseId, lease); + } + }); + } + + /** + * Drop kv snapshot lease. + * + * @param leaseId the lease id + * @return true if clear success, false if lease not exist + */ + public boolean drop(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); + metadataHelper.deleteLease(leaseId); + + LOG.info("kv snapshot lease '" + leaseId + "' has been dropped."); + 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(); + List expiredLeases = + kvSnapshotLeaseMap.entrySet().stream() + .filter(entry -> entry.getValue().getExpirationTime() < currentTime) + .map(Map.Entry::getKey) + .collect(Collectors.toList()); + for (String leaseId : expiredLeases) { + try { + drop(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, + () -> { + if (!refCount.containsKey(kvSnapshotLeaseForBucket)) { + return 0; + } else { + return refCount.get(kvSnapshotLeaseForBucket).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/DropKvSnapshotLeaseEvent.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/DropKvSnapshotLeaseEvent.java new file mode 100644 index 0000000000..7354085f7f --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/DropKvSnapshotLeaseEvent.java @@ -0,0 +1,42 @@ +/* + * 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.rpc.messages.DropKvSnapshotLeaseResponse; + +import java.util.concurrent.CompletableFuture; + +/** An event for drop a kv snapshot lease. */ +public class DropKvSnapshotLeaseEvent implements CoordinatorEvent { + private final String leaseId; + private final CompletableFuture respCallback; + + public DropKvSnapshotLeaseEvent( + String leaseId, CompletableFuture respCallback) { + this.leaseId = leaseId; + this.respCallback = respCallback; + } + + public String getLeaseId() { + return leaseId; + } + + 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..a12a3b325a 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 CanSubsume canSubsume; private final ReentrantLock lock = new ReentrantLock(); @@ -80,12 +82,14 @@ public CompletedSnapshotStore( SharedKvFileRegistry sharedKvFileRegistry, Collection completedSnapshots, CompletedSnapshotHandleStore completedSnapshotHandleStore, - Executor executor) { + Executor executor, + CanSubsume canSubsume) { this.maxNumberOfSnapshotsToRetain = maxNumberOfSnapshotsToRetain; this.sharedKvFileRegistry = sharedKvFileRegistry; this.completedSnapshots = new ArrayDeque<>(); this.completedSnapshots.addAll(completedSnapshots); this.completedSnapshotHandleStore = completedSnapshotHandleStore; + this.canSubsume = canSubsume; this.ioExecutor = executor; this.snapshotsCleaner = new SnapshotsCleaner(); } @@ -144,7 +148,8 @@ CompletedSnapshot addSnapshotAndSubsumeOldestOne( completedSnapshot.getTableBucket(), completedSnapshot.getSnapshotID()); snapshotsCleaner.addSubsumedSnapshot(completedSnapshot); - }); + }, + canSubsume); 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, + CanSubsume canSubsume) { 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, canSubsume)) { // always return the subsumed snapshot with larger snapshot id. if (!lastSubsumedSnapshot.isPresent() || next.getSnapshotID() > lastSubsumedSnapshot.get().getSnapshotID()) { @@ -200,14 +208,21 @@ interface SubsumeAction { void subsume(CompletedSnapshot snapshot) throws Exception; } - private static boolean canSubsume(CompletedSnapshot next, CompletedSnapshot latest) { + /** A function to determine whether a snapshot can be subsumed. */ + @FunctionalInterface + public interface CanSubsume { + boolean canSubsume(KvSnapshotLeaseForBucket bucket); + } + + private static boolean canSubsume( + CompletedSnapshot next, CompletedSnapshot latest, CanSubsume canSubsume) { // 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 canSubsume.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 8ee46545b0..21b73e76a9 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 @@ -26,6 +26,7 @@ import org.apache.fluss.config.cluster.ConfigEntry; import org.apache.fluss.fs.FsPath; import org.apache.fluss.fs.token.ObtainedSecurityToken; +import org.apache.fluss.metadata.KvSnapshotLeaseForBucket; import org.apache.fluss.metadata.PartitionSpec; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.ResolvedPartitionSpec; @@ -51,6 +52,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; @@ -87,6 +89,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; @@ -100,6 +103,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; @@ -126,6 +131,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; @@ -138,6 +144,7 @@ import org.apache.fluss.rpc.messages.ProduceLogResponse; import org.apache.fluss.rpc.messages.PutKvRequest; import org.apache.fluss.rpc.messages.PutKvResponse; +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; @@ -1790,6 +1797,48 @@ public static List toPbConfigEntries(List describ .collect(Collectors.toList()); } + 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 40bffe8171..6bffa7a767 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 @@ -59,6 +59,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; @@ -79,6 +81,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.shaded.curator5.org.apache.curator.framework.CuratorFramework; import org.apache.fluss.shaded.curator5.org.apache.curator.framework.api.BackgroundCallback; import org.apache.fluss.shaded.curator5.org.apache.curator.framework.api.CuratorEvent; @@ -992,6 +995,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 4798623a74..a2c5464c6f 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.utils.json.JsonSerdeUtils; import org.apache.fluss.utils.types.Tuple2; @@ -818,4 +820,38 @@ public static RebalancePlan decode(byte[] json) { return JsonSerdeUtils.readValue(json, RebalancePlanJsonSerde.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..9b1258ed9a --- /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 leaseDuration) { + this(leaseDuration, 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/KvSnapshotLeaseMetadataHelper.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataHelper.java new file mode 100644 index 0000000000..cc0474b9bd --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataHelper.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 helper to handle {@link KvSnapshotLease} to register/update/delete metadata from zk and + * remote fs. + */ +public class KvSnapshotLeaseMetadataHelper { + private static final Logger LOG = LoggerFactory.getLogger(KvSnapshotLeaseMetadataHelper.class); + + private final ZooKeeperClient zkClient; + private final String remoteDataDir; + + public KvSnapshotLeaseMetadataHelper(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/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/KvSnapshotTableLease.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotTableLease.java new file mode 100644 index 0000000000..2bb4533bca --- /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 += 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 dcfa6b5b3a..900595a26a 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 @@ -74,6 +74,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; @@ -911,7 +912,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..f90fc297c0 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/KvSnapshotLeaseManagerTest.java @@ -0,0 +1,481 @@ +/* + * 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.metadata.TableInfo; +import org.apache.fluss.metadata.TablePath; +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.KvSnapshotLeaseMetadataHelper; +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_PARTITIONED_TABLE_DESCRIPTOR; +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.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 TablePath PARTITION_TABLE_PATH = + new TablePath("test_db_1", "test_partition_table"); + private static final long PARTITION_TABLE_ID = 150008L; + private static final TableInfo PARTITION_TABLE_INFO = + TableInfo.of( + PARTITION_TABLE_PATH, + PARTITION_TABLE_ID, + 1, + DATA1_PARTITIONED_TABLE_DESCRIPTOR, + System.currentTimeMillis(), + System.currentTimeMillis()); + 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 KvSnapshotLeaseMetadataHelper metadataHelper; + + 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(); + metadataHelper = new KvSnapshotLeaseMetadataHelper(zookeeperClient, tempDir.toString()); + kvSnapshotLeaseManager = + new KvSnapshotLeaseManager( + conf, + metadataHelper, + 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)); + metadataHelper.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(metadataHelper.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(metadataHelper.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(metadataHelper.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.drop("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.drop("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.drop("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 06d15d95ed..bbaf9ed81c 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; @@ -58,6 +60,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; @@ -94,6 +98,8 @@ import org.apache.fluss.rpc.messages.PrepareLakeTableSnapshotResponse; import org.apache.fluss.rpc.messages.RebalanceRequest; import org.apache.fluss.rpc.messages.RebalanceResponse; +import org.apache.fluss.rpc.messages.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; @@ -391,6 +397,24 @@ 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 dropKvSnapshotLease( + DropKvSnapshotLeaseRequest 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/statemachine/TableBucketStateMachineTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachineTest.java index c57c9950e5..b76a1618db 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 @@ -43,6 +43,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; @@ -263,7 +264,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 9f9ea078da..dff452cb26 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 bab31f10b2..44623bf5b3 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 @@ -257,7 +257,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 64da4b95cb..2a1627e56f 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.Schema; import org.apache.fluss.metadata.SchemaInfo; import org.apache.fluss.metadata.TableBucket; @@ -40,6 +41,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; @@ -488,6 +491,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 @@ -664,4 +697,13 @@ void testZookeeperConfigPath() throws Exception { .isEqualTo("zookeeper2"); } } + + 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/KvSnapshotLeaseMetadataHelperTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataHelperTest.java new file mode 100644 index 0000000000..1c121a5cc4 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataHelperTest.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 KvSnapshotLeaseMetadataHelper}. */ +public class KvSnapshotLeaseMetadataHelperTest { + + @RegisterExtension + public static final AllCallbackWrapper ZOO_KEEPER_EXTENSION_WRAPPER = + new AllCallbackWrapper<>(new ZooKeeperExtension()); + + protected static ZooKeeperClient zookeeperClient; + private @TempDir Path tempDir; + private KvSnapshotLeaseMetadataHelper metadataHelper; + + @BeforeAll + static void beforeAll() { + zookeeperClient = + ZOO_KEEPER_EXTENSION_WRAPPER + .getCustomExtension() + .getZooKeeperClient(NOPErrorHandler.INSTANCE); + } + + @BeforeEach + void beforeEach() { + metadataHelper = new KvSnapshotLeaseMetadataHelper(zookeeperClient, tempDir.toString()); + } + + @AfterEach + void afterEach() { + ZOO_KEEPER_EXTENSION_WRAPPER.getCustomExtension().cleanupRoot(); + } + + @Test + void testGetLeasesList() throws Exception { + List leasesList = metadataHelper.getLeasesList(); + assertThat(leasesList).isEmpty(); + + metadataHelper.registerLease("leaseId1", new KvSnapshotLease(1000L)); + + Map tableIdToTableLease = new HashMap<>(); + tableIdToTableLease.put(1L, new KvSnapshotTableLease(1L, new Long[] {100L, -1L})); + metadataHelper.registerLease("leaseId2", new KvSnapshotLease(2000L, tableIdToTableLease)); + leasesList = metadataHelper.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); + metadataHelper.registerLease("leaseId1", expectedLease); + + Optional lease = metadataHelper.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); + metadataHelper.updateLease("leaseId1", expectedLease); + lease = metadataHelper.getLease("leaseId1"); + assertThat(lease).hasValue(expectedLease); + // assert zk and remote fs. + assertRemoteFsAndZkEquals("leaseId1", expectedLease); + + // test delete lease. + metadataHelper.deleteLease("leaseId1"); + lease = metadataHelper.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/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/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 16e942860e..1cd28525b1 100644 --- a/fluss-test-coverage/pom.xml +++ b/fluss-test-coverage/pom.xml @@ -398,6 +398,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 a7114af492..81dfabe6f8 100644 --- a/website/docs/engine-flink/options.md +++ b/website/docs/engine-flink/options.md @@ -91,20 +91,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/maintenance/configuration.md b/website/docs/maintenance/configuration.md index 97da3457d0..fc52eeb8b1 100644 --- a/website/docs/maintenance/configuration.md +++ b/website/docs/maintenance/configuration.md @@ -137,6 +137,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`. | @@ -157,7 +158,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 From da631094fb8846fb733839d29141db9fe6e65c3b Mon Sep 17 00:00:00 2001 From: yunhong <337361684@qq.com> Date: Sat, 3 Jan 2026 16:33:20 +0800 Subject: [PATCH 2/2] address's Yang Wang's comments --- .../org/apache/fluss/client/admin/Admin.java | 4 +- .../batch/KvSnapshotBatchScannerITCase.java | 8 +-- .../org/apache/fluss/metrics/MetricNames.java | 1 + .../org/apache/fluss/record/TestData.java | 14 ++++++ .../DropKvSnapshotLeaseProcedure.java | 2 +- .../CoordinatorEventProcessor.java | 4 +- .../coordinator/KvSnapshotLeaseManager.java | 50 ++++++++----------- ...va => KvSnapshotLeaseMetadataManager.java} | 8 +-- .../KvSnapshotLeaseManagerTest.java | 23 +++------ ...> KvSnapshotLeaseMetadataManagerTest.java} | 8 +-- .../observability/monitor-metrics.md | 12 ++++- 11 files changed, 69 insertions(+), 65 deletions(-) rename fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/{KvSnapshotLeaseMetadataHelper.java => KvSnapshotLeaseMetadataManager.java} (97%) rename fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/{KvSnapshotLeaseMetadataHelperTest.java => KvSnapshotLeaseMetadataManagerTest.java} (95%) 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 0e43bc603d..2703747bb0 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 @@ -452,12 +452,12 @@ CompletableFuture releaseKvSnapshotLease( String leaseId, Set bucketsToRelease); /** - * Drops the entire lease asynchronously. + * Releases the entire lease asynchronously. * *

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 drop. + * @param leaseId The lease id to release. */ CompletableFuture dropKvSnapshotLease(String leaseId); 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 b0d2768baa..228dbfe537 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 @@ -37,7 +37,7 @@ 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.KvSnapshotLeaseMetadataHelper; +import org.apache.fluss.server.zk.data.lease.KvSnapshotLeaseMetadataManager; import org.apache.fluss.server.zk.data.lease.KvSnapshotTableLease; import org.apache.fluss.types.DataTypes; @@ -205,8 +205,8 @@ public void testKvSnapshotLease() throws Exception { ZooKeeperClient zkClient = FLUSS_CLUSTER_EXTENSION.getZooKeeperClient(); String remoteDataDir = FLUSS_CLUSTER_EXTENSION.getRemoteDataDir(); - KvSnapshotLeaseMetadataHelper metadataHelper = - new KvSnapshotLeaseMetadataHelper(zkClient, remoteDataDir); + KvSnapshotLeaseMetadataManager metadataHelper = + new KvSnapshotLeaseMetadataManager(zkClient, remoteDataDir); assertThat(zkClient.getKvSnapshotLeasesList()).isEmpty(); @@ -346,7 +346,7 @@ private void waitUntilAllSnapshotFinished(Set tableBuckets, long sn } private void checkKvSnapshotLeaseEquals( - KvSnapshotLeaseMetadataHelper metadataHelper, + KvSnapshotLeaseMetadataManager metadataHelper, String leaseId, long tableId, Long[] expectedBucketIndex) 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 8ff19d0130..e42abc1330 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,7 @@ public class MetricNames { 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"; 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 8743771257..011ca9802f 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/procedure/DropKvSnapshotLeaseProcedure.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/DropKvSnapshotLeaseProcedure.java index 9d932c83a1..3bd38e121f 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/DropKvSnapshotLeaseProcedure.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/DropKvSnapshotLeaseProcedure.java @@ -22,7 +22,7 @@ import org.apache.flink.table.annotation.ProcedureHint; import org.apache.flink.table.procedure.ProcedureContext; -/** Procedure to drop kv snapshot lease. */ +/** Procedure to release kv snapshot lease. */ public class DropKvSnapshotLeaseProcedure extends ProcedureBase { @ProcedureHint( 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 abd40698a9..4965336ca4 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 @@ -111,7 +111,7 @@ 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.KvSnapshotLeaseMetadataHelper; +import org.apache.fluss.server.zk.data.lease.KvSnapshotLeaseMetadataManager; import org.apache.fluss.utils.clock.Clock; import org.apache.fluss.utils.types.Tuple2; @@ -225,7 +225,7 @@ public CoordinatorEventProcessor( this.kvSnapshotLeaseManager = new KvSnapshotLeaseManager( conf, - new KvSnapshotLeaseMetadataHelper(zooKeeperClient, remoteDataDir), + new KvSnapshotLeaseMetadataManager(zooKeeperClient, remoteDataDir), coordinatorContext, clock, coordinatorMetricGroup); 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 index 853e62a426..650f6be928 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/KvSnapshotLeaseManager.java @@ -26,7 +26,7 @@ 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.KvSnapshotLeaseMetadataHelper; +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; @@ -48,7 +48,6 @@ 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; @@ -57,8 +56,9 @@ @ThreadSafe public class KvSnapshotLeaseManager { private static final Logger LOG = LoggerFactory.getLogger(KvSnapshotLeaseManager.class); + private static final AtomicInteger ZERO = new AtomicInteger(0); - private final KvSnapshotLeaseMetadataHelper metadataHelper; + private final KvSnapshotLeaseMetadataManager metadataHelper; private final CoordinatorContext coordinatorContext; private final Clock clock; private final ScheduledExecutorService scheduledExecutor; @@ -75,7 +75,6 @@ public class KvSnapshotLeaseManager { * KvSnapshotLeaseForBucket to the ref count, which means this table bucket + snapshotId has * been leased by how many lease id. */ - @GuardedBy("refCountLock") private final Map refCount = MapUtils.newConcurrentHashMap(); @@ -84,7 +83,7 @@ public class KvSnapshotLeaseManager { public KvSnapshotLeaseManager( Configuration conf, - KvSnapshotLeaseMetadataHelper metadataHelper, + KvSnapshotLeaseMetadataManager metadataHelper, CoordinatorContext coordinatorContext, Clock clock, CoordinatorMetricGroup coordinatorMetricGroup) { @@ -101,7 +100,7 @@ public KvSnapshotLeaseManager( @VisibleForTesting public KvSnapshotLeaseManager( Configuration conf, - KvSnapshotLeaseMetadataHelper metadataHelper, + KvSnapshotLeaseMetadataManager metadataHelper, CoordinatorContext coordinatorContext, ScheduledExecutorService scheduledExecutor, Clock clock, @@ -126,8 +125,7 @@ public void start() { } public void initialize() throws Exception { - List leaseIds = metadataHelper.getLeasesList(); - for (String leaseId : leaseIds) { + for (String leaseId : metadataHelper.getLeasesList()) { Optional kvSnapshotLeaseOpt = metadataHelper.getLease(leaseId); if (kvSnapshotLeaseOpt.isPresent()) { KvSnapshotLease kvSnapshotLease = kvSnapshotLeaseOpt.get(); @@ -145,8 +143,8 @@ public boolean snapshotLeaseNotExist(KvSnapshotLeaseForBucket kvSnapshotLeaseFor return inReadLock( refCountLock, () -> - !refCount.containsKey(kvSnapshotLeaseForBucket) - || refCount.get(kvSnapshotLeaseForBucket).get() <= 0); + refCount.getOrDefault(kvSnapshotLeaseForBucket, new AtomicInteger(0)).get() + <= 0); } /** @@ -402,18 +400,17 @@ private void decrementRefCount(KvSnapshotLeaseForBucket kvSnapshotLeaseForBucket private void expireLeases() { long currentTime = clock.milliseconds(); - List expiredLeases = - kvSnapshotLeaseMap.entrySet().stream() - .filter(entry -> entry.getValue().getExpirationTime() < currentTime) - .map(Map.Entry::getKey) - .collect(Collectors.toList()); - for (String leaseId : expiredLeases) { - try { - drop(leaseId); - } catch (Exception e) { - LOG.error("Failed to clear kv snapshot lease {}", leaseId, e); - } - } + kvSnapshotLeaseMap.entrySet().stream() + .filter(entry -> entry.getValue().getExpirationTime() < currentTime) + .map(Map.Entry::getKey) + .forEach( + leaseId -> { + try { + drop(leaseId); + } catch (Exception e) { + LOG.error("Failed to clear kv snapshot lease {}", leaseId, e); + } + }); } private void registerMetrics(CoordinatorMetricGroup coordinatorMetricGroup) { @@ -436,14 +433,7 @@ int getLeasedBucketCount() { @VisibleForTesting int getRefCount(KvSnapshotLeaseForBucket kvSnapshotLeaseForBucket) { return inReadLock( - refCountLock, - () -> { - if (!refCount.containsKey(kvSnapshotLeaseForBucket)) { - return 0; - } else { - return refCount.get(kvSnapshotLeaseForBucket).get(); - } - }); + refCountLock, () -> refCount.getOrDefault(kvSnapshotLeaseForBucket, ZERO).get()); } @VisibleForTesting diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataHelper.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataManager.java similarity index 97% rename from fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataHelper.java rename to fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataManager.java index cc0474b9bd..d269e8d763 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataHelper.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataManager.java @@ -41,16 +41,16 @@ import static org.apache.fluss.utils.Preconditions.checkNotNull; /** - * The helper to handle {@link KvSnapshotLease} to register/update/delete metadata from zk and + * The manager to handle {@link KvSnapshotLease} to register/update/delete metadata from zk and * remote fs. */ -public class KvSnapshotLeaseMetadataHelper { - private static final Logger LOG = LoggerFactory.getLogger(KvSnapshotLeaseMetadataHelper.class); +public class KvSnapshotLeaseMetadataManager { + private static final Logger LOG = LoggerFactory.getLogger(KvSnapshotLeaseMetadataManager.class); private final ZooKeeperClient zkClient; private final String remoteDataDir; - public KvSnapshotLeaseMetadataHelper(ZooKeeperClient zkClient, String remoteDataDir) { + public KvSnapshotLeaseMetadataManager(ZooKeeperClient zkClient, String remoteDataDir) { this.zkClient = zkClient; this.remoteDataDir = remoteDataDir; } 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 index f90fc297c0..45b2bebef8 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/KvSnapshotLeaseManagerTest.java @@ -22,15 +22,13 @@ import org.apache.fluss.metadata.KvSnapshotLeaseForBucket; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.TableBucket; -import org.apache.fluss.metadata.TableInfo; -import org.apache.fluss.metadata.TablePath; 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.KvSnapshotLeaseMetadataHelper; +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; @@ -52,10 +50,12 @@ import java.util.Map; import java.util.concurrent.TimeUnit; -import static org.apache.fluss.record.TestData.DATA1_PARTITIONED_TABLE_DESCRIPTOR; 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; @@ -66,17 +66,6 @@ public class KvSnapshotLeaseManagerTest { public static final AllCallbackWrapper ZOO_KEEPER_EXTENSION_WRAPPER = new AllCallbackWrapper<>(new ZooKeeperExtension()); - private static final TablePath PARTITION_TABLE_PATH = - new TablePath("test_db_1", "test_partition_table"); - private static final long PARTITION_TABLE_ID = 150008L; - private static final TableInfo PARTITION_TABLE_INFO = - TableInfo.of( - PARTITION_TABLE_PATH, - PARTITION_TABLE_ID, - 1, - DATA1_PARTITIONED_TABLE_DESCRIPTOR, - System.currentTimeMillis(), - System.currentTimeMillis()); private static final long PARTITION_ID_1 = 19001L; private static final PhysicalTablePath PARTITION_TABLE_PATH_1 = PhysicalTablePath.of(PARTITION_TABLE_PATH, "2024"); @@ -101,7 +90,7 @@ public class KvSnapshotLeaseManagerTest { private ManualClock manualClock; private ManuallyTriggeredScheduledExecutorService clearLeaseScheduler; private KvSnapshotLeaseManager kvSnapshotLeaseManager; - private KvSnapshotLeaseMetadataHelper metadataHelper; + private KvSnapshotLeaseMetadataManager metadataHelper; private @TempDir Path tempDir; @@ -121,7 +110,7 @@ void beforeEach() throws Exception { conf.set(ConfigOptions.KV_SNAPSHOT_LEASE_EXPIRATION_CHECK_INTERVAL, Duration.ofDays(7)); manualClock = new ManualClock(System.currentTimeMillis()); clearLeaseScheduler = new ManuallyTriggeredScheduledExecutorService(); - metadataHelper = new KvSnapshotLeaseMetadataHelper(zookeeperClient, tempDir.toString()); + metadataHelper = new KvSnapshotLeaseMetadataManager(zookeeperClient, tempDir.toString()); kvSnapshotLeaseManager = new KvSnapshotLeaseManager( conf, diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataHelperTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataManagerTest.java similarity index 95% rename from fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataHelperTest.java rename to fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataManagerTest.java index 1c121a5cc4..3d7f138945 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataHelperTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lease/KvSnapshotLeaseMetadataManagerTest.java @@ -41,8 +41,8 @@ import static org.assertj.core.api.Assertions.assertThat; -/** Test for {@link KvSnapshotLeaseMetadataHelper}. */ -public class KvSnapshotLeaseMetadataHelperTest { +/** Test for {@link KvSnapshotLeaseMetadataManager}. */ +public class KvSnapshotLeaseMetadataManagerTest { @RegisterExtension public static final AllCallbackWrapper ZOO_KEEPER_EXTENSION_WRAPPER = @@ -50,7 +50,7 @@ public class KvSnapshotLeaseMetadataHelperTest { protected static ZooKeeperClient zookeeperClient; private @TempDir Path tempDir; - private KvSnapshotLeaseMetadataHelper metadataHelper; + private KvSnapshotLeaseMetadataManager metadataHelper; @BeforeAll static void beforeAll() { @@ -62,7 +62,7 @@ static void beforeAll() { @BeforeEach void beforeEach() { - metadataHelper = new KvSnapshotLeaseMetadataHelper(zookeeperClient, tempDir.toString()); + metadataHelper = new KvSnapshotLeaseMetadataManager(zookeeperClient, tempDir.toString()); } @AfterEach diff --git a/website/docs/maintenance/observability/monitor-metrics.md b/website/docs/maintenance/observability/monitor-metrics.md index 2d8ef80771..1d8c632805 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 snapshots 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.