Uploaded image for project: 'Hadoop HDFS'
  1. Hadoop HDFS
  2. HDFS-13145

SBN crash when transition to ANN with in-progress edit tailing enabled

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 3.0.0
    • 3.1.0, 3.0.3
    • ha, namenode
    • None
    • Reviewed

    Description

      With edit log in-progress edit log tailing enabled, QuorumOutputStream will send two batches to JNs, one normal edit batch followed by a dummy batch to update the commit ID on JNs.

            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);
      
            // If we don't have this dummy send, committed TxId might be one-batch
            // stale on the Journal Nodes
            if (updateCommittedTxId) {
              QuorumCall<AsyncLogger, Void> fakeCall = loggers.sendEdits(
                  segmentTxId, firstTxToFlush,
                  0, new byte[0]);
              loggers.waitForWriteQuorum(fakeCall, writeTimeoutMs, "sendEdits");
            }
      

      Between each batch, it will wait for the JNs to reach a quorum. However, if the ANN crashes in between, then SBN will crash while transiting to ANN:

      java.lang.IllegalStateException: Cannot start writing at txid 24312595802 when there is a stream available for read: ......
              at org.apache.hadoop.hdfs.server.namenode.FSEditLog.openForWrite(FSEditLog.java:329)
              at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startActiveServices(FSNamesystem.java:1196)
              at org.apache.hadoop.hdfs.server.namenode.NameNode$NameNodeHAContext.startActiveServices(NameNode.java:1839)
              at org.apache.hadoop.hdfs.server.namenode.ha.ActiveState.enterState(ActiveState.java:61)
              at org.apache.hadoop.hdfs.server.namenode.ha.HAState.setStateInternal(HAState.java:64)
              at org.apache.hadoop.hdfs.server.namenode.ha.StandbyState.setState(StandbyState.java:49)
              at org.apache.hadoop.hdfs.server.namenode.NameNode.transitionToActive(NameNode.java:1707)
              at org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.transitionToActive(NameNodeRpcServer.java:1622)
              at org.apache.hadoop.ha.protocolPB.HAServiceProtocolServerSideTranslatorPB.transitionToActive(HAServiceProtocolServerSideTranslatorPB.java:107)
              at org.apache.hadoop.ha.proto.HAServiceProtocolProtos$HAServiceProtocolService$2.callBlockingMethod(HAServiceProtocolProtos.java:4460)
              at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:447)
              at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:989)
              at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:851)
              at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:794)
              at java.security.AccessController.doPrivileged(Native Method)
              at javax.security.auth.Subject.doAs(Subject.java:422)
              at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1836)
              at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2490)
      2018-02-13 00:43:20,728 INFO org.apache.hadoop.util.ExitUtil: Exiting with status 1
      

      This is because without the dummy batch, the commitTxnId will lag behind the endTxId, which caused the check in openForWrite to fail:

          List<EditLogInputStream> streams = new ArrayList<EditLogInputStream>();
          journalSet.selectInputStreams(streams, segmentTxId, true, false);
          if (!streams.isEmpty()) {
            String error = String.format("Cannot start writing at txid %s " +
              "when there is a stream available for read: %s",
              segmentTxId, streams.get(0));
            IOUtils.cleanupWithLogger(LOG,
                streams.toArray(new EditLogInputStream[0]));
            throw new IllegalStateException(error);
          }
      

      In our environment, this can be reproduced pretty consistently, which will leave the cluster with no running namenodes. Even though we are using a 2.8.2 backport, I believe the same issue also exist in 3.0.x.

      Attachments

        1. HDFS-13145.000.patch
          3 kB
          Chao Sun
        2. HDFS-13145.001.patch
          3 kB
          Chao Sun

        Issue Links

          Activity

            People

              csun Chao Sun
              csun Chao Sun
              Votes:
              0 Vote for this issue
              Watchers:
              9 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: