Uploaded image for project: 'ZooKeeper'
  1. ZooKeeper
  2. ZOOKEEPER-4673

Deadlock in zookeeper client.

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

Details

    • Bug
    • Status: Open
    • Major
    • Resolution: Unresolved
    • 3.4.13
    • None
    • java client
    • None

    Description

      We use apache curator 2.10.0 version
      We use zookeeper some version of 3.4.13 (I know its very old. ) We also have cherry-picked ZOOKEEPER-2251 in our branch.

      Threads in deadlock.

      Found one Java-level deadlock:
      =============================
      "Curator-Framework-0":
        waiting to lock monitor 0x00007f34f2163a00 (object 0x00007f344228db00, a java.util.LinkedList),
        which is held by "Curator-Framework-0-SendThread(<zk server>:2181)"
      "Curator-Framework-0-SendThread(<zk server>:2181)":
        waiting to lock monitor 0x00007f2109731980 (object 0x00007f2270fd02a8, a org.apache.curator.ConnectionState),
        which is held by "Curator-Framework-0"
      
      
      
      "Curator-Framework-0" #1493 daemon prio=5 os_prio=0 cpu=98.73ms elapsed=50812.45s tid=0x00007f2128da5de0 nid=0x2889 waiting for monitor entry  [0x00007f2053436000]
         java.lang.Thread.State: BLOCKED (on object monitor)
              at org.apache.zookeeper.ClientCnxn$SendThread.cleanup(ClientCnxn.java:1245)
              - waiting to lock <0x00007f344228db00> (a java.util.LinkedList)
              at org.apache.zookeeper.ClientCnxn$SendThread.cleanAndNotifyState(ClientCnxn.java:1174)
              at org.apache.zookeeper.ClientCnxn$SendThread.access$2300(ClientCnxn.java:700)
              at org.apache.zookeeper.ClientCnxn.submitRequest(ClientCnxn.java:1420)
              at org.apache.zookeeper.ClientCnxn.close(ClientCnxn.java:1373)
              at org.apache.zookeeper.ZooKeeper.close(ZooKeeper.java:788)
              - locked <0x00007f344228cd78> (a org.apache.zookeeper.ZooKeeper)
              at org.apache.curator.HandleHolder.internalClose(HandleHolder.java:139)
              at org.apache.curator.HandleHolder.closeAndReset(HandleHolder.java:77)
              at org.apache.curator.ConnectionState.reset(ConnectionState.java:217)
              - locked <0x00007f2270fd02a8> (a org.apache.curator.ConnectionState)
              at org.apache.curator.ConnectionState.checkTimeouts(ConnectionState.java:193)
              - locked <0x00007f2270fd02a8> (a org.apache.curator.ConnectionState)
              at org.apache.curator.ConnectionState.getZooKeeper(ConnectionState.java:88)
              at org.apache.curator.CuratorZookeeperClient.getZooKeeper(CuratorZookeeperClient.java:116)
              at org.apache.curator.framework.imps.CuratorFrameworkImpl.performBackgroundOperation(CuratorFrameworkImpl.java:835)
              at org.apache.curator.framework.imps.CuratorFrameworkImpl.backgroundOperationsLoop(CuratorFrameworkImpl.java:809)
              at org.apache.curator.framework.imps.CuratorFrameworkImpl.access$300(CuratorFrameworkImpl.java:64)
              at org.apache.curator.framework.imps.CuratorFrameworkImpl$4.call(CuratorFrameworkImpl.java:267)
              at java.util.concurrent.FutureTask.run(java.base@11.0.16.1.1/FutureTask.java:264)
              at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(java.base@11.0.16.1.1/ScheduledThreadPoolExecutor.java:304)
              at java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.16.1.1/ThreadPoolExecutor.java:1128)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@11.0.16.1.1/ThreadPoolExecutor.java:628)
              at java.lang.Thread.run(java.base@11.0.16.1.1/Thread.java:829)
      
      
      "Curator-Framework-0-SendThread(<zkserver>:2181)" #1524807 daemon prio=5 os_prio=0 cpu=25.58ms elapsed=101.61s tid=0x00007f1f3836d0c0 nid=0x113ad waiting for monitor entry  [0x00007f212767b000]
         java.lang.Thread.State: BLOCKED (on object monitor)
              at org.apache.curator.ConnectionState.checkTimeouts(ConnectionState.java:176)
              - waiting to lock <0x00007f2270fd02a8> (a org.apache.curator.ConnectionState)
              at org.apache.curator.ConnectionState.getZooKeeper(ConnectionState.java:88)
              at org.apache.curator.CuratorZookeeperClient.getZooKeeper(CuratorZookeeperClient.java:116)
              at org.apache.curator.framework.imps.CuratorFrameworkImpl.performBackgroundOperation(CuratorFrameworkImpl.java:835)
              at org.apache.curator.framework.imps.CuratorFrameworkImpl.doSyncForSuspendedConnection(CuratorFrameworkImpl.java:701)
              at org.apache.curator.framework.imps.CuratorFrameworkImpl.access$700(CuratorFrameworkImpl.java:64)
              at org.apache.curator.framework.imps.CuratorFrameworkImpl$7.retriesExhausted(CuratorFrameworkImpl.java:697)
              at org.apache.curator.framework.imps.CuratorFrameworkImpl.checkBackgroundRetry(CuratorFrameworkImpl.java:716)
              at org.apache.curator.framework.imps.CuratorFrameworkImpl.processBackgroundOperation(CuratorFrameworkImpl.java:516)
              at org.apache.curator.framework.imps.BackgroundSyncImpl$1.processResult(BackgroundSyncImpl.java:50)
              at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:611)
              at org.apache.zookeeper.ClientCnxn$EventThread.queuePacket(ClientCnxn.java:462)
              - locked <0x00007f344228e0c0> (a java.util.concurrent.LinkedBlockingQueue)
              at org.apache.zookeeper.ClientCnxn.finishPacket(ClientCnxn.java:632)
              at org.apache.zookeeper.ClientCnxn.conLossPacket(ClientCnxn.java:650)
              at org.apache.zookeeper.ClientCnxn.access$2100(ClientCnxn.java:94)
              at org.apache.zookeeper.ClientCnxn$SendThread.cleanup(ClientCnxn.java:1246)
              - locked <0x00007f344228db00> (a java.util.LinkedList)
              at org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1159)
      
      

      In 3.4.13 version, ClientCnxn#outgoingQueue type is LinkedList NOT LinkedBlockingDeque and we have many places where we synchronize on outgoingQueue.

      The client application was spending lot of time in garbage collection and the request to server were timing out.
      Logline from CuratorFramework-0 thread before the jstack

       2023-02-03 08:29:41:973 #1493 Timeout error occurred for the packet clientPath:null serverPath:null finished:false header:: 0,-11  replyHeader:: 0,0,0  request:: null response:: null
      

      Logline from CuratorFramework-0-SendThread thread before the jstack.

      2023-02-03 08:29:26:371 #1524807 Opening socket connection to server <zk server>:2181. Will attempt to SASL-authenticate using Login Context section 'Client'
      2023-02-03 08:29:26.371 #1524807 Socket connection established to <zk server>:2181, initiating session
      2023-02-03 08:29:26.388 #1524807 Unable to reconnect to ZooKeeper service, session 0x1002232ea037548 has expired
      

      Curator-Framework-0 thread acquired org.apache.curator.ConnectionState lock and waiting for ClientCnxn#outgoingQueue lock
      Curator-Framework-0-SendThread acquired ClientCnxn#outgoingQueue lock and waiting for org.apache.curator.ConnectionState lock.

      From the above thread dump, Curator-Framework-0 is trying to call SendThread#cleanAndNotifyState which is NOT correct. In my opinion it should just interrupt the sendThread and let SendThread cleanup its state.
      Here is the relevant code link

              if (r.getErr() == Code.REQUESTTIMEOUT.intValue()) {
                  sendThread.cleanAndNotifyState(); ---> We should NOT call this directly instead call sendThread.interrupt and let it clean its state.
              }
      

      Attachments

        Activity

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

          People

            Unassigned Unassigned
            shahrs87 Rushabh Shah

            Dates

              Created:
              Updated:

              Slack

                Issue deployment