HDFS Multiple Standby原理分析

前言


HDFS在早期实现HA时,是标准的一主一备的服务模式,主的叫Active NameNode,备的叫Standby NameNode。Standby/Active NN间可以互相切换以此达到服务高可用的目的。但是这种双节点的HA模式是否能够满足更高的高可用性的要求呢?在标准的HA模式下,其实只有1个Standby的NN作为bak来使用。假设在极端情况下,Active和Stanby同时出现crash的情况(这种概率在实际生产环境中也是有几率发生的),那么此时HDFS集群将处于不能服务的状态。因此,我们是否能够实现一种具有更高HA特性的服务模式呢,比如1个Active Service+多Standby的运行模式?倘若我们有了多Standby的支持,毫无疑问将会大大提高NN服务的高可用性。目前社区在3.x版本中已经实现了此功能,相关JIRA HDFS-6440(Support more than 2 NameNodes)。本文笔者对此功能做一个简单的原理分析,在了解了其原理实现后,能够帮助我们更好地去使用这个功能特性。

HDFS Multiple Standby的实现要素


首先我们要知道实现HDFS Multiple Standby的前提是基于HDFS原有的HA实现之上的。它只是将原有的Active-Standby模式扩展为Active-多Standby模式。

从Single Standby到Multiple Standby的关系转变上,有一个核心的点是不变的:Active和Standby服务之间的交互行为。简单来说,就是原本Active和Standby NN服务之间的交互通信理应是不变的。在HA核心代码实现上我们并不需要做额外的改动。

因此在这里面,我们要重点关注以下Active、Standby NN之间的交互行为:

  • NN Bootstrap行为
  • Standby NN的checkpoint upload到Active NN的行为
  • Active/Standby NN间的failover行为
  • Standby NN向ActiveNN发起的log roll操作行为

上述四类操作在原有1对1的Active/Standby模式时,是比较简单的单向直接调用行为。但是在多Standby时,这个时候target会变成多个,而且这些 target的状态事先也不确定,可能是Standby也可能是Active。这部分也是多Standby实现的一个重点也是难点。

下面我们结合实际场景代码做逐一分析。

Multiple Standby实现分析


此小节部分我们主要对上面提到的4个交互行为做具体分析。

Bootstrap行为


在单Standby模式里,Standby NN的直接向另外一个Active NN发起download image的行为即可。但是在多Standby时,操作行为转变为:只需寻找到一个正在服务的NN进行bootstrp即可,不管这个NN是真正的Active还是Standby。因为bootstrap行为只是去NN的metadata文件数据。

相关代码如下:
BootstrapStandby.java

  private int doRun() throws IOException {
   
    // find the active NN
    NamenodeProtocol proxy = null;
    NamespaceInfo nsInfo = null;
    boolean isUpgradeFinalized = false;
    RemoteNameNodeInfo proxyInfo = null;
    for (int i = 0; i < remoteNNs.size(); i++) {
   
      proxyInfo = remoteNNs.get(i);
      InetSocketAddress otherIpcAddress = proxyInfo.getIpcAddress();
      proxy = createNNProtocolProxy(otherIpcAddress);
      try {
   
        // Get the namespace from any active NN. If you just formatted the primary NN and are
        // bootstrapping the other NNs from that layout, it will only contact the single NN.
        // However, if there cluster is already running and you are adding a NN later (e.g.
        // replacing a failed NN), then this will bootstrap from any node in the cluster.
        nsInfo = proxy.versionRequest();
        isUpgradeFinalized = proxy.isUpgradeFinalized();
        // 寻找到一个可用的NN即可
        break;
      } catch (IOException ioe) {
   
        LOG.warn("Unable to fetch namespace information from remote NN at " + otherIpcAddress
            + ": " + ioe.getMessage());
        if (LOG.isDebugEnabled()) {
   
          LOG.debug("Full exception trace", ioe);
        }
      }
    }

    ...

    // download the fsimage from active namenode
    int download = downloadImage(storage, proxy, proxyInfo);
    if (download != 0) {
   
      return download;
    }

    // finish the upgrade: rename previous.tmp to previous
    ...
    return 0;
  }

