HDFS NAMENODE QJM HA方案实现-1

问题背景

因为线上集群扩容,扩容后的NameNode每隔几个小时出现告警,检查NameNode日志,发现RPC 请求会出现上百秒的卡顿。每次告警都是 NameNode在回传 FSImage文件(大小9G左右)。经过源码跟踪,FSImage在回传的时候会占用单磁盘的全部IO,同时这块磁盘上配置了 EditLog目录,造成EditLog日志无法正常Flush,RPC卡死。
解决方法:

  • 将 FSImage目录和 EditLog目录分别使用单独的磁盘进行存储,分散IO
  • 改用SSD磁盘来加速 NameNode IO
2019-05-13 18:19:06,445 INFO org.apache.hadoop.hdfs.server.namenode.TransferFsImage: Transfer took 172.50s at 54678.20 KB/s
2019-05-13 18:19:06,445 INFO org.apache.hadoop.hdfs.server.namenode.TransferFsImage: Downloaded file fsimage.ckpt_0000000021366706701 size 9658525975 bytes.

2019-05-13 18:19:07,493 INFO org.apache.hadoop.hdfs.server.namenode.FSNamesystem: FSNamesystem write lock held for 89620 ms via
java.lang.Thread.getStackTrace(Thread.java:1559)
org.apache.hadoop.util.StringUtils.getStackTrace(StringUtils.java:945)
org.apache.hadoop.hdfs.server.namenode.FSNamesystemLock.writeUnlock(FSNamesystemLock.java:195)
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.writeUnlock(FSNamesystem.java:1655)
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.rollEditLog(FSNamesystem.java:6549)
org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.rollEditLog(NameNodeRpcServer.java:1009)
org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolServerSideTranslatorPB.rollEditLog(NamenodeProtocolServerSideTranslatorPB.java:142)
org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos$NamenodeProtocolService$2.callBlockingMethod(NamenodeProtocolProtos.java:12025)
org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:617)
org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1073)
org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2217)
org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2213)
java.security.AccessController.doPrivileged(Native Method)
javax.security.auth.Subject.doAs(Subject.java:422)
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1917)
org.apache.hadoop.ipc.Server$Handler.run(Server.java:2211)
        Number of suppressed write-lock reports: 0
        Longest write-lock held interval: 89620

JournalNode

  • main()
  • run()
  • start()
    • 启动 httpServer
      httpServer 启动服务 GetJournalEditServlet

    • 启动rpcServer

this.server = new RPC.Builder(confCopy)
.setProtocol(QJournalProtocolPB.class)
.setInstance(service)
.setBindAddress(addr.getHostName())
.setPort(addr.getPort())
.setNumHandlers(HANDLER_COUNT)
.setVerbose(false)
.build();

对应的proto协议类

JournalProtocol.proto
QJournalProtocol.proto

QJM 方案对象实现过程

确定 EditsDirs

<!-- hdfs-default.xml dfs.namenode.edits.dir 的默认值就是 dfs.namenode.name.dir 参数值 -->
<property>
  <name>dfs.namenode.edits.dir</name>
  <value>${dfs.namenode.name.dir}</value>
  <description>Determines where on the local filesystem the DFS name node
      should store the transaction (edits) file. If this is a comma-delimited list
      of directories then the transaction file is replicated in all of the 
      directories, for redundancy. Default value is same as dfs.namenode.name.dir
  </description>
</property>

NameNode初始化

// NameNode启动
NameNode : main()
NameNode : createNameNode()
NameNode : return new NameNode(conf);
NameNode : initialize(conf);
NameNode : loadNamesystem(conf)
FSNamesystem : loadFromDisk(conf)

实例 FSImage 和 FSEditLog

  // NameNode 启动时,在加载FSImage文件的时候初始化,相关配置目录
  // sharedDirs : FSNamesystem.getNamespaceEditsDirs(conf) 
  //    - < dfs.namenode.shared.edits.dir : qjournal://node1:8485;node2:8485;node3:8485/mycluster >
  //    - < dfs.namenode.edits.dir : ${dfs.namenode.name.dir}> 
  FSImage fsImage = new FSImage(conf,
        FSNamesystem.getNamespaceDirs(conf),
        FSNamesystem.getNamespaceEditsDirs(conf));

  // FSImage.java  protected FSImage()
  this.editLog = FSEditLog.newInstance(conf, storage, editsDirs);

  // FSEditLog.java
  FSEditLog(Configuration conf, NNStorage storage, List<URI> editsDirs) {
      this.editsDirs = Lists.newArrayList(editsDirs);
      this.sharedEditsDirs = FSNamesystem.getSharedEditsDirs(conf);
  }

