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 0d8a68cb22..9bb48a4f17 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; @@ -71,7 +72,9 @@ import java.util.Collection; import java.util.List; +import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.concurrent.CompletableFuture; /** @@ -412,6 +415,57 @@ CompletableFuture dropPartition( CompletableFuture getKvSnapshotMetadata( TableBucket bucket, long snapshotId); + /** + * Acquires a lease for specific KV snapshots of the given tableBuckets asynchronously. + * + *

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

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

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

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

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

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

All snapshots locked under this {@code leaseId} will be released immediately. This is + * equivalent to calling {@link #releaseKvSnapshotLease} with all held buckets. + * + * @param leaseId The lease id to release. + */ + CompletableFuture releaseAllKvSnapshotLease(String leaseId); + /** * Get table lake snapshot info of the given table asynchronously. * diff --git a/fluss-client/src/main/java/org/apache/fluss/client/admin/FlussAdmin.java b/fluss-client/src/main/java/org/apache/fluss/client/admin/FlussAdmin.java index 88aec06c8d..2c1fa69686 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/admin/FlussAdmin.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/admin/FlussAdmin.java @@ -18,6 +18,7 @@ package org.apache.fluss.client.admin; import org.apache.fluss.annotation.VisibleForTesting; +import org.apache.fluss.client.metadata.AcquireKvSnapshotLeaseResult; import org.apache.fluss.client.metadata.KvSnapshotMetadata; import org.apache.fluss.client.metadata.KvSnapshots; import org.apache.fluss.client.metadata.LakeSnapshot; @@ -81,6 +82,7 @@ import org.apache.fluss.rpc.messages.PbTablePath; import org.apache.fluss.rpc.messages.RebalanceRequest; import org.apache.fluss.rpc.messages.RebalanceResponse; +import org.apache.fluss.rpc.messages.ReleaseKvSnapshotLeaseRequest; import org.apache.fluss.rpc.messages.RemoveServerTagRequest; import org.apache.fluss.rpc.messages.TableExistsRequest; import org.apache.fluss.rpc.messages.TableExistsResponse; @@ -98,13 +100,16 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.concurrent.CompletableFuture; +import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makeAcquireKvSnapshotLeaseRequest; import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makeAlterTableRequest; import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makeCreatePartitionRequest; import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makeDropPartitionRequest; import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makeListOffsetsRequest; import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makePbPartitionSpec; +import static org.apache.fluss.client.utils.ClientRpcMessageUtils.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; @@ -388,6 +393,34 @@ public CompletableFuture getKvSnapshotMetadata( .thenApply(ClientRpcMessageUtils::toKvSnapshotMetadata); } + @Override + public CompletableFuture acquireKvSnapshotLease( + String leaseId, Map snapshotIds, long leaseDuration) { + if (snapshotIds.isEmpty()) { + throw new IllegalArgumentException( + "The snapshotIds to acquire kv snapshot lease is empty"); + } + + return gateway.acquireKvSnapshotLease( + makeAcquireKvSnapshotLeaseRequest(leaseId, snapshotIds, leaseDuration)) + .thenApply(ClientRpcMessageUtils::toAcquireKvSnapshotLeaseResult); + } + + @Override + public CompletableFuture releaseKvSnapshotLease( + String leaseId, Set bucketsToRelease) { + return gateway.releaseKvSnapshotLease( + makeReleaseKvSnapshotLeaseRequest(leaseId, bucketsToRelease)) + .thenApply(r -> null); + } + + @Override + public CompletableFuture releaseAllKvSnapshotLease(String leaseId) { + ReleaseKvSnapshotLeaseRequest request = + new ReleaseKvSnapshotLeaseRequest().setLeaseId(leaseId); + return gateway.releaseKvSnapshotLease(request).thenApply(r -> null); + } + @Override public CompletableFuture getLatestLakeSnapshot(TablePath tablePath) { GetLatestLakeSnapshotRequest request = new GetLatestLakeSnapshotRequest(); diff --git a/fluss-client/src/main/java/org/apache/fluss/client/metadata/AcquireKvSnapshotLeaseResult.java b/fluss-client/src/main/java/org/apache/fluss/client/metadata/AcquireKvSnapshotLeaseResult.java new file mode 100644 index 0000000000..3852acd78c --- /dev/null +++ b/fluss-client/src/main/java/org/apache/fluss/client/metadata/AcquireKvSnapshotLeaseResult.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.client.metadata; + +import org.apache.fluss.annotation.PublicEvolving; +import org.apache.fluss.metadata.TableBucket; + +import java.util.Map; +import java.util.Set; + +/** + * A class to represent the result of acquire kv snapshot lease. It contains: + * + *

    + *
  • A map of unavailable snapshots. Such as the specify snapshotId is not exist for this table + * bucket. + *
+ * + * @since 0.9 + */ +@PublicEvolving +public class AcquireKvSnapshotLeaseResult { + private final Map unavailableSnapshots; + + public AcquireKvSnapshotLeaseResult(Map unavailableSnapshots) { + this.unavailableSnapshots = unavailableSnapshots; + } + + /** + * Returns the set of buckets that could not be locked (e.g., snapshot ID doesn't exist or has + * already been GC'ed). + */ + public Map getUnavailableSnapshots() { + return unavailableSnapshots; + } + + public Set getUnavailableTableBucketSet() { + return unavailableSnapshots.keySet(); + } +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/metadata/KvSnapshots.java b/fluss-client/src/main/java/org/apache/fluss/client/metadata/KvSnapshots.java index 9f01ed8f19..91428dcb5e 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/metadata/KvSnapshots.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/metadata/KvSnapshots.java @@ -18,12 +18,14 @@ package org.apache.fluss.client.metadata; import org.apache.fluss.annotation.PublicEvolving; +import org.apache.fluss.metadata.TableBucket; import javax.annotation.Nullable; import java.util.Map; import java.util.OptionalLong; import java.util.Set; +import java.util.stream.Collectors; /** * A class representing the kv snapshots of a table or a partition. It contains multiple snapshots @@ -71,6 +73,12 @@ public Set getBucketIds() { return snapshotIds.keySet(); } + public Set getTableBuckets() { + return snapshotIds.keySet().stream() + .map(bucketId -> new TableBucket(tableId, partitionId, bucketId)) + .collect(Collectors.toSet()); + } + /** * Get the latest snapshot id for this kv tablet (bucket), or empty if there are no snapshots. */ diff --git a/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java b/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java index 2fae90d4b5..844c3f3d70 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; @@ -41,6 +42,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; @@ -55,10 +58,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; @@ -74,6 +80,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; @@ -381,6 +388,75 @@ public static AlterTableRequest makeAlterTableRequest( return request; } + public static AcquireKvSnapshotLeaseRequest makeAcquireKvSnapshotLeaseRequest( + String leaseId, Map snapshotIds, long leaseDuration) { + AcquireKvSnapshotLeaseRequest request = new AcquireKvSnapshotLeaseRequest(); + request.setLeaseId(leaseId).setLeaseDuration(leaseDuration); + + Map> pbLeaseForTables = new HashMap<>(); + for (Map.Entry entry : snapshotIds.entrySet()) { + TableBucket tableBucket = entry.getKey(); + Long snapshotId = entry.getValue(); + PbKvSnapshotLeaseForBucket pbLeaseForBucket = + new PbKvSnapshotLeaseForBucket() + .setBucketId(tableBucket.getBucket()) + .setSnapshotId(snapshotId); + if (tableBucket.getPartitionId() != null) { + pbLeaseForBucket.setPartitionId(tableBucket.getPartitionId()); + } + pbLeaseForTables + .computeIfAbsent(tableBucket.getTableId(), k -> new ArrayList<>()) + .add(pbLeaseForBucket); + } + + for (Map.Entry> entry : + pbLeaseForTables.entrySet()) { + request.addTableLeaseReq() + .setTableId(entry.getKey()) + .addAllBucketsReqs(entry.getValue()); + } + return request; + } + + public static AcquireKvSnapshotLeaseResult toAcquireKvSnapshotLeaseResult( + AcquireKvSnapshotLeaseResponse response) { + Map unavailableSnapshots = new HashMap<>(); + for (PbKvSnapshotLeaseForTable leaseForTable : response.getTablesLeaseResList()) { + long tableId = leaseForTable.getTableId(); + for (PbKvSnapshotLeaseForBucket leaseForBucket : leaseForTable.getBucketsReqsList()) { + TableBucket tableBucket = + new TableBucket( + tableId, + leaseForBucket.hasPartitionId() + ? leaseForBucket.getPartitionId() + : null, + leaseForBucket.getBucketId()); + unavailableSnapshots.put(tableBucket, leaseForBucket.getSnapshotId()); + } + } + return new AcquireKvSnapshotLeaseResult(unavailableSnapshots); + } + + public static ReleaseKvSnapshotLeaseRequest makeReleaseKvSnapshotLeaseRequest( + String leaseId, Set bucketsToRelease) { + ReleaseKvSnapshotLeaseRequest request = new ReleaseKvSnapshotLeaseRequest(); + request.setLeaseId(leaseId); + + Map> pbLeasedTable = new HashMap<>(); + for (TableBucket tb : bucketsToRelease) { + PbBucket pbBucket = new PbBucket().setBucketId(tb.getBucket()); + if (tb.getPartitionId() != null) { + pbBucket.setPartitionId(tb.getPartitionId()); + } + pbLeasedTable.computeIfAbsent(tb.getTableId(), k -> new ArrayList<>()).add(pbBucket); + } + + for (Map.Entry> entry : pbLeasedTable.entrySet()) { + request.addReleaseTable().setTableId(entry.getKey()).addAllBuckets(entry.getValue()); + } + return request; + } + public static Optional toRebalanceProgress( ListRebalanceProgressResponse response) { if (!response.hasRebalanceId()) { diff --git a/fluss-client/src/test/java/org/apache/fluss/client/metadata/TestingClientSchemaGetter.java b/fluss-client/src/test/java/org/apache/fluss/client/metadata/TestingClientSchemaGetter.java index a983eea11e..ffb2c54dc2 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/metadata/TestingClientSchemaGetter.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/metadata/TestingClientSchemaGetter.java @@ -32,13 +32,13 @@ public class TestingClientSchemaGetter extends ClientSchemaGetter { public TestingClientSchemaGetter( TablePath tablePath, SchemaInfo latestSchemaInfo, - TestingMetadataUpdater metadataUpdater) { + TestingMetadataUpdater metadataUpdater, + Configuration conf) { super( tablePath, latestSchemaInfo, new FlussAdmin( - RpcClient.create( - new Configuration(), TestingClientMetricGroup.newInstance(), false), + RpcClient.create(conf, TestingClientMetricGroup.newInstance(), false), metadataUpdater)); } diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvSnapshotBatchScannerITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvSnapshotBatchScannerITCase.java index 2ab2f37265..2e99f86a20 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvSnapshotBatchScannerITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvSnapshotBatchScannerITCase.java @@ -24,6 +24,7 @@ import org.apache.fluss.client.table.scanner.RemoteFileDownloader; import org.apache.fluss.client.table.writer.UpsertWriter; import org.apache.fluss.client.write.HashBucketAssigner; +import org.apache.fluss.exception.KvSnapshotLeaseNotExistException; import org.apache.fluss.metadata.DataLakeFormat; import org.apache.fluss.metadata.Schema; import org.apache.fluss.metadata.TableBucket; @@ -34,21 +35,28 @@ import org.apache.fluss.row.ProjectedRow; import org.apache.fluss.row.encode.CompactedKeyEncoder; import org.apache.fluss.row.encode.KeyEncoder; +import org.apache.fluss.server.zk.ZooKeeperClient; +import org.apache.fluss.server.zk.data.lease.KvSnapshotLease; +import org.apache.fluss.server.zk.data.lease.KvSnapshotLeaseMetadataManager; +import org.apache.fluss.server.zk.data.lease.KvSnapshotTableLease; import org.apache.fluss.types.DataTypes; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import java.time.Duration; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import static org.apache.fluss.record.TestData.DATA1_ROW_TYPE; import static org.apache.fluss.testutils.DataTestUtils.compactedRow; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** IT Case for {@link KvSnapshotBatchScanner}. */ class KvSnapshotBatchScannerITCase extends ClientToServerITCaseBase { @@ -180,6 +188,97 @@ void testScanSnapshotDuringSchemaChange() throws Exception { testSnapshotRead(tablePath, expectedRowByBuckets); } + @Test + public void testKvSnapshotLease() throws Exception { + TablePath tablePath = TablePath.of(DEFAULT_DB, "test-kv-snapshot-lease"); + long tableId = createTable(tablePath, DEFAULT_TABLE_DESCRIPTOR, true); + + String kvSnapshotLease1 = "test-lease"; + String kvSnapshotLease2 = "test-lease2"; + + // scan the snapshot + Map> expectedRowByBuckets = putRows(tableId, tablePath, 10); + + // wait snapshot finish + FLUSS_CLUSTER_EXTENSION.triggerAndWaitSnapshots(expectedRowByBuckets.keySet()); + + ZooKeeperClient zkClient = FLUSS_CLUSTER_EXTENSION.getZooKeeperClient(); + String remoteDataDir = FLUSS_CLUSTER_EXTENSION.getRemoteDataDir(); + KvSnapshotLeaseMetadataManager metadataManager = + new KvSnapshotLeaseMetadataManager(zkClient, remoteDataDir); + + assertThat(zkClient.getKvSnapshotLeasesList()).isEmpty(); + + // test register kv snapshot lease for snapshot 0. + Map consumeBuckets = new HashMap<>(); + KvSnapshots kvSnapshots = admin.getLatestKvSnapshots(tablePath).get(); + for (int bucketId : kvSnapshots.getBucketIds()) { + TableBucket tableBucket = new TableBucket(kvSnapshots.getTableId(), bucketId); + consumeBuckets.put(tableBucket, kvSnapshots.getSnapshotId(bucketId).getAsLong()); + } + admin.acquireKvSnapshotLease( + kvSnapshotLease1, consumeBuckets, Duration.ofDays(1).toMillis()) + .get(); + checkKvSnapshotLeaseEquals( + metadataManager, kvSnapshotLease1, tableId, new Long[] {0L, 0L, 0L}); + + expectedRowByBuckets = putRows(tableId, tablePath, 10); + // wait snapshot2 finish + FLUSS_CLUSTER_EXTENSION.triggerAndWaitSnapshots(expectedRowByBuckets.keySet()); + + // test register kv snapshot lease for snapshot 1. + consumeBuckets = new HashMap<>(); + kvSnapshots = admin.getLatestKvSnapshots(tablePath).get(); + for (int bucketId : kvSnapshots.getBucketIds()) { + TableBucket tableBucket = new TableBucket(kvSnapshots.getTableId(), bucketId); + consumeBuckets.put(tableBucket, kvSnapshots.getSnapshotId(bucketId).getAsLong()); + } + admin.acquireKvSnapshotLease( + kvSnapshotLease2, consumeBuckets, Duration.ofDays(1).toMillis()) + .get(); + checkKvSnapshotLeaseEquals( + metadataManager, kvSnapshotLease2, tableId, new Long[] {1L, 1L, 1L}); + // check even snapshot1 is generated, snapshot0 also retained as lease exists. + for (TableBucket tb : expectedRowByBuckets.keySet()) { + assertThat(zkClient.getTableBucketSnapshot(tb, 0L).isPresent()).isTrue(); + assertThat(zkClient.getTableBucketSnapshot(tb, 1L).isPresent()).isTrue(); + } + + expectedRowByBuckets = putRows(tableId, tablePath, 10); + // wait snapshot3 finish + FLUSS_CLUSTER_EXTENSION.triggerAndWaitSnapshots(expectedRowByBuckets.keySet()); + + // release lease1. + admin.releaseKvSnapshotLease( + kvSnapshotLease1, Collections.singleton(new TableBucket(tableId, 0))) + .get(); + checkKvSnapshotLeaseEquals( + metadataManager, kvSnapshotLease1, tableId, new Long[] {-1L, 0L, 0L}); + + // release lease2. + admin.releaseKvSnapshotLease(kvSnapshotLease2, consumeBuckets.keySet()).get(); + assertThat(zkClient.getKvSnapshotLeasesList()).doesNotContain(kvSnapshotLease2); + + // release all kv snapshot lease of lease1 + admin.releaseAllKvSnapshotLease(kvSnapshotLease1).get(); + assertThat(zkClient.getKvSnapshotLeasesList()).isEmpty(); + + expectedRowByBuckets = putRows(tableId, tablePath, 10); + // wait snapshot2 finish + FLUSS_CLUSTER_EXTENSION.triggerAndWaitSnapshots(expectedRowByBuckets.keySet()); + // as all leases are dropped, and new snapshot is generated, all old snapshot are + // cleared. + for (TableBucket tb : expectedRowByBuckets.keySet()) { + assertThat(zkClient.getTableBucketSnapshot(tb, 0L).isPresent()).isFalse(); + assertThat(zkClient.getTableBucketSnapshot(tb, 1L).isPresent()).isFalse(); + } + + assertThatThrownBy(() -> admin.releaseAllKvSnapshotLease("no-exist-lease").get()) + .rootCause() + .isInstanceOf(KvSnapshotLeaseNotExistException.class) + .hasMessageContaining("kv snapshot lease 'no-exist-lease' not exits"); + } + private Map> putRows( long tableId, TablePath tablePath, int rowNumber) throws Exception { List rows = new ArrayList<>(); @@ -238,4 +337,20 @@ private static int getBucketId(InternalRow row) { byte[] key = keyEncoder.encodeKey(row); return function.bucketing(key, DEFAULT_BUCKET_NUM); } + + private void checkKvSnapshotLeaseEquals( + KvSnapshotLeaseMetadataManager metadataManager, + String leaseId, + long tableId, + Long[] expectedBucketIndex) + throws Exception { + assertThat(metadataManager.getLeasesList()).contains(leaseId); + Optional leaseOpt = metadataManager.getLease(leaseId); + assertThat(leaseOpt).isPresent(); + KvSnapshotLease actualLease = leaseOpt.get(); + Map tableIdToTableLease = actualLease.getTableIdToTableLease(); + KvSnapshotTableLease tableLease = tableIdToTableLease.get(tableId); + assertThat(tableLease).isNotNull(); + assertThat(tableLease.getBucketSnapshots()).isEqualTo(expectedBucketIndex); + } } diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherTest.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherTest.java index f06f886142..a01809c58c 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherTest.java @@ -70,7 +70,10 @@ public void setup() { metadataUpdater = initializeMetadataUpdater(); ClientSchemaGetter clientSchemaGetter = new TestingClientSchemaGetter( - DATA1_TABLE_PATH, new SchemaInfo(DATA1_SCHEMA, 0), metadataUpdater); + DATA1_TABLE_PATH, + new SchemaInfo(DATA1_SCHEMA, 0), + metadataUpdater, + new Configuration()); LogScannerStatus logScannerStatus = initializeLogScannerStatus(); logFetcher = new LogFetcher( diff --git a/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java b/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java index 7e04c3a1cc..520b17b40e 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 @@ -1506,9 +1506,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 60eb942e8b..50f6a4d27c 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,11 @@ public class MetricNames { public static final String KV_ALL_SNAPSHOT_SIZE = "allKvSnapshotSize"; public static final String SERVER_PHYSICAL_STORAGE_REMOTE_KV_SIZE = "remoteKvSize"; + // for kv snapshot lease. + // TODO implemented it at the table level. Trace by: https://github.com/apache/fluss/issues/2297 + public static final String KV_SNAPSHOT_LEASE_COUNT = "kvSnapshotLeaseCount"; + public static final String LEASED_KV_SNAPSHOT_COUNT = "leasedKvSnapshotCount"; + // -------------------------------------------------------------------------------------------- // metrics for tablet server // -------------------------------------------------------------------------------------------- diff --git a/fluss-common/src/main/java/org/apache/fluss/utils/FlussPaths.java b/fluss-common/src/main/java/org/apache/fluss/utils/FlussPaths.java index 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-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/FlinkConnectorOptions.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/FlinkConnectorOptions.java index c62292c69b..7772dce562 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/FlinkConnectorOptions.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/FlinkConnectorOptions.java @@ -31,6 +31,7 @@ import java.time.Duration; import java.util.Arrays; import java.util.List; +import java.util.UUID; import static org.apache.flink.configuration.description.TextElement.text; @@ -76,6 +77,23 @@ public class FlinkConnectorOptions { "A list of host/port pairs to use for establishing the initial connection to the Fluss cluster. " + "The list should be in the form host1:port1,host2:port2,...."); + public static final ConfigOption SCAN_KV_SNAPSHOT_LEASE_ID = + ConfigOptions.key("scan.kv.snapshot.lease.id") + .stringType() + .defaultValue(String.valueOf(UUID.randomUUID())) + .withDescription( + "The lease id to ping kv snapshots. If set, the acquired kv snapshots will not be deleted " + + "until the consumer finished consuming all the snapshots or the lease duration time " + + "is reached. If not set, an UUID will be set."); + + public static final ConfigOption SCAN_KV_SNAPSHOT_LEASE_DURATION = + ConfigOptions.key("scan.kv.snapshot.lease.duration") + .durationType() + .defaultValue(Duration.ofDays(1)) + .withDescription( + "The time period how long to wait before expiring the kv snapshot lease to " + + "avoid kv snapshot blocking to delete."); + // -------------------------------------------------------------------------------------------- // Lookup specific options // -------------------------------------------------------------------------------------------- diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/catalog/FlinkTableFactory.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/catalog/FlinkTableFactory.java index 48d74e712e..389ae7690a 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 @@ -25,6 +25,7 @@ import org.apache.fluss.flink.sink.FlinkTableSink; import org.apache.fluss.flink.sink.shuffle.DistributionMode; 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; @@ -138,6 +139,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( @@ -153,7 +163,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 @@ -215,6 +226,8 @@ public Set> optionalOptions() { FlinkConnectorOptions.SCAN_STARTUP_MODE, FlinkConnectorOptions.SCAN_STARTUP_TIMESTAMP, FlinkConnectorOptions.SCAN_PARTITION_DISCOVERY_INTERVAL, + FlinkConnectorOptions.SCAN_KV_SNAPSHOT_LEASE_ID, + FlinkConnectorOptions.SCAN_KV_SNAPSHOT_LEASE_DURATION, FlinkConnectorOptions.LOOKUP_ASYNC, FlinkConnectorOptions.SINK_IGNORE_DELETE, FlinkConnectorOptions.SINK_BUCKET_SHUFFLE, diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/ProcedureManager.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/ProcedureManager.java index b97b3cd8e0..a0d0d54e0d 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/ProcedureManager.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/ProcedureManager.java @@ -77,7 +77,9 @@ private enum ProcedureEnum { REMOVE_SERVER_TAG("sys.remove_server_tag", RemoveServerTagProcedure.class), REBALANCE("sys.rebalance", RebalanceProcedure.class), CANCEL_REBALANCE("sys.cancel_rebalance", CancelRebalanceProcedure.class), - LIST_REBALANCE_PROGRESS("sys.list_rebalance", ListRebalanceProcessProcedure.class); + LIST_REBALANCE_PROGRESS("sys.list_rebalance", ListRebalanceProcessProcedure.class), + RELEASE_ALL_KV_SNAPSHOT_LEASE( + "sys.release_all_kv_snapshot_lease", ReleaseAllKvSnapshotLeaseProcedure.class); private final String path; private final Class procedureClass; diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/ReleaseAllKvSnapshotLeaseProcedure.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/ReleaseAllKvSnapshotLeaseProcedure.java new file mode 100644 index 0000000000..ba6e9c76ff --- /dev/null +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/ReleaseAllKvSnapshotLeaseProcedure.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.flink.procedure; + +import org.apache.flink.table.annotation.ArgumentHint; +import org.apache.flink.table.annotation.DataTypeHint; +import org.apache.flink.table.annotation.ProcedureHint; +import org.apache.flink.table.procedure.ProcedureContext; + +/** + * Procedure to release all kv snapshots leased of specified leaseId. See {@link + * org.apache.fluss.client.admin.Admin#releaseAllKvSnapshotLease(String)} for more details. + * + *

Usage examples: + * + *

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