diff --git a/fluss-common/src/main/java/com/alibaba/fluss/config/ConfigOptions.java b/fluss-common/src/main/java/com/alibaba/fluss/config/ConfigOptions.java index 73c3fd054d..d5f6c7c091 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/config/ConfigOptions.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/config/ConfigOptions.java @@ -285,6 +285,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."); + public static final ConfigOption COORDINATOR_IO_POOL_SIZE = key("coordinator.io-pool.size") .intType() diff --git a/fluss-common/src/main/java/com/alibaba/fluss/metrics/MetricNames.java b/fluss-common/src/main/java/com/alibaba/fluss/metrics/MetricNames.java index dee2aa1648..a8e8eb68c8 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/metrics/MetricNames.java +++ b/fluss-common/src/main/java/com/alibaba/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/com/alibaba/fluss/server/coordinator/CoordinatorContext.java b/fluss-server/src/main/java/com/alibaba/fluss/server/coordinator/CoordinatorContext.java index 01df39a048..acff0e11d4 100644 --- a/fluss-server/src/main/java/com/alibaba/fluss/server/coordinator/CoordinatorContext.java +++ b/fluss-server/src/main/java/com/alibaba/fluss/server/coordinator/CoordinatorContext.java @@ -66,6 +66,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<>(); // a map from the table bucket to the state of the bucket. @@ -110,6 +111,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/com/alibaba/fluss/server/coordinator/CoordinatorEventProcessor.java b/fluss-server/src/main/java/com/alibaba/fluss/server/coordinator/CoordinatorEventProcessor.java index afadb7508f..e28d1ec711 100644 --- a/fluss-server/src/main/java/com/alibaba/fluss/server/coordinator/CoordinatorEventProcessor.java +++ b/fluss-server/src/main/java/com/alibaba/fluss/server/coordinator/CoordinatorEventProcessor.java @@ -49,16 +49,20 @@ import com.alibaba.fluss.server.coordinator.event.CoordinatorEventManager; import com.alibaba.fluss.server.coordinator.event.CreatePartitionEvent; import com.alibaba.fluss.server.coordinator.event.CreateTableEvent; +import com.alibaba.fluss.server.coordinator.event.DeadCoordinatorServerEvent; import com.alibaba.fluss.server.coordinator.event.DeadTabletServerEvent; import com.alibaba.fluss.server.coordinator.event.DeleteReplicaResponseReceivedEvent; import com.alibaba.fluss.server.coordinator.event.DropPartitionEvent; import com.alibaba.fluss.server.coordinator.event.DropTableEvent; import com.alibaba.fluss.server.coordinator.event.EventProcessor; import com.alibaba.fluss.server.coordinator.event.FencedCoordinatorEvent; +import com.alibaba.fluss.server.coordinator.event.NewCoordinatorServerEvent; import com.alibaba.fluss.server.coordinator.event.NewTabletServerEvent; import com.alibaba.fluss.server.coordinator.event.NotifyLeaderAndIsrResponseReceivedEvent; +import com.alibaba.fluss.server.coordinator.event.watcher.CoordinatorServerChangeWatcher; import com.alibaba.fluss.server.coordinator.event.watcher.TableChangeWatcher; import com.alibaba.fluss.server.coordinator.event.watcher.TabletServerChangeWatcher; +import com.alibaba.fluss.server.coordinator.statemachine.ReplicaState; import com.alibaba.fluss.server.coordinator.statemachine.ReplicaStateMachine; import com.alibaba.fluss.server.coordinator.statemachine.TableBucketStateMachine; import com.alibaba.fluss.server.entity.AdjustIsrResultForBucket; @@ -90,6 +94,7 @@ import javax.annotation.concurrent.NotThreadSafe; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashSet; import java.util.List; @@ -126,6 +131,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; @@ -174,6 +180,8 @@ public CoordinatorEventProcessor( replicaStateMachine, tableBucketStateMachine, new RemoteStorageCleaner(conf, ioExecutor)); + this.coordinatorServerChangeWatcher = + new CoordinatorServerChangeWatcher(zooKeeperClient, coordinatorEventManager); this.tableChangeWatcher = new TableChangeWatcher(zooKeeperClient, coordinatorEventManager); this.tabletServerChangeWatcher = new TabletServerChangeWatcher(zooKeeperClient, coordinatorEventManager); @@ -198,6 +206,7 @@ public CoordinatorEventManager getCoordinatorEventManager() { 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."); @@ -235,7 +244,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 @@ -263,6 +272,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<>(); @@ -419,6 +433,7 @@ private void onShutdown() { tableManager.shutdown(); // then stop watchers + coordinatorServerChangeWatcher.stop(); tableChangeWatcher.stop(); tabletServerChangeWatcher.stop(); } @@ -438,6 +453,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) { @@ -477,6 +496,11 @@ public void process(CoordinatorEvent event) { } } + private boolean isReplicaToDelete(TableBucketReplica replica) { + ReplicaState replicaState = coordinatorContext.getReplicaState(replica); + return replicaState != null && replicaState != ReplicaDeletionSuccessful; + } + private void processCreateTable(CreateTableEvent createTableEvent) { long tableId = createTableEvent.getTableInfo().getTableId(); // skip the table if it already exists @@ -696,10 +720,33 @@ 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 - // listener the children created and deleted in zk node. + // listen the children created and deleted in zk node. // Also, Kafka use broker epoch to make it can reject the LeaderAndIsrRequest, // UpdateMetadataRequest and StopReplicaRequest @@ -718,7 +765,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/com/alibaba/fluss/server/coordinator/CoordinatorLeaderElection.java b/fluss-server/src/main/java/com/alibaba/fluss/server/coordinator/CoordinatorLeaderElection.java new file mode 100644 index 0000000000..e404f72f6b --- /dev/null +++ b/fluss-server/src/main/java/com/alibaba/fluss/server/coordinator/CoordinatorLeaderElection.java @@ -0,0 +1,90 @@ +/* + * 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.LeaderLatch; +import com.alibaba.fluss.shaded.curator5.org.apache.curator.framework.recipes.leader.LeaderLatchListener; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +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 implements AutoCloseable { + private static final Logger LOG = LoggerFactory.getLogger(CoordinatorLeaderElection.class); + + private final int serverId; + private final LeaderLatch leaderLatch; + private final AtomicBoolean isLeader = new AtomicBoolean(false); + + public CoordinatorLeaderElection(CuratorFramework zkClient, int serverId) { + this.serverId = serverId; + this.leaderLatch = + new LeaderLatch( + zkClient, ZkData.CoordinatorElectionZNode.path(), String.valueOf(serverId)); + } + + public void startElectLeader(Runnable initLeaderServices) { + leaderLatch.addListener( + new LeaderLatchListener() { + @Override + public void isLeader() { + LOG.info("Coordinator server {} has become the leader.", serverId); + isLeader.set(true); + } + + @Override + public void notLeader() { + LOG.warn("Coordinator server {} has lost the leadership.", serverId); + isLeader.set(false); + } + }); + + try { + leaderLatch.start(); + LOG.info("Coordinator server {} started leader election.", serverId); + + // 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(); + + } 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(); + } + } + + public boolean isLeader() { + return this.isLeader.get(); + } +} diff --git a/fluss-server/src/main/java/com/alibaba/fluss/server/coordinator/CoordinatorServer.java b/fluss-server/src/main/java/com/alibaba/fluss/server/coordinator/CoordinatorServer.java index e8388f10f5..00067d35ef 100644 --- a/fluss-server/src/main/java/com/alibaba/fluss/server/coordinator/CoordinatorServer.java +++ b/fluss-server/src/main/java/com/alibaba/fluss/server/coordinator/CoordinatorServer.java @@ -60,7 +60,7 @@ import java.util.Collection; import java.util.List; import java.util.Map; -import java.util.UUID; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -94,7 +94,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; @@ -146,6 +146,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) { @@ -157,10 +158,45 @@ public static void main(String[] args) { @Override protected void startServices() throws Exception { + electCoordinatorLeader(); + } + + private void electCoordinatorLeader() throws Exception { + 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(); + 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); + } + }); + } + } + + 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); @@ -171,8 +207,6 @@ protected void startServices() throws Exception { endpoints.get(0).getHost(), serverId); - this.zkClient = ZooKeeperUtils.startZookeeperClient(conf, this); - this.coordinatorContext = new CoordinatorContext(); this.metadataCache = new CoordinatorMetadataCache(); @@ -281,6 +315,54 @@ 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 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(); @@ -487,6 +569,11 @@ public RpcServer getRpcServer() { return rpcServer; } + @VisibleForTesting + public int getServerId() { + return serverId; + } + @VisibleForTesting public ServerMetadataCache getMetadataCache() { return metadataCache; @@ -498,6 +585,19 @@ public ServerMetadataCache getMetadataCache() { } 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/com/alibaba/fluss/server/coordinator/event/CoordinatorEventManager.java b/fluss-server/src/main/java/com/alibaba/fluss/server/coordinator/event/CoordinatorEventManager.java index c220a9f63f..a17ee2a0b4 100644 --- a/fluss-server/src/main/java/com/alibaba/fluss/server/coordinator/event/CoordinatorEventManager.java +++ b/fluss-server/src/main/java/com/alibaba/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; @@ -93,6 +94,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); diff --git a/fluss-server/src/main/java/com/alibaba/fluss/server/coordinator/event/DeadCoordinatorServerEvent.java b/fluss-server/src/main/java/com/alibaba/fluss/server/coordinator/event/DeadCoordinatorServerEvent.java new file mode 100644 index 0000000000..24850c7981 --- /dev/null +++ b/fluss-server/src/main/java/com/alibaba/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/com/alibaba/fluss/server/coordinator/event/NewCoordinatorServerEvent.java b/fluss-server/src/main/java/com/alibaba/fluss/server/coordinator/event/NewCoordinatorServerEvent.java new file mode 100644 index 0000000000..cf6abcf56c --- /dev/null +++ b/fluss-server/src/main/java/com/alibaba/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/com/alibaba/fluss/server/coordinator/event/watcher/CoordinatorServerChangeWatcher.java b/fluss-server/src/main/java/com/alibaba/fluss/server/coordinator/event/watcher/CoordinatorServerChangeWatcher.java new file mode 100644 index 0000000000..1ddb3e37da --- /dev/null +++ b/fluss-server/src/main/java/com/alibaba/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/com/alibaba/fluss/server/coordinator/event/watcher/TabletServerChangeWatcher.java b/fluss-server/src/main/java/com/alibaba/fluss/server/coordinator/event/watcher/TabletServerChangeWatcher.java index e44b61f7da..158d83904d 100644 --- a/fluss-server/src/main/java/com/alibaba/fluss/server/coordinator/event/watcher/TabletServerChangeWatcher.java +++ b/fluss-server/src/main/java/com/alibaba/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/com/alibaba/fluss/server/metrics/ServerMetricUtils.java b/fluss-server/src/main/java/com/alibaba/fluss/server/metrics/ServerMetricUtils.java index 89ad183387..0d0f137d5d 100644 --- a/fluss-server/src/main/java/com/alibaba/fluss/server/metrics/ServerMetricUtils.java +++ b/fluss-server/src/main/java/com/alibaba/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/com/alibaba/fluss/server/metrics/group/CoordinatorMetricGroup.java b/fluss-server/src/main/java/com/alibaba/fluss/server/metrics/group/CoordinatorMetricGroup.java index 88bcd46414..4c628dec50 100644 --- a/fluss-server/src/main/java/com/alibaba/fluss/server/metrics/group/CoordinatorMetricGroup.java +++ b/fluss-server/src/main/java/com/alibaba/fluss/server/metrics/group/CoordinatorMetricGroup.java @@ -30,10 +30,10 @@ public class CoordinatorMetricGroup extends AbstractMetricGroup { protected final String clusterId; protected final String hostname; - protected final String serverId; + protected final int serverId; 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; @@ -49,6 +49,6 @@ 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)); } } diff --git a/fluss-server/src/main/java/com/alibaba/fluss/server/zk/ZooKeeperClient.java b/fluss-server/src/main/java/com/alibaba/fluss/server/zk/ZooKeeperClient.java index 0782012120..b8a3150eca 100644 --- a/fluss-server/src/main/java/com/alibaba/fluss/server/zk/ZooKeeperClient.java +++ b/fluss-server/src/main/java/com/alibaba/fluss/server/zk/ZooKeeperClient.java @@ -46,7 +46,6 @@ import com.alibaba.fluss.server.zk.data.ZkData.BucketRemoteLogsZNode; import com.alibaba.fluss.server.zk.data.ZkData.BucketSnapshotIdZNode; import com.alibaba.fluss.server.zk.data.ZkData.BucketSnapshotsZNode; -import com.alibaba.fluss.server.zk.data.ZkData.CoordinatorZNode; import com.alibaba.fluss.server.zk.data.ZkData.DatabaseZNode; import com.alibaba.fluss.server.zk.data.ZkData.DatabasesZNode; import com.alibaba.fluss.server.zk.data.ZkData.LakeTableZNode; @@ -127,20 +126,46 @@ 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 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 = CoordinatorZNode.path(); + String path = ZkData.CoordinatorLeaderZNode.path(); zkClient.create() .creatingParentsIfNeeded() .withMode(CreateMode.EPHEMERAL) - .forPath(path, CoordinatorZNode.encode(coordinatorAddress)); - LOG.info("Registered leader {} at path {}.", coordinatorAddress, path); + .forPath(path, ZkData.CoordinatorLeaderZNode.encode(coordinatorAddress)); + LOG.info("Registered Coordinator leader {} at path {}.", coordinatorAddress, path); } /** Get the leader address registered in ZK. */ - public Optional getCoordinatorAddress() throws Exception { - Optional bytes = getOrEmpty(CoordinatorZNode.path()); - return bytes.map(CoordinatorZNode::decode); + 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 + 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(); } // -------------------------------------------------------------------------------------------- diff --git a/fluss-server/src/main/java/com/alibaba/fluss/server/zk/data/CoordinatorAddress.java b/fluss-server/src/main/java/com/alibaba/fluss/server/zk/data/CoordinatorAddress.java index c39a4860e1..ee2b236fc8 100644 --- a/fluss-server/src/main/java/com/alibaba/fluss/server/zk/data/CoordinatorAddress.java +++ b/fluss-server/src/main/java/com/alibaba/fluss/server/zk/data/CoordinatorAddress.java @@ -23,20 +23,20 @@ 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. */ 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/com/alibaba/fluss/server/zk/data/CoordinatorAddressJsonSerde.java b/fluss-server/src/main/java/com/alibaba/fluss/server/zk/data/CoordinatorAddressJsonSerde.java index 09302098a6..0711d0a27a 100644 --- a/fluss-server/src/main/java/com/alibaba/fluss/server/zk/data/CoordinatorAddressJsonSerde.java +++ b/fluss-server/src/main/java/com/alibaba/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/com/alibaba/fluss/server/zk/data/ZkData.java b/fluss-server/src/main/java/com/alibaba/fluss/server/zk/data/ZkData.java index a8259e3ccd..f2b689cfd5 100644 --- a/fluss-server/src/main/java/com/alibaba/fluss/server/zk/data/ZkData.java +++ b/fluss-server/src/main/java/com/alibaba/fluss/server/zk/data/ZkData.java @@ -248,13 +248,44 @@ 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 + */ + public static final class CoordinatorIdsZNode { + public static String path() { + return "/coordinators/ids"; + } + } + + /** + * The znode for a registered Coordinator information. The znode path is: * - *

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

/coordinators/ids/[serverId] */ - public static final class CoordinatorZNode { + public static final class CoordinatorIdZNode { + public static String path(int serverId) { + return CoordinatorIdsZNode.path() + "/" + 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: + * + *

/coordinators/leader + */ + public static final class CoordinatorLeaderZNode { public static String path() { return "/coordinators/active"; } @@ -268,7 +299,6 @@ public static CoordinatorAddress decode(byte[] json) { return JsonSerdeUtils.readValue(json, CoordinatorAddressJsonSerde.INSTANCE); } } - // ------------------------------------------------------------------------------------------ // ZNodes under "/tabletservers/" // ------------------------------------------------------------------------------------------ diff --git a/fluss-server/src/test/java/com/alibaba/fluss/server/ServerTestBase.java b/fluss-server/src/test/java/com/alibaba/fluss/server/ServerTestBase.java index 9a5a0da8e6..c56d6fa5e0 100644 --- a/fluss-server/src/test/java/com/alibaba/fluss/server/ServerTestBase.java +++ b/fluss-server/src/test/java/com/alibaba/fluss/server/ServerTestBase.java @@ -25,7 +25,7 @@ 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.ZkData.CoordinatorZNode; +import com.alibaba.fluss.server.zk.data.ZkData; import com.alibaba.fluss.server.zk.data.ZkData.ServerIdZNode; import com.alibaba.fluss.shaded.zookeeper3.org.apache.zookeeper.data.Stat; import com.alibaba.fluss.testutils.common.AllCallbackWrapper; @@ -95,7 +95,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(); @@ -143,6 +144,7 @@ protected void verifyEndpoint( } public static CoordinatorServer startCoordinatorServer(Configuration conf) throws Exception { + conf.set(ConfigOptions.COORDINATOR_ID, 0); CoordinatorServer coordinatorServer = new CoordinatorServer(conf); coordinatorServer.start(); return coordinatorServer; diff --git a/fluss-server/src/test/java/com/alibaba/fluss/server/coordinator/CoordinatorEventProcessorTest.java b/fluss-server/src/test/java/com/alibaba/fluss/server/coordinator/CoordinatorEventProcessorTest.java index b2ff021e93..f34e328a48 100644 --- a/fluss-server/src/test/java/com/alibaba/fluss/server/coordinator/CoordinatorEventProcessorTest.java +++ b/fluss-server/src/test/java/com/alibaba/fluss/server/coordinator/CoordinatorEventProcessorTest.java @@ -148,7 +148,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++) { @@ -309,7 +309,7 @@ void testServerBecomeOnlineAndOfflineLine() throws Exception { System.currentTimeMillis()); client.registerTabletServer(newlyServerId, tabletServerRegistration); - // retry until the tablet server register event is been handled + // retry until the tablet server register event has been handled retryVerifyContext( ctx -> assertThat(ctx.getLiveTabletServers()).containsKey(newlyServerId)); @@ -377,7 +377,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/com/alibaba/fluss/server/coordinator/CoordinatorServerElectionTest.java b/fluss-server/src/test/java/com/alibaba/fluss/server/coordinator/CoordinatorServerElectionTest.java new file mode 100644 index 0000000000..fb9516e8d1 --- /dev/null +++ b/fluss-server/src/test/java/com/alibaba/fluss/server/coordinator/CoordinatorServerElectionTest.java @@ -0,0 +1,129 @@ +/* + * 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.waitUntil; +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); + + waitUntilCoordinatorServerElected(); + + CoordinatorAddress firstLeaderAddress = zookeeperClient.getCoordinatorLeaderAddress().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. + waitUntilCoordinatorServerReelected(firstLeaderAddress); + CoordinatorAddress secondLeaderAddress = + zookeeperClient.getCoordinatorLeaderAddress().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 waitUntilCoordinatorServerElected() { + waitUntil( + () -> { + return zookeeperClient.getCoordinatorLeaderAddress().isPresent(); + }, + Duration.ofMinutes(1), + "Fail to wait coordinator server elected"); + } + + public void waitUntilCoordinatorServerReelected(CoordinatorAddress address) { + waitUntil( + () -> { + 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/com/alibaba/fluss/server/coordinator/CoordinatorServerITCase.java b/fluss-server/src/test/java/com/alibaba/fluss/server/coordinator/CoordinatorServerITCase.java index ddaeb4cc80..1cca53c55d 100644 --- a/fluss-server/src/test/java/com/alibaba/fluss/server/coordinator/CoordinatorServerITCase.java +++ b/fluss-server/src/test/java/com/alibaba/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/com/alibaba/fluss/server/coordinator/CoordinatorServerTest.java b/fluss-server/src/test/java/com/alibaba/fluss/server/coordinator/CoordinatorServerTest.java index 1855bf7142..43fb537c7d 100644 --- a/fluss-server/src/test/java/com/alibaba/fluss/server/coordinator/CoordinatorServerTest.java +++ b/fluss-server/src/test/java/com/alibaba/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); } @@ -63,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/com/alibaba/fluss/server/coordinator/event/watcher/CoordinatorServerChangeWatcherTest.java b/fluss-server/src/test/java/com/alibaba/fluss/server/coordinator/event/watcher/CoordinatorServerChangeWatcherTest.java new file mode 100644 index 0000000000..7101a4e9bb --- /dev/null +++ b/fluss-server/src/test/java/com/alibaba/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/com/alibaba/fluss/server/coordinator/event/watcher/TabletServerChangeWatcherTest.java b/fluss-server/src/test/java/com/alibaba/fluss/server/coordinator/event/watcher/TabletServerChangeWatcherTest.java index 70bf82ccf8..dfd75158a9 100644 --- a/fluss-server/src/test/java/com/alibaba/fluss/server/coordinator/event/watcher/TabletServerChangeWatcherTest.java +++ b/fluss-server/src/test/java/com/alibaba/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/com/alibaba/fluss/server/metrics/group/TestingMetricGroups.java b/fluss-server/src/test/java/com/alibaba/fluss/server/metrics/group/TestingMetricGroups.java index 0ef97e641c..3a0236d84c 100644 --- a/fluss-server/src/test/java/com/alibaba/fluss/server/metrics/group/TestingMetricGroups.java +++ b/fluss-server/src/test/java/com/alibaba/fluss/server/metrics/group/TestingMetricGroups.java @@ -28,7 +28,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 PhysicalTableMetricGroup TABLE_METRICS = new PhysicalTableMetricGroup( diff --git a/fluss-server/src/test/java/com/alibaba/fluss/server/testutils/FlussClusterExtension.java b/fluss-server/src/test/java/com/alibaba/fluss/server/testutils/FlussClusterExtension.java index 70ed70220f..2bcefc2f7e 100644 --- a/fluss-server/src/test/java/com/alibaba/fluss/server/testutils/FlussClusterExtension.java +++ b/fluss-server/src/test/java/com/alibaba/fluss/server/testutils/FlussClusterExtension.java @@ -249,6 +249,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/com/alibaba/fluss/server/zk/ZooKeeperClientTest.java b/fluss-server/src/test/java/com/alibaba/fluss/server/zk/ZooKeeperClientTest.java index 8f119f1afb..22dcb0b42f 100644 --- a/fluss-server/src/test/java/com/alibaba/fluss/server/zk/ZooKeeperClientTest.java +++ b/fluss-server/src/test/java/com/alibaba/fluss/server/zk/ZooKeeperClientTest.java @@ -94,14 +94,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")); + 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/fluss-server/src/test/java/com/alibaba/fluss/server/zk/data/CoordinatorAddressJsonSerdeTest.java b/fluss-server/src/test/java/com/alibaba/fluss/server/zk/data/CoordinatorAddressJsonSerdeTest.java index 98fb17858c..06c598e96a 100644 --- a/fluss-server/src/test/java/com/alibaba/fluss/server/zk/data/CoordinatorAddressJsonSerdeTest.java +++ b/fluss-server/src/test/java/com/alibaba/fluss/server/zk/data/CoordinatorAddressJsonSerdeTest.java @@ -39,7 +39,7 @@ public class CoordinatorAddressJsonSerdeTest extends JsonSerdeTestBase - 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.