Skip to content

[server] Coordinator Server Supports High-Available #1401

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 7 commits into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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<Integer> COORDINATOR_ID =
key("coordinator.id")
.intType()
.noDefaultValue()
.withDescription("The id for the coordinator server.");

public static final ConfigOption<Integer> COORDINATOR_IO_POOL_SIZE =
key("coordinator.io-pool.size")
.intType()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,7 @@ public class CoordinatorContext {
// a success deletion.
private final Map<TableBucketReplica, Integer> failDeleteNumbers = new HashMap<>();

private final Set<Integer> liveCoordinatorServers = new HashSet<>();
private final Map<Integer, ServerInfo> liveTabletServers = new HashMap<>();

// a map from the table bucket to the state of the bucket.
Expand Down Expand Up @@ -110,6 +111,24 @@ public int getCoordinatorEpoch() {
return coordinatorEpoch;
}

public Set<Integer> getLiveCoordinatorServers() {
return liveCoordinatorServers;
}

@VisibleForTesting
public void setLiveCoordinatorServers(Set<Integer> 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<Integer, ServerInfo> getLiveTabletServers() {
return liveTabletServers;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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;
Expand Down Expand Up @@ -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;
Expand Down Expand Up @@ -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);
Expand All @@ -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.");
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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<ServerInfo> tabletServerInfos = new ArrayList<>();
Expand Down Expand Up @@ -419,6 +433,7 @@ private void onShutdown() {
tableManager.shutdown();

// then stop watchers
coordinatorServerChangeWatcher.stop();
tableChangeWatcher.stop();
tabletServerChangeWatcher.stop();
}
Expand All @@ -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) {
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -696,10 +720,33 @@ private void onReplicaBecomeOffline(Set<TableBucketReplica> 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
Expand All @@ -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;
}
Expand Down
Original file line number Diff line number Diff line change
@@ -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();
}
}
Loading
Loading