NameNode服务启动和journalSet 实例初始化

  NameNode  public NameNode(Configuration conf)
  ActiveState  public void enterState(HAContext context)
  NameNodeHAContext public void startActiveServices()
  FSNamesystem  void startActiveServices()
  FSEditLog public synchronized void initJournalsForWrite()
  FSEditLog private synchronized void initJournals(List<URI> dirs) 

journalSet 初始化

journalSet 代表一组 JournalAndStream(JournalManager),一般要包含一个FileJournalManager和一个QuorumJournalManager

  • FileJournalManager :dfs.namenode.edits.dir 会有一个默认目录,然后在该目录下创建一个FileJournalManager

  • QuorumJournalManager : 通过解析的dfs.namenode.shared.edits.dir 来创建一个管理 journals集群的QuorumJournalManager

  • JournalAndStream

    • 内部包含一个 JournalManager 和 一个 EditLogOutputStream 。
    • JournalManager journal 在构造器中初始化,
    • EditLogOutputStream 通过 jas.startLogSegment() 调用 journal.startLogSegment() 初始化

JournalManager 本身设计是支持在各种存储上代码实现。所以这里使用了一个小技巧,通过在 createJournal() 方法中使用自己配置的协议类schema 来动态得到JournalManager实现类,再取初始化。

举例:

  • 配置参数 dfs.namenode.shared.edits.dir = qjournal://node1:8485;node2:8485;node3:8485/mycluster
  • 计算得到 schema : qjournal
  • 到 configuration 中找 dfs.namenode.edits.journal-plugin.qjournal 对应的class 类
<property>
  <name>dfs.namenode.edits.journal-plugin.qjournal</name>
  <value>org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager</value>
</property>
  private synchronized void initJournals(List<URI> dirs) {
      // minimumRedundantJournals 默认值1,要求至少要有JAS 可以使用
      journalSet = new JournalSet(minimumRedundantJournals);

      for (URI u : dirs) {
        boolean required = FSNamesystem.getRequiredNamespaceEditsDirs(conf)
            .contains(u);
        if (u.getScheme().equals(NNStorage.LOCAL_URI_SCHEME)) {
          StorageDirectory sd = storage.getStorageDirectory(u);
          if (sd != null) {
            journalSet.add(new FileJournalManager(conf, sd, storage),
                required, sharedEditsDirs.contains(u));
          }
        } else {
          journalSet.add(createJournal(u), required,
              sharedEditsDirs.contains(u));
        }
      }
  }

  /**
   * Construct a custom journal manager.
   * The class to construct is taken from the configuration.
   * @param uri Uri to construct
   * @return The constructed journal manager
   * @throws IllegalArgumentException if no class is configured for uri
   */
  private JournalManager createJournal(URI uri) {
    Class<? extends JournalManager> clazz
      = getJournalClass(conf, uri.getScheme());

    try {
      Constructor<? extends JournalManager> cons
        = clazz.getConstructor(Configuration.class, URI.class,
            NamespaceInfo.class);
      return cons.newInstance(conf, uri, storage.getNamespaceInfo());
    } catch (Exception e) {
      throw new IllegalArgumentException("Unable to construct journal, "
                                         + uri, e);
    }
  }

JournalManager / QuorumJournalManager 初始化

QuorumJournalManager内部主要管理对所有AsyncLogger loggers 的rpc连接。
创建过程 : 解析配置文件中的journalnodes的URI,并和每个journalnode建立RPC socket proxy

  • 对 QuorumJournalManager 类实例化一个JournalManager,来管理node1,node2,nod3 对应的三个 AsyncLogger
  public QuorumJournalManager(Configuration conf,
      URI uri, NamespaceInfo nsInfo) throws IOException {
    this(conf, uri, nsInfo, IPCLoggerChannel.FACTORY);
  }

AsyncLoggerSet

wrapper around a set of logger

下面这些方法会对内部的logger依次调用对应logger的方法

  • void setEpoch()
  • void setCommittedTxId()
  • void close()
  this.loggers = new AsyncLoggerSet(createLoggers(loggerFactory));

  static List<AsyncLogger> createLoggers(Configuration conf,
      URI uri, NamespaceInfo nsInfo, AsyncLogger.Factory factory)
          throws IOException {
    List<AsyncLogger> ret = Lists.newArrayList();
    List<InetSocketAddress> addrs = getLoggerAddresses(uri);
    String jid = parseJournalId(uri);
    for (InetSocketAddress addr : addrs) {
      ret.add(factory.createLogger(conf, nsInfo, jid, addr));
    }
    return ret;
  }

  static final Factory FACTORY = new AsyncLogger.Factory() {
    @Override
    public AsyncLogger createLogger(Configuration conf, NamespaceInfo nsInfo,
        String journalId, InetSocketAddress addr) {
      return new IPCLoggerChannel(conf, nsInfo, journalId, addr);
    }
  };

