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


##########
fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java:
##########
@@ -259,6 +288,12 @@ protected void startServices() throws Exception {
                     new AutoPartitionManager(metadataCache, metadataManager, 
conf);
             autoPartitionManager.start();
 
+            int ioExecutorPoolSize = 
conf.get(ConfigOptions.COORDINATOR_IO_POOL_SIZE);
+            this.ioExecutor =
+                    Executors.newFixedThreadPool(
+                            ioExecutorPoolSize, new 
ExecutorThreadFactory("coordinator-io"));
+

Review Comment:
   `ioExecutor` is initialized twice; it was already serialized once 
previously. This will cause the first thread pool to never be shutdown.



##########
fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.exception.CoordinatorEpochFencedException;
+import org.apache.fluss.server.zk.ZkEpoch;
+import org.apache.fluss.server.zk.ZooKeeperClient;
+import org.apache.fluss.server.zk.data.ZkData;
+import 
org.apache.fluss.shaded.curator5.org.apache.curator.framework.recipes.leader.LeaderLatch;
+import 
org.apache.fluss.shaded.curator5.org.apache.curator.framework.recipes.leader.LeaderLatchListener;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** Using by coordinator server. Coordinator servers listen ZK node and elect 
leadership. */
+public class CoordinatorLeaderElection implements AutoCloseable {
+    private static final Logger LOG = 
LoggerFactory.getLogger(CoordinatorLeaderElection.class);
+
+    private final int serverId;
+    private final ZooKeeperClient zkClient;
+    private final CoordinatorContext coordinatorContext;
+    private final LeaderLatch leaderLatch;
+    private final AtomicBoolean isLeader = new AtomicBoolean(false);
+    private final CompletableFuture<Void> leaderReadyFuture = new 
CompletableFuture<>();
+    private volatile Thread electionThread;
+
+    public CoordinatorLeaderElection(
+            ZooKeeperClient zkClient, int serverId, CoordinatorContext 
coordinatorContext) {
+        this.serverId = serverId;
+        this.zkClient = zkClient;
+        this.coordinatorContext = coordinatorContext;
+        this.leaderLatch =
+                new LeaderLatch(
+                        zkClient.getCuratorClient(),
+                        ZkData.CoordinatorElectionZNode.path(),
+                        String.valueOf(serverId));
+    }
+
+    /**
+     * Starts the leader election process asynchronously. The returned future 
completes when this
+     * server becomes the leader and initializes the leader services.
+     *
+     * @param initLeaderServices the runnable to initialize leader services 
once elected
+     * @return a CompletableFuture that completes when this server becomes 
leader
+     */
+    public CompletableFuture<Void> startElectLeaderAsync(Runnable 
initLeaderServices) {
+        leaderLatch.addListener(
+                new LeaderLatchListener() {
+                    @Override
+                    public void isLeader() {
+                        LOG.info("Coordinator server {} has become the 
leader.", serverId);
+                        isLeader.set(true);
+                    }
+
+                    @Override
+                    public void notLeader() {
+                        relinquishLeadership();
+                        LOG.warn("Coordinator server {} has lost the 
leadership.", serverId);
+                    }
+                });
+
+        try {
+            leaderLatch.start();
+            LOG.info("Coordinator server {} started leader election.", 
serverId);
+        } catch (Exception e) {
+            LOG.error("Failed to start LeaderLatch for server {}", serverId, 
e);
+            leaderReadyFuture.completeExceptionally(
+                    new RuntimeException("Leader election start failed", e));
+            return leaderReadyFuture;
+        }
+
+        // Run the await and initialization in a separate thread to avoid 
blocking
+        electionThread =
+                new Thread(
+                        () -> {
+                            try {
+                                // todo: Currently, we await the leader latch 
and do nothing until
+                                // it becomes leader.
+                                // Later we can make it as a hot backup server 
to continuously
+                                // synchronize metadata from
+                                // Zookeeper, which save time from recovering 
context
+                                leaderLatch.await();
+                                doInitLeaderServices(initLeaderServices);
+                                leaderReadyFuture.complete(null);
+                            } catch (InterruptedException e) {
+                                Thread.currentThread().interrupt();
+                                LOG.info(
+                                        "Leader election for server {} was 
interrupted.", serverId);
+                                leaderReadyFuture.completeExceptionally(e);
+                            } catch (Exception e) {
+                                LOG.error(
+                                        "Failed during leader election for 
server {}", serverId, e);
+                                leaderReadyFuture.completeExceptionally(e);
+                            }
+                        },
+                        "coordinator-leader-election-" + serverId);
+        electionThread.start();
+
+        return leaderReadyFuture;
+    }
+
+    public void doInitLeaderServices(Runnable initLeaderServices) {
+        try {
+            // to avoid split-brain
+            Optional<ZkEpoch> optionalEpoch = 
zkClient.fenceBecomeCoordinatorLeader(serverId);
+            optionalEpoch.ifPresent(

Review Comment:
    The handling when returning `Optional.empty()` here is problematic? When 
`BadVersionException` is caught,
       `fenceBecomeCoordinatorLeader` returns `Optional.empty()`, but 
`doInitLeaderServices` only relinquishes leadership on
       `CoordinatorEpochFencedException`. If the epoch update fails (returning 
empty), the code continues to execute
       `initLeaderServices.run()`, causing a coordinator that did not 
successfully acquire the epoch to still run as a leader, which may result in a 
split-brain.



##########
fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorContext.java:
##########
@@ -67,6 +68,7 @@ public class CoordinatorContext {
     // a success deletion.
     private final Map<TableBucketReplica, Integer> failDeleteNumbers = new 
HashMap<>();
 
+    private final Set<Integer> liveCoordinatorServers = new HashSet<>();

Review Comment:
   `resetContext()` need to clear `liveCoordinatorServers`?  also need to clear 
`coordinatorEpochZkVersion`.



##########
fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java:
##########
@@ -178,20 +189,105 @@ public Optional<byte[]> getOrEmpty(String path) throws 
Exception {
     // Coordinator server
     // 
--------------------------------------------------------------------------------------------
 
-    /** Register a coordinator leader server to ZK. */
+    /** Register a coordinator server to ZK. */
+    public void registerCoordinatorServer(int coordinatorId) throws Exception {
+        String path = ZkData.CoordinatorIdZNode.path(coordinatorId);
+        
zkClient.create().creatingParentsIfNeeded().withMode(CreateMode.EPHEMERAL).forPath(path);
+        LOG.info("Registered Coordinator server {} at path {}.", 
coordinatorId, path);
+    }
+
+    /**
+     * Become coordinator leader. This method is a step after 
electCoordinatorLeader() and before
+     * registerCoordinatorLeader(). This is to ensure the coordinator get and 
update the coordinator
+     * epoch and coordinator epoch zk version.
+     */
+    public Optional<ZkEpoch> fenceBecomeCoordinatorLeader(int coordinatorId) 
throws Exception {
+        ensureEpochZnodeExists();
+
+        try {
+            ZkEpoch getEpoch = getCurrentEpoch();
+            int currentCoordinatorEpoch = getEpoch.getCoordinatorEpoch();
+            int currentCoordinatorEpochZkVersion = 
getEpoch.getCoordinatorEpochZkVersion();
+            int newCoordinatorEpoch = currentCoordinatorEpoch + 1;
+            LOG.info(
+                    "Coordinator leader {} tries to update epoch. Current 
epoch={}, Zookeeper version={}, new epoch={}",
+                    coordinatorId,
+                    currentCoordinatorEpoch,
+                    currentCoordinatorEpochZkVersion,
+                    newCoordinatorEpoch);
+
+            // atomically update epoch
+            zkClient.setData()
+                    .withVersion(currentCoordinatorEpochZkVersion)
+                    .forPath(
+                            ZkData.CoordinatorEpochZNode.path(),
+                            
ZkData.CoordinatorEpochZNode.encode(newCoordinatorEpoch));
+
+            return Optional.of(getEpoch.nextZkEpoch());
+        } catch (KeeperException.BadVersionException e) {
+            // Other coordinator leader has updated epoch.
+            // If this happens, it means our fence is in effect.
+            LOG.info("Coordinator leader {} failed to update epoch.", 
coordinatorId);
+        }
+
+        return Optional.empty();
+    }
+
+    /** Register a coordinator leader to ZK. */
     public void registerCoordinatorLeader(CoordinatorAddress 
coordinatorAddress) throws Exception {
-        String path = CoordinatorZNode.path();
+        String path = ZkData.CoordinatorLeaderZNode.path();
         zkClient.create()
                 .creatingParentsIfNeeded()
                 .withMode(CreateMode.EPHEMERAL)
-                .forPath(path, CoordinatorZNode.encode(coordinatorAddress));
-        LOG.info("Registered leader {} at path {}.", coordinatorAddress, path);
+                .forPath(path, 
ZkData.CoordinatorLeaderZNode.encode(coordinatorAddress));
+        LOG.info("Registered Coordinator leader {} at path {}.", 
coordinatorAddress, path);
     }
 
     /** Get the leader address registered in ZK. */
-    public Optional<CoordinatorAddress> getCoordinatorAddress() throws 
Exception {
-        Optional<byte[]> bytes = getOrEmpty(CoordinatorZNode.path());
-        return bytes.map(CoordinatorZNode::decode);
+    public Optional<CoordinatorAddress> getCoordinatorLeaderAddress() throws 
Exception {
+        Optional<byte[]> bytes = 
getOrEmpty(ZkData.CoordinatorLeaderZNode.path());
+        return bytes.map(
+                data ->
+                        // maybe an empty node when a leader is elected but 
not registered
+                        data.length == 0 ? null : 
ZkData.CoordinatorLeaderZNode.decode(data));
+    }
+
+    /** Gets the list of coordinator server Ids. */
+    public int[] getCoordinatorServerList() throws Exception {
+        List<String> coordinatorServers = 
getChildren(ZkData.CoordinatorIdsZNode.path());
+        return 
coordinatorServers.stream().mapToInt(Integer::parseInt).toArray();
+    }
+
+    /** Ensure epoch znode exists. */
+    public void ensureEpochZnodeExists() throws Exception {
+        String path = ZkData.CoordinatorEpochZNode.path();
+        if (zkClient.checkExists().forPath(path) == null) {

Review Comment:
    If two `coordinators` execute almost simultaneously and both pass 
`checkExists == null`, the first one creates successfully, while the second one 
will throw `NodeExistsException` which is wrapped as `RuntimeException` causing 
startup failure. So I think `NodeExistsException` should be ignored here, not 
thrown `RuntimeException`.



##########
fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.exception.CoordinatorEpochFencedException;
+import org.apache.fluss.server.zk.ZkEpoch;
+import org.apache.fluss.server.zk.ZooKeeperClient;
+import org.apache.fluss.server.zk.data.ZkData;
+import 
org.apache.fluss.shaded.curator5.org.apache.curator.framework.recipes.leader.LeaderLatch;
+import 
org.apache.fluss.shaded.curator5.org.apache.curator.framework.recipes.leader.LeaderLatchListener;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** Using by coordinator server. Coordinator servers listen ZK node and elect 
leadership. */
+public class CoordinatorLeaderElection implements AutoCloseable {
+    private static final Logger LOG = 
LoggerFactory.getLogger(CoordinatorLeaderElection.class);
+
+    private final int serverId;
+    private final ZooKeeperClient zkClient;
+    private final CoordinatorContext coordinatorContext;
+    private final LeaderLatch leaderLatch;
+    private final AtomicBoolean isLeader = new AtomicBoolean(false);
+    private final CompletableFuture<Void> leaderReadyFuture = new 
CompletableFuture<>();
+    private volatile Thread electionThread;
+
+    public CoordinatorLeaderElection(
+            ZooKeeperClient zkClient, int serverId, CoordinatorContext 
coordinatorContext) {
+        this.serverId = serverId;
+        this.zkClient = zkClient;
+        this.coordinatorContext = coordinatorContext;
+        this.leaderLatch =
+                new LeaderLatch(
+                        zkClient.getCuratorClient(),
+                        ZkData.CoordinatorElectionZNode.path(),
+                        String.valueOf(serverId));
+    }
+
+    /**
+     * Starts the leader election process asynchronously. The returned future 
completes when this
+     * server becomes the leader and initializes the leader services.
+     *
+     * @param initLeaderServices the runnable to initialize leader services 
once elected
+     * @return a CompletableFuture that completes when this server becomes 
leader
+     */
+    public CompletableFuture<Void> startElectLeaderAsync(Runnable 
initLeaderServices) {
+        leaderLatch.addListener(
+                new LeaderLatchListener() {
+                    @Override
+                    public void isLeader() {
+                        LOG.info("Coordinator server {} has become the 
leader.", serverId);
+                        isLeader.set(true);
+                    }
+
+                    @Override
+                    public void notLeader() {
+                        relinquishLeadership();
+                        LOG.warn("Coordinator server {} has lost the 
leadership.", serverId);
+                    }
+                });
+
+        try {
+            leaderLatch.start();
+            LOG.info("Coordinator server {} started leader election.", 
serverId);
+        } catch (Exception e) {
+            LOG.error("Failed to start LeaderLatch for server {}", serverId, 
e);
+            leaderReadyFuture.completeExceptionally(
+                    new RuntimeException("Leader election start failed", e));
+            return leaderReadyFuture;
+        }
+
+        // Run the await and initialization in a separate thread to avoid 
blocking
+        electionThread =
+                new Thread(
+                        () -> {
+                            try {
+                                // todo: Currently, we await the leader latch 
and do nothing until
+                                // it becomes leader.
+                                // Later we can make it as a hot backup server 
to continuously
+                                // synchronize metadata from
+                                // Zookeeper, which save time from recovering 
context
+                                leaderLatch.await();
+                                doInitLeaderServices(initLeaderServices);
+                                leaderReadyFuture.complete(null);
+                            } catch (InterruptedException e) {
+                                Thread.currentThread().interrupt();
+                                LOG.info(
+                                        "Leader election for server {} was 
interrupted.", serverId);
+                                leaderReadyFuture.completeExceptionally(e);
+                            } catch (Exception e) {
+                                LOG.error(
+                                        "Failed during leader election for 
server {}", serverId, e);
+                                leaderReadyFuture.completeExceptionally(e);
+                            }
+                        },
+                        "coordinator-leader-election-" + serverId);
+        electionThread.start();
+
+        return leaderReadyFuture;
+    }
+
+    public void doInitLeaderServices(Runnable initLeaderServices) {
+        try {
+            // to avoid split-brain
+            Optional<ZkEpoch> optionalEpoch = 
zkClient.fenceBecomeCoordinatorLeader(serverId);
+            optionalEpoch.ifPresent(
+                    integer ->
+                            coordinatorContext.setCoordinatorEpochAndZkVersion(
+                                    optionalEpoch.get().getCoordinatorEpoch(),
+                                    
optionalEpoch.get().getCoordinatorEpochZkVersion()));
+        } catch (CoordinatorEpochFencedException e) {
+            relinquishLeadership();
+            LOG.warn(
+                    "Coordinator server {} has been fence and not become 
leader successfully.",

Review Comment:
   `has been fence` -> `has been fenced`.



##########
fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.exception.CoordinatorEpochFencedException;
+import org.apache.fluss.server.zk.ZkEpoch;
+import org.apache.fluss.server.zk.ZooKeeperClient;
+import org.apache.fluss.server.zk.data.ZkData;
+import 
org.apache.fluss.shaded.curator5.org.apache.curator.framework.recipes.leader.LeaderLatch;
+import 
org.apache.fluss.shaded.curator5.org.apache.curator.framework.recipes.leader.LeaderLatchListener;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** Using by coordinator server. Coordinator servers listen ZK node and elect 
leadership. */
+public class CoordinatorLeaderElection implements AutoCloseable {
+    private static final Logger LOG = 
LoggerFactory.getLogger(CoordinatorLeaderElection.class);
+
+    private final int serverId;
+    private final ZooKeeperClient zkClient;
+    private final CoordinatorContext coordinatorContext;
+    private final LeaderLatch leaderLatch;
+    private final AtomicBoolean isLeader = new AtomicBoolean(false);
+    private final CompletableFuture<Void> leaderReadyFuture = new 
CompletableFuture<>();
+    private volatile Thread electionThread;
+
+    public CoordinatorLeaderElection(
+            ZooKeeperClient zkClient, int serverId, CoordinatorContext 
coordinatorContext) {
+        this.serverId = serverId;
+        this.zkClient = zkClient;
+        this.coordinatorContext = coordinatorContext;
+        this.leaderLatch =
+                new LeaderLatch(
+                        zkClient.getCuratorClient(),
+                        ZkData.CoordinatorElectionZNode.path(),
+                        String.valueOf(serverId));
+    }
+
+    /**
+     * Starts the leader election process asynchronously. The returned future 
completes when this
+     * server becomes the leader and initializes the leader services.
+     *
+     * @param initLeaderServices the runnable to initialize leader services 
once elected
+     * @return a CompletableFuture that completes when this server becomes 
leader
+     */
+    public CompletableFuture<Void> startElectLeaderAsync(Runnable 
initLeaderServices) {
+        leaderLatch.addListener(
+                new LeaderLatchListener() {
+                    @Override
+                    public void isLeader() {
+                        LOG.info("Coordinator server {} has become the 
leader.", serverId);
+                        isLeader.set(true);
+                    }
+
+                    @Override
+                    public void notLeader() {
+                        relinquishLeadership();
+                        LOG.warn("Coordinator server {} has lost the 
leadership.", serverId);
+                    }
+                });
+
+        try {
+            leaderLatch.start();
+            LOG.info("Coordinator server {} started leader election.", 
serverId);
+        } catch (Exception e) {
+            LOG.error("Failed to start LeaderLatch for server {}", serverId, 
e);
+            leaderReadyFuture.completeExceptionally(
+                    new RuntimeException("Leader election start failed", e));
+            return leaderReadyFuture;
+        }
+
+        // Run the await and initialization in a separate thread to avoid 
blocking
+        electionThread =
+                new Thread(
+                        () -> {
+                            try {
+                                // todo: Currently, we await the leader latch 
and do nothing until
+                                // it becomes leader.
+                                // Later we can make it as a hot backup server 
to continuously
+                                // synchronize metadata from
+                                // Zookeeper, which save time from recovering 
context
+                                leaderLatch.await();
+                                doInitLeaderServices(initLeaderServices);
+                                leaderReadyFuture.complete(null);
+                            } catch (InterruptedException e) {
+                                Thread.currentThread().interrupt();
+                                LOG.info(
+                                        "Leader election for server {} was 
interrupted.", serverId);
+                                leaderReadyFuture.completeExceptionally(e);
+                            } catch (Exception e) {
+                                LOG.error(
+                                        "Failed during leader election for 
server {}", serverId, e);
+                                leaderReadyFuture.completeExceptionally(e);
+                            }
+                        },
+                        "coordinator-leader-election-" + serverId);
+        electionThread.start();
+
+        return leaderReadyFuture;
+    }
+
+    public void doInitLeaderServices(Runnable initLeaderServices) {

Review Comment:
   private ? 



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