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

NameNode failover doesn't happen when running fsfreeze for the NameNode dir (dfs.namenode.name.dir)

Add voteVotersWatch issueWatchersCreate sub-taskLinkCloneUpdate Comment AuthorReplace String in CommentUpdate Comment VisibilityDelete Comments
    XMLWordPrintableJSON

Details

    • Bug
    • Status: Patch Available
    • Major
    • Resolution: Unresolved
    • None
    • None
    • ha
    • None

    Description

      I ran fsfreeze for the NameNode dir (dfs.namenode.name.dir) in my cluster for test purpose, but NameNode failover didn't happen.

      fsfreeze -f /mnt
      

      /mnt is a separate filesystem partition from /. And the NameNode dir "dfs.namenode.name.dir" is /mnt/hadoop/hdfs/namenode.

      I checked the source code, and I found monitorHealth RPC from ZKFC doesn't fail even if the NameNode dir is frozen. I think that's why the failover doesn't happen.

      Also if the NameNode dir is frozen, it looks like FSImage.rollEditLog() gets stuck like the following, and it keeps holding the write lock of FSNamesystem, which causes HDFS service down:

      "IPC Server handler 5 on default port 8020" #53 daemon prio=5 os_prio=0 tid=0x00007f56b96e2000 nid=0x5042 in Object.wait() [0x00007f56937bb000]
         java.lang.Thread.State: TIMED_WAITING (on object monitor)
              at java.lang.Object.wait(Native Method)
              at org.apache.hadoop.hdfs.server.namenode.FSEditLogAsync$SyncEdit.logSyncWait(FSEditLogAsync.java:317)
              - locked <0x00000000c58ca268> (a org.apache.hadoop.hdfs.server.namenode.FSEditLogAsync)
              at org.apache.hadoop.hdfs.server.namenode.FSEditLogAsync.logSyncAll(FSEditLogAsync.java:147)
              at org.apache.hadoop.hdfs.server.namenode.FSEditLog.endCurrentLogSegment(FSEditLog.java:1422)
              - locked <0x00000000c58ca268> (a org.apache.hadoop.hdfs.server.namenode.FSEditLogAsync)
              at org.apache.hadoop.hdfs.server.namenode.FSEditLog.rollEditLog(FSEditLog.java:1316)
              - locked <0x00000000c58ca268> (a org.apache.hadoop.hdfs.server.namenode.FSEditLogAsync)
              at org.apache.hadoop.hdfs.server.namenode.FSImage.rollEditLog(FSImage.java:1322)
              at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.rollEditLog(FSNamesystem.java:4740)
              at org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.rollEditLog(NameNodeRpcServer.java:1307)
              at org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolServerSideTranslatorPB.rollEditLog(NamenodeProtocolServerSideTranslatorPB.java:148)
              at org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos$NamenodeProtocolService$2.callBlockingMethod(NamenodeProtocolProtos.java:14726)
              at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:524)
              at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1025)
              at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:898)
              at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:844)
              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:1876)
              at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2727)
      
         Locked ownable synchronizers:
              - <0x00000000c5f4ca10> (a java.util.concurrent.locks.ReentrantReadWriteLock$FairSync)
      

      I believe NameNode failover should happen in this case. One idea is to check if the NameNode dir is working when NameNode receives monitorHealth RPC from ZKFC.

      I will attach a patch for this idea.

      Attachments

        1. HDFS-14123.01.patch
          2 kB
          Toshihiro Suzuki
        2. HDFS-14123.01.patch
          2 kB
          Toshihiro Suzuki
        3. HDFS-14123.01.patch
          2 kB
          Toshihiro Suzuki

        Activity

          This comment will be Viewable by All Users Viewable by All Users
          Cancel

          People

            brfrn169 Toshihiro Suzuki
            brfrn169 Toshihiro Suzuki

            Dates

              Created:
              Updated:

              Slack

                Issue deployment