Details

    • Type: Bug Bug
    • Status: Closed
    • Priority: Major Major
    • Resolution: Fixed
    • Affects Version/s: None
    • Fix Version/s: 4.0.0
    • Component/s: None
    • Labels:
      None

      Description

      In one my experiments, I found that a BookKeeper object was locked after I tried to halt it. By searching the Web, I found that the issue is described here:

      http://www.jboss.org/netty/community.html#nabble-td5492010

      I'll upload a patch to fix it. For now, I'm marking it for 3.4.0, but if there is any chance we can get it in 3.3.3, it would be nice.

      1. ZOOKEEPER-998.patch
        1.0 kB
        Flavio Junqueira
      2. BOOKKEEPER-5.patch
        5 kB
        Flavio Junqueira
      3. BOOKKEEPER-5.patch
        8 kB
        Flavio Junqueira
      4. BOOKKEEPER-5.diff
        7 kB
        Ivan Kelly
      5. BOOKKEEPER-5.diff
        6 kB
        Ivan Kelly
      6. BOOKKEEPER-5.diff
        5 kB
        Ivan Kelly
      7. BOOKKEEPER-5.diff
        2 kB
        Ivan Kelly

        Activity

        Hide
        Hadoop QA added a comment -

        -1 overall. Here are the results of testing the latest attachment
        http://issues.apache.org/jira/secure/attachment/12471577/ZOOKEEPER-998.patch
        against trunk revision 1072085.

        +1 @author. The patch does not contain any @author tags.

        -1 tests included. The patch doesn't appear to include any new or modified tests.
        Please justify why no new tests are needed for this patch.
        Also please list what manual steps were performed to verify this patch.

        +1 javadoc. The javadoc tool did not generate any warning messages.

        +1 javac. The applied patch does not increase the total number of javac compiler warnings.

        +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings.

        +1 release audit. The applied patch does not increase the total number of release audit warnings.

        +1 core tests. The patch passed core unit tests.

        +1 contrib tests. The patch passed contrib unit tests.

        Test results: https://hudson.apache.org/hudson/job/PreCommit-ZOOKEEPER-Build/154//testReport/
        Findbugs warnings: https://hudson.apache.org/hudson/job/PreCommit-ZOOKEEPER-Build/154//artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
        Console output: https://hudson.apache.org/hudson/job/PreCommit-ZOOKEEPER-Build/154//console

        This message is automatically generated.

        Show
        Hadoop QA added a comment - -1 overall. Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12471577/ZOOKEEPER-998.patch against trunk revision 1072085. +1 @author. The patch does not contain any @author tags. -1 tests included. The patch doesn't appear to include any new or modified tests. Please justify why no new tests are needed for this patch. Also please list what manual steps were performed to verify this patch. +1 javadoc. The javadoc tool did not generate any warning messages. +1 javac. The applied patch does not increase the total number of javac compiler warnings. +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings. +1 release audit. The applied patch does not increase the total number of release audit warnings. +1 core tests. The patch passed core unit tests. +1 contrib tests. The patch passed contrib unit tests. Test results: https://hudson.apache.org/hudson/job/PreCommit-ZOOKEEPER-Build/154//testReport/ Findbugs warnings: https://hudson.apache.org/hudson/job/PreCommit-ZOOKEEPER-Build/154//artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html Console output: https://hudson.apache.org/hudson/job/PreCommit-ZOOKEEPER-Build/154//console This message is automatically generated.
        Hide
        Hadoop QA added a comment -

        -1 overall. Here are the results of testing the latest attachment
        http://issues.apache.org/jira/secure/attachment/12471577/ZOOKEEPER-998.patch
        against trunk revision 1072085.

        +1 @author. The patch does not contain any @author tags.

        -1 tests included. The patch doesn't appear to include any new or modified tests.
        Please justify why no new tests are needed for this patch.
        Also please list what manual steps were performed to verify this patch.

        +1 javadoc. The javadoc tool did not generate any warning messages.

        +1 javac. The applied patch does not increase the total number of javac compiler warnings.

        +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings.

        +1 release audit. The applied patch does not increase the total number of release audit warnings.

        +1 core tests. The patch passed core unit tests.

        +1 contrib tests. The patch passed contrib unit tests.

        Test results: https://hudson.apache.org/hudson/job/PreCommit-ZOOKEEPER-Build/157//testReport/
        Findbugs warnings: https://hudson.apache.org/hudson/job/PreCommit-ZOOKEEPER-Build/157//artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
        Console output: https://hudson.apache.org/hudson/job/PreCommit-ZOOKEEPER-Build/157//console

        This message is automatically generated.

        Show
        Hadoop QA added a comment - -1 overall. Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12471577/ZOOKEEPER-998.patch against trunk revision 1072085. +1 @author. The patch does not contain any @author tags. -1 tests included. The patch doesn't appear to include any new or modified tests. Please justify why no new tests are needed for this patch. Also please list what manual steps were performed to verify this patch. +1 javadoc. The javadoc tool did not generate any warning messages. +1 javac. The applied patch does not increase the total number of javac compiler warnings. +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings. +1 release audit. The applied patch does not increase the total number of release audit warnings. +1 core tests. The patch passed core unit tests. +1 contrib tests. The patch passed contrib unit tests. Test results: https://hudson.apache.org/hudson/job/PreCommit-ZOOKEEPER-Build/157//testReport/ Findbugs warnings: https://hudson.apache.org/hudson/job/PreCommit-ZOOKEEPER-Build/157//artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html Console output: https://hudson.apache.org/hudson/job/PreCommit-ZOOKEEPER-Build/157//console This message is automatically generated.
        Hide
        Flavio Junqueira added a comment -

        It is still halting, so I'm canceling for now.

        Show
        Flavio Junqueira added a comment - It is still halting, so I'm canceling for now.
        Hide
        Flavio Junqueira added a comment -

        Preliminary patch to fix this problem. It does not include a test yet.

        Show
        Flavio Junqueira added a comment - Preliminary patch to fix this problem. It does not include a test yet.
        Hide
        Flavio Junqueira added a comment -

        Patch with test. The patch essentially introduces a ChannelGroup as the Netty documentation suggests:

        http://docs.jboss.org/netty/3.2/api/org/jboss/netty/channel/ChannelFactory.html

        Show
        Flavio Junqueira added a comment - Patch with test. The patch essentially introduces a ChannelGroup as the Netty documentation suggests: http://docs.jboss.org/netty/3.2/api/org/jboss/netty/channel/ChannelFactory.html
        Hide
        Ivan Kelly added a comment -

        Is the locking on the object the same as the locking in the netty thread you posted?

        I think the reason this patch works is that you call .awaitUninterruptibly() on the channel group. You could just do awaitingUninterruptibly() on the channel.close call in PerChannelBookieClient#close for the same effect. Also, the fact that you call close on the group rather than the individual channels means PerChannelBookieClient#close gets bypassed completely, which could be a potential source of bugs, as someone could put cleanup in there that never gets called.

        Who calls the halt() code in the test code? It might be a good idea to move the releaseExternalResources() call into a new thread to allow halt to be called from callbacks.

          public void halt() throws InterruptedException {
            bookieClient.close();
            bookieWatcher.halt();
            if (ownChannelFactory) {
              new Thread() {
                public void run() {
                  channelFactory.releaseExternalResources();
                }
              }.start();
            }
            if (ownZKHandle) {
              zk.close();
            }
            callbackWorker.shutdown();
            mainWorkerPool.shutdown();
          }
        

        Also, indentation in general is inconsistent over BK, 4 space in some places, 2 space in others. We should standardize. It may even be a good idea to go through the whole tree and reindent now, rather than later when such an action would destroy history (we currently don't have very much history since the split).

        Finally, I think halt() should be called close(). halt() seems to imply that the bookkeeper service is being stopped, not that you're just closing your connection to it.

        Show
        Ivan Kelly added a comment - Is the locking on the object the same as the locking in the netty thread you posted? I think the reason this patch works is that you call .awaitUninterruptibly() on the channel group. You could just do awaitingUninterruptibly() on the channel.close call in PerChannelBookieClient#close for the same effect. Also, the fact that you call close on the group rather than the individual channels means PerChannelBookieClient#close gets bypassed completely, which could be a potential source of bugs, as someone could put cleanup in there that never gets called. Who calls the halt() code in the test code? It might be a good idea to move the releaseExternalResources() call into a new thread to allow halt to be called from callbacks. public void halt() throws InterruptedException { bookieClient.close(); bookieWatcher.halt(); if (ownChannelFactory) { new Thread () { public void run() { channelFactory.releaseExternalResources(); } }.start(); } if (ownZKHandle) { zk.close(); } callbackWorker.shutdown(); mainWorkerPool.shutdown(); } Also, indentation in general is inconsistent over BK, 4 space in some places, 2 space in others. We should standardize. It may even be a good idea to go through the whole tree and reindent now, rather than later when such an action would destroy history (we currently don't have very much history since the split). Finally, I think halt() should be called close(). halt() seems to imply that the bookkeeper service is being stopped, not that you're just closing your connection to it.
        Hide
        Flavio Junqueira added a comment -

        The trunk code in BookiClient.close() is this:

        for (PerChannelBookieClient channel: channels.values()){
                    channel.close();
                }
        

        and I have just replaced it with channelGroup.close(). The documentation of ChannelFactory says:

        Graceful shutdown
        
        To shut down a network application service which is managed by a factory. you should follow the following steps:
        
           1- close all channels created by the factory and their child channels usually using ChannelGroup.close(), and
           2- call releaseExternalResources().
        

        and if you look at the ChannelGroup documentation, I'm just using the way they suggest. Finally, in my understanding of awaitUninterruptibly() is that the thread waits and can't be interrupted.

        On the test, BookKeeper.halt() is invoked from the tearDown method of BaseTestCase, which the test class extends. I tried applying only the new test, and it hangs on my computer.

        Show
        Flavio Junqueira added a comment - The trunk code in BookiClient.close() is this: for (PerChannelBookieClient channel: channels.values()){ channel.close(); } and I have just replaced it with channelGroup.close(). The documentation of ChannelFactory says: Graceful shutdown To shut down a network application service which is managed by a factory. you should follow the following steps: 1- close all channels created by the factory and their child channels usually using ChannelGroup.close(), and 2- call releaseExternalResources(). and if you look at the ChannelGroup documentation, I'm just using the way they suggest. Finally, in my understanding of awaitUninterruptibly() is that the thread waits and can't be interrupted. On the test, BookKeeper.halt() is invoked from the tearDown method of BaseTestCase, which the test class extends. I tried applying only the new test, and it hangs on my computer.
        Hide
        Flavio Junqueira added a comment -

        Until issues brought up are resolved.

        Show
        Flavio Junqueira added a comment - Until issues brought up are resolved.
        Hide
        Ivan Kelly added a comment -

        BookieClient has a list of PerChannelBookieClient which in turn holds the netty Channel. My concern is that, by using ChannelGroup, PerChannelBookieClient#close is bypassed completely. Its not a problem now, as all it does it close the channel, but if in future it is used, it will never be called. I think the core of the problem is that it introduces an asymmetry in the lifetime management of the Channel object. PerChannelBookieClient creates it, but BookieClient destroys it.

        My understanding of awaitUninterruptibly() is the same, and I think that is where the bug was. Since channel.close() is called but not awaited on, the thread moves on and calls releaseExternalResources() while the close is possibly still in progress.

        Show
        Ivan Kelly added a comment - BookieClient has a list of PerChannelBookieClient which in turn holds the netty Channel. My concern is that, by using ChannelGroup, PerChannelBookieClient#close is bypassed completely. Its not a problem now, as all it does it close the channel, but if in future it is used, it will never be called. I think the core of the problem is that it introduces an asymmetry in the lifetime management of the Channel object. PerChannelBookieClient creates it, but BookieClient destroys it. My understanding of awaitUninterruptibly() is the same, and I think that is where the bug was. Since channel.close() is called but not awaited on, the thread moves on and calls releaseExternalResources() while the close is possibly still in progress.
        Hide
        Flavio Junqueira added a comment -

        I'm not sure I understand why you're saying that PerChannelBookieClient#close is bypassed completely. The original trunk code only calls close on the channel of a PerChannelBookieClient. According to the documentation of ChannelGroup, calling close on it causes it to close all channels in the group. I believe the change I'm proposing achieves the same behavior, no?

        Perhaps you're saying that we should just call awaitUninterruptibly() on every PerChannelBookieClient#close instead of creating a channel group? I think it should work.

        Show
        Flavio Junqueira added a comment - I'm not sure I understand why you're saying that PerChannelBookieClient#close is bypassed completely. The original trunk code only calls close on the channel of a PerChannelBookieClient. According to the documentation of ChannelGroup, calling close on it causes it to close all channels in the group. I believe the change I'm proposing achieves the same behavior, no? Perhaps you're saying that we should just call awaitUninterruptibly() on every PerChannelBookieClient#close instead of creating a channel group? I think it should work.
        Hide
        Ivan Kelly added a comment -

        The original code in BookieClient is:

            public void close(){
                for (PerChannelBookieClient channel: channels.values()){
                    channel.close();
                }
            }
        

        which called close on PerChannelBookieClient, not just its channel. The names are confusing and should be changed.

        Perhaps you're saying that we should just call awaitUninterruptibly() on every PerChannelBookieClient#close instead of creating a channel group? I think it should work.

        Yup, this is what I was suggesting.

        Show
        Ivan Kelly added a comment - The original code in BookieClient is: public void close(){ for (PerChannelBookieClient channel: channels.values()){ channel.close(); } } which called close on PerChannelBookieClient, not just its channel. The names are confusing and should be changed. Perhaps you're saying that we should just call awaitUninterruptibly() on every PerChannelBookieClient#close instead of creating a channel group? I think it should work. Yup, this is what I was suggesting.
        Hide
        Flavio Junqueira added a comment -

        Sounds good to me. I'll fix and propose a new patch.

        Show
        Flavio Junqueira added a comment - Sounds good to me. I'll fix and propose a new patch.
        Hide
        Ivan Kelly added a comment -

        Problem was a race between the creation/connection of channels in BookieClient#lookupClient. This meant a client was created, connection and discarded, and therefore never closed.

        This patch defers connection until the channel has been safely put into the channels hashmap.

        Show
        Ivan Kelly added a comment - Problem was a race between the creation/connection of channels in BookieClient#lookupClient. This meant a client was created, connection and discarded, and therefore never closed. This patch defers connection until the channel has been safely put into the channels hashmap.
        Hide
        Flavio Junqueira added a comment -

        Good catch, Ivan. It is actually not even necessary to connect before invoking an operation, since we call connectIfNeededAndDoOp() upon every addEntry and readEntry. It works even without the else block in your version of the patch. There are also a few other changes unrelated to this issue, like a change to pom.xml and fixed typos. Do you want those to get in or they ended up there by accident?

        Show
        Flavio Junqueira added a comment - Good catch, Ivan. It is actually not even necessary to connect before invoking an operation, since we call connectIfNeededAndDoOp() upon every addEntry and readEntry. It works even without the else block in your version of the patch. There are also a few other changes unrelated to this issue, like a change to pom.xml and fixed typos. Do you want those to get in or they ended up there by accident?
        Hide
        Ivan Kelly added a comment -

        They were uncommitted changes. I've regenerated the patch to only include whats committed.

        Show
        Ivan Kelly added a comment - They were uncommitted changes. I've regenerated the patch to only include whats committed.
        Hide
        Benjamin Reed added a comment -

        if i understand this patch correctly, there are two big fixes:

        1) we wait for close to complete
        2) we proactively initiate a connection on lookup

        was 1) really causing a problem?

        the rest is really just cleanup right?

        it looks like all of the additions to BookKeeper.java are just debug messages that should be removed.

        other than these small items the patch looks good.

        Show
        Benjamin Reed added a comment - if i understand this patch correctly, there are two big fixes: 1) we wait for close to complete 2) we proactively initiate a connection on lookup was 1) really causing a problem? the rest is really just cleanup right? it looks like all of the additions to BookKeeper.java are just debug messages that should be removed. other than these small items the patch looks good.
        Hide
        Flavio Junqueira added a comment -

        On 1): According to the discussions I found online about this problem with Netty, if the thread is interrupted while waiting on close, then it can go into an infinite loop. Honestly, I didn't spend time trying to understand what exactly is the problem with netty, only that folks recommend to do it in the way we have in the patch, which makes a difference.

        The real problem, though, is that we were creating new channels, but not keeping track of the, so when shutting down, we were not closing those channels and triggering the netty problem.

        I rarely have a good sense of where log messages have to be inserted, but I do find useful to say what we are shutting down when we call halt() in the BookKeeper object. Since these messages only appear when we shut down the object, I don't see a problem with keeping them, but at the same time I don't feel strongly about removing them if you feel they are unnecessary.

        Show
        Flavio Junqueira added a comment - On 1): According to the discussions I found online about this problem with Netty, if the thread is interrupted while waiting on close, then it can go into an infinite loop. Honestly, I didn't spend time trying to understand what exactly is the problem with netty, only that folks recommend to do it in the way we have in the patch, which makes a difference. The real problem, though, is that we were creating new channels, but not keeping track of the, so when shutting down, we were not closing those channels and triggering the netty problem. I rarely have a good sense of where log messages have to be inserted, but I do find useful to say what we are shutting down when we call halt() in the BookKeeper object. Since these messages only appear when we shut down the object, I don't see a problem with keeping them, but at the same time I don't feel strongly about removing them if you feel they are unnecessary.
        Hide
        Benjamin Reed added a comment -

        i'm fine with 1). we do need to either remove those messages or change them to debug especially since this is a client library. on the server being verbose only messes up the server log, but on the client it can be really annoying for the client application.

        Show
        Benjamin Reed added a comment - i'm fine with 1). we do need to either remove those messages or change them to debug especially since this is a client library. on the server being verbose only messes up the server log, but on the client it can be really annoying for the client application.
        Hide
        Ivan Kelly added a comment -

        Removed trace statements

        Show
        Ivan Kelly added a comment - Removed trace statements
        Hide
        Benjamin Reed added a comment -

        +1 looks good

        Show
        Benjamin Reed added a comment - +1 looks good
        Hide
        Flavio Junqueira added a comment -

        Thanks Ivan for working on the patch and Ben for reviewing it. Committed revision 1143850.

        Show
        Flavio Junqueira added a comment - Thanks Ivan for working on the patch and Ben for reviewing it. Committed revision 1143850.
        Hide
        Ivan Kelly added a comment -

        This issue still seems to be hitting if you run for long enough. To repro,

        true; while [ $? = 0 ]; do mvn test -Dtest=BookieReadWriteTest; done

        Leave this running for 5-6 hours, and the bug should hit. From looking at the code it could be that connect is unsynchronized, so resources could be allocated and lost by concurrent executions of connect(). I'm currently testing a patch to see if this is the case.

        Show
        Ivan Kelly added a comment - This issue still seems to be hitting if you run for long enough. To repro, true; while [ $? = 0 ]; do mvn test -Dtest=BookieReadWriteTest; done Leave this running for 5-6 hours, and the bug should hit. From looking at the code it could be that connect is unsynchronized, so resources could be allocated and lost by concurrent executions of connect(). I'm currently testing a patch to see if this is the case.
        Hide
        Ivan Kelly added a comment -

        I've attached a patch to fix this. I'm not sure how to proceed procedurally with this though, as the original fix is in trunk, and the original fix is good, just incomplete. Perhaps this should get a whole new JIRA.

        Show
        Ivan Kelly added a comment - I've attached a patch to fix this. I'm not sure how to proceed procedurally with this though, as the original fix is in trunk, and the original fix is good, just incomplete. Perhaps this should get a whole new JIRA.
        Hide
        Ivan Kelly added a comment - - edited

        Going to fix in BOOKKEEPER-59. To do it under BK-5 we'd have to pull the former patch and then reapply it with extra changes. This is ugly, especially since the original patch does good stuff.

        Show
        Ivan Kelly added a comment - - edited Going to fix in BOOKKEEPER-59 . To do it under BK-5 we'd have to pull the former patch and then reapply it with extra changes. This is ugly, especially since the original patch does good stuff.

          People

          • Assignee:
            Ivan Kelly
            Reporter:
            Flavio Junqueira
          • Votes:
            0 Vote for this issue
            Watchers:
            1 Start watching this issue

            Dates

            • Created:
              Updated:
              Resolved:

              Development