Details

    Description

      handling zookeeper session expire event in bookie server & bookie watcher.

      Attachments

        1. BOOKKEEPER-855.patch
          25 kB
          Sijie Guo

        Issue Links

          Activity

            mmerli Matteo Merli added a comment -

            Closed as part of BookKeeper-4.4.0 release

            mmerli Matteo Merli added a comment - Closed as part of BookKeeper-4.4.0 release
            hudson Hudson added a comment -

            FAILURE: Integrated in bookkeeper-master #1268 (See https://builds.apache.org/job/bookkeeper-master/1268/)
            BOOKKEEPER-855: handle session expire event in bookie (sijie) (sijie: rev 92722ee9c34b069e23d1a87d7fc78256b8540268)

            • bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ReadOnlyBookieTest.java
            • bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java
            • bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestBookieWatcher.java
            • bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ReadOnlyBookie.java
            • bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieZKExpireTest.java
            • bookkeeper-server/src/test/java/org/apache/bookkeeper/zookeeper/TestZooKeeperClient.java
            • bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java
            • bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java
            • bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java
            • bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieInitializationTest.java
            hudson Hudson added a comment - FAILURE: Integrated in bookkeeper-master #1268 (See https://builds.apache.org/job/bookkeeper-master/1268/ ) BOOKKEEPER-855 : handle session expire event in bookie (sijie) (sijie: rev 92722ee9c34b069e23d1a87d7fc78256b8540268) bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ReadOnlyBookieTest.java bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestBookieWatcher.java bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ReadOnlyBookie.java bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieZKExpireTest.java bookkeeper-server/src/test/java/org/apache/bookkeeper/zookeeper/TestZooKeeperClient.java bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieInitializationTest.java
            hustlmsp Sijie Guo added a comment -

            commit as 92722ee9c34b069e23d1a87d7fc78256b8540268 in master.

            thanks ikelly and merlimat for reviewing it.

            hustlmsp Sijie Guo added a comment - commit as 92722ee9c34b069e23d1a87d7fc78256b8540268 in master. thanks ikelly and merlimat for reviewing it.
            githubbot ASF GitHub Bot added a comment -

            Github user asfgit closed the pull request at:

            https://github.com/apache/bookkeeper/pull/1

            githubbot ASF GitHub Bot added a comment - Github user asfgit closed the pull request at: https://github.com/apache/bookkeeper/pull/1
            githubbot ASF GitHub Bot added a comment - - edited

            Github user ivankelly commented on the pull request:

            https://github.com/apache/bookkeeper/pull/1#issuecomment-174041727

            lgtm +1. There seems to be a failing test though.

            githubbot ASF GitHub Bot added a comment - - edited Github user ivankelly commented on the pull request: https://github.com/apache/bookkeeper/pull/1#issuecomment-174041727 lgtm +1. There seems to be a failing test though.
            hustlmsp Sijie Guo added a comment -

            fpj ikelly could any one review this since merlimat is not available?

            hustlmsp Sijie Guo added a comment - fpj ikelly could any one review this since merlimat is not available?
            githubbot ASF GitHub Bot added a comment -

            Github user sijie commented on the pull request:

            https://github.com/apache/bookkeeper/pull/1#issuecomment-171234734

            @merlimat I've updated the pr to shutdown state service when closing bookie. let me know if that looks good to you.

            githubbot ASF GitHub Bot added a comment - Github user sijie commented on the pull request: https://github.com/apache/bookkeeper/pull/1#issuecomment-171234734 @merlimat I've updated the pr to shutdown state service when closing bookie. let me know if that looks good to you.
            githubbot ASF GitHub Bot added a comment -

            Github user sijie commented on a diff in the pull request:

            https://github.com/apache/bookkeeper/pull/1#discussion_r49566517

            — Diff: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java —
            @@ -142,7 +145,11 @@
            final protected String zkBookieRegPath;
            final protected String zkBookieReadOnlyPath;

            + final private AtomicBoolean zkRegistered = new AtomicBoolean(false);
            final protected AtomicBoolean readOnly = new AtomicBoolean(false);
            + // executor to manage the state changes for a bookie.
            + final ExecutorService stateService = Executors.newSingleThreadExecutor(
            — End diff –

            @merlimat yup, that's a good catch. will fix it.

            githubbot ASF GitHub Bot added a comment - Github user sijie commented on a diff in the pull request: https://github.com/apache/bookkeeper/pull/1#discussion_r49566517 — Diff: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java — @@ -142,7 +145,11 @@ final protected String zkBookieRegPath; final protected String zkBookieReadOnlyPath; + final private AtomicBoolean zkRegistered = new AtomicBoolean(false); final protected AtomicBoolean readOnly = new AtomicBoolean(false); + // executor to manage the state changes for a bookie. + final ExecutorService stateService = Executors.newSingleThreadExecutor( — End diff – @merlimat yup, that's a good catch. will fix it.
            hadoopqa Hadoop QA added a comment -

            Testing JIRA BOOKKEEPER-855

            Patch BOOKKEEPER-855.patch downloaded at Tue Dec 15 04:45:49 UTC 2015

            ----------------------------

            +1 PATCH_APPLIES
            +1 CLEAN
            +1 RAW_PATCH_ANALYSIS
            . +1 the patch does not introduce any @author tags
            . +1 the patch does not introduce any tabs
            . +1 the patch does not introduce any trailing spaces
            . +1 the patch does not introduce any line longer than 120
            . +1 the patch does adds/modifies 5 testcase(s)
            +1 RAT
            . +1 the patch does not seem to introduce new RAT warnings
            +1 JAVADOC
            . +1 the patch does not seem to introduce new Javadoc warnings
            +1 COMPILE
            . +1 HEAD compiles
            . +1 patch compiles
            . +1 the patch does not seem to introduce new javac warnings
            +1 FINDBUGS
            . +1 the patch does not seem to introduce new Findbugs warnings
            -1 TESTS
            . Tests run: 997
            . Tests failed: 3
            . Tests errors: 2

            . The patch failed the following testcases:

            . testThrottleWithServerSideFilter[0](org.apache.hedwig.server.delivery.TestThrottlingDelivery)
            . testSimpleChat(org.apache.hedwig.jms.BasicJMSTest)
            . testBookieShouldTurnWritableFromReadOnly(org.apache.bookkeeper.test.ReadOnlyBookieTest)

            +1 DISTRO
            . +1 distro tarball builds with the patch

            ----------------------------
            -1 Overall result, please check the reported -1(s)

            The full output of the test-patch run is available at

            . https://builds.apache.org/job/bookkeeper-trunk-precommit-build/974/

            hadoopqa Hadoop QA added a comment - Testing JIRA BOOKKEEPER-855 Patch BOOKKEEPER-855.patch downloaded at Tue Dec 15 04:45:49 UTC 2015 ---------------------------- +1 PATCH_APPLIES +1 CLEAN +1 RAW_PATCH_ANALYSIS . +1 the patch does not introduce any @author tags . +1 the patch does not introduce any tabs . +1 the patch does not introduce any trailing spaces . +1 the patch does not introduce any line longer than 120 . +1 the patch does adds/modifies 5 testcase(s) +1 RAT . +1 the patch does not seem to introduce new RAT warnings +1 JAVADOC . +1 the patch does not seem to introduce new Javadoc warnings +1 COMPILE . +1 HEAD compiles . +1 patch compiles . +1 the patch does not seem to introduce new javac warnings +1 FINDBUGS . +1 the patch does not seem to introduce new Findbugs warnings -1 TESTS . Tests run: 997 . Tests failed: 3 . Tests errors: 2 . The patch failed the following testcases: . testThrottleWithServerSideFilter [0] (org.apache.hedwig.server.delivery.TestThrottlingDelivery) . testSimpleChat(org.apache.hedwig.jms.BasicJMSTest) . testBookieShouldTurnWritableFromReadOnly(org.apache.bookkeeper.test.ReadOnlyBookieTest) +1 DISTRO . +1 distro tarball builds with the patch ---------------------------- -1 Overall result, please check the reported -1(s) The full output of the test-patch run is available at . https://builds.apache.org/job/bookkeeper-trunk-precommit-build/974/
            githubbot ASF GitHub Bot added a comment -

            Github user merlimat commented on the pull request:

            https://github.com/apache/bookkeeper/pull/1#issuecomment-161300318

            Looks good to me, just the executor shutdown thing

            githubbot ASF GitHub Bot added a comment - Github user merlimat commented on the pull request: https://github.com/apache/bookkeeper/pull/1#issuecomment-161300318 Looks good to me, just the executor shutdown thing
            githubbot ASF GitHub Bot added a comment -

            Github user merlimat commented on a diff in the pull request:

            https://github.com/apache/bookkeeper/pull/1#discussion_r46415317

            — Diff: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java —
            @@ -142,7 +145,11 @@
            final protected String zkBookieRegPath;
            final protected String zkBookieReadOnlyPath;

            + final private AtomicBoolean zkRegistered = new AtomicBoolean(false);
            final protected AtomicBoolean readOnly = new AtomicBoolean(false);
            + // executor to manage the state changes for a bookie.
            + final ExecutorService stateService = Executors.newSingleThreadExecutor(
            — End diff –

            `stateService` needs to be shut down at bookie stop

            githubbot ASF GitHub Bot added a comment - Github user merlimat commented on a diff in the pull request: https://github.com/apache/bookkeeper/pull/1#discussion_r46415317 — Diff: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java — @@ -142,7 +145,11 @@ final protected String zkBookieRegPath; final protected String zkBookieReadOnlyPath; + final private AtomicBoolean zkRegistered = new AtomicBoolean(false); final protected AtomicBoolean readOnly = new AtomicBoolean(false); + // executor to manage the state changes for a bookie. + final ExecutorService stateService = Executors.newSingleThreadExecutor( — End diff – `stateService` needs to be shut down at bookie stop
            hustlmsp Sijie Guo added a comment -

            ping. could anyone review this? fpj merlimat ikelly

            hustlmsp Sijie Guo added a comment - ping. could anyone review this? fpj merlimat ikelly
            hustlmsp Sijie Guo added a comment -

            Sent a pull request via GitHub.

            hustlmsp Sijie Guo added a comment - Sent a pull request via GitHub.
            githubbot ASF GitHub Bot added a comment -

            GitHub user sijie opened a pull request:

            https://github.com/apache/bookkeeper/pull/1

            BOOKKEEPER-855: handle session expire event in bookie

            This change is to retry bookie registration when zookeeper session expired.

            You can merge this pull request into a Git repository by running:

            $ git pull https://github.com/sijie/bookkeeper sijie/BOOKKEEPER-855

            Alternatively you can review and apply these changes as the patch at:

            https://github.com/apache/bookkeeper/pull/1.patch

            To close this pull request, make a commit to your master/trunk branch
            with (at least) the following in the commit message:

            This closes #1


            commit 726e44554c7bcd839b275c0b626ebda54133c221
            Author: Sijie Guo <sijie@apache.org>
            Date: 2015-05-23T06:45:24Z

            ZooKeeper Client

            commit a74561d003227a47026a328ad308dc774c8107bf
            Author: Sijie Guo <sijie@apache.org>
            Date: 2015-05-24T06:18:35Z

            Test case for bookie watcher

            commit 03cb0863484ead1181e928357fd83cdd0448397b
            Author: Sijie Guo <sijie@apache.org>
            Date: 2015-05-24T06:22:40Z

            new line

            commit 85a0ff1f316f8871c29f550cf9d42116f0031687
            Author: Sijie Guo <sijie@apache.org>
            Date: 2015-05-24T08:01:25Z

            Handle zookeeper session expiry on bookies

            commit 984d19ba67d4e1835416b614e235b4dae53d124b
            Author: Sijie Guo <sijie@apache.org>
            Date: 2015-10-29T05:15:13Z

            Merge branch 'master' into sijie/BOOKKEEPER-855

            Conflicts:
            bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java
            bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java
            bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java
            bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AutoRecoveryMain.java
            bookkeeper-server/src/main/java/org/apache/bookkeeper/zookeeper/ZooKeeperClient.java
            bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieInitializationTest.java
            bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperClientZKSessionExpiry.java

            commit d39dfa47b08248c3851d6e9c1fc8bb8fb8b93449
            Author: Sijie Guo <sijie@apache.org>
            Date: 2015-11-03T07:09:56Z

            Fix


            githubbot ASF GitHub Bot added a comment - GitHub user sijie opened a pull request: https://github.com/apache/bookkeeper/pull/1 BOOKKEEPER-855 : handle session expire event in bookie This change is to retry bookie registration when zookeeper session expired. You can merge this pull request into a Git repository by running: $ git pull https://github.com/sijie/bookkeeper sijie/ BOOKKEEPER-855 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/bookkeeper/pull/1.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #1 commit 726e44554c7bcd839b275c0b626ebda54133c221 Author: Sijie Guo <sijie@apache.org> Date: 2015-05-23T06:45:24Z ZooKeeper Client commit a74561d003227a47026a328ad308dc774c8107bf Author: Sijie Guo <sijie@apache.org> Date: 2015-05-24T06:18:35Z Test case for bookie watcher commit 03cb0863484ead1181e928357fd83cdd0448397b Author: Sijie Guo <sijie@apache.org> Date: 2015-05-24T06:22:40Z new line commit 85a0ff1f316f8871c29f550cf9d42116f0031687 Author: Sijie Guo <sijie@apache.org> Date: 2015-05-24T08:01:25Z Handle zookeeper session expiry on bookies commit 984d19ba67d4e1835416b614e235b4dae53d124b Author: Sijie Guo <sijie@apache.org> Date: 2015-10-29T05:15:13Z Merge branch 'master' into sijie/ BOOKKEEPER-855 Conflicts: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AutoRecoveryMain.java bookkeeper-server/src/main/java/org/apache/bookkeeper/zookeeper/ZooKeeperClient.java bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieInitializationTest.java bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperClientZKSessionExpiry.java commit d39dfa47b08248c3851d6e9c1fc8bb8fb8b93449 Author: Sijie Guo <sijie@apache.org> Date: 2015-11-03T07:09:56Z Fix
            hustlmsp Sijie Guo added a comment - - edited

            rakeshr fpj this patch was generated based on BOOKKEEPER-705 & BOOKKEEPER-796. Could you review BOOKKEEPER-796 first?

            hustlmsp Sijie Guo added a comment - - edited rakeshr fpj this patch was generated based on BOOKKEEPER-705 & BOOKKEEPER-796 . Could you review BOOKKEEPER-796 first?

            As Rakesh has also observed, the patch isn't applying cleanly to master. Some changes to Bookie.java are failing to apply.

            fpj Flavio Paiva Junqueira added a comment - As Rakesh has also observed, the patch isn't applying cleanly to master. Some changes to Bookie.java are failing to apply.

            Thanks hustlmsp for the contribution.

            Could you please point me to the bookkeeper repository, am referring to apache bookkeeper repo
            I think the patch is not generated out of this and is not properly applying to me. For example, I couldn't see the below logic in this repo.

            .operationRetryPolicy(new BoundExponentialBackoffRetryPolicy(conf.getZkTimeout(),
                                    conf.getZkTimeout(), Integer.MAX_VALUE))
            

            Just few comments based on the attached patch. Probably will do more reviews once I get the actual repo.

            1. After looking at the patch, it handles the expiry case of Bookie server. In that case, please remove the & bookie watcher part from jira description.
            2. Please shutdown the stateService executor in Bookie#shutdown.
            3. Please make the stateService executor threads as daemon by giving the new DaemonThreadFactory()
            4. Before re-register operation, its good to add a log saying:
                 "ZK client connection expired and scheduling a re-register operation. It may take few seconds!"
              
            5. In BookieZKExpireTest, instead of sleeping Thread.sleep(3000);, can we think of sliced sleeping 500millis and do retries couple of time to see bk server is running?
            rakeshr Rakesh Radhakrishnan added a comment - Thanks hustlmsp for the contribution. Could you please point me to the bookkeeper repository, am referring to apache bookkeeper repo I think the patch is not generated out of this and is not properly applying to me. For example, I couldn't see the below logic in this repo. .operationRetryPolicy( new BoundExponentialBackoffRetryPolicy(conf.getZkTimeout(), conf.getZkTimeout(), Integer .MAX_VALUE)) Just few comments based on the attached patch. Probably will do more reviews once I get the actual repo. After looking at the patch, it handles the expiry case of Bookie server. In that case, please remove the & bookie watcher part from jira description. Please shutdown the stateService executor in Bookie#shutdown. Please make the stateService executor threads as daemon by giving the new DaemonThreadFactory() Before re-register operation, its good to add a log saying: "ZK client connection expired and scheduling a re-register operation. It may take few seconds!" In BookieZKExpireTest, instead of sleeping Thread.sleep(3000); , can we think of sliced sleeping 500millis and do retries couple of time to see bk server is running?
            hadoopqa Hadoop QA added a comment -

            Testing JIRA BOOKKEEPER-855

            Patch BOOKKEEPER-855.patch downloaded at Sun May 24 08:16:20 UTC 2015

            ----------------------------

            -1 Patch failed to apply to head of branch

            ----------------------------

            hadoopqa Hadoop QA added a comment - Testing JIRA BOOKKEEPER-855 Patch BOOKKEEPER-855.patch downloaded at Sun May 24 08:16:20 UTC 2015 ---------------------------- -1 Patch failed to apply to head of branch ----------------------------
            hustlmsp Sijie Guo added a comment -

            attach a patch on handle zookeeper session expiry on bookies. it is based on BOOKKEEPER-705.

            hustlmsp Sijie Guo added a comment - attach a patch on handle zookeeper session expiry on bookies. it is based on BOOKKEEPER-705 .

            People

              hustlmsp Sijie Guo
              hustlmsp Sijie Guo
              Votes:
              0 Vote for this issue
              Watchers:
              7 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: