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

Fix dead lock during async editlog rolling if edit queue is full

    XMLWordPrintableJSON

Details

    • Reviewed

    Description

      when doing rolleditlog it acquires  fs write lock,then acquire FSEditLogAsync lock object,and write 3 EDIT(the second one override logEdit method and return true)

      in extremely case,when FSEditLogAsync's logSync is very slow,editPendingQ(default size 4096)is full,it case IPC thread can not offer edit object into editPendingQ when doing rolleditlog,it block on editPendingQ .put  method,however it does't release FSEditLogAsync object lock, and edit.logEdit method in FSEditLogAsync.run thread can never acquire FSEditLogAsync object lock, it case dead lock

      stack trace like below

      "Thread[Thread-44528,5,main]" #130093 daemon prio=5 os_prio=0 tid=0x0000000002377000 nid=0x13fda waiting on condition [0x00007fb3297de000]
      java.lang.Thread.State: WAITING (parking)
      at sun.misc.Unsafe.park(Native Method)

      • parking to wait for <0x00007fbd3cb96f58> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
        at java.util.concurrent.ArrayBlockingQueue.put(ArrayBlockingQueue.java:353)
        at org.apache.hadoop.hdfs.server.namenode.FSEditLogAsync.enqueueEdit(FSEditLogAsync.java:156)
        at org.apache.hadoop.hdfs.server.namenode.FSEditLogAsync.logEdit(FSEditLogAsync.java:118)
        at org.apache.hadoop.hdfs.server.namenode.FSEditLog.logCancelDelegationToken(FSEditLog.java:1008)
        at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.logExpireDelegationToken(FSNamesystem.java:7635)
        at org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager.logExpireToken(DelegationTokenSecretManager.java:395)
      • locked <0x00007fbd3cbae500> (a java.lang.Object)
        at org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager.logExpireToken(DelegationTokenSecretManager.java:62)
        at org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager.removeExpiredToken(AbstractDelegationTokenSecretManager.java:604)
        at org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager.access$400(AbstractDelegationTokenSecretManager.java:54)
        at org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager$ExpiredTokenRemover.run(AbstractDelegationTokenSecretManager.java:656)
        at java.lang.Thread.run(Thread.java:745)

      "FSEditLogAsync" #130072 daemon prio=5 os_prio=0 tid=0x000000000715b800 nid=0x13fbf waiting for monitor entry [0x00007fb32c51a000]
      java.lang.Thread.State: BLOCKED (on object monitor)
      at org.apache.hadoop.hdfs.server.namenode.FSEditLog.doEditTransaction(FSEditLog.java:443)

      • waiting to lock <0x00007fbcbc131000> (a org.apache.hadoop.hdfs.server.namenode.FSEditLogAsync)
        at org.apache.hadoop.hdfs.server.namenode.FSEditLogAsync$Edit.logEdit(FSEditLogAsync.java:233)
        at org.apache.hadoop.hdfs.server.namenode.FSEditLogAsync.run(FSEditLogAsync.java:177)
        at java.lang.Thread.run(Thread.java:745)

      "IPC Server handler 47 on 53310" #337 daemon prio=5 os_prio=0 tid=0x00007fe659d46000 nid=0x4c62 waiting on condition [0x00007fb32fe52000]
      java.lang.Thread.State: WAITING (parking)
      at sun.misc.Unsafe.park(Native Method)

      • parking to wait for <0x00007fbd3cb96f58> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
        at java.util.concurrent.ArrayBlockingQueue.put(ArrayBlockingQueue.java:353)
        at org.apache.hadoop.hdfs.server.namenode.FSEditLogAsync.enqueueEdit(FSEditLogAsync.java:156)
        at org.apache.hadoop.hdfs.server.namenode.FSEditLogAsync.logEdit(FSEditLogAsync.java:118)
        at org.apache.hadoop.hdfs.server.namenode.FSEditLog.endCurrentLogSegment(FSEditLog.java:1251)
      • locked <0x00007fbcbc131000> (a org.apache.hadoop.hdfs.server.namenode.FSEditLogAsync)
        at org.apache.hadoop.hdfs.server.namenode.FSEditLog.rollEditLog(FSEditLog.java:1190)
      • locked <0x00007fbcbc131000> (a org.apache.hadoop.hdfs.server.namenode.FSEditLogAsync)
        at org.apache.hadoop.hdfs.server.namenode.FSImage.rollEditLog(FSImage.java:1235)
        at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.rollEditLog(FSNamesystem.java:6463)
        at org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.rollEditLog(NameNodeRpcServer.java:1062)
        at org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolServerSideTranslatorPB.rollEditLog(NamenodeProtocolServerSideTranslatorPB.java:142)
        at org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos$NamenodeProtocolService$2.callBlockingMethod(NamenodeProtocolProtos.java:12025)
        at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:617)
        at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1073)
        at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2141)
        at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2137)
        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:1743)
        at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2135)

      Attachments

        1. deadlock.patch
          10 kB
          zhangwei
        2. HDFS-13112.patch
          11 kB
          Daryn Sharp

        Issue Links

          Activity

            People

              daryn Daryn Sharp
              photogamrun zhangwei
              Votes:
              0 Vote for this issue
              Watchers:
              20 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: