platinumhamburg commented on code in PR #2179:
URL: https://github.com/apache/fluss/pull/2179#discussion_r2654917690


##########
fluss-client/src/main/java/org/apache/fluss/client/admin/Admin.java:
##########
@@ -408,6 +411,56 @@ CompletableFuture<Void> dropPartition(
     CompletableFuture<KvSnapshotMetadata> getKvSnapshotMetadata(
             TableBucket bucket, long snapshotId);
 
+    /**
+     * Acquires a lease for specific KV snapshots of the given tableBuckets 
asynchronously.
+     *
+     * <p>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.
+     *
+     * <p>If the lease expires (no renew received within duration), the server 
is free to delete the
+     * snapshot files.
+     *
+     * <p>The following exceptions can be anticipated when calling {@code 
get()} on returned future:
+     *
+     * <ul>
+     *   <li>{@link TableNotExistException} if the table does not exist.
+     *   <li>{@link PartitionNotExistException} if the partition does not 
exist.
+     * </ul>
+     *
+     * @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<AcquireKvSnapshotLeaseResult> acquireKvSnapshotLease(
+            String leaseId, Map<TableBucket, Long> snapshotIds, long 
leaseDuration);
+
+    /**
+     * Releases the lease for specific tableBuckets asynchronously.
+     *
+     * <p>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.
+     *
+     * <p>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<Void> releaseKvSnapshotLease(
+            String leaseId, Set<TableBucket> bucketsToRelease);
+
+    /**
+     * Drops the entire lease asynchronously.
+     *
+     * <p>All snapshots locked under this {@code leaseId} will be released 
immediately. This is
+     * equivalent to calling {@link #releaseKvSnapshotLease} with all held 
buckets.
+     *
+     * @param leaseId The lease id to drop.
+     */
+    CompletableFuture<Void> dropKvSnapshotLease(String leaseId);

Review Comment:
   ditto



##########
fluss-client/src/main/java/org/apache/fluss/client/admin/Admin.java:
##########
@@ -408,6 +411,56 @@ CompletableFuture<Void> dropPartition(
     CompletableFuture<KvSnapshotMetadata> getKvSnapshotMetadata(
             TableBucket bucket, long snapshotId);
 
+    /**
+     * Acquires a lease for specific KV snapshots of the given tableBuckets 
asynchronously.
+     *
+     * <p>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.
+     *
+     * <p>If the lease expires (no renew received within duration), the server 
is free to delete the
+     * snapshot files.
+     *
+     * <p>The following exceptions can be anticipated when calling {@code 
get()} on returned future:
+     *
+     * <ul>
+     *   <li>{@link TableNotExistException} if the table does not exist.
+     *   <li>{@link PartitionNotExistException} if the partition does not 
exist.
+     * </ul>
+     *
+     * @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<AcquireKvSnapshotLeaseResult> acquireKvSnapshotLease(
+            String leaseId, Map<TableBucket, Long> snapshotIds, long 
leaseDuration);
+
+    /**
+     * Releases the lease for specific tableBuckets asynchronously.
+     *
+     * <p>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.
+     *
+     * <p>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<Void> releaseKvSnapshotLease(
+            String leaseId, Set<TableBucket> bucketsToRelease);
+
+    /**
+     * Drops the entire lease asynchronously.

Review Comment:
   Prefer 'releases' over 'drops'—more precise and idiomatic for leases.



##########
fluss-server/src/main/java/org/apache/fluss/server/coordinator/KvSnapshotLeaseManager.java:
##########
@@ -0,0 +1,453 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.fluss.server.coordinator;
+
+import org.apache.fluss.annotation.VisibleForTesting;
+import org.apache.fluss.config.ConfigOptions;
+import org.apache.fluss.config.Configuration;
+import org.apache.fluss.metadata.KvSnapshotLeaseForBucket;
+import org.apache.fluss.metadata.TableBucket;
+import org.apache.fluss.metadata.TableInfo;
+import org.apache.fluss.metrics.MetricNames;
+import org.apache.fluss.server.metrics.group.CoordinatorMetricGroup;
+import org.apache.fluss.server.zk.data.lease.KvSnapshotLease;
+import org.apache.fluss.server.zk.data.lease.KvSnapshotLeaseMetadataHelper;
+import org.apache.fluss.server.zk.data.lease.KvSnapshotTableLease;
+import org.apache.fluss.utils.MapUtils;
+import org.apache.fluss.utils.clock.Clock;
+import org.apache.fluss.utils.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.fluss.utils.concurrent.LockUtils.inReadLock;
+import static org.apache.fluss.utils.concurrent.LockUtils.inWriteLock;
+
+/** A manager to manage kv snapshot lease acquire, renew, release and drop. */
+@ThreadSafe
+public class KvSnapshotLeaseManager {
+    private static final Logger LOG = 
LoggerFactory.getLogger(KvSnapshotLeaseManager.class);
+
+    private final KvSnapshotLeaseMetadataHelper metadataHelper;
+    private final CoordinatorContext coordinatorContext;
+    private final Clock clock;
+    private final ScheduledExecutorService scheduledExecutor;
+    private final Configuration conf;
+
+    private final Map<String, ReadWriteLock> leaseLocks = 
MapUtils.newConcurrentHashMap();
+    /** lease id to kv snapshot lease. */
+    @GuardedBy("leaseLocks")
+    private final Map<String, KvSnapshotLease> kvSnapshotLeaseMap;
+
+    private final ReadWriteLock refCountLock = new ReentrantReadWriteLock();
+
+    /**
+     * KvSnapshotLeaseForBucket to the ref count, which means this table 
bucket + snapshotId has
+     * been leased by how many lease id.
+     */
+    @GuardedBy("refCountLock")

Review Comment:
   I'm not entirely sure why the refCountLock is needed—it doesn't seem to be 
protecting multiple atomic states. Could using just a ConcurrentHashMap along 
with Atomic types lead to consistency issues?
   
   



##########
fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/procedure/DropKvSnapshotLeaseProcedure.java:
##########
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.fluss.flink.procedure;
+
+import org.apache.flink.table.annotation.ArgumentHint;
+import org.apache.flink.table.annotation.DataTypeHint;
+import org.apache.flink.table.annotation.ProcedureHint;
+import org.apache.flink.table.procedure.ProcedureContext;
+
+/** Procedure to drop kv snapshot lease. */
+public class DropKvSnapshotLeaseProcedure extends ProcedureBase {

Review Comment:
   Ditto—“drop” isn’t the most appropriate verb to use with “lease.”
   
   



##########
fluss-common/src/main/java/org/apache/fluss/metrics/MetricNames.java:
##########
@@ -53,6 +53,10 @@ public class MetricNames {
     public static final String KV_ALL_SNAPSHOT_SIZE = "allKvSnapshotSize";
     public static final String SERVER_PHYSICAL_STORAGE_REMOTE_KV_SIZE = 
"remoteKvSize";
 
+    // for kv snapshot lease.
+    public static final String KV_SNAPSHOT_LEASE_COUNT = 
"kvSnapshotLeaseCount";

Review Comment:
   I think it should be implemented at the table level so we can quickly 
identify the affected business when lease leaks occur.
   
   



##########
fluss-server/src/main/java/org/apache/fluss/server/coordinator/KvSnapshotLeaseManager.java:
##########
@@ -0,0 +1,453 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.fluss.server.coordinator;
+
+import org.apache.fluss.annotation.VisibleForTesting;
+import org.apache.fluss.config.ConfigOptions;
+import org.apache.fluss.config.Configuration;
+import org.apache.fluss.metadata.KvSnapshotLeaseForBucket;
+import org.apache.fluss.metadata.TableBucket;
+import org.apache.fluss.metadata.TableInfo;
+import org.apache.fluss.metrics.MetricNames;
+import org.apache.fluss.server.metrics.group.CoordinatorMetricGroup;
+import org.apache.fluss.server.zk.data.lease.KvSnapshotLease;
+import org.apache.fluss.server.zk.data.lease.KvSnapshotLeaseMetadataHelper;
+import org.apache.fluss.server.zk.data.lease.KvSnapshotTableLease;
+import org.apache.fluss.utils.MapUtils;
+import org.apache.fluss.utils.clock.Clock;
+import org.apache.fluss.utils.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.fluss.utils.concurrent.LockUtils.inReadLock;
+import static org.apache.fluss.utils.concurrent.LockUtils.inWriteLock;
+
+/** A manager to manage kv snapshot lease acquire, renew, release and drop. */
+@ThreadSafe
+public class KvSnapshotLeaseManager {
+    private static final Logger LOG = 
LoggerFactory.getLogger(KvSnapshotLeaseManager.class);
+
+    private final KvSnapshotLeaseMetadataHelper metadataHelper;
+    private final CoordinatorContext coordinatorContext;
+    private final Clock clock;
+    private final ScheduledExecutorService scheduledExecutor;
+    private final Configuration conf;
+
+    private final Map<String, ReadWriteLock> leaseLocks = 
MapUtils.newConcurrentHashMap();
+    /** lease id to kv snapshot lease. */
+    @GuardedBy("leaseLocks")
+    private final Map<String, KvSnapshotLease> kvSnapshotLeaseMap;
+
+    private final ReadWriteLock refCountLock = new ReentrantReadWriteLock();
+
+    /**
+     * KvSnapshotLeaseForBucket to the ref count, which means this table 
bucket + snapshotId has
+     * been leased by how many lease id.
+     */
+    @GuardedBy("refCountLock")
+    private final Map<KvSnapshotLeaseForBucket, AtomicInteger> refCount =
+            MapUtils.newConcurrentHashMap();
+
+    /** For metrics. */
+    private final AtomicInteger leasedBucketCount = new AtomicInteger(0);
+
+    public KvSnapshotLeaseManager(
+            Configuration conf,
+            KvSnapshotLeaseMetadataHelper metadataHelper,
+            CoordinatorContext coordinatorContext,
+            Clock clock,
+            CoordinatorMetricGroup coordinatorMetricGroup) {
+        this(
+                conf,
+                metadataHelper,
+                coordinatorContext,
+                Executors.newScheduledThreadPool(
+                        1, new 
ExecutorThreadFactory("kv-snapshot-lease-cleaner")),
+                clock,
+                coordinatorMetricGroup);
+    }
+
+    @VisibleForTesting
+    public KvSnapshotLeaseManager(
+            Configuration conf,
+            KvSnapshotLeaseMetadataHelper metadataHelper,
+            CoordinatorContext coordinatorContext,
+            ScheduledExecutorService scheduledExecutor,
+            Clock clock,
+            CoordinatorMetricGroup coordinatorMetricGroup) {
+        this.metadataHelper = metadataHelper;
+        this.conf = conf;
+        this.scheduledExecutor = scheduledExecutor;
+        this.coordinatorContext = coordinatorContext;
+        this.clock = clock;
+        this.kvSnapshotLeaseMap = MapUtils.newConcurrentHashMap();
+
+        registerMetrics(coordinatorMetricGroup);
+    }
+
+    public void start() {
+        LOG.info("kv snapshot lease manager has been started.");
+        scheduledExecutor.scheduleWithFixedDelay(
+                this::expireLeases,
+                0L,
+                
conf.get(ConfigOptions.KV_SNAPSHOT_LEASE_EXPIRATION_CHECK_INTERVAL).toMillis(),
+                TimeUnit.MILLISECONDS);
+    }
+
+    public void initialize() throws Exception {
+        List<String> leaseIds = metadataHelper.getLeasesList();

Review Comment:
   Instead of getLeasesList() returning a List<String>, it could directly 
return a List<KvSnapshotLease>.



##########
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<KvSnapshotLeaseMetadata>,
+                JsonDeserializer<KvSnapshotLeaseMetadata> {
+
+    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)

Review Comment:
   I noticed we're using Jackson's very low-level APIs, which makes the 
serialization/deserialization logic quite complex. What's the reason behind 
this approach?
   
   



##########
fluss-server/src/main/java/org/apache/fluss/server/coordinator/KvSnapshotLeaseManager.java:
##########
@@ -0,0 +1,453 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.fluss.server.coordinator;
+
+import org.apache.fluss.annotation.VisibleForTesting;
+import org.apache.fluss.config.ConfigOptions;
+import org.apache.fluss.config.Configuration;
+import org.apache.fluss.metadata.KvSnapshotLeaseForBucket;
+import org.apache.fluss.metadata.TableBucket;
+import org.apache.fluss.metadata.TableInfo;
+import org.apache.fluss.metrics.MetricNames;
+import org.apache.fluss.server.metrics.group.CoordinatorMetricGroup;
+import org.apache.fluss.server.zk.data.lease.KvSnapshotLease;
+import org.apache.fluss.server.zk.data.lease.KvSnapshotLeaseMetadataHelper;
+import org.apache.fluss.server.zk.data.lease.KvSnapshotTableLease;
+import org.apache.fluss.utils.MapUtils;
+import org.apache.fluss.utils.clock.Clock;
+import org.apache.fluss.utils.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.fluss.utils.concurrent.LockUtils.inReadLock;
+import static org.apache.fluss.utils.concurrent.LockUtils.inWriteLock;
+
+/** A manager to manage kv snapshot lease acquire, renew, release and drop. */
+@ThreadSafe
+public class KvSnapshotLeaseManager {
+    private static final Logger LOG = 
LoggerFactory.getLogger(KvSnapshotLeaseManager.class);
+
+    private final KvSnapshotLeaseMetadataHelper metadataHelper;

Review Comment:
   The usage of KvSnapshotLeaseMetadataHelper involves both metadata queries 
and updates. The suffix "Helper" typically implies a stateless utility class, 
which doesn't align well with the actual role of KvSnapshotLeaseMetadataHelper. 
It might be more appropriate to rename it to KvSnapshotLeaseMetadataManager.
   
   



##########
fluss-server/src/main/java/org/apache/fluss/server/coordinator/KvSnapshotLeaseManager.java:
##########
@@ -0,0 +1,453 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.fluss.server.coordinator;
+
+import org.apache.fluss.annotation.VisibleForTesting;
+import org.apache.fluss.config.ConfigOptions;
+import org.apache.fluss.config.Configuration;
+import org.apache.fluss.metadata.KvSnapshotLeaseForBucket;
+import org.apache.fluss.metadata.TableBucket;
+import org.apache.fluss.metadata.TableInfo;
+import org.apache.fluss.metrics.MetricNames;
+import org.apache.fluss.server.metrics.group.CoordinatorMetricGroup;
+import org.apache.fluss.server.zk.data.lease.KvSnapshotLease;
+import org.apache.fluss.server.zk.data.lease.KvSnapshotLeaseMetadataHelper;
+import org.apache.fluss.server.zk.data.lease.KvSnapshotTableLease;
+import org.apache.fluss.utils.MapUtils;
+import org.apache.fluss.utils.clock.Clock;
+import org.apache.fluss.utils.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.fluss.utils.concurrent.LockUtils.inReadLock;
+import static org.apache.fluss.utils.concurrent.LockUtils.inWriteLock;
+
+/** A manager to manage kv snapshot lease acquire, renew, release and drop. */
+@ThreadSafe
+public class KvSnapshotLeaseManager {
+    private static final Logger LOG = 
LoggerFactory.getLogger(KvSnapshotLeaseManager.class);
+
+    private final KvSnapshotLeaseMetadataHelper metadataHelper;
+    private final CoordinatorContext coordinatorContext;
+    private final Clock clock;
+    private final ScheduledExecutorService scheduledExecutor;
+    private final Configuration conf;
+
+    private final Map<String, ReadWriteLock> leaseLocks = 
MapUtils.newConcurrentHashMap();
+    /** lease id to kv snapshot lease. */
+    @GuardedBy("leaseLocks")
+    private final Map<String, KvSnapshotLease> kvSnapshotLeaseMap;
+
+    private final ReadWriteLock refCountLock = new ReentrantReadWriteLock();
+
+    /**
+     * KvSnapshotLeaseForBucket to the ref count, which means this table 
bucket + snapshotId has
+     * been leased by how many lease id.
+     */
+    @GuardedBy("refCountLock")
+    private final Map<KvSnapshotLeaseForBucket, AtomicInteger> refCount =
+            MapUtils.newConcurrentHashMap();
+
+    /** For metrics. */
+    private final AtomicInteger leasedBucketCount = new AtomicInteger(0);
+
+    public KvSnapshotLeaseManager(
+            Configuration conf,
+            KvSnapshotLeaseMetadataHelper metadataHelper,
+            CoordinatorContext coordinatorContext,
+            Clock clock,
+            CoordinatorMetricGroup coordinatorMetricGroup) {
+        this(
+                conf,
+                metadataHelper,
+                coordinatorContext,
+                Executors.newScheduledThreadPool(
+                        1, new 
ExecutorThreadFactory("kv-snapshot-lease-cleaner")),
+                clock,
+                coordinatorMetricGroup);
+    }
+
+    @VisibleForTesting
+    public KvSnapshotLeaseManager(
+            Configuration conf,
+            KvSnapshotLeaseMetadataHelper metadataHelper,
+            CoordinatorContext coordinatorContext,
+            ScheduledExecutorService scheduledExecutor,
+            Clock clock,
+            CoordinatorMetricGroup coordinatorMetricGroup) {
+        this.metadataHelper = metadataHelper;
+        this.conf = conf;
+        this.scheduledExecutor = scheduledExecutor;
+        this.coordinatorContext = coordinatorContext;
+        this.clock = clock;
+        this.kvSnapshotLeaseMap = MapUtils.newConcurrentHashMap();
+
+        registerMetrics(coordinatorMetricGroup);
+    }
+
+    public void start() {
+        LOG.info("kv snapshot lease manager has been started.");
+        scheduledExecutor.scheduleWithFixedDelay(
+                this::expireLeases,
+                0L,
+                
conf.get(ConfigOptions.KV_SNAPSHOT_LEASE_EXPIRATION_CHECK_INTERVAL).toMillis(),
+                TimeUnit.MILLISECONDS);
+    }
+
+    public void initialize() throws Exception {
+        List<String> leaseIds = metadataHelper.getLeasesList();
+        for (String leaseId : leaseIds) {
+            Optional<KvSnapshotLease> kvSnapshotLeaseOpt = 
metadataHelper.getLease(leaseId);
+            if (kvSnapshotLeaseOpt.isPresent()) {
+                KvSnapshotLease kvSnapshotLease = kvSnapshotLeaseOpt.get();
+                this.leaseLocks.put(leaseId, new ReentrantReadWriteLock());
+                this.kvSnapshotLeaseMap.put(leaseId, kvSnapshotLease);
+
+                initializeRefCount(kvSnapshotLease);
+
+                
leasedBucketCount.addAndGet(kvSnapshotLease.getLeasedSnapshotCount());
+            }
+        }
+    }
+
+    public boolean snapshotLeaseNotExist(KvSnapshotLeaseForBucket 
kvSnapshotLeaseForBucket) {
+        return inReadLock(
+                refCountLock,
+                () ->
+                        !refCount.containsKey(kvSnapshotLeaseForBucket)
+                                || 
refCount.get(kvSnapshotLeaseForBucket).get() <= 0);

Review Comment:
   This code can be optimized to call get only once.



##########
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 {

Review Comment:
   A lease inherently implies a time-bound reservation—without a TTL, it’s just 
a static reference. Consider adding a ttl (e.g., in milliseconds) to this class 
to properly reflect lease semantics.
   
   



##########
fluss-server/src/test/java/org/apache/fluss/server/coordinator/KvSnapshotLeaseManagerTest.java:
##########
@@ -0,0 +1,481 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.fluss.server.coordinator;
+
+import org.apache.fluss.config.ConfigOptions;
+import org.apache.fluss.config.Configuration;
+import org.apache.fluss.metadata.KvSnapshotLeaseForBucket;
+import org.apache.fluss.metadata.PhysicalTablePath;
+import org.apache.fluss.metadata.TableBucket;
+import org.apache.fluss.metadata.TableInfo;
+import org.apache.fluss.metadata.TablePath;
+import org.apache.fluss.server.metrics.group.TestingMetricGroups;
+import org.apache.fluss.server.zk.NOPErrorHandler;
+import org.apache.fluss.server.zk.ZooKeeperClient;
+import org.apache.fluss.server.zk.ZooKeeperExtension;
+import org.apache.fluss.server.zk.data.BucketSnapshot;
+import org.apache.fluss.server.zk.data.lease.KvSnapshotLease;
+import org.apache.fluss.server.zk.data.lease.KvSnapshotLeaseMetadataHelper;
+import org.apache.fluss.server.zk.data.lease.KvSnapshotTableLease;
+import org.apache.fluss.testutils.common.AllCallbackWrapper;
+import 
org.apache.fluss.testutils.common.ManuallyTriggeredScheduledExecutorService;
+import org.apache.fluss.utils.clock.ManualClock;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.nio.file.Path;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import static 
org.apache.fluss.record.TestData.DATA1_PARTITIONED_TABLE_DESCRIPTOR;
+import static org.apache.fluss.record.TestData.DATA1_TABLE_ID_PK;
+import static org.apache.fluss.record.TestData.DATA1_TABLE_INFO_PK;
+import static org.apache.fluss.record.TestData.DATA1_TABLE_PATH_PK;
+import static 
org.apache.fluss.server.coordinator.CoordinatorTestUtils.createServers;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test for {@link KvSnapshotLeaseManager}. */
+public class KvSnapshotLeaseManagerTest {
+
+    @RegisterExtension
+    public static final AllCallbackWrapper<ZooKeeperExtension> 
ZOO_KEEPER_EXTENSION_WRAPPER =
+            new AllCallbackWrapper<>(new ZooKeeperExtension());
+
+    private static final TablePath PARTITION_TABLE_PATH =
+            new TablePath("test_db_1", "test_partition_table");
+    private static final long PARTITION_TABLE_ID = 150008L;
+    private static final TableInfo PARTITION_TABLE_INFO =

Review Comment:
   Can we reuse TestData, or should we add the test data into TestData?
   
   



##########
fluss-common/src/main/java/org/apache/fluss/metrics/MetricNames.java:
##########
@@ -53,6 +53,10 @@ public class MetricNames {
     public static final String KV_ALL_SNAPSHOT_SIZE = "allKvSnapshotSize";
     public static final String SERVER_PHYSICAL_STORAGE_REMOTE_KV_SIZE = 
"remoteKvSize";
 
+    // for kv snapshot lease.
+    public static final String KV_SNAPSHOT_LEASE_COUNT = 
"kvSnapshotLeaseCount";
+    public static final String LEASED_KV_SNAPSHOT_COUNT = 
"leasedKvSnapshotCount";

Review Comment:
   Please add documentation to monitor-metrics.md.
   
   
   



##########
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<KvSnapshotTableLease>, 
JsonDeserializer<KvSnapshotTableLease> {
+
+    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)

Review Comment:
   ditto



##########
fluss-server/src/main/java/org/apache/fluss/server/coordinator/KvSnapshotLeaseManager.java:
##########
@@ -0,0 +1,453 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.fluss.server.coordinator;
+
+import org.apache.fluss.annotation.VisibleForTesting;
+import org.apache.fluss.config.ConfigOptions;
+import org.apache.fluss.config.Configuration;
+import org.apache.fluss.metadata.KvSnapshotLeaseForBucket;
+import org.apache.fluss.metadata.TableBucket;
+import org.apache.fluss.metadata.TableInfo;
+import org.apache.fluss.metrics.MetricNames;
+import org.apache.fluss.server.metrics.group.CoordinatorMetricGroup;
+import org.apache.fluss.server.zk.data.lease.KvSnapshotLease;
+import org.apache.fluss.server.zk.data.lease.KvSnapshotLeaseMetadataHelper;
+import org.apache.fluss.server.zk.data.lease.KvSnapshotTableLease;
+import org.apache.fluss.utils.MapUtils;
+import org.apache.fluss.utils.clock.Clock;
+import org.apache.fluss.utils.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.fluss.utils.concurrent.LockUtils.inReadLock;
+import static org.apache.fluss.utils.concurrent.LockUtils.inWriteLock;
+
+/** A manager to manage kv snapshot lease acquire, renew, release and drop. */
+@ThreadSafe
+public class KvSnapshotLeaseManager {
+    private static final Logger LOG = 
LoggerFactory.getLogger(KvSnapshotLeaseManager.class);
+
+    private final KvSnapshotLeaseMetadataHelper metadataHelper;
+    private final CoordinatorContext coordinatorContext;
+    private final Clock clock;
+    private final ScheduledExecutorService scheduledExecutor;
+    private final Configuration conf;
+
+    private final Map<String, ReadWriteLock> leaseLocks = 
MapUtils.newConcurrentHashMap();
+    /** lease id to kv snapshot lease. */
+    @GuardedBy("leaseLocks")
+    private final Map<String, KvSnapshotLease> kvSnapshotLeaseMap;
+
+    private final ReadWriteLock refCountLock = new ReentrantReadWriteLock();
+
+    /**
+     * KvSnapshotLeaseForBucket to the ref count, which means this table 
bucket + snapshotId has
+     * been leased by how many lease id.
+     */
+    @GuardedBy("refCountLock")
+    private final Map<KvSnapshotLeaseForBucket, AtomicInteger> refCount =
+            MapUtils.newConcurrentHashMap();
+
+    /** For metrics. */
+    private final AtomicInteger leasedBucketCount = new AtomicInteger(0);
+
+    public KvSnapshotLeaseManager(
+            Configuration conf,
+            KvSnapshotLeaseMetadataHelper metadataHelper,
+            CoordinatorContext coordinatorContext,
+            Clock clock,
+            CoordinatorMetricGroup coordinatorMetricGroup) {
+        this(
+                conf,
+                metadataHelper,
+                coordinatorContext,
+                Executors.newScheduledThreadPool(
+                        1, new 
ExecutorThreadFactory("kv-snapshot-lease-cleaner")),
+                clock,
+                coordinatorMetricGroup);
+    }
+
+    @VisibleForTesting
+    public KvSnapshotLeaseManager(
+            Configuration conf,
+            KvSnapshotLeaseMetadataHelper metadataHelper,
+            CoordinatorContext coordinatorContext,
+            ScheduledExecutorService scheduledExecutor,
+            Clock clock,
+            CoordinatorMetricGroup coordinatorMetricGroup) {
+        this.metadataHelper = metadataHelper;
+        this.conf = conf;
+        this.scheduledExecutor = scheduledExecutor;
+        this.coordinatorContext = coordinatorContext;
+        this.clock = clock;
+        this.kvSnapshotLeaseMap = MapUtils.newConcurrentHashMap();
+
+        registerMetrics(coordinatorMetricGroup);
+    }
+
+    public void start() {
+        LOG.info("kv snapshot lease manager has been started.");
+        scheduledExecutor.scheduleWithFixedDelay(
+                this::expireLeases,
+                0L,
+                
conf.get(ConfigOptions.KV_SNAPSHOT_LEASE_EXPIRATION_CHECK_INTERVAL).toMillis(),
+                TimeUnit.MILLISECONDS);
+    }
+
+    public void initialize() throws Exception {
+        List<String> leaseIds = metadataHelper.getLeasesList();
+        for (String leaseId : leaseIds) {
+            Optional<KvSnapshotLease> kvSnapshotLeaseOpt = 
metadataHelper.getLease(leaseId);
+            if (kvSnapshotLeaseOpt.isPresent()) {
+                KvSnapshotLease kvSnapshotLease = kvSnapshotLeaseOpt.get();
+                this.leaseLocks.put(leaseId, new ReentrantReadWriteLock());
+                this.kvSnapshotLeaseMap.put(leaseId, kvSnapshotLease);
+
+                initializeRefCount(kvSnapshotLease);
+
+                
leasedBucketCount.addAndGet(kvSnapshotLease.getLeasedSnapshotCount());
+            }
+        }
+    }
+
+    public boolean snapshotLeaseNotExist(KvSnapshotLeaseForBucket 
kvSnapshotLeaseForBucket) {
+        return inReadLock(
+                refCountLock,
+                () ->
+                        !refCount.containsKey(kvSnapshotLeaseForBucket)
+                                || 
refCount.get(kvSnapshotLeaseForBucket).get() <= 0);
+    }
+
+    /**
+     * Acquire kv snapshot lease.
+     *
+     * @param leaseId the lease id
+     * @param leaseDuration the lease duration
+     * @param tableIdToLeaseBucket the table id to lease bucket
+     * @return the map of unavailable snapshots that failed to be leased
+     */
+    public Map<TableBucket, Long> acquireLease(
+            String leaseId,
+            long leaseDuration,
+            Map<Long, List<KvSnapshotLeaseForBucket>> tableIdToLeaseBucket)
+            throws Exception {
+        ReadWriteLock lock = leaseLocks.computeIfAbsent(leaseId, k -> new 
ReentrantReadWriteLock());

Review Comment:
   I noticed that the management state for individual leases is scattered 
across multiple maps, such as leaseLocks and kvSnapshotLeaseMap. Perhaps all 
lease-related state—including locks—should be encapsulated into a single 
cohesive class. This would effectively reduce both code complexity and the 
complexity of managing concurrent access safely.
   
   



##########
fluss-server/src/main/java/org/apache/fluss/server/coordinator/KvSnapshotLeaseManager.java:
##########
@@ -0,0 +1,453 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.fluss.server.coordinator;
+
+import org.apache.fluss.annotation.VisibleForTesting;
+import org.apache.fluss.config.ConfigOptions;
+import org.apache.fluss.config.Configuration;
+import org.apache.fluss.metadata.KvSnapshotLeaseForBucket;
+import org.apache.fluss.metadata.TableBucket;
+import org.apache.fluss.metadata.TableInfo;
+import org.apache.fluss.metrics.MetricNames;
+import org.apache.fluss.server.metrics.group.CoordinatorMetricGroup;
+import org.apache.fluss.server.zk.data.lease.KvSnapshotLease;
+import org.apache.fluss.server.zk.data.lease.KvSnapshotLeaseMetadataHelper;
+import org.apache.fluss.server.zk.data.lease.KvSnapshotTableLease;
+import org.apache.fluss.utils.MapUtils;
+import org.apache.fluss.utils.clock.Clock;
+import org.apache.fluss.utils.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.fluss.utils.concurrent.LockUtils.inReadLock;
+import static org.apache.fluss.utils.concurrent.LockUtils.inWriteLock;
+
+/** A manager to manage kv snapshot lease acquire, renew, release and drop. */
+@ThreadSafe
+public class KvSnapshotLeaseManager {
+    private static final Logger LOG = 
LoggerFactory.getLogger(KvSnapshotLeaseManager.class);
+
+    private final KvSnapshotLeaseMetadataHelper metadataHelper;
+    private final CoordinatorContext coordinatorContext;
+    private final Clock clock;
+    private final ScheduledExecutorService scheduledExecutor;
+    private final Configuration conf;
+
+    private final Map<String, ReadWriteLock> leaseLocks = 
MapUtils.newConcurrentHashMap();
+    /** lease id to kv snapshot lease. */
+    @GuardedBy("leaseLocks")
+    private final Map<String, KvSnapshotLease> kvSnapshotLeaseMap;
+
+    private final ReadWriteLock refCountLock = new ReentrantReadWriteLock();
+
+    /**
+     * KvSnapshotLeaseForBucket to the ref count, which means this table 
bucket + snapshotId has
+     * been leased by how many lease id.
+     */
+    @GuardedBy("refCountLock")
+    private final Map<KvSnapshotLeaseForBucket, AtomicInteger> refCount =
+            MapUtils.newConcurrentHashMap();
+
+    /** For metrics. */
+    private final AtomicInteger leasedBucketCount = new AtomicInteger(0);
+
+    public KvSnapshotLeaseManager(
+            Configuration conf,
+            KvSnapshotLeaseMetadataHelper metadataHelper,
+            CoordinatorContext coordinatorContext,
+            Clock clock,
+            CoordinatorMetricGroup coordinatorMetricGroup) {
+        this(
+                conf,
+                metadataHelper,
+                coordinatorContext,
+                Executors.newScheduledThreadPool(
+                        1, new 
ExecutorThreadFactory("kv-snapshot-lease-cleaner")),
+                clock,
+                coordinatorMetricGroup);
+    }
+
+    @VisibleForTesting
+    public KvSnapshotLeaseManager(
+            Configuration conf,
+            KvSnapshotLeaseMetadataHelper metadataHelper,
+            CoordinatorContext coordinatorContext,
+            ScheduledExecutorService scheduledExecutor,
+            Clock clock,
+            CoordinatorMetricGroup coordinatorMetricGroup) {
+        this.metadataHelper = metadataHelper;
+        this.conf = conf;
+        this.scheduledExecutor = scheduledExecutor;
+        this.coordinatorContext = coordinatorContext;
+        this.clock = clock;
+        this.kvSnapshotLeaseMap = MapUtils.newConcurrentHashMap();
+
+        registerMetrics(coordinatorMetricGroup);
+    }
+
+    public void start() {
+        LOG.info("kv snapshot lease manager has been started.");
+        scheduledExecutor.scheduleWithFixedDelay(
+                this::expireLeases,
+                0L,
+                
conf.get(ConfigOptions.KV_SNAPSHOT_LEASE_EXPIRATION_CHECK_INTERVAL).toMillis(),
+                TimeUnit.MILLISECONDS);
+    }
+
+    public void initialize() throws Exception {
+        List<String> leaseIds = metadataHelper.getLeasesList();
+        for (String leaseId : leaseIds) {
+            Optional<KvSnapshotLease> kvSnapshotLeaseOpt = 
metadataHelper.getLease(leaseId);
+            if (kvSnapshotLeaseOpt.isPresent()) {
+                KvSnapshotLease kvSnapshotLease = kvSnapshotLeaseOpt.get();
+                this.leaseLocks.put(leaseId, new ReentrantReadWriteLock());
+                this.kvSnapshotLeaseMap.put(leaseId, kvSnapshotLease);
+
+                initializeRefCount(kvSnapshotLease);
+
+                
leasedBucketCount.addAndGet(kvSnapshotLease.getLeasedSnapshotCount());
+            }
+        }
+    }
+
+    public boolean snapshotLeaseNotExist(KvSnapshotLeaseForBucket 
kvSnapshotLeaseForBucket) {
+        return inReadLock(
+                refCountLock,
+                () ->
+                        !refCount.containsKey(kvSnapshotLeaseForBucket)
+                                || 
refCount.get(kvSnapshotLeaseForBucket).get() <= 0);
+    }
+
+    /**
+     * Acquire kv snapshot lease.
+     *
+     * @param leaseId the lease id
+     * @param leaseDuration the lease duration
+     * @param tableIdToLeaseBucket the table id to lease bucket
+     * @return the map of unavailable snapshots that failed to be leased
+     */
+    public Map<TableBucket, Long> acquireLease(
+            String leaseId,
+            long leaseDuration,
+            Map<Long, List<KvSnapshotLeaseForBucket>> 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<TableBucket, Long> unavailableSnapshots = new 
HashMap<>();
+
+                    boolean update = kvSnapshotLeaseMap.containsKey(leaseId);
+                    KvSnapshotLease kvSnapshotLease;
+                    long newLeaseDuration = clock.milliseconds() + 
leaseDuration;
+                    if (!update) {
+                        // set the expiration time as: current time + 
leaseDuration
+                        kvSnapshotLease = new 
KvSnapshotLease(newLeaseDuration);
+                        kvSnapshotLeaseMap.put(leaseId, kvSnapshotLease);
+                        LOG.info(
+                                "kv snapshot lease '"
+                                        + leaseId
+                                        + "' has been acquired. The lease 
expiration time is "
+                                        + kvSnapshotLease.getExpirationTime());
+                    } else {
+                        kvSnapshotLease = kvSnapshotLeaseMap.get(leaseId);
+                        kvSnapshotLease.setExpirationTime(newLeaseDuration);
+                    }
+
+                    for (Map.Entry<Long, List<KvSnapshotLeaseForBucket>> entry 
:
+                            tableIdToLeaseBucket.entrySet()) {
+                        Long tableId = entry.getKey();
+                        TableInfo tableInfo = 
coordinatorContext.getTableInfoById(tableId);
+                        int numBuckets = tableInfo.getNumBuckets();
+                        List<KvSnapshotLeaseForBucket> buckets = 
entry.getValue();
+                        for (KvSnapshotLeaseForBucket bucket : buckets) {
+
+                            TableBucket tableBucket = bucket.getTableBucket();
+                            long kvSnapshotId = bucket.getKvSnapshotId();
+                            try {
+                                boolean snapshotExists =
+                                        
metadataHelper.isSnapshotExists(tableBucket, kvSnapshotId);
+                                if (!snapshotExists) {
+                                    unavailableSnapshots.put(tableBucket, 
kvSnapshotId);
+                                    continue;
+                                }
+                            } catch (Exception e) {
+                                LOG.error(
+                                        "Failed to check snapshotExists for 
tableBucket when acquire kv "
+                                                + "snapshot kvSnapshotLease 
{}.",
+                                        tableBucket,
+                                        e);
+                                unavailableSnapshots.put(tableBucket, 
kvSnapshotId);
+                                continue;
+                            }
+
+                            long originalSnapshotId =
+                                    kvSnapshotLease.acquireBucket(
+                                            tableBucket, kvSnapshotId, 
numBuckets);
+                            if (originalSnapshotId == -1L) {
+                                leasedBucketCount.incrementAndGet();
+                            } else {
+                                // clear the original ref.
+                                decrementRefCount(
+                                        new KvSnapshotLeaseForBucket(
+                                                tableBucket, 
originalSnapshotId));
+                            }
+                            incrementRefCount(bucket);
+                        }
+                    }
+
+                    if (update) {
+                        metadataHelper.updateLease(leaseId, kvSnapshotLease);
+                    } else {
+                        metadataHelper.registerLease(leaseId, kvSnapshotLease);
+                    }
+
+                    return unavailableSnapshots;
+                });
+    }
+
+    public void release(String leaseId, Map<Long, List<TableBucket>> 
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<Long, List<TableBucket>> entry :
+                            tableIdToUnregisterBucket.entrySet()) {
+                        List<TableBucket> buckets = entry.getValue();
+                        for (TableBucket bucket : buckets) {
+                            long snapshotId = lease.releaseBucket(bucket);
+                            if (snapshotId != -1L) {
+                                leasedBucketCount.decrementAndGet();
+                                decrementRefCount(new 
KvSnapshotLeaseForBucket(bucket, snapshotId));
+                            }
+                        }
+                    }
+
+                    if (lease.isEmpty()) {
+                        drop(leaseId);
+                    } else {
+                        metadataHelper.updateLease(leaseId, lease);
+                    }
+                });
+    }
+
+    /**
+     * Drop kv snapshot lease.
+     *
+     * @param leaseId the lease id
+     * @return true if clear success, false if lease not exist
+     */
+    public boolean drop(String leaseId) throws Exception {
+        ReadWriteLock lock = leaseLocks.get(leaseId);
+        if (lock == null) {
+            return false;
+        }
+
+        boolean exist =
+                inWriteLock(
+                        lock,
+                        () -> {
+                            KvSnapshotLease kvSnapshotLease = 
kvSnapshotLeaseMap.remove(leaseId);
+                            if (kvSnapshotLease == null) {
+                                return false;
+                            }
+
+                            clearRefCount(kvSnapshotLease);
+                            metadataHelper.deleteLease(leaseId);
+
+                            LOG.info("kv snapshot lease '" + leaseId + "' has 
been dropped.");
+                            return true;
+                        });
+
+        leaseLocks.remove(leaseId);
+        return exist;
+    }
+
+    private void initializeRefCount(KvSnapshotLease lease) {
+        for (Map.Entry<Long, KvSnapshotTableLease> 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<Long, Long[]> partitionSnapshots = 
tableLease.getPartitionSnapshots();
+                for (Map.Entry<Long, Long[]> 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<Long, KvSnapshotTableLease> 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<Long, Long[]> partitionSnapshots = 
tableLease.getPartitionSnapshots();
+                for (Map.Entry<Long, Long[]> entry : 
partitionSnapshots.entrySet()) {
+                    Long partitionId = entry.getKey();
+                    Long[] snapshots = entry.getValue();
+                    for (int i = 0; i < snapshots.length; i++) {
+                        if (snapshots[i] == -1L) {
+                            continue;
+                        }
+
+                        decrementRefCount(
+                                new KvSnapshotLeaseForBucket(
+                                        new TableBucket(tableId, partitionId, 
i), snapshots[i]));
+                        leasedBucketCount.decrementAndGet();
+                    }
+                }
+            }
+        }
+    }
+
+    private void incrementRefCount(KvSnapshotLeaseForBucket 
kvSnapshotLeaseForBucket) {
+        inWriteLock(
+                refCountLock,
+                () ->
+                        refCount.computeIfAbsent(
+                                        kvSnapshotLeaseForBucket, k -> new 
AtomicInteger(0))
+                                .incrementAndGet());
+    }
+
+    private void decrementRefCount(KvSnapshotLeaseForBucket 
kvSnapshotLeaseForBucket) {
+        inWriteLock(
+                refCountLock,
+                () -> {
+                    AtomicInteger atomicInteger = 
refCount.get(kvSnapshotLeaseForBucket);
+                    if (atomicInteger != null) {
+                        int decrementAndGet = atomicInteger.decrementAndGet();
+                        if (decrementAndGet <= 0) {
+                            refCount.remove(kvSnapshotLeaseForBucket);
+                        }
+                    }
+                });
+    }
+
+    private void expireLeases() {
+        long currentTime = clock.milliseconds();
+        List<String> expiredLeases =
+                kvSnapshotLeaseMap.entrySet().stream()
+                        .filter(entry -> entry.getValue().getExpirationTime() 
< currentTime)
+                        .map(Map.Entry::getKey)
+                        .collect(Collectors.toList());
+        for (String leaseId : expiredLeases) {
+            try {
+                drop(leaseId);
+            } catch (Exception e) {
+                LOG.error("Failed to clear kv snapshot lease {}", leaseId, e);
+            }
+        }
+    }
+
+    private void registerMetrics(CoordinatorMetricGroup 
coordinatorMetricGroup) {
+        coordinatorMetricGroup.gauge(MetricNames.KV_SNAPSHOT_LEASE_COUNT, 
this::getLeaseCount);
+        // TODO register as table or bucket level.
+        coordinatorMetricGroup.gauge(
+                MetricNames.LEASED_KV_SNAPSHOT_COUNT, 
this::getLeasedBucketCount);
+    }
+
+    @VisibleForTesting
+    int getLeaseCount() {
+        return kvSnapshotLeaseMap.size();
+    }
+
+    @VisibleForTesting
+    int getLeasedBucketCount() {
+        return leasedBucketCount.get();
+    }
+
+    @VisibleForTesting
+    int getRefCount(KvSnapshotLeaseForBucket kvSnapshotLeaseForBucket) {
+        return inReadLock(
+                refCountLock,
+                () -> {
+                    if (!refCount.containsKey(kvSnapshotLeaseForBucket)) {
+                        return 0;
+                    } else {
+                        return refCount.get(kvSnapshotLeaseForBucket).get();

Review Comment:
   This will result in two get calls.
   
   



##########
fluss-server/src/main/java/org/apache/fluss/server/coordinator/KvSnapshotLeaseManager.java:
##########
@@ -0,0 +1,453 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.fluss.server.coordinator;
+
+import org.apache.fluss.annotation.VisibleForTesting;
+import org.apache.fluss.config.ConfigOptions;
+import org.apache.fluss.config.Configuration;
+import org.apache.fluss.metadata.KvSnapshotLeaseForBucket;
+import org.apache.fluss.metadata.TableBucket;
+import org.apache.fluss.metadata.TableInfo;
+import org.apache.fluss.metrics.MetricNames;
+import org.apache.fluss.server.metrics.group.CoordinatorMetricGroup;
+import org.apache.fluss.server.zk.data.lease.KvSnapshotLease;
+import org.apache.fluss.server.zk.data.lease.KvSnapshotLeaseMetadataHelper;
+import org.apache.fluss.server.zk.data.lease.KvSnapshotTableLease;
+import org.apache.fluss.utils.MapUtils;
+import org.apache.fluss.utils.clock.Clock;
+import org.apache.fluss.utils.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.fluss.utils.concurrent.LockUtils.inReadLock;
+import static org.apache.fluss.utils.concurrent.LockUtils.inWriteLock;
+
+/** A manager to manage kv snapshot lease acquire, renew, release and drop. */
+@ThreadSafe
+public class KvSnapshotLeaseManager {
+    private static final Logger LOG = 
LoggerFactory.getLogger(KvSnapshotLeaseManager.class);
+
+    private final KvSnapshotLeaseMetadataHelper metadataHelper;
+    private final CoordinatorContext coordinatorContext;
+    private final Clock clock;
+    private final ScheduledExecutorService scheduledExecutor;
+    private final Configuration conf;
+
+    private final Map<String, ReadWriteLock> leaseLocks = 
MapUtils.newConcurrentHashMap();
+    /** lease id to kv snapshot lease. */
+    @GuardedBy("leaseLocks")
+    private final Map<String, KvSnapshotLease> kvSnapshotLeaseMap;
+
+    private final ReadWriteLock refCountLock = new ReentrantReadWriteLock();
+
+    /**
+     * KvSnapshotLeaseForBucket to the ref count, which means this table 
bucket + snapshotId has
+     * been leased by how many lease id.
+     */
+    @GuardedBy("refCountLock")
+    private final Map<KvSnapshotLeaseForBucket, AtomicInteger> refCount =
+            MapUtils.newConcurrentHashMap();
+
+    /** For metrics. */
+    private final AtomicInteger leasedBucketCount = new AtomicInteger(0);
+
+    public KvSnapshotLeaseManager(
+            Configuration conf,
+            KvSnapshotLeaseMetadataHelper metadataHelper,
+            CoordinatorContext coordinatorContext,
+            Clock clock,
+            CoordinatorMetricGroup coordinatorMetricGroup) {
+        this(
+                conf,
+                metadataHelper,
+                coordinatorContext,
+                Executors.newScheduledThreadPool(
+                        1, new 
ExecutorThreadFactory("kv-snapshot-lease-cleaner")),
+                clock,
+                coordinatorMetricGroup);
+    }
+
+    @VisibleForTesting
+    public KvSnapshotLeaseManager(
+            Configuration conf,
+            KvSnapshotLeaseMetadataHelper metadataHelper,
+            CoordinatorContext coordinatorContext,
+            ScheduledExecutorService scheduledExecutor,
+            Clock clock,
+            CoordinatorMetricGroup coordinatorMetricGroup) {
+        this.metadataHelper = metadataHelper;
+        this.conf = conf;
+        this.scheduledExecutor = scheduledExecutor;
+        this.coordinatorContext = coordinatorContext;
+        this.clock = clock;
+        this.kvSnapshotLeaseMap = MapUtils.newConcurrentHashMap();
+
+        registerMetrics(coordinatorMetricGroup);
+    }
+
+    public void start() {
+        LOG.info("kv snapshot lease manager has been started.");
+        scheduledExecutor.scheduleWithFixedDelay(
+                this::expireLeases,
+                0L,
+                
conf.get(ConfigOptions.KV_SNAPSHOT_LEASE_EXPIRATION_CHECK_INTERVAL).toMillis(),
+                TimeUnit.MILLISECONDS);
+    }
+
+    public void initialize() throws Exception {
+        List<String> leaseIds = metadataHelper.getLeasesList();
+        for (String leaseId : leaseIds) {
+            Optional<KvSnapshotLease> kvSnapshotLeaseOpt = 
metadataHelper.getLease(leaseId);
+            if (kvSnapshotLeaseOpt.isPresent()) {
+                KvSnapshotLease kvSnapshotLease = kvSnapshotLeaseOpt.get();
+                this.leaseLocks.put(leaseId, new ReentrantReadWriteLock());
+                this.kvSnapshotLeaseMap.put(leaseId, kvSnapshotLease);
+
+                initializeRefCount(kvSnapshotLease);
+
+                
leasedBucketCount.addAndGet(kvSnapshotLease.getLeasedSnapshotCount());
+            }
+        }
+    }
+
+    public boolean snapshotLeaseNotExist(KvSnapshotLeaseForBucket 
kvSnapshotLeaseForBucket) {
+        return inReadLock(
+                refCountLock,
+                () ->
+                        !refCount.containsKey(kvSnapshotLeaseForBucket)
+                                || 
refCount.get(kvSnapshotLeaseForBucket).get() <= 0);
+    }
+
+    /**
+     * Acquire kv snapshot lease.
+     *
+     * @param leaseId the lease id
+     * @param leaseDuration the lease duration
+     * @param tableIdToLeaseBucket the table id to lease bucket
+     * @return the map of unavailable snapshots that failed to be leased
+     */
+    public Map<TableBucket, Long> acquireLease(
+            String leaseId,
+            long leaseDuration,
+            Map<Long, List<KvSnapshotLeaseForBucket>> 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<TableBucket, Long> unavailableSnapshots = new 
HashMap<>();
+
+                    boolean update = kvSnapshotLeaseMap.containsKey(leaseId);
+                    KvSnapshotLease kvSnapshotLease;
+                    long newLeaseDuration = clock.milliseconds() + 
leaseDuration;
+                    if (!update) {
+                        // set the expiration time as: current time + 
leaseDuration
+                        kvSnapshotLease = new 
KvSnapshotLease(newLeaseDuration);
+                        kvSnapshotLeaseMap.put(leaseId, kvSnapshotLease);
+                        LOG.info(
+                                "kv snapshot lease '"
+                                        + leaseId
+                                        + "' has been acquired. The lease 
expiration time is "
+                                        + kvSnapshotLease.getExpirationTime());
+                    } else {
+                        kvSnapshotLease = kvSnapshotLeaseMap.get(leaseId);
+                        kvSnapshotLease.setExpirationTime(newLeaseDuration);
+                    }
+
+                    for (Map.Entry<Long, List<KvSnapshotLeaseForBucket>> entry 
:
+                            tableIdToLeaseBucket.entrySet()) {
+                        Long tableId = entry.getKey();
+                        TableInfo tableInfo = 
coordinatorContext.getTableInfoById(tableId);
+                        int numBuckets = tableInfo.getNumBuckets();
+                        List<KvSnapshotLeaseForBucket> buckets = 
entry.getValue();
+                        for (KvSnapshotLeaseForBucket bucket : buckets) {
+
+                            TableBucket tableBucket = bucket.getTableBucket();
+                            long kvSnapshotId = bucket.getKvSnapshotId();
+                            try {
+                                boolean snapshotExists =
+                                        
metadataHelper.isSnapshotExists(tableBucket, kvSnapshotId);
+                                if (!snapshotExists) {
+                                    unavailableSnapshots.put(tableBucket, 
kvSnapshotId);
+                                    continue;
+                                }
+                            } catch (Exception e) {
+                                LOG.error(
+                                        "Failed to check snapshotExists for 
tableBucket when acquire kv "
+                                                + "snapshot kvSnapshotLease 
{}.",
+                                        tableBucket,
+                                        e);
+                                unavailableSnapshots.put(tableBucket, 
kvSnapshotId);
+                                continue;
+                            }
+
+                            long originalSnapshotId =
+                                    kvSnapshotLease.acquireBucket(
+                                            tableBucket, kvSnapshotId, 
numBuckets);
+                            if (originalSnapshotId == -1L) {
+                                leasedBucketCount.incrementAndGet();
+                            } else {
+                                // clear the original ref.
+                                decrementRefCount(
+                                        new KvSnapshotLeaseForBucket(

Review Comment:
   If the number of buckets reaches hundreds of thousands, would managing 
buckets via KvSnapshotLeaseForBucket become a memory-expensive model?
   
   



##########
fluss-server/src/main/java/org/apache/fluss/server/coordinator/KvSnapshotLeaseManager.java:
##########
@@ -0,0 +1,453 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.fluss.server.coordinator;
+
+import org.apache.fluss.annotation.VisibleForTesting;
+import org.apache.fluss.config.ConfigOptions;
+import org.apache.fluss.config.Configuration;
+import org.apache.fluss.metadata.KvSnapshotLeaseForBucket;
+import org.apache.fluss.metadata.TableBucket;
+import org.apache.fluss.metadata.TableInfo;
+import org.apache.fluss.metrics.MetricNames;
+import org.apache.fluss.server.metrics.group.CoordinatorMetricGroup;
+import org.apache.fluss.server.zk.data.lease.KvSnapshotLease;
+import org.apache.fluss.server.zk.data.lease.KvSnapshotLeaseMetadataHelper;
+import org.apache.fluss.server.zk.data.lease.KvSnapshotTableLease;
+import org.apache.fluss.utils.MapUtils;
+import org.apache.fluss.utils.clock.Clock;
+import org.apache.fluss.utils.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.fluss.utils.concurrent.LockUtils.inReadLock;
+import static org.apache.fluss.utils.concurrent.LockUtils.inWriteLock;
+
+/** A manager to manage kv snapshot lease acquire, renew, release and drop. */
+@ThreadSafe
+public class KvSnapshotLeaseManager {
+    private static final Logger LOG = 
LoggerFactory.getLogger(KvSnapshotLeaseManager.class);
+
+    private final KvSnapshotLeaseMetadataHelper metadataHelper;
+    private final CoordinatorContext coordinatorContext;
+    private final Clock clock;
+    private final ScheduledExecutorService scheduledExecutor;
+    private final Configuration conf;
+
+    private final Map<String, ReadWriteLock> leaseLocks = 
MapUtils.newConcurrentHashMap();
+    /** lease id to kv snapshot lease. */
+    @GuardedBy("leaseLocks")
+    private final Map<String, KvSnapshotLease> kvSnapshotLeaseMap;
+
+    private final ReadWriteLock refCountLock = new ReentrantReadWriteLock();
+
+    /**
+     * KvSnapshotLeaseForBucket to the ref count, which means this table 
bucket + snapshotId has
+     * been leased by how many lease id.
+     */
+    @GuardedBy("refCountLock")
+    private final Map<KvSnapshotLeaseForBucket, AtomicInteger> refCount =
+            MapUtils.newConcurrentHashMap();
+
+    /** For metrics. */
+    private final AtomicInteger leasedBucketCount = new AtomicInteger(0);
+
+    public KvSnapshotLeaseManager(
+            Configuration conf,
+            KvSnapshotLeaseMetadataHelper metadataHelper,
+            CoordinatorContext coordinatorContext,
+            Clock clock,
+            CoordinatorMetricGroup coordinatorMetricGroup) {
+        this(
+                conf,
+                metadataHelper,
+                coordinatorContext,
+                Executors.newScheduledThreadPool(
+                        1, new 
ExecutorThreadFactory("kv-snapshot-lease-cleaner")),
+                clock,
+                coordinatorMetricGroup);
+    }
+
+    @VisibleForTesting
+    public KvSnapshotLeaseManager(
+            Configuration conf,
+            KvSnapshotLeaseMetadataHelper metadataHelper,
+            CoordinatorContext coordinatorContext,
+            ScheduledExecutorService scheduledExecutor,
+            Clock clock,
+            CoordinatorMetricGroup coordinatorMetricGroup) {
+        this.metadataHelper = metadataHelper;
+        this.conf = conf;
+        this.scheduledExecutor = scheduledExecutor;
+        this.coordinatorContext = coordinatorContext;
+        this.clock = clock;
+        this.kvSnapshotLeaseMap = MapUtils.newConcurrentHashMap();
+
+        registerMetrics(coordinatorMetricGroup);
+    }
+
+    public void start() {
+        LOG.info("kv snapshot lease manager has been started.");
+        scheduledExecutor.scheduleWithFixedDelay(
+                this::expireLeases,
+                0L,
+                
conf.get(ConfigOptions.KV_SNAPSHOT_LEASE_EXPIRATION_CHECK_INTERVAL).toMillis(),
+                TimeUnit.MILLISECONDS);
+    }
+
+    public void initialize() throws Exception {
+        List<String> leaseIds = metadataHelper.getLeasesList();
+        for (String leaseId : leaseIds) {
+            Optional<KvSnapshotLease> kvSnapshotLeaseOpt = 
metadataHelper.getLease(leaseId);
+            if (kvSnapshotLeaseOpt.isPresent()) {
+                KvSnapshotLease kvSnapshotLease = kvSnapshotLeaseOpt.get();
+                this.leaseLocks.put(leaseId, new ReentrantReadWriteLock());
+                this.kvSnapshotLeaseMap.put(leaseId, kvSnapshotLease);
+
+                initializeRefCount(kvSnapshotLease);
+
+                
leasedBucketCount.addAndGet(kvSnapshotLease.getLeasedSnapshotCount());
+            }
+        }
+    }
+
+    public boolean snapshotLeaseNotExist(KvSnapshotLeaseForBucket 
kvSnapshotLeaseForBucket) {
+        return inReadLock(
+                refCountLock,
+                () ->
+                        !refCount.containsKey(kvSnapshotLeaseForBucket)
+                                || 
refCount.get(kvSnapshotLeaseForBucket).get() <= 0);
+    }
+
+    /**
+     * Acquire kv snapshot lease.
+     *
+     * @param leaseId the lease id
+     * @param leaseDuration the lease duration
+     * @param tableIdToLeaseBucket the table id to lease bucket
+     * @return the map of unavailable snapshots that failed to be leased
+     */
+    public Map<TableBucket, Long> acquireLease(
+            String leaseId,
+            long leaseDuration,
+            Map<Long, List<KvSnapshotLeaseForBucket>> 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<TableBucket, Long> unavailableSnapshots = new 
HashMap<>();
+
+                    boolean update = kvSnapshotLeaseMap.containsKey(leaseId);
+                    KvSnapshotLease kvSnapshotLease;
+                    long newLeaseDuration = clock.milliseconds() + 
leaseDuration;
+                    if (!update) {
+                        // set the expiration time as: current time + 
leaseDuration
+                        kvSnapshotLease = new 
KvSnapshotLease(newLeaseDuration);
+                        kvSnapshotLeaseMap.put(leaseId, kvSnapshotLease);
+                        LOG.info(
+                                "kv snapshot lease '"
+                                        + leaseId
+                                        + "' has been acquired. The lease 
expiration time is "
+                                        + kvSnapshotLease.getExpirationTime());
+                    } else {
+                        kvSnapshotLease = kvSnapshotLeaseMap.get(leaseId);
+                        kvSnapshotLease.setExpirationTime(newLeaseDuration);
+                    }
+
+                    for (Map.Entry<Long, List<KvSnapshotLeaseForBucket>> entry 
:
+                            tableIdToLeaseBucket.entrySet()) {
+                        Long tableId = entry.getKey();
+                        TableInfo tableInfo = 
coordinatorContext.getTableInfoById(tableId);
+                        int numBuckets = tableInfo.getNumBuckets();
+                        List<KvSnapshotLeaseForBucket> buckets = 
entry.getValue();
+                        for (KvSnapshotLeaseForBucket bucket : buckets) {
+
+                            TableBucket tableBucket = bucket.getTableBucket();
+                            long kvSnapshotId = bucket.getKvSnapshotId();
+                            try {
+                                boolean snapshotExists =
+                                        
metadataHelper.isSnapshotExists(tableBucket, kvSnapshotId);
+                                if (!snapshotExists) {
+                                    unavailableSnapshots.put(tableBucket, 
kvSnapshotId);
+                                    continue;
+                                }
+                            } catch (Exception e) {
+                                LOG.error(
+                                        "Failed to check snapshotExists for 
tableBucket when acquire kv "
+                                                + "snapshot kvSnapshotLease 
{}.",
+                                        tableBucket,
+                                        e);
+                                unavailableSnapshots.put(tableBucket, 
kvSnapshotId);
+                                continue;
+                            }
+
+                            long originalSnapshotId =
+                                    kvSnapshotLease.acquireBucket(
+                                            tableBucket, kvSnapshotId, 
numBuckets);
+                            if (originalSnapshotId == -1L) {
+                                leasedBucketCount.incrementAndGet();
+                            } else {
+                                // clear the original ref.
+                                decrementRefCount(
+                                        new KvSnapshotLeaseForBucket(
+                                                tableBucket, 
originalSnapshotId));
+                            }
+                            incrementRefCount(bucket);
+                        }
+                    }
+
+                    if (update) {
+                        metadataHelper.updateLease(leaseId, kvSnapshotLease);
+                    } else {
+                        metadataHelper.registerLease(leaseId, kvSnapshotLease);
+                    }
+
+                    return unavailableSnapshots;
+                });
+    }
+
+    public void release(String leaseId, Map<Long, List<TableBucket>> 
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<Long, List<TableBucket>> entry :
+                            tableIdToUnregisterBucket.entrySet()) {
+                        List<TableBucket> buckets = entry.getValue();
+                        for (TableBucket bucket : buckets) {
+                            long snapshotId = lease.releaseBucket(bucket);
+                            if (snapshotId != -1L) {
+                                leasedBucketCount.decrementAndGet();
+                                decrementRefCount(new 
KvSnapshotLeaseForBucket(bucket, snapshotId));
+                            }
+                        }
+                    }
+
+                    if (lease.isEmpty()) {
+                        drop(leaseId);
+                    } else {
+                        metadataHelper.updateLease(leaseId, lease);
+                    }
+                });
+    }
+
+    /**
+     * Drop kv snapshot lease.
+     *
+     * @param leaseId the lease id
+     * @return true if clear success, false if lease not exist
+     */
+    public boolean drop(String leaseId) throws Exception {
+        ReadWriteLock lock = leaseLocks.get(leaseId);
+        if (lock == null) {
+            return false;
+        }
+
+        boolean exist =
+                inWriteLock(
+                        lock,
+                        () -> {
+                            KvSnapshotLease kvSnapshotLease = 
kvSnapshotLeaseMap.remove(leaseId);
+                            if (kvSnapshotLease == null) {
+                                return false;
+                            }
+
+                            clearRefCount(kvSnapshotLease);
+                            metadataHelper.deleteLease(leaseId);
+
+                            LOG.info("kv snapshot lease '" + leaseId + "' has 
been dropped.");
+                            return true;
+                        });
+
+        leaseLocks.remove(leaseId);
+        return exist;
+    }
+
+    private void initializeRefCount(KvSnapshotLease lease) {
+        for (Map.Entry<Long, KvSnapshotTableLease> 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<Long, Long[]> partitionSnapshots = 
tableLease.getPartitionSnapshots();
+                for (Map.Entry<Long, Long[]> 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<Long, KvSnapshotTableLease> 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<Long, Long[]> partitionSnapshots = 
tableLease.getPartitionSnapshots();
+                for (Map.Entry<Long, Long[]> entry : 
partitionSnapshots.entrySet()) {
+                    Long partitionId = entry.getKey();
+                    Long[] snapshots = entry.getValue();
+                    for (int i = 0; i < snapshots.length; i++) {
+                        if (snapshots[i] == -1L) {
+                            continue;
+                        }
+
+                        decrementRefCount(
+                                new KvSnapshotLeaseForBucket(
+                                        new TableBucket(tableId, partitionId, 
i), snapshots[i]));
+                        leasedBucketCount.decrementAndGet();
+                    }
+                }
+            }
+        }
+    }
+
+    private void incrementRefCount(KvSnapshotLeaseForBucket 
kvSnapshotLeaseForBucket) {
+        inWriteLock(
+                refCountLock,
+                () ->
+                        refCount.computeIfAbsent(
+                                        kvSnapshotLeaseForBucket, k -> new 
AtomicInteger(0))
+                                .incrementAndGet());
+    }
+
+    private void decrementRefCount(KvSnapshotLeaseForBucket 
kvSnapshotLeaseForBucket) {
+        inWriteLock(
+                refCountLock,
+                () -> {
+                    AtomicInteger atomicInteger = 
refCount.get(kvSnapshotLeaseForBucket);
+                    if (atomicInteger != null) {
+                        int decrementAndGet = atomicInteger.decrementAndGet();
+                        if (decrementAndGet <= 0) {
+                            refCount.remove(kvSnapshotLeaseForBucket);
+                        }
+                    }
+                });
+    }
+
+    private void expireLeases() {
+        long currentTime = clock.milliseconds();
+        List<String> expiredLeases =
+                kvSnapshotLeaseMap.entrySet().stream()
+                        .filter(entry -> entry.getValue().getExpirationTime() 
< currentTime)
+                        .map(Map.Entry::getKey)
+                        .collect(Collectors.toList());
+        for (String leaseId : expiredLeases) {

Review Comment:
   The logic here is somewhat inefficient—it first creates a list and then 
iterates over it, resulting in multiple loops and unnecessary copying.
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to