下面这些方法都是用来生成QuorumCall,并使用 waitForWriteQuorum() 方法进行调用,例如

      QuorumCall<AsyncLogger, Void> qcall = loggers.sendEdits(
          segmentTxId, firstTxToFlush,
          numReadyTxns, data);
      loggers.waitForWriteQuorum(qcall, writeTimeoutMs, "sendEdits");

      // Since we successfully wrote this batch, let the loggers know. Any future
      // RPCs will thus let the loggers know of the most recent transaction, even
      // if a logger has fallen behind.
      loggers.setCommittedTxId(firstTxToFlush + numReadyTxns - 1);
  • public QuorumCall<AsyncLogger, GetJournalStateResponseProto> getJournalState()
  • public QuorumCall<AsyncLogger, Boolean> isFormatted()
  • public QuorumCall<AsyncLogger,NewEpochResponseProto> newEpoch(NamespaceInfo nsInfo,long epoch)
  • public QuorumCall<AsyncLogger, Void> startLogSegment(long txid, int layoutVersion)
  • public QuorumCall<AsyncLogger, Void> finalizeLogSegment(long firstTxId,long lastTxId)
  • public QuorumCall<AsyncLogger, Void> sendEdits(long segmentTxId, long firstTxnId, int numTxns, byte[] data)
  • public QuorumCall<AsyncLogger, RemoteEditLogManifest> getEditLogManifest(long fromTxnId, boolean inProgressOk)
  • QuorumCall<AsyncLogger, PrepareRecoveryResponseProto> prepareRecovery(long segmentTxId)
  • QuorumCall<AsyncLogger,Void> acceptRecovery(SegmentStateProto log, URL fromURL)
  • QuorumCall<AsyncLogger,Void> format(NamespaceInfo nsInfo)
  • public QuorumCall<AsyncLogger, Void> discardSegments(long startTxId)
  • QuorumCall<AsyncLogger, Void> doPreUpgrade()
  • public QuorumCall<AsyncLogger, Void> doUpgrade(StorageInfo sInfo)
  • public QuorumCall<AsyncLogger, Void> doFinalize()
  • public QuorumCall<AsyncLogger, Boolean> canRollBack(StorageInfo storage,StorageInfo prevStorage, int targetLayoutVersion)
  • public QuorumCall<AsyncLogger, Void> doRollback()
  • public QuorumCall<AsyncLogger, Long> getJournalCTime()

方法内部 QuorumCall q.waitFor() 方法,要求至少一半以上的logger执行成功

