platinumhamburg commented on code in PR #2179: URL: https://github.com/apache/fluss/pull/2179#discussion_r2654962686
########## 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? -- 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]
