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


##########
fluss-server/src/main/java/org/apache/fluss/server/coordinator/KvSnapshotConsumerManager.java:
##########
@@ -0,0 +1,425 @@
+/*
+ * 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.ConsumeKvSnapshotForBucket;
+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.ZooKeeperClient;
+import org.apache.fluss.server.zk.data.KvSnapshotConsumer;
+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.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+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 consumer register/unregister/clear. */
+@ThreadSafe
+public class KvSnapshotConsumerManager {
+    private static final Logger LOG = 
LoggerFactory.getLogger(KvSnapshotConsumerManager.class);
+
+    private final ZooKeeperClient zkClient;
+    private final CoordinatorContext coordinatorContext;
+    private final Clock clock;
+    private final ScheduledExecutorService scheduledExecutor;
+    private final Configuration conf;
+
+    private final Map<String, ReadWriteLock> consumerLocks = 
MapUtils.newConcurrentHashMap();
+    /** Consumer id to Consumer. */
+    @GuardedBy("consumerLocks")
+    private final Map<String, KvSnapshotConsumer> consumers;
+
+    private final ReadWriteLock refCountLock = new ReentrantReadWriteLock();
+
+    /**
+     * ConsumeKvSnapshotForBucket to the ref count, which means this table 
bucket + snapshotId has
+     * been consumed by how many consumers.
+     */
+    @GuardedBy("refCountLock")
+    private final Map<ConsumeKvSnapshotForBucket, AtomicInteger> refCount =
+            MapUtils.newConcurrentHashMap();
+
+    /** For metrics. */
+    private final AtomicInteger consumedBucketCount = new AtomicInteger(0);
+
+    public KvSnapshotConsumerManager(
+            Configuration conf,
+            ZooKeeperClient zkClient,
+            CoordinatorContext coordinatorContext,
+            Clock clock,
+            CoordinatorMetricGroup coordinatorMetricGroup) {
+        this(
+                conf,
+                zkClient,
+                coordinatorContext,
+                Executors.newScheduledThreadPool(
+                        1, new 
ExecutorThreadFactory("kv-snapshot-consumer-cleaner")),
+                clock,
+                coordinatorMetricGroup);
+    }
+
+    @VisibleForTesting
+    public KvSnapshotConsumerManager(
+            Configuration conf,
+            ZooKeeperClient zkClient,
+            CoordinatorContext coordinatorContext,
+            ScheduledExecutorService scheduledExecutor,
+            Clock clock,
+            CoordinatorMetricGroup coordinatorMetricGroup) {
+        this.zkClient = zkClient;
+        this.conf = conf;
+        this.scheduledExecutor = scheduledExecutor;
+        this.coordinatorContext = coordinatorContext;
+        this.clock = clock;
+        this.consumers = MapUtils.newConcurrentHashMap();
+
+        registerMetrics(coordinatorMetricGroup);
+    }
+
+    public void start() {
+        scheduledExecutor.scheduleWithFixedDelay(
+                this::expireConsumers,
+                0L,
+                
conf.get(ConfigOptions.KV_SNAPSHOT_CONSUMER_EXPIRATION_CHECK_INTERVAL).toMillis(),
+                TimeUnit.MILLISECONDS);
+    }
+
+    public void initialize() throws Exception {
+        List<String> consumers = zkClient.getKvSnapshotConsumerList();
+        for (String consumer : consumers) {
+            Optional<KvSnapshotConsumer> kvSnapshotConsumerOpt =
+                    zkClient.getKvSnapshotConsumer(consumer);
+            if (kvSnapshotConsumerOpt.isPresent()) {
+                KvSnapshotConsumer kvSnapshotConsumer = 
kvSnapshotConsumerOpt.get();
+                this.consumerLocks.put(consumer, new ReentrantReadWriteLock());

Review Comment:
   Hi, copilot. I think this variable is thread safe as I already announce it 
as ConcurrentHashMap.



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