前往小程序,Get更优阅读体验!
立即前往
首页
学习
活动
专区
工具
TVP
发布
社区首页 >专栏 >zk session expire会引起HA模式的rm一直处于standby吗

zk session expire会引起HA模式的rm一直处于standby吗

作者头像
陈猿解码
发布2023-02-28 15:06:52
5030
发布2023-02-28 15:06:52
举报
文章被收录于专栏:陈猿解码陈猿解码

【概述】


最近连续在多个环境中遇到了同一个问题:在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的重连动作。

代码语言:javascript
复制
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。

代码语言:javascript
复制
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,并抛出异常,这样发送线程也就会退出循环从而结束运行。

代码语言:javascript
复制
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的选举。

代码语言:javascript
复制
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,其完整的日志如下所示:

代码语言:javascript
复制
// 超时会接收到任何数据
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的重连动作了。

对应的代码如下所示:

代码语言:javascript
复制
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。

代码语言:javascript
复制
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的连接断开一段时间后,再执行下面的脚本命令,这样,问题现象大概率就复现出来了。

代码语言:javascript
复制
#!/bin/bash
# 恢复网络
iptables -F
# 短暂睡眠,使其可以重连成功
sleep 0.3
# 再次模拟与ZK的网络异常
iptables -A INPUT -p tcp --sport 2181 -j DROP

【问题解决】

问题的解决其实也很简单,比如去除原子布尔变量的判断逻辑,同时在后续的执行动作中加锁保护,避免多线程并发操作;另一种更简单的方式是启用curator框架,新版本中大多引入了该框架,只是默认为false,即没有使用,可以配置使用该框架,也能对这个问题进行规避。

本文参与 腾讯云自媒体同步曝光计划,分享自微信公众号。
原始发表:2022-09-03,如有侵权请联系 cloudcommunity@tencent.com 删除

本文分享自 陈猿解码 微信公众号,前往查看

如有侵权,请联系 cloudcommunity@tencent.com 删除。

本文参与 腾讯云自媒体同步曝光计划  ,欢迎热爱写作的你一起参与!

评论
登录后参与评论
0 条评论
热度
最新
推荐阅读
领券
问题归档专栏文章快讯文章归档关键词归档开发者手册归档开发者手册 Section 归档