From 5aed00cdc455092c020fde11e6fc813c87fbc7b0 Mon Sep 17 00:00:00 2001 From: "whenzhou.zc" Date: Thu, 24 Jul 2025 16:08:18 +0800 Subject: [PATCH 01/26] [server] Coordinator Server Supports High-Available --- .../apache/fluss/config/ConfigOptions.java | 6 + .../CoordinatorLeaderElection.java | 69 ++++++++++ .../server/coordinator/CoordinatorServer.java | 99 +++++++++++++- .../server/metrics/ServerMetricUtils.java | 2 +- .../metrics/group/CoordinatorMetricGroup.java | 6 +- .../fluss/server/zk/ZooKeeperClient.java | 29 ++-- .../server/zk/data/CoordinatorAddress.java | 6 +- .../zk/data/CoordinatorAddressJsonSerde.java | 4 +- .../apache/fluss/server/zk/data/ZkData.java | 31 ++++- .../apache/fluss/server/ServerTestBase.java | 3 +- .../CoordinatorEventProcessorTest.java | 2 +- .../CoordinatorServerElectionTest.java | 128 ++++++++++++++++++ .../metrics/group/TestingMetricGroups.java | 2 +- .../testutils/FlussClusterExtension.java | 1 + .../fluss/server/zk/ZooKeeperClientTest.java | 2 +- .../data/CoordinatorAddressJsonSerdeTest.java | 5 +- 16 files changed, 358 insertions(+), 37 deletions(-) create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java create mode 100644 fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java diff --git a/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java b/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java index f121539224..48272c9aed 100644 --- a/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java +++ b/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java @@ -341,6 +341,12 @@ public class ConfigOptions { + " (“50100,50101”), ranges (“50100-50200”) or a combination of both." + "This option is deprecated. Please use bind.listeners instead, which provides a more flexible configuration for multiple ports"); + public static final ConfigOption COORDINATOR_ID = + key("coordinator.id") + .intType() + .noDefaultValue() + .withDescription("The id for the coordinator server."); + /** * @deprecated This option is deprecated. Please use {@link ConfigOptions#SERVER_IO_POOL_SIZE} * instead. diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java new file mode 100644 index 0000000000..ea3e676674 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java @@ -0,0 +1,69 @@ +/* + * 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 com.alibaba.fluss.server.coordinator; + +import com.alibaba.fluss.server.zk.data.ZkData; +import com.alibaba.fluss.shaded.curator5.org.apache.curator.framework.CuratorFramework; +import com.alibaba.fluss.shaded.curator5.org.apache.curator.framework.recipes.leader.LeaderSelector; +import com.alibaba.fluss.shaded.curator5.org.apache.curator.framework.recipes.leader.LeaderSelectorListener; +import com.alibaba.fluss.shaded.curator5.org.apache.curator.framework.state.ConnectionState; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Using by coordinator server. Coordinator servers listen ZK node and elect leadership. */ +public class CoordinatorLeaderElection { + private static final Logger LOG = LoggerFactory.getLogger(CoordinatorLeaderElection.class); + + private final CuratorFramework zkClient; + private final int serverId; + + public CoordinatorLeaderElection(CuratorFramework zkClient, int serverId) { + this.zkClient = zkClient; + this.serverId = serverId; + } + + public void startElectLeader(Runnable startLeaderServices) { + LeaderSelector leaderSelector = + new LeaderSelector( + zkClient, + ZkData.CoordinatorLeaderZNode.path(), + new LeaderSelectorListener() { + @Override + public void takeLeadership(CuratorFramework client) { + LOG.info( + "Coordinator server {} win the leader in election now.", + serverId); + startLeaderServices.run(); + } + + @Override + public void stateChanged( + CuratorFramework client, ConnectionState newState) { + if (newState == ConnectionState.LOST) { + LOG.info("Coordinator leader {} lost connection", serverId); + } + } + }); + + // allow reelection + leaderSelector.autoRequeue(); + leaderSelector.start(); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java index 7e06f61091..2dfcd3a452 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java @@ -56,6 +56,8 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; +import java.util.Map; +import java.util.Optional; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; @@ -87,7 +89,7 @@ public class CoordinatorServer extends ServerBase { private final AtomicBoolean isShutDown = new AtomicBoolean(false); @GuardedBy("lock") - private String serverId; + private int serverId; @GuardedBy("lock") private MetricRegistry metricRegistry; @@ -101,6 +103,9 @@ public class CoordinatorServer extends ServerBase { @GuardedBy("lock") private RpcClient rpcClient; + @GuardedBy("lock") + private List endpoints; + @GuardedBy("lock") private ClientMetricGroup clientMetricGroup; @@ -145,6 +150,7 @@ public CoordinatorServer(Configuration conf) { super(conf); validateConfigs(conf); this.terminationFuture = new CompletableFuture<>(); + this.serverId = conf.getInt(ConfigOptions.COORDINATOR_ID); } public static void main(String[] args) { @@ -156,10 +162,38 @@ public static void main(String[] args) { @Override protected void startServices() throws Exception { + electCoordinatorLeader(); + } + + private void electCoordinatorLeader() throws Exception { + this.endpoints = Endpoint.loadBindEndpoints(conf, ServerType.COORDINATOR); + this.zkClient = ZooKeeperUtils.startZookeeperClient(conf, this); + + // Coordinator Server supports high availability. If 3 coordinator servers are alive, + // one of them will be elected as leader and the other two will be standby. + // When leader fails, one of standby coordinators will be elected as new leader. + // All of them register to ZK like tablet servers in path + // "/coordinators/ids/1","/coordinators/ids/2","/coordinators/ids/3". + // but the leader will be elected in path "/coordinators/leader" additionally. + registerCoordinatorServer(); + + CoordinatorLeaderElection coordinatorLeaderElection = + new CoordinatorLeaderElection(zkClient.getCuratorClient(), serverId); + coordinatorLeaderElection.startElectLeader( + () -> { + try { + startCoordinatorLeaderService(); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + } + + protected void startCoordinatorLeaderService() throws Exception { + synchronized (lock) { LOG.info("Initializing Coordinator services."); List endpoints = Endpoint.loadBindEndpoints(conf, ServerType.COORDINATOR); - this.serverId = UUID.randomUUID().toString(); // for metrics this.metricRegistry = MetricRegistry.create(conf, pluginManager); @@ -170,8 +204,6 @@ protected void startServices() throws Exception { endpoints.get(0).getHost(), serverId); - this.zkClient = ZooKeeperUtils.startZookeeperClient(conf, this); - this.lakeCatalogDynamicLoader = new LakeCatalogDynamicLoader(conf, pluginManager, true); this.dynamicConfigManager = new DynamicConfigManager(zkClient, conf, true); @@ -221,9 +253,9 @@ protected void startServices() throws Exception { rpcServer.start(); registerCoordinatorLeader(); - // when init session, register coordinator server again - ZooKeeperUtils.registerZookeeperClientReInitSessionListener( - zkClient, this::registerCoordinatorLeader, this); + // // when init session, register coordinator server again + // ZooKeeperUtils.registerZookeeperClientReInitSessionListener( + // zkClient, this::registerCoordinatorLeader, this); this.clientMetricGroup = new ClientMetricGroup(metricRegistry, SERVER_NAME); this.rpcClient = RpcClient.create(conf, clientMetricGroup, true); @@ -276,6 +308,41 @@ protected CompletableFuture closeAsync(Result result) { return terminationFuture; } + private void registerCoordinatorServer() throws Exception { + long startTime = System.currentTimeMillis(); + + // we need to retry to register since although + // zkClient reconnect, the ephemeral node may still exist + // for a while time, retry to wait the ephemeral node removed + // see ZOOKEEPER-2985 + while (true) { + try { + zkClient.registerCoordinatorServer(this.serverId); + break; + } catch (KeeperException.NodeExistsException nodeExistsException) { + long elapsedTime = System.currentTimeMillis() - startTime; + if (elapsedTime >= ZOOKEEPER_REGISTER_TOTAL_WAIT_TIME_MS) { + LOG.error( + "Coordinator Server register to Zookeeper exceeded total retry time of {} ms. " + + "Aborting registration attempts.", + ZOOKEEPER_REGISTER_TOTAL_WAIT_TIME_MS); + throw nodeExistsException; + } + + LOG.warn( + "Coordinator server already registered in Zookeeper. " + + "retrying register after {} ms....", + ZOOKEEPER_REGISTER_RETRY_INTERVAL_MS); + try { + Thread.sleep(ZOOKEEPER_REGISTER_RETRY_INTERVAL_MS); + } catch (InterruptedException interruptedException) { + Thread.currentThread().interrupt(); + break; + } + } + } + } + private void registerCoordinatorLeader() throws Exception { long startTime = System.currentTimeMillis(); List bindEndpoints = rpcServer.getBindEndpoints(); @@ -500,6 +567,11 @@ public RpcServer getRpcServer() { return rpcServer; } + @VisibleForTesting + public int getServerId() { + return serverId; + } + @VisibleForTesting public ServerMetadataCache getMetadataCache() { return metadataCache; @@ -520,6 +592,19 @@ public RebalanceManager getRebalanceManager() { } private static void validateConfigs(Configuration conf) { + Optional serverId = conf.getOptional(ConfigOptions.COORDINATOR_ID); + if (!serverId.isPresent()) { + throw new IllegalConfigurationException( + String.format("Configuration %s must be set.", ConfigOptions.COORDINATOR_ID)); + } + + if (serverId.get() < 0) { + throw new IllegalConfigurationException( + String.format( + "Invalid configuration for %s, it must be greater than or equal 0.", + ConfigOptions.COORDINATOR_ID.key())); + } + if (conf.get(ConfigOptions.DEFAULT_REPLICATION_FACTOR) < 1) { throw new IllegalConfigurationException( String.format( diff --git a/fluss-server/src/main/java/org/apache/fluss/server/metrics/ServerMetricUtils.java b/fluss-server/src/main/java/org/apache/fluss/server/metrics/ServerMetricUtils.java index 770a900014..2f013a7312 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/metrics/ServerMetricUtils.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/metrics/ServerMetricUtils.java @@ -72,7 +72,7 @@ public class ServerMetricUtils { @VisibleForTesting static final String METRIC_GROUP_MEMORY = "memory"; public static CoordinatorMetricGroup createCoordinatorGroup( - MetricRegistry registry, String clusterId, String hostname, String serverId) { + MetricRegistry registry, String clusterId, String hostname, int serverId) { CoordinatorMetricGroup coordinatorMetricGroup = new CoordinatorMetricGroup(registry, clusterId, hostname, serverId); createAndInitializeStatusMetricGroup(coordinatorMetricGroup); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/metrics/group/CoordinatorMetricGroup.java b/fluss-server/src/main/java/org/apache/fluss/server/metrics/group/CoordinatorMetricGroup.java index 4045fbae78..5425c16788 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/metrics/group/CoordinatorMetricGroup.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/metrics/group/CoordinatorMetricGroup.java @@ -47,13 +47,13 @@ public class CoordinatorMetricGroup extends AbstractMetricGroup { protected final String clusterId; protected final String hostname; - protected final String serverId; + protected final int serverId; private final Map, CoordinatorEventMetricGroup> eventMetricGroups = MapUtils.newConcurrentHashMap(); public CoordinatorMetricGroup( - MetricRegistry registry, String clusterId, String hostname, String serverId) { + MetricRegistry registry, String clusterId, String hostname, int serverId) { super(registry, new String[] {clusterId, hostname, NAME}, null); this.clusterId = clusterId; this.hostname = hostname; @@ -69,7 +69,7 @@ protected String getGroupName(CharacterFilter filter) { protected final void putVariables(Map variables) { variables.put("cluster_id", clusterId); variables.put("host", hostname); - variables.put("server_id", serverId); + variables.put("server_id", String.valueOf(serverId)); } public CoordinatorEventMetricGroup getOrAddEventTypeMetricGroup( diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java index 86f3b952f6..8c14e348d2 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java @@ -175,20 +175,31 @@ public Optional 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); + } + + /** + * Register a coordinator leader to ZK. Don't need to create node because leader election + * process already do it. + */ public void registerCoordinatorLeader(CoordinatorAddress coordinatorAddress) throws Exception { - String path = CoordinatorZNode.path(); - zkClient.create() - .creatingParentsIfNeeded() - .withMode(CreateMode.EPHEMERAL) - .forPath(path, CoordinatorZNode.encode(coordinatorAddress)); - LOG.info("Registered leader {} at path {}.", coordinatorAddress, path); + String path = ZkData.CoordinatorLeaderZNode.path(); + zkClient.setData().forPath(path, ZkData.CoordinatorLeaderZNode.encode(coordinatorAddress)); + LOG.info("Registered Coordinator leader {} at path {}.", coordinatorAddress, path); } /** Get the leader address registered in ZK. */ public Optional getCoordinatorAddress() throws Exception { - Optional bytes = getOrEmpty(CoordinatorZNode.path()); - return bytes.map(CoordinatorZNode::decode); + Optional bytes = getOrEmpty(ZkData.CoordinatorLeaderZNode.path()); + // return bytes.map(CoordinatorZNode::decode); + return bytes.map( + data -> + // maybe a empty node when a leader is elected but not registered + data.length == 0 ? null : ZkData.CoordinatorLeaderZNode.decode(data)); } // -------------------------------------------------------------------------------------------- diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/CoordinatorAddress.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/CoordinatorAddress.java index 5953f33666..415be68b1e 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/CoordinatorAddress.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/CoordinatorAddress.java @@ -28,15 +28,15 @@ * @see CoordinatorAddressJsonSerde for json serialization and deserialization. */ public class CoordinatorAddress { - private final String id; + private final int id; private final List endpoints; - public CoordinatorAddress(String id, List endpoints) { + public CoordinatorAddress(int id, List endpoints) { this.id = id; this.endpoints = endpoints; } - public String getId() { + public int getId() { return id; } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/CoordinatorAddressJsonSerde.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/CoordinatorAddressJsonSerde.java index 9b577df5a2..c2863fe610 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/CoordinatorAddressJsonSerde.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/CoordinatorAddressJsonSerde.java @@ -51,7 +51,7 @@ public void serialize(CoordinatorAddress coordinatorAddress, JsonGenerator gener throws IOException { generator.writeStartObject(); writeVersion(generator); - generator.writeStringField(ID, coordinatorAddress.getId()); + generator.writeNumberField(ID, coordinatorAddress.getId()); generator.writeStringField( LISTENERS, Endpoint.toListenersString(coordinatorAddress.getEndpoints())); generator.writeEndObject(); @@ -60,7 +60,7 @@ public void serialize(CoordinatorAddress coordinatorAddress, JsonGenerator gener @Override public CoordinatorAddress deserialize(JsonNode node) { int version = node.get(VERSION_KEY).asInt(); - String id = node.get(ID).asText(); + int id = node.get(ID).asInt(); List endpoints; if (version == 1) { String host = node.get(HOST).asText(); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java index 43c1acbf44..42ed9a82c0 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java @@ -303,13 +303,35 @@ public static String path() { // ------------------------------------------------------------------------------------------ /** - * The znode for the active coordinator. The znode path is: + * The znode for alive coordinators. The znode path is: * - *

/coordinators/active + *

/coordinators/ids * - *

Note: introduce standby coordinators in the future for znode "/coordinators/standby/". + *

Note: leader coordinator for znode "/coordinators/leader". */ - public static final class CoordinatorZNode { + public static final class CoordinatorIdsZNode { + public static String path() { + return "/coordinators/ids"; + } + } + + /** + * The znode for a registered Coordinator information. The znode path is: + * + *

/coordinators/ids/[serverId] + */ + public static final class CoordinatorIdZNode { + public static String path(int serverId) { + return CoordinatorIdsZNode.path() + "/" + serverId; + } + } + + /** + * The znode for the active coordinator leader. The znode path is: + * + *

/coordinators/leader + */ + public static final class CoordinatorLeaderZNode { public static String path() { return "/coordinators/active"; } @@ -323,7 +345,6 @@ public static CoordinatorAddress decode(byte[] json) { return JsonSerdeUtils.readValue(json, CoordinatorAddressJsonSerde.INSTANCE); } } - // ------------------------------------------------------------------------------------------ // ZNodes under "/tabletservers/" // ------------------------------------------------------------------------------------------ diff --git a/fluss-server/src/test/java/org/apache/fluss/server/ServerTestBase.java b/fluss-server/src/test/java/org/apache/fluss/server/ServerTestBase.java index 2504aebed3..a9183c8b45 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/ServerTestBase.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/ServerTestBase.java @@ -94,7 +94,8 @@ void registerServerNodeWhenZkClientReInitSession() throws Exception { // get the EPHEMERAL node of server String path = server instanceof CoordinatorServer - ? CoordinatorZNode.path() + ? ZkData.CoordinatorIdZNode.path( + server.conf.getInt(ConfigOptions.COORDINATOR_ID)) : ServerIdZNode.path(server.conf.getInt(ConfigOptions.TABLET_SERVER_ID)); long oldNodeCtime = zookeeperClient.getStat(path).get().getCtime(); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java index d4ec8f8db2..c7b7fd8099 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java @@ -170,7 +170,7 @@ static void baseBeforeAll() throws Exception { // register coordinator server zookeeperClient.registerCoordinatorLeader( new CoordinatorAddress( - "2", Endpoint.fromListenersString("CLIENT://localhost:10012"))); + 2, Endpoint.fromListenersString("CLIENT://localhost:10012"))); // register 3 tablet servers for (int i = 0; i < 3; i++) { diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java new file mode 100644 index 0000000000..ae217d5d38 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java @@ -0,0 +1,128 @@ +/* + * 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 com.alibaba.fluss.server.coordinator; + +import com.alibaba.fluss.config.ConfigOptions; +import com.alibaba.fluss.config.Configuration; +import com.alibaba.fluss.server.zk.NOPErrorHandler; +import com.alibaba.fluss.server.zk.ZooKeeperClient; +import com.alibaba.fluss.server.zk.ZooKeeperExtension; +import com.alibaba.fluss.server.zk.data.CoordinatorAddress; +import com.alibaba.fluss.testutils.common.AllCallbackWrapper; + +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +import java.time.Duration; +import java.util.Arrays; +import java.util.List; + +import static com.alibaba.fluss.testutils.common.CommonTestUtils.waitUtil; +import static org.assertj.core.api.Assertions.assertThat; + +class CoordinatorServerElectionTest { + @RegisterExtension + public static final AllCallbackWrapper ZOO_KEEPER_EXTENSION_WRAPPER = + new AllCallbackWrapper<>(new ZooKeeperExtension()); + + protected static ZooKeeperClient zookeeperClient; + + @BeforeAll + static void baseBeforeAll() { + zookeeperClient = + ZOO_KEEPER_EXTENSION_WRAPPER + .getCustomExtension() + .getZooKeeperClient(NOPErrorHandler.INSTANCE); + } + + @Test + void testCoordinatorServerElection() throws Exception { + CoordinatorServer coordinatorServer1 = new CoordinatorServer(createConfiguration(1)); + CoordinatorServer coordinatorServer2 = new CoordinatorServer(createConfiguration(2)); + CoordinatorServer coordinatorServer3 = new CoordinatorServer(createConfiguration(3)); + coordinatorServer1.start(); + coordinatorServer2.start(); + coordinatorServer3.start(); + + List coordinatorServerList = + Arrays.asList(coordinatorServer1, coordinatorServer2, coordinatorServer3); + + waitUtilCoordinatorServerElected(); + + CoordinatorAddress firstLeaderAddress = zookeeperClient.getCoordinatorAddress().get(); + + // Find the Coordinator server leader + // and try to close it. + // Then we should get another Coordinator server leader elected + CoordinatorServer elected = null; + for (CoordinatorServer coordinatorServer : coordinatorServerList) { + if (coordinatorServer.getServerId() == firstLeaderAddress.getId()) { + elected = coordinatorServer; + break; + } + } + assertThat(elected).isNotNull(); + elected.close(); + + // coordinator leader changed. + waitUtilCoordinatorServerReelected(firstLeaderAddress); + CoordinatorAddress secondLeaderAddress = zookeeperClient.getCoordinatorAddress().get(); + assertThat(secondLeaderAddress).isNotEqualTo(firstLeaderAddress); + } + + /** Create a configuration with Zookeeper address setting. */ + protected static Configuration createConfiguration(int serverId) { + Configuration configuration = new Configuration(); + configuration.setString( + ConfigOptions.ZOOKEEPER_ADDRESS, + ZOO_KEEPER_EXTENSION_WRAPPER.getCustomExtension().getConnectString()); + configuration.setString( + ConfigOptions.BIND_LISTENERS, "CLIENT://localhost:0,FLUSS://localhost:0"); + configuration.setString(ConfigOptions.ADVERTISED_LISTENERS, "CLIENT://198.168.0.1:100"); + configuration.set(ConfigOptions.REMOTE_DATA_DIR, "/tmp/fluss/remote-data"); + + // set to small timout to verify the case that zk session is timeout + configuration.set(ConfigOptions.ZOOKEEPER_SESSION_TIMEOUT, Duration.ofMillis(500)); + configuration.set(ConfigOptions.ZOOKEEPER_CONNECTION_TIMEOUT, Duration.ofMillis(500)); + configuration.set(ConfigOptions.ZOOKEEPER_RETRY_WAIT, Duration.ofMillis(500)); + + configuration.set(ConfigOptions.COORDINATOR_ID, serverId); + return configuration; + } + + public void waitUtilCoordinatorServerElected() { + waitUtil( + () -> { + return zookeeperClient.getCoordinatorAddress().isPresent(); + }, + Duration.ofMinutes(1), + String.format("Fail to wait coordinator server elected")); + } + + public void waitUtilCoordinatorServerReelected(CoordinatorAddress address) { + waitUtil( + () -> { + return zookeeperClient.getCoordinatorAddress().isPresent() + && !zookeeperClient.getCoordinatorAddress().get().equals(address); + }, + Duration.ofMinutes(1), + String.format("Fail to wait coordinator server reelected")); + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/metrics/group/TestingMetricGroups.java b/fluss-server/src/test/java/org/apache/fluss/server/metrics/group/TestingMetricGroups.java index b2e93f2bbc..4bbff888e7 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/metrics/group/TestingMetricGroups.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/metrics/group/TestingMetricGroups.java @@ -32,7 +32,7 @@ public class TestingMetricGroups { new TabletServerMetricGroup(NOPMetricRegistry.INSTANCE, "fluss", "host", "rack", 0); public static final CoordinatorMetricGroup COORDINATOR_METRICS = - new CoordinatorMetricGroup(NOPMetricRegistry.INSTANCE, "cluster1", "host", "0"); + new CoordinatorMetricGroup(NOPMetricRegistry.INSTANCE, "cluster1", "host", 0); public static final TableMetricGroup TABLE_METRICS = new TableMetricGroup( diff --git a/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java b/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java index 8eaf29a0bb..713f049b33 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java @@ -265,6 +265,7 @@ public void startCoordinatorServer() throws Exception { if (coordinatorServer == null) { // if no coordinator server exists, create a new coordinator server and start Configuration conf = new Configuration(clusterConf); + conf.set(ConfigOptions.COORDINATOR_ID, 0); conf.setString(ConfigOptions.ZOOKEEPER_ADDRESS, zooKeeperServer.getConnectString()); conf.setString(ConfigOptions.BIND_LISTENERS, coordinatorServerListeners); setRemoteDataDir(conf); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java index 94cb34b233..0d41355e39 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java @@ -108,7 +108,7 @@ void testCoordinatorLeader() throws Exception { assertThat(zookeeperClient.getCoordinatorAddress()).isEmpty(); CoordinatorAddress coordinatorAddress = new CoordinatorAddress( - "2", Endpoint.fromListenersString("CLIENT://localhost1:10012")); + 2, Endpoint.fromListenersString("CLIENT://localhost1:10012")); // register leader address zookeeperClient.registerCoordinatorLeader(coordinatorAddress); // check get leader address diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/CoordinatorAddressJsonSerdeTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/CoordinatorAddressJsonSerdeTest.java index d90151d838..a3ae878ac9 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/CoordinatorAddressJsonSerdeTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/CoordinatorAddressJsonSerdeTest.java @@ -39,7 +39,7 @@ public class CoordinatorAddressJsonSerdeTest extends JsonSerdeTestBase Date: Thu, 24 Jul 2025 17:08:42 +0800 Subject: [PATCH 02/26] [server] Coordinator Server Supports High-Available --- .../fluss/server/zk/data/CoordinatorAddress.java | 2 +- .../java/org/apache/fluss/server/zk/data/ZkData.java | 2 -- .../coordinator/CoordinatorServerElectionTest.java | 10 +++++----- .../fluss/server/testutils/FlussClusterExtension.java | 10 ++++++++++ 4 files changed, 16 insertions(+), 8 deletions(-) diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/CoordinatorAddress.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/CoordinatorAddress.java index 415be68b1e..41dc6ec2ad 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/CoordinatorAddress.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/CoordinatorAddress.java @@ -23,7 +23,7 @@ import java.util.Objects; /** - * The address information of an active coordinator stored in {@link ZkData.CoordinatorZNode}. + * The address information of an active coordinator stored in {@link ZkData.CoordinatorLeaderZNode}. * * @see CoordinatorAddressJsonSerde for json serialization and deserialization. */ diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java index 42ed9a82c0..f5ed263b61 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java @@ -306,8 +306,6 @@ public static String path() { * The znode for alive coordinators. The znode path is: * *

/coordinators/ids - * - *

Note: leader coordinator for znode "/coordinators/leader". */ public static final class CoordinatorIdsZNode { public static String path() { diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java index ae217d5d38..818883f91a 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java @@ -64,7 +64,7 @@ void testCoordinatorServerElection() throws Exception { List coordinatorServerList = Arrays.asList(coordinatorServer1, coordinatorServer2, coordinatorServer3); - waitUtilCoordinatorServerElected(); + waitUntilCoordinatorServerElected(); CoordinatorAddress firstLeaderAddress = zookeeperClient.getCoordinatorAddress().get(); @@ -82,7 +82,7 @@ void testCoordinatorServerElection() throws Exception { elected.close(); // coordinator leader changed. - waitUtilCoordinatorServerReelected(firstLeaderAddress); + waitUntilCoordinatorServerReelected(firstLeaderAddress); CoordinatorAddress secondLeaderAddress = zookeeperClient.getCoordinatorAddress().get(); assertThat(secondLeaderAddress).isNotEqualTo(firstLeaderAddress); } @@ -107,16 +107,16 @@ protected static Configuration createConfiguration(int serverId) { return configuration; } - public void waitUtilCoordinatorServerElected() { + public void waitUntilCoordinatorServerElected() { waitUtil( () -> { return zookeeperClient.getCoordinatorAddress().isPresent(); }, Duration.ofMinutes(1), - String.format("Fail to wait coordinator server elected")); + "Fail to wait coordinator server elected"); } - public void waitUtilCoordinatorServerReelected(CoordinatorAddress address) { + public void waitUntilCoordinatorServerReelected(CoordinatorAddress address) { waitUtil( () -> { return zookeeperClient.getCoordinatorAddress().isPresent() diff --git a/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java b/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java index 713f049b33..d802ff5727 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java @@ -271,6 +271,7 @@ public void startCoordinatorServer() throws Exception { setRemoteDataDir(conf); coordinatorServer = new CoordinatorServer(conf); coordinatorServer.start(); + waitUntilCoordinatorServerElected(); coordinatorServerInfo = // TODO, Currently, we use 0 as coordinator server id. new ServerInfo( @@ -515,6 +516,15 @@ private TabletServerGateway newTabletServerClientForNode(ServerNode serverNode) () -> serverNode, rpcClient, TabletServerGateway.class); } + private void waitUntilCoordinatorServerElected() { + waitUtil( + () -> { + return zooKeeperClient.getCoordinatorAddress().isPresent(); + }, + Duration.ofSeconds(30), + "Fail to wait coordinator server elected"); + } + /** * Wait until coordinator server and all the tablet servers have the same metadata (Only need to * make sure same server info not to make sure table metadata). This method needs to be called From e774d2a589c6d04909910cfff405562b5d28cd97 Mon Sep 17 00:00:00 2001 From: "whenzhou.zc" Date: Thu, 24 Jul 2025 19:18:37 +0800 Subject: [PATCH 03/26] [server] Coordinator Server Supports High-Available --- .../fluss/server/coordinator/CoordinatorServer.java | 4 ---- .../java/org/apache/fluss/server/ServerTestBase.java | 11 +++++++++++ .../server/coordinator/CoordinatorServerTest.java | 1 + .../fluss/server/testutils/FlussClusterExtension.java | 5 +++-- 4 files changed, 15 insertions(+), 6 deletions(-) diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java index 2dfcd3a452..4be5d4cdd1 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java @@ -103,9 +103,6 @@ public class CoordinatorServer extends ServerBase { @GuardedBy("lock") private RpcClient rpcClient; - @GuardedBy("lock") - private List endpoints; - @GuardedBy("lock") private ClientMetricGroup clientMetricGroup; @@ -166,7 +163,6 @@ protected void startServices() throws Exception { } private void electCoordinatorLeader() throws Exception { - this.endpoints = Endpoint.loadBindEndpoints(conf, ServerType.COORDINATOR); this.zkClient = ZooKeeperUtils.startZookeeperClient(conf, this); // Coordinator Server supports high availability. If 3 coordinator servers are alive, diff --git a/fluss-server/src/test/java/org/apache/fluss/server/ServerTestBase.java b/fluss-server/src/test/java/org/apache/fluss/server/ServerTestBase.java index a9183c8b45..7e8b49c55c 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/ServerTestBase.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/ServerTestBase.java @@ -141,8 +141,19 @@ protected void verifyEndpoint( } public static CoordinatorServer startCoordinatorServer(Configuration conf) throws Exception { + conf.set(ConfigOptions.COORDINATOR_ID, 0); CoordinatorServer coordinatorServer = new CoordinatorServer(conf); coordinatorServer.start(); + waitUntilCoordinatorLeaderElected(); return coordinatorServer; } + + private static void waitUntilCoordinatorLeaderElected() { + waitUtil( + () -> { + return zookeeperClient.getCoordinatorAddress().isPresent(); + }, + Duration.ofSeconds(30), + "Fail to wait coordinator server elected"); + } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java index aee7f0d479..29718adc46 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java @@ -55,6 +55,7 @@ protected ServerBase getServer() { @Override protected ServerBase getStartFailServer() { Configuration configuration = createConfiguration(); + configuration.set(ConfigOptions.COORDINATOR_ID, 0); configuration.set(ConfigOptions.BIND_LISTENERS, "CLIENT://localhost:-12"); return new CoordinatorServer(configuration); } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java b/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java index d802ff5727..d3ef0d0f8f 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java @@ -271,7 +271,7 @@ public void startCoordinatorServer() throws Exception { setRemoteDataDir(conf); coordinatorServer = new CoordinatorServer(conf); coordinatorServer.start(); - waitUntilCoordinatorServerElected(); + waitUntilCoordinatorLeaderElected(); coordinatorServerInfo = // TODO, Currently, we use 0 as coordinator server id. new ServerInfo( @@ -282,6 +282,7 @@ public void startCoordinatorServer() throws Exception { } else { // start the existing coordinator server coordinatorServer.start(); + waitUntilCoordinatorLeaderElected(); coordinatorServerInfo = new ServerInfo( 0, @@ -516,7 +517,7 @@ private TabletServerGateway newTabletServerClientForNode(ServerNode serverNode) () -> serverNode, rpcClient, TabletServerGateway.class); } - private void waitUntilCoordinatorServerElected() { + private void waitUntilCoordinatorLeaderElected() { waitUtil( () -> { return zooKeeperClient.getCoordinatorAddress().isPresent(); From 19559e73e44baeb3835cfe5386e76b086cede983 Mon Sep 17 00:00:00 2001 From: "whenzhou.zc" Date: Fri, 25 Jul 2025 14:37:16 +0800 Subject: [PATCH 04/26] [server] Coordinator Server Supports High-Available --- .../CoordinatorLeaderElection.java | 2 +- .../server/coordinator/CoordinatorServer.java | 47 ++++++++++++++----- .../fluss/server/zk/ZooKeeperClient.java | 11 ++++- .../apache/fluss/server/zk/data/ZkData.java | 11 +++++ .../apache/fluss/server/ServerTestBase.java | 10 ---- .../coordinator/CoordinatorServerITCase.java | 2 + .../testutils/FlussClusterExtension.java | 11 ----- .../data/CoordinatorAddressJsonSerdeTest.java | 4 +- 8 files changed, 60 insertions(+), 38 deletions(-) diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java index ea3e676674..4dddf6c410 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java @@ -43,7 +43,7 @@ public void startElectLeader(Runnable startLeaderServices) { LeaderSelector leaderSelector = new LeaderSelector( zkClient, - ZkData.CoordinatorLeaderZNode.path(), + ZkData.CoordinatorElectionZNode.path(), new LeaderSelectorListener() { @Override public void takeLeadership(CuratorFramework client) { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java index 4be5d4cdd1..915fa25108 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java @@ -172,17 +172,25 @@ private void electCoordinatorLeader() throws Exception { // "/coordinators/ids/1","/coordinators/ids/2","/coordinators/ids/3". // but the leader will be elected in path "/coordinators/leader" additionally. registerCoordinatorServer(); + ZooKeeperUtils.registerZookeeperClientReInitSessionListener( + zkClient, this::registerCoordinatorServer, this); - CoordinatorLeaderElection coordinatorLeaderElection = - new CoordinatorLeaderElection(zkClient.getCuratorClient(), serverId); - coordinatorLeaderElection.startElectLeader( - () -> { - try { - startCoordinatorLeaderService(); - } catch (Exception e) { - throw new RuntimeException(e); - } - }); + // try to register Coordinator leader once + if (tryElectCoordinatorLeaderOnce()) { + startCoordinatorLeaderService(); + } else { + // standby + CoordinatorLeaderElection coordinatorLeaderElection = + new CoordinatorLeaderElection(zkClient.getCuratorClient(), serverId); + coordinatorLeaderElection.startElectLeader( + () -> { + try { + startCoordinatorLeaderService(); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + } } protected void startCoordinatorLeaderService() throws Exception { @@ -249,9 +257,9 @@ protected void startCoordinatorLeaderService() throws Exception { rpcServer.start(); registerCoordinatorLeader(); - // // when init session, register coordinator server again - // ZooKeeperUtils.registerZookeeperClientReInitSessionListener( - // zkClient, this::registerCoordinatorLeader, this); + // when init session, register coordinator server again + ZooKeeperUtils.registerZookeeperClientReInitSessionListener( + zkClient, this::registerCoordinatorLeader, this); this.clientMetricGroup = new ClientMetricGroup(metricRegistry, SERVER_NAME); this.rpcClient = RpcClient.create(conf, clientMetricGroup, true); @@ -339,6 +347,19 @@ private void registerCoordinatorServer() throws Exception { } } + private boolean tryElectCoordinatorLeaderOnce() throws Exception { + try { + zkClient.electCoordinatorLeader(); + LOG.info("Coordinator server {} win the leader in election now.", serverId); + return true; + } catch (KeeperException.NodeExistsException nodeExistsException) { + LOG.warn( + "Coordinator leader already registered in Zookeeper. Coordinator {} will be standby", + serverId); + return false; + } + } + private void registerCoordinatorLeader() throws Exception { long startTime = System.currentTimeMillis(); List bindEndpoints = rpcServer.getBindEndpoints(); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java index 8c14e348d2..d21698b95c 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java @@ -186,9 +186,18 @@ public void registerCoordinatorServer(int coordinatorId) throws Exception { * Register a coordinator leader to ZK. Don't need to create node because leader election * process already do it. */ + public void electCoordinatorLeader() throws Exception { + String path = ZkData.CoordinatorElectionZNode.path(); + zkClient.create().creatingParentsIfNeeded().withMode(CreateMode.EPHEMERAL).forPath(path); + } + + /** Register a coordinator leader to ZK. */ public void registerCoordinatorLeader(CoordinatorAddress coordinatorAddress) throws Exception { String path = ZkData.CoordinatorLeaderZNode.path(); - zkClient.setData().forPath(path, ZkData.CoordinatorLeaderZNode.encode(coordinatorAddress)); + zkClient.create() + .creatingParentsIfNeeded() + .withMode(CreateMode.EPHEMERAL) + .forPath(path, ZkData.CoordinatorLeaderZNode.encode(coordinatorAddress)); LOG.info("Registered Coordinator leader {} at path {}.", coordinatorAddress, path); } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java index f5ed263b61..be68123a8d 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java @@ -324,6 +324,17 @@ public static String path(int serverId) { } } + /** + * The znode for the coordinator leader election. The znode path is: + * + *

/coordinators/election + */ + public static final class CoordinatorElectionZNode { + public static String path() { + return "/coordinators/election"; + } + } + /** * The znode for the active coordinator leader. The znode path is: * diff --git a/fluss-server/src/test/java/org/apache/fluss/server/ServerTestBase.java b/fluss-server/src/test/java/org/apache/fluss/server/ServerTestBase.java index 7e8b49c55c..aec981a963 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/ServerTestBase.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/ServerTestBase.java @@ -144,16 +144,6 @@ public static CoordinatorServer startCoordinatorServer(Configuration conf) throw conf.set(ConfigOptions.COORDINATOR_ID, 0); CoordinatorServer coordinatorServer = new CoordinatorServer(conf); coordinatorServer.start(); - waitUntilCoordinatorLeaderElected(); return coordinatorServer; } - - private static void waitUntilCoordinatorLeaderElected() { - waitUtil( - () -> { - return zookeeperClient.getCoordinatorAddress().isPresent(); - }, - Duration.ofSeconds(30), - "Fail to wait coordinator server elected"); - } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerITCase.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerITCase.java index bd060d6eed..21c0af8f76 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerITCase.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerITCase.java @@ -63,6 +63,8 @@ protected Configuration getServerConfig() { ConfigOptions.BIND_LISTENERS, String.format("%s://%s:%d", DEFAULT_LISTENER_NAME, HOSTNAME, getPort())); conf.set(ConfigOptions.REMOTE_DATA_DIR, "/tmp/fluss/remote-data"); + conf.set(ConfigOptions.COORDINATOR_ID, 0); + return conf; } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java b/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java index d3ef0d0f8f..713f049b33 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java @@ -271,7 +271,6 @@ public void startCoordinatorServer() throws Exception { setRemoteDataDir(conf); coordinatorServer = new CoordinatorServer(conf); coordinatorServer.start(); - waitUntilCoordinatorLeaderElected(); coordinatorServerInfo = // TODO, Currently, we use 0 as coordinator server id. new ServerInfo( @@ -282,7 +281,6 @@ public void startCoordinatorServer() throws Exception { } else { // start the existing coordinator server coordinatorServer.start(); - waitUntilCoordinatorLeaderElected(); coordinatorServerInfo = new ServerInfo( 0, @@ -517,15 +515,6 @@ private TabletServerGateway newTabletServerClientForNode(ServerNode serverNode) () -> serverNode, rpcClient, TabletServerGateway.class); } - private void waitUntilCoordinatorLeaderElected() { - waitUtil( - () -> { - return zooKeeperClient.getCoordinatorAddress().isPresent(); - }, - Duration.ofSeconds(30), - "Fail to wait coordinator server elected"); - } - /** * Wait until coordinator server and all the tablet servers have the same metadata (Only need to * make sure same server info not to make sure table metadata). This method needs to be called diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/CoordinatorAddressJsonSerdeTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/CoordinatorAddressJsonSerdeTest.java index a3ae878ac9..de1cd6555a 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/CoordinatorAddressJsonSerdeTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/CoordinatorAddressJsonSerdeTest.java @@ -49,7 +49,7 @@ protected CoordinatorAddress[] createObjects() { @Override protected String[] expectedJsons() { return new String[] { - "{\"version\":2,\"id\":\"1\",\"listeners\":\"CLIENT://localhost:1001,FLUSS://127.0.0.1:9124\"}" + "{\"version\":2,\"id\":1,\"listeners\":\"CLIENT://localhost:1001,FLUSS://127.0.0.1:9124\"}" }; } @@ -58,7 +58,7 @@ void testCompatibility() throws IOException { JsonNode jsonInVersion1 = new ObjectMapper() .readTree( - "{\"version\":1,\"id\":\"1\",\"host\":\"localhost\",\"port\":1001}" + "{\"version\":1,\"id\":1,\"host\":\"localhost\",\"port\":1001}" .getBytes(StandardCharsets.UTF_8)); CoordinatorAddress coordinatorAddress = From bca4a1d2f9a795361ae464b4b1f675b66f82296e Mon Sep 17 00:00:00 2001 From: "whenzhou.zc" Date: Mon, 28 Jul 2025 15:30:00 +0800 Subject: [PATCH 05/26] [server] Coordinator Server Supports High-Available --- .../org/apache/fluss/metrics/MetricNames.java | 1 + .../coordinator/CoordinatorContext.java | 19 +++ .../CoordinatorEventProcessor.java | 42 ++++++- .../CoordinatorLeaderElection.java | 32 ++++- .../event/DeadCoordinatorServerEvent.java | 56 +++++++++ .../event/NewCoordinatorServerEvent.java | 56 +++++++++ .../CoordinatorServerChangeWatcher.java | 110 ++++++++++++++++++ .../watcher/TabletServerChangeWatcher.java | 2 +- .../fluss/server/zk/ZooKeeperClient.java | 8 +- .../CoordinatorEventProcessorTest.java | 2 +- .../CoordinatorServerElectionTest.java | 11 +- .../coordinator/CoordinatorServerTest.java | 3 +- .../CoordinatorServerChangeWatcherTest.java | 86 ++++++++++++++ .../TabletServerChangeWatcherTest.java | 2 +- .../fluss/server/zk/ZooKeeperClientTest.java | 4 +- .../observability/monitor-metrics.md | 9 +- 16 files changed, 425 insertions(+), 18 deletions(-) create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/DeadCoordinatorServerEvent.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/NewCoordinatorServerEvent.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/CoordinatorServerChangeWatcher.java create mode 100644 fluss-server/src/test/java/org/apache/fluss/server/coordinator/event/watcher/CoordinatorServerChangeWatcherTest.java diff --git a/fluss-common/src/main/java/org/apache/fluss/metrics/MetricNames.java b/fluss-common/src/main/java/org/apache/fluss/metrics/MetricNames.java index 7f4ae5a03b..c2811719a5 100644 --- a/fluss-common/src/main/java/org/apache/fluss/metrics/MetricNames.java +++ b/fluss-common/src/main/java/org/apache/fluss/metrics/MetricNames.java @@ -36,6 +36,7 @@ public class MetricNames { // metrics for coordinator server // -------------------------------------------------------------------------------------------- public static final String ACTIVE_COORDINATOR_COUNT = "activeCoordinatorCount"; + public static final String ALIVE_COORDINATOR_COUNT = "aliveCoordinatorCount"; public static final String ACTIVE_TABLET_SERVER_COUNT = "activeTabletServerCount"; public static final String OFFLINE_BUCKET_COUNT = "offlineBucketCount"; public static final String TABLE_COUNT = "tableCount"; diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorContext.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorContext.java index bece7d9dc3..b41b4769ed 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorContext.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorContext.java @@ -67,6 +67,7 @@ public class CoordinatorContext { // a success deletion. private final Map failDeleteNumbers = new HashMap<>(); + private final Set liveCoordinatorServers = new HashSet<>(); private final Map liveTabletServers = new HashMap<>(); private final Set shuttingDownTabletServers = new HashSet<>(); @@ -115,6 +116,24 @@ public int getCoordinatorEpoch() { return coordinatorEpoch; } + public Set getLiveCoordinatorServers() { + return liveCoordinatorServers; + } + + @VisibleForTesting + public void setLiveCoordinatorServers(Set servers) { + liveCoordinatorServers.clear(); + liveCoordinatorServers.addAll(servers); + } + + public void addLiveCoordinatorServer(int serverId) { + this.liveCoordinatorServers.add(serverId); + } + + public void removeLiveCoordinatorServer(int serverId) { + this.liveCoordinatorServers.remove(serverId); + } + public Map getLiveTabletServers() { return liveTabletServers; } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java index 813c1f10ed..c97aa0d77b 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java @@ -128,6 +128,7 @@ import java.time.Duration; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -172,6 +173,7 @@ public class CoordinatorEventProcessor implements EventProcessor { private final LakeTableTieringManager lakeTableTieringManager; private final TableChangeWatcher tableChangeWatcher; private final CoordinatorChannelManager coordinatorChannelManager; + private final CoordinatorServerChangeWatcher coordinatorServerChangeWatcher; private final TabletServerChangeWatcher tabletServerChangeWatcher; private final CoordinatorMetadataCache serverMetadataCache; private final CoordinatorRequestBatch coordinatorRequestBatch; @@ -224,6 +226,8 @@ public CoordinatorEventProcessor( tableBucketStateMachine, new RemoteStorageCleaner(conf, ioExecutor), ioExecutor); + this.coordinatorServerChangeWatcher = + new CoordinatorServerChangeWatcher(zooKeeperClient, coordinatorEventManager); this.tableChangeWatcher = new TableChangeWatcher(zooKeeperClient, coordinatorEventManager); this.tabletServerChangeWatcher = new TabletServerChangeWatcher(zooKeeperClient, coordinatorEventManager); @@ -261,6 +265,7 @@ public CoordinatorContext getCoordinatorContext() { public void startup() { coordinatorContext.setCoordinatorServerInfo(getCoordinatorServerInfo()); // start watchers first so that we won't miss node in zk; + coordinatorServerChangeWatcher.start(); tabletServerChangeWatcher.start(); tableChangeWatcher.start(); LOG.info("Initializing coordinator context."); @@ -304,7 +309,7 @@ public void shutdown() { private ServerInfo getCoordinatorServerInfo() { try { return zooKeeperClient - .getCoordinatorAddress() + .getCoordinatorLeaderAddress() .map( coordinatorAddress -> // TODO we set id to 0 as that CoordinatorServer don't support @@ -332,6 +337,11 @@ public int getCoordinatorEpoch() { private void initCoordinatorContext() throws Exception { long start = System.currentTimeMillis(); + // get all coordinator servers + int[] currentCoordinatorServers = zooKeeperClient.getCoordinatorServerList(); + coordinatorContext.setLiveCoordinatorServers( + Arrays.stream(currentCoordinatorServers).boxed().collect(Collectors.toSet())); + // get all tablet server's int[] currentServers = zooKeeperClient.getSortedTabletServerList(); List tabletServerInfos = new ArrayList<>(); @@ -546,6 +556,7 @@ private void onShutdown() { tableManager.shutdown(); // then stop watchers + coordinatorServerChangeWatcher.stop(); tableChangeWatcher.stop(); tabletServerChangeWatcher.stop(); } @@ -570,6 +581,10 @@ public void process(CoordinatorEvent event) { (NotifyLeaderAndIsrResponseReceivedEvent) event); } else if (event instanceof DeleteReplicaResponseReceivedEvent) { processDeleteReplicaResponseReceived((DeleteReplicaResponseReceivedEvent) event); + } else if (event instanceof NewCoordinatorServerEvent) { + processNewCoordinatorServer((NewCoordinatorServerEvent) event); + } else if (event instanceof DeadCoordinatorServerEvent) { + processDeadCoordinatorServer((DeadCoordinatorServerEvent) event); } else if (event instanceof NewTabletServerEvent) { processNewTabletServer((NewTabletServerEvent) event); } else if (event instanceof DeadTabletServerEvent) { @@ -981,6 +996,29 @@ private void onReplicaBecomeOffline(Set offlineReplicas) { replicaStateMachine.handleStateChanges(offlineReplicas, OfflineReplica); } + private void processNewCoordinatorServer(NewCoordinatorServerEvent newCoordinatorServerEvent) { + int coordinatorServerId = newCoordinatorServerEvent.getServerId(); + if (coordinatorContext.getLiveCoordinatorServers().contains(coordinatorServerId)) { + return; + } + + // process new coordinator server + LOG.info("New coordinator server callback for coordinator server {}", coordinatorServerId); + + coordinatorContext.addLiveCoordinatorServer(coordinatorServerId); + } + + private void processDeadCoordinatorServer( + DeadCoordinatorServerEvent deadCoordinatorServerEvent) { + int coordinatorServerId = deadCoordinatorServerEvent.getServerId(); + if (!coordinatorContext.getLiveCoordinatorServers().contains(coordinatorServerId)) { + return; + } + // process dead coordinator server + LOG.info("Coordinator server failure callback for {}.", coordinatorServerId); + coordinatorContext.removeLiveCoordinatorServer(coordinatorServerId); + } + private void processNewTabletServer(NewTabletServerEvent newTabletServerEvent) { // NOTE: we won't need to detect bounced tablet servers like Kafka as we won't // miss the event of tablet server un-register and register again since we can @@ -1003,7 +1041,7 @@ private void processNewTabletServer(NewTabletServerEvent newTabletServerEvent) { // it may happen during coordinator server initiation, the watcher watch a new tablet // server register event and put it to event manager, but after that, the coordinator // server read - // all tablet server nodes registered which contain the tablet server a; in this case, + // all tablet server nodes registered which contain the tablet server; in this case, // we can ignore it. return; } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java index 4dddf6c410..635c936121 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java @@ -23,23 +23,43 @@ import com.alibaba.fluss.shaded.curator5.org.apache.curator.framework.recipes.leader.LeaderSelector; import com.alibaba.fluss.shaded.curator5.org.apache.curator.framework.recipes.leader.LeaderSelectorListener; import com.alibaba.fluss.shaded.curator5.org.apache.curator.framework.state.ConnectionState; +import com.alibaba.fluss.utils.concurrent.ExecutorThreadFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + /** Using by coordinator server. Coordinator servers listen ZK node and elect leadership. */ public class CoordinatorLeaderElection { private static final Logger LOG = LoggerFactory.getLogger(CoordinatorLeaderElection.class); private final CuratorFramework zkClient; private final int serverId; + private final ScheduledExecutorService executor; public CoordinatorLeaderElection(CuratorFramework zkClient, int serverId) { + this( + zkClient, + serverId, + Executors.newSingleThreadScheduledExecutor( + new ExecutorThreadFactory("fluss-coordinator-leader-election"))); + } + + protected CoordinatorLeaderElection( + CuratorFramework zkClient, int serverId, ScheduledExecutorService executor) { this.zkClient = zkClient; this.serverId = serverId; + this.executor = executor; } - public void startElectLeader(Runnable startLeaderServices) { + public void startElectLeader(Runnable initLeaderServices) { + executor.schedule(() -> electLeader(initLeaderServices), 0, TimeUnit.MILLISECONDS); + } + + private void electLeader(Runnable initLeaderServices) { LeaderSelector leaderSelector = new LeaderSelector( zkClient, @@ -50,7 +70,15 @@ public void takeLeadership(CuratorFramework client) { LOG.info( "Coordinator server {} win the leader in election now.", serverId); - startLeaderServices.run(); + initLeaderServices.run(); + + // Do not return, otherwise the leader will be released immediately. + while (true) { + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + } + } } @Override diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/DeadCoordinatorServerEvent.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/DeadCoordinatorServerEvent.java new file mode 100644 index 0000000000..24850c7981 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/DeadCoordinatorServerEvent.java @@ -0,0 +1,56 @@ +/* + * 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 com.alibaba.fluss.server.coordinator.event; + +import java.util.Objects; + +/** An event for coordinator server became dead. */ +public class DeadCoordinatorServerEvent implements CoordinatorEvent { + + private final int serverId; + + public DeadCoordinatorServerEvent(int serverId) { + this.serverId = serverId; + } + + public int getServerId() { + return serverId; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DeadCoordinatorServerEvent that = (DeadCoordinatorServerEvent) o; + return serverId == that.serverId; + } + + @Override + public int hashCode() { + return Objects.hash(serverId); + } + + @Override + public String toString() { + return "DeadCoordinatorServerEvent{" + "serverId=" + serverId + '}'; + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/NewCoordinatorServerEvent.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/NewCoordinatorServerEvent.java new file mode 100644 index 0000000000..cf6abcf56c --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/NewCoordinatorServerEvent.java @@ -0,0 +1,56 @@ +/* + * 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 com.alibaba.fluss.server.coordinator.event; + +import java.util.Objects; + +/** An event for new coordinator server. */ +public class NewCoordinatorServerEvent implements CoordinatorEvent { + + private final int serverId; + + public NewCoordinatorServerEvent(int serverId) { + this.serverId = serverId; + } + + public int getServerId() { + return serverId; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + NewCoordinatorServerEvent that = (NewCoordinatorServerEvent) o; + return serverId == that.serverId; + } + + @Override + public int hashCode() { + return Objects.hash(serverId); + } + + @Override + public String toString() { + return "NewCoordinatorServerEvent{" + "serverId=" + serverId + '}'; + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/CoordinatorServerChangeWatcher.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/CoordinatorServerChangeWatcher.java new file mode 100644 index 0000000000..1ddb3e37da --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/CoordinatorServerChangeWatcher.java @@ -0,0 +1,110 @@ +/* + * 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 com.alibaba.fluss.server.coordinator.event.watcher; + +import com.alibaba.fluss.exception.FlussRuntimeException; +import com.alibaba.fluss.server.coordinator.event.DeadCoordinatorServerEvent; +import com.alibaba.fluss.server.coordinator.event.EventManager; +import com.alibaba.fluss.server.coordinator.event.NewCoordinatorServerEvent; +import com.alibaba.fluss.server.zk.ZooKeeperClient; +import com.alibaba.fluss.server.zk.data.ZkData; +import com.alibaba.fluss.shaded.curator5.org.apache.curator.framework.recipes.cache.ChildData; +import com.alibaba.fluss.shaded.curator5.org.apache.curator.framework.recipes.cache.CuratorCache; +import com.alibaba.fluss.shaded.curator5.org.apache.curator.framework.recipes.cache.CuratorCacheListener; +import com.alibaba.fluss.shaded.curator5.org.apache.curator.utils.ZKPaths; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** A watcher to watch the coordinator server changes(new/delete) in zookeeper. */ +public class CoordinatorServerChangeWatcher { + + private static final Logger LOG = LoggerFactory.getLogger(CoordinatorServerChangeWatcher.class); + private final CuratorCache curatorCache; + + private volatile boolean running; + + private final EventManager eventManager; + + public CoordinatorServerChangeWatcher( + ZooKeeperClient zooKeeperClient, EventManager eventManager) { + this.curatorCache = + CuratorCache.build( + zooKeeperClient.getCuratorClient(), ZkData.CoordinatorIdsZNode.path()); + this.eventManager = eventManager; + this.curatorCache.listenable().addListener(new CoordinatorServerChangeListener()); + } + + public void start() { + running = true; + curatorCache.start(); + } + + public void stop() { + if (!running) { + return; + } + running = false; + LOG.info("Stopping CoordinatorServerChangeWatcher"); + curatorCache.close(); + } + + private final class CoordinatorServerChangeListener implements CuratorCacheListener { + + @Override + public void event(Type type, ChildData oldData, ChildData newData) { + if (newData != null) { + LOG.debug("Received {} event (path: {})", type, newData.getPath()); + } else { + LOG.debug("Received {} event", type); + } + + switch (type) { + case NODE_CREATED: + { + if (newData != null && newData.getData().length > 0) { + int serverId = getServerIdFromEvent(newData); + LOG.info("Received CHILD_ADDED event for server {}.", serverId); + eventManager.put(new NewCoordinatorServerEvent(serverId)); + } + break; + } + case NODE_DELETED: + { + if (oldData != null && oldData.getData().length > 0) { + int serverId = getServerIdFromEvent(oldData); + LOG.info("Received CHILD_REMOVED event for server {}.", serverId); + eventManager.put(new DeadCoordinatorServerEvent(serverId)); + } + break; + } + default: + break; + } + } + } + + private int getServerIdFromEvent(ChildData data) { + try { + return Integer.parseInt(ZKPaths.getNodeFromPath(data.getPath())); + } catch (NumberFormatException e) { + throw new FlussRuntimeException( + "Invalid server id in zookeeper path: " + data.getPath(), e); + } + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/TabletServerChangeWatcher.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/TabletServerChangeWatcher.java index a8f4dcb153..0bee350eff 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/TabletServerChangeWatcher.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/TabletServerChangeWatcher.java @@ -62,7 +62,7 @@ public void stop() { return; } running = false; - LOG.info("Stopping TableChangeWatcher"); + LOG.info("Stopping TabletServerChangeWatcher"); curatorCache.close(); } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java index d21698b95c..c1d764499b 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java @@ -202,7 +202,7 @@ public void registerCoordinatorLeader(CoordinatorAddress coordinatorAddress) thr } /** Get the leader address registered in ZK. */ - public Optional getCoordinatorAddress() throws Exception { + public Optional getCoordinatorLeaderAddress() throws Exception { Optional bytes = getOrEmpty(ZkData.CoordinatorLeaderZNode.path()); // return bytes.map(CoordinatorZNode::decode); return bytes.map( @@ -211,6 +211,12 @@ public Optional getCoordinatorAddress() throws Exception { data.length == 0 ? null : ZkData.CoordinatorLeaderZNode.decode(data)); } + /** Gets the list of coordinator server Ids. */ + public int[] getCoordinatorServerList() throws Exception { + List coordinatorServers = getChildren(ZkData.CoordinatorIdsZNode.path()); + return coordinatorServers.stream().mapToInt(Integer::parseInt).toArray(); + } + // -------------------------------------------------------------------------------------------- // Tablet server // -------------------------------------------------------------------------------------------- diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java index c7b7fd8099..6b57633e7f 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java @@ -402,7 +402,7 @@ void testServerBecomeOnlineAndOfflineLine() throws Exception { BucketState t1Bucket0State = fromCtx(ctx -> ctx.getBucketState(t1Bucket0)); assertThat(t1Bucket0State).isEqualTo(OnlineBucket); // t1 bucket 1 should reelect a leader since the leader is not alive - // the bucket whose leader is in the server should be online a again, but the leadership + // the bucket whose leader is in the server should be online again, but the leadership // should change the leader for bucket2 of t1 should change since the leader fail BucketState t1Bucket1State = fromCtx(ctx -> ctx.getBucketState(t1Bucket1)); assertThat(t1Bucket1State).isEqualTo(OnlineBucket); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java index 818883f91a..854b3de8fe 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java @@ -66,7 +66,7 @@ void testCoordinatorServerElection() throws Exception { waitUntilCoordinatorServerElected(); - CoordinatorAddress firstLeaderAddress = zookeeperClient.getCoordinatorAddress().get(); + CoordinatorAddress firstLeaderAddress = zookeeperClient.getCoordinatorLeaderAddress().get(); // Find the Coordinator server leader // and try to close it. @@ -83,7 +83,8 @@ void testCoordinatorServerElection() throws Exception { // coordinator leader changed. waitUntilCoordinatorServerReelected(firstLeaderAddress); - CoordinatorAddress secondLeaderAddress = zookeeperClient.getCoordinatorAddress().get(); + CoordinatorAddress secondLeaderAddress = + zookeeperClient.getCoordinatorLeaderAddress().get(); assertThat(secondLeaderAddress).isNotEqualTo(firstLeaderAddress); } @@ -110,7 +111,7 @@ protected static Configuration createConfiguration(int serverId) { public void waitUntilCoordinatorServerElected() { waitUtil( () -> { - return zookeeperClient.getCoordinatorAddress().isPresent(); + return zookeeperClient.getCoordinatorLeaderAddress().isPresent(); }, Duration.ofMinutes(1), "Fail to wait coordinator server elected"); @@ -119,8 +120,8 @@ public void waitUntilCoordinatorServerElected() { public void waitUntilCoordinatorServerReelected(CoordinatorAddress address) { waitUtil( () -> { - return zookeeperClient.getCoordinatorAddress().isPresent() - && !zookeeperClient.getCoordinatorAddress().get().equals(address); + return zookeeperClient.getCoordinatorLeaderAddress().isPresent() + && !zookeeperClient.getCoordinatorLeaderAddress().get().equals(address); }, Duration.ofMinutes(1), String.format("Fail to wait coordinator server reelected")); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java index 29718adc46..96e07889f7 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java @@ -64,7 +64,8 @@ protected ServerBase getStartFailServer() { protected void checkAfterStartServer() throws Exception { assertThat(coordinatorServer.getRpcServer()).isNotNull(); // check the data put in zk after coordinator server start - Optional optCoordinatorAddr = zookeeperClient.getCoordinatorAddress(); + Optional optCoordinatorAddr = + zookeeperClient.getCoordinatorLeaderAddress(); assertThat(optCoordinatorAddr).isNotEmpty(); verifyEndpoint( optCoordinatorAddr.get().getEndpoints(), diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/event/watcher/CoordinatorServerChangeWatcherTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/event/watcher/CoordinatorServerChangeWatcherTest.java new file mode 100644 index 0000000000..7101a4e9bb --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/event/watcher/CoordinatorServerChangeWatcherTest.java @@ -0,0 +1,86 @@ +/* + * 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 com.alibaba.fluss.server.coordinator.event.watcher; + +import com.alibaba.fluss.server.coordinator.event.CoordinatorEvent; +import com.alibaba.fluss.server.coordinator.event.DeadCoordinatorServerEvent; +import com.alibaba.fluss.server.coordinator.event.NewCoordinatorServerEvent; +import com.alibaba.fluss.server.coordinator.event.TestingEventManager; +import com.alibaba.fluss.server.zk.NOPErrorHandler; +import com.alibaba.fluss.server.zk.ZooKeeperClient; +import com.alibaba.fluss.server.zk.ZooKeeperExtension; +import com.alibaba.fluss.testutils.common.AllCallbackWrapper; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; + +import static com.alibaba.fluss.testutils.common.CommonTestUtils.retry; +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link CoordinatorServerChangeWatcher} . */ +class CoordinatorServerChangeWatcherTest { + + @RegisterExtension + public static final AllCallbackWrapper ZOO_KEEPER_EXTENSION_WRAPPER = + new AllCallbackWrapper<>(new ZooKeeperExtension()); + + @Test + void testServerChanges() throws Exception { + ZooKeeperClient zookeeperClient = + ZOO_KEEPER_EXTENSION_WRAPPER + .getCustomExtension() + .getZooKeeperClient(NOPErrorHandler.INSTANCE); + TestingEventManager eventManager = new TestingEventManager(); + CoordinatorServerChangeWatcher coordinatorServerChangeWatcher = + new CoordinatorServerChangeWatcher(zookeeperClient, eventManager); + coordinatorServerChangeWatcher.start(); + + // register new servers + List expectedEvents = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + expectedEvents.add(new NewCoordinatorServerEvent(i)); + zookeeperClient.registerCoordinatorServer(i); + } + + retry( + Duration.ofMinutes(1), + () -> + assertThat(eventManager.getEvents()) + .containsExactlyInAnyOrderElementsOf(expectedEvents)); + + // close it to mock the servers become down + zookeeperClient.close(); + + // unregister servers + for (int i = 0; i < 10; i++) { + expectedEvents.add(new DeadCoordinatorServerEvent(i)); + } + + retry( + Duration.ofMinutes(1), + () -> + assertThat(eventManager.getEvents()) + .containsExactlyInAnyOrderElementsOf(expectedEvents)); + + coordinatorServerChangeWatcher.stop(); + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/event/watcher/TabletServerChangeWatcherTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/event/watcher/TabletServerChangeWatcherTest.java index d7739028d7..6da8dfacd8 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/event/watcher/TabletServerChangeWatcherTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/event/watcher/TabletServerChangeWatcherTest.java @@ -49,7 +49,7 @@ class TabletServerChangeWatcherTest { new AllCallbackWrapper<>(new ZooKeeperExtension()); @Test - void testServetChanges() throws Exception { + void testServerChanges() throws Exception { ZooKeeperClient zookeeperClient = ZOO_KEEPER_EXTENSION_WRAPPER .getCustomExtension() diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java index 0d41355e39..eb97a1145e 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java @@ -105,14 +105,14 @@ static void afterAll() { void testCoordinatorLeader() throws Exception { // try to get leader address, should return empty since node leader address stored in // zk - assertThat(zookeeperClient.getCoordinatorAddress()).isEmpty(); + assertThat(zookeeperClient.getCoordinatorLeaderAddress()).isEmpty(); CoordinatorAddress coordinatorAddress = new CoordinatorAddress( 2, Endpoint.fromListenersString("CLIENT://localhost1:10012")); // register leader address zookeeperClient.registerCoordinatorLeader(coordinatorAddress); // check get leader address - CoordinatorAddress gottenAddress = zookeeperClient.getCoordinatorAddress().get(); + CoordinatorAddress gottenAddress = zookeeperClient.getCoordinatorLeaderAddress().get(); assertThat(gottenAddress).isEqualTo(coordinatorAddress); } diff --git a/website/docs/maintenance/observability/monitor-metrics.md b/website/docs/maintenance/observability/monitor-metrics.md index 4b082b2f2e..0a590789ab 100644 --- a/website/docs/maintenance/observability/monitor-metrics.md +++ b/website/docs/maintenance/observability/monitor-metrics.md @@ -294,12 +294,17 @@ Some metrics might not be exposed when using other JVM implementations (e.g. IBM - coordinator - - + coordinator + - activeCoordinatorCount The number of active CoordinatorServer in this cluster. Gauge + + aliveCoordinatorServerCount + The number of alive CoordinatorServer in this cluster. + Gauge + activeTabletServerCount The number of active TabletServer in this cluster. From 7a8ffc21fc50b4254c766e273ae4627734c11719 Mon Sep 17 00:00:00 2001 From: "whenzhou.zc" Date: Tue, 29 Jul 2025 09:44:30 +0800 Subject: [PATCH 06/26] [server] Coordinator Server Supports High-Available --- .../server/coordinator/CoordinatorServerElectionTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java index 854b3de8fe..fb9516e8d1 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java @@ -34,7 +34,7 @@ import java.util.Arrays; import java.util.List; -import static com.alibaba.fluss.testutils.common.CommonTestUtils.waitUtil; +import static com.alibaba.fluss.testutils.common.CommonTestUtils.waitUntil; import static org.assertj.core.api.Assertions.assertThat; class CoordinatorServerElectionTest { @@ -109,7 +109,7 @@ protected static Configuration createConfiguration(int serverId) { } public void waitUntilCoordinatorServerElected() { - waitUtil( + waitUntil( () -> { return zookeeperClient.getCoordinatorLeaderAddress().isPresent(); }, @@ -118,7 +118,7 @@ public void waitUntilCoordinatorServerElected() { } public void waitUntilCoordinatorServerReelected(CoordinatorAddress address) { - waitUtil( + waitUntil( () -> { return zookeeperClient.getCoordinatorLeaderAddress().isPresent() && !zookeeperClient.getCoordinatorLeaderAddress().get().equals(address); From d76949672f5e7e98312db25bfa5b02caf7e45c71 Mon Sep 17 00:00:00 2001 From: "whenzhou.zc" Date: Fri, 8 Aug 2025 14:36:41 +0800 Subject: [PATCH 07/26] user LeaderLatch instead of LeaderSelector --- .../CoordinatorLeaderElection.java | 101 ++++++++---------- .../event/CoordinatorEventManager.java | 3 + 2 files changed, 50 insertions(+), 54 deletions(-) diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java index 635c936121..e404f72f6b 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java @@ -20,78 +20,71 @@ import com.alibaba.fluss.server.zk.data.ZkData; import com.alibaba.fluss.shaded.curator5.org.apache.curator.framework.CuratorFramework; -import com.alibaba.fluss.shaded.curator5.org.apache.curator.framework.recipes.leader.LeaderSelector; -import com.alibaba.fluss.shaded.curator5.org.apache.curator.framework.recipes.leader.LeaderSelectorListener; -import com.alibaba.fluss.shaded.curator5.org.apache.curator.framework.state.ConnectionState; -import com.alibaba.fluss.utils.concurrent.ExecutorThreadFactory; +import com.alibaba.fluss.shaded.curator5.org.apache.curator.framework.recipes.leader.LeaderLatch; +import com.alibaba.fluss.shaded.curator5.org.apache.curator.framework.recipes.leader.LeaderLatchListener; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; +import java.io.IOException; +import java.util.concurrent.atomic.AtomicBoolean; /** Using by coordinator server. Coordinator servers listen ZK node and elect leadership. */ -public class CoordinatorLeaderElection { +public class CoordinatorLeaderElection implements AutoCloseable { private static final Logger LOG = LoggerFactory.getLogger(CoordinatorLeaderElection.class); - private final CuratorFramework zkClient; private final int serverId; - private final ScheduledExecutorService executor; + private final LeaderLatch leaderLatch; + private final AtomicBoolean isLeader = new AtomicBoolean(false); public CoordinatorLeaderElection(CuratorFramework zkClient, int serverId) { - this( - zkClient, - serverId, - Executors.newSingleThreadScheduledExecutor( - new ExecutorThreadFactory("fluss-coordinator-leader-election"))); - } - - protected CoordinatorLeaderElection( - CuratorFramework zkClient, int serverId, ScheduledExecutorService executor) { - this.zkClient = zkClient; this.serverId = serverId; - this.executor = executor; + this.leaderLatch = + new LeaderLatch( + zkClient, ZkData.CoordinatorElectionZNode.path(), String.valueOf(serverId)); } public void startElectLeader(Runnable initLeaderServices) { - executor.schedule(() -> electLeader(initLeaderServices), 0, TimeUnit.MILLISECONDS); - } + leaderLatch.addListener( + new LeaderLatchListener() { + @Override + public void isLeader() { + LOG.info("Coordinator server {} has become the leader.", serverId); + isLeader.set(true); + } + + @Override + public void notLeader() { + LOG.warn("Coordinator server {} has lost the leadership.", serverId); + isLeader.set(false); + } + }); - private void electLeader(Runnable initLeaderServices) { - LeaderSelector leaderSelector = - new LeaderSelector( - zkClient, - ZkData.CoordinatorElectionZNode.path(), - new LeaderSelectorListener() { - @Override - public void takeLeadership(CuratorFramework client) { - LOG.info( - "Coordinator server {} win the leader in election now.", - serverId); - initLeaderServices.run(); + try { + leaderLatch.start(); + LOG.info("Coordinator server {} started leader election.", serverId); - // Do not return, otherwise the leader will be released immediately. - while (true) { - try { - Thread.sleep(1000); - } catch (InterruptedException e) { - } - } - } + // 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 initializing context + leaderLatch.await(); + initLeaderServices.run(); - @Override - public void stateChanged( - CuratorFramework client, ConnectionState newState) { - if (newState == ConnectionState.LOST) { - LOG.info("Coordinator leader {} lost connection", serverId); - } - } - }); + } catch (Exception e) { + LOG.error("Failed to start LeaderLatch for server {}", serverId, e); + throw new RuntimeException("Leader election start failed", e); + } + } + + @Override + public void close() throws IOException { + LOG.info("Closing LeaderLatch for server {}.", serverId); + if (leaderLatch != null) { + leaderLatch.close(); + } + } - // allow reelection - leaderSelector.autoRequeue(); - leaderSelector.start(); + public boolean isLeader() { + return this.isLeader.get(); } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/CoordinatorEventManager.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/CoordinatorEventManager.java index ad22f6d1c9..d222490eda 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/CoordinatorEventManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/CoordinatorEventManager.java @@ -62,6 +62,7 @@ public final class CoordinatorEventManager implements EventManager { private Histogram eventQueueTime; // Coordinator metrics moved from CoordinatorEventProcessor + private volatile int aliveCoordinatorServerCount; private volatile int tabletServerCount; private volatile int offlineBucketCount; private volatile int tableCount; @@ -88,6 +89,8 @@ private void registerMetrics() { // Register coordinator metrics coordinatorMetricGroup.gauge(MetricNames.ACTIVE_COORDINATOR_COUNT, () -> 1); + coordinatorMetricGroup.gauge( + MetricNames.ALIVE_COORDINATOR_COUNT, () -> aliveCoordinatorServerCount); coordinatorMetricGroup.gauge( MetricNames.ACTIVE_TABLET_SERVER_COUNT, () -> tabletServerCount); coordinatorMetricGroup.gauge(MetricNames.OFFLINE_BUCKET_COUNT, () -> offlineBucketCount); From 656ebae35c9843ceb78ca56f29f37d89a89e0e96 Mon Sep 17 00:00:00 2001 From: "whenzhou.zc" Date: Mon, 18 Aug 2025 16:30:42 +0800 Subject: [PATCH 08/26] [server] Coordinator Server Supports High-Available --- .../server/coordinator/CoordinatorServer.java | 40 +++++-------------- .../fluss/server/zk/ZooKeeperClient.java | 9 ----- .../CoordinatorServerElectionTest.java | 18 +++++++-- 3 files changed, 26 insertions(+), 41 deletions(-) diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java index 915fa25108..221ef04c28 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java @@ -175,22 +175,17 @@ private void electCoordinatorLeader() throws Exception { ZooKeeperUtils.registerZookeeperClientReInitSessionListener( zkClient, this::registerCoordinatorServer, this); - // try to register Coordinator leader once - if (tryElectCoordinatorLeaderOnce()) { - startCoordinatorLeaderService(); - } else { - // standby - CoordinatorLeaderElection coordinatorLeaderElection = - new CoordinatorLeaderElection(zkClient.getCuratorClient(), serverId); - coordinatorLeaderElection.startElectLeader( - () -> { - try { - startCoordinatorLeaderService(); - } catch (Exception e) { - throw new RuntimeException(e); - } - }); - } + // standby + CoordinatorLeaderElection coordinatorLeaderElection = + new CoordinatorLeaderElection(zkClient.getCuratorClient(), serverId); + coordinatorLeaderElection.startElectLeader( + () -> { + try { + startCoordinatorLeaderService(); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); } protected void startCoordinatorLeaderService() throws Exception { @@ -347,19 +342,6 @@ private void registerCoordinatorServer() throws Exception { } } - private boolean tryElectCoordinatorLeaderOnce() throws Exception { - try { - zkClient.electCoordinatorLeader(); - LOG.info("Coordinator server {} win the leader in election now.", serverId); - return true; - } catch (KeeperException.NodeExistsException nodeExistsException) { - LOG.warn( - "Coordinator leader already registered in Zookeeper. Coordinator {} will be standby", - serverId); - return false; - } - } - private void registerCoordinatorLeader() throws Exception { long startTime = System.currentTimeMillis(); List bindEndpoints = rpcServer.getBindEndpoints(); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java index c1d764499b..ce3af0c13a 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java @@ -182,15 +182,6 @@ public void registerCoordinatorServer(int coordinatorId) throws Exception { LOG.info("Registered Coordinator server {} at path {}.", coordinatorId, path); } - /** - * Register a coordinator leader to ZK. Don't need to create node because leader election - * process already do it. - */ - public void electCoordinatorLeader() throws Exception { - String path = ZkData.CoordinatorElectionZNode.path(); - zkClient.create().creatingParentsIfNeeded().withMode(CreateMode.EPHEMERAL).forPath(path); - } - /** Register a coordinator leader to ZK. */ public void registerCoordinatorLeader(CoordinatorAddress coordinatorAddress) throws Exception { String path = ZkData.CoordinatorLeaderZNode.path(); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java index fb9516e8d1..ed3128564e 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java @@ -33,6 +33,8 @@ import java.time.Duration; import java.util.Arrays; import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import static com.alibaba.fluss.testutils.common.CommonTestUtils.waitUntil; import static org.assertj.core.api.Assertions.assertThat; @@ -57,13 +59,23 @@ void testCoordinatorServerElection() throws Exception { CoordinatorServer coordinatorServer1 = new CoordinatorServer(createConfiguration(1)); CoordinatorServer coordinatorServer2 = new CoordinatorServer(createConfiguration(2)); CoordinatorServer coordinatorServer3 = new CoordinatorServer(createConfiguration(3)); - coordinatorServer1.start(); - coordinatorServer2.start(); - coordinatorServer3.start(); List coordinatorServerList = Arrays.asList(coordinatorServer1, coordinatorServer2, coordinatorServer3); + ExecutorService executor = Executors.newFixedThreadPool(3); + for (int i = 0; i < 3; i++) { + CoordinatorServer server = coordinatorServerList.get(i); + executor.submit( + () -> { + try { + server.start(); + } catch (Exception e) { + e.printStackTrace(); + } + }); + } + waitUntilCoordinatorServerElected(); CoordinatorAddress firstLeaderAddress = zookeeperClient.getCoordinatorLeaderAddress().get(); From 5e40d8b087be48275031bafcec69c7f3dd5f50b4 Mon Sep 17 00:00:00 2001 From: "whenzhou.zc" Date: Tue, 2 Sep 2025 10:54:41 +0800 Subject: [PATCH 09/26] merge --- .../CoordinatorEpochFencedException.java | 26 +++++++++++++++++++ .../CoordinatorLeaderElection.java | 10 +++---- .../event/DeadCoordinatorServerEvent.java | 2 +- .../event/NewCoordinatorServerEvent.java | 2 +- .../CoordinatorServerChangeWatcher.java | 22 ++++++++-------- .../apache/fluss/server/ServerTestBase.java | 2 +- .../CoordinatorServerElectionTest.java | 18 ++++++------- .../CoordinatorServerChangeWatcherTest.java | 20 +++++++------- 8 files changed, 64 insertions(+), 38 deletions(-) create mode 100644 fluss-common/src/main/java/org/apache/fluss/exception/CoordinatorEpochFencedException.java diff --git a/fluss-common/src/main/java/org/apache/fluss/exception/CoordinatorEpochFencedException.java b/fluss-common/src/main/java/org/apache/fluss/exception/CoordinatorEpochFencedException.java new file mode 100644 index 0000000000..b6201c6159 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/exception/CoordinatorEpochFencedException.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.exception; + +/** Exception thrown when the Coordinator leader epoch is invalid. */ +public class CoordinatorEpochFencedException extends RuntimeException { + public CoordinatorEpochFencedException(String message) { + super(message); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java index e404f72f6b..2712c37b9d 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java @@ -16,12 +16,12 @@ * limitations under the License. */ -package com.alibaba.fluss.server.coordinator; +package org.apache.fluss.server.coordinator; -import com.alibaba.fluss.server.zk.data.ZkData; -import com.alibaba.fluss.shaded.curator5.org.apache.curator.framework.CuratorFramework; -import com.alibaba.fluss.shaded.curator5.org.apache.curator.framework.recipes.leader.LeaderLatch; -import com.alibaba.fluss.shaded.curator5.org.apache.curator.framework.recipes.leader.LeaderLatchListener; +import org.apache.fluss.server.zk.data.ZkData; +import org.apache.fluss.shaded.curator5.org.apache.curator.framework.CuratorFramework; +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; diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/DeadCoordinatorServerEvent.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/DeadCoordinatorServerEvent.java index 24850c7981..2ffc0110cc 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/DeadCoordinatorServerEvent.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/DeadCoordinatorServerEvent.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package com.alibaba.fluss.server.coordinator.event; +package org.apache.fluss.server.coordinator.event; import java.util.Objects; diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/NewCoordinatorServerEvent.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/NewCoordinatorServerEvent.java index cf6abcf56c..fd53bc412d 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/NewCoordinatorServerEvent.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/NewCoordinatorServerEvent.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package com.alibaba.fluss.server.coordinator.event; +package org.apache.fluss.server.coordinator.event; import java.util.Objects; diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/CoordinatorServerChangeWatcher.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/CoordinatorServerChangeWatcher.java index 1ddb3e37da..f0008d5186 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/CoordinatorServerChangeWatcher.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/CoordinatorServerChangeWatcher.java @@ -15,18 +15,18 @@ * limitations under the License. */ -package com.alibaba.fluss.server.coordinator.event.watcher; +package org.apache.fluss.server.coordinator.event.watcher; -import com.alibaba.fluss.exception.FlussRuntimeException; -import com.alibaba.fluss.server.coordinator.event.DeadCoordinatorServerEvent; -import com.alibaba.fluss.server.coordinator.event.EventManager; -import com.alibaba.fluss.server.coordinator.event.NewCoordinatorServerEvent; -import com.alibaba.fluss.server.zk.ZooKeeperClient; -import com.alibaba.fluss.server.zk.data.ZkData; -import com.alibaba.fluss.shaded.curator5.org.apache.curator.framework.recipes.cache.ChildData; -import com.alibaba.fluss.shaded.curator5.org.apache.curator.framework.recipes.cache.CuratorCache; -import com.alibaba.fluss.shaded.curator5.org.apache.curator.framework.recipes.cache.CuratorCacheListener; -import com.alibaba.fluss.shaded.curator5.org.apache.curator.utils.ZKPaths; +import org.apache.fluss.exception.FlussRuntimeException; +import org.apache.fluss.server.coordinator.event.DeadCoordinatorServerEvent; +import org.apache.fluss.server.coordinator.event.EventManager; +import org.apache.fluss.server.coordinator.event.NewCoordinatorServerEvent; +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.cache.ChildData; +import org.apache.fluss.shaded.curator5.org.apache.curator.framework.recipes.cache.CuratorCache; +import org.apache.fluss.shaded.curator5.org.apache.curator.framework.recipes.cache.CuratorCacheListener; +import org.apache.fluss.shaded.curator5.org.apache.curator.utils.ZKPaths; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/fluss-server/src/test/java/org/apache/fluss/server/ServerTestBase.java b/fluss-server/src/test/java/org/apache/fluss/server/ServerTestBase.java index aec981a963..c95f1d7cb7 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/ServerTestBase.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/ServerTestBase.java @@ -25,7 +25,7 @@ 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.ZkData.CoordinatorZNode; +import org.apache.fluss.server.zk.data.ZkData; import org.apache.fluss.server.zk.data.ZkData.ServerIdZNode; import org.apache.fluss.shaded.zookeeper3.org.apache.zookeeper.data.Stat; import org.apache.fluss.testutils.common.AllCallbackWrapper; diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java index ed3128564e..57a4754925 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java @@ -16,15 +16,15 @@ * limitations under the License. */ -package com.alibaba.fluss.server.coordinator; +package org.apache.fluss.server.coordinator; -import com.alibaba.fluss.config.ConfigOptions; -import com.alibaba.fluss.config.Configuration; -import com.alibaba.fluss.server.zk.NOPErrorHandler; -import com.alibaba.fluss.server.zk.ZooKeeperClient; -import com.alibaba.fluss.server.zk.ZooKeeperExtension; -import com.alibaba.fluss.server.zk.data.CoordinatorAddress; -import com.alibaba.fluss.testutils.common.AllCallbackWrapper; +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.config.Configuration; +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.CoordinatorAddress; +import org.apache.fluss.testutils.common.AllCallbackWrapper; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -36,7 +36,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import static com.alibaba.fluss.testutils.common.CommonTestUtils.waitUntil; +import static org.apache.fluss.testutils.common.CommonTestUtils.waitUntil; import static org.assertj.core.api.Assertions.assertThat; class CoordinatorServerElectionTest { diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/event/watcher/CoordinatorServerChangeWatcherTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/event/watcher/CoordinatorServerChangeWatcherTest.java index 7101a4e9bb..f35206c3d3 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/event/watcher/CoordinatorServerChangeWatcherTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/event/watcher/CoordinatorServerChangeWatcherTest.java @@ -15,16 +15,16 @@ * limitations under the License. */ -package com.alibaba.fluss.server.coordinator.event.watcher; +package org.apache.fluss.server.coordinator.event.watcher; -import com.alibaba.fluss.server.coordinator.event.CoordinatorEvent; -import com.alibaba.fluss.server.coordinator.event.DeadCoordinatorServerEvent; -import com.alibaba.fluss.server.coordinator.event.NewCoordinatorServerEvent; -import com.alibaba.fluss.server.coordinator.event.TestingEventManager; -import com.alibaba.fluss.server.zk.NOPErrorHandler; -import com.alibaba.fluss.server.zk.ZooKeeperClient; -import com.alibaba.fluss.server.zk.ZooKeeperExtension; -import com.alibaba.fluss.testutils.common.AllCallbackWrapper; +import org.apache.fluss.server.coordinator.event.CoordinatorEvent; +import org.apache.fluss.server.coordinator.event.DeadCoordinatorServerEvent; +import org.apache.fluss.server.coordinator.event.NewCoordinatorServerEvent; +import org.apache.fluss.server.coordinator.event.TestingEventManager; +import org.apache.fluss.server.zk.NOPErrorHandler; +import org.apache.fluss.server.zk.ZooKeeperClient; +import org.apache.fluss.server.zk.ZooKeeperExtension; +import org.apache.fluss.testutils.common.AllCallbackWrapper; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; @@ -33,7 +33,7 @@ import java.util.ArrayList; import java.util.List; -import static com.alibaba.fluss.testutils.common.CommonTestUtils.retry; +import static org.apache.fluss.testutils.common.CommonTestUtils.retry; import static org.assertj.core.api.Assertions.assertThat; /** Test for {@link CoordinatorServerChangeWatcher} . */ From 48b6f8ebbb4a72881957348aa2be7dbee8db3f16 Mon Sep 17 00:00:00 2001 From: "whenzhou.zc" Date: Tue, 2 Sep 2025 11:24:10 +0800 Subject: [PATCH 10/26] support coordinator epoch --- .../coordinator/CoordinatorContext.java | 11 ++++ .../CoordinatorLeaderElection.java | 50 ++++++++++++++- .../coordinator/CoordinatorRequestBatch.java | 1 + .../server/coordinator/CoordinatorServer.java | 7 +-- .../server/utils/ServerRpcMessageUtils.java | 4 ++ .../fluss/server/zk/ZooKeeperClient.java | 62 ++++++++++++++++++- .../apache/fluss/server/zk/data/ZkData.java | 19 ++++++ .../CoordinatorChannelManagerTest.java | 1 + .../coordinator/TableManagerITCase.java | 1 + .../server/tablet/TabletServiceITCase.java | 1 + 10 files changed, 148 insertions(+), 9 deletions(-) diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorContext.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorContext.java index b41b4769ed..62a9a45805 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorContext.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorContext.java @@ -55,6 +55,7 @@ public class CoordinatorContext { private static final Logger LOG = LoggerFactory.getLogger(CoordinatorContext.class); public static final int INITIAL_COORDINATOR_EPOCH = 0; + public static final int INITIAL_COORDINATOR_EPOCH_ZKVERSION = 0; // for simplicity, we just use retry time, may consider make it a configurable value // and use combine retry times and retry delay @@ -109,6 +110,7 @@ public class CoordinatorContext { private ServerInfo coordinatorServerInfo = null; private int coordinatorEpoch = INITIAL_COORDINATOR_EPOCH; + private int coordinatorEpochZkVersion = INITIAL_COORDINATOR_EPOCH_ZKVERSION; public CoordinatorContext() {} @@ -116,6 +118,15 @@ public int getCoordinatorEpoch() { return coordinatorEpoch; } + public int getCoordinatorEpochZkVersion() { + return coordinatorEpochZkVersion; + } + + public void setCoordinatorEpochAndZkVersion(int newEpoch, int newZkVersion) { + this.coordinatorEpoch = newEpoch; + this.coordinatorEpochZkVersion = newZkVersion; + } + public Set getLiveCoordinatorServers() { return liveCoordinatorServers; } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java index 2712c37b9d..73b1e55dc2 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java @@ -18,8 +18,9 @@ package org.apache.fluss.server.coordinator; +import org.apache.fluss.exception.CoordinatorEpochFencedException; +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.CuratorFramework; 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; @@ -27,6 +28,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.Optional; import java.util.concurrent.atomic.AtomicBoolean; /** Using by coordinator server. Coordinator servers listen ZK node and elect leadership. */ @@ -34,14 +36,26 @@ 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 CoordinatorServer server; private final AtomicBoolean isLeader = new AtomicBoolean(false); - public CoordinatorLeaderElection(CuratorFramework zkClient, int serverId) { + public CoordinatorLeaderElection( + ZooKeeperClient zkClient, + int serverId, + CoordinatorContext coordinatorContext, + CoordinatorServer server) { this.serverId = serverId; + this.zkClient = zkClient; + this.coordinatorContext = coordinatorContext; + this.server = server; this.leaderLatch = new LeaderLatch( - zkClient, ZkData.CoordinatorElectionZNode.path(), String.valueOf(serverId)); + zkClient.getCuratorClient(), + ZkData.CoordinatorElectionZNode.path(), + String.valueOf(serverId)); } public void startElectLeader(Runnable initLeaderServices) { @@ -51,10 +65,28 @@ public void startElectLeader(Runnable initLeaderServices) { public void isLeader() { LOG.info("Coordinator server {} has become the leader.", serverId); isLeader.set(true); + try { + // to avoid split-brain + Optional optionalEpoch = + zkClient.fenceBecomeCoordinatorLeader(serverId); + if (optionalEpoch.isPresent()) { + coordinatorContext.setCoordinatorEpochAndZkVersion( + optionalEpoch.get(), + coordinatorContext.getCoordinatorEpochZkVersion() + 1); + } else { + throw new CoordinatorEpochFencedException( + "Fenced to become coordinator leader."); + } + } catch (Exception e) { + relinquishLeadership(); + throw new CoordinatorEpochFencedException( + "Fenced to become coordinator leader."); + } } @Override public void notLeader() { + relinquishLeadership(); LOG.warn("Coordinator server {} has lost the leadership.", serverId); isLeader.set(false); } @@ -87,4 +119,16 @@ public void close() throws IOException { public boolean isLeader() { return this.isLeader.get(); } + + private void relinquishLeadership() { + isLeader.set(false); + LOG.info("Coordinator server {} has been fenced.", serverId); + + try { + leaderLatch.close(); + server.closeAsync(); + leaderLatch.start(); + } catch (Exception e) { + } + } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorRequestBatch.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorRequestBatch.java index fe85ff391e..2a83a5c24e 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorRequestBatch.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorRequestBatch.java @@ -682,6 +682,7 @@ private UpdateMetadataRequest buildUpdateMetadataRequest() { // tablet servers. return makeUpdateMetadataRequest( coordinatorContext.getCoordinatorServerInfo(), + coordinatorContext.getCoordinatorEpoch(), new HashSet<>(coordinatorContext.getLiveTabletServers().values()), tableMetadataList, partitionMetadataList); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java index 221ef04c28..66a8aaf5d4 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java @@ -159,6 +159,7 @@ public static void main(String[] args) { @Override protected void startServices() throws Exception { + this.coordinatorContext = new CoordinatorContext(); electCoordinatorLeader(); } @@ -168,16 +169,13 @@ private void electCoordinatorLeader() throws Exception { // Coordinator Server supports high availability. If 3 coordinator servers are alive, // one of them will be elected as leader and the other two will be standby. // When leader fails, one of standby coordinators will be elected as new leader. - // All of them register to ZK like tablet servers in path - // "/coordinators/ids/1","/coordinators/ids/2","/coordinators/ids/3". - // but the leader will be elected in path "/coordinators/leader" additionally. registerCoordinatorServer(); ZooKeeperUtils.registerZookeeperClientReInitSessionListener( zkClient, this::registerCoordinatorServer, this); // standby CoordinatorLeaderElection coordinatorLeaderElection = - new CoordinatorLeaderElection(zkClient.getCuratorClient(), serverId); + new CoordinatorLeaderElection(zkClient, serverId, coordinatorContext, this); coordinatorLeaderElection.startElectLeader( () -> { try { @@ -211,7 +209,6 @@ protected void startCoordinatorLeaderService() throws Exception { dynamicConfigManager.startup(); - this.coordinatorContext = new CoordinatorContext(); this.metadataCache = new CoordinatorMetadataCache(); this.authorizer = AuthorizerLoader.createAuthorizer(conf, zkClient, pluginManager); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java b/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java index 3a8dd5c240..0656413634 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java @@ -410,6 +410,7 @@ public static MetadataResponse buildMetadataResponse( public static UpdateMetadataRequest makeUpdateMetadataRequest( @Nullable ServerInfo coordinatorServer, + @Nullable Integer coordinatorEpoch, Set aliveTableServers, List tableMetadataList, List partitionMetadataList) { @@ -452,6 +453,9 @@ public static UpdateMetadataRequest makeUpdateMetadataRequest( updateMetadataRequest.addAllTableMetadatas(pbTableMetadataList); updateMetadataRequest.addAllPartitionMetadatas(pbPartitionMetadataList); + if (coordinatorEpoch != null) { + updateMetadataRequest.setCoordinatorEpoch(coordinatorEpoch); + } return updateMetadataRequest; } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java index ce3af0c13a..bc946cfc3a 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java @@ -33,6 +33,7 @@ import org.apache.fluss.security.acl.Resource; import org.apache.fluss.security.acl.ResourceType; import org.apache.fluss.server.authorizer.DefaultAuthorizer.VersionedAcls; +import org.apache.fluss.server.coordinator.CoordinatorContext; import org.apache.fluss.server.entity.RegisterTableBucketLeadAndIsrInfo; import org.apache.fluss.server.metadata.BucketMetadata; import org.apache.fluss.server.zk.ZkAsyncRequest.ZkCheckExistsRequest; @@ -182,6 +183,49 @@ public void registerCoordinatorServer(int coordinatorId) throws Exception { 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 fenceBecomeCoordinatorLeader(int coordinatorId) throws Exception { + try { + ensureEpochZnodeExists(); + + try { + Stat currentStat = new Stat(); + byte[] bytes = + zkClient.getData() + .storingStatIn(currentStat) + .forPath(ZkData.CoordinatorEpochZNode.path()); + int currentEpoch = ZkData.CoordinatorEpochZNode.decode(bytes); + int currentVersion = currentStat.getVersion(); + int newEpoch = currentEpoch + 1; + LOG.info( + "Coordinator leader {} tries to update epoch. Current epoch={}, Zookeeper version={}, new epoch={}", + coordinatorId, + currentEpoch, + currentVersion, + newEpoch); + + // atomically update epoch + zkClient.setData() + .withVersion(currentVersion) + .forPath( + ZkData.CoordinatorEpochZNode.path(), + ZkData.CoordinatorEpochZNode.encode(newEpoch)); + + return Optional.of(newEpoch); + } 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); + } + } catch (KeeperException.NodeExistsException e) { + } + return Optional.empty(); + } + /** Register a coordinator leader to ZK. */ public void registerCoordinatorLeader(CoordinatorAddress coordinatorAddress) throws Exception { String path = ZkData.CoordinatorLeaderZNode.path(); @@ -195,7 +239,6 @@ public void registerCoordinatorLeader(CoordinatorAddress coordinatorAddress) thr /** Get the leader address registered in ZK. */ public Optional getCoordinatorLeaderAddress() throws Exception { Optional bytes = getOrEmpty(ZkData.CoordinatorLeaderZNode.path()); - // return bytes.map(CoordinatorZNode::decode); return bytes.map( data -> // maybe a empty node when a leader is elected but not registered @@ -208,6 +251,23 @@ public int[] getCoordinatorServerList() throws Exception { 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) { + try { + zkClient.create() + .creatingParentsIfNeeded() + .withMode(CreateMode.PERSISTENT) + .forPath( + path, + ZkData.CoordinatorEpochZNode.encode( + CoordinatorContext.INITIAL_COORDINATOR_EPOCH)); + } catch (KeeperException.NodeExistsException e) { + } + } + } + // -------------------------------------------------------------------------------------------- // Tablet server // -------------------------------------------------------------------------------------------- diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java index be68123a8d..e290711059 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java @@ -354,6 +354,25 @@ public static CoordinatorAddress decode(byte[] json) { return JsonSerdeUtils.readValue(json, CoordinatorAddressJsonSerde.INSTANCE); } } + + /** + * The znode for the coordinator epoch. The znode path is: + * + *

/coordinators/epoch + */ + public static final class CoordinatorEpochZNode { + public static String path() { + return "/coordinators/epoch"; + } + + public static byte[] encode(int epoch) { + return String.valueOf(epoch).getBytes(); + } + + public static int decode(byte[] bytes) { + return Integer.parseInt(new String(bytes)); + } + } // ------------------------------------------------------------------------------------------ // ZNodes under "/tabletservers/" // ------------------------------------------------------------------------------------------ diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorChannelManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorChannelManagerTest.java index bd4467c0a2..be937226b0 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorChannelManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorChannelManagerTest.java @@ -93,6 +93,7 @@ private void checkSendRequest( // we use update metadata request to test for simplicity UpdateMetadataRequest updateMetadataRequest = makeUpdateMetadataRequest( + null, null, Collections.emptySet(), Collections.emptyList(), diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TableManagerITCase.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TableManagerITCase.java index e49636cf43..a6031736df 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TableManagerITCase.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TableManagerITCase.java @@ -511,6 +511,7 @@ void testMetadata(boolean isCoordinatorServer) throws Exception { .updateMetadata( makeUpdateMetadataRequest( coordinatorServerInfo, + null, new HashSet<>(tabletServerInfos), Collections.emptyList(), Collections.emptyList())) diff --git a/fluss-server/src/test/java/org/apache/fluss/server/tablet/TabletServiceITCase.java b/fluss-server/src/test/java/org/apache/fluss/server/tablet/TabletServiceITCase.java index 1137e499a2..7bdd781072 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/tablet/TabletServiceITCase.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/tablet/TabletServiceITCase.java @@ -890,6 +890,7 @@ void testBecomeLeaderOrFollowerWithOneTabletServerOffline() throws Exception { .updateMetadata( makeUpdateMetadataRequest( coordinatorServerInfo, + null, newTabletServerInfos, Collections.emptyList(), Collections.emptyList())) From dfc9aca57eaaafd7d0d5f013ca6f05484e668ea9 Mon Sep 17 00:00:00 2001 From: "whenzhou.zc" Date: Tue, 2 Sep 2025 11:46:27 +0800 Subject: [PATCH 11/26] support coordinator epoch --- .../fluss/server/coordinator/CoordinatorLeaderElection.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java index 73b1e55dc2..33a15e0710 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java @@ -73,6 +73,7 @@ public void isLeader() { coordinatorContext.setCoordinatorEpochAndZkVersion( optionalEpoch.get(), coordinatorContext.getCoordinatorEpochZkVersion() + 1); + initLeaderServices.run(); } else { throw new CoordinatorEpochFencedException( "Fenced to become coordinator leader."); @@ -99,8 +100,8 @@ public void notLeader() { // 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 initializing context - leaderLatch.await(); - initLeaderServices.run(); + // leaderLatch.await(); + // initLeaderServices.run(); } catch (Exception e) { LOG.error("Failed to start LeaderLatch for server {}", serverId, e); From 862bbde78999949608159fc0a1f11a193d93233d Mon Sep 17 00:00:00 2001 From: "whenzhou.zc" Date: Mon, 20 Oct 2025 15:22:53 +0800 Subject: [PATCH 12/26] support coordinator epoch --- .../fluss/server/coordinator/CoordinatorLeaderElection.java | 1 - .../org/apache/fluss/server/coordinator/CoordinatorServer.java | 2 -- .../main/java/org/apache/fluss/server/zk/ZooKeeperClient.java | 1 - 3 files changed, 4 deletions(-) diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java index 33a15e0710..8172f2dd8d 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java @@ -128,7 +128,6 @@ private void relinquishLeadership() { try { leaderLatch.close(); server.closeAsync(); - leaderLatch.start(); } catch (Exception e) { } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java index 66a8aaf5d4..4b7988ada9 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java @@ -56,9 +56,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; -import java.util.Map; import java.util.Optional; -import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java index bc946cfc3a..05b7e0224c 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java @@ -61,7 +61,6 @@ import org.apache.fluss.server.zk.data.ZkData.BucketSnapshotIdZNode; import org.apache.fluss.server.zk.data.ZkData.BucketSnapshotsZNode; import org.apache.fluss.server.zk.data.ZkData.ConfigEntityZNode; -import org.apache.fluss.server.zk.data.ZkData.CoordinatorZNode; import org.apache.fluss.server.zk.data.ZkData.DatabaseZNode; import org.apache.fluss.server.zk.data.ZkData.DatabasesZNode; import org.apache.fluss.server.zk.data.ZkData.LakeTableZNode; From beccef907b61d2e317e60f88d53d9a627787122f Mon Sep 17 00:00:00 2001 From: "whenzhou.zc" Date: Mon, 20 Oct 2025 15:49:03 +0800 Subject: [PATCH 13/26] modify test --- .../server/coordinator/CoordinatorServerTest.java | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java index 96e07889f7..c25dacb404 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java @@ -26,8 +26,10 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; +import java.time.Duration; import java.util.Optional; +import static org.apache.fluss.testutils.common.CommonTestUtils.waitUntil; import static org.assertj.core.api.Assertions.assertThat; /** Test for {@link CoordinatorServer} . */ @@ -38,6 +40,7 @@ class CoordinatorServerTest extends ServerTestBase { @BeforeEach void beforeEach() throws Exception { coordinatorServer = startCoordinatorServer(createConfiguration()); + waitUtilCoordinatorServerElected(); } @AfterEach @@ -71,4 +74,13 @@ protected void checkAfterStartServer() throws Exception { optCoordinatorAddr.get().getEndpoints(), coordinatorServer.getRpcServer().getBindEndpoints()); } + + public void waitUtilCoordinatorServerElected() { + waitUntil( + () -> { + return zookeeperClient.getCoordinatorLeaderAddress().isPresent(); + }, + Duration.ofSeconds(10), + String.format("Fail to wait coordinator server reelected")); + } } From 23beccda624013f156415dc5621b27673b5f4fca Mon Sep 17 00:00:00 2001 From: "whenzhou.zc" Date: Mon, 20 Oct 2025 15:59:02 +0800 Subject: [PATCH 14/26] modify test --- .../coordinator/CoordinatorServerElectionTest.java | 2 +- .../server/coordinator/CoordinatorServerTest.java | 2 +- .../fluss/server/testutils/FlussClusterExtension.java | 10 ++++++++++ 3 files changed, 12 insertions(+), 2 deletions(-) diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java index 57a4754925..05a535b400 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java @@ -136,6 +136,6 @@ public void waitUntilCoordinatorServerReelected(CoordinatorAddress address) { && !zookeeperClient.getCoordinatorLeaderAddress().get().equals(address); }, Duration.ofMinutes(1), - String.format("Fail to wait coordinator server reelected")); + "Fail to wait coordinator server reelected"); } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java index c25dacb404..dbe7dfb710 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java @@ -81,6 +81,6 @@ public void waitUtilCoordinatorServerElected() { return zookeeperClient.getCoordinatorLeaderAddress().isPresent(); }, Duration.ofSeconds(10), - String.format("Fail to wait coordinator server reelected")); + "Fail to wait coordinator server elected"); } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java b/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java index 713f049b33..d3393de3f2 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java @@ -271,6 +271,7 @@ public void startCoordinatorServer() throws Exception { setRemoteDataDir(conf); coordinatorServer = new CoordinatorServer(conf); coordinatorServer.start(); + waitUtilCoordinatorServerElected(); coordinatorServerInfo = // TODO, Currently, we use 0 as coordinator server id. new ServerInfo( @@ -938,6 +939,15 @@ public CoordinatorServer getCoordinatorServer() { return coordinatorServer; } + public void waitUtilCoordinatorServerElected() { + waitUntil( + () -> { + return zooKeeperClient.getCoordinatorLeaderAddress().isPresent(); + }, + Duration.ofSeconds(10), + "Fail to wait coordinator server elected"); + } + // -------------------------------------------------------------------------------------------- /** Builder for {@link FlussClusterExtension}. */ From 638ab655f55c4865a6db59859f171dbdb6b7e82b Mon Sep 17 00:00:00 2001 From: "whenzhou.zc" Date: Tue, 21 Oct 2025 20:09:18 +0800 Subject: [PATCH 15/26] fix test --- .../CoordinatorLeaderElection.java | 15 +++++++------- .../server/coordinator/CoordinatorServer.java | 4 ++-- .../CoordinatorServerElectionTest.java | 20 ++++++++++++++++--- 3 files changed, 26 insertions(+), 13 deletions(-) diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java index 8172f2dd8d..40dfc7d60d 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java @@ -27,7 +27,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.util.Optional; import java.util.concurrent.atomic.AtomicBoolean; @@ -110,10 +109,14 @@ public void notLeader() { } @Override - public void close() throws IOException { + public void close() { LOG.info("Closing LeaderLatch for server {}.", serverId); if (leaderLatch != null) { - leaderLatch.close(); + try { + leaderLatch.close(); + } catch (Exception e) { + LOG.error("Failed to close LeaderLatch for server {}.", serverId, e); + } } } @@ -125,10 +128,6 @@ private void relinquishLeadership() { isLeader.set(false); LOG.info("Coordinator server {} has been fenced.", serverId); - try { - leaderLatch.close(); - server.closeAsync(); - } catch (Exception e) { - } + this.close(); } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java index 4b7988ada9..12b81b3ec5 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java @@ -248,8 +248,8 @@ protected void startCoordinatorLeaderService() throws Exception { registerCoordinatorLeader(); // when init session, register coordinator server again - ZooKeeperUtils.registerZookeeperClientReInitSessionListener( - zkClient, this::registerCoordinatorLeader, this); + // ZooKeeperUtils.registerZookeeperClientReInitSessionListener( + // zkClient, this::registerCoordinatorLeader, this); this.clientMetricGroup = new ClientMetricGroup(metricRegistry, SERVER_NAME); this.rpcClient = RpcClient.create(conf, clientMetricGroup, true); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java index 05a535b400..99031e95ff 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java @@ -63,6 +63,7 @@ void testCoordinatorServerElection() throws Exception { List coordinatorServerList = Arrays.asList(coordinatorServer1, coordinatorServer2, coordinatorServer3); + // start 3 coordinator servers ExecutorService executor = Executors.newFixedThreadPool(3); for (int i = 0; i < 3; i++) { CoordinatorServer server = coordinatorServerList.get(i); @@ -76,13 +77,13 @@ void testCoordinatorServerElection() throws Exception { }); } + // random 1 become leader waitUntilCoordinatorServerElected(); CoordinatorAddress firstLeaderAddress = zookeeperClient.getCoordinatorLeaderAddress().get(); - // Find the Coordinator server leader + // Find the leader // and try to close it. - // Then we should get another Coordinator server leader elected CoordinatorServer elected = null; for (CoordinatorServer coordinatorServer : coordinatorServerList) { if (coordinatorServer.getServerId() == firstLeaderAddress.getId()) { @@ -92,12 +93,25 @@ void testCoordinatorServerElection() throws Exception { } assertThat(elected).isNotNull(); elected.close(); + elected.start(); - // coordinator leader changed. + // Then we should get another Coordinator server leader elected waitUntilCoordinatorServerReelected(firstLeaderAddress); CoordinatorAddress secondLeaderAddress = zookeeperClient.getCoordinatorLeaderAddress().get(); assertThat(secondLeaderAddress).isNotEqualTo(firstLeaderAddress); + + // kill other 2 coordinator servers + for (CoordinatorServer coordinatorServer : coordinatorServerList) { + if (coordinatorServer.getServerId() != firstLeaderAddress.getId()) { + coordinatorServer.close(); + } + } + // the origin coordinator server should become leader again + waitUntilCoordinatorServerElected(); + CoordinatorAddress thirdLeaderAddress = zookeeperClient.getCoordinatorLeaderAddress().get(); + + assertThat(thirdLeaderAddress.getId()).isEqualTo(firstLeaderAddress.getId()); } /** Create a configuration with Zookeeper address setting. */ From 6f2eae9e0f2af97539fd6346f3a7a04da71e1908 Mon Sep 17 00:00:00 2001 From: "whenzhou.zc" Date: Wed, 22 Oct 2025 16:11:11 +0800 Subject: [PATCH 16/26] support coordinator epoch --- .../coordinator/CoordinatorContext.java | 5 +- .../CoordinatorEventProcessor.java | 6 ++- .../server/coordinator/CoordinatorServer.java | 3 -- .../event/CoordinatorEventManager.java | 6 +++ .../statemachine/ReplicaStateMachine.java | 3 +- .../statemachine/TableBucketStateMachine.java | 5 +- .../fluss/server/zk/ZooKeeperClient.java | 53 +++++++++++++++---- .../CoordinatorServerElectionTest.java | 9 +++- .../coordinator/CoordinatorServerTest.java | 4 +- .../testutils/FlussClusterExtension.java | 4 +- .../fluss/server/zk/ZooKeeperClientTest.java | 22 ++++---- 11 files changed, 82 insertions(+), 38 deletions(-) diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorContext.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorContext.java index 62a9a45805..a6c28ebc24 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorContext.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorContext.java @@ -55,7 +55,7 @@ public class CoordinatorContext { private static final Logger LOG = LoggerFactory.getLogger(CoordinatorContext.class); public static final int INITIAL_COORDINATOR_EPOCH = 0; - public static final int INITIAL_COORDINATOR_EPOCH_ZKVERSION = 0; + public static final int INITIAL_COORDINATOR_EPOCH_ZK_VERSION = 0; // for simplicity, we just use retry time, may consider make it a configurable value // and use combine retry times and retry delay @@ -110,7 +110,7 @@ public class CoordinatorContext { private ServerInfo coordinatorServerInfo = null; private int coordinatorEpoch = INITIAL_COORDINATOR_EPOCH; - private int coordinatorEpochZkVersion = INITIAL_COORDINATOR_EPOCH_ZKVERSION; + private int coordinatorEpochZkVersion = INITIAL_COORDINATOR_EPOCH_ZK_VERSION; public CoordinatorContext() {} @@ -131,7 +131,6 @@ public Set getLiveCoordinatorServers() { return liveCoordinatorServers; } - @VisibleForTesting public void setLiveCoordinatorServers(Set servers) { liveCoordinatorServers.clear(); liveCoordinatorServers.addAll(servers); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java index c97aa0d77b..b85156983c 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java @@ -1660,7 +1660,8 @@ private List tryProcessAdjustIsr( } try { - zooKeeperClient.batchUpdateLeaderAndIsr(newLeaderAndIsrList); + zooKeeperClient.batchUpdateLeaderAndIsr( + newLeaderAndIsrList, coordinatorContext.getCoordinatorEpoch()); newLeaderAndIsrList.forEach( (tableBucket, newLeaderAndIsr) -> result.add(new AdjustIsrResultForBucket(tableBucket, newLeaderAndIsr))); @@ -1671,7 +1672,8 @@ private List tryProcessAdjustIsr( TableBucket tableBucket = entry.getKey(); LeaderAndIsr newLeaderAndIsr = entry.getValue(); try { - zooKeeperClient.updateLeaderAndIsr(tableBucket, newLeaderAndIsr); + zooKeeperClient.updateLeaderAndIsr( + tableBucket, newLeaderAndIsr, coordinatorContext.getCoordinatorEpoch()); } catch (Exception e) { LOG.error("Error when register leader and isr.", e); result.add( diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java index 12b81b3ec5..8f59ce9aa6 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java @@ -247,9 +247,6 @@ protected void startCoordinatorLeaderService() throws Exception { rpcServer.start(); registerCoordinatorLeader(); - // when init session, register coordinator server again - // ZooKeeperUtils.registerZookeeperClientReInitSessionListener( - // zkClient, this::registerCoordinatorLeader, this); this.clientMetricGroup = new ClientMetricGroup(metricRegistry, SERVER_NAME); this.rpcClient = RpcClient.create(conf, clientMetricGroup, true); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/CoordinatorEventManager.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/CoordinatorEventManager.java index d222490eda..74a49a0da1 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/CoordinatorEventManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/CoordinatorEventManager.java @@ -108,6 +108,7 @@ private void updateMetricsViaAccessContext() { AccessContextEvent accessContextEvent = new AccessContextEvent<>( context -> { + int coordinatorServerCount = context.getLiveCoordinatorServers().size(); int tabletServerCount = context.getLiveTabletServers().size(); int tableCount = context.allTables().size(); int lakeTableCount = context.getLakeTableCount(); @@ -141,6 +142,7 @@ private void updateMetricsViaAccessContext() { } return new MetricsData( + coordinatorServerCount, tabletServerCount, tableCount, lakeTableCount, @@ -155,6 +157,7 @@ private void updateMetricsViaAccessContext() { // Wait for the result and update local metrics try { MetricsData metricsData = accessContextEvent.getResultFuture().get(); + this.aliveCoordinatorServerCount = metricsData.coordinatorServerCount; this.tabletServerCount = metricsData.tabletServerCount; this.tableCount = metricsData.tableCount; this.lakeTableCount = metricsData.lakeTableCount; @@ -278,6 +281,7 @@ public QueuedEvent(CoordinatorEvent event, long enqueueTimeMs) { } private static class MetricsData { + private final int coordinatorServerCount; private final int tabletServerCount; private final int tableCount; private final int lakeTableCount; @@ -287,6 +291,7 @@ private static class MetricsData { private final int replicasToDeleteCount; public MetricsData( + int coordinatorServerCount, int tabletServerCount, int tableCount, int lakeTableCount, @@ -294,6 +299,7 @@ public MetricsData( int partitionCount, int offlineBucketCount, int replicasToDeleteCount) { + this.coordinatorServerCount = coordinatorServerCount; this.tabletServerCount = tabletServerCount; this.tableCount = tableCount; this.lakeTableCount = lakeTableCount; diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachine.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachine.java index 091c2cbc9c..fd7b731330 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachine.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachine.java @@ -487,7 +487,8 @@ private Map doRemoveReplicaFromIsr( toUpdateLeaderAndIsrList.put(tableBucket, adjustLeaderAndIsr); } try { - zooKeeperClient.batchUpdateLeaderAndIsr(toUpdateLeaderAndIsrList); + zooKeeperClient.batchUpdateLeaderAndIsr( + toUpdateLeaderAndIsrList, coordinatorContext.getCoordinatorEpoch()); toUpdateLeaderAndIsrList.forEach(coordinatorContext::putBucketLeaderAndIsr); return adjustedLeaderAndIsr; } catch (Exception e) { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachine.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachine.java index 85dcc434f4..5d1d5b31c6 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachine.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachine.java @@ -496,7 +496,10 @@ private Optional electNewLeaderForTableBuckets( } ElectionResult electionResult = optionalElectionResult.get(); try { - zooKeeperClient.updateLeaderAndIsr(tableBucket, electionResult.leaderAndIsr); + zooKeeperClient.updateLeaderAndIsr( + tableBucket, + electionResult.leaderAndIsr, + coordinatorContext.getCoordinatorEpoch()); } catch (Exception e) { LOG.error( "Fail to update bucket LeaderAndIsr for table bucket {}.", diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java index 05b7e0224c..c3fbcc3389 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java @@ -21,6 +21,7 @@ import org.apache.fluss.annotation.VisibleForTesting; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; +import org.apache.fluss.exception.InvalidCoordinatorException; import org.apache.fluss.metadata.DatabaseSummary; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.ResolvedPartitionSpec; @@ -192,13 +193,9 @@ public Optional fenceBecomeCoordinatorLeader(int coordinatorId) throws ensureEpochZnodeExists(); try { - Stat currentStat = new Stat(); - byte[] bytes = - zkClient.getData() - .storingStatIn(currentStat) - .forPath(ZkData.CoordinatorEpochZNode.path()); - int currentEpoch = ZkData.CoordinatorEpochZNode.decode(bytes); - int currentVersion = currentStat.getVersion(); + Tuple2 getEpoch = getCurrentEpoch(); + int currentEpoch = getEpoch.f0; + int currentVersion = getEpoch.f1; int newEpoch = currentEpoch + 1; LOG.info( "Coordinator leader {} tries to update epoch. Current epoch={}, Zookeeper version={}, new epoch={}", @@ -240,7 +237,7 @@ public Optional getCoordinatorLeaderAddress() throws Excepti Optional bytes = getOrEmpty(ZkData.CoordinatorLeaderZNode.path()); return bytes.map( data -> - // maybe a empty node when a leader is elected but not registered + // maybe an empty node when a leader is elected but not registered data.length == 0 ? null : ZkData.CoordinatorLeaderZNode.decode(data)); } @@ -261,12 +258,24 @@ public void ensureEpochZnodeExists() throws Exception { .forPath( path, ZkData.CoordinatorEpochZNode.encode( - CoordinatorContext.INITIAL_COORDINATOR_EPOCH)); + CoordinatorContext.INITIAL_COORDINATOR_EPOCH - 1)); } catch (KeeperException.NodeExistsException e) { } } } + /** Get epoch now in ZK. */ + public Tuple2 getCurrentEpoch() throws Exception { + Stat currentStat = new Stat(); + byte[] bytes = + zkClient.getData() + .storingStatIn(currentStat) + .forPath(ZkData.CoordinatorEpochZNode.path()); + int currentEpoch = ZkData.CoordinatorEpochZNode.decode(bytes); + int currentVersion = currentStat.getVersion(); + return new Tuple2<>(currentEpoch, currentVersion); + } + // -------------------------------------------------------------------------------------------- // Tablet server // -------------------------------------------------------------------------------------------- @@ -493,14 +502,25 @@ public Map getLeaderAndIsrs(Collection t "leader and isr"); } - public void updateLeaderAndIsr(TableBucket tableBucket, LeaderAndIsr leaderAndIsr) + public void updateLeaderAndIsr( + TableBucket tableBucket, LeaderAndIsr leaderAndIsr, int currentCoordinatorEpoch) throws Exception { + // check coordinator epoch to ensure no other Coordinator leader exists. + if (leaderAndIsr.coordinatorEpoch() != currentCoordinatorEpoch) { + throw new InvalidCoordinatorException( + String.format( + "LeaderAndIsr coordinator epoch %d does not match current coordinator epoch %d for bucket %s. " + + "This coordinator may no longer be the leader.", + leaderAndIsr.coordinatorEpoch(), currentCoordinatorEpoch, tableBucket)); + } + String path = LeaderAndIsrZNode.path(tableBucket); zkClient.setData().forPath(path, LeaderAndIsrZNode.encode(leaderAndIsr)); LOG.info("Updated {} for bucket {} in Zookeeper.", leaderAndIsr, tableBucket); } - public void batchUpdateLeaderAndIsr(Map leaderAndIsrList) + public void batchUpdateLeaderAndIsr( + Map leaderAndIsrList, int currentCoordinatorEpoch) throws Exception { if (leaderAndIsrList.isEmpty()) { return; @@ -511,6 +531,17 @@ public void batchUpdateLeaderAndIsr(Map leaderAndIsrL TableBucket tableBucket = entry.getKey(); LeaderAndIsr leaderAndIsr = entry.getValue(); + // check coordinator epoch to ensure no other Coordinator leader exists. + if (leaderAndIsr.coordinatorEpoch() != currentCoordinatorEpoch) { + throw new InvalidCoordinatorException( + String.format( + "LeaderAndIsr coordinator epoch %d does not match current coordinator epoch %d for bucket %s. " + + "This coordinator may no longer be the leader.", + leaderAndIsr.coordinatorEpoch(), + currentCoordinatorEpoch, + tableBucket)); + } + LOG.info("Batch Update {} for bucket {} in Zookeeper.", leaderAndIsr, tableBucket); String path = LeaderAndIsrZNode.path(tableBucket); byte[] data = LeaderAndIsrZNode.encode(leaderAndIsr); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java index 99031e95ff..032455c688 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java @@ -82,8 +82,7 @@ void testCoordinatorServerElection() throws Exception { CoordinatorAddress firstLeaderAddress = zookeeperClient.getCoordinatorLeaderAddress().get(); - // Find the leader - // and try to close it. + // Find the leader and try to close it. CoordinatorServer elected = null; for (CoordinatorServer coordinatorServer : coordinatorServerList) { if (coordinatorServer.getServerId() == firstLeaderAddress.getId()) { @@ -92,6 +91,8 @@ void testCoordinatorServerElection() throws Exception { } } assertThat(elected).isNotNull(); + assertThat(zookeeperClient.getCurrentEpoch().f0) + .isEqualTo(CoordinatorContext.INITIAL_COORDINATOR_EPOCH); elected.close(); elected.start(); @@ -100,6 +101,8 @@ void testCoordinatorServerElection() throws Exception { CoordinatorAddress secondLeaderAddress = zookeeperClient.getCoordinatorLeaderAddress().get(); assertThat(secondLeaderAddress).isNotEqualTo(firstLeaderAddress); + assertThat(zookeeperClient.getCurrentEpoch().f0) + .isEqualTo(CoordinatorContext.INITIAL_COORDINATOR_EPOCH + 1); // kill other 2 coordinator servers for (CoordinatorServer coordinatorServer : coordinatorServerList) { @@ -112,6 +115,8 @@ void testCoordinatorServerElection() throws Exception { CoordinatorAddress thirdLeaderAddress = zookeeperClient.getCoordinatorLeaderAddress().get(); assertThat(thirdLeaderAddress.getId()).isEqualTo(firstLeaderAddress.getId()); + assertThat(zookeeperClient.getCurrentEpoch().f0) + .isEqualTo(CoordinatorContext.INITIAL_COORDINATOR_EPOCH + 2); } /** Create a configuration with Zookeeper address setting. */ diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java index dbe7dfb710..7bf32dae67 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java @@ -77,9 +77,7 @@ protected void checkAfterStartServer() throws Exception { public void waitUtilCoordinatorServerElected() { waitUntil( - () -> { - return zookeeperClient.getCoordinatorLeaderAddress().isPresent(); - }, + () -> zookeeperClient.getCoordinatorLeaderAddress().isPresent(), Duration.ofSeconds(10), "Fail to wait coordinator server elected"); } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java b/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java index d3393de3f2..57b3867ff7 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java @@ -941,9 +941,7 @@ public CoordinatorServer getCoordinatorServer() { public void waitUtilCoordinatorServerElected() { waitUntil( - () -> { - return zooKeeperClient.getCoordinatorLeaderAddress().isPresent(); - }, + () -> zooKeeperClient.getCoordinatorLeaderAddress().isPresent(), Duration.ofSeconds(10), "Fail to wait coordinator server elected"); } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java index eb97a1145e..df03152463 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java @@ -30,6 +30,7 @@ import org.apache.fluss.metadata.TableDescriptor; import org.apache.fluss.metadata.TablePartition; import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.server.coordinator.CoordinatorContext; import org.apache.fluss.server.entity.RegisterTableBucketLeadAndIsrInfo; import org.apache.fluss.server.zk.data.BucketAssignment; import org.apache.fluss.server.zk.data.BucketSnapshot; @@ -189,8 +190,8 @@ void testLeaderAndIsr() throws Exception { assertThat(zookeeperClient.getLeaderAndIsr(tableBucket2)).isEmpty(); // try to register bucket leaderAndIsr - LeaderAndIsr leaderAndIsr1 = new LeaderAndIsr(1, 10, Arrays.asList(1, 2, 3), 100, 1000); - LeaderAndIsr leaderAndIsr2 = new LeaderAndIsr(2, 10, Arrays.asList(4, 5, 6), 100, 1000); + LeaderAndIsr leaderAndIsr1 = new LeaderAndIsr(1, 10, Arrays.asList(1, 2, 3), 0, 1000); + LeaderAndIsr leaderAndIsr2 = new LeaderAndIsr(2, 10, Arrays.asList(4, 5, 6), 0, 1000); zookeeperClient.registerLeaderAndIsr(tableBucket1, leaderAndIsr1); zookeeperClient.registerLeaderAndIsr(tableBucket2, leaderAndIsr2); @@ -200,8 +201,9 @@ void testLeaderAndIsr() throws Exception { .containsValues(leaderAndIsr1, leaderAndIsr2); // test update - leaderAndIsr1 = new LeaderAndIsr(2, 20, Collections.emptyList(), 200, 2000); - zookeeperClient.updateLeaderAndIsr(tableBucket1, leaderAndIsr1); + leaderAndIsr1 = new LeaderAndIsr(2, 20, Collections.emptyList(), 0, 2000); + zookeeperClient.updateLeaderAndIsr( + tableBucket1, leaderAndIsr1, CoordinatorContext.INITIAL_COORDINATOR_EPOCH); assertThat(zookeeperClient.getLeaderAndIsr(tableBucket1)).hasValue(leaderAndIsr1); // test delete @@ -218,7 +220,7 @@ void testBatchCreateAndUpdateLeaderAndIsr(boolean isPartitionTable) throws Excep TableBucket tableBucket = isPartitionTable ? new TableBucket(1, 2L, i) : new TableBucket(1, i); LeaderAndIsr leaderAndIsr = - new LeaderAndIsr(i, 10, Arrays.asList(i + 1, i + 2, i + 3), 100, 1000); + new LeaderAndIsr(i, 10, Arrays.asList(i + 1, i + 2, i + 3), 0, 1000); leaderAndIsrList.add(leaderAndIsr); RegisterTableBucketLeadAndIsrInfo info = isPartitionTable @@ -259,7 +261,8 @@ void testBatchCreateAndUpdateLeaderAndIsr(boolean isPartitionTable) throws Excep entry.setValue(adjustLeaderAndIsr); }); // batch update - zookeeperClient.batchUpdateLeaderAndIsr(updateMap); + zookeeperClient.batchUpdateLeaderAndIsr( + updateMap, CoordinatorContext.INITIAL_COORDINATOR_EPOCH); for (int i = 0; i < 100; i++) { // each should update successful Optional optionalLeaderAndIsr = @@ -278,7 +281,7 @@ void testBatchUpdateLeaderAndIsr() throws Exception { for (int i = 0; i < totalCount; i++) { TableBucket tableBucket = new TableBucket(1, i); LeaderAndIsr leaderAndIsr = - new LeaderAndIsr(i, 10, Arrays.asList(i + 1, i + 2, i + 3), 100, 1000); + new LeaderAndIsr(i, 10, Arrays.asList(i + 1, i + 2, i + 3), 0, 1000); leaderAndIsrList.put(tableBucket, leaderAndIsr); zookeeperClient.registerLeaderAndIsr(tableBucket, leaderAndIsr); } @@ -295,10 +298,11 @@ void testBatchUpdateLeaderAndIsr() throws Exception { old.leader() + 1, old.leaderEpoch() + 1, old.isr(), - old.coordinatorEpoch() + 1, + old.coordinatorEpoch(), old.bucketEpoch() + 1); })); - zookeeperClient.batchUpdateLeaderAndIsr(updateLeaderAndIsrList); + zookeeperClient.batchUpdateLeaderAndIsr( + updateLeaderAndIsrList, CoordinatorContext.INITIAL_COORDINATOR_EPOCH); for (Map.Entry entry : updateLeaderAndIsrList.entrySet()) { TableBucket tableBucket = entry.getKey(); LeaderAndIsr leaderAndIsr = entry.getValue(); From e3350835680ee8db74aa0ad96e00296cce2c4b26 Mon Sep 17 00:00:00 2001 From: "whenzhou.zc" Date: Tue, 28 Oct 2025 09:37:17 +0800 Subject: [PATCH 17/26] support coordinator epoch2 --- .../CoordinatorEventProcessor.java | 6 +- .../statemachine/ReplicaStateMachine.java | 2 +- .../statemachine/TableBucketStateMachine.java | 12 +- .../fluss/server/zk/ZooKeeperClient.java | 106 ++++++++++++------ .../apache/fluss/server/zk/ZooKeeperOp.java | 60 ++++++++++ .../fluss/server/zk/data/ZkVersion.java | 35 ++++++ .../CoordinatorEventProcessorTest.java | 2 +- .../coordinator/CoordinatorServerTest.java | 4 +- .../coordinator/TestCoordinatorContext.java | 39 +++++++ .../statemachine/ReplicaStateMachineTest.java | 18 +-- .../TableBucketStateMachineTest.java | 14 ++- .../metadata/ZkBasedMetadataProviderTest.java | 25 +++-- .../testutils/FlussClusterExtension.java | 4 +- .../fluss/server/zk/ZooKeeperClientTest.java | 17 +-- 14 files changed, 277 insertions(+), 67 deletions(-) create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperOp.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkVersion.java create mode 100644 fluss-server/src/test/java/org/apache/fluss/server/coordinator/TestCoordinatorContext.java diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java index b85156983c..b43a7ec5f3 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java @@ -1661,7 +1661,7 @@ private List tryProcessAdjustIsr( try { zooKeeperClient.batchUpdateLeaderAndIsr( - newLeaderAndIsrList, coordinatorContext.getCoordinatorEpoch()); + newLeaderAndIsrList, coordinatorContext.getCoordinatorEpochZkVersion()); newLeaderAndIsrList.forEach( (tableBucket, newLeaderAndIsr) -> result.add(new AdjustIsrResultForBucket(tableBucket, newLeaderAndIsr))); @@ -1673,7 +1673,9 @@ private List tryProcessAdjustIsr( LeaderAndIsr newLeaderAndIsr = entry.getValue(); try { zooKeeperClient.updateLeaderAndIsr( - tableBucket, newLeaderAndIsr, coordinatorContext.getCoordinatorEpoch()); + tableBucket, + newLeaderAndIsr, + coordinatorContext.getCoordinatorEpochZkVersion()); } catch (Exception e) { LOG.error("Error when register leader and isr.", e); result.add( diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachine.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachine.java index fd7b731330..2416b163b5 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachine.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachine.java @@ -488,7 +488,7 @@ private Map doRemoveReplicaFromIsr( } try { zooKeeperClient.batchUpdateLeaderAndIsr( - toUpdateLeaderAndIsrList, coordinatorContext.getCoordinatorEpoch()); + toUpdateLeaderAndIsrList, coordinatorContext.getCoordinatorEpochZkVersion()); toUpdateLeaderAndIsrList.forEach(coordinatorContext::putBucketLeaderAndIsr); return adjustedLeaderAndIsr; } catch (Exception e) { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachine.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachine.java index 5d1d5b31c6..561cd527bc 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachine.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachine.java @@ -311,7 +311,10 @@ private Optional initLeaderForTableBuckets( ElectionResult electionResult = optionalElectionResult.get(); LeaderAndIsr leaderAndIsr = electionResult.leaderAndIsr; try { - zooKeeperClient.registerLeaderAndIsr(tableBucket, leaderAndIsr); + zooKeeperClient.registerLeaderAndIsr( + tableBucket, + leaderAndIsr, + coordinatorContext.getCoordinatorEpochZkVersion()); } catch (Exception e) { LOG.error( "Fail to create state node for table bucket {} in zookeeper.", @@ -454,7 +457,10 @@ private List tryRegisterLeaderAndIsrOneByOne( List registerSuccessList = new ArrayList<>(); for (RegisterTableBucketLeadAndIsrInfo info : registerList) { try { - zooKeeperClient.registerLeaderAndIsr(info.getTableBucket(), info.getLeaderAndIsr()); + zooKeeperClient.registerLeaderAndIsr( + info.getTableBucket(), + info.getLeaderAndIsr(), + coordinatorContext.getCoordinatorEpochZkVersion()); registerSuccessList.add(info); } catch (Exception e) { LOG.error( @@ -499,7 +505,7 @@ private Optional electNewLeaderForTableBuckets( zooKeeperClient.updateLeaderAndIsr( tableBucket, electionResult.leaderAndIsr, - coordinatorContext.getCoordinatorEpoch()); + coordinatorContext.getCoordinatorEpochZkVersion()); } catch (Exception e) { LOG.error( "Fail to update bucket LeaderAndIsr for table bucket {}.", diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java index c3fbcc3389..14ea248a98 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java @@ -84,6 +84,7 @@ import org.apache.fluss.server.zk.data.ZkData.TableZNode; import org.apache.fluss.server.zk.data.ZkData.TablesZNode; import org.apache.fluss.server.zk.data.ZkData.WriterIdZNode; +import org.apache.fluss.server.zk.data.ZkVersion; import org.apache.fluss.server.zk.data.lake.LakeTable; import org.apache.fluss.server.zk.data.lake.LakeTableSnapshot; import org.apache.fluss.server.zk.data.producer.ProducerOffsets; @@ -124,6 +125,7 @@ import static java.util.stream.Collectors.toMap; import static org.apache.fluss.metadata.ResolvedPartitionSpec.fromPartitionName; +import static org.apache.fluss.server.zk.ZooKeeperOp.multiRequest; import static org.apache.fluss.utils.Preconditions.checkNotNull; /** @@ -141,6 +143,7 @@ public class ZooKeeperClient implements AutoCloseable { private final CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper; private final CuratorFramework zkClient; + private final ZooKeeperOp zkOp; private final ZkSequenceIDCounter tableIdCounter; private final ZkSequenceIDCounter partitionIdCounter; private final ZkSequenceIDCounter writerIdCounter; @@ -153,6 +156,7 @@ public ZooKeeperClient( Configuration configuration) { this.curatorFrameworkWrapper = curatorFrameworkWrapper; this.zkClient = curatorFrameworkWrapper.asCuratorFramework(); + this.zkOp = new ZooKeeperOp(zkClient); this.tableIdCounter = new ZkSequenceIDCounter(zkClient, TableSequenceIdZNode.path()); this.partitionIdCounter = new ZkSequenceIDCounter(zkClient, PartitionSequenceIdZNode.path()); @@ -418,13 +422,14 @@ public void deletePartitionAssignment(long partitionId) throws Exception { // -------------------------------------------------------------------------------------------- /** Register bucket LeaderAndIsr to ZK. */ - public void registerLeaderAndIsr(TableBucket tableBucket, LeaderAndIsr leaderAndIsr) + public void registerLeaderAndIsr( + TableBucket tableBucket, LeaderAndIsr leaderAndIsr, int expectedZkVersion) throws Exception { + String path = LeaderAndIsrZNode.path(tableBucket); - zkClient.create() - .creatingParentsIfNeeded() - .withMode(CreateMode.PERSISTENT) - .forPath(path, LeaderAndIsrZNode.encode(leaderAndIsr)); + byte[] data = LeaderAndIsrZNode.encode(leaderAndIsr); + + createRecursive(path, data, expectedZkVersion, false); LOG.info("Registered {} for bucket {} in Zookeeper.", leaderAndIsr, tableBucket); } @@ -503,24 +508,20 @@ public Map getLeaderAndIsrs(Collection t } public void updateLeaderAndIsr( - TableBucket tableBucket, LeaderAndIsr leaderAndIsr, int currentCoordinatorEpoch) + TableBucket tableBucket, LeaderAndIsr leaderAndIsr, int expectedZkVersion) throws Exception { - // check coordinator epoch to ensure no other Coordinator leader exists. - if (leaderAndIsr.coordinatorEpoch() != currentCoordinatorEpoch) { - throw new InvalidCoordinatorException( - String.format( - "LeaderAndIsr coordinator epoch %d does not match current coordinator epoch %d for bucket %s. " - + "This coordinator may no longer be the leader.", - leaderAndIsr.coordinatorEpoch(), currentCoordinatorEpoch, tableBucket)); - } - String path = LeaderAndIsrZNode.path(tableBucket); - zkClient.setData().forPath(path, LeaderAndIsrZNode.encode(leaderAndIsr)); + byte[] data = LeaderAndIsrZNode.encode(leaderAndIsr); + + CuratorOp updateOp = zkOp.updateOp(path, data); + List ops = wrapRequestWithCoordinatorEpochCheck(updateOp, expectedZkVersion); + + zkClient.transaction().forOperations(ops); LOG.info("Updated {} for bucket {} in Zookeeper.", leaderAndIsr, tableBucket); } public void batchUpdateLeaderAndIsr( - Map leaderAndIsrList, int currentCoordinatorEpoch) + Map leaderAndIsrList, int expectedZkVersion) throws Exception { if (leaderAndIsrList.isEmpty()) { return; @@ -530,30 +531,21 @@ public void batchUpdateLeaderAndIsr( for (Map.Entry entry : leaderAndIsrList.entrySet()) { TableBucket tableBucket = entry.getKey(); LeaderAndIsr leaderAndIsr = entry.getValue(); - - // check coordinator epoch to ensure no other Coordinator leader exists. - if (leaderAndIsr.coordinatorEpoch() != currentCoordinatorEpoch) { - throw new InvalidCoordinatorException( - String.format( - "LeaderAndIsr coordinator epoch %d does not match current coordinator epoch %d for bucket %s. " - + "This coordinator may no longer be the leader.", - leaderAndIsr.coordinatorEpoch(), - currentCoordinatorEpoch, - tableBucket)); - } - LOG.info("Batch Update {} for bucket {} in Zookeeper.", leaderAndIsr, tableBucket); String path = LeaderAndIsrZNode.path(tableBucket); byte[] data = LeaderAndIsrZNode.encode(leaderAndIsr); CuratorOp updateOp = zkClient.transactionOp().setData().forPath(path, data); ops.add(updateOp); - if (ops.size() == MAX_BATCH_SIZE) { - zkClient.transaction().forOperations(ops); + if (ops.size() == MAX_BATCH_SIZE - 1) { + List wrapOps = + wrapRequestsWithCoordinatorEpochCheck(ops, expectedZkVersion); + zkClient.transaction().forOperations(wrapOps); ops.clear(); } } if (!ops.isEmpty()) { - zkClient.transaction().forOperations(ops); + List wrapOps = wrapRequestsWithCoordinatorEpochCheck(ops, expectedZkVersion); + zkClient.transaction().forOperations(wrapOps); } } @@ -1749,6 +1741,56 @@ public static Map> processGetChildrenResponses( return result; } + public void createRecursive( + String path, byte[] data, int expectedZkVersion, boolean throwIfPathExists) + throws Exception { + CuratorOp createOp = zkOp.createOp(path, data, CreateMode.PERSISTENT); + List ops = wrapRequestWithCoordinatorEpochCheck(createOp, expectedZkVersion); + + try { + // try to directly create + zkClient.transaction().forOperations(ops); + } catch (KeeperException.NodeExistsException e) { + // should not exist + if (throwIfPathExists) { + throw e; + } + } catch (KeeperException.NoNodeException e) { + // if parent does not exist, create parent first + int indexOfLastSlash = path.lastIndexOf("/"); + if (indexOfLastSlash == -1) { + throw new IllegalArgumentException("Invalid path {}" + path); + } + String parentPath = path.substring(0, indexOfLastSlash); + createRecursive(parentPath, null, expectedZkVersion, throwIfPathExists); + // After creating parent, retry creating the original path + zkClient.transaction().forOperations(ops); + } + } + + public List wrapRequestWithCoordinatorEpochCheck( + CuratorOp request, int expectedZkVersion) throws Exception { + return wrapRequestsWithCoordinatorEpochCheck( + Collections.singletonList(request), expectedZkVersion); + } + + public List wrapRequestsWithCoordinatorEpochCheck( + List requestList, int expectedZkVersion) throws Exception { + if (ZkVersion.MATCH_ANY_VERSION.getVersion() == expectedZkVersion) { + return requestList; + } else if (expectedZkVersion >= 0) { + CuratorOp checkOp = + zkOp.checkOp(ZkData.CoordinatorEpochZNode.path(), expectedZkVersion); + return multiRequest(checkOp, requestList); + } else { + throw new IllegalArgumentException( + "Expected coordinator epoch zkVersion " + + expectedZkVersion + + " should be non-negative or equal to " + + ZkVersion.MATCH_ANY_VERSION.getVersion()); + } + } + // -------------------------------------------------------------------------------------------- // Producer Offset Snapshot // -------------------------------------------------------------------------------------------- diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperOp.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperOp.java new file mode 100644 index 0000000000..7dc171de63 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperOp.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.zk; + +import org.apache.fluss.shaded.curator5.org.apache.curator.framework.CuratorFramework; +import org.apache.fluss.shaded.curator5.org.apache.curator.framework.api.transaction.CuratorOp; +import org.apache.fluss.shaded.zookeeper3.org.apache.zookeeper.CreateMode; + +import java.util.ArrayList; +import java.util.List; + +/** This class contains some utility methods for wrap/unwrap operations for Zookeeper. */ +public class ZooKeeperOp { + private final CuratorFramework zkClient; + + public ZooKeeperOp(CuratorFramework zkClient) { + this.zkClient = zkClient; + } + + public CuratorOp checkOp(String path, int expectZkVersion) throws Exception { + return zkClient.transactionOp().check().withVersion(expectZkVersion).forPath(path); + } + + public CuratorOp createOp(String path, byte[] data, CreateMode createMode) throws Exception { + return zkClient.transactionOp().create().withMode(createMode).forPath(path, data); + } + + public CuratorOp updateOp(String path, byte[] data) throws Exception { + return zkClient.transactionOp().setData().forPath(path, data); + } + + public static List multiRequest(CuratorOp op1, CuratorOp op2) { + List ops = new ArrayList<>(); + ops.add(op1); + ops.add(op2); + return ops; + } + + public static List multiRequest(CuratorOp op, List ops) { + List result = new ArrayList<>(); + result.add(op); + result.addAll(ops); + return result; + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkVersion.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkVersion.java new file mode 100644 index 0000000000..26242df0f3 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkVersion.java @@ -0,0 +1,35 @@ +/* + * 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; + +/** Enum to represent the type of special Zookeeper version. */ +public enum ZkVersion { + MATCH_ANY_VERSION(-1), + UNKNOWN_VERSION(-2); + + private final int version; + + ZkVersion(int version) { + this.version = version; + } + + public int getVersion() { + return version; + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java index 6b57633e7f..1c916a728d 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java @@ -1045,7 +1045,7 @@ private CoordinatorEventProcessor buildCoordinatorEventProcessor() { zookeeperClient, serverMetadataCache, testCoordinatorChannelManager, - new CoordinatorContext(), + new TestCoordinatorContext(), autoPartitionManager, lakeTableTieringManager, TestingMetricGroups.COORDINATOR_METRICS, diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java index 7bf32dae67..76f0e8a582 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java @@ -40,7 +40,7 @@ class CoordinatorServerTest extends ServerTestBase { @BeforeEach void beforeEach() throws Exception { coordinatorServer = startCoordinatorServer(createConfiguration()); - waitUtilCoordinatorServerElected(); + waitUntilCoordinatorServerElected(); } @AfterEach @@ -75,7 +75,7 @@ protected void checkAfterStartServer() throws Exception { coordinatorServer.getRpcServer().getBindEndpoints()); } - public void waitUtilCoordinatorServerElected() { + public void waitUntilCoordinatorServerElected() { waitUntil( () -> zookeeperClient.getCoordinatorLeaderAddress().isPresent(), Duration.ofSeconds(10), diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TestCoordinatorContext.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TestCoordinatorContext.java new file mode 100644 index 0000000000..b931d564ff --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TestCoordinatorContext.java @@ -0,0 +1,39 @@ +/* + * 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.server.zk.data.ZkVersion; + +/** A coordinator context for test purpose which can set epoch manually. */ +public class TestCoordinatorContext extends CoordinatorContext { + public TestCoordinatorContext() { + // When create or modify ZooKeeper node, it should check ZooKeeper epoch node version + // to ensure the coordinator is still holding the leadership. However, in the test + // cases, we don't register epoch node, so we skip the check process by setting + // "coordinatorEpochZkVersion" to ZkVersion.MATCH_ANY_VERSION + super(); + this.setCoordinatorEpochAndZkVersion( + INITIAL_COORDINATOR_EPOCH, ZkVersion.MATCH_ANY_VERSION.getVersion()); + } + + public TestCoordinatorContext(int coordinatorEpoch, int coordinatorEpochZkVersion) { + super(); + this.setCoordinatorEpochAndZkVersion(coordinatorEpoch, coordinatorEpochZkVersion); + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachineTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachineTest.java index 454ec5de4b..51f2ba4948 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachineTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachineTest.java @@ -30,6 +30,7 @@ import org.apache.fluss.server.coordinator.CoordinatorRequestBatch; import org.apache.fluss.server.coordinator.CoordinatorTestUtils; import org.apache.fluss.server.coordinator.TestCoordinatorChannelManager; +import org.apache.fluss.server.coordinator.TestCoordinatorContext; import org.apache.fluss.server.coordinator.event.DeleteReplicaResponseReceivedEvent; import org.apache.fluss.server.entity.DeleteReplicaResultForBucket; import org.apache.fluss.server.metadata.ServerInfo; @@ -37,6 +38,7 @@ import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.server.zk.ZooKeeperExtension; import org.apache.fluss.server.zk.data.LeaderAndIsr; +import org.apache.fluss.server.zk.data.ZkVersion; import org.apache.fluss.testutils.common.AllCallbackWrapper; import org.junit.jupiter.api.BeforeAll; @@ -79,7 +81,7 @@ static void baseBeforeAll() { @Test void testStartup() { - CoordinatorContext coordinatorContext = new CoordinatorContext(); + CoordinatorContext coordinatorContext = new TestCoordinatorContext(); // init coordinator server context with a table assignment TableBucket tableBucket = new TableBucket(1, 0); @@ -104,7 +106,7 @@ void testStartup() { @Test void testReplicaStateChange() { - CoordinatorContext coordinatorContext = new CoordinatorContext(); + CoordinatorContext coordinatorContext = new TestCoordinatorContext(); ReplicaStateMachine replicaStateMachine = createReplicaStateMachine(coordinatorContext); // test check valid replica state change @@ -133,7 +135,7 @@ void testReplicaStateChange() { @Test void testDeleteReplicaStateChange() { Map isReplicaDeleteSuccess = new HashMap<>(); - CoordinatorContext coordinatorContext = new CoordinatorContext(); + CoordinatorContext coordinatorContext = new TestCoordinatorContext(); coordinatorContext.setLiveTabletServers( CoordinatorTestUtils.createServers(Arrays.asList(0, 1))); // use a context that will return a gateway that always get success ack @@ -190,7 +192,7 @@ void testDeleteReplicaStateChange() { @Test void testOfflineReplicasShouldBeRemovedFromIsr() throws Exception { - CoordinatorContext coordinatorContext = new CoordinatorContext(); + CoordinatorContext coordinatorContext = new TestCoordinatorContext(); coordinatorContext.setLiveTabletServers(createServers(new int[] {0, 1, 2})); ReplicaStateMachine replicaStateMachine = createReplicaStateMachine(coordinatorContext); @@ -214,7 +216,8 @@ void testOfflineReplicasShouldBeRemovedFromIsr() throws Exception { } // put leader and isr LeaderAndIsr leaderAndIsr = new LeaderAndIsr(0, 0, Arrays.asList(0, 1, 2), 0, 0); - zookeeperClient.registerLeaderAndIsr(tableBucket, leaderAndIsr); + zookeeperClient.registerLeaderAndIsr( + tableBucket, leaderAndIsr, ZkVersion.MATCH_ANY_VERSION.getVersion()); coordinatorContext.updateBucketReplicaAssignment(tableBucket, Arrays.asList(0, 1, 2)); coordinatorContext.putBucketLeaderAndIsr(tableBucket, leaderAndIsr); @@ -228,7 +231,7 @@ void testOfflineReplicasShouldBeRemovedFromIsr() throws Exception { @Test void testOfflineReplicaShouldBeRemovedFromIsr() throws Exception { - CoordinatorContext coordinatorContext = new CoordinatorContext(); + CoordinatorContext coordinatorContext = new TestCoordinatorContext(); coordinatorContext.setLiveTabletServers(createServers(new int[] {0, 1, 2})); ReplicaStateMachine replicaStateMachine = createReplicaStateMachine(coordinatorContext); @@ -250,7 +253,8 @@ void testOfflineReplicaShouldBeRemovedFromIsr() throws Exception { } // put leader and isr LeaderAndIsr leaderAndIsr = new LeaderAndIsr(0, 0, Arrays.asList(0, 1, 2), 0, 0); - zookeeperClient.registerLeaderAndIsr(tableBucket, leaderAndIsr); + zookeeperClient.registerLeaderAndIsr( + tableBucket, leaderAndIsr, ZkVersion.MATCH_ANY_VERSION.getVersion()); coordinatorContext.updateBucketReplicaAssignment(tableBucket, Arrays.asList(0, 1, 2)); coordinatorContext.putBucketLeaderAndIsr(tableBucket, leaderAndIsr); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachineTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachineTest.java index bafb477c54..dc11450bf9 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachineTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachineTest.java @@ -34,6 +34,7 @@ import org.apache.fluss.server.coordinator.LakeTableTieringManager; import org.apache.fluss.server.coordinator.MetadataManager; import org.apache.fluss.server.coordinator.TestCoordinatorChannelManager; +import org.apache.fluss.server.coordinator.TestCoordinatorContext; import org.apache.fluss.server.coordinator.event.CoordinatorEventManager; import org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElection.ControlledShutdownLeaderElection; import org.apache.fluss.server.coordinator.statemachine.TableBucketStateMachine.ElectionResult; @@ -43,6 +44,7 @@ import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.server.zk.ZooKeeperExtension; import org.apache.fluss.server.zk.data.LeaderAndIsr; +import org.apache.fluss.server.zk.data.ZkVersion; import org.apache.fluss.shaded.guava32.com.google.common.collect.Sets; import org.apache.fluss.testutils.common.AllCallbackWrapper; import org.apache.fluss.utils.concurrent.ExecutorThreadFactory; @@ -100,7 +102,7 @@ void beforeEach() throws IOException { Configuration conf = new Configuration(); conf.setString(ConfigOptions.COORDINATOR_HOST, "localhost"); conf.setString(ConfigOptions.REMOTE_DATA_DIR, "/tmp/fluss/remote-data"); - coordinatorContext = new CoordinatorContext(); + coordinatorContext = new TestCoordinatorContext(); testCoordinatorChannelManager = new TestCoordinatorChannelManager(); coordinatorRequestBatch = new CoordinatorRequestBatch( @@ -142,9 +144,13 @@ void testStartup() throws Exception { // create LeaderAndIsr for t10/t11 info in zk, zookeeperClient.registerLeaderAndIsr( - new TableBucket(t1Id, 0), new LeaderAndIsr(0, 0, Arrays.asList(0, 1), 0, 0)); + new TableBucket(t1Id, 0), + new LeaderAndIsr(0, 0, Arrays.asList(0, 1), 0, 0), + ZkVersion.MATCH_ANY_VERSION.getVersion()); zookeeperClient.registerLeaderAndIsr( - new TableBucket(t1Id, 1), new LeaderAndIsr(2, 0, Arrays.asList(2, 3), 0, 0)); + new TableBucket(t1Id, 1), + new LeaderAndIsr(2, 0, Arrays.asList(2, 3), 0, 0), + ZkVersion.MATCH_ANY_VERSION.getVersion()); // update the LeaderAndIsr to context coordinatorContext.putBucketLeaderAndIsr( t1b0, zookeeperClient.getLeaderAndIsr(new TableBucket(t1Id, 0)).get()); @@ -207,6 +213,8 @@ void testStateChangeToOnline() throws Exception { coordinatorContext.putTablePath(tableId, fakeTablePath); coordinatorContext.updateBucketReplicaAssignment(tableBucket, Arrays.asList(0, 1, 2)); coordinatorContext.putBucketState(tableBucket, NewBucket); + coordinatorContext.setCoordinatorEpochAndZkVersion( + 0, ZkVersion.MATCH_ANY_VERSION.getVersion()); // case1: init a new leader for NewBucket to OnlineBucket tableBucketStateMachine.handleStateChange(Collections.singleton(tableBucket), OnlineBucket); // non any alive servers, the state change fail diff --git a/fluss-server/src/test/java/org/apache/fluss/server/metadata/ZkBasedMetadataProviderTest.java b/fluss-server/src/test/java/org/apache/fluss/server/metadata/ZkBasedMetadataProviderTest.java index 1ebb0c3e03..3c7bf8e240 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/metadata/ZkBasedMetadataProviderTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/metadata/ZkBasedMetadataProviderTest.java @@ -36,6 +36,7 @@ import org.apache.fluss.server.zk.data.PartitionAssignment; import org.apache.fluss.server.zk.data.TableAssignment; import org.apache.fluss.server.zk.data.TableRegistration; +import org.apache.fluss.server.zk.data.ZkVersion; import org.apache.fluss.testutils.common.AllCallbackWrapper; import org.junit.jupiter.api.AfterAll; @@ -109,8 +110,10 @@ void testGetTableMetadataFromZk() throws Exception { LeaderAndIsr leaderAndIsr0 = new LeaderAndIsr(1, 10, Arrays.asList(1, 2, 3), 100, 1000); LeaderAndIsr leaderAndIsr1 = new LeaderAndIsr(2, 20, Arrays.asList(2, 3, 4), 200, 2000); - zookeeperClient.registerLeaderAndIsr(tableBucket0, leaderAndIsr0); - zookeeperClient.registerLeaderAndIsr(tableBucket1, leaderAndIsr1); + zookeeperClient.registerLeaderAndIsr( + tableBucket0, leaderAndIsr0, ZkVersion.MATCH_ANY_VERSION.getVersion()); + zookeeperClient.registerLeaderAndIsr( + tableBucket1, leaderAndIsr1, ZkVersion.MATCH_ANY_VERSION.getVersion()); List tablesMetadataFromZK = metadataProvider.getTablesMetadataFromZK( @@ -170,8 +173,10 @@ void testGetPartitionMetadataFromZk() throws Exception { LeaderAndIsr leaderAndIsr0 = new LeaderAndIsr(1, 10, Arrays.asList(1, 2), 100, 1000); LeaderAndIsr leaderAndIsr1 = new LeaderAndIsr(2, 20, Arrays.asList(2, 3), 200, 2000); - zookeeperClient.registerLeaderAndIsr(partitionBucket0, leaderAndIsr0); - zookeeperClient.registerLeaderAndIsr(partitionBucket1, leaderAndIsr1); + zookeeperClient.registerLeaderAndIsr( + partitionBucket0, leaderAndIsr0, ZkVersion.MATCH_ANY_VERSION.getVersion()); + zookeeperClient.registerLeaderAndIsr( + partitionBucket1, leaderAndIsr1, ZkVersion.MATCH_ANY_VERSION.getVersion()); // Test getPartitionMetadataFromZkAsync PhysicalTablePath partitionPath = PhysicalTablePath.of(tablePath, partitionName); @@ -254,11 +259,17 @@ void testBatchGetPartitionMetadataFromZkAsync() throws Exception { TableBucket bucket3 = new TableBucket(tableId2, partitionId3, 0); zookeeperClient.registerLeaderAndIsr( - bucket1, new LeaderAndIsr(1, 10, Arrays.asList(1, 2), 100, 1000)); + bucket1, + new LeaderAndIsr(1, 10, Arrays.asList(1, 2), 100, 1000), + ZkVersion.MATCH_ANY_VERSION.getVersion()); zookeeperClient.registerLeaderAndIsr( - bucket2, new LeaderAndIsr(2, 20, Arrays.asList(2, 3), 200, 2000)); + bucket2, + new LeaderAndIsr(2, 20, Arrays.asList(2, 3), 200, 2000), + ZkVersion.MATCH_ANY_VERSION.getVersion()); zookeeperClient.registerLeaderAndIsr( - bucket3, new LeaderAndIsr(1, 30, Arrays.asList(1, 3), 300, 3000)); + bucket3, + new LeaderAndIsr(1, 30, Arrays.asList(1, 3), 300, 3000), + ZkVersion.MATCH_ANY_VERSION.getVersion()); // Test getPartitionsMetadataFromZK List partitionPaths = diff --git a/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java b/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java index 57b3867ff7..145e89561f 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java @@ -271,7 +271,7 @@ public void startCoordinatorServer() throws Exception { setRemoteDataDir(conf); coordinatorServer = new CoordinatorServer(conf); coordinatorServer.start(); - waitUtilCoordinatorServerElected(); + waitUntilCoordinatorServerElected(); coordinatorServerInfo = // TODO, Currently, we use 0 as coordinator server id. new ServerInfo( @@ -939,7 +939,7 @@ public CoordinatorServer getCoordinatorServer() { return coordinatorServer; } - public void waitUtilCoordinatorServerElected() { + public void waitUntilCoordinatorServerElected() { waitUntil( () -> zooKeeperClient.getCoordinatorLeaderAddress().isPresent(), Duration.ofSeconds(10), diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java index df03152463..42dc766a03 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java @@ -30,7 +30,6 @@ import org.apache.fluss.metadata.TableDescriptor; import org.apache.fluss.metadata.TablePartition; import org.apache.fluss.metadata.TablePath; -import org.apache.fluss.server.coordinator.CoordinatorContext; import org.apache.fluss.server.entity.RegisterTableBucketLeadAndIsrInfo; import org.apache.fluss.server.zk.data.BucketAssignment; import org.apache.fluss.server.zk.data.BucketSnapshot; @@ -42,6 +41,7 @@ import org.apache.fluss.server.zk.data.TableAssignment; import org.apache.fluss.server.zk.data.TableRegistration; import org.apache.fluss.server.zk.data.TabletServerRegistration; +import org.apache.fluss.server.zk.data.ZkVersion; import org.apache.fluss.shaded.curator5.org.apache.curator.CuratorZookeeperClient; import org.apache.fluss.shaded.curator5.org.apache.curator.framework.CuratorFramework; import org.apache.fluss.shaded.zookeeper3.org.apache.zookeeper.KeeperException; @@ -193,8 +193,10 @@ void testLeaderAndIsr() throws Exception { LeaderAndIsr leaderAndIsr1 = new LeaderAndIsr(1, 10, Arrays.asList(1, 2, 3), 0, 1000); LeaderAndIsr leaderAndIsr2 = new LeaderAndIsr(2, 10, Arrays.asList(4, 5, 6), 0, 1000); - zookeeperClient.registerLeaderAndIsr(tableBucket1, leaderAndIsr1); - zookeeperClient.registerLeaderAndIsr(tableBucket2, leaderAndIsr2); + zookeeperClient.registerLeaderAndIsr( + tableBucket1, leaderAndIsr1, ZkVersion.MATCH_ANY_VERSION.getVersion()); + zookeeperClient.registerLeaderAndIsr( + tableBucket2, leaderAndIsr2, ZkVersion.MATCH_ANY_VERSION.getVersion()); assertThat(zookeeperClient.getLeaderAndIsr(tableBucket1)).hasValue(leaderAndIsr1); assertThat(zookeeperClient.getLeaderAndIsr(tableBucket2)).hasValue(leaderAndIsr2); assertThat(zookeeperClient.getLeaderAndIsrs(Arrays.asList(tableBucket1, tableBucket2))) @@ -203,7 +205,7 @@ void testLeaderAndIsr() throws Exception { // test update leaderAndIsr1 = new LeaderAndIsr(2, 20, Collections.emptyList(), 0, 2000); zookeeperClient.updateLeaderAndIsr( - tableBucket1, leaderAndIsr1, CoordinatorContext.INITIAL_COORDINATOR_EPOCH); + tableBucket1, leaderAndIsr1, ZkVersion.MATCH_ANY_VERSION.getVersion()); assertThat(zookeeperClient.getLeaderAndIsr(tableBucket1)).hasValue(leaderAndIsr1); // test delete @@ -262,7 +264,7 @@ void testBatchCreateAndUpdateLeaderAndIsr(boolean isPartitionTable) throws Excep }); // batch update zookeeperClient.batchUpdateLeaderAndIsr( - updateMap, CoordinatorContext.INITIAL_COORDINATOR_EPOCH); + updateMap, ZkVersion.MATCH_ANY_VERSION.getVersion()); for (int i = 0; i < 100; i++) { // each should update successful Optional optionalLeaderAndIsr = @@ -283,7 +285,8 @@ void testBatchUpdateLeaderAndIsr() throws Exception { LeaderAndIsr leaderAndIsr = new LeaderAndIsr(i, 10, Arrays.asList(i + 1, i + 2, i + 3), 0, 1000); leaderAndIsrList.put(tableBucket, leaderAndIsr); - zookeeperClient.registerLeaderAndIsr(tableBucket, leaderAndIsr); + zookeeperClient.registerLeaderAndIsr( + tableBucket, leaderAndIsr, ZkVersion.MATCH_ANY_VERSION.getVersion()); } // try to batch update @@ -302,7 +305,7 @@ void testBatchUpdateLeaderAndIsr() throws Exception { old.bucketEpoch() + 1); })); zookeeperClient.batchUpdateLeaderAndIsr( - updateLeaderAndIsrList, CoordinatorContext.INITIAL_COORDINATOR_EPOCH); + updateLeaderAndIsrList, ZkVersion.MATCH_ANY_VERSION.getVersion()); for (Map.Entry entry : updateLeaderAndIsrList.entrySet()) { TableBucket tableBucket = entry.getKey(); LeaderAndIsr leaderAndIsr = entry.getValue(); From 283754cbf92221ab5d59e7e9cceda279aa2a80f8 Mon Sep 17 00:00:00 2001 From: "whenzhou.zc" Date: Mon, 10 Nov 2025 10:11:59 +0800 Subject: [PATCH 18/26] support coordinator epoch2 --- .../server/coordinator/CoordinatorServerElectionTest.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java index 032455c688..93aeb604fa 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java @@ -77,12 +77,12 @@ void testCoordinatorServerElection() throws Exception { }); } - // random 1 become leader + // random coordinator become leader waitUntilCoordinatorServerElected(); CoordinatorAddress firstLeaderAddress = zookeeperClient.getCoordinatorLeaderAddress().get(); - // Find the leader and try to close it. + // Find the leader and try to restart it. CoordinatorServer elected = null; for (CoordinatorServer coordinatorServer : coordinatorServerList) { if (coordinatorServer.getServerId() == firstLeaderAddress.getId()) { @@ -104,14 +104,14 @@ void testCoordinatorServerElection() throws Exception { assertThat(zookeeperClient.getCurrentEpoch().f0) .isEqualTo(CoordinatorContext.INITIAL_COORDINATOR_EPOCH + 1); - // kill other 2 coordinator servers + // kill other 2 coordinator servers except the first one for (CoordinatorServer coordinatorServer : coordinatorServerList) { if (coordinatorServer.getServerId() != firstLeaderAddress.getId()) { coordinatorServer.close(); } } // the origin coordinator server should become leader again - waitUntilCoordinatorServerElected(); + waitUntilCoordinatorServerReelected(secondLeaderAddress); CoordinatorAddress thirdLeaderAddress = zookeeperClient.getCoordinatorLeaderAddress().get(); assertThat(thirdLeaderAddress.getId()).isEqualTo(firstLeaderAddress.getId()); From 70958bc5057416b7be89c56ee091b5d9ba4af769 Mon Sep 17 00:00:00 2001 From: "whenzhou.zc" Date: Mon, 10 Nov 2025 10:36:39 +0800 Subject: [PATCH 19/26] support coordinator epoch2 --- .../server/coordinator/MetadataManager.java | 12 +- .../statemachine/TableBucketStateMachine.java | 3 +- .../org/apache/fluss/server/zk/ZkEpoch.java | 38 ++++++ .../fluss/server/zk/ZooKeeperClient.java | 126 +++++++++++++----- .../apache/fluss/server/zk/ZooKeeperOp.java | 4 + .../CoordinatorServerElectionTest.java | 6 +- .../server/coordinator/TableManagerTest.java | 14 +- .../fluss/server/zk/ZooKeeperClientTest.java | 22 +-- 8 files changed, 168 insertions(+), 57 deletions(-) create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/zk/ZkEpoch.java diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/MetadataManager.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/MetadataManager.java index c9537136f8..9099604bfd 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/MetadataManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/MetadataManager.java @@ -53,6 +53,7 @@ import org.apache.fluss.server.zk.data.PartitionAssignment; import org.apache.fluss.server.zk.data.TableAssignment; import org.apache.fluss.server.zk.data.TableRegistration; +import org.apache.fluss.server.zk.data.ZkVersion; import org.apache.fluss.shaded.zookeeper3.org.apache.zookeeper.KeeperException; import org.apache.fluss.utils.function.RunnableWithException; import org.apache.fluss.utils.function.ThrowingRunnable; @@ -257,7 +258,9 @@ public void completeDeleteTable(long tableId) { // delete bucket assignments node, which will also delete the bucket state node, // so that all the zk nodes related to this table are deleted. rethrowIfIsNotNoNodeException( - () -> zookeeperClient.deleteTableAssignment(tableId), + () -> + zookeeperClient.deleteTableAssignment( + tableId, ZkVersion.MATCH_ANY_VERSION.getVersion()), String.format("Delete tablet assignment meta fail for table %s.", tableId)); } @@ -266,7 +269,9 @@ public void completeDeletePartition(long partitionId) { // delete partition assignments node, which will also delete the bucket state node, // so that all the zk nodes related to this partition are deleted. rethrowIfIsNotNoNodeException( - () -> zookeeperClient.deletePartitionAssignment(partitionId), + () -> + zookeeperClient.deletePartitionAssignment( + partitionId, ZkVersion.MATCH_ANY_VERSION.getVersion()), String.format("Delete tablet assignment meta fail for partition %s.", partitionId)); } @@ -319,7 +324,8 @@ public long createTable( long tableId = zookeeperClient.getTableIdAndIncrement(); if (tableAssignment != null) { // register table assignment - zookeeperClient.registerTableAssignment(tableId, tableAssignment); + zookeeperClient.registerTableAssignment( + tableId, tableAssignment, ZkVersion.MATCH_ANY_VERSION.getVersion()); } // register the table zookeeperClient.registerTable( diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachine.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachine.java index 561cd527bc..b01e65c92d 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachine.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachine.java @@ -382,7 +382,8 @@ public void batchHandleOnlineChangeAndInitLeader(Set tableBuckets) if (!tableBucketLeadAndIsrInfos.isEmpty()) { try { zooKeeperClient.batchRegisterLeaderAndIsrForTablePartition( - tableBucketLeadAndIsrInfos); + tableBucketLeadAndIsrInfos, + coordinatorContext.getCoordinatorEpochZkVersion()); registerSuccessList.addAll(tableBucketLeadAndIsrInfos); } catch (Exception e) { LOG.error( diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZkEpoch.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZkEpoch.java new file mode 100644 index 0000000000..b198f30f08 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZkEpoch.java @@ -0,0 +1,38 @@ +/* + * 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; + +/** Class for coordinator epoch and coordinator epoch zk version. */ +public class ZkEpoch { + private final int coordinatorEpoch; + private final int coordinatorEpochZkVersion; + + public ZkEpoch(int coordinatorEpoch, int coordinatorEpochZkVersion) { + this.coordinatorEpoch = coordinatorEpoch; + this.coordinatorEpochZkVersion = coordinatorEpochZkVersion; + } + + public int getCoordinatorEpoch() { + return coordinatorEpoch; + } + + public int getCoordinatorEpochZkVersion() { + return coordinatorEpochZkVersion; + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java index 14ea248a98..a776910569 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java @@ -131,6 +131,13 @@ /** * This class includes methods for write/read various metadata (leader address, tablet server * registration, table assignment, table, schema) in Zookeeper. + * + *

In some method, 'expectedZkVersion' is used to execute an epoch Zookeeper version check. We + * have the following principals to judge if it's necessary to execute epoch Zookeeper version + * check. If all condition met, we need to execute epoch Zookeeper version check. 1. The method + * create/modify/delete Zk node. 2. It's executed by coordinator server. 3. It is about + * metadata(table/partition/leaderAndIsr) rather than server info or ACL info. 4. The Zk node is + * persistent rather than ephemeral. */ @Internal public class ZooKeeperClient implements AutoCloseable { @@ -197,9 +204,9 @@ public Optional fenceBecomeCoordinatorLeader(int coordinatorId) throws ensureEpochZnodeExists(); try { - Tuple2 getEpoch = getCurrentEpoch(); - int currentEpoch = getEpoch.f0; - int currentVersion = getEpoch.f1; + ZkEpoch getEpoch = getCurrentEpoch(); + int currentEpoch = getEpoch.getCoordinatorEpoch(); + int currentVersion = getEpoch.getCoordinatorEpochZkVersion(); int newEpoch = currentEpoch + 1; LOG.info( "Coordinator leader {} tries to update epoch. Current epoch={}, Zookeeper version={}, new epoch={}", @@ -269,7 +276,7 @@ public void ensureEpochZnodeExists() throws Exception { } /** Get epoch now in ZK. */ - public Tuple2 getCurrentEpoch() throws Exception { + public ZkEpoch getCurrentEpoch() throws Exception { Stat currentStat = new Stat(); byte[] bytes = zkClient.getData() @@ -277,7 +284,7 @@ public Tuple2 getCurrentEpoch() throws Exception { .forPath(ZkData.CoordinatorEpochZNode.path()); int currentEpoch = ZkData.CoordinatorEpochZNode.decode(bytes); int currentVersion = currentStat.getVersion(); - return new Tuple2<>(currentEpoch, currentVersion); + return new ZkEpoch(currentEpoch, currentVersion); } // -------------------------------------------------------------------------------------------- @@ -333,13 +340,13 @@ public int[] getSortedTabletServerList() throws Exception { // -------------------------------------------------------------------------------------------- /** Register table assignment to ZK. */ - public void registerTableAssignment(long tableId, TableAssignment tableAssignment) - throws Exception { + public void registerTableAssignment( + long tableId, TableAssignment tableAssignment, int expectedZkVersion) throws Exception { String path = TableIdZNode.path(tableId); - zkClient.create() - .creatingParentsIfNeeded() - .withMode(CreateMode.PERSISTENT) - .forPath(path, TableIdZNode.encode(tableAssignment)); + byte[] data = TableIdZNode.encode(tableAssignment); + + createRecursiveWithEpochCheck(path, data, expectedZkVersion, false); + LOG.info("Registered table assignment {} for table id {}.", tableAssignment, tableId); } @@ -388,10 +395,14 @@ public Map getPartitionsAssignments(Collection "partition assignment"); } - public void updateTableAssignment(long tableId, TableAssignment tableAssignment) - throws Exception { + public void updateTableAssignment( + long tableId, TableAssignment tableAssignment, int expectedZkVersion) throws Exception { String path = TableIdZNode.path(tableId); - zkClient.setData().forPath(path, TableIdZNode.encode(tableAssignment)); + byte[] data = TableIdZNode.encode(tableAssignment); + CuratorOp updateOp = zkOp.updateOp(path, data); + List ops = wrapRequestWithEpochCheck(updateOp, expectedZkVersion); + + zkClient.transaction().forOperations(ops); LOG.debug("Updated table assignment {} for table id {}.", tableAssignment, tableId); } @@ -405,15 +416,16 @@ public void updatePartitionAssignment(long partitionId, PartitionAssignment part partitionId); } - public void deleteTableAssignment(long tableId) throws Exception { + public void deleteTableAssignment(long tableId, int expectedZkVersion) throws Exception { String path = TableIdZNode.path(tableId); - zkClient.delete().deletingChildrenIfNeeded().forPath(path); + deleteRecursiveWithEpochCheck(path, expectedZkVersion, false); LOG.info("Deleted table assignment for table id {}.", tableId); } - public void deletePartitionAssignment(long partitionId) throws Exception { + public void deletePartitionAssignment(long partitionId, int expectedZkVersion) + throws Exception { String path = PartitionIdZNode.path(partitionId); - zkClient.delete().deletingChildrenIfNeeded().forPath(path); + deleteRecursiveWithEpochCheck(path, expectedZkVersion, false); LOG.info("Deleted table assignment for partition id {}.", partitionId); } @@ -429,12 +441,13 @@ public void registerLeaderAndIsr( String path = LeaderAndIsrZNode.path(tableBucket); byte[] data = LeaderAndIsrZNode.encode(leaderAndIsr); - createRecursive(path, data, expectedZkVersion, false); + createRecursiveWithEpochCheck(path, data, expectedZkVersion, false); LOG.info("Registered {} for bucket {} in Zookeeper.", leaderAndIsr, tableBucket); } public void batchRegisterLeaderAndIsrForTablePartition( - List registerList) throws Exception { + List registerList, int expectedZkVersion) + throws Exception { if (registerList.isEmpty()) { return; } @@ -470,12 +483,14 @@ public void batchRegisterLeaderAndIsrForTablePartition( ops.add(parentNodeCreate); ops.add(currentNodeCreate); if (ops.size() == MAX_BATCH_SIZE) { - zkClient.transaction().forOperations(ops); + List wrapOps = wrapRequestsWithEpochCheck(ops, expectedZkVersion); + zkClient.transaction().forOperations(wrapOps); ops.clear(); } } if (!ops.isEmpty()) { - zkClient.transaction().forOperations(ops); + List wrapOps = wrapRequestsWithEpochCheck(ops, expectedZkVersion); + zkClient.transaction().forOperations(wrapOps); } LOG.info( "Batch registered leadAndIsr for tableId: {}, partitionId: {}, partitionName: {} in Zookeeper.", @@ -514,7 +529,7 @@ public void updateLeaderAndIsr( byte[] data = LeaderAndIsrZNode.encode(leaderAndIsr); CuratorOp updateOp = zkOp.updateOp(path, data); - List ops = wrapRequestWithCoordinatorEpochCheck(updateOp, expectedZkVersion); + List ops = wrapRequestWithEpochCheck(updateOp, expectedZkVersion); zkClient.transaction().forOperations(ops); LOG.info("Updated {} for bucket {} in Zookeeper.", leaderAndIsr, tableBucket); @@ -536,22 +551,22 @@ public void batchUpdateLeaderAndIsr( byte[] data = LeaderAndIsrZNode.encode(leaderAndIsr); CuratorOp updateOp = zkClient.transactionOp().setData().forPath(path, data); ops.add(updateOp); - if (ops.size() == MAX_BATCH_SIZE - 1) { - List wrapOps = - wrapRequestsWithCoordinatorEpochCheck(ops, expectedZkVersion); + if (ops.size() == MAX_BATCH_SIZE) { + List wrapOps = wrapRequestsWithEpochCheck(ops, expectedZkVersion); zkClient.transaction().forOperations(wrapOps); ops.clear(); } } if (!ops.isEmpty()) { - List wrapOps = wrapRequestsWithCoordinatorEpochCheck(ops, expectedZkVersion); + List wrapOps = wrapRequestsWithEpochCheck(ops, expectedZkVersion); zkClient.transaction().forOperations(wrapOps); } } - public void deleteLeaderAndIsr(TableBucket tableBucket) throws Exception { + public void deleteLeaderAndIsr(TableBucket tableBucket, int expectedZkVersion) + throws Exception { String path = LeaderAndIsrZNode.path(tableBucket); - zkClient.delete().forPath(path); + deleteRecursiveWithEpochCheck(path, expectedZkVersion, false); LOG.info("Deleted LeaderAndIsr for bucket {} in Zookeeper.", tableBucket); } @@ -1741,11 +1756,19 @@ public static Map> processGetChildrenResponses( return result; } - public void createRecursive( + /** + * create a node (recursively if parent path not exists) with Zk epoch version check. + * + * @param path the path to create + * @param data the data to write + * @param throwIfPathExists whether to throw exception if path exist + * @throws Exception if any error occurs + */ + public void createRecursiveWithEpochCheck( String path, byte[] data, int expectedZkVersion, boolean throwIfPathExists) throws Exception { CuratorOp createOp = zkOp.createOp(path, data, CreateMode.PERSISTENT); - List ops = wrapRequestWithCoordinatorEpochCheck(createOp, expectedZkVersion); + List ops = wrapRequestWithEpochCheck(createOp, expectedZkVersion); try { // try to directly create @@ -1762,19 +1785,48 @@ public void createRecursive( throw new IllegalArgumentException("Invalid path {}" + path); } String parentPath = path.substring(0, indexOfLastSlash); - createRecursive(parentPath, null, expectedZkVersion, throwIfPathExists); + createRecursiveWithEpochCheck(parentPath, null, expectedZkVersion, throwIfPathExists); // After creating parent, retry creating the original path zkClient.transaction().forOperations(ops); } } - public List wrapRequestWithCoordinatorEpochCheck( - CuratorOp request, int expectedZkVersion) throws Exception { - return wrapRequestsWithCoordinatorEpochCheck( - Collections.singletonList(request), expectedZkVersion); + /** + * Delete a node (and recursively delete children) with Zk epoch version check. + * + * @param path the path to delete + * @param expectedZkVersion the expected coordinator epoch zk version + * @param throwIfPathNotExists whether to throw exception if path does not exist + * @throws Exception if any error occurs + */ + public void deleteRecursiveWithEpochCheck( + String path, int expectedZkVersion, boolean throwIfPathNotExists) throws Exception { + // delete children recursively + List children = getChildren(path); + for (String child : children) { + deleteRecursiveWithEpochCheck(path + "/" + child, expectedZkVersion, false); + } + + CuratorOp deleteOp = zkOp.deleteOp(path); + List ops = wrapRequestWithEpochCheck(deleteOp, expectedZkVersion); + + try { + // delete itself + zkClient.transaction().forOperations(ops); + } catch (KeeperException.NoNodeException e) { + // should exist + if (throwIfPathNotExists) { + throw e; + } + } + } + + public List wrapRequestWithEpochCheck(CuratorOp request, int expectedZkVersion) + throws Exception { + return wrapRequestsWithEpochCheck(Collections.singletonList(request), expectedZkVersion); } - public List wrapRequestsWithCoordinatorEpochCheck( + public List wrapRequestsWithEpochCheck( List requestList, int expectedZkVersion) throws Exception { if (ZkVersion.MATCH_ANY_VERSION.getVersion() == expectedZkVersion) { return requestList; diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperOp.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperOp.java index 7dc171de63..e44670d643 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperOp.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperOp.java @@ -44,6 +44,10 @@ public CuratorOp updateOp(String path, byte[] data) throws Exception { return zkClient.transactionOp().setData().forPath(path, data); } + public CuratorOp deleteOp(String path) throws Exception { + return zkClient.transactionOp().delete().forPath(path); + } + public static List multiRequest(CuratorOp op1, CuratorOp op2) { List ops = new ArrayList<>(); ops.add(op1); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java index 93aeb604fa..119b371999 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java @@ -91,7 +91,7 @@ void testCoordinatorServerElection() throws Exception { } } assertThat(elected).isNotNull(); - assertThat(zookeeperClient.getCurrentEpoch().f0) + assertThat(zookeeperClient.getCurrentEpoch().getCoordinatorEpoch()) .isEqualTo(CoordinatorContext.INITIAL_COORDINATOR_EPOCH); elected.close(); elected.start(); @@ -101,7 +101,7 @@ void testCoordinatorServerElection() throws Exception { CoordinatorAddress secondLeaderAddress = zookeeperClient.getCoordinatorLeaderAddress().get(); assertThat(secondLeaderAddress).isNotEqualTo(firstLeaderAddress); - assertThat(zookeeperClient.getCurrentEpoch().f0) + assertThat(zookeeperClient.getCurrentEpoch().getCoordinatorEpoch()) .isEqualTo(CoordinatorContext.INITIAL_COORDINATOR_EPOCH + 1); // kill other 2 coordinator servers except the first one @@ -115,7 +115,7 @@ void testCoordinatorServerElection() throws Exception { CoordinatorAddress thirdLeaderAddress = zookeeperClient.getCoordinatorLeaderAddress().get(); assertThat(thirdLeaderAddress.getId()).isEqualTo(firstLeaderAddress.getId()); - assertThat(zookeeperClient.getCurrentEpoch().f0) + assertThat(zookeeperClient.getCurrentEpoch().getCoordinatorEpoch()) .isEqualTo(CoordinatorContext.INITIAL_COORDINATOR_EPOCH + 2); } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TableManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TableManagerTest.java index 3c417efe8d..c2c4f2aecd 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TableManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TableManagerTest.java @@ -36,6 +36,7 @@ import org.apache.fluss.server.zk.data.BucketAssignment; import org.apache.fluss.server.zk.data.PartitionAssignment; import org.apache.fluss.server.zk.data.TableAssignment; +import org.apache.fluss.server.zk.data.ZkVersion; import org.apache.fluss.testutils.common.AllCallbackWrapper; import org.junit.jupiter.api.AfterAll; @@ -112,7 +113,7 @@ static void afterAll() { private void initTableManager() { testingEventManager = new TestingEventManager(); - coordinatorContext = new CoordinatorContext(); + coordinatorContext = new TestCoordinatorContext(); testCoordinatorChannelManager = new TestCoordinatorChannelManager(); Configuration conf = new Configuration(); conf.setString(ConfigOptions.REMOTE_DATA_DIR, "/tmp/fluss/remote-data"); @@ -169,7 +170,7 @@ void testCreateTable() throws Exception { // all replica should be online checkReplicaOnline(tableId, null, assignment); // clear the assignment for the table - zookeeperClient.deleteTableAssignment(tableId); + zookeeperClient.deleteTableAssignment(tableId, ZkVersion.MATCH_ANY_VERSION.getVersion()); } @Test @@ -177,7 +178,8 @@ void testDeleteTable() throws Exception { // first, create a table long tableId = zookeeperClient.getTableIdAndIncrement(); TableAssignment assignment = createAssignment(); - zookeeperClient.registerTableAssignment(tableId, assignment); + zookeeperClient.registerTableAssignment( + tableId, assignment, ZkVersion.MATCH_ANY_VERSION.getVersion()); coordinatorContext.putTableInfo( TableInfo.of( @@ -216,7 +218,8 @@ void testResumeDeletionAfterRestart() throws Exception { // first, create a table long tableId = zookeeperClient.getTableIdAndIncrement(); TableAssignment assignment = createAssignment(); - zookeeperClient.registerTableAssignment(tableId, assignment); + zookeeperClient.registerTableAssignment( + tableId, assignment, ZkVersion.MATCH_ANY_VERSION.getVersion()); coordinatorContext.putTableInfo( TableInfo.of( @@ -263,7 +266,8 @@ void testCreateAndDropPartition() throws Exception { // create a table long tableId = zookeeperClient.getTableIdAndIncrement(); TableAssignment assignment = TableAssignment.builder().build(); - zookeeperClient.registerTableAssignment(tableId, assignment); + zookeeperClient.registerTableAssignment( + tableId, assignment, ZkVersion.MATCH_ANY_VERSION.getVersion()); coordinatorContext.putTableInfo( TableInfo.of( diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java index 42dc766a03..41b561ef0b 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java @@ -163,8 +163,10 @@ void testTabletAssignments() throws Exception { .add(0, BucketAssignment.of(1, 2)) .add(1, BucketAssignment.of(3, 4, 5)) .build(); - zookeeperClient.registerTableAssignment(tableId1, tableAssignment1); - zookeeperClient.registerTableAssignment(tableId2, tableAssignment2); + zookeeperClient.registerTableAssignment( + tableId1, tableAssignment1, ZkVersion.MATCH_ANY_VERSION.getVersion()); + zookeeperClient.registerTableAssignment( + tableId2, tableAssignment2, ZkVersion.MATCH_ANY_VERSION.getVersion()); assertThat(zookeeperClient.getTableAssignment(tableId1)).contains(tableAssignment1); assertThat(zookeeperClient.getTableAssignment(tableId2)).contains(tableAssignment2); assertThat(zookeeperClient.getTablesAssignments(Arrays.asList(tableId1, tableId2))) @@ -173,11 +175,12 @@ void testTabletAssignments() throws Exception { // test update TableAssignment tableAssignment3 = TableAssignment.builder().add(3, BucketAssignment.of(1, 5)).build(); - zookeeperClient.updateTableAssignment(tableId1, tableAssignment3); + zookeeperClient.updateTableAssignment( + tableId1, tableAssignment3, ZkVersion.MATCH_ANY_VERSION.getVersion()); assertThat(zookeeperClient.getTableAssignment(tableId1)).contains(tableAssignment3); // test delete - zookeeperClient.deleteTableAssignment(tableId1); + zookeeperClient.deleteTableAssignment(tableId1, ZkVersion.MATCH_ANY_VERSION.getVersion()); assertThat(zookeeperClient.getTableAssignment(tableId1)).isEmpty(); } @@ -209,7 +212,7 @@ void testLeaderAndIsr() throws Exception { assertThat(zookeeperClient.getLeaderAndIsr(tableBucket1)).hasValue(leaderAndIsr1); // test delete - zookeeperClient.deleteLeaderAndIsr(tableBucket1); + zookeeperClient.deleteLeaderAndIsr(tableBucket1, ZkVersion.MATCH_ANY_VERSION.getVersion()); assertThat(zookeeperClient.getLeaderAndIsr(tableBucket1)).isEmpty(); } @@ -233,7 +236,8 @@ void testBatchCreateAndUpdateLeaderAndIsr(boolean isPartitionTable) throws Excep tableBucketInfo.add(info); } // batch create - zookeeperClient.batchRegisterLeaderAndIsrForTablePartition(tableBucketInfo); + zookeeperClient.batchRegisterLeaderAndIsrForTablePartition( + tableBucketInfo, ZkVersion.MATCH_ANY_VERSION.getVersion()); for (int i = 0; i < 100; i++) { // each should register successful @@ -310,7 +314,8 @@ void testBatchUpdateLeaderAndIsr() throws Exception { TableBucket tableBucket = entry.getKey(); LeaderAndIsr leaderAndIsr = entry.getValue(); assertThat(zookeeperClient.getLeaderAndIsr(tableBucket)).hasValue(leaderAndIsr); - zookeeperClient.deleteLeaderAndIsr(tableBucket); + zookeeperClient.deleteLeaderAndIsr( + tableBucket, ZkVersion.MATCH_ANY_VERSION.getVersion()); assertThat(zookeeperClient.getLeaderAndIsr(tableBucket)).isEmpty(); } } @@ -457,7 +462,8 @@ void testTableBucketSnapshot() throws Exception { table1Bucket2.getTableId(), TableAssignment.builder() .add(table1Bucket2.getBucket(), BucketAssignment.of(0, 1, 2)) - .build()); + .build(), + ZkVersion.MATCH_ANY_VERSION.getVersion()); BucketSnapshot snapshot1 = new BucketSnapshot(1L, 10L, "oss://test/cp1"); BucketSnapshot snapshot2 = new BucketSnapshot(2L, 20L, "oss://test/cp2"); zookeeperClient.registerTableBucketSnapshot(table1Bucket2, snapshot1); From 913c3bd24036b2d5112140630bd42e9d29d34a69 Mon Sep 17 00:00:00 2001 From: "whenzhou.zc" Date: Thu, 13 Nov 2025 16:07:33 +0800 Subject: [PATCH 20/26] support coordinator epoch2 --- .../org/apache/fluss/server/zk/ZooKeeperClient.java | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java index a776910569..ecbee97452 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java @@ -1782,11 +1782,16 @@ public void createRecursiveWithEpochCheck( // if parent does not exist, create parent first int indexOfLastSlash = path.lastIndexOf("/"); if (indexOfLastSlash == -1) { - throw new IllegalArgumentException("Invalid path {}" + path); + throw new IllegalArgumentException("Invalid path: " + path); } - String parentPath = path.substring(0, indexOfLastSlash); - createRecursiveWithEpochCheck(parentPath, null, expectedZkVersion, throwIfPathExists); - // After creating parent, retry creating the original path + // If indexOfLastSlash is 0, it means the parent is root "/" which should already exist + // We should not try to create it, just retry creating the current path + if (indexOfLastSlash > 0) { + String parentPath = path.substring(0, indexOfLastSlash); + createRecursiveWithEpochCheck( + parentPath, null, expectedZkVersion, throwIfPathExists); + } + // After creating parent (or if parent is root), retry creating the original path zkClient.transaction().forOperations(ops); } } From 47761b73ce2e2d9ebfe82a54f30f8024382a082e Mon Sep 17 00:00:00 2001 From: "whenzhou.zc" Date: Thu, 13 Nov 2025 17:07:37 +0800 Subject: [PATCH 21/26] support coordinator epoch2 --- .../java/org/apache/fluss/server/zk/ZooKeeperClient.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java index ecbee97452..546c536c5f 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java @@ -1783,6 +1783,12 @@ public void createRecursiveWithEpochCheck( int indexOfLastSlash = path.lastIndexOf("/"); if (indexOfLastSlash == -1) { throw new IllegalArgumentException("Invalid path: " + path); + } else if (indexOfLastSlash == 0) { + // root path can be directly create without fence + zkClient.create() + .creatingParentsIfNeeded() + .withMode(CreateMode.PERSISTENT) + .forPath(path); } // If indexOfLastSlash is 0, it means the parent is root "/" which should already exist // We should not try to create it, just retry creating the current path From 3a922cf9b11c920ce530d769657444dd548f9cf5 Mon Sep 17 00:00:00 2001 From: "whenzhou.zc" Date: Thu, 13 Nov 2025 19:09:00 +0800 Subject: [PATCH 22/26] support coordinator epoch2 --- .../fluss/server/zk/ZooKeeperClient.java | 21 +++++++------- .../CoordinatorServerElectionTest.java | 29 ++++++++++++++----- 2 files changed, 32 insertions(+), 18 deletions(-) diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java index 546c536c5f..c48c0f679e 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java @@ -1785,20 +1785,21 @@ public void createRecursiveWithEpochCheck( throw new IllegalArgumentException("Invalid path: " + path); } else if (indexOfLastSlash == 0) { // root path can be directly create without fence - zkClient.create() - .creatingParentsIfNeeded() - .withMode(CreateMode.PERSISTENT) - .forPath(path); - } - // If indexOfLastSlash is 0, it means the parent is root "/" which should already exist - // We should not try to create it, just retry creating the current path - if (indexOfLastSlash > 0) { + try { + zkClient.create() + .creatingParentsIfNeeded() + .withMode(CreateMode.PERSISTENT) + .forPath(path); + } catch (KeeperException.NodeExistsException ignored) { + } + } else { + // indexOfLastSlash > 0 String parentPath = path.substring(0, indexOfLastSlash); createRecursiveWithEpochCheck( parentPath, null, expectedZkVersion, throwIfPathExists); + // After creating parent (or if parent is root), retry creating the original path + zkClient.transaction().forOperations(ops); } - // After creating parent (or if parent is root), retry creating the original path - zkClient.transaction().forOperations(ops); } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java index 119b371999..cf36b70738 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java @@ -83,18 +83,18 @@ void testCoordinatorServerElection() throws Exception { CoordinatorAddress firstLeaderAddress = zookeeperClient.getCoordinatorLeaderAddress().get(); // Find the leader and try to restart it. - CoordinatorServer elected = null; + CoordinatorServer firstLeader = null; for (CoordinatorServer coordinatorServer : coordinatorServerList) { if (coordinatorServer.getServerId() == firstLeaderAddress.getId()) { - elected = coordinatorServer; + firstLeader = coordinatorServer; break; } } - assertThat(elected).isNotNull(); + assertThat(firstLeader).isNotNull(); assertThat(zookeeperClient.getCurrentEpoch().getCoordinatorEpoch()) .isEqualTo(CoordinatorContext.INITIAL_COORDINATOR_EPOCH); - elected.close(); - elected.start(); + firstLeader.close(); + firstLeader.start(); // Then we should get another Coordinator server leader elected waitUntilCoordinatorServerReelected(firstLeaderAddress); @@ -104,12 +104,25 @@ void testCoordinatorServerElection() throws Exception { assertThat(zookeeperClient.getCurrentEpoch().getCoordinatorEpoch()) .isEqualTo(CoordinatorContext.INITIAL_COORDINATOR_EPOCH + 1); - // kill other 2 coordinator servers except the first one + CoordinatorServer secondLeader = null; + for (CoordinatorServer coordinatorServer : coordinatorServerList) { + if (coordinatorServer.getServerId() == secondLeaderAddress.getId()) { + secondLeader = coordinatorServer; + break; + } + } + CoordinatorServer nonLeader = null; for (CoordinatorServer coordinatorServer : coordinatorServerList) { - if (coordinatorServer.getServerId() != firstLeaderAddress.getId()) { - coordinatorServer.close(); + if (coordinatorServer.getServerId() != firstLeaderAddress.getId() + && coordinatorServer.getServerId() != secondLeaderAddress.getId()) { + nonLeader = coordinatorServer; + break; } } + // kill other 2 coordinator servers except the first one + nonLeader.close(); + secondLeader.close(); + // the origin coordinator server should become leader again waitUntilCoordinatorServerReelected(secondLeaderAddress); CoordinatorAddress thirdLeaderAddress = zookeeperClient.getCoordinatorLeaderAddress().get(); From 3c36ce68fba88850922b2f82f18be085cba4178c Mon Sep 17 00:00:00 2001 From: "whenzhou.zc" Date: Fri, 14 Nov 2025 16:13:15 +0800 Subject: [PATCH 23/26] support coordinator epoch2 --- .../fluss/server/coordinator/CoordinatorServer.java | 8 ++++++-- .../fluss/server/testutils/FlussClusterExtension.java | 1 + 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java index 8f59ce9aa6..d6d6edd310 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java @@ -246,8 +246,6 @@ protected void startCoordinatorLeaderService() throws Exception { serverMetricGroup)); rpcServer.start(); - registerCoordinatorLeader(); - this.clientMetricGroup = new ClientMetricGroup(metricRegistry, SERVER_NAME); this.rpcClient = RpcClient.create(conf, clientMetricGroup, true); @@ -257,6 +255,12 @@ protected void startCoordinatorLeaderService() 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")); + + registerCoordinatorLeader(); // start coordinator event processor after we register coordinator leader to zk // so that the event processor can get the coordinator leader node from zk during start // up. diff --git a/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java b/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java index 145e89561f..5c9ce0c532 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java @@ -282,6 +282,7 @@ public void startCoordinatorServer() throws Exception { } else { // start the existing coordinator server coordinatorServer.start(); + waitUntilCoordinatorServerElected(); coordinatorServerInfo = new ServerInfo( 0, From c31b093d5b2082d780b6bdf8a970d309af1f2cc9 Mon Sep 17 00:00:00 2001 From: "whenzhou.zc" Date: Mon, 15 Dec 2025 14:28:58 +0800 Subject: [PATCH 24/26] merge --- .../coordinator/CoordinatorEventProcessor.java | 12 ++++++++++-- .../coordinator/CoordinatorLeaderElection.java | 8 +------- .../server/coordinator/CoordinatorServer.java | 15 +++++++++++++-- .../server/coordinator/CoordinatorServerTest.java | 7 +++++++ .../server/testutils/FlussClusterExtension.java | 7 +++++++ 5 files changed, 38 insertions(+), 11 deletions(-) diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java index b43a7ec5f3..e12a601d4e 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java @@ -71,6 +71,7 @@ import org.apache.fluss.server.coordinator.event.CoordinatorEventManager; import org.apache.fluss.server.coordinator.event.CreatePartitionEvent; import org.apache.fluss.server.coordinator.event.CreateTableEvent; +import org.apache.fluss.server.coordinator.event.DeadCoordinatorServerEvent; import org.apache.fluss.server.coordinator.event.DeadTabletServerEvent; import org.apache.fluss.server.coordinator.event.DeleteReplicaResponseReceivedEvent; import org.apache.fluss.server.coordinator.event.DropPartitionEvent; @@ -78,6 +79,7 @@ import org.apache.fluss.server.coordinator.event.EventProcessor; import org.apache.fluss.server.coordinator.event.FencedCoordinatorEvent; import org.apache.fluss.server.coordinator.event.ListRebalanceProgressEvent; +import org.apache.fluss.server.coordinator.event.NewCoordinatorServerEvent; import org.apache.fluss.server.coordinator.event.NewTabletServerEvent; import org.apache.fluss.server.coordinator.event.NotifyKvSnapshotOffsetEvent; import org.apache.fluss.server.coordinator.event.NotifyLakeTableOffsetEvent; @@ -86,6 +88,7 @@ import org.apache.fluss.server.coordinator.event.RemoveServerTagEvent; import org.apache.fluss.server.coordinator.event.SchemaChangeEvent; import org.apache.fluss.server.coordinator.event.TableRegistrationChangeEvent; +import org.apache.fluss.server.coordinator.event.watcher.CoordinatorServerChangeWatcher; import org.apache.fluss.server.coordinator.event.watcher.TableChangeWatcher; import org.apache.fluss.server.coordinator.event.watcher.TabletServerChangeWatcher; import org.apache.fluss.server.coordinator.rebalance.RebalanceManager; @@ -341,6 +344,7 @@ private void initCoordinatorContext() throws Exception { int[] currentCoordinatorServers = zooKeeperClient.getCoordinatorServerList(); coordinatorContext.setLiveCoordinatorServers( Arrays.stream(currentCoordinatorServers).boxed().collect(Collectors.toSet())); + LOG.info("Load coordinator servers success when initializing coordinator context."); // get all tablet server's int[] currentServers = zooKeeperClient.getSortedTabletServerList(); @@ -1603,7 +1607,10 @@ private void updateReplicaAssignmentForBucket( tableAssignment.forEach( (bucket, replicas) -> newTableAssignment.put(bucket, new BucketAssignment(replicas))); - zooKeeperClient.updateTableAssignment(tableId, new TableAssignment(newTableAssignment)); + zooKeeperClient.updateTableAssignment( + tableId, + new TableAssignment(newTableAssignment), + coordinatorContext.getCoordinatorEpochZkVersion()); } else { Map> partitionAssignment = coordinatorContext.getPartitionAssignment( @@ -2193,7 +2200,8 @@ private void updateBucketEpochAndSendRequest(TableBucket tableBucket, List { try { @@ -515,6 +518,14 @@ CompletableFuture stopServices() { exception = ExceptionUtils.firstOrSuppressed(t, exception); } + try { + if (coordinatorLeaderElection != null) { + coordinatorLeaderElection.close(); + } + } catch (Throwable t) { + exception = ExceptionUtils.firstOrSuppressed(t, exception); + } + try { if (zkClient != null) { zkClient.close(); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java index 76f0e8a582..1a8326ae77 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java @@ -80,5 +80,12 @@ public void waitUntilCoordinatorServerElected() { () -> zookeeperClient.getCoordinatorLeaderAddress().isPresent(), Duration.ofSeconds(10), "Fail to wait coordinator server elected"); + // Sleep 1 second to make sure coordinator server has been started and event processor + // started. + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java b/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java index 5c9ce0c532..0e76dee4d9 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java @@ -945,6 +945,13 @@ public void waitUntilCoordinatorServerElected() { () -> zooKeeperClient.getCoordinatorLeaderAddress().isPresent(), Duration.ofSeconds(10), "Fail to wait coordinator server elected"); + // Sleep 1 second to make sure coordinator server has been started and event processor + // started. + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } } // -------------------------------------------------------------------------------------------- From 27d23bdced0bb48facb2f0d0d3bda983fb35dd28 Mon Sep 17 00:00:00 2001 From: "whenzhou.zc" Date: Mon, 19 Jan 2026 19:57:23 +0800 Subject: [PATCH 25/26] code refactor --- .../CoordinatorEpochFencedException.java | 2 +- .../coordinator/CoordinatorContext.java | 6 +- .../CoordinatorEventProcessor.java | 44 +++++------ ...erEvent.java => DeadCoordinatorEvent.java} | 8 +- ...verEvent.java => NewCoordinatorEvent.java} | 8 +- ...her.java => CoordinatorChangeWatcher.java} | 56 ++++--------- .../watcher/ServerBaseChangeWatcher.java | 78 +++++++++++++++++++ .../watcher/TabletServerChangeWatcher.java | 40 ++-------- .../fluss/server/zk/ZooKeeperClient.java | 61 ++++++++------- .../zk/data/CoordinatorAddressJsonSerde.java | 9 ++- ...java => CoordinatorChangeWatcherTest.java} | 20 ++--- .../data/CoordinatorAddressJsonSerdeTest.java | 26 +++++-- 12 files changed, 204 insertions(+), 154 deletions(-) rename fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/{DeadCoordinatorServerEvent.java => DeadCoordinatorEvent.java} (84%) rename fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/{NewCoordinatorServerEvent.java => NewCoordinatorEvent.java} (84%) rename fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/{CoordinatorServerChangeWatcher.java => CoordinatorChangeWatcher.java} (61%) create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/ServerBaseChangeWatcher.java rename fluss-server/src/test/java/org/apache/fluss/server/coordinator/event/watcher/{CoordinatorServerChangeWatcherTest.java => CoordinatorChangeWatcherTest.java} (83%) diff --git a/fluss-common/src/main/java/org/apache/fluss/exception/CoordinatorEpochFencedException.java b/fluss-common/src/main/java/org/apache/fluss/exception/CoordinatorEpochFencedException.java index b6201c6159..0e0e4e409a 100644 --- a/fluss-common/src/main/java/org/apache/fluss/exception/CoordinatorEpochFencedException.java +++ b/fluss-common/src/main/java/org/apache/fluss/exception/CoordinatorEpochFencedException.java @@ -18,7 +18,7 @@ package org.apache.fluss.exception; -/** Exception thrown when the Coordinator leader epoch is invalid. */ +/** Exception thrown when the Coordinator leader epoch is fenced. */ public class CoordinatorEpochFencedException extends RuntimeException { public CoordinatorEpochFencedException(String message) { super(message); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorContext.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorContext.java index a6c28ebc24..2197388f5e 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorContext.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorContext.java @@ -131,16 +131,16 @@ public Set getLiveCoordinatorServers() { return liveCoordinatorServers; } - public void setLiveCoordinatorServers(Set servers) { + public void setLiveCoordinators(Set servers) { liveCoordinatorServers.clear(); liveCoordinatorServers.addAll(servers); } - public void addLiveCoordinatorServer(int serverId) { + public void addLiveCoordinator(int serverId) { this.liveCoordinatorServers.add(serverId); } - public void removeLiveCoordinatorServer(int serverId) { + public void removeLiveCoordinator(int serverId) { this.liveCoordinatorServers.remove(serverId); } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java index e12a601d4e..45cff09458 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java @@ -71,7 +71,7 @@ import org.apache.fluss.server.coordinator.event.CoordinatorEventManager; import org.apache.fluss.server.coordinator.event.CreatePartitionEvent; import org.apache.fluss.server.coordinator.event.CreateTableEvent; -import org.apache.fluss.server.coordinator.event.DeadCoordinatorServerEvent; +import org.apache.fluss.server.coordinator.event.DeadCoordinatorEvent; import org.apache.fluss.server.coordinator.event.DeadTabletServerEvent; import org.apache.fluss.server.coordinator.event.DeleteReplicaResponseReceivedEvent; import org.apache.fluss.server.coordinator.event.DropPartitionEvent; @@ -79,7 +79,7 @@ import org.apache.fluss.server.coordinator.event.EventProcessor; import org.apache.fluss.server.coordinator.event.FencedCoordinatorEvent; import org.apache.fluss.server.coordinator.event.ListRebalanceProgressEvent; -import org.apache.fluss.server.coordinator.event.NewCoordinatorServerEvent; +import org.apache.fluss.server.coordinator.event.NewCoordinatorEvent; import org.apache.fluss.server.coordinator.event.NewTabletServerEvent; import org.apache.fluss.server.coordinator.event.NotifyKvSnapshotOffsetEvent; import org.apache.fluss.server.coordinator.event.NotifyLakeTableOffsetEvent; @@ -88,7 +88,7 @@ import org.apache.fluss.server.coordinator.event.RemoveServerTagEvent; import org.apache.fluss.server.coordinator.event.SchemaChangeEvent; import org.apache.fluss.server.coordinator.event.TableRegistrationChangeEvent; -import org.apache.fluss.server.coordinator.event.watcher.CoordinatorServerChangeWatcher; +import org.apache.fluss.server.coordinator.event.watcher.CoordinatorChangeWatcher; import org.apache.fluss.server.coordinator.event.watcher.TableChangeWatcher; import org.apache.fluss.server.coordinator.event.watcher.TabletServerChangeWatcher; import org.apache.fluss.server.coordinator.rebalance.RebalanceManager; @@ -176,7 +176,7 @@ public class CoordinatorEventProcessor implements EventProcessor { private final LakeTableTieringManager lakeTableTieringManager; private final TableChangeWatcher tableChangeWatcher; private final CoordinatorChannelManager coordinatorChannelManager; - private final CoordinatorServerChangeWatcher coordinatorServerChangeWatcher; + private final CoordinatorChangeWatcher coordinatorChangeWatcher; private final TabletServerChangeWatcher tabletServerChangeWatcher; private final CoordinatorMetadataCache serverMetadataCache; private final CoordinatorRequestBatch coordinatorRequestBatch; @@ -229,8 +229,8 @@ public CoordinatorEventProcessor( tableBucketStateMachine, new RemoteStorageCleaner(conf, ioExecutor), ioExecutor); - this.coordinatorServerChangeWatcher = - new CoordinatorServerChangeWatcher(zooKeeperClient, coordinatorEventManager); + this.coordinatorChangeWatcher = + new CoordinatorChangeWatcher(zooKeeperClient, coordinatorEventManager); this.tableChangeWatcher = new TableChangeWatcher(zooKeeperClient, coordinatorEventManager); this.tabletServerChangeWatcher = new TabletServerChangeWatcher(zooKeeperClient, coordinatorEventManager); @@ -268,7 +268,7 @@ public CoordinatorContext getCoordinatorContext() { public void startup() { coordinatorContext.setCoordinatorServerInfo(getCoordinatorServerInfo()); // start watchers first so that we won't miss node in zk; - coordinatorServerChangeWatcher.start(); + coordinatorChangeWatcher.start(); tabletServerChangeWatcher.start(); tableChangeWatcher.start(); LOG.info("Initializing coordinator context."); @@ -315,11 +315,8 @@ private ServerInfo getCoordinatorServerInfo() { .getCoordinatorLeaderAddress() .map( coordinatorAddress -> - // TODO we set id to 0 as that CoordinatorServer don't support - // HA, if we support HA, we need to set id to the config - // CoordinatorServer id to avoid node drift. new ServerInfo( - 0, + coordinatorAddress.getId(), null, // For coordinatorServer, no rack info coordinatorAddress.getEndpoints(), ServerType.COORDINATOR)) @@ -342,7 +339,7 @@ private void initCoordinatorContext() throws Exception { long start = System.currentTimeMillis(); // get all coordinator servers int[] currentCoordinatorServers = zooKeeperClient.getCoordinatorServerList(); - coordinatorContext.setLiveCoordinatorServers( + coordinatorContext.setLiveCoordinators( Arrays.stream(currentCoordinatorServers).boxed().collect(Collectors.toSet())); LOG.info("Load coordinator servers success when initializing coordinator context."); @@ -560,7 +557,7 @@ private void onShutdown() { tableManager.shutdown(); // then stop watchers - coordinatorServerChangeWatcher.stop(); + coordinatorChangeWatcher.stop(); tableChangeWatcher.stop(); tabletServerChangeWatcher.stop(); } @@ -585,10 +582,10 @@ public void process(CoordinatorEvent event) { (NotifyLeaderAndIsrResponseReceivedEvent) event); } else if (event instanceof DeleteReplicaResponseReceivedEvent) { processDeleteReplicaResponseReceived((DeleteReplicaResponseReceivedEvent) event); - } else if (event instanceof NewCoordinatorServerEvent) { - processNewCoordinatorServer((NewCoordinatorServerEvent) event); - } else if (event instanceof DeadCoordinatorServerEvent) { - processDeadCoordinatorServer((DeadCoordinatorServerEvent) event); + } else if (event instanceof NewCoordinatorEvent) { + processNewCoordinator((NewCoordinatorEvent) event); + } else if (event instanceof DeadCoordinatorEvent) { + processDeadCoordinator((DeadCoordinatorEvent) event); } else if (event instanceof NewTabletServerEvent) { processNewTabletServer((NewTabletServerEvent) event); } else if (event instanceof DeadTabletServerEvent) { @@ -1000,8 +997,8 @@ private void onReplicaBecomeOffline(Set offlineReplicas) { replicaStateMachine.handleStateChanges(offlineReplicas, OfflineReplica); } - private void processNewCoordinatorServer(NewCoordinatorServerEvent newCoordinatorServerEvent) { - int coordinatorServerId = newCoordinatorServerEvent.getServerId(); + private void processNewCoordinator(NewCoordinatorEvent newCoordinatorEvent) { + int coordinatorServerId = newCoordinatorEvent.getServerId(); if (coordinatorContext.getLiveCoordinatorServers().contains(coordinatorServerId)) { return; } @@ -1009,18 +1006,17 @@ private void processNewCoordinatorServer(NewCoordinatorServerEvent newCoordinato // process new coordinator server LOG.info("New coordinator server callback for coordinator server {}", coordinatorServerId); - coordinatorContext.addLiveCoordinatorServer(coordinatorServerId); + coordinatorContext.addLiveCoordinator(coordinatorServerId); } - private void processDeadCoordinatorServer( - DeadCoordinatorServerEvent deadCoordinatorServerEvent) { - int coordinatorServerId = deadCoordinatorServerEvent.getServerId(); + private void processDeadCoordinator(DeadCoordinatorEvent deadCoordinatorEvent) { + int coordinatorServerId = deadCoordinatorEvent.getServerId(); if (!coordinatorContext.getLiveCoordinatorServers().contains(coordinatorServerId)) { return; } // process dead coordinator server LOG.info("Coordinator server failure callback for {}.", coordinatorServerId); - coordinatorContext.removeLiveCoordinatorServer(coordinatorServerId); + coordinatorContext.removeLiveCoordinator(coordinatorServerId); } private void processNewTabletServer(NewTabletServerEvent newTabletServerEvent) { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/DeadCoordinatorServerEvent.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/DeadCoordinatorEvent.java similarity index 84% rename from fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/DeadCoordinatorServerEvent.java rename to fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/DeadCoordinatorEvent.java index 2ffc0110cc..850c21293d 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/DeadCoordinatorServerEvent.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/DeadCoordinatorEvent.java @@ -20,11 +20,11 @@ import java.util.Objects; /** An event for coordinator server became dead. */ -public class DeadCoordinatorServerEvent implements CoordinatorEvent { +public class DeadCoordinatorEvent implements CoordinatorEvent { private final int serverId; - public DeadCoordinatorServerEvent(int serverId) { + public DeadCoordinatorEvent(int serverId) { this.serverId = serverId; } @@ -40,7 +40,7 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } - DeadCoordinatorServerEvent that = (DeadCoordinatorServerEvent) o; + DeadCoordinatorEvent that = (DeadCoordinatorEvent) o; return serverId == that.serverId; } @@ -51,6 +51,6 @@ public int hashCode() { @Override public String toString() { - return "DeadCoordinatorServerEvent{" + "serverId=" + serverId + '}'; + return "DeadCoordinatorEvent{" + "serverId=" + serverId + '}'; } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/NewCoordinatorServerEvent.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/NewCoordinatorEvent.java similarity index 84% rename from fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/NewCoordinatorServerEvent.java rename to fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/NewCoordinatorEvent.java index fd53bc412d..82cbdde139 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/NewCoordinatorServerEvent.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/NewCoordinatorEvent.java @@ -20,11 +20,11 @@ import java.util.Objects; /** An event for new coordinator server. */ -public class NewCoordinatorServerEvent implements CoordinatorEvent { +public class NewCoordinatorEvent implements CoordinatorEvent { private final int serverId; - public NewCoordinatorServerEvent(int serverId) { + public NewCoordinatorEvent(int serverId) { this.serverId = serverId; } @@ -40,7 +40,7 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } - NewCoordinatorServerEvent that = (NewCoordinatorServerEvent) o; + NewCoordinatorEvent that = (NewCoordinatorEvent) o; return serverId == that.serverId; } @@ -51,6 +51,6 @@ public int hashCode() { @Override public String toString() { - return "NewCoordinatorServerEvent{" + "serverId=" + serverId + '}'; + return "NewCoordinatorEvent{" + "serverId=" + serverId + '}'; } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/CoordinatorServerChangeWatcher.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/CoordinatorChangeWatcher.java similarity index 61% rename from fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/CoordinatorServerChangeWatcher.java rename to fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/CoordinatorChangeWatcher.java index f0008d5186..6d3a860cc7 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/CoordinatorServerChangeWatcher.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/CoordinatorChangeWatcher.java @@ -17,54 +17,37 @@ package org.apache.fluss.server.coordinator.event.watcher; -import org.apache.fluss.exception.FlussRuntimeException; -import org.apache.fluss.server.coordinator.event.DeadCoordinatorServerEvent; +import org.apache.fluss.server.coordinator.event.DeadCoordinatorEvent; import org.apache.fluss.server.coordinator.event.EventManager; -import org.apache.fluss.server.coordinator.event.NewCoordinatorServerEvent; +import org.apache.fluss.server.coordinator.event.NewCoordinatorEvent; 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.cache.ChildData; -import org.apache.fluss.shaded.curator5.org.apache.curator.framework.recipes.cache.CuratorCache; import org.apache.fluss.shaded.curator5.org.apache.curator.framework.recipes.cache.CuratorCacheListener; -import org.apache.fluss.shaded.curator5.org.apache.curator.utils.ZKPaths; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** A watcher to watch the coordinator server changes(new/delete) in zookeeper. */ -public class CoordinatorServerChangeWatcher { +public class CoordinatorChangeWatcher extends ServerBaseChangeWatcher { - private static final Logger LOG = LoggerFactory.getLogger(CoordinatorServerChangeWatcher.class); - private final CuratorCache curatorCache; + private static final Logger LOG = LoggerFactory.getLogger(CoordinatorChangeWatcher.class); - private volatile boolean running; - - private final EventManager eventManager; - - public CoordinatorServerChangeWatcher( - ZooKeeperClient zooKeeperClient, EventManager eventManager) { - this.curatorCache = - CuratorCache.build( - zooKeeperClient.getCuratorClient(), ZkData.CoordinatorIdsZNode.path()); - this.eventManager = eventManager; - this.curatorCache.listenable().addListener(new CoordinatorServerChangeListener()); + public CoordinatorChangeWatcher(ZooKeeperClient zooKeeperClient, EventManager eventManager) { + super(zooKeeperClient, eventManager, ZkData.CoordinatorIdsZNode.path()); } - public void start() { - running = true; - curatorCache.start(); + @Override + protected CuratorCacheListener createListener() { + return new CoordinatorChangeListener(); } - public void stop() { - if (!running) { - return; - } - running = false; - LOG.info("Stopping CoordinatorServerChangeWatcher"); - curatorCache.close(); + @Override + protected String getWatcherName() { + return "CoordinatorChangeWatcher"; } - private final class CoordinatorServerChangeListener implements CuratorCacheListener { + private final class CoordinatorChangeListener implements CuratorCacheListener { @Override public void event(Type type, ChildData oldData, ChildData newData) { @@ -80,7 +63,7 @@ public void event(Type type, ChildData oldData, ChildData newData) { if (newData != null && newData.getData().length > 0) { int serverId = getServerIdFromEvent(newData); LOG.info("Received CHILD_ADDED event for server {}.", serverId); - eventManager.put(new NewCoordinatorServerEvent(serverId)); + eventManager.put(new NewCoordinatorEvent(serverId)); } break; } @@ -89,7 +72,7 @@ public void event(Type type, ChildData oldData, ChildData newData) { if (oldData != null && oldData.getData().length > 0) { int serverId = getServerIdFromEvent(oldData); LOG.info("Received CHILD_REMOVED event for server {}.", serverId); - eventManager.put(new DeadCoordinatorServerEvent(serverId)); + eventManager.put(new DeadCoordinatorEvent(serverId)); } break; } @@ -98,13 +81,4 @@ public void event(Type type, ChildData oldData, ChildData newData) { } } } - - private int getServerIdFromEvent(ChildData data) { - try { - return Integer.parseInt(ZKPaths.getNodeFromPath(data.getPath())); - } catch (NumberFormatException e) { - throw new FlussRuntimeException( - "Invalid server id in zookeeper path: " + data.getPath(), e); - } - } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/ServerBaseChangeWatcher.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/ServerBaseChangeWatcher.java new file mode 100644 index 0000000000..6bc5ca77eb --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/ServerBaseChangeWatcher.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.coordinator.event.watcher; + +import org.apache.fluss.exception.FlussRuntimeException; +import org.apache.fluss.server.coordinator.event.EventManager; +import org.apache.fluss.server.zk.ZooKeeperClient; +import org.apache.fluss.shaded.curator5.org.apache.curator.framework.recipes.cache.ChildData; +import org.apache.fluss.shaded.curator5.org.apache.curator.framework.recipes.cache.CuratorCache; +import org.apache.fluss.shaded.curator5.org.apache.curator.framework.recipes.cache.CuratorCacheListener; +import org.apache.fluss.shaded.curator5.org.apache.curator.utils.ZKPaths; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An abstract base server class for {@link CoordinatorChangeWatcher} and {@link + * TabletServerChangeWatcher}. + */ +public abstract class ServerBaseChangeWatcher { + + private static final Logger LOG = LoggerFactory.getLogger(ServerBaseChangeWatcher.class); + + protected final CuratorCache curatorCache; + protected final EventManager eventManager; + protected volatile boolean running; + + public ServerBaseChangeWatcher( + ZooKeeperClient zooKeeperClient, EventManager eventManager, String zkPath) { + this.curatorCache = CuratorCache.build(zooKeeperClient.getCuratorClient(), zkPath); + this.eventManager = eventManager; + this.curatorCache.listenable().addListener(createListener()); + } + + /** Creates the listener for server change events. */ + protected abstract CuratorCacheListener createListener(); + + /** Returns the watcher name for logging. */ + protected abstract String getWatcherName(); + + public void start() { + running = true; + curatorCache.start(); + } + + public void stop() { + if (!running) { + return; + } + running = false; + LOG.info("Stopping {}", getWatcherName()); + curatorCache.close(); + } + + protected int getServerIdFromEvent(ChildData data) { + try { + return Integer.parseInt(ZKPaths.getNodeFromPath(data.getPath())); + } catch (NumberFormatException e) { + throw new FlussRuntimeException( + "Invalid server id in zookeeper path: " + data.getPath(), e); + } + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/TabletServerChangeWatcher.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/TabletServerChangeWatcher.java index 0bee350eff..bed958eb74 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/TabletServerChangeWatcher.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/TabletServerChangeWatcher.java @@ -18,7 +18,6 @@ package org.apache.fluss.server.coordinator.event.watcher; import org.apache.fluss.cluster.ServerType; -import org.apache.fluss.exception.FlussRuntimeException; import org.apache.fluss.server.coordinator.event.DeadTabletServerEvent; import org.apache.fluss.server.coordinator.event.EventManager; import org.apache.fluss.server.coordinator.event.NewTabletServerEvent; @@ -28,42 +27,28 @@ import org.apache.fluss.server.zk.data.ZkData.ServerIdZNode; import org.apache.fluss.server.zk.data.ZkData.ServerIdsZNode; import org.apache.fluss.shaded.curator5.org.apache.curator.framework.recipes.cache.ChildData; -import org.apache.fluss.shaded.curator5.org.apache.curator.framework.recipes.cache.CuratorCache; import org.apache.fluss.shaded.curator5.org.apache.curator.framework.recipes.cache.CuratorCacheListener; -import org.apache.fluss.shaded.curator5.org.apache.curator.utils.ZKPaths; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** A watcher to watch the tablet server changes(new/delete) in zookeeper. */ -public class TabletServerChangeWatcher { +public class TabletServerChangeWatcher extends ServerBaseChangeWatcher { private static final Logger LOG = LoggerFactory.getLogger(TabletServerChangeWatcher.class); - private final CuratorCache curatorCache; - - private volatile boolean running; - - private final EventManager eventManager; public TabletServerChangeWatcher(ZooKeeperClient zooKeeperClient, EventManager eventManager) { - this.curatorCache = - CuratorCache.build(zooKeeperClient.getCuratorClient(), ServerIdsZNode.path()); - this.eventManager = eventManager; - this.curatorCache.listenable().addListener(new TabletServerChangeListener()); + super(zooKeeperClient, eventManager, ServerIdsZNode.path()); } - public void start() { - running = true; - curatorCache.start(); + @Override + protected CuratorCacheListener createListener() { + return new TabletServerChangeListener(); } - public void stop() { - if (!running) { - return; - } - running = false; - LOG.info("Stopping TabletServerChangeWatcher"); - curatorCache.close(); + @Override + protected String getWatcherName() { + return "TabletServerChangeWatcher"; } private final class TabletServerChangeListener implements CuratorCacheListener { @@ -108,13 +93,4 @@ public void event(Type type, ChildData oldData, ChildData newData) { } } } - - private int getServerIdFromEvent(ChildData data) { - try { - return Integer.parseInt(ZKPaths.getNodeFromPath(data.getPath())); - } catch (NumberFormatException e) { - throw new FlussRuntimeException( - "Invalid server id in zookeeper path: " + data.getPath(), e); - } - } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java index c48c0f679e..1a70b88279 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java @@ -200,36 +200,34 @@ public void registerCoordinatorServer(int coordinatorId) throws Exception { * epoch and coordinator epoch zk version. */ public Optional fenceBecomeCoordinatorLeader(int coordinatorId) throws Exception { - try { - ensureEpochZnodeExists(); + ensureEpochZnodeExists(); - try { - ZkEpoch getEpoch = getCurrentEpoch(); - int currentEpoch = getEpoch.getCoordinatorEpoch(); - int currentVersion = getEpoch.getCoordinatorEpochZkVersion(); - int newEpoch = currentEpoch + 1; - LOG.info( - "Coordinator leader {} tries to update epoch. Current epoch={}, Zookeeper version={}, new epoch={}", - coordinatorId, - currentEpoch, - currentVersion, - newEpoch); - - // atomically update epoch - zkClient.setData() - .withVersion(currentVersion) - .forPath( - ZkData.CoordinatorEpochZNode.path(), - ZkData.CoordinatorEpochZNode.encode(newEpoch)); - - return Optional.of(newEpoch); - } 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); - } - } catch (KeeperException.NodeExistsException e) { + try { + ZkEpoch getEpoch = getCurrentEpoch(); + int currentEpoch = getEpoch.getCoordinatorEpoch(); + int currentVersion = getEpoch.getCoordinatorEpochZkVersion(); + int newEpoch = currentEpoch + 1; + LOG.info( + "Coordinator leader {} tries to update epoch. Current epoch={}, Zookeeper version={}, new epoch={}", + coordinatorId, + currentEpoch, + currentVersion, + newEpoch); + + // atomically update epoch + zkClient.setData() + .withVersion(currentVersion) + .forPath( + ZkData.CoordinatorEpochZNode.path(), + ZkData.CoordinatorEpochZNode.encode(newEpoch)); + + return Optional.of(newEpoch); + } 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(); } @@ -271,6 +269,9 @@ public void ensureEpochZnodeExists() throws Exception { ZkData.CoordinatorEpochZNode.encode( CoordinatorContext.INITIAL_COORDINATOR_EPOCH - 1)); } catch (KeeperException.NodeExistsException e) { + // This should not happen. + throw new RuntimeException( + "Coordinator leader try to init epoch znode failed. Epoch znode should not exist."); } } } @@ -1791,6 +1792,7 @@ public void createRecursiveWithEpochCheck( .withMode(CreateMode.PERSISTENT) .forPath(path); } catch (KeeperException.NodeExistsException ignored) { + // ignore } } else { // indexOfLastSlash > 0 @@ -1800,6 +1802,9 @@ public void createRecursiveWithEpochCheck( // After creating parent (or if parent is root), retry creating the original path zkClient.transaction().forOperations(ops); } + } catch (KeeperException.BadVersionException e) { + LOG.error("Bad version for path {}, expected version {} ", path, expectedZkVersion); + throw e; } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/CoordinatorAddressJsonSerde.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/CoordinatorAddressJsonSerde.java index c2863fe610..5429f8d1c5 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/CoordinatorAddressJsonSerde.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/CoordinatorAddressJsonSerde.java @@ -35,7 +35,7 @@ public class CoordinatorAddressJsonSerde public static final CoordinatorAddressJsonSerde INSTANCE = new CoordinatorAddressJsonSerde(); private static final String VERSION_KEY = "version"; - private static final int VERSION = 2; + private static final int VERSION = 3; private static final String ID = "id"; private static final String HOST = "host"; @@ -60,13 +60,18 @@ public void serialize(CoordinatorAddress coordinatorAddress, JsonGenerator gener @Override public CoordinatorAddress deserialize(JsonNode node) { int version = node.get(VERSION_KEY).asInt(); - int id = node.get(ID).asInt(); + int id; List endpoints; if (version == 1) { + id = Integer.parseInt(node.get(ID).asText()); String host = node.get(HOST).asText(); int port = node.get(PORT).asInt(); endpoints = Collections.singletonList(new Endpoint(host, port, "CLIENT")); + } else if (version == 2) { + id = Integer.parseInt(node.get(ID).asText()); + endpoints = Endpoint.fromListenersString(node.get(LISTENERS).asText()); } else { + id = node.get(ID).asInt(); endpoints = Endpoint.fromListenersString(node.get(LISTENERS).asText()); } return new CoordinatorAddress(id, endpoints); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/event/watcher/CoordinatorServerChangeWatcherTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/event/watcher/CoordinatorChangeWatcherTest.java similarity index 83% rename from fluss-server/src/test/java/org/apache/fluss/server/coordinator/event/watcher/CoordinatorServerChangeWatcherTest.java rename to fluss-server/src/test/java/org/apache/fluss/server/coordinator/event/watcher/CoordinatorChangeWatcherTest.java index f35206c3d3..ba482c6f05 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/event/watcher/CoordinatorServerChangeWatcherTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/event/watcher/CoordinatorChangeWatcherTest.java @@ -18,8 +18,8 @@ package org.apache.fluss.server.coordinator.event.watcher; import org.apache.fluss.server.coordinator.event.CoordinatorEvent; -import org.apache.fluss.server.coordinator.event.DeadCoordinatorServerEvent; -import org.apache.fluss.server.coordinator.event.NewCoordinatorServerEvent; +import org.apache.fluss.server.coordinator.event.DeadCoordinatorEvent; +import org.apache.fluss.server.coordinator.event.NewCoordinatorEvent; import org.apache.fluss.server.coordinator.event.TestingEventManager; import org.apache.fluss.server.zk.NOPErrorHandler; import org.apache.fluss.server.zk.ZooKeeperClient; @@ -36,8 +36,8 @@ import static org.apache.fluss.testutils.common.CommonTestUtils.retry; import static org.assertj.core.api.Assertions.assertThat; -/** Test for {@link CoordinatorServerChangeWatcher} . */ -class CoordinatorServerChangeWatcherTest { +/** Test for {@link CoordinatorChangeWatcher} . */ +class CoordinatorChangeWatcherTest { @RegisterExtension public static final AllCallbackWrapper ZOO_KEEPER_EXTENSION_WRAPPER = @@ -50,14 +50,14 @@ void testServerChanges() throws Exception { .getCustomExtension() .getZooKeeperClient(NOPErrorHandler.INSTANCE); TestingEventManager eventManager = new TestingEventManager(); - CoordinatorServerChangeWatcher coordinatorServerChangeWatcher = - new CoordinatorServerChangeWatcher(zookeeperClient, eventManager); - coordinatorServerChangeWatcher.start(); + CoordinatorChangeWatcher coordinatorChangeWatcher = + new CoordinatorChangeWatcher(zookeeperClient, eventManager); + coordinatorChangeWatcher.start(); // register new servers List expectedEvents = new ArrayList<>(); for (int i = 0; i < 10; i++) { - expectedEvents.add(new NewCoordinatorServerEvent(i)); + expectedEvents.add(new NewCoordinatorEvent(i)); zookeeperClient.registerCoordinatorServer(i); } @@ -72,7 +72,7 @@ void testServerChanges() throws Exception { // unregister servers for (int i = 0; i < 10; i++) { - expectedEvents.add(new DeadCoordinatorServerEvent(i)); + expectedEvents.add(new DeadCoordinatorEvent(i)); } retry( @@ -81,6 +81,6 @@ void testServerChanges() throws Exception { assertThat(eventManager.getEvents()) .containsExactlyInAnyOrderElementsOf(expectedEvents)); - coordinatorServerChangeWatcher.stop(); + coordinatorChangeWatcher.stop(); } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/CoordinatorAddressJsonSerdeTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/CoordinatorAddressJsonSerdeTest.java index de1cd6555a..979f86713d 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/CoordinatorAddressJsonSerdeTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/CoordinatorAddressJsonSerdeTest.java @@ -49,22 +49,38 @@ protected CoordinatorAddress[] createObjects() { @Override protected String[] expectedJsons() { return new String[] { - "{\"version\":2,\"id\":1,\"listeners\":\"CLIENT://localhost:1001,FLUSS://127.0.0.1:9124\"}" + "{\"version\":3,\"id\":1,\"listeners\":\"CLIENT://localhost:1001,FLUSS://127.0.0.1:9124\"}" }; } @Test void testCompatibility() throws IOException { + // version 1 JsonNode jsonInVersion1 = new ObjectMapper() .readTree( - "{\"version\":1,\"id\":1,\"host\":\"localhost\",\"port\":1001}" + "{\"version\":1,\"id\":\"1\",\"host\":\"localhost\",\"port\":1001}" .getBytes(StandardCharsets.UTF_8)); - CoordinatorAddress coordinatorAddress = + CoordinatorAddress coordinatorAddress1 = CoordinatorAddressJsonSerde.INSTANCE.deserialize(jsonInVersion1); - CoordinatorAddress expectedCoordinator = + CoordinatorAddress expectedCoordinator1 = new CoordinatorAddress(1, Endpoint.fromListenersString("CLIENT://localhost:1001")); - assertEquals(coordinatorAddress, expectedCoordinator); + assertEquals(coordinatorAddress1, expectedCoordinator1); + + // version 2 + JsonNode jsonInVersion2 = + new ObjectMapper() + .readTree( + "{\"version\":2,\"id\":\"2\",\"listeners\":\"CLIENT://localhost:1001,FLUSS://127.0.0.1:9124\"}" + .getBytes(StandardCharsets.UTF_8)); + CoordinatorAddress coordinatorAddress2 = + CoordinatorAddressJsonSerde.INSTANCE.deserialize(jsonInVersion2); + CoordinatorAddress expectedCoordinator2 = + new CoordinatorAddress( + 2, + Endpoint.fromListenersString( + "CLIENT://localhost:1001,FLUSS://127.0.0.1:9124")); + assertEquals(coordinatorAddress2, expectedCoordinator2); } } From 5fae68004da4cc686423e9226f749f1e174322d0 Mon Sep 17 00:00:00 2001 From: "whenzhou.zc" Date: Mon, 15 Dec 2025 14:28:58 +0800 Subject: [PATCH 26/26] rebase and fix test --- .../CoordinatorLeaderElection.java | 101 +++++++++++++----- .../server/coordinator/CoordinatorServer.java | 30 ++++-- .../fluss/server/replica/ReplicaManager.java | 5 + .../org/apache/fluss/server/zk/ZkEpoch.java | 4 + .../fluss/server/zk/ZooKeeperClient.java | 21 ++-- .../CoordinatorServerElectionTest.java | 17 +-- .../coordinator/CoordinatorServerTest.java | 9 +- .../testutils/FlussClusterExtension.java | 5 +- 8 files changed, 121 insertions(+), 71 deletions(-) diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java index ca78f6aa8f..12f1791e6f 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorLeaderElection.java @@ -19,6 +19,7 @@ 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; @@ -28,6 +29,7 @@ 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. */ @@ -39,6 +41,8 @@ public class CoordinatorLeaderElection implements AutoCloseable { private final CoordinatorContext coordinatorContext; private final LeaderLatch leaderLatch; private final AtomicBoolean isLeader = new AtomicBoolean(false); + private final CompletableFuture leaderReadyFuture = new CompletableFuture<>(); + private volatile Thread electionThread; public CoordinatorLeaderElection( ZooKeeperClient zkClient, int serverId, CoordinatorContext coordinatorContext) { @@ -52,31 +56,20 @@ public CoordinatorLeaderElection( String.valueOf(serverId)); } - public void startElectLeader(Runnable initLeaderServices) { + /** + * 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 startElectLeaderAsync(Runnable initLeaderServices) { leaderLatch.addListener( new LeaderLatchListener() { @Override public void isLeader() { LOG.info("Coordinator server {} has become the leader.", serverId); isLeader.set(true); - try { - // to avoid split-brain - Optional optionalEpoch = - zkClient.fenceBecomeCoordinatorLeader(serverId); - if (optionalEpoch.isPresent()) { - coordinatorContext.setCoordinatorEpochAndZkVersion( - optionalEpoch.get(), - coordinatorContext.getCoordinatorEpochZkVersion() + 1); - initLeaderServices.run(); - } else { - throw new CoordinatorEpochFencedException( - "Fenced to become coordinator leader."); - } - } catch (Exception e) { - relinquishLeadership(); - throw new CoordinatorEpochFencedException( - "Fenced to become coordinator leader."); - } } @Override @@ -90,21 +83,75 @@ public void notLeader() { 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; + } - // 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 initializing context - // leaderLatch.await(); + // 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 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.", + serverId); + throw e; } catch (Exception e) { - LOG.error("Failed to start LeaderLatch for server {}", serverId, e); - throw new RuntimeException("Leader election start failed", e); + LOG.warn("Coordinator server {} failed to become leader successfully.", serverId, e); + relinquishLeadership(); + throw new RuntimeException("Failed to become leader", e); } + initLeaderServices.run(); } @Override public void close() { LOG.info("Closing LeaderLatch for server {}.", serverId); + + // Interrupt the election thread if it's waiting + if (electionThread != null && electionThread.isAlive()) { + electionThread.interrupt(); + } + if (leaderLatch != null) { try { leaderLatch.close(); @@ -112,6 +159,10 @@ public void close() { LOG.error("Failed to close LeaderLatch for server {}.", serverId, e); } } + + // Complete the future exceptionally if it hasn't been completed yet + leaderReadyFuture.completeExceptionally( + new RuntimeException("Leader election closed for server " + serverId)); } public boolean isLeader() { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java index 9a4113bf60..9cc43dc8a5 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java @@ -144,6 +144,9 @@ public class CoordinatorServer extends ServerBase { @GuardedBy("lock") private CoordinatorLeaderElection coordinatorLeaderElection; + @GuardedBy("lock") + private CompletableFuture leaderElectionFuture; + public CoordinatorServer(Configuration conf) { super(conf); validateConfigs(conf); @@ -161,10 +164,10 @@ public static void main(String[] args) { @Override protected void startServices() throws Exception { this.coordinatorContext = new CoordinatorContext(); - electCoordinatorLeader(); + electCoordinatorLeaderAsync(); } - private void electCoordinatorLeader() throws Exception { + private CompletableFuture electCoordinatorLeaderAsync() throws Exception { this.zkClient = ZooKeeperUtils.startZookeeperClient(conf, this); // Coordinator Server supports high availability. If 3 coordinator servers are alive, @@ -177,14 +180,16 @@ private void electCoordinatorLeader() throws Exception { // standby this.coordinatorLeaderElection = new CoordinatorLeaderElection(zkClient, serverId, coordinatorContext); - coordinatorLeaderElection.startElectLeader( - () -> { - try { - startCoordinatorLeaderService(); - } catch (Exception e) { - throw new RuntimeException(e); - } - }); + this.leaderElectionFuture = + coordinatorLeaderElection.startElectLeaderAsync( + () -> { + try { + startCoordinatorLeaderService(); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + return leaderElectionFuture; } protected void startCoordinatorLeaderService() throws Exception { @@ -563,6 +568,11 @@ public CoordinatorService getCoordinatorService() { return coordinatorService; } + @VisibleForTesting + public CompletableFuture getLeaderElectionFuture() { + return leaderElectionFuture; + } + @Override protected String getServerName() { return SERVER_NAME; diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java index d8ddb92c4d..ec363731a9 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java @@ -1901,6 +1901,11 @@ public TabletServerMetricGroup getServerMetricGroup() { return serverMetricGroup; } + @VisibleForTesting + public void resetCoordinatorEpoch() { + this.coordinatorEpoch = CoordinatorContext.INITIAL_COORDINATOR_EPOCH; + } + /** * Interface to represent the state of hosted {@link Replica}. We create a concrete (active) * {@link Replica} instance when the TabletServer receives a createLogLeader request or diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZkEpoch.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZkEpoch.java index b198f30f08..f3d401a255 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZkEpoch.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZkEpoch.java @@ -28,6 +28,10 @@ public ZkEpoch(int coordinatorEpoch, int coordinatorEpochZkVersion) { this.coordinatorEpochZkVersion = coordinatorEpochZkVersion; } + public ZkEpoch nextZkEpoch() { + return new ZkEpoch(coordinatorEpoch + 1, coordinatorEpochZkVersion + 1); + } + public int getCoordinatorEpoch() { return coordinatorEpoch; } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java index 1a70b88279..55a53c623a 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java @@ -21,7 +21,6 @@ import org.apache.fluss.annotation.VisibleForTesting; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; -import org.apache.fluss.exception.InvalidCoordinatorException; import org.apache.fluss.metadata.DatabaseSummary; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.ResolvedPartitionSpec; @@ -199,29 +198,29 @@ public void registerCoordinatorServer(int coordinatorId) throws Exception { * registerCoordinatorLeader(). This is to ensure the coordinator get and update the coordinator * epoch and coordinator epoch zk version. */ - public Optional fenceBecomeCoordinatorLeader(int coordinatorId) throws Exception { + public Optional fenceBecomeCoordinatorLeader(int coordinatorId) throws Exception { ensureEpochZnodeExists(); try { ZkEpoch getEpoch = getCurrentEpoch(); - int currentEpoch = getEpoch.getCoordinatorEpoch(); - int currentVersion = getEpoch.getCoordinatorEpochZkVersion(); - int newEpoch = currentEpoch + 1; + 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, - currentEpoch, - currentVersion, - newEpoch); + currentCoordinatorEpoch, + currentCoordinatorEpochZkVersion, + newCoordinatorEpoch); // atomically update epoch zkClient.setData() - .withVersion(currentVersion) + .withVersion(currentCoordinatorEpochZkVersion) .forPath( ZkData.CoordinatorEpochZNode.path(), - ZkData.CoordinatorEpochZNode.encode(newEpoch)); + ZkData.CoordinatorEpochZNode.encode(newCoordinatorEpoch)); - return Optional.of(newEpoch); + 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. diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java index cf36b70738..9630a2d7ee 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerElectionTest.java @@ -33,8 +33,6 @@ import java.time.Duration; import java.util.Arrays; import java.util.List; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import static org.apache.fluss.testutils.common.CommonTestUtils.waitUntil; import static org.assertj.core.api.Assertions.assertThat; @@ -64,17 +62,9 @@ void testCoordinatorServerElection() throws Exception { Arrays.asList(coordinatorServer1, coordinatorServer2, coordinatorServer3); // start 3 coordinator servers - ExecutorService executor = Executors.newFixedThreadPool(3); for (int i = 0; i < 3; i++) { CoordinatorServer server = coordinatorServerList.get(i); - executor.submit( - () -> { - try { - server.start(); - } catch (Exception e) { - e.printStackTrace(); - } - }); + server.start(); } // random coordinator become leader @@ -143,11 +133,6 @@ protected static Configuration createConfiguration(int serverId) { configuration.setString(ConfigOptions.ADVERTISED_LISTENERS, "CLIENT://198.168.0.1:100"); configuration.set(ConfigOptions.REMOTE_DATA_DIR, "/tmp/fluss/remote-data"); - // set to small timout to verify the case that zk session is timeout - configuration.set(ConfigOptions.ZOOKEEPER_SESSION_TIMEOUT, Duration.ofMillis(500)); - configuration.set(ConfigOptions.ZOOKEEPER_CONNECTION_TIMEOUT, Duration.ofMillis(500)); - configuration.set(ConfigOptions.ZOOKEEPER_RETRY_WAIT, Duration.ofMillis(500)); - configuration.set(ConfigOptions.COORDINATOR_ID, serverId); return configuration; } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java index 1a8326ae77..0340ebe82b 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorServerTest.java @@ -78,14 +78,7 @@ protected void checkAfterStartServer() throws Exception { public void waitUntilCoordinatorServerElected() { waitUntil( () -> zookeeperClient.getCoordinatorLeaderAddress().isPresent(), - Duration.ofSeconds(10), + Duration.ofSeconds(5), "Fail to wait coordinator server elected"); - // Sleep 1 second to make sure coordinator server has been started and event processor - // started. - try { - Thread.sleep(1000); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java b/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java index 0e76dee4d9..90145e863c 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java @@ -205,6 +205,7 @@ public void afterEach(ExtensionContext extensionContext) throws Exception { // metadata. for (TabletServer tabletServer : tabletServers.values()) { tabletServer.getMetadataCache().clearTableMetadata(); + tabletServer.getReplicaManager().resetCoordinatorEpoch(); } } @@ -940,7 +941,9 @@ public CoordinatorServer getCoordinatorServer() { return coordinatorServer; } - public void waitUntilCoordinatorServerElected() { + public void waitUntilCoordinatorServerElected() throws Exception { + coordinatorServer.getLeaderElectionFuture().get(); + waitUntil( () -> zooKeeperClient.getCoordinatorLeaderAddress().isPresent(), Duration.ofSeconds(10),