Details

    • Type: Bug
    • Status: Resolved
    • Priority: Blocker
    • Resolution: Fixed
    • Affects Version/s: 3.4.7
    • Fix Version/s: 3.4.8
    • Component/s: None
    • Labels:
      None

      Description

      HBase recently upgraded to zookeeper 3.4.7

      In one of the tests, TestSplitLogManager, there is reproducible hang at the end of the test.
      Below is snippet from stack trace related to zookeeper:

      "main-EventThread" daemon prio=5 tid=0x00007fd27488a800 nid=0x6f1f waiting on condition [0x000000011834b000]
         java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000007c5b8d3a0> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2043)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:501)
      
      "main-SendThread(localhost:59510)" daemon prio=5 tid=0x00007fd274eb4000 nid=0x9513 waiting on condition [0x0000000118042000]
         java.lang.Thread.State: TIMED_WAITING (sleeping)
        at java.lang.Thread.sleep(Native Method)
        at org.apache.zookeeper.client.StaticHostProvider.next(StaticHostProvider.java:101)
        at org.apache.zookeeper.ClientCnxn$SendThread.startConnect(ClientCnxn.java:997)
        at org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1060)
      
      "SyncThread:0" prio=5 tid=0x00007fd274d02000 nid=0x730f waiting for monitor entry [0x00000001170ac000]
         java.lang.Thread.State: BLOCKED (on object monitor)
        at org.apache.zookeeper.server.ZooKeeperServer.decInProcess(ZooKeeperServer.java:512)
        - waiting to lock <0x00000007c5b62128> (a org.apache.zookeeper.server.ZooKeeperServer)
        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:144)
        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:200)
        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:131)
      
      "main-EventThread" daemon prio=5 tid=0x00007fd2753a3800 nid=0x711b waiting on condition [0x0000000117a30000]
         java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000007c9b106b8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2043)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:501)
      
      "main" prio=5 tid=0x00007fd276000000 nid=0x1903 in Object.wait() [0x0000000108aa1000]
         java.lang.Thread.State: WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        - waiting on <0x00000007c5b66400> (a org.apache.zookeeper.server.SyncRequestProcessor)
        at java.lang.Thread.join(Thread.java:1281)
        - locked <0x00000007c5b66400> (a org.apache.zookeeper.server.SyncRequestProcessor)
        at java.lang.Thread.join(Thread.java:1355)
        at org.apache.zookeeper.server.SyncRequestProcessor.shutdown(SyncRequestProcessor.java:213)
        at org.apache.zookeeper.server.PrepRequestProcessor.shutdown(PrepRequestProcessor.java:770)
        at org.apache.zookeeper.server.ZooKeeperServer.shutdown(ZooKeeperServer.java:478)
        - locked <0x00000007c5b62128> (a org.apache.zookeeper.server.ZooKeeperServer)
        at org.apache.zookeeper.server.NIOServerCnxnFactory.shutdown(NIOServerCnxnFactory.java:266)
        at org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster.shutdown(MiniZooKeeperCluster.java:301)
      

      Note the address (0x00000007c5b66400) in the last hunk which seems to indicate some form of deadlock.

      According to Camille Fournier:

      We made shutdown synchronized. But decrementing the requests is
      also synchronized and called from a different thread. So yeah, deadlock.
      This came in with ZOOKEEPER-1907

      1. ZOOKEEPER-2347-br-3.4.patch
        12 kB
        Rakesh R
      2. ZOOKEEPER-2347-br-3.4.patch
        12 kB
        Rakesh R
      3. ZOOKEEPER-2347-br-3.4.patch
        11 kB
        Rakesh R
      4. ZOOKEEPER-2347-br-3.4.patch
        5 kB
        Rakesh R
      5. testSplitLogManager.stack
        16 kB
        Ted Yu

        Activity

        Hide
        yuzhihong@gmail.com Ted Yu added a comment -

        Stack trace showing the issue

        Show
        yuzhihong@gmail.com Ted Yu added a comment - Stack trace showing the issue
        Hide
        rakeshr Rakesh R added a comment -

        Thanks Ted Yu for reporting this issue. Also, thanks Camille Fournier, Flavio Junqueira for the analysis/advice. I'm attaching a patch fixing the issue. I've tried an attempt to simulate the scenario through unit testing, I ran test multiple times and didn't see any deadlock after applying the patch. Please review.

        Show
        rakeshr Rakesh R added a comment - Thanks Ted Yu for reporting this issue. Also, thanks Camille Fournier , Flavio Junqueira for the analysis/advice. I'm attaching a patch fixing the issue. I've tried an attempt to simulate the scenario through unit testing, I ran test multiple times and didn't see any deadlock after applying the patch. Please review.
        Hide
        fpj Flavio Junqueira added a comment -

        To be consistent, I'm reposting the comment I made in the other jira here.

        We have made requestsInProcess an AtomicInteger in ZOOKEEPER-1504, removing the synchronization of the decInProcess method. We should just make the same change here for the 3.4 branch.

        Show
        fpj Flavio Junqueira added a comment - To be consistent, I'm reposting the comment I made in the other jira here. We have made requestsInProcess an AtomicInteger in ZOOKEEPER-1504 , removing the synchronization of the decInProcess method. We should just make the same change here for the 3.4 branch.
        Hide
        yuzhihong@gmail.com Ted Yu added a comment -

        Not sure how I can test this with hbase unit test(s).

        As far as I know, zookeeper still uses ant to build while hbase dependency is expressed through maven.

        Show
        yuzhihong@gmail.com Ted Yu added a comment - Not sure how I can test this with hbase unit test(s). As far as I know, zookeeper still uses ant to build while hbase dependency is expressed through maven.
        Hide
        cnauroth Chris Nauroth added a comment -

        As far as I know, zookeeper still uses ant to build while hbase dependency is expressed through maven.

        Hi Ted. The Ant build has a mvn-install target. If you're interested in testing with HBase, then I think you could get the current branch-3.4 ZooKeeper code, apply the patch, run ant mvn-install to install a 3.4.8-SNAPSHOT build to your local repository, and then set up your HBase build to link against ZooKeeper 3.4.8-SNAPSHOT.

        Show
        cnauroth Chris Nauroth added a comment - As far as I know, zookeeper still uses ant to build while hbase dependency is expressed through maven. Hi Ted. The Ant build has a mvn-install target. If you're interested in testing with HBase, then I think you could get the current branch-3.4 ZooKeeper code, apply the patch, run ant mvn-install to install a 3.4.8-SNAPSHOT build to your local repository, and then set up your HBase build to link against ZooKeeper 3.4.8-SNAPSHOT.
        Hide
        jbrosenberg@gmail.com Jason Rosenberg added a comment -

        What are the conditions that trigger this issue? We've been running with 3.4.7 and so far have not seen any dead-locks with routine server shutdowns, or with tests. Trying to judge whether we should revert or not.

        Show
        jbrosenberg@gmail.com Jason Rosenberg added a comment - What are the conditions that trigger this issue? We've been running with 3.4.7 and so far have not seen any dead-locks with routine server shutdowns, or with tests. Trying to judge whether we should revert or not.
        Hide
        yuzhihong@gmail.com Ted Yu added a comment -

        Thanks for the pointer, Chris.

        I ran TestSplitLogManager after modifying pom.xml twice which passed. Previously the test hung quite reliably on Mac.

        Show
        yuzhihong@gmail.com Ted Yu added a comment - Thanks for the pointer, Chris. I ran TestSplitLogManager after modifying pom.xml twice which passed. Previously the test hung quite reliably on Mac.
        Hide
        cnauroth Chris Nauroth added a comment -

        Ted Yu, thank you for the help with testing!

        Show
        cnauroth Chris Nauroth added a comment - Ted Yu , thank you for the help with testing!
        Hide
        fpj Flavio Junqueira added a comment -

        The patch looks good, but I'm not really convinced about the test case. It relies on the interleaving of events to possibly trigger the problem, so it isn't deterministically reproducing the problem in the case it exists. I was thinking that maybe a better way of testing this is to set up a pipeline, populate toFlush directly, and just call shutdown on the ZooKeeperServer. If it is possible to do this, then it will be more reliable than submitting a bunch of operations and hoping for the race to kick in. What do you think?

        Show
        fpj Flavio Junqueira added a comment - The patch looks good, but I'm not really convinced about the test case. It relies on the interleaving of events to possibly trigger the problem, so it isn't deterministically reproducing the problem in the case it exists. I was thinking that maybe a better way of testing this is to set up a pipeline, populate toFlush directly, and just call shutdown on the ZooKeeperServer . If it is possible to do this, then it will be more reliable than submitting a bunch of operations and hoping for the race to kick in. What do you think?
        Hide
        rakeshr Rakesh R added a comment -

        yes, agreed. I've attached another patch modifying the unit tests, please review it again. Thanks!

        Show
        rakeshr Rakesh R added a comment - yes, agreed. I've attached another patch modifying the unit tests, please review it again. Thanks!
        Hide
        yuzhihong@gmail.com Ted Yu added a comment -

        Rakesh:
        Thanks for updating the test case.

        Show
        yuzhihong@gmail.com Ted Yu added a comment - Rakesh: Thanks for updating the test case.
        Hide
        yuzhihong@gmail.com Ted Yu added a comment -

        Flavio Junqueira:
        Can you review the patch ?

        Show
        yuzhihong@gmail.com Ted Yu added a comment - Flavio Junqueira : Can you review the patch ?
        Hide
        fpj Flavio Junqueira added a comment -

        Rakesh R thanks for the update. It looks much better, I have tested and the new test case does hang without the other changes, but there are a few small points I want to raise:

        1. Do we really need a timeout of 90s? I'd rather have something like 30s or less.
        2. Typo in LOG.error("Exception while waiting to proess req", e);
        3. Please add a description of the dependency cycle that we are testing for. For example, in step 7, you could say that we are testing that SyncRequestProcessor#shutdown holds a lock and waits on FinalRequestProcessor to complete a pending operation, which in turn also needs the ZooKeeperServer lock. This is to emphasize where the problem was and make it very clear.
        4. Replace "Waiting for FinalReqProcessor to be called" with "Waiting for FinalRequestProcessor to start processing request" and "Waiting for SyncReqProcessor#shutdown to be called" with "Waiting for SyncRequestProcessor to shut down".
        5. There are a couple of exceptions that we catch but do nothing because we rely on the timeout. It is better to simply fail the test case directly if it is a failure rather than rely on a timeout. If you don't like the idea of calling fail() from an auxiliary class, then we need to at least propagate the exception so that we can catch and fail rather than wait.

        I also would feel more comfortable if we get another review here. I'm fairly confident, but given that we've missed this issue before, I'd rather have another +1 before we check in.

        Show
        fpj Flavio Junqueira added a comment - Rakesh R thanks for the update. It looks much better, I have tested and the new test case does hang without the other changes, but there are a few small points I want to raise: Do we really need a timeout of 90s? I'd rather have something like 30s or less. Typo in LOG.error("Exception while waiting to proess req", e); Please add a description of the dependency cycle that we are testing for. For example, in step 7, you could say that we are testing that SyncRequestProcessor#shutdown holds a lock and waits on FinalRequestProcessor to complete a pending operation, which in turn also needs the ZooKeeperServer lock. This is to emphasize where the problem was and make it very clear. Replace "Waiting for FinalReqProcessor to be called" with "Waiting for FinalRequestProcessor to start processing request" and "Waiting for SyncReqProcessor#shutdown to be called" with "Waiting for SyncRequestProcessor to shut down" . There are a couple of exceptions that we catch but do nothing because we rely on the timeout. It is better to simply fail the test case directly if it is a failure rather than rely on a timeout. If you don't like the idea of calling fail() from an auxiliary class, then we need to at least propagate the exception so that we can catch and fail rather than wait. I also would feel more comfortable if we get another review here. I'm fairly confident, but given that we've missed this issue before, I'd rather have another +1 before we check in.
        Hide
        rakeshr Rakesh R added a comment -

        Thanks Flavio Junqueira for the detailed comments. I've modified the description . Also, used 30 secs timeout. I feel bigger timeout would be needed in case running the tests in slower machines. Could you please review the latest patch again when you get a chance.

        Show
        rakeshr Rakesh R added a comment - Thanks Flavio Junqueira for the detailed comments. I've modified the description . Also, used 30 secs timeout. I feel bigger timeout would be needed in case running the tests in slower machines. Could you please review the latest patch again when you get a chance.
        Hide
        fpj Flavio Junqueira added a comment -

        Rakesh R looks good, I just have a few minor asks, please replace accordingly:

        1. “Tests to verify that ZooKeeper server should be able to shutdown properly…” -> “Test case to verify that ZooKeeper server is able to shutdown properly…”
        2. “errOccurred” -> “interrupted”
        3. “InterruptedException while waiting to process request!” -> “Interrupted while waiting to process request”

        I still would like to have another committer having a look at this to have a second opinion. Any volunteer, please?

        Show
        fpj Flavio Junqueira added a comment - Rakesh R looks good, I just have a few minor asks, please replace accordingly: “Tests to verify that ZooKeeper server should be able to shutdown properly…” -> “Test case to verify that ZooKeeper server is able to shutdown properly…” “errOccurred” -> “interrupted” “InterruptedException while waiting to process request!” -> “Interrupted while waiting to process request” I still would like to have another committer having a look at this to have a second opinion. Any volunteer, please?
        Hide
        rakeshr Rakesh R added a comment -

        Thanks again Flavio Junqueira for the review comments. Attached another patch addressing the same.

        Show
        rakeshr Rakesh R added a comment - Thanks again Flavio Junqueira for the review comments. Attached another patch addressing the same.
        Hide
        rgs Raul Gutierrez Segales added a comment -

        It lgtm - thanks Rakesh R and Flavio Junqueira. One question though, why use NettyServerCnxnFactory for the test instead of the NIO one (which much more used)?

        Chris Nauroth: mind taking a look as well?

        Also, how can we validate if the HBase tests now pass?

        Show
        rgs Raul Gutierrez Segales added a comment - It lgtm - thanks Rakesh R and Flavio Junqueira . One question though, why use NettyServerCnxnFactory for the test instead of the NIO one (which much more used)? Chris Nauroth : mind taking a look as well? Also, how can we validate if the HBase tests now pass?
        Hide
        rakeshr Rakesh R added a comment -

        Thanks Raul Gutierrez Segales for the reviews.

        One question though, why use NettyServerCnxnFactory for the test instead of the NIO one (which much more used)?

        No specific reason. Test scenario has no relation with either Netty or NIO.

        Also, how can we validate if the HBase tests now pass?

        Sometime back Ted has updated Hbase test status in jira, please see the comments. Thanks Ted Yu for the test results.

        Show
        rakeshr Rakesh R added a comment - Thanks Raul Gutierrez Segales for the reviews. One question though, why use NettyServerCnxnFactory for the test instead of the NIO one (which much more used)? No specific reason. Test scenario has no relation with either Netty or NIO. Also, how can we validate if the HBase tests now pass? Sometime back Ted has updated Hbase test status in jira, please see the comments . Thanks Ted Yu for the test results.
        Hide
        yuzhihong@gmail.com Ted Yu added a comment -

        Assuming there was only test change since I performed validation last year, this should be good to go.

        Show
        yuzhihong@gmail.com Ted Yu added a comment - Assuming there was only test change since I performed validation last year, this should be good to go.
        Hide
        cnauroth Chris Nauroth added a comment -

        +1 for the patch. Nice work, Rakesh R!

        Show
        cnauroth Chris Nauroth added a comment - +1 for the patch. Nice work, Rakesh R !
        Show
        rgs Raul Gutierrez Segales added a comment - Merged: https://github.com/apache/zookeeper/commit/8fc106507b767c28530a1028b27bd3b25e3aaab8 Thanks Rakesh R !

          People

          • Assignee:
            rakeshr Rakesh R
            Reporter:
            yuzhihong@gmail.com Ted Yu
          • Votes:
            0 Vote for this issue
            Watchers:
            9 Start watching this issue

            Dates

            • Created:
              Updated:
              Resolved:

              Development