Details

    • Type: Sub-task Sub-task
    • Status: Closed
    • Priority: Major Major
    • Resolution: Fixed
    • Affects Version/s: None
    • Fix Version/s: 4.2.0
    • Labels:
      None

      Description

      This JIRA discusses how the bookkeeper system will detect underreplication of ledger entries.

      1. BOOKKEEPER-247.diff
        33 kB
        Ivan Kelly
      2. BOOKKEEPER-247-1.patch
        28 kB
        Uma Maheswara Rao G
      3. BOOKKEEPER-247.patch
        25 kB
        Uma Maheswara Rao G
      4. BOOKKEEPER-247.diff
        19 kB
        Ivan Kelly
      5. BOOKKEEPER-247.diff
        19 kB
        Ivan Kelly
      6. BOOKKEEPER-247.patch
        16 kB
        Uma Maheswara Rao G

        Issue Links

          Activity

          Hide
          Ivan Kelly added a comment -

          @rakesh, are you working on this checking now? if not i'd like to take a run at it.

          Show
          Ivan Kelly added a comment - @rakesh, are you working on this checking now? if not i'd like to take a run at it.
          Hide
          Rakesh R added a comment -

          @Ivan, presently I'm working on the BOOKKEEPER-272, as it is the beginning. I'have done the initial draft version and likely will upload the patch to BOOKKEEPER-272. I think we would be able to do the regression on this.

          Please take a look on the comments and attached docs on umbrella JIRA and would like to know your suggestions.

          Thanks,
          Rakesh

          Show
          Rakesh R added a comment - @Ivan, presently I'm working on the BOOKKEEPER-272 , as it is the beginning. I'have done the initial draft version and likely will upload the patch to BOOKKEEPER-272 . I think we would be able to do the regression on this. Please take a look on the comments and attached docs on umbrella JIRA and would like to know your suggestions. Thanks, Rakesh
          Hide
          Ivan Kelly added a comment -

          I've put a first pass at the detection algorithm up onto github[1][2].

          It doesn't make any assumption about where it'll be run from, as it runs through the standard bookkeeper client code. The entry point is LedgerChecker#checkLedger which you pass a LedgerHandle and a callback. On completion, the callback is given a set of LedgerFragmentReplicas, which are the fragments which are underrepliced.

          [1] https://github.com/ivankelly/bookkeeper/tree/BOOKKEEPER-247
          [2] https://github.com/ivankelly/bookkeeper/commit/5b2d079b8792f7bdb63f4f9ae7d78cede85c58b7

          Show
          Ivan Kelly added a comment - I've put a first pass at the detection algorithm up onto github [1] [2] . It doesn't make any assumption about where it'll be run from, as it runs through the standard bookkeeper client code. The entry point is LedgerChecker#checkLedger which you pass a LedgerHandle and a callback. On completion, the callback is given a set of LedgerFragmentReplicas, which are the fragments which are underrepliced. [1] https://github.com/ivankelly/bookkeeper/tree/BOOKKEEPER-247 [2] https://github.com/ivankelly/bookkeeper/commit/5b2d079b8792f7bdb63f4f9ae7d78cede85c58b7
          Hide
          Uma Maheswara Rao G added a comment -

          Thanks a lot Ivan,
          Currently we are planing to detect the replication based on Bookie failures. If one bookie shutdown or failed, then we will be able to detect with help of auditor.
          This code is checking each ledger for under-replication. If one bookie failed, then all ledgers in that bookie should be replicated.

          Use of this ledger checking would be only when some disks failed in bookie and if still continue running the Bookie. That time, same bookie(partial disk failures Bookie) can use this class and find the underreplicated ledgers?

          If that is the case, worth editing the JIRA title as Detect under replication of ledgers on bookie disk failures?

          Show
          Uma Maheswara Rao G added a comment - Thanks a lot Ivan, Currently we are planing to detect the replication based on Bookie failures. If one bookie shutdown or failed, then we will be able to detect with help of auditor. This code is checking each ledger for under-replication. If one bookie failed, then all ledgers in that bookie should be replicated. Use of this ledger checking would be only when some disks failed in bookie and if still continue running the Bookie. That time, same bookie(partial disk failures Bookie) can use this class and find the underreplicated ledgers? If that is the case, worth editing the JIRA title as Detect under replication of ledgers on bookie disk failures?
          Hide
          Ivan Kelly added a comment -

          A bookie failure, is really the failure of a lot of ledger fragments. I think the direction of BOOKKEEPER-272 matches that. The sequence of events for a bookie failure is.

          1. Bookie fails
          2. Auditor puts list of affected ledgers in suspected ledgers znode
          3. Recovery worker takes a ledger from the list, and runs this detection on it. Puts underreplicated ledger fragments in underreplicated znode.
          4. Recovery worker takes an underreplicated ledger fragment, and rereplicates it.

          Each bookie is running a recovery worker, so the work of detection and rereplication will be distributed, while the auditor for checking the bookies will be centralized. Also, i think bookies should run this detection on all their ledgers, every few hours, to detect disk issues.

          Show
          Ivan Kelly added a comment - A bookie failure, is really the failure of a lot of ledger fragments. I think the direction of BOOKKEEPER-272 matches that. The sequence of events for a bookie failure is. Bookie fails Auditor puts list of affected ledgers in suspected ledgers znode Recovery worker takes a ledger from the list, and runs this detection on it. Puts underreplicated ledger fragments in underreplicated znode. Recovery worker takes an underreplicated ledger fragment, and rereplicates it. Each bookie is running a recovery worker, so the work of detection and rereplication will be distributed, while the auditor for checking the bookies will be centralized. Also, i think bookies should run this detection on all their ledgers, every few hours, to detect disk issues.
          Hide
          Uma Maheswara Rao G added a comment -

          We are thinking about the sequence till now like below:

          1.Bookie fails
          2.Auditor puts list of affected ledgers in suspected/underreplicated ledgers znode
          3.Replication worker will take one by one ledger from suspected ledgers znode and re-replicate it.
          If we are able reuse the BookKeeperAdmin code to re-replicate, then BookKeeperAdmin #recoverLedger already finding the fragments and replicating then and there. Am I missing some thing here?

          Otherwise Recovery worker/Replication worker may need to watch two level of data. 1. suspected ledgers znode 2. underreplicated znode.

          Also, i think bookies should run this detection on all their ledgers, every few hours, to detect disk issues

          I agree. I think work can be triggered on disk failures and will run hourly basis by default.

          Show
          Uma Maheswara Rao G added a comment - We are thinking about the sequence till now like below: 1.Bookie fails 2.Auditor puts list of affected ledgers in suspected/underreplicated ledgers znode 3.Replication worker will take one by one ledger from suspected ledgers znode and re-replicate it. If we are able reuse the BookKeeperAdmin code to re-replicate, then BookKeeperAdmin #recoverLedger already finding the fragments and replicating then and there. Am I missing some thing here? Otherwise Recovery worker/Replication worker may need to watch two level of data. 1. suspected ledgers znode 2. underreplicated znode. Also, i think bookies should run this detection on all their ledgers, every few hours, to detect disk issues I agree. I think work can be triggered on disk failures and will run hourly basis by default.
          Hide
          Ivan Kelly added a comment -

          This also works.

          Show
          Ivan Kelly added a comment - This also works.
          Hide
          Rakesh R added a comment -

          @Ivan @Uma
          Just to summarize:- As per my understanding, the Auditor will identify the suspected ledgers and publish to everyone. The re-replicator will take this under replicated ledgers and do the re-replication.

          Then I feel we can use this JIRA for handling self-check on disk failures and do re-replication. Also, it would be good to modify the JIRA title accordingly. How does it sound?

          Show
          Rakesh R added a comment - @Ivan @Uma Just to summarize:- As per my understanding, the Auditor will identify the suspected ledgers and publish to everyone. The re-replicator will take this under replicated ledgers and do the re-replication. Then I feel we can use this JIRA for handling self-check on disk failures and do re-replication. Also, it would be good to modify the JIRA title accordingly. How does it sound?
          Hide
          Ivan Kelly added a comment -

          I think what is in the patch now works as a standalone patch. I'd prefer to keep the patches small and modular like this and do the work over many JIRAs. It makes it easier to get code through the review and testing process. I'll open new JIRAs for the other bits.

          Show
          Ivan Kelly added a comment - I think what is in the patch now works as a standalone patch. I'd prefer to keep the patches small and modular like this and do the work over many JIRAs. It makes it easier to get code through the review and testing process. I'll open new JIRAs for the other bits.
          Hide
          Uma Maheswara Rao G added a comment -

          Hi Ivan, Are you planning to make LedgerChecker as a patch?
          BOOKKEEPER-299 is using LedgerChecker for finding the missed fragments.

          Show
          Uma Maheswara Rao G added a comment - Hi Ivan, Are you planning to make LedgerChecker as a patch? BOOKKEEPER-299 is using LedgerChecker for finding the missed fragments.
          Hide
          Ivan Kelly added a comment -

          @Uma, yes, but there's a chain of JIRAs which need to be submitted first though. BOOKKEEPER-292 needs to go in. Once that is in, I can start working on tests for BOOKKEEPER-2 (I already have the changes to the code itself). Once BOOKKEEPER-2 is in, this can go in, and then BOOKKEEPER-299.

          Perhaps you could review BOOKKEEPER-292 to get the ball rolling. Sijie had looked at it, but hadn't +1'd.

          Show
          Ivan Kelly added a comment - @Uma, yes, but there's a chain of JIRAs which need to be submitted first though. BOOKKEEPER-292 needs to go in. Once that is in, I can start working on tests for BOOKKEEPER-2 (I already have the changes to the code itself). Once BOOKKEEPER-2 is in, this can go in, and then BOOKKEEPER-299 . Perhaps you could review BOOKKEEPER-292 to get the ball rolling. Sijie had looked at it, but hadn't +1'd.
          Hide
          Uma Maheswara Rao G added a comment -

          Yep, I missed BK-2. It was mentioned in BK-2's description already.
          BTW, I have just taken a look on BK-292. Seems Sijie +1'd on it now. and tests looks great.

          Show
          Uma Maheswara Rao G added a comment - Yep, I missed BK-2. It was mentioned in BK-2's description already. BTW, I have just taken a look on BK-292. Seems Sijie +1'd on it now. and tests looks great.
          Hide
          Uma Maheswara Rao G added a comment -

          I think we have to handle one special case in LedgerChecker.

          Take a case, creating the ledger with ensemble 3 and quorum is 2.

          Add a first entry:
          Now ensemble should look like '0 A B C'
          Entry should have added to A, B. Now kill the bookie C.

          Add one more entry. Now Writer will get the exception when writing to C and will lead to ensemble updation.
          Now new ensemble should look like '1 A B D'

          Writer can continue with this ensemble util there is no failure again.

          Now if you run the ledger checker on this Ledger, It will consider '0 A B C' as UnderReplicated Fragment. But here first entry already met the quorum, we need not reoplicate any entries.

          I think we should skip such cases here.

          Some grepped logs related to this issue:

          First entry write:
          
          2012-06-28 14:23:46,797 - INFO  - [main:BookKeeperClusterTestCase@336] - New bookie on port 5002 has been created.
          2012-06-28 14:23:46,970 - INFO  - [New I/O client worker #1-1:PerChannelBookieClient$1@146] - Successfully connected to bookie: /10.18.47.127:5000
          2012-06-28 14:23:46,970 - INFO  - [New I/O client worker #1-2:PerChannelBookieClient$1@146] - Successfully connected to bookie: /10.18.47.127:5001
          2012-06-28 14:23:47,064 - INFO  - [main:TestLedgerChecker@137] - Killing /10.18.47.127:5002 from ensemble=[/10.18.47.127:5000, /10.18.47.127:5001, /10.18.47.127:5002]
          Ensembles after first entry : {0=[/10.18.47.127:5000, /10.18.47.127:5001, /10.18.47.127:5002]}
          .......................
          .......................
          
          
          2012-06-28 14:23:47,549 - INFO  - [main:BookKeeperClusterTestCase@336] - New bookie on port 5003 has been created.
          
          
          Second erntry write:
          
          
          First entry write:
          
          2012-06-28 14:23:46,797 - INFO  - [main:BookKeeperClusterTestCase@336] - New bookie on port 5002 has been created.
          2012-06-28 14:23:46,970 - INFO  - [New I/O client worker #1-1:PerChannelBookieClient$1@146] - Successfully connected to bookie: /XX.XX.XX.127:5000
          2012-06-28 14:23:46,970 - INFO  - [New I/O client worker #1-2:PerChannelBookieClient$1@146] - Successfully connected to bookie: /XX.XX.XX.127:5001
          2012-06-28 14:23:47,064 - INFO  - [main:TestLedgerChecker@137] - Killing /XX.XX.XX.127:5002 from ensemble=[/XX.XX.XX.127:5000, /XX.XX.XX.127:5001, /XX.XX.XX.127:5002]
          Ensembles after first entry : {0=[/XX.XX.XX.127:5000, /XX.XX.XX.127:5001, /XX.XX.XX.127:5002]}
          .......................
          .......................
          
          
          2012-06-28 14:23:47,549 - INFO  - [main:BookKeeperClusterTestCase@336] - New bookie on port 5003 has been created.
          
          
          Second erntry write:
          
          

          2012-06-28 14:23:48,537 - ERROR - [New I/O client boss #1:PerChannelBookieClient$1@151] - Could not connect to bookie: /XX.XX.XX.127:5002
          2012-06-28 14:23:48,537 - WARN - [New I/O client boss #1:PendingAddOp@146] - Write did not succeed: 3, 1
          2012-06-28 14:23:48,584 - INFO - [New I/O client worker #1-4:PerChannelBookieClient$1@146] - Successfully connected to bookie: /XX.XX.XX.127:5003
          Ensembles after second entry :

          {0=[/XX.XX.XX.127:5000, /XX.XX.XX.127:5001, /XX.XX.XX.127:5002], 1=[/XX.XX.XX.127:5000, /XX.XX.XX.127:5001, /XX.XX.XX.127:5003]}

          2012-06-28 14:23:48,631 - ERROR - [pool-4-thread-1:PerChannelBookieClient@618] - Unexpected read response received from bookie: /XX.XX.XX.127:5000 for ledger: 3, entry: 0 , ignoring
          2012-06-28 14:23:49,633 - ERROR - [New I/O client boss #1:PerChannelBookieClient$1@151] - Could not connect to bookie: /XX.XX.XX.127:5002
          2012-06-28 14:23:49,633 - INFO - [main:TestLedgerChecker@160] - unreplicated fragment: Fragment(LedgerID: 3, FirstEntryID: 1[2], LastEntryID: 1[0], Host: /XX.XX.XX.127:5000)
          2012-06-28 14:23:49,633 - INFO - [main:TestLedgerChecker@160] - unreplicated fragment: Fragment(LedgerID: 3, FirstEntryID: 0[1], LastEntryID: 0[-1], Host: /XX.XX.XX.127:5002)
          2012-06-28 14:23:49,633 - INFO - [main:BookKeeperClusterTestCase@92] - TearDown

          
          
          Show
          Uma Maheswara Rao G added a comment - I think we have to handle one special case in LedgerChecker. Take a case, creating the ledger with ensemble 3 and quorum is 2. Add a first entry: Now ensemble should look like '0 A B C' Entry should have added to A, B. Now kill the bookie C. Add one more entry. Now Writer will get the exception when writing to C and will lead to ensemble updation. Now new ensemble should look like '1 A B D' Writer can continue with this ensemble util there is no failure again. Now if you run the ledger checker on this Ledger, It will consider '0 A B C' as UnderReplicated Fragment. But here first entry already met the quorum, we need not reoplicate any entries. I think we should skip such cases here. Some grepped logs related to this issue: First entry write: 2012-06-28 14:23:46,797 - INFO - [main:BookKeeperClusterTestCase@336] - New bookie on port 5002 has been created. 2012-06-28 14:23:46,970 - INFO - [New I/O client worker #1-1:PerChannelBookieClient$1@146] - Successfully connected to bookie: /10.18.47.127:5000 2012-06-28 14:23:46,970 - INFO - [New I/O client worker #1-2:PerChannelBookieClient$1@146] - Successfully connected to bookie: /10.18.47.127:5001 2012-06-28 14:23:47,064 - INFO - [main:TestLedgerChecker@137] - Killing /10.18.47.127:5002 from ensemble=[/10.18.47.127:5000, /10.18.47.127:5001, /10.18.47.127:5002] Ensembles after first entry : {0=[/10.18.47.127:5000, /10.18.47.127:5001, /10.18.47.127:5002]} ....................... ....................... 2012-06-28 14:23:47,549 - INFO - [main:BookKeeperClusterTestCase@336] - New bookie on port 5003 has been created. Second erntry write: First entry write: 2012-06-28 14:23:46,797 - INFO - [main:BookKeeperClusterTestCase@336] - New bookie on port 5002 has been created. 2012-06-28 14:23:46,970 - INFO - [New I/O client worker #1-1:PerChannelBookieClient$1@146] - Successfully connected to bookie: /XX.XX.XX.127:5000 2012-06-28 14:23:46,970 - INFO - [New I/O client worker #1-2:PerChannelBookieClient$1@146] - Successfully connected to bookie: /XX.XX.XX.127:5001 2012-06-28 14:23:47,064 - INFO - [main:TestLedgerChecker@137] - Killing /XX.XX.XX.127:5002 from ensemble=[/XX.XX.XX.127:5000, /XX.XX.XX.127:5001, /XX.XX.XX.127:5002] Ensembles after first entry : {0=[/XX.XX.XX.127:5000, /XX.XX.XX.127:5001, /XX.XX.XX.127:5002]} ....................... ....................... 2012-06-28 14:23:47,549 - INFO - [main:BookKeeperClusterTestCase@336] - New bookie on port 5003 has been created. Second erntry write: 2012-06-28 14:23:48,537 - ERROR - [New I/O client boss #1:PerChannelBookieClient$1@151] - Could not connect to bookie: /XX.XX.XX.127:5002 2012-06-28 14:23:48,537 - WARN - [New I/O client boss #1:PendingAddOp@146] - Write did not succeed: 3, 1 2012-06-28 14:23:48,584 - INFO - [New I/O client worker #1-4:PerChannelBookieClient$1@146] - Successfully connected to bookie: /XX.XX.XX.127:5003 Ensembles after second entry : {0=[/XX.XX.XX.127:5000, /XX.XX.XX.127:5001, /XX.XX.XX.127:5002], 1=[/XX.XX.XX.127:5000, /XX.XX.XX.127:5001, /XX.XX.XX.127:5003]} 2012-06-28 14:23:48,631 - ERROR - [pool-4-thread-1:PerChannelBookieClient@618] - Unexpected read response received from bookie: /XX.XX.XX.127:5000 for ledger: 3, entry: 0 , ignoring 2012-06-28 14:23:49,633 - ERROR - [New I/O client boss #1:PerChannelBookieClient$1@151] - Could not connect to bookie: /XX.XX.XX.127:5002 2012-06-28 14:23:49,633 - INFO - [main:TestLedgerChecker@160] - unreplicated fragment: Fragment(LedgerID: 3, FirstEntryID: 1 [2] , LastEntryID: 1 [0] , Host: /XX.XX.XX.127:5000) 2012-06-28 14:23:49,633 - INFO - [main:TestLedgerChecker@160] - unreplicated fragment: Fragment(LedgerID: 3, FirstEntryID: 0 [1] , LastEntryID: 0 [-1] , Host: /XX.XX.XX.127:5002) 2012-06-28 14:23:49,633 - INFO - [main:BookKeeperClusterTestCase@92] - TearDown
          Hide
          Uma Maheswara Rao G added a comment -

          I think we have to handle one special case in LedgerChecker.

          Take a case, creating the ledger with ensemble 3 and quorum is 2.

          Add a first entry:
          Now ensemble should look like '0 A B C'
          Entry should have added to A, B. Now kill the bookie C.

          Add one more entry. Now Writer will get the exception when writing to C and will lead to ensemble updation.
          Now new ensemble should look like '1 A B D'

          Writer can continue with this ensemble util there is no failure again.

          Now if you run the ledger checker on this Ledger, It will consider '0 A B C' as UnderReplicated Fragment. But here first entry already met the quorum, we need not reoplicate any entries.

          I think we should skip such cases here.

          Some grepped logs related to this issue:

          First entry write:
          
          2012-06-28 14:23:46,797 - INFO  - [main:BookKeeperClusterTestCase@336] - New bookie on port 5002 has been created.
          2012-06-28 14:23:46,970 - INFO  - [New I/O client worker #1-1:PerChannelBookieClient$1@146] - Successfully connected to bookie: /XX.XX.XX.127:5000
          2012-06-28 14:23:46,970 - INFO  - [New I/O client worker #1-2:PerChannelBookieClient$1@146] - Successfully connected to bookie: /XX.XX.XX.127:5001
          2012-06-28 14:23:47,064 - INFO  - [main:TestLedgerChecker@137] - Killing /XX.XX.XX.127:5002 from ensemble=[/XX.XX.XX.127:5000, /XX.XX.XX.127:5001, /XX.XX.XX.127:5002]
          Ensembles after first entry : {0=[/XX.XX.XX.127:5000, /XX.XX.XX.127:5001, /XX.XX.XX.127:5002]}
          .......................
          .......................
          
          
          2012-06-28 14:23:47,549 - INFO  - [main:BookKeeperClusterTestCase@336] - New bookie on port 5003 has been created.
          
          
          Second erntry write:
          
          2012-06-28 14:23:48,537 - ERROR - [New I/O client boss #1:PerChannelBookieClient$1@151] - Could not connect to bookie: /XX.XX.XX.127:5002
          2012-06-28 14:23:48,537 - WARN  - [New I/O client boss #1:PendingAddOp@146] - Write did not succeed: 3, 1
          2012-06-28 14:23:48,584 - INFO  - [New I/O client worker #1-4:PerChannelBookieClient$1@146] - Successfully connected to bookie: /XX.XX.XX.127:5003
          Ensembles after second entry : {0=[/XX.XX.XX.127:5000, /XX.XX.XX.127:5001, /XX.XX.XX.127:5002], 1=[/XX.XX.XX.127:5000, /XX.XX.XX.127:5001, /XX.XX.XX.127:5003]}
          2012-06-28 14:23:48,631 - ERROR - [pool-4-thread-1:PerChannelBookieClient@618] - Unexpected read response received from bookie: /XX.XX.XX.127:5000 for ledger: 3, entry: 0 , ignoring
          2012-06-28 14:23:49,633 - ERROR - [New I/O client boss #1:PerChannelBookieClient$1@151] - Could not connect to bookie: /XX.XX.XX.127:5002
          2012-06-28 14:23:49,633 - INFO  - [main:TestLedgerChecker@160] - unreplicated fragment: Fragment(LedgerID: 3, FirstEntryID: 1[2], LastEntryID: 1[0], Host: /XX.XX.XX.127:5000)
          2012-06-28 14:23:49,633 - INFO  - [main:TestLedgerChecker@160] - unreplicated fragment: Fragment(LedgerID: 3, FirstEntryID: 0[1], LastEntryID: 0[-1], Host: /XX.XX.XX.127:5002)
          2012-06-28 14:23:49,633 - INFO  - [main:BookKeeperClusterTestCase@92] - TearDown
          
          Show
          Uma Maheswara Rao G added a comment - I think we have to handle one special case in LedgerChecker. Take a case, creating the ledger with ensemble 3 and quorum is 2. Add a first entry: Now ensemble should look like '0 A B C' Entry should have added to A, B. Now kill the bookie C. Add one more entry. Now Writer will get the exception when writing to C and will lead to ensemble updation. Now new ensemble should look like '1 A B D' Writer can continue with this ensemble util there is no failure again. Now if you run the ledger checker on this Ledger, It will consider '0 A B C' as UnderReplicated Fragment. But here first entry already met the quorum, we need not reoplicate any entries. I think we should skip such cases here. Some grepped logs related to this issue: First entry write: 2012-06-28 14:23:46,797 - INFO - [main:BookKeeperClusterTestCase@336] - New bookie on port 5002 has been created. 2012-06-28 14:23:46,970 - INFO - [New I/O client worker #1-1:PerChannelBookieClient$1@146] - Successfully connected to bookie: /XX.XX.XX.127:5000 2012-06-28 14:23:46,970 - INFO - [New I/O client worker #1-2:PerChannelBookieClient$1@146] - Successfully connected to bookie: /XX.XX.XX.127:5001 2012-06-28 14:23:47,064 - INFO - [main:TestLedgerChecker@137] - Killing /XX.XX.XX.127:5002 from ensemble=[/XX.XX.XX.127:5000, /XX.XX.XX.127:5001, /XX.XX.XX.127:5002] Ensembles after first entry : {0=[/XX.XX.XX.127:5000, /XX.XX.XX.127:5001, /XX.XX.XX.127:5002]} ....................... ....................... 2012-06-28 14:23:47,549 - INFO - [main:BookKeeperClusterTestCase@336] - New bookie on port 5003 has been created. Second erntry write: 2012-06-28 14:23:48,537 - ERROR - [New I/O client boss #1:PerChannelBookieClient$1@151] - Could not connect to bookie: /XX.XX.XX.127:5002 2012-06-28 14:23:48,537 - WARN - [New I/O client boss #1:PendingAddOp@146] - Write did not succeed: 3, 1 2012-06-28 14:23:48,584 - INFO - [New I/O client worker #1-4:PerChannelBookieClient$1@146] - Successfully connected to bookie: /XX.XX.XX.127:5003 Ensembles after second entry : {0=[/XX.XX.XX.127:5000, /XX.XX.XX.127:5001, /XX.XX.XX.127:5002], 1=[/XX.XX.XX.127:5000, /XX.XX.XX.127:5001, /XX.XX.XX.127:5003]} 2012-06-28 14:23:48,631 - ERROR - [pool-4-thread-1:PerChannelBookieClient@618] - Unexpected read response received from bookie: /XX.XX.XX.127:5000 for ledger: 3, entry: 0 , ignoring 2012-06-28 14:23:49,633 - ERROR - [New I/O client boss #1:PerChannelBookieClient$1@151] - Could not connect to bookie: /XX.XX.XX.127:5002 2012-06-28 14:23:49,633 - INFO - [main:TestLedgerChecker@160] - unreplicated fragment: Fragment(LedgerID: 3, FirstEntryID: 1[2], LastEntryID: 1[0], Host: /XX.XX.XX.127:5000) 2012-06-28 14:23:49,633 - INFO - [main:TestLedgerChecker@160] - unreplicated fragment: Fragment(LedgerID: 3, FirstEntryID: 0[1], LastEntryID: 0[-1], Host: /XX.XX.XX.127:5002) 2012-06-28 14:23:49,633 - INFO - [main:BookKeeperClusterTestCase@92] - TearDown
          Hide
          Uma Maheswara Rao G added a comment -

          Ivan, I have generated a patch with your GitHub code and attached patch with Test, that will fail in the above explained scenario.

          Show
          Uma Maheswara Rao G added a comment - Ivan, I have generated a patch with your GitHub code and attached patch with Test, that will fail in the above explained scenario.
          Hide
          Uma Maheswara Rao G added a comment -

          How about skipping this kind of fragments like below
          while finding fault index from ensemble start ID, it should have a check that should not cross end entry ID also. If it is not able to find the failed BK index with in this range then we can skip this fragment right?

          Show
          Uma Maheswara Rao G added a comment - How about skipping this kind of fragments like below while finding fault index from ensemble start ID, it should have a check that should not cross end entry ID also. If it is not able to find the failed BK index with in this range then we can skip this fragment right?
          Hide
          Ivan Kelly added a comment -

          This shouldn't happen, as I only check the replicas which should have the entry, for each entry. I'll check your test now. I probably missed something. Well spotted

          Show
          Ivan Kelly added a comment - This shouldn't happen, as I only check the replicas which should have the entry, for each entry. I'll check your test now. I probably missed something. Well spotted
          Hide
          Ivan Kelly added a comment -

          The problem was quite simple in the end. The code that ensures I only checked the correct replicas, only worked if the number of entries in the ledger was greater than the number of bookies in the ensemble. Adding two checks fixed it. I also moved around the code a bit. I've pushed it to github and attached a new patch. I still need to add tests to this, so the patch shouldn't be considered ready for submission yet.

          Show
          Ivan Kelly added a comment - The problem was quite simple in the end. The code that ensures I only checked the correct replicas, only worked if the number of entries in the ledger was greater than the number of bookies in the ensemble. Adding two checks fixed it. I also moved around the code a bit. I've pushed it to github and attached a new patch. I still need to add tests to this, so the patch shouldn't be considered ready for submission yet.
          Hide
          Uma Maheswara Rao G added a comment -

          Great

          Thanks a lot for addressing the issue.

          I have some scenarios verified tests, I should have included them in my previous patch, but don't want to messup with the patch which explains the problem.
          Now I have updated the patch with the below minor modifications and newly added tests.

          • added javadoc for hasEntry methods.
          • updated correct header comments in DistributedSchedule, RRDSchedule algorithm.
          • Included some more tests which I have added recently.
          • Made the inner classes of LedgerChecker as private static.
          • also updated the class level javadocs for LedgerChecker and LedgerFragment.

          If you like the changes, you can push this to your git and make it ready for review with your added tests. And many of the scenarios works well for me.

          Show
          Uma Maheswara Rao G added a comment - Great Thanks a lot for addressing the issue. I have some scenarios verified tests, I should have included them in my previous patch, but don't want to messup with the patch which explains the problem. Now I have updated the patch with the below minor modifications and newly added tests. added javadoc for hasEntry methods. updated correct header comments in DistributedSchedule, RRDSchedule algorithm. Included some more tests which I have added recently. Made the inner classes of LedgerChecker as private static. also updated the class level javadocs for LedgerChecker and LedgerFragment. If you like the changes, you can push this to your git and make it ready for review with your added tests. And many of the scenarios works well for me.
          Hide
          Uma Maheswara Rao G added a comment -

          Hi Ivan, Is it possible for you to mark the last fragment in ledger from LedgerChecker?
          So, that i can use this information in ReplicationWorker for fencing the current writer.

          This is the point what we have discussed in mailing list recently.
          Also we have seen that case rarely 2 or 3 times in our testing till now.
          So, I will try to fence if last fragment is underReplicated fragment.

          Still I am thinking about introduce some delay(say 30sec, can be configarable) for this kind of ledgers and if still ledger is in that situation then I will fence.
          Here hope is that, within that delay period, client may write the entries and may reform the ensemble with good bookies, then that fragment would not be last fragment in ensemble.

          Show
          Uma Maheswara Rao G added a comment - Hi Ivan, Is it possible for you to mark the last fragment in ledger from LedgerChecker? So, that i can use this information in ReplicationWorker for fencing the current writer. This is the point what we have discussed in mailing list recently. Also we have seen that case rarely 2 or 3 times in our testing till now. So, I will try to fence if last fragment is underReplicated fragment. Still I am thinking about introduce some delay(say 30sec, can be configarable) for this kind of ledgers and if still ledger is in that situation then I will fence. Here hope is that, within that delay period, client may write the entries and may reform the ensemble with good bookies, then that fragment would not be last fragment in ensemble.
          Hide
          Ivan Kelly added a comment -

          This may not be the clearest place to check this though. I think it would be clearer to check this at the point at which the replication worker is selecting the ledger to rereplicate. If it sees that the ledger is still open, it can wait for a grace period before running the checker and rereplicating any missing fragments.

          Show
          Ivan Kelly added a comment - This may not be the clearest place to check this though. I think it would be clearer to check this at the point at which the replication worker is selecting the ledger to rereplicate. If it sees that the ledger is still open, it can wait for a grace period before running the checker and rereplicating any missing fragments.
          Hide
          Uma Maheswara Rao G added a comment -

          Thanks Ivan, Make sense to me. I have filed a JIRA for grace period delay in replication for opned underReplicated(last fragment) ledgers. BK-325. I will handle along with that JIRA.

          Show
          Uma Maheswara Rao G added a comment - Thanks Ivan, Make sense to me. I have filed a JIRA for grace period delay in replication for opned underReplicated(last fragment) ledgers. BK-325. I will handle along with that JIRA.
          Hide
          Uma Maheswara Rao G added a comment -

          Hi Ivan,

          Here is one boundary case came across. When client written single entry and waiting, at this time if one BK goes down, then Ledger checker is not able to find that as underReplicated fragment.

          I think it should detect that as under replicated, then I can wait in PendingReplicationWorker for grace period and fence the ledger. If it is not able to detect as underReplicated, we can not know whether really there is no fragments underReplicated or some one else already replicated them.

          Here is test to reproduce:

          /**
               * Tests that LedgerChecker should one fragment as underReplicated
               * if there is an open ledger with single entry written.
               */
              @Test(timeout = 3000)
              public void testShouldGetOneFragmentWithSingleEntryOpenedLedger() throws Exception {
                  LedgerHandle lh = bkc.createLedger(3, 3, BookKeeper.DigestType.CRC32,
                          TEST_LEDGER_PASSWORD);
                  lh.addEntry(TEST_LEDGER_ENTRY_DATA);
                  ArrayList<InetSocketAddress> firstEnsemble = lh.getLedgerMetadata()
                          .getEnsembles().get(0L);
                  InetSocketAddress lastBookieFromEnsemble = firstEnsemble.get(0);
                  LOG.info("Killing " + lastBookieFromEnsemble + " from ensemble="
                          + firstEnsemble);
                  killBookie(lastBookieFromEnsemble);
          
                  startNewBookie();
                  
                  //Open ledger separately for Ledger checker.
                  LedgerHandle lh1 =bkc.openLedgerNoRecovery(lh.getId(), BookKeeper.DigestType.CRC32,
                          TEST_LEDGER_PASSWORD);
                  
                  Set<LedgerFragment> result = getUnderReplicatedFragments(lh1);
                  assertNotNull("Result shouldn't be null", result);
                  assertEquals("There should be 1 fragment. But returned fragments are "
                          + result, 1, result.size());
              }
          
             private Set<LedgerFragment> getUnderReplicatedFragments(LedgerHandle lh)
                      throws InterruptedException {
                  LedgerChecker checker = new LedgerChecker(bkc);
                  CheckerCallback cb = new CheckerCallback();
                  checker.checkLedger(lh, cb);
                  Set<LedgerFragment> result = cb.waitAndGetResult();
                  return result;
              }
          

          I think the problem is, when ledger is not closed then getLastConfirmed may not give real last entry. we will get one lesser than real last entry confirmed. If the ledger is closed, then only we can get real last entry. In this case also, it has written only one entry and it was in open state. so, it may get last confirmed is nothing. Finally it is not detecting ledger any fragments from the ledger as underReplicated.

          If I write one more entry extra, then it can detect as underReplicated.

          Thanks
          Uma

          Show
          Uma Maheswara Rao G added a comment - Hi Ivan, Here is one boundary case came across. When client written single entry and waiting, at this time if one BK goes down, then Ledger checker is not able to find that as underReplicated fragment. I think it should detect that as under replicated, then I can wait in PendingReplicationWorker for grace period and fence the ledger. If it is not able to detect as underReplicated, we can not know whether really there is no fragments underReplicated or some one else already replicated them. Here is test to reproduce: /** * Tests that LedgerChecker should one fragment as underReplicated * if there is an open ledger with single entry written. */ @Test(timeout = 3000) public void testShouldGetOneFragmentWithSingleEntryOpenedLedger() throws Exception { LedgerHandle lh = bkc.createLedger(3, 3, BookKeeper.DigestType.CRC32, TEST_LEDGER_PASSWORD); lh.addEntry(TEST_LEDGER_ENTRY_DATA); ArrayList<InetSocketAddress> firstEnsemble = lh.getLedgerMetadata() .getEnsembles().get(0L); InetSocketAddress lastBookieFromEnsemble = firstEnsemble.get(0); LOG.info( "Killing " + lastBookieFromEnsemble + " from ensemble=" + firstEnsemble); killBookie(lastBookieFromEnsemble); startNewBookie(); //Open ledger separately for Ledger checker. LedgerHandle lh1 =bkc.openLedgerNoRecovery(lh.getId(), BookKeeper.DigestType.CRC32, TEST_LEDGER_PASSWORD); Set<LedgerFragment> result = getUnderReplicatedFragments(lh1); assertNotNull( "Result shouldn't be null " , result); assertEquals( "There should be 1 fragment. But returned fragments are " + result, 1, result.size()); } private Set<LedgerFragment> getUnderReplicatedFragments(LedgerHandle lh) throws InterruptedException { LedgerChecker checker = new LedgerChecker(bkc); CheckerCallback cb = new CheckerCallback(); checker.checkLedger(lh, cb); Set<LedgerFragment> result = cb.waitAndGetResult(); return result; } I think the problem is, when ledger is not closed then getLastConfirmed may not give real last entry. we will get one lesser than real last entry confirmed. If the ledger is closed, then only we can get real last entry. In this case also, it has written only one entry and it was in open state. so, it may get last confirmed is nothing. Finally it is not detecting ledger any fragments from the ledger as underReplicated. If I write one more entry extra, then it can detect as underReplicated. Thanks Uma
          Hide
          Rakesh R added a comment -

          @Uma @Ivan

          Here is one boundary case came across. When client written single entry and waiting, at this time if one BK goes down, then Ledger checker is not able to find that as underReplicated fragment.

          I've seen the LedgerRecoveryOp.java is doing the following logic to identify the lastAddConfirmed entry. Can we have similar stuff here also in the replication logic if the ledger is in open state.

              /**
               * Try to read past the last confirmed.
               */
              private void doRecoveryRead() {
                  lh.lastAddConfirmed++;
                  lh.asyncReadEntries(lh.lastAddConfirmed, lh.lastAddConfirmed, this, null);
              }
          
          Show
          Rakesh R added a comment - @Uma @Ivan Here is one boundary case came across. When client written single entry and waiting, at this time if one BK goes down, then Ledger checker is not able to find that as underReplicated fragment. I've seen the LedgerRecoveryOp.java is doing the following logic to identify the lastAddConfirmed entry. Can we have similar stuff here also in the replication logic if the ledger is in open state. /** * Try to read past the last confirmed. */ private void doRecoveryRead() { lh.lastAddConfirmed++; lh.asyncReadEntries(lh.lastAddConfirmed, lh.lastAddConfirmed, this , null ); }
          Hide
          Uma Maheswara Rao G added a comment -

          Current writer only can get the lastConfirmed entry correctly.
          Here LedgerChecker will not be a Writer in any case, considering one entry extra will the option to go i feel.

           long lastAddConfirmed = lh
                                  .getLastAddConfirmed();
                          if(lh.metadata.isClosed() == false){
                              lastAddConfirmed++;
                          }
                          fragments.add(new LedgerFragment(lh.getId(), curEntryId, lastAddConfirmed, i, curEnsemble, lh
                                  .getDistributionSchedule()));
          

          with this, above given test passed. Infact all other tests also should modify to use different LedgerHandle. otherwise this will make the test failures , because writer will anyway get lastConfirmed correctly.

          Show
          Uma Maheswara Rao G added a comment - Current writer only can get the lastConfirmed entry correctly. Here LedgerChecker will not be a Writer in any case, considering one entry extra will the option to go i feel. long lastAddConfirmed = lh .getLastAddConfirmed(); if (lh.metadata.isClosed() == false ){ lastAddConfirmed++; } fragments.add( new LedgerFragment(lh.getId(), curEntryId, lastAddConfirmed, i, curEnsemble, lh .getDistributionSchedule())); with this, above given test passed. Infact all other tests also should modify to use different LedgerHandle. otherwise this will make the test failures , because writer will anyway get lastConfirmed correctly.
          Hide
          Uma Maheswara Rao G added a comment -

          One more case here, the above proposal may break that. For example if the ledger is really empty, it just created the ledger. then also, there will be one ensemble created i guess then, above incrementing will give unnecessary assumption that ledger freagment is in underReplicated.
          Any alternatives?

          Show
          Uma Maheswara Rao G added a comment - One more case here, the above proposal may break that. For example if the ledger is really empty, it just created the ledger. then also, there will be one ensemble created i guess then, above incrementing will give unnecessary assumption that ledger freagment is in underReplicated. Any alternatives?
          Hide
          Uma Maheswara Rao G added a comment -

          Even though we treat this empty ledger as underReplicated due to the above lastAddConfirmed++, we will post pone this ledger as last fragment is underReplicated and it is in open state. After pendingReplication timeout, this ledger will get force fenced.

          That means, after adding the PendingReplicationWorker logic and ReplicationWorker logic, there won't be any ledger in open state more than PendingReplication grace period interval. Ofcource this is configurable. Is this behaviour fine with you all?

          Show
          Uma Maheswara Rao G added a comment - Even though we treat this empty ledger as underReplicated due to the above lastAddConfirmed++, we will post pone this ledger as last fragment is underReplicated and it is in open state. After pendingReplication timeout, this ledger will get force fenced. That means, after adding the PendingReplicationWorker logic and ReplicationWorker logic, there won't be any ledger in open state more than PendingReplication grace period interval. Ofcource this is configurable. Is this behaviour fine with you all?
          Hide
          Uma Maheswara Rao G added a comment -

          Attached the patch with above change and tests correction. Added one more test with this special case.

          Show
          Uma Maheswara Rao G added a comment - Attached the patch with above change and tests correction. Added one more test with this special case.
          Hide
          Ivan Kelly added a comment -

          The first case (single entry in ledger) is actually straightforward to fix. The problem is that the fragment to check has the last entry set wrong, so it never actually checks.
          https://github.com/ivankelly/bookkeeper/commit/73b55018efdc451b356781fbe9b25a148878b308

          The empty ledger problem im not so sure of how to fix. It's not obvious how to tell the difference between. There's already a test case for this #testShouldNotGetAnyFragmentWithEmptyLedger

          Btw, if you guys want to add changes to the patch, could you clone the github branch, and make the mods there. It makes it easier to track what the changes between the patches are.

          Show
          Ivan Kelly added a comment - The first case (single entry in ledger) is actually straightforward to fix. The problem is that the fragment to check has the last entry set wrong, so it never actually checks. https://github.com/ivankelly/bookkeeper/commit/73b55018efdc451b356781fbe9b25a148878b308 The empty ledger problem im not so sure of how to fix. It's not obvious how to tell the difference between. There's already a test case for this #testShouldNotGetAnyFragmentWithEmptyLedger Btw, if you guys want to add changes to the patch, could you clone the github branch, and make the mods there. It makes it easier to track what the changes between the patches are.
          Hide
          Uma Maheswara Rao G added a comment -

          Hi Ivan,

          Btw, if you guys want to add changes to the patch, could you clone the github branch, and make the mods there. It makes it easier to track what the changes between the patches are.

          Yeah, I should have done that. Will do from next if any more changes on it.

          The empty ledger problem im not so sure of how to fix. It's not obvious how to tell the difference between. There's already a test case for this #testShouldNotGetAnyFragmentWithEmptyLedger

          The current fix should solve the single entry problem. But with empty ledger also we may get the entry as currentEntryId. This will make ledger checker mark that fragment as underReplicated.
          Anyway this fragment will be postponed by worker as lastFragment is in underReplication state. Once that pendingReplication timedout, this will get force fenced.

          So, with PendingReplications logic, it turns out that, there won't be any ledger idle more than the pendingReplicationTime out period.

          Only my worry is that, In Namenode case,

          Just start the namenode and dont write any data. By this time, if any Bookie goes down from selected quoram. Then after 30sec, if user startes writing then it will fail with fenced exception because Replication worker already would have fenced. then this will make one switch unnecessarily.

          On startup, keeping system idle for some time may be normal scenario as OM may start the process one another. To start all processes, it may take some time. Because of this idleness, it will cause one switch. Another argument is, since this is startup, one switch should be ok.

          Show
          Uma Maheswara Rao G added a comment - Hi Ivan, Btw, if you guys want to add changes to the patch, could you clone the github branch, and make the mods there. It makes it easier to track what the changes between the patches are. Yeah, I should have done that. Will do from next if any more changes on it. The empty ledger problem im not so sure of how to fix. It's not obvious how to tell the difference between. There's already a test case for this #testShouldNotGetAnyFragmentWithEmptyLedger The current fix should solve the single entry problem. But with empty ledger also we may get the entry as currentEntryId. This will make ledger checker mark that fragment as underReplicated. Anyway this fragment will be postponed by worker as lastFragment is in underReplication state. Once that pendingReplication timedout, this will get force fenced. So, with PendingReplications logic, it turns out that, there won't be any ledger idle more than the pendingReplicationTime out period. Only my worry is that, In Namenode case, Just start the namenode and dont write any data. By this time, if any Bookie goes down from selected quoram. Then after 30sec, if user startes writing then it will fail with fenced exception because Replication worker already would have fenced. then this will make one switch unnecessarily. On startup, keeping system idle for some time may be normal scenario as OM may start the process one another. To start all processes, it may take some time. Because of this idleness, it will cause one switch. Another argument is, since this is startup, one switch should be ok.
          Hide
          Ivan Kelly added a comment -

          Fixed the empty ledger case.

          https://github.com/ivankelly/bookkeeper/commit/b85c94930a325b1cc275fee26d7d62e9e0cdc778

          Regarding the namenode, even without this fix, you dont need to worry about that, as the namenode always writes a START_SEGMENT entry after starting a new log segment.

          Show
          Ivan Kelly added a comment - Fixed the empty ledger case. https://github.com/ivankelly/bookkeeper/commit/b85c94930a325b1cc275fee26d7d62e9e0cdc778 Regarding the namenode, even without this fix, you dont need to worry about that, as the namenode always writes a START_SEGMENT entry after starting a new log segment.
          Hide
          Uma Maheswara Rao G added a comment -

          Yep, you are right It should not be a problem. openEditLogForWrite()should do that on startingActiveServices.

          On quick look change looks great. I like the idea to confirm the empty ledger.

          Thanks a lot for the update.

          small nit: testShouldGet3FragmentWithEmptyLedgerButBookiesDead --> testShouldGet2FragmentsWithEmptyLedgerButBookiesDead ?

          Show
          Uma Maheswara Rao G added a comment - Yep, you are right It should not be a problem. openEditLogForWrite()should do that on startingActiveServices. On quick look change looks great. I like the idea to confirm the empty ledger. Thanks a lot for the update. small nit: testShouldGet3FragmentWithEmptyLedgerButBookiesDead --> testShouldGet2FragmentsWithEmptyLedgerButBookiesDead ?
          Hide
          Uma Maheswara Rao G added a comment -

          Ivan, as per the order of JIRAs to go in, I think now turn is for this JIRA.
          Could you please generate a patch with all our latest changes discussed? It should be simple as you already added the changed code in github.
          So, that we can continue our further reviews.

          Show
          Uma Maheswara Rao G added a comment - Ivan, as per the order of JIRAs to go in, I think now turn is for this JIRA. Could you please generate a patch with all our latest changes discussed? It should be simple as you already added the changed code in github. So, that we can continue our further reviews.
          Hide
          Ivan Kelly added a comment -

          @Uma,

          I'll get to this today. Yesterday was a bank holiday here, which is why there was no movement.

          Show
          Ivan Kelly added a comment - @Uma, I'll get to this today. Yesterday was a bank holiday here, which is why there was no movement.
          Hide
          Uma Maheswara Rao G added a comment -

          Great, Thanks Ivan.

          Show
          Uma Maheswara Rao G added a comment - Great, Thanks Ivan.
          Hide
          Ivan Kelly added a comment -

          New patch is simply the old patch, rebased onto trunk and one findbug error fixed.

          Show
          Ivan Kelly added a comment - New patch is simply the old patch, rebased onto trunk and one findbug error fixed.
          Hide
          Uma Maheswara Rao G added a comment -

          Latest patch looks great to me. Majority of the issues we already found and fixed above. I don't see any issues now in the patch.

          I am +1 to push this in.

          Others, could you please add your comments, if we miss anything here.

          Thanks,
          Uma

          Show
          Uma Maheswara Rao G added a comment - Latest patch looks great to me. Majority of the issues we already found and fixed above. I don't see any issues now in the patch. I am +1 to push this in. Others, could you please add your comments, if we miss anything here. Thanks, Uma
          Hide
          Rakesh R added a comment -

          Thanks Ivan and the new Patch looks good.
          +1 from me. As we discussed and covered all the known scenarios it would be fine to go.

          Show
          Rakesh R added a comment - Thanks Ivan and the new Patch looks good. +1 from me. As we discussed and covered all the known scenarios it would be fine to go.
          Hide
          Ivan Kelly added a comment -

          Committed as r1374195. Thanks for reviewing Uma & Rakesh

          Show
          Ivan Kelly added a comment - Committed as r1374195. Thanks for reviewing Uma & Rakesh
          Hide
          Hudson added a comment -

          Integrated in bookkeeper-trunk #651 (See https://builds.apache.org/job/bookkeeper-trunk/651/)
          BOOKKEEPER-247: Detection of under replication (ivank) (Revision 1374195)

          Result = ABORTED
          ivank :
          Files :

          • /zookeeper/bookkeeper/trunk/CHANGES.txt
          • /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DistributionSchedule.java
          • /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerChecker.java
          • /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerFragment.java
          • /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RoundRobinDistributionSchedule.java
          • /zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerChecker.java
          Show
          Hudson added a comment - Integrated in bookkeeper-trunk #651 (See https://builds.apache.org/job/bookkeeper-trunk/651/ ) BOOKKEEPER-247 : Detection of under replication (ivank) (Revision 1374195) Result = ABORTED ivank : Files : /zookeeper/bookkeeper/trunk/CHANGES.txt /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DistributionSchedule.java /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerChecker.java /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerFragment.java /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RoundRobinDistributionSchedule.java /zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerChecker.java

            People

            • Assignee:
              Ivan Kelly
              Reporter:
              Ivan Kelly
            • Votes:
              0 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development