Standby NN的checkpoint upload行为


我们知道在HDFS HA机制里,Standby会定期checkpoint出一个新的image,然后upload到Active NN中。那么在多Standby NN的情况下,需要解决下面2点主要问题:

  • Standby NN如何找到谁是真正的Active NN,然后再进行image的upload。
  • 找到谁是真正的Active NN后,各个Standby如何协调进行image的upload,这里面会存在潜在的冲突问题。

对于第一个问题,社区的解决思路是假定所有的NN都是潜在的Active NN(里面肯定包括Standby NN)。然后检查每个NN上次发送成功image的状态信息,这个状态信息主要记录以下2点信息:

1)是否是Active节点,这个是基于上次image upload结果发现的,如果后面发生了failover,这个属性值会被更新。
2)上次image upload成功后的时间。

检查的条件:要么target NN是Active节点,要么或者是距上次image upload时间超过指定阈值范围内的即可。相关代码如下:

StandbyCheckpointer.java

  private void doCheckpoint() throws InterruptedException, IOException {
   
    assert canceler != null;
    final long txid;
    final NameNodeFile imageType;
       ...
      // checkpoint操作生成新的image文件
      img.saveNamespace(namesystem, imageType, canceler);
      txid = img.getStorage().getMostRecentCheckpointTxId();
      assert txid == thisCheckpointTxId : "expected to save checkpoint at txid=" +
          thisCheckpointTxId + " but instead saved at txid=" + txid;

      ...

    // Upload the saved checkpoint back to the active
    // Do this in a separate thread to avoid blocking transition to active, but don't allow more
    // than the expected number of tasks to run or queue up
    // See HDFS-4816
    ExecutorService executor = new ThreadPoolExecutor(0, activeNNAddresses.size(), 100,
        TimeUnit.MILLISECONDS, new LinkedBlockingQueue<Runnable>(activeNNAddresses.size()),
        uploadThreadFactory);
    // for right now, just match the upload to the nn address by convention. There is no need to
    // directly tie them together by adding a pair class.
    HashMap<String, Future<TransferFsImage.TransferResult>> uploads =
        new HashMap<>();
    for (final URL activeNNAddress : activeNNAddresses) {
   
      // Upload image if at least 1 of 2 following conditions met:
      // 1. has been quiet for long enough, try to contact the node.
      // 2. this standby IS the primary checkpointer of target NN.
      String addressString = activeNNAddress.toString();
      assert checkpointReceivers.containsKey(addressString);
      // 1)获取目标NN上次checkpoint的状态
      CheckpointReceiverEntry receiverEntry =
          checkpointReceivers.get(addressString);
      long secsSinceLastUpload =
          TimeUnit.MILLISECONDS.toSeconds(
              monotonicNow() - receiverEntry.getLastUploadTime());
      // 2)如果目标NN是Active或者上次checkpoint时间超出阈值外,则应该upload checkpoint到此NN
      boolean shouldUpload = receiverEntry.isPrimary() ||
          secsSinceLastUpload >= checkpointConf.getQuietPeriod();
      if (shouldUpload) {
   
        // 3)提交执行upload image到指定NN的任务
        Future<TransferFsImage.TransferResult> upload =
            execut
  • 3
    点赞
  • 2
    收藏
    觉得还不错? 一键收藏
  • 1
    评论

“相关推荐”对你有帮助么?

  • 非常没帮助
  • 没帮助
  • 一般
  • 有帮助
  • 非常有帮助
提交
评论 1
添加红包

请填写红包祝福语或标题

红包个数最小为10个

红包金额最低5元

当前余额3.43前往充值 >
需支付:10.00
成就一亿技术人!
领取后你会自动成为博主和红包主的粉丝 规则
hope_wisdom
发出的红包
实付
使用余额支付
点击重新获取
扫码支付
钱包余额 0

抵扣说明:

1.余额是钱包充值的虚拟货币,按照1:1的比例进行支付金额的抵扣。
2.余额无法直接购买下载,可以购买VIP、付费专栏及课程。

余额充值