zk session expire会引起HA模式的rm一直处于standby吗
【概述】最近连续在多个环境中遇到了同一个问题:在HA模式下,两个resourcemanager均为standby,并且持续没有选举出新的leader。经过一番分析,并对照源码梳理问题出现前后的逻辑流程,最后发现是因为zk会话过期(session expire)引起的问题,本文就复盘总结下。
【RM的正常选举流程】
在很早之前的文章中,介绍过hadoop里namenode的HA机制(戳这里),RM的选举流程其实是复用了同样的框架,只是以一个独立线程的方式运行,而不是像namenode一样,有个独立的进程(zkfc)负责与zk连接并选举。
因此,整体的选举流程会和namenode的选举方式基本雷同,即首先向zk建立连接,当连接建立成功后,在zk上竞争创建临时锁节点,成功创建的rm成为active,失败的则成为standby。
【与zk之间网络异常后的情况】
正常逻辑是相对简单的,那我们再来看看与zk之间网络出现异常,以及网络异常恢复之后的处理逻辑,具体如下图所示:
1. 当ZK服务出现故障,或者网络出现故障,导致网络完全不可达时,客户端与ZK的连接会出现在指定时间内没有读到任何数据,从而引发会话超时。(也可能是读异常,此时产生的是EndOfStreamException,后续处理逻辑与会话超时的逻辑一样)。
这个时候,zk客户端的发送线程会抛会话超时的异常,同时内部捕获该异常, 向事件回调线程的队列中插入连接断开的事件。此后,循环执行与zk的重连动作。
while (state.isAlive()) {
try {
...
if (to <= 0) {
String warnInfo;
warnInfo =
"Client session timed out, have not heard from server in " +
clientCnxnSocket.getIdleRecv() + "ms" +
" for sessionid 0x" + Long.toHexString(sessionId);
LOG.warn(warnInfo);
throw new SessionTimeoutException(warnInfo);
}
} catch (Throwable e) {
...
if (state.isAlive()) {
eventThread.queueEvent(
new WatchedEvent(Event.EventType.None, Event.KeeperState.Disconnected, null));
}
...
}
}
2. zk客户端中的事件回调线程接收到事件后,向上进行回调通知。在RM的回调处理中,启动定时器线程,触发成为standby。
synchronized void processWatchEvent(ZooKeeper zk, WatchedEvent event) {
...
if (eventType == Event.EventType.None) {
switch (event.getState()) {
case Disconnected:
LOG.info("Session disconnected. Entering neutral mode...");
zkConnectionState = ConnectionState.DISCONNECTED;
enterNeutralMode();
break;
...
}
}
}
private void enterNeutralMode() {
if (state != State.NEUTRAL) {
if (LOG.isDebugEnabled()) {
LOG.debug("Entering neutral mode for " + this);
}
state = State.NEUTRAL;
appClient.enterNeutralMode();
}
}
public void enterNeutralMode() {
LOG.warn("Lost contact with Zookeeper. Transitioning to standby in "
+ zkSessionTimeout + " ms if connection is not reestablished.");
// If we've just become disconnected, start a timer. When the time's up,
// we'll transition to standby.
synchronized (zkDisconnectLock) {
if (zkDisconnectTimer == null) {
zkDisconnectTimer = new Timer("Zookeeper disconnect timer");
zkDisconnectTimer.schedule(new TimerTask() {
@Override
public void run() {
synchronized (zkDisconnectLock) {
// Only run if the timer hasn't been cancelled
if (zkDisconnectTimer != null) {
becomeStandby();
}
}
}
}, zkSessionTimeout);
}
}
}
3. 当网络恢复后,ZK客户端重连成功, 但仍旧是携带老的会话ID发送注册请求,如果重连时间超过了会话过期的时间,那么服务端会给出相应应答,告知会话过期,同时断开连接。
此时,ZK客户端内部发送线程会从响应中得到知道会话过期,向事件线程发送会话过期事件以及线程退出事件,同时将自身状态置为CLOSED,并抛出异常,这样发送线程也就会退出循环从而结束运行。
void onConnected(
int _negotiatedSessionTimeout,
long _sessionId,
byte[] _sessionPasswd,
boolean isRO)
throws IOException {
negotiatedSessionTimeout = _negotiatedSessionTimeout;
if(negotiatedSessionTimeout <= 0) {
state = States.CLOSED;
eventThread.queueEvent(new WatchedEvent(
Watcher.Event.EventType.None,
Watcher.Event.KeeperState.Expired, null));
eventThread.queueEventOfDeath();
String warnInfo;
warnInfo =
"Unable to reconnect to ZooKeeper service, session 0x" +
Long.toHexString(sessionId) + " has expired";
LOG.warn(warnInfo);
throw new SessionExpiredException(warnInfo);
}
}
4. 在会话过期的回调处理中,修改自身状态,并重新参与选举,这包括关闭当前的客户端,重新创建新的zk客户端进行连接,如果能成功连接,则继续创建锁节点来进行leader的选举。
synchronized void processWatchEvent(ZooKeeper zk, WatchedEvent event) {
...
if (eventType == Event.EventType.None) {
switch (event.getState()) {
case Expired:
LOG.info("Session expired. Entering neutral mode and rejoining...");
enterNeutralMode();
reJoinElection(0);
break;
...
}
}
}
private void reJoinElection(int sleepTime) {
LOG.info("Trying to re-establish ZK session");
sessionReestablishLockForTests.lock();
try {
terminateConnection();
sleepFor(sleepTime);
// Should not join election even before the SERVICE is reported
// as HEALTHY from ZKFC monitoring.
if (appData != null) {
joinElectionInternal();
} else {
LOG.info("Not joining election since service has not yet been " +
"reported as healthy.");
}
} finally {
sessionReestablishLockForTests.unlock();
}
}
private void joinElectionInternal() {
Preconditions.checkState(appData != null,
"trying to join election without any app data");
if (zkClient == null) {
if (!reEstablishSession()) {
fatalError("Failed to reEstablish connection with ZooKeeper");
return;
}
}
createRetryCount = 0;
wantToBeInElection = true;
createLockNodeAsync();
}
对于standby的RM,其完整的日志如下所示:
// 超时会接收到任何数据
2022-09-01 19:10:25,230 WARN org.apache.zookeeper.ClientCnxn: Client session timed out, have not heard from server in 6668ms for sessionid 0x10054aa9d110000
// 异常捕获
2022-09-01 19:10:25,230 INFO org.apache.zookeeper.ClientCnxn: Client session timed out, have not heard from server in 6668ms for sessionid 0x10054aa9d110000, closing socket connection and attempting reconnect
// RM的回调处理
2022-09-01 19:10:25,331 INFO org.apache.hadoop.ha.ActiveStandbyElector: Session disconnected. Entering neutral mode...
// 触发定时器线程
2022-09-01 19:10:25,331 WARN org.apache.hadoop.yarn.server.resourcemanager.ActiveStandbyElectorBasedElectorService: Lost contact with Zookeeper. Transitioning to standby in 10000 ms if connection is not reestablished.
// ZK客户端的发送线程尝试重连
2022-09-01 19:10:26,905 INFO org.apache.zookeeper.ClientCnxn: Opening socket connection to server zk-0-hncscwc.network-hncscwc/172.168.1.1:2181. Will not attempt to authenticate using SASL (unknown error)
// 定时器线程触发进行状态的状态, 但当前状态已经是standby状态
2022-09-01 19:10:35,334 INFO org.apache.hadoop.yarn.server.resourcemanager.ResourceManager: Already in standby state
// 重连成功
2022-09-01 19:13:51,101 INFO org.apache.zookeeper.ClientCnxn: Socket connection established to zk-0-hncscwc.network-hncscwc/172.168.1.1:2181, initiating session
// 会话过期, 向事件回调线程队列插入会话过期的事件
2022-09-01 19:13:51,104 WARN org.apache.zookeeper.ClientCnxn: Unable to reconnect to ZooKeeper service, session 0x10054aa9d110000 has expired
// 回调处理, 并触发重新选举
2022-09-01 19:13:51,104 INFO org.apache.hadoop.ha.ActiveStandbyElector: Session expired. Entering neutral mode and rejoining...
// 发送线程捕获异常
2022-09-01 19:13:51,105 INFO org.apache.zookeeper.ClientCnxn: Unable to reconnect to ZooKeeper service, session 0x10054aa9d110000 has expired, closing socket connection
// 重新建立连接并进行选举
2022-09-01 19:13:51,105 INFO org.apache.hadoop.ha.ActiveStandbyElector: Trying to re-establish ZK session
2022-09-01 19:13:51,109 INFO org.apache.zookeeper.ClientCnxn: Socket connection established to zk-0-hncscwc.network-hncscwc/172.168.1.1:2181, initiating session
// 成功建立连接(注意会话ID不同)
2022-09-01 19:13:51,122 INFO org.apache.zookeeper.ClientCnxn: Session establishment complete on server zk-0-hncscwc.network-hncscwc/172.168.1.1:2181, sessionid = 0x10054aa9d110006, negotiated timeout = 10000
// 连接成功建立的回调
2022-09-01 19:13:51,123 INFO org.apache.hadoop.ha.ActiveStandbyElector: Session connected.
【极端情况引起的BUG】
上面的逻辑分析中提到了,首次重连ZK后触发会话过期后,会重新创建新的客户端进行重连动作(毕竟老的会话已经过期,无法再继续使用)。通常情况下, 网络都是相对稳定的,创建新的客户端连接肯定可以重连成功,毕竟这一系列动作是连贯的,中间没有任何睡眠操作。
但如果真的有极端情况,会话过期后重连ZK失败,并且达到重连的最大次数后仍旧未成功连接ZK。那么此时,会再向上层回调一个致命错误,对于这类型错误的处理,则是创建一个线程先进行standby状态的转换,然后再进行重新选举的动作。
在这个线程中,会对一个原子变量进行判断(初始值为false)。如果为false,表示当前没有线程在执行这个动作,将该变量置为true,然后进行后续动作。
然而,这个地方,也是BUG所在的地方。在这个线程中重新进行选举,其逻辑和之前一样,依旧是先尝试连接ZK,如果持续无法连接到ZK,并且达到最大重连次数,则再触发回调,创建新线程进行后续逻辑。但此时,原子变量的值已经被置为true,新的线程运行后,判断该值为true,则直接退出。此后就没有机会再进行与ZK的重连动作了。
对应的代码如下所示:
private void fatalError(String errorMessage) {
LOG.error(errorMessage);
reset();
appClient.notifyFatalError(errorMessage);
}
public void notifyFatalError(String errorMessage) {
rm.getRMContext().getDispatcher().getEventHandler().handle(
new RMFatalEvent(RMFatalEventType.EMBEDDED_ELECTOR_FAILED,
errorMessage));
}
private class RMFatalEventDispatcher implements EventHandler<RMFatalEvent> {
@Override
public void handle(RMFatalEvent event) {
LOG.error("Received " + event);
if (HAUtil.isHAEnabled(getConfig())) {
LOG.warn("Transitioning the resource manager to standby.");
handleTransitionToStandByInNewThread();
}
...
}
}
private void handleTransitionToStandByInNewThread() {
Thread standByTransitionThread =
new Thread(activeServices.standByTransitionRunnable);
standByTransitionThread.setName("StandByTransitionThread");
standByTransitionThread.start();
}
private class StandByTransitionRunnable implements Runnable {
// The atomic variable to make sure multiple threads with the same runnable
// run only once.
private final AtomicBoolean hasAlreadyRun = new AtomicBoolean(false);
@Override
public void run() {
// Run this only once, even if multiple threads end up triggering
// this simultaneously.
if (hasAlreadyRun.getAndSet(true)) {
return;
}
if (rmContext.isHAEnabled()) {
try {
// Transition to standby and reinit active services
LOG.info("Transitioning RM to Standby mode");
transitionToStandby(true);
EmbeddedElector elector = rmContext.getLeaderElectorService();
if (elector != null) {
elector.rejoinElection();
}
} catch (Exception e) {
LOG.fatal("Failed to transition RM to Standby mode.", e);
ExitUtil.terminate(1, e);
}
}
}
}
在线程中进行状态转换的过程中,有个细节需要注意:
如果进行转换时,RM的当前状态为active,那么此时会停止activeService并重新初始化,即重新创建一个新的实例对象出来。而前面的原子变量,也会随着新的实例对象重新被赋值为false。
synchronized void transitionToStandby(boolean initialize)
throws Exception {
if (rmContext.getHAServiceState() ==
HAServiceProtocol.HAServiceState.STANDBY) {
LOG.info("Already in standby state");
return;
}
LOG.info("Transitioning to standby state");
HAServiceState state = rmContext.getHAServiceState();
rmContext.setHAServiceState(HAServiceProtocol.HAServiceState.STANDBY);
if (state == HAServiceProtocol.HAServiceState.ACTIVE) {
stopActiveServices();
reinitialize(initialize);
}
LOG.info("Transitioned to standby state");
}
void reinitialize(boolean initialize) {
ClusterMetrics.destroy();
QueueMetrics.clearQueueMetrics();
getResourceScheduler().resetSchedulerMetrics();
if (initialize) {
resetRMContext();
createAndInitActiveServices(true);
}
}
protected void createAndInitActiveServices(boolean fromActive) {
activeServices = new RMActiveServices(this);
activeServices.fromActive = fromActive;
activeServices.init(conf);
}
同时,此时会走重新初始化建立连接的逻辑流程,因此,这里是可以正确进行重连。但此后,active的状态切换为standby,在未成为active之前,如果继续出现会话过期后的重连ZK失败,那么仍旧会出现无法再重连zk的问题。
【可以稳定复现的方式】
清楚问题产生的场景后,也就能比较容易的进行问题复现了,我们可以通过iptables丢弃从zk过来的数据包进行模拟。例如在与ZK的连接断开一段时间后,再执行下面的脚本命令,这样,问题现象大概率就复现出来了。
#!/bin/bash
# 恢复网络
iptables -F
# 短暂睡眠,使其可以重连成功
sleep 0.3
# 再次模拟与ZK的网络异常
iptables -A INPUT -p tcp --sport 2181 -j DROP
【问题解决】
问题的解决其实也很简单,比如去除原子布尔变量的判断逻辑,同时在后续的执行动作中加锁保护,避免多线程并发操作;另一种更简单的方式是启用curator框架,新版本中大多引入了该框架,只是默认为false,即没有使用,可以配置使用该框架,也能对这个问题进行规避。
好了,这就是本文的全部内容,如果觉得本文对您有帮助,请点赞+转发,也欢迎加我微信交流~
本文分享自微信公众号 - hncscwc(gh_383bc7486c1a)。
如有侵权,请联系 support@oschina.cn 删除。
本文参与“OSC源创计划”,欢迎正在阅读的你也加入,一起分享。
https://my.oschina.net/hncscwc/blog/5573506
页:
[1]