<V> Map<AsyncLogger, V> waitForWriteQuorum(QuorumCall<AsyncLogger, V> q,int timeoutMs, String operationName)

  // QuorumCall.java
  /**
   * Wait for the quorum to achieve a certain number of responses.
   * 
   * Note that, even after this returns, more responses may arrive,
   * causing the return value of other methods in this class to change.
   *
   * @param minResponses return as soon as this many responses have been
   * received, regardless of whether they are successes or exceptions
   * @param minSuccesses return as soon as this many successful (non-exception)
   * responses have been received
   * @param maxExceptions return as soon as this many exception responses
   * have been received. Pass 0 to return immediately if any exception is
   * received.
   * @param millis the number of milliseconds to wait for
   * @throws InterruptedException if the thread is interrupted while waiting
   * @throws TimeoutException if the specified timeout elapses before
   * achieving the desired conditions
   */
  public synchronized void waitFor(
      int minResponses, int minSuccesses, int maxExceptions,
      int millis, String operationName)
      throws InterruptedException, TimeoutException {
    long st = Time.monotonicNow();
    long nextLogTime = st + (long)(millis * WAIT_PROGRESS_INFO_THRESHOLD);
    long et = st + millis;
    while (true) {
      restartQuorumStopWatch();
      checkAssertionErrors();
      if (minResponses > 0 && countResponses() >= minResponses) return;
      if (minSuccesses > 0 && countSuccesses() >= minSuccesses) return;
      if (maxExceptions >= 0 && countExceptions() > maxExceptions) return;
      long now = Time.monotonicNow();
      
      if (now > nextLogTime) {
        long waited = now - st;
        String msg = String.format(
            "Waited %s ms (timeout=%s ms) for a response for %s",
            waited, millis, operationName);
        if (!successes.isEmpty()) {
          msg += ". Succeeded so far: [" + Joiner.on(",").join(successes.keySet()) + "]";
        }
        if (!exceptions.isEmpty()) {
          msg += ". Exceptions so far: [" + getExceptionMapString() + "]";
        }
        if (successes.isEmpty() && exceptions.isEmpty()) {
          msg += ". No responses yet.";
        }
        if (waited > millis * WAIT_PROGRESS_WARN_THRESHOLD) {
          QuorumJournalManager.LOG.warn(msg);
        } else {
          QuorumJournalManager.LOG.info(msg);
        }
        nextLogTime = now + WAIT_PROGRESS_INTERVAL_MILLIS;
      }
      long rem = et - now;
      if (rem <= 0) {
        // Increase timeout if a full GC occurred after restarting stopWatch
        if (shouldIncreaseQuorumTimeout(0, millis)) {
          et = et + millis;
        } else {
          throw new TimeoutException();
        }
      }
      restartQuorumStopWatch();
      rem = Math.min(rem, nextLogTime - now);
      rem = Math.max(rem, 1);
      wait(rem);
      // Increase timeout if a full GC occurred after restarting stopWatch
      if (shouldIncreaseQuorumTimeout(-rem, millis)) {
        et = et + millis;
      }
    }
  }

class IPCLoggerChannel implements AsyncLogger

Channel to a remote JournalNode using Hadoop IPC

protected QJournalProtocol createProxy() throws IOException {
    final Configuration confCopy = new Configuration(conf);
    
    // Need to set NODELAY or else batches larger than MTU can trigger 
    // 40ms nagling delays.
    confCopy.setBoolean(
        CommonConfigurationKeysPublic.IPC_CLIENT_TCPNODELAY_KEY,
        true);
    
    RPC.setProtocolEngine(confCopy,
        QJournalProtocolPB.class, ProtobufRpcEngine.class);
    return SecurityUtil.doAsLoginUser(
        new PrivilegedExceptionAction<QJournalProtocol>() {
          @Override
          public QJournalProtocol run() throws IOException {
            RPC.setProtocolEngine(confCopy,
                QJournalProtocolPB.class, ProtobufRpcEngine.class);
            QJournalProtocolPB pbproxy = RPC.getProxy(
                QJournalProtocolPB.class,
                RPC.getProtocolVersion(QJournalProtocolPB.class),
                addr, confCopy);
            return new QJournalProtocolTranslatorPB(pbproxy);
          }
        });
  }

startLogSegment 建立输出流

FSEditLog synchronized void startLogSegment()
JournalSet public EditLogOutputStream startLogSegment()
JournalAndStream public void startLogSegment(long txId, int layoutVersion)

QuorumJournalManager 实现

QuorumJournalManager public EditLogOutputStream startLogSegment()
AsyncLoggerSet public QuorumCall<AsyncLogger, Void> startLogSegment()
IPCLoggerChannel public ListenableFuture<Void> startLogSegment()
  // RPC Call
  getProxy().startLogSegment(createReqInfo(), txid, layoutVersion);
JournalNodeRpcServer public void startLogSegment()
Journal public synchronized void startLogSegment()
// Journal
public synchronized void startLogSegment(RequestInfo reqInfo, long txid,
      int layoutVersion) throws IOException {
    
    EditLogFile existing = fjm.getLogFile(txid);
    if (existing != null) {
      if (!existing.isInProgress()) {
        throw new IllegalStateException("Already have a finalized segment " +
            existing + " beginning at " + txid);
      }
      
      // If it's in-progress, it should only contain one transaction,
      // because the "startLogSegment" transaction is written alone at the
      // start of each segment. 
      existing.scanLog();
      if (existing.getLastTxId() != existing.getFirstTxId()) {
        throw new IllegalStateException("The log file " +
            existing + " seems to contain valid transactions");
      }
    }
    
    long curLastWriterEpoch = lastWriterEpoch.get();
    if (curLastWriterEpoch != reqInfo.getEpoch()) {
      LOG.info("Updating lastWriterEpoch from " + curLastWriterEpoch +
          " to " + reqInfo.getEpoch() + " for client " +
          Server.getRemoteIp());
      lastWriterEpoch.set(reqInfo.getEpoch());
    }

    // The fact that we are starting a segment at this txid indicates
    // that any previous recovery for this same segment was aborted.
    // Otherwise, no writer would have started writing. So, we can
    // remove the record of the older segment here.
    purgePaxosDecision(txid);
    
    curSegment = fjm.startLogSegment(txid, layoutVersion);
    curSegmentTxId = txid;
    nextTxId = txid;
  }

FileJournalManager 实现

  synchronized public EditLogOutputStream startLogSegment(long txid,
      int layoutVersion) throws IOException {
    try {
      currentInProgress = NNStorage.getInProgressEditsFile(sd, txid);
      EditLogOutputStream stm = new EditLogFileOutputStream(conf,
          currentInProgress, outputBufferCapacity);
      stm.create(layoutVersion);
      return stm;
    } catch (IOException e) {
      // ...
    }
  }

创建物理EditLog文件,fc,buf

// EditLogFileOutputStream
// doubleBuf : 设置为512K的缓存
// fc : 基于随机文件IO的Channel
  public EditLogFileOutputStream(Configuration conf, File name, int size)
      throws IOException {
    super();
    shouldSyncWritesAndSkipFsync = conf.getBoolean(
            DFSConfigKeys.DFS_NAMENODE_EDITS_NOEDITLOGCHANNELFLUSH,
            DFSConfigKeys.DFS_NAMENODE_EDITS_NOEDITLOGCHANNELFLUSH_DEFAULT);

    file = name;
    doubleBuf = new EditsDoubleBuffer(size);
    RandomAccessFile rp;
    if (shouldSyncWritesAndSkipFsync) {
      rp = new RandomAccessFile(name, "rws");
    } else {
      rp = new RandomAccessFile(name, "rw");
    }
    fp = new FileOutputStream(rp.getFD()); // open for append
    fc = rp.getChannel();
    fc.position(fc.size());
  }

FSEditLog 异步写JournalNodes editLog信息

FSEditLog log 操作

FSEditLog logAddBlock | logUpdateBlocks | logMkDir 等
FSEditLog void logEdit(final FSEditLogOp op)

* FSEditLog synchronized boolean doEditTransaction(final FSEditLogOp op) 
      FSEditLog : editLogStream.write(op)
      QuorumOutputStream : public void write(FSEditLogOp op)  // 写本地缓存,等待Flush
      QuorumOutputStream : buf.writeOp(op);
      EditsDoubleBuffer : bufCurrent.writeOp(op);

      // 由stream 确定是否ForceSync(), EditLogFileOutputStream : doubleBuf满,则强制Flush;
      // JournalAndStream : 内部 journals 中活跃的jas中,如果存在需要Flush,则全部flush
      FSEditLog : editLogStream.shouldForceSync()  

* FSEditLog public void logSync()

FSEditLog protected void logSync(long mytxid)
EditLogOutputStream public void flush()
EditLogOutputStream public void flush(boolean durable)
EditLogOutputStream abstract protected void flushAndSync(boolean durable)

// QuorumOutputStream 实现flushAndSync()
      QuorumCall<AsyncLogger, Void> qcall = loggers.sendEdits(
          segmentTxId, firstTxToFlush,
          numReadyTxns, data);
      loggers.waitForWriteQuorum(qcall, writeTimeoutMs, "sendEdits");
      
      // Since we successfully wrote this batch, let the loggers know. Any future
      // RPCs will thus let the loggers know of the most recent transaction, even
      // if a logger has fallen behind.
      loggers.setCommittedTxId(firstTxToFlush + numReadyTxns - 1);

AsyncLoggerSet public QuorumCall<AsyncLogger, Void> sendEdits
IPCLoggerChannel public ListenableFuture<Void> sendEdits()
// RPC Call
      getProxy().journal(createReqInfo(),segmentTxId, firstTxnId, numTxns, data);
JournalNodeRpcServer public void journal()
Journal 
synchronized void journal() {
    curSegment.writeRaw(records, 0, records.length); // 这里写入doubleBuf缓存
    curSegment.setReadyToFlush(); // 切换一个新的Buf,然后就可以异步对老的buf进行flush了
    curSegment.flush(shouldFsync);  // 调用 EditLogFileOutputStream : flushAndSync 将buf中数据拷贝到FC 中
}

这里 curSegment 实现参考[QuorumJournalManager 实现](## QuorumJournalManager 实现)

评论
添加红包

请填写红包祝福语或标题

红包个数最小为10个

红包金额最低5元

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

抵扣说明:

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

余额充值