Details

    • Type: Improvement Improvement
    • Status: Resolved
    • Priority: Major Major
    • Resolution: Fixed
    • Affects Version/s: None
    • Fix Version/s: 4.3.0
    • Component/s: bookkeeper-server
    • Labels:
      None

      Description

      Currently, SyncThread made a checkpoint too frequently, which affects performance. data is writing to entry logger file might be blocked by syncing same entry logger file, which affect bookie to achieve higher throughput. We could schedule checkpoint only when rotating an entry log file. so new incoming entries would be written to newer entry log file and old entry log file could be synced.

      1. 0001-BOOKKEEPER-564-CheckpointSource.patch
        57 kB
        Ivan Kelly
      2. 0001-BOOKKEEPER-564-CheckpointSource.patch
        52 kB
        Ivan Kelly
      3. 0001-BOOKKEEPER-564-CheckpointSource.patch
        53 kB
        Ivan Kelly
      4. 0001-BOOKKEEPER-584-CheckpointSource.patch
        53 kB
        Ivan Kelly
      5. BOOKKEEPER-564.patch
        66 kB
        Sijie Guo
      6. 0001-BOOKKEEPER-564-Better-checkpoint-mechanism.patch
        80 kB
        Ivan Kelly
      7. 0001-BOOKKEEPER-564-Better-checkpoint-mechanism.patch
        75 kB
        Ivan Kelly
      8. BOOKKEEPER-564.patch
        65 kB
        Sijie Guo
      9. 0002-BOOKKEEPER-564-Better-checkpoint-mechanism.patch
        30 kB
        Ivan Kelly
      10. BOOKKEEPER-564.patch
        60 kB
        Sijie Guo

        Issue Links

          Activity

          Hide
          Sijie Guo added a comment -

          attach a patch to provide Checkpoint abstraction to make checkpoint. to unblock the pending dependency for BOOKKEEPER-572.

          Show
          Sijie Guo added a comment - attach a patch to provide Checkpoint abstraction to make checkpoint. to unblock the pending dependency for BOOKKEEPER-572 .
          Hide
          Hadoop QA added a comment -

          Testing JIRA BOOKKEEPER-564

          Patch BOOKKEEPER-564.patch downloaded at Mon Mar 18 07:10:43 UTC 2013

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

          +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 contains 5 line(s) with trailing spaces
          . +1 the patch does not introduce any line longer than 120
          . +1 the patch does adds/modifies 6 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: 818
          +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/289/

          Show
          Hadoop QA added a comment - Testing JIRA BOOKKEEPER-564 Patch BOOKKEEPER-564.patch downloaded at Mon Mar 18 07:10:43 UTC 2013 ---------------------------- +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 contains 5 line(s) with trailing spaces . +1 the patch does not introduce any line longer than 120 . +1 the patch does adds/modifies 6 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: 818 +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/289/
          Hide
          Ivan Kelly added a comment -

          The patch seems to mix a couple of changes together, and doesn't actually make it much easier to address BOOKKEEPER-572's problem. However, the general idea of the changes is quite good. I have the following comments

          • SyncThread javadoc is out of date now.
          • SyncThread#startCheckpoint calls offsetCheckPoint directly. offsetCheckPoint only called by startCheckpoint. offsetCheckPoint should be removed and the code put directly in #startCheckpoint
          • the names requestCheckpoint and startCheckpoint are too synonymous. requestCheckpoint suggests that the checkpoint should begin at that point in time. It would be better to rename requestCheckpoint to newCheckpoint.
          • rename CheckpointProgress to Checkpointer
          • in InterleavedLedgerStorage, assert checkPointer != null in the constructor. It's not nice to be passing around nulls. Create a Null implementation of checkpointer if you want not to use a full one in tests.
          • InterleavedLedgerStorage#processEntry() is never used. Remove from this patch.
          • flushOptional(boolean, boolean) is ugly. It's impossible to tell how flushOptional(false, true) is different to flushOptional(true,false) by just looking at it. For this reason, boolean parameters should be used very sparingly. The force parameter is used incorrectly. In LedgerCacheImpl#flushLedger(boolean), the parameter is used to specify whether all indices should be flushed or only one. In fact, this is a perfect example of why boolean parameters suck. I would suggest breaking flushOptional into two methods and call them directly from #checkpoint() and #flush().
          • In fact, I think you should get rid of separate #flush() and #checkpoint() completely, as #flush only seems to be used when shutting down. So you can do whatever flush is doing in the #shutdown() method.
          • Why make entrylogger preallocation optional?
          • #checkpoint.checkpointComplete(running); so we only cleanup old journals on shutdown? what if we never shutdown? I think this job may be better served by a background thread or something in the journal, and never exposed to the upper level.
          Show
          Ivan Kelly added a comment - The patch seems to mix a couple of changes together, and doesn't actually make it much easier to address BOOKKEEPER-572 's problem. However, the general idea of the changes is quite good. I have the following comments SyncThread javadoc is out of date now. SyncThread#startCheckpoint calls offsetCheckPoint directly. offsetCheckPoint only called by startCheckpoint. offsetCheckPoint should be removed and the code put directly in #startCheckpoint the names requestCheckpoint and startCheckpoint are too synonymous. requestCheckpoint suggests that the checkpoint should begin at that point in time. It would be better to rename requestCheckpoint to newCheckpoint. rename CheckpointProgress to Checkpointer in InterleavedLedgerStorage, assert checkPointer != null in the constructor. It's not nice to be passing around nulls. Create a Null implementation of checkpointer if you want not to use a full one in tests. InterleavedLedgerStorage#processEntry() is never used. Remove from this patch. flushOptional(boolean, boolean) is ugly. It's impossible to tell how flushOptional(false, true) is different to flushOptional(true,false) by just looking at it. For this reason, boolean parameters should be used very sparingly. The force parameter is used incorrectly. In LedgerCacheImpl#flushLedger(boolean), the parameter is used to specify whether all indices should be flushed or only one. In fact, this is a perfect example of why boolean parameters suck. I would suggest breaking flushOptional into two methods and call them directly from #checkpoint() and #flush(). In fact, I think you should get rid of separate #flush() and #checkpoint() completely, as #flush only seems to be used when shutting down. So you can do whatever flush is doing in the #shutdown() method. Why make entrylogger preallocation optional? #checkpoint.checkpointComplete(running); so we only cleanup old journals on shutdown? what if we never shutdown? I think this job may be better served by a background thread or something in the journal, and never exposed to the upper level.
          Hide
          Ivan Kelly added a comment -

          I've rejigged this to work on top of the decoupled checkpoint mechanism from BOOKKEEPER-572. The patch does the same thing about only flushing the entrylogs when they're rolled, but take a different approach. Instead of the entrylogger triggering the sync, the bookie monitors the amount of unsynced data it has and when it reaches a threshold it triggers the sync.

          Show
          Ivan Kelly added a comment - I've rejigged this to work on top of the decoupled checkpoint mechanism from BOOKKEEPER-572 . The patch does the same thing about only flushing the entrylogs when they're rolled, but take a different approach. Instead of the entrylogger triggering the sync, the bookie monitors the amount of unsynced data it has and when it reaches a threshold it triggers the sync.
          Hide
          Ivan Kelly added a comment -

          This is just a preliminary patch btw, it needs tests and some polishing (I still haven't updated the syncThread javadoc, etc).

          Show
          Ivan Kelly added a comment - This is just a preliminary patch btw, it needs tests and some polishing (I still haven't updated the syncThread javadoc, etc).
          Hide
          Sijie Guo added a comment -

          the jira is to separate checkpoint with flush, a checkpoint just need to sync old data before checkpoint without affecting current written data. in your patch, each time you synced data, you still flush everything dirty in ledger storage which affects under writing operations. the point why we trigger checkpoint when rolling the data is new data will be added to a new entry logger (which is in filesystem page cache) when old data is syncing to disk. so adding operation will not be affected by background syncing.

          And this is also important when adapting skiplist proposal. if you triggered checkpoint in sync thread, you might end up flushing current in-use skiplist, which is bad.

          BTW, checkpoint here doesn't resolve BOOKKEEPER-572 directly, but it provides clear interface for checkpoints. BOOKKEEPER-572 could use its interface, this is the point why I mentioned this jira in BOOKKEEPER-572.

          Show
          Sijie Guo added a comment - the jira is to separate checkpoint with flush, a checkpoint just need to sync old data before checkpoint without affecting current written data. in your patch, each time you synced data, you still flush everything dirty in ledger storage which affects under writing operations. the point why we trigger checkpoint when rolling the data is new data will be added to a new entry logger (which is in filesystem page cache) when old data is syncing to disk. so adding operation will not be affected by background syncing. And this is also important when adapting skiplist proposal. if you triggered checkpoint in sync thread, you might end up flushing current in-use skiplist, which is bad. BTW, checkpoint here doesn't resolve BOOKKEEPER-572 directly, but it provides clear interface for checkpoints. BOOKKEEPER-572 could use its interface, this is the point why I mentioned this jira in BOOKKEEPER-572 .
          Hide
          Ivan Kelly added a comment -

          With BOOKKEEPER-572 the way we checkpoint fundamentally changes though. It's no longer the case that we can flush/checkpoint the ledger storage and assume that everything that was in the journal before the flush/checkpoint has now been persisted. I couldn't see how to fix this in BOOKKEEPER-572, even with the checkpoint mechanism.

          I need to think about this more. I hadn't considered the skiplist case deeply. Perhaps we need some sort of transaction id. I would like to avoid the storage making calls to the journal though (even if it is through interfaces).

          Show
          Ivan Kelly added a comment - With BOOKKEEPER-572 the way we checkpoint fundamentally changes though. It's no longer the case that we can flush/checkpoint the ledger storage and assume that everything that was in the journal before the flush/checkpoint has now been persisted. I couldn't see how to fix this in BOOKKEEPER-572 , even with the checkpoint mechanism. I need to think about this more. I hadn't considered the skiplist case deeply. Perhaps we need some sort of transaction id. I would like to avoid the storage making calls to the journal though (even if it is through interfaces).
          Hide
          Sijie Guo added a comment -

          the patch here already covered the most part you made the change in BOOKKEEPER-572. the only thing is that we need a place to tracking checkpoints to ensure data in ledger storage are synced to disk before that checkpoint. (in bookkeeper-572, you tracks the checkpoints in SyncThread). but since checkpointing is a behavior of ledger storage, it would be better to keep checkpoint in ledger storage rather than in SyncThread, so it could control how to checkpoint. so either interleaved ledger storage or skiplist based on ledger storage could control when to do checkpointing based on its storage characteristics, rather than flushing everything in ledger storage in SyncThread.

          Show
          Sijie Guo added a comment - the patch here already covered the most part you made the change in BOOKKEEPER-572 . the only thing is that we need a place to tracking checkpoints to ensure data in ledger storage are synced to disk before that checkpoint. (in bookkeeper-572, you tracks the checkpoints in SyncThread). but since checkpointing is a behavior of ledger storage, it would be better to keep checkpoint in ledger storage rather than in SyncThread, so it could control how to checkpoint. so either interleaved ledger storage or skiplist based on ledger storage could control when to do checkpointing based on its storage characteristics, rather than flushing everything in ledger storage in SyncThread.
          Hide
          Ivan Kelly added a comment -

          but since checkpointing is a behavior of ledger storage, it would be better to keep checkpoint in ledger storage rather than in SyncThread,

          So why have a sync thread? The checkpoint/mark is a property of the ledger storage. It signifies the point in history to which the ledger storage has synced to disk, and therefore the point from which the bookie needs to start applying entries from the journal on a restart. The actual journal only uses it as a convenience mechanism for GC.

          So we should give the ledger storage absolute control over checkpointing. The ledger storage should also take care of storing the mark also, since we store it in the ledger directories. The LedgerStorage should have a method, LedgerStorage#getSyncedMark() which gives back the mark. This method can be used on recovery to find where to start applying the journal from. Also there can be a background thread to gc the journals based on what #getSyncedMark returns.

          Also we'll need to give the mark to the ledger storage at some point. pre BOOKKEEPER-572, this can be done after adding to ledger storage, but before adding to the journal. post BOOKKEEPER-572, the mark should be taken from the journal before adding to it, and give to the ledger storage after it has been applied to the journal.

          Show
          Ivan Kelly added a comment - but since checkpointing is a behavior of ledger storage, it would be better to keep checkpoint in ledger storage rather than in SyncThread, So why have a sync thread? The checkpoint/mark is a property of the ledger storage. It signifies the point in history to which the ledger storage has synced to disk, and therefore the point from which the bookie needs to start applying entries from the journal on a restart. The actual journal only uses it as a convenience mechanism for GC. So we should give the ledger storage absolute control over checkpointing. The ledger storage should also take care of storing the mark also, since we store it in the ledger directories. The LedgerStorage should have a method, LedgerStorage#getSyncedMark() which gives back the mark. This method can be used on recovery to find where to start applying the journal from. Also there can be a background thread to gc the journals based on what #getSyncedMark returns. Also we'll need to give the mark to the ledger storage at some point. pre BOOKKEEPER-572 , this can be done after adding to ledger storage, but before adding to the journal. post BOOKKEEPER-572 , the mark should be taken from the journal before adding to it, and give to the ledger storage after it has been applied to the journal.
          Hide
          Sijie Guo added a comment -

          In fact, I think you should get rid of separate #flush() and #checkpoint() completely, as #flush only seems to be used when shutting down. So you can do whatever flush is doing in the #shutdown() method.

          separate #flush and #checkpoint would make interface clearly rather than mixing things in #shutdown method.

          Why make entrylogger preallocation optional?

          for testing stuffs.

          #checkpoint.checkpointComplete(running); so we only cleanup old journals on shutdown?

          we expose as a compact flag, indicated whether we need to compact journals after checkpoint. running means if sync thread is running, we compact each time checkpoint. if running is false, it means shutting down. we don't compact journal, which speed up shutting down process.

          Show
          Sijie Guo added a comment - In fact, I think you should get rid of separate #flush() and #checkpoint() completely, as #flush only seems to be used when shutting down. So you can do whatever flush is doing in the #shutdown() method. separate #flush and #checkpoint would make interface clearly rather than mixing things in #shutdown method. Why make entrylogger preallocation optional? for testing stuffs. #checkpoint.checkpointComplete(running); so we only cleanup old journals on shutdown? we expose as a compact flag, indicated whether we need to compact journals after checkpoint. running means if sync thread is running, we compact each time checkpoint. if running is false, it means shutting down. we don't compact journal, which speed up shutting down process.
          Hide
          Sijie Guo added a comment -

          new patch addressed comments.

          Show
          Sijie Guo added a comment - new patch addressed comments.
          Hide
          Sijie Guo added a comment -

          So we should give the ledger storage absolute control over checkpointing. The ledger storage should also take care of storing the mark also, since we store it in the ledger directories. The LedgerStorage should have a method, LedgerStorage#getSyncedMark() which gives back the mark. This method can be used on recovery to find where to start applying the journal from. Also there can be a background thread to gc the journals based on what #getSyncedMark returns.

          Also we'll need to give the mark to the ledger storage at some point. pre BOOKKEEPER-572, this can be done after adding to ledger storage, but before adding to the journal. post BOOKKEEPER-572, the mark should be taken from the journal before adding to it, and give to the ledger storage after it has been applied to the journal.

          you could just pass the checkpoint when adding entry to ledger storage. so for interleaved ledger storage, when it rolls an entry log file, it used its kept checkpoint to do checkpointing; for skip list based ledger storage, each skip list kept a checkpoint, when it snapshot current skip list, it used the checkpoint kept in snapshotted skip list to do checkpointing. I don't see how difficult that BOOKKEEPER-572 could be addressed based on this jira.

          Show
          Sijie Guo added a comment - So we should give the ledger storage absolute control over checkpointing. The ledger storage should also take care of storing the mark also, since we store it in the ledger directories. The LedgerStorage should have a method, LedgerStorage#getSyncedMark() which gives back the mark. This method can be used on recovery to find where to start applying the journal from. Also there can be a background thread to gc the journals based on what #getSyncedMark returns. Also we'll need to give the mark to the ledger storage at some point. pre BOOKKEEPER-572 , this can be done after adding to ledger storage, but before adding to the journal. post BOOKKEEPER-572 , the mark should be taken from the journal before adding to it, and give to the ledger storage after it has been applied to the journal. you could just pass the checkpoint when adding entry to ledger storage. so for interleaved ledger storage, when it rolls an entry log file, it used its kept checkpoint to do checkpointing; for skip list based ledger storage, each skip list kept a checkpoint, when it snapshot current skip list, it used the checkpoint kept in snapshotted skip list to do checkpointing. I don't see how difficult that BOOKKEEPER-572 could be addressed based on this jira.
          Hide
          Hadoop QA added a comment -

          Testing JIRA BOOKKEEPER-564

          Patch BOOKKEEPER-564.patch downloaded at Tue Apr 9 07:21:39 UTC 2013

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

          +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 6 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: 833
          . Tests failed: 0
          . Tests errors: 1

          . The patch failed the following testcases:

          .

          +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/320/

          Show
          Hadoop QA added a comment - Testing JIRA BOOKKEEPER-564 Patch BOOKKEEPER-564.patch downloaded at Tue Apr 9 07:21:39 UTC 2013 ---------------------------- +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 6 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: 833 . Tests failed: 0 . Tests errors: 1 . The patch failed the following testcases: . +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/320/
          Hide
          Ivan Kelly added a comment -

          The more I look at it, the less I like this Checkpointer interface. It's doing exactly the type of thing I was trying to get rid of with BOOKKEEPER-572 originally. Specifically, the ledger storage is reaching around to make a call on itself through this interface (SafeEntryLogAdder does this, and it's nasty).

          The motivation for this interface is to allow the ledger storage itself to control when it makes a checkpoint/flush. As I said in a previous comment we can remove the sync thread and move the logic into the ledger storage directly. I'm trying this now, should have a patch later.

          separate #flush and #checkpoint would make interface clearly rather than mixing things in #shutdown method.

          They can be removed from the interface completely if we allow ledger storage to handle its own flushing.

          Show
          Ivan Kelly added a comment - The more I look at it, the less I like this Checkpointer interface. It's doing exactly the type of thing I was trying to get rid of with BOOKKEEPER-572 originally. Specifically, the ledger storage is reaching around to make a call on itself through this interface (SafeEntryLogAdder does this, and it's nasty). The motivation for this interface is to allow the ledger storage itself to control when it makes a checkpoint/flush. As I said in a previous comment we can remove the sync thread and move the logic into the ledger storage directly. I'm trying this now, should have a patch later. separate #flush and #checkpoint would make interface clearly rather than mixing things in #shutdown method. They can be removed from the interface completely if we allow ledger storage to handle its own flushing.
          Hide
          Ivan Kelly added a comment -

          New patch moves the sync thread into the ledger storage, removing the need for the checkpointer interface.

          org.apache.bookkeeper.test.ReadOnlyBookieTest still failing, as in Sijie's patch.

          Show
          Ivan Kelly added a comment - New patch moves the sync thread into the ledger storage, removing the need for the checkpointer interface. org.apache.bookkeeper.test.ReadOnlyBookieTest still failing, as in Sijie's patch.
          Hide
          Hadoop QA added a comment -

          Testing JIRA BOOKKEEPER-564

          Patch 0001-BOOKKEEPER-564-Better-checkpoint-mechanism.patch downloaded at Tue Apr 9 14:47:53 UTC 2013

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

          +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 9 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 seems to introduce 2 new Findbugs warning(s) in module(s) [bookkeeper-server]
          +1 TESTS
          . Tests run: 833
          +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/323/

          Show
          Hadoop QA added a comment - Testing JIRA BOOKKEEPER-564 Patch 0001-BOOKKEEPER-564-Better-checkpoint-mechanism.patch downloaded at Tue Apr 9 14:47:53 UTC 2013 ---------------------------- +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 9 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 seems to introduce 2 new Findbugs warning(s) in module(s) [bookkeeper-server] +1 TESTS . Tests run: 833 +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/323/
          Hide
          Ivan Kelly added a comment -

          New patch fixes findbugs, and ReadOnlyBookieTest.

          Show
          Ivan Kelly added a comment - New patch fixes findbugs, and ReadOnlyBookieTest.
          Hide
          Sijie Guo added a comment -

          The more I look at it, the less I like this Checkpointer interface. It's doing exactly the type of thing I was trying to get rid of with BOOKKEEPER-572 originally.

          this is also what I pointed in previous comment "the patch here already covered the most part you made the change in BOOKKEEPER-572.".

          Show
          Sijie Guo added a comment - The more I look at it, the less I like this Checkpointer interface. It's doing exactly the type of thing I was trying to get rid of with BOOKKEEPER-572 originally. this is also what I pointed in previous comment "the patch here already covered the most part you made the change in BOOKKEEPER-572 .".
          Hide
          Hadoop QA added a comment -

          Testing JIRA BOOKKEEPER-564

          Patch 0001-BOOKKEEPER-564-Better-checkpoint-mechanism.patch downloaded at Tue Apr 9 16:41:31 UTC 2013

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

          +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 9 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: 833
          +1 DISTRO
          . +1 distro tarball builds with the patch

          ----------------------------
          +1 Overall result, good!, no -1s

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

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

          Show
          Hadoop QA added a comment - Testing JIRA BOOKKEEPER-564 Patch 0001-BOOKKEEPER-564-Better-checkpoint-mechanism.patch downloaded at Tue Apr 9 16:41:31 UTC 2013 ---------------------------- +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 9 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: 833 +1 DISTRO . +1 distro tarball builds with the patch ---------------------------- +1 Overall result, good!, no -1s The full output of the test-patch run is available at . https://builds.apache.org/job/bookkeeper-trunk-precommit-build/324/
          Hide
          Ivan Kelly added a comment -

          The more I look at it, the less I like this Checkpointer interface. It's doing exactly the type of thing I was trying to get rid of with BOOKKEEPER-572 originally.

          this is also what I pointed in previous comment "the patch here already covered the most part you made the change in BOOKKEEPER-572.".

          You've confused me now . are you saying you don't like the checkpointer interface also?

          Show
          Ivan Kelly added a comment - The more I look at it, the less I like this Checkpointer interface. It's doing exactly the type of thing I was trying to get rid of with BOOKKEEPER-572 originally. this is also what I pointed in previous comment "the patch here already covered the most part you made the change in BOOKKEEPER-572 .". You've confused me now . are you saying you don't like the checkpointer interface also?
          Hide
          Sijie Guo added a comment -

          I don't like putting SyncThread in ledger storage itself, since for other ledger storage implementation, we had to same thing. The value of #checkpoint #flush interfaces and CheckPointer interface here is to bridge the journal with the ledger storage, it allows plugging different implementation for either journal or ledger storage, at least for skip list based ledger storage.

          The comment I raised is to say that BOOKKEEPER-572 could be resolved easily based on this jira, since it already did lots of things you tried to do in BOOKKEEPER-572.

          Show
          Sijie Guo added a comment - I don't like putting SyncThread in ledger storage itself, since for other ledger storage implementation, we had to same thing. The value of #checkpoint #flush interfaces and CheckPointer interface here is to bridge the journal with the ledger storage, it allows plugging different implementation for either journal or ledger storage, at least for skip list based ledger storage. The comment I raised is to say that BOOKKEEPER-572 could be resolved easily based on this jira, since it already did lots of things you tried to do in BOOKKEEPER-572 .
          Hide
          Ivan Kelly added a comment -

          The bridge between journal and ledger storage is the bookie, which uses both. Take a look at the last patch I posted. The sync thread as we know it has gone away completely. InterleavedLedgerStorage has an executor on which it runs checkpoints, and it also take care of storing the log mark. All the running, and flushing flag stuff that was in the SyncThread is taken care of by the executor.

          The log mark is a property of the ledger storage, as it defines how much of the history is contained within it. Therefore, it belongs there. Also, it is stored to the ledger dirs, which are now only written to by ledger storage.

          With the patch I attached, Journal and LedgerStorage are completely decoupled. The only thing they have in common is the LogMark objects, which I should be possible to make completely opaque to the LedgerStorage in future if needed.

          Show
          Ivan Kelly added a comment - The bridge between journal and ledger storage is the bookie, which uses both. Take a look at the last patch I posted. The sync thread as we know it has gone away completely. InterleavedLedgerStorage has an executor on which it runs checkpoints, and it also take care of storing the log mark. All the running, and flushing flag stuff that was in the SyncThread is taken care of by the executor. The log mark is a property of the ledger storage, as it defines how much of the history is contained within it. Therefore, it belongs there. Also, it is stored to the ledger dirs, which are now only written to by ledger storage. With the patch I attached, Journal and LedgerStorage are completely decoupled. The only thing they have in common is the LogMark objects, which I should be possible to make completely opaque to the LedgerStorage in future if needed.
          Hide
          Sijie Guo added a comment -

          you didn't answer the question for skiplist based ledger storage. this is my concern on your latest patch on previous comment.

          The only thing they have in common is the LogMark objects, which I should be possible to make completely opaque to the LedgerStorage in future if needed.

          don't you think the opaque thing is already addressed by CheckPoint interface?

          Show
          Sijie Guo added a comment - you didn't answer the question for skiplist based ledger storage. this is my concern on your latest patch on previous comment. The only thing they have in common is the LogMark objects, which I should be possible to make completely opaque to the LedgerStorage in future if needed. don't you think the opaque thing is already addressed by CheckPoint interface?
          Hide
          Ivan Kelly added a comment -

          you didn't answer the question for skiplist based ledger storage. this is my concern on your latest patch on previous comment.

          I assume you mean "it allows plugging different implementation for either journal or ledger storage, at least for skip list based ledger storage.". Moving the sync thread out of Bookie makes supporting different implementations much easier, as Journal and LedgerStorage know absolutely nothing about each other now, and the LedgerStorage has 100% control over when it runs a checkpoint.

          don't you think the opaque thing is already addressed by CheckPoint interface?

          opacity yes. I was referring to something like what Checkpoint interface does in that regard. However, it also has the #checkpointComplete() which I found unnecessary once the storage of LogMark was moved into the LedgerStorage.

          Show
          Ivan Kelly added a comment - you didn't answer the question for skiplist based ledger storage. this is my concern on your latest patch on previous comment. I assume you mean "it allows plugging different implementation for either journal or ledger storage, at least for skip list based ledger storage.". Moving the sync thread out of Bookie makes supporting different implementations much easier, as Journal and LedgerStorage know absolutely nothing about each other now, and the LedgerStorage has 100% control over when it runs a checkpoint. don't you think the opaque thing is already addressed by CheckPoint interface? opacity yes. I was referring to something like what Checkpoint interface does in that regard. However, it also has the #checkpointComplete() which I found unnecessary once the storage of LogMark was moved into the LedgerStorage.
          Hide
          Ivan Kelly added a comment -

          If you're concerned that not having checkpointer will make skiplist harder to integrate, see
          https://github.com/ivankelly/bookkeeper/commit/dfa5a47291ad228f4aa5fdd9029a1ebc1709d7bf

          It took about an hour to do the integration.

          Show
          Ivan Kelly added a comment - If you're concerned that not having checkpointer will make skiplist harder to integrate, see https://github.com/ivankelly/bookkeeper/commit/dfa5a47291ad228f4aa5fdd9029a1ebc1709d7bf It took about an hour to do the integration.
          Hide
          Sijie Guo added a comment -

          the integration is easy due to skiplist extends InterleavedLedgerStorage. but if another implementation of LedgerStorage is not based InterleavedLedgerStorage, it has to implement all the things in InterleavedLedgerStorage again.

          -1 for the part that you moved LogMark stuffs (like reading, write marks, which is part of Journal) into ledger storage, which make the journal constructor of "Journal(ServerConfiguration conf, LogMark lastLogMark)" looks like that if I passed any mark, the journal could replay starting from the passed mark. but it couldn't, since Journal gc its journal files. The mark is part of journal, which is used to gc its journal files. And Checkpoint#complete is the way to bridge ledger storage and journal, which tells journal that ledger storage already synced until this checkpoint and now it is ok for journal to move on and gc journal files.

          Show
          Sijie Guo added a comment - the integration is easy due to skiplist extends InterleavedLedgerStorage. but if another implementation of LedgerStorage is not based InterleavedLedgerStorage, it has to implement all the things in InterleavedLedgerStorage again. -1 for the part that you moved LogMark stuffs (like reading, write marks, which is part of Journal) into ledger storage, which make the journal constructor of "Journal(ServerConfiguration conf, LogMark lastLogMark)" looks like that if I passed any mark, the journal could replay starting from the passed mark. but it couldn't, since Journal gc its journal files. The mark is part of journal, which is used to gc its journal files. And Checkpoint#complete is the way to bridge ledger storage and journal, which tells journal that ledger storage already synced until this checkpoint and now it is ok for journal to move on and gc journal files.
          Hide
          Ivan Kelly added a comment -

          the integration is easy due to skiplist extends InterleavedLedgerStorage. but if another implementation of LedgerStorage is not based InterleavedLedgerStorage, it has to implement all the things in InterleavedLedgerStorage again.

          Which part needs to be reimplemented? storing the log mark? Perhaps the new ledger storage stores it in a different way. It's quite conceivable that it may store the log mark in the footer of a log file, rather than in an individual file for example.

          I can't think of anything else I've moved into ledger storage, which a new ledger storage wouldn't have to implement anyhow, like checkpointing and flushing, if it even considers them different things.

          "Journal(ServerConfiguration conf, LogMark lastLogMark)" looks like that if I passed any mark, the journal could replay starting from the passed mark. but it couldn't, since Journal gc its journal files.

          It couldn't have gc'd unless the ledger storage had indicated that that mark had been synced to. And if the ledger storage has indicated this, it shouldn't expect though journals still to be there.
          This doesn't change whether you're having bookie asked for the last sync mark and gc, or passing it through Checkpoint#complete. Whats more, with Checkpoint#complete, you're having the ledger storage driving actions on the journal, rather than the bookie driving the action. The bookie owns the journal, so it can tell it what to do. For the ledger storage, this is not the case

          And Checkpoint#complete is the way to bridge ledger storage and journal, which tells journal that ledger storage already synced until this checkpoint

          So the journal storage tells the journal that it has synced to the checkpoint. This means that the checkpoint is something that is inheritly a property of the ledger storage. The ledger storage + the mark/checkpoint tells you something about the state of the system. The journal + mark/checkpoint tells you nothing.

          Show
          Ivan Kelly added a comment - the integration is easy due to skiplist extends InterleavedLedgerStorage. but if another implementation of LedgerStorage is not based InterleavedLedgerStorage, it has to implement all the things in InterleavedLedgerStorage again. Which part needs to be reimplemented? storing the log mark? Perhaps the new ledger storage stores it in a different way. It's quite conceivable that it may store the log mark in the footer of a log file, rather than in an individual file for example. I can't think of anything else I've moved into ledger storage, which a new ledger storage wouldn't have to implement anyhow, like checkpointing and flushing, if it even considers them different things. "Journal(ServerConfiguration conf, LogMark lastLogMark)" looks like that if I passed any mark, the journal could replay starting from the passed mark. but it couldn't, since Journal gc its journal files. It couldn't have gc'd unless the ledger storage had indicated that that mark had been synced to. And if the ledger storage has indicated this, it shouldn't expect though journals still to be there. This doesn't change whether you're having bookie asked for the last sync mark and gc, or passing it through Checkpoint#complete. Whats more, with Checkpoint#complete, you're having the ledger storage driving actions on the journal, rather than the bookie driving the action. The bookie owns the journal, so it can tell it what to do. For the ledger storage, this is not the case And Checkpoint#complete is the way to bridge ledger storage and journal, which tells journal that ledger storage already synced until this checkpoint So the journal storage tells the journal that it has synced to the checkpoint. This means that the checkpoint is something that is inheritly a property of the ledger storage. The ledger storage + the mark/checkpoint tells you something about the state of the system. The journal + mark/checkpoint tells you nothing.
          Hide
          Ivan Kelly added a comment -

          But actually, to clarify because I think this has got off on a tangent, what I dislike about Checkpointer/CheckpointProgress/Checkpoint is the way that ledger storage is controlling the journal. This appears to me as coupling, and coupling is bad.

          Show
          Ivan Kelly added a comment - But actually, to clarify because I think this has got off on a tangent, what I dislike about Checkpointer/CheckpointProgress/Checkpoint is the way that ledger storage is controlling the journal. This appears to me as coupling, and coupling is bad.
          Hide
          Sijie Guo added a comment -

          I don't see how coupling as you said. a consistent state of ledger storage, mark/checkpoint and journal file state (gc files ) are all controlled by bookie (SyncThread):

          1) SyncThread ask ledger storage to persist all the data added before a mark/checkpoint by calling LedgerStorage#checkpoint
          2) after LedgerStorage finished persisting all the data, SyncThread tells journal that it is safe to gc all the journal entries before that mark/checkpoint.

          the ledger storage does nothing on this above flow, which just tell SyncThread: "hey, I am dirty enough and it might be the time to do a checkpoint, which reduces recovery time facing failure".

          Show
          Sijie Guo added a comment - I don't see how coupling as you said. a consistent state of ledger storage, mark/checkpoint and journal file state (gc files ) are all controlled by bookie (SyncThread): 1) SyncThread ask ledger storage to persist all the data added before a mark/checkpoint by calling LedgerStorage#checkpoint 2) after LedgerStorage finished persisting all the data, SyncThread tells journal that it is safe to gc all the journal entries before that mark/checkpoint. the ledger storage does nothing on this above flow, which just tell SyncThread: "hey, I am dirty enough and it might be the time to do a checkpoint, which reduces recovery time facing failure".
          Hide
          Ivan Kelly added a comment -

          1) SyncThread ask ledger storage to persist all the data added before a mark/checkpoint by calling LedgerStorage#checkpoint

          If this was how it happened it wouldn't be so bad. But what actually happens is that LedgerStorage decides it wants a checkpoint, and makes a call that controls the SyncThread (#startCheckpoint). Eventually this also calls #checkpointComplete. This is control coupling. It makes LedgerStorage dependent on the behaviour of SyncThread and makes it impossible to reason about or test the LedgerStorage in isolation. While this has always been partially the case, previously SyncThread did control the checkpointing completely, so the flow was simpler. Now the ledger storage is controlling the checkpointing by means of the sync thread.

          And then, since the journal is controlled by the sync thread in this design, by inheritance, the journal is controlled by the Ledger storage. I'd actually prefer a design where the sync thread and the journal were owned by the ledger storage. At least then the control flow would match the ownership, and the coupling would be explicit rather than the implicit coupling we have here.

          the ledger storage does nothing on this above flow, which just tell SyncThread: "hey, I am dirty enough and it might be the time to do a checkpoint, which reduces recovery time facing failure".

          Again, this wouldn't be so bad. But its not the case. What actually happens is that the ledger storage is saying "hey, checkpoint to this point now!"

          Show
          Ivan Kelly added a comment - 1) SyncThread ask ledger storage to persist all the data added before a mark/checkpoint by calling LedgerStorage#checkpoint If this was how it happened it wouldn't be so bad. But what actually happens is that LedgerStorage decides it wants a checkpoint, and makes a call that controls the SyncThread (#startCheckpoint). Eventually this also calls #checkpointComplete. This is control coupling. It makes LedgerStorage dependent on the behaviour of SyncThread and makes it impossible to reason about or test the LedgerStorage in isolation. While this has always been partially the case, previously SyncThread did control the checkpointing completely, so the flow was simpler. Now the ledger storage is controlling the checkpointing by means of the sync thread. And then, since the journal is controlled by the sync thread in this design, by inheritance, the journal is controlled by the Ledger storage. I'd actually prefer a design where the sync thread and the journal were owned by the ledger storage. At least then the control flow would match the ownership, and the coupling would be explicit rather than the implicit coupling we have here. the ledger storage does nothing on this above flow, which just tell SyncThread: "hey, I am dirty enough and it might be the time to do a checkpoint, which reduces recovery time facing failure". Again, this wouldn't be so bad. But its not the case. What actually happens is that the ledger storage is saying "hey, checkpoint to this point now!"
          Hide
          Sijie Guo added a comment -

          This is control coupling. It makes LedgerStorage dependent on the behaviour of SyncThread and makes it impossible to reason about or test the LedgerStorage in isolation.

          How it worked in previous flow on testing LedgerStorage in isolation? I don't quite understand about this part.

          What actually happens is that the ledger storage is saying "hey, checkpoint to this point now!"

          in previous flow, sync thread just checkpoint blindly. in current flow, ledger storage would tell the syncthread (the checkpointer) the point which is a better point to do checkpoint. the only thing that ledger storage gave out is the point to guide checkpoint. but if you mean the guide is coupling, I have nothing more to say.

          Show
          Sijie Guo added a comment - This is control coupling. It makes LedgerStorage dependent on the behaviour of SyncThread and makes it impossible to reason about or test the LedgerStorage in isolation. How it worked in previous flow on testing LedgerStorage in isolation? I don't quite understand about this part. What actually happens is that the ledger storage is saying "hey, checkpoint to this point now!" in previous flow, sync thread just checkpoint blindly. in current flow, ledger storage would tell the syncthread (the checkpointer) the point which is a better point to do checkpoint. the only thing that ledger storage gave out is the point to guide checkpoint. but if you mean the guide is coupling, I have nothing more to say.
          Hide
          Ivan Kelly added a comment -

          How it worked in previous flow on testing LedgerStorage in isolation? I don't quite understand about this part.

          We don't. We always construct a full bookie, because it's impossible to test the ledger storage without the bookie.

          grep -r "new InterleavedLedgerStorage" bookkeeper-server/src/test/java
          

          Another place I came across this was with the bkvhbase benchmark. I had to implement my own SyncThread.


          in previous flow, sync thread just checkpoint blindly. in current flow, ledger storage would tell the syncthread (the checkpointer) the point which is a better point to do checkpoint. the only thing that ledger storage gave out is the point to guide checkpoint. but if you mean the guide is coupling, I have nothing more to say.

          It's a command rather than a guide. And how the ledger storage behaves is dependent on the sync thread. This is coupling.

          Show
          Ivan Kelly added a comment - How it worked in previous flow on testing LedgerStorage in isolation? I don't quite understand about this part. We don't. We always construct a full bookie, because it's impossible to test the ledger storage without the bookie. grep -r " new InterleavedLedgerStorage" bookkeeper-server/src/test/java Another place I came across this was with the bkvhbase benchmark. I had to implement my own SyncThread. in previous flow, sync thread just checkpoint blindly. in current flow, ledger storage would tell the syncthread (the checkpointer) the point which is a better point to do checkpoint. the only thing that ledger storage gave out is the point to guide checkpoint. but if you mean the guide is coupling, I have nothing more to say. It's a command rather than a guide. And how the ledger storage behaves is dependent on the sync thread. This is coupling.
          Hide
          Sijie Guo added a comment -

          We don't. We always construct a full bookie, because it's impossible to test the ledger storage without the bookie.
          Another place I came across this was with the bkvhbase benchmark. I had to implement my own SyncThread.

          you could use ledger storage add/read/flush independently. it is a full-functioned module. You could use it as an independent module in other place if you like for different purpose. I don't understand how is bad as you said.

          It's a command rather than a guide. And how the ledger storage behaves is dependent on the sync thread. This is coupling.

          if its behavior is part to a bookie's behavior, I don't think it is coupling. so I don't know what behaviors that is just belonged to ledger storage and not belonged to bookie.

          again, why I think it is a guide rather than a command. in checkpointer interface, ledger storage just tell a better pointer to the implementation. the really execution is decided by checkpointer itself (bookie), it could use the guide offered by ledger storage, or could use a different checkpoint based on other condition (for example, CheckPoint.MAX), so the control part is up to checkpointer.

          Show
          Sijie Guo added a comment - We don't. We always construct a full bookie, because it's impossible to test the ledger storage without the bookie. Another place I came across this was with the bkvhbase benchmark. I had to implement my own SyncThread. you could use ledger storage add/read/flush independently. it is a full-functioned module. You could use it as an independent module in other place if you like for different purpose. I don't understand how is bad as you said. It's a command rather than a guide. And how the ledger storage behaves is dependent on the sync thread. This is coupling. if its behavior is part to a bookie's behavior, I don't think it is coupling. so I don't know what behaviors that is just belonged to ledger storage and not belonged to bookie. again, why I think it is a guide rather than a command. in checkpointer interface, ledger storage just tell a better pointer to the implementation. the really execution is decided by checkpointer itself (bookie), it could use the guide offered by ledger storage, or could use a different checkpoint based on other condition (for example, CheckPoint.MAX), so the control part is up to checkpointer.
          Hide
          Sijie Guo added a comment -

          one more point that using the CheckPointer interface, you could still implement a Periodical sync thread, if you don't like using the checkpoint offered by ledger storage. but it would allow us using the optimization.

          Show
          Sijie Guo added a comment - one more point that using the CheckPointer interface, you could still implement a Periodical sync thread, if you don't like using the checkpoint offered by ledger storage. but it would allow us using the optimization.
          Hide
          Ivan Kelly added a comment -

          We don't. We always construct a full bookie, because it's impossible to test the ledger storage without the bookie. Another place I came across this was with the bkvhbase benchmark. I had to implement my own SyncThread.

          you could use ledger storage add/read/flush independently. it is a full-functioned module. You could use it as an independent module in other place if you like for different purpose. I don't understand how is bad as you said.

          Its not a complete module. We're going to be introducing a new ledger storage to trunk hopefully soon. This is going to need to be tested and benchmarked extensively. If the behaviour of the ledger storage is dependendent on the behaviour of the sync thread, which it is, then it's going to make this job much more awkward. We will have to reimplement the sync thread. But of course, the sync thread will have to match the behaviour of the existing sync thread. We can't use the existing sync thread, because it's coupled with the journal and we can't benchmark using the entire bookie, because the journal throughput will interfere with the throughput of the ledger storage (i.e. it will throttle it as the journal should be the bottleneck). This is the core reason why I want to avoid this coupling.

          It's a command rather than a guide. And how the ledger storage behaves is dependent on the sync thread. This is coupling.

          if its behavior is part to a bookie's behavior, I don't think it is coupling. so I don't know what behaviors that is just belonged to ledger storage and not belonged to bookie.

          again, why I think it is a guide rather than a command. in checkpointer interface, ledger storage just tell a better pointer to the implementation. the really execution is decided by checkpointer itself (bookie), it could use the guide offered by ledger storage, or could use a different checkpoint based on other condition (for example, CheckPoint.MAX), so the control part is up to checkpointer.

          With checkpointer, an #onRotateEntryLog event in InterleavedLedgerStorage triggers a call to #startCheckpoint which pushes a sync request onto the request queue which causes a #checkpoint() to be called in the InterleavedLedgerStorage. This isn't a guide. A guide would be something that whoever triggering the checkpoint would request and based on that decide whether to checkpoint or not. LedgerStorage#isFlushRequired() was a guide. But checkpointer is not. Checkpointer uses a push mechanism, a guide would use a pull mechanism.

          But there's a deeper issue here. The decision to checkpoint is taken inside of the ledger storage. This suggests that the actual action to checkpoint should take place there too.

          one more point that using the CheckPointer interface, you could still implement a Periodical sync thread, if you don't like using the checkpoint offered by ledger storage. but it would allow us using the optimization.

          This doesn't require the checkpoint interface. It can be achieved by simply exposing #flush(). I don't think we should do this until there's a strong requirement for it though.

          Show
          Ivan Kelly added a comment - We don't. We always construct a full bookie, because it's impossible to test the ledger storage without the bookie. Another place I came across this was with the bkvhbase benchmark. I had to implement my own SyncThread. you could use ledger storage add/read/flush independently. it is a full-functioned module. You could use it as an independent module in other place if you like for different purpose. I don't understand how is bad as you said. Its not a complete module. We're going to be introducing a new ledger storage to trunk hopefully soon. This is going to need to be tested and benchmarked extensively. If the behaviour of the ledger storage is dependendent on the behaviour of the sync thread, which it is, then it's going to make this job much more awkward. We will have to reimplement the sync thread. But of course, the sync thread will have to match the behaviour of the existing sync thread. We can't use the existing sync thread, because it's coupled with the journal and we can't benchmark using the entire bookie, because the journal throughput will interfere with the throughput of the ledger storage (i.e. it will throttle it as the journal should be the bottleneck). This is the core reason why I want to avoid this coupling . It's a command rather than a guide. And how the ledger storage behaves is dependent on the sync thread. This is coupling. if its behavior is part to a bookie's behavior, I don't think it is coupling. so I don't know what behaviors that is just belonged to ledger storage and not belonged to bookie. again, why I think it is a guide rather than a command. in checkpointer interface, ledger storage just tell a better pointer to the implementation. the really execution is decided by checkpointer itself (bookie), it could use the guide offered by ledger storage, or could use a different checkpoint based on other condition (for example, CheckPoint.MAX), so the control part is up to checkpointer. With checkpointer, an #onRotateEntryLog event in InterleavedLedgerStorage triggers a call to #startCheckpoint which pushes a sync request onto the request queue which causes a #checkpoint() to be called in the InterleavedLedgerStorage. This isn't a guide. A guide would be something that whoever triggering the checkpoint would request and based on that decide whether to checkpoint or not. LedgerStorage#isFlushRequired() was a guide. But checkpointer is not. Checkpointer uses a push mechanism, a guide would use a pull mechanism. But there's a deeper issue here. The decision to checkpoint is taken inside of the ledger storage. This suggests that the actual action to checkpoint should take place there too. one more point that using the CheckPointer interface, you could still implement a Periodical sync thread, if you don't like using the checkpoint offered by ledger storage. but it would allow us using the optimization. This doesn't require the checkpoint interface. It can be achieved by simply exposing #flush(). I don't think we should do this until there's a strong requirement for it though.
          Hide
          Ivan Kelly added a comment -

          Also, the fact that SyncThread needs to be changed to support the new ledger storage indicates that there is a strong coupling between the two.

          Show
          Ivan Kelly added a comment - Also, the fact that SyncThread needs to be changed to support the new ledger storage indicates that there is a strong coupling between the two.
          Hide
          Sijie Guo added a comment -

          We're going to be introducing a new ledger storage to trunk hopefully soon. This is going to need to be tested and benchmarked extensively. If the behaviour of the ledger storage is dependendent on the behaviour of the sync thread, which it is, then it's going to make this job much more awkward. We will have to reimplement the sync thread.

          I don't see how we need to reimplement the sync thread for the new ledger storage. we used the same sync thread for both interleaved and skiplist ledger storage.

          This doesn't require the checkpoint interface. It can be achieved by simply exposing #flush(). I don't think we should do this until there's a strong requirement for it though.

          The ticket created here is for such requirement (if you read the description of this ticket), since we found that we could not get a sustain high throughput on heavy workload due to blind syncing (in a long run on heavy workload, we observed zero throughput at some points, which is totally bad for latency sensitive case. because the ledger storage is hanging on syncing too much data (old data which need to be synced and newly added data) due to blind syncing).

          As I said in last comment, you could use the checkpointer interface to do periodical sync as before or based on the calculation of number of bytes unflushed (if you don't like the way let ledger storage offering the hint). but this interface allowed us and it already helped us to use such hint to optimize the bookie to get a sustained higher throughput.

          Show
          Sijie Guo added a comment - We're going to be introducing a new ledger storage to trunk hopefully soon. This is going to need to be tested and benchmarked extensively. If the behaviour of the ledger storage is dependendent on the behaviour of the sync thread, which it is, then it's going to make this job much more awkward. We will have to reimplement the sync thread. I don't see how we need to reimplement the sync thread for the new ledger storage. we used the same sync thread for both interleaved and skiplist ledger storage. This doesn't require the checkpoint interface. It can be achieved by simply exposing #flush(). I don't think we should do this until there's a strong requirement for it though. The ticket created here is for such requirement (if you read the description of this ticket), since we found that we could not get a sustain high throughput on heavy workload due to blind syncing (in a long run on heavy workload, we observed zero throughput at some points, which is totally bad for latency sensitive case. because the ledger storage is hanging on syncing too much data (old data which need to be synced and newly added data) due to blind syncing). As I said in last comment, you could use the checkpointer interface to do periodical sync as before or based on the calculation of number of bytes unflushed (if you don't like the way let ledger storage offering the hint). but this interface allowed us and it already helped us to use such hint to optimize the bookie to get a sustained higher throughput.
          Hide
          Ivan Kelly added a comment -

          I don't see how we need to reimplement the sync thread for the new ledger storage. we used the same sync thread for both interleaved and skiplist ledger storage.

          We would need to reimplement it in any benchmark/test as the logic for flushing is distributed across the two places. And any reimplementation would have to behave exactly like the original SyncThread for the bench to be valid. And this will drift over time.


          As I said in last comment, you could use the checkpointer interface to do periodical sync as before or based on the calculation of number of bytes unflushed (if you don't like the way let ledger storage offering the hint). but this interface allowed us and it already helped us to use such hint to optimize the bookie to get a sustained higher throughput.

          My point was that there's nothing inherent in the checkpointer interface which makes this possible. Syncing periodically can already be done calling #flush, by size can be done by comparing marks. Adding the checkpointer interface doesn't make any of this easier to do.

          Show
          Ivan Kelly added a comment - I don't see how we need to reimplement the sync thread for the new ledger storage. we used the same sync thread for both interleaved and skiplist ledger storage. We would need to reimplement it in any benchmark/test as the logic for flushing is distributed across the two places. And any reimplementation would have to behave exactly like the original SyncThread for the bench to be valid. And this will drift over time. As I said in last comment, you could use the checkpointer interface to do periodical sync as before or based on the calculation of number of bytes unflushed (if you don't like the way let ledger storage offering the hint). but this interface allowed us and it already helped us to use such hint to optimize the bookie to get a sustained higher throughput. My point was that there's nothing inherent in the checkpointer interface which makes this possible. Syncing periodically can already be done calling #flush, by size can be done by comparing marks. Adding the checkpointer interface doesn't make any of this easier to do.
          Hide
          Sijie Guo added a comment -

          My point was that there's nothing inherent in the checkpointer interface which makes this possible. Syncing periodically can already be done calling #flush, by size can be done by comparing marks. Adding the checkpointer interface doesn't make any of this easier to do.

          I don't want to argument on this interface, since it is just an interface. if you could do the optimization without ledger storage offering the hint, show the code. And my point is still that ledger storage and journal are just bookie's components, the coordinate thing and semantic guarantee is better to be done in bookie itself.

          Show
          Sijie Guo added a comment - My point was that there's nothing inherent in the checkpointer interface which makes this possible. Syncing periodically can already be done calling #flush, by size can be done by comparing marks. Adding the checkpointer interface doesn't make any of this easier to do. I don't want to argument on this interface, since it is just an interface. if you could do the optimization without ledger storage offering the hint, show the code. And my point is still that ledger storage and journal are just bookie's components, the coordinate thing and semantic guarantee is better to be done in bookie itself.
          Hide
          Ivan Kelly added a comment -

          if you could do the optimization without ledger storage offering the hint, show the code.

          I'm not sure which exact optimization you're referring to. My patch from the 9th of April has all the things I've discussed except for periodic syncs, which, as I understand it, we don't want anymore. To summarize what is in the patch.

          • LedgerStorage decides when to checkpoint, and runs it autonomously
          • The LogMark storage is managed by LedgerStorage
          • The bookie coordinates LogMark usage between the Journal and LedgerStorage.
          • Both components can now be tested 100% independently. Journal doesn't even need the ledger storage dirs anymore.

          ledger storage and journal are just bookie's components, the coordinate thing and semantic guarantee is better to be done in bookie itself.

          And my patch conveys this. The bookie owns the journal and the ledger storage. And controls both. The journal doesn't control the ledger storage. The ledger storage doesn't control the journal. The ledger storage doesn't control the bookie. The journal doesn't control the bookie. The control flow follows the ownership. Bookie controls ledger storage and the journal, and that's it.

          Show
          Ivan Kelly added a comment - if you could do the optimization without ledger storage offering the hint, show the code. I'm not sure which exact optimization you're referring to. My patch from the 9th of April has all the things I've discussed except for periodic syncs, which, as I understand it, we don't want anymore. To summarize what is in the patch. LedgerStorage decides when to checkpoint, and runs it autonomously The LogMark storage is managed by LedgerStorage The bookie coordinates LogMark usage between the Journal and LedgerStorage. Both components can now be tested 100% independently. Journal doesn't even need the ledger storage dirs anymore. ledger storage and journal are just bookie's components, the coordinate thing and semantic guarantee is better to be done in bookie itself. And my patch conveys this. The bookie owns the journal and the ledger storage. And controls both. The journal doesn't control the ledger storage. The ledger storage doesn't control the journal. The ledger storage doesn't control the bookie. The journal doesn't control the bookie. The control flow follows the ownership. Bookie controls ledger storage and the journal, and that's it.
          Hide
          Rakesh R added a comment -

          Thanks Ivan and Sijie for the great effort and your time. Sorry for pitching late, I'm busy with our internal releases and other schedules

          Its very nice idea to avoid frequent checkpointing, which affects performance.

          Hi Ivan Kelly, patch looks good and I'm just having one thought.

          •LedgerStorage decides when to checkpoint, and runs it autonomously

          If I understand your approach, LedgerStorage owns the checkpoint thread. #onRotateEntryLog it will run the checkpointing and updates the lastSyncedMark.

          There is no problem with this approach. In this case, when a new LedgerStorage implementation comes in, it should again re-define the checkpointing algo. IMHO, instead of this can we think of an approach where we can decouple the checkpointing algo from Interleaved storage. Bookie can own this checkpointing logic and let him control. With this approach Bookie will have more control over the checkpointing irrespective of the plugged-in ledger storage. How does it sound?. Sijie Guo, are you also thinking in similar way?

          Thanks,
          Rakesh

          Show
          Rakesh R added a comment - Thanks Ivan and Sijie for the great effort and your time. Sorry for pitching late, I'm busy with our internal releases and other schedules Its very nice idea to avoid frequent checkpointing, which affects performance. Hi Ivan Kelly , patch looks good and I'm just having one thought. •LedgerStorage decides when to checkpoint, and runs it autonomously If I understand your approach, LedgerStorage owns the checkpoint thread. #onRotateEntryLog it will run the checkpointing and updates the lastSyncedMark. There is no problem with this approach. In this case, when a new LedgerStorage implementation comes in, it should again re-define the checkpointing algo. IMHO, instead of this can we think of an approach where we can decouple the checkpointing algo from Interleaved storage. Bookie can own this checkpointing logic and let him control. With this approach Bookie will have more control over the checkpointing irrespective of the plugged-in ledger storage. How does it sound?. Sijie Guo , are you also thinking in similar way? Thanks, Rakesh
          Hide
          Sijie Guo added a comment - - edited

          Ivan Kelly

          I already commented on your patch in previous comment.

          1) you moved LogMark to ledger storage, which makes journal contructor "Journal(conf, logmark)" behavior unclear, which looks like whatever the caller that give a mark, the journal would replay starting from the given mark. but it couldn't, since journal gc its files. It is against what you said Journal is a independent component.

          2) you are making ledger storage too ambitious. sync thread (checkpointing) logic should be maintained by Bookie itself, as the sync(checkpointing) logic is part of bookie not ledger storage, it should be common across different ledger storage implementations. And, Rakesh R yes, I agreed with you.

          > The LogMark storage is managed by LedgerStorage
          > Journal doesn't even need the ledger storage dirs anymore.

          as I pointed at 1), making LogMark as a part to journal would make Journal clearer on the replaying behavior. Although we used ledger directories in Journal, it doesn't mean that LogMark is part of ledger storage. The directories recording LogMark could be any directory. the only thing why not putting the journal directory is not to affect the performance of journal.

          Show
          Sijie Guo added a comment - - edited Ivan Kelly I already commented on your patch in previous comment. 1) you moved LogMark to ledger storage, which makes journal contructor "Journal(conf, logmark)" behavior unclear, which looks like whatever the caller that give a mark, the journal would replay starting from the given mark. but it couldn't, since journal gc its files. It is against what you said Journal is a independent component. 2) you are making ledger storage too ambitious. sync thread (checkpointing) logic should be maintained by Bookie itself, as the sync(checkpointing) logic is part of bookie not ledger storage, it should be common across different ledger storage implementations. And, Rakesh R yes, I agreed with you. > The LogMark storage is managed by LedgerStorage > Journal doesn't even need the ledger storage dirs anymore. as I pointed at 1), making LogMark as a part to journal would make Journal clearer on the replaying behavior. Although we used ledger directories in Journal, it doesn't mean that LogMark is part of ledger storage. The directories recording LogMark could be any directory. the only thing why not putting the journal directory is not to affect the performance of journal.
          Hide
          Ivan Kelly added a comment - - edited


          In this case, when a new LedgerStorage implementation comes in, it should again re-define the checkpointing algo. IMHO, instead of this can we think of an approach where we can decouple the checkpointing algo from Interleaved storage. Bookie can own this checkpointing logic and let him control. With this approach Bookie will have more control over the checkpointing irrespective of the plugged-in ledger storage. How does it sound?. Sijie Guo, are you also thinking in similar way?

          If we want the LedgerStorage to control when checkpointing should occur, then LedgerStorage has to run the checkpoint itself. Otherwise you have coupled the LedgerStorage to the Bookie.SyncThread. There's no problem with breaking the sync thread out into a separate class, which multiple LedgerStorage implementations can use, but it should be owned by the LedgerStorage


          1) you moved LogMark to ledger storage, which makes journal contructor "Journal(conf, logmark)" behavior unclear,

          This should be better. The journal should just be constructed with Journal(conf). LastSyncedLogMark should only come into play for Journal#replay(JournalScanner) which should become Journal#replay(LogMark from, JournalScanner).


          sync thread (checkpointing) logic should be maintained by Bookie itself

          I strongly disagree with this because...

          as the sync(checkpointing) logic is part of bookie not ledger storage

          ...all the logic to do the checkpoint is in the LedgerStorage. The decision to make the checkpoint is taken from within the ledger storage. So this is false. The logic is part of ledger storage.


          it should be common across different ledger storage implementations.

          It can be broken out into a different class which can be shared by different implementations. It should be owned by the ledger storage though.

          1), making LogMark as a part to journal would make Journal clearer on the replaying behaviour.

          The log mark is dependent on the ledger storage and only means anything in the context of the ledger storage. It should only be stored when a checkpoint has occurred. This means that the ledger storage is what decides which log mark to store. If the journal is storing the mark, the ledger storage is triggering behaviour on the journal. Again, this is another piece that could be broken out into a separate class to be used by multiple ledger storage implementations, but it should remain owned by the ledger storage.

          To reiterate, this changes need to be done to make it possible to benchmark the ledger storage in a way that the ledger storage will behave the same as it does when running under a bookie.

          Show
          Ivan Kelly added a comment - - edited In this case, when a new LedgerStorage implementation comes in, it should again re-define the checkpointing algo. IMHO, instead of this can we think of an approach where we can decouple the checkpointing algo from Interleaved storage. Bookie can own this checkpointing logic and let him control. With this approach Bookie will have more control over the checkpointing irrespective of the plugged-in ledger storage. How does it sound?. Sijie Guo, are you also thinking in similar way? If we want the LedgerStorage to control when checkpointing should occur, then LedgerStorage has to run the checkpoint itself. Otherwise you have coupled the LedgerStorage to the Bookie.SyncThread. There's no problem with breaking the sync thread out into a separate class, which multiple LedgerStorage implementations can use, but it should be owned by the LedgerStorage 1) you moved LogMark to ledger storage, which makes journal contructor "Journal(conf, logmark)" behavior unclear, This should be better. The journal should just be constructed with Journal(conf). LastSyncedLogMark should only come into play for Journal#replay(JournalScanner) which should become Journal#replay(LogMark from, JournalScanner). sync thread (checkpointing) logic should be maintained by Bookie itself I strongly disagree with this because... as the sync(checkpointing) logic is part of bookie not ledger storage ...all the logic to do the checkpoint is in the LedgerStorage. The decision to make the checkpoint is taken from within the ledger storage. So this is false. The logic is part of ledger storage. it should be common across different ledger storage implementations. It can be broken out into a different class which can be shared by different implementations. It should be owned by the ledger storage though. 1), making LogMark as a part to journal would make Journal clearer on the replaying behaviour. The log mark is dependent on the ledger storage and only means anything in the context of the ledger storage. It should only be stored when a checkpoint has occurred. This means that the ledger storage is what decides which log mark to store. If the journal is storing the mark, the ledger storage is triggering behaviour on the journal. Again, this is another piece that could be broken out into a separate class to be used by multiple ledger storage implementations, but it should remain owned by the ledger storage. To reiterate, this changes need to be done to make it possible to benchmark the ledger storage in a way that the ledger storage will behave the same as it does when running under a bookie.
          Hide
          Ivan Kelly added a comment -

          I was thinking about this some more last night. There's no way of having the checkpointer outside of LedgerStorage without having a circular dependency. To demonstrate, if I wanted to implement a checkpointer, I would have to do something like the following.

          class MyCheckpointer extends CheckPointer {
              public CheckPoint newCheckpoint() {
                  return generateCheckpointSomehow();
              }
          
              public void startCheckpoint(CheckPoint checkpoint) {
                  ledgerStorage.checkpoint(checkpoint);
              }
          }
          
          LedgerStorage ledgerStorage = new InterleavedLedgerStorage(conf, dirs, new MyCheckpointer());
          

          There's a circular dependency here and there's no way to get rid of it. This circular dependency also exists in the Sijie's patch from 9 Apr, though it's hidden behind inner classes. This is just bad software design, and I'm going to -1 any patch that I find with such antipatterns.

          Whats more, any solution that has the syncthread and ledgerstorage as side by side objects rather than ledgerstorage owning the syncthread will run into this.

          However, we've been arguing this for weeks now, and we need to find some middle ground. I realise my patch isn't going to go in as is either, due to the points made earlier. Specifically, referring to Sijie's comments on the 23rd, for 1) this can go either way, I prefer LogMark in ledger storage, or at least out of Journal, but I can live with it. However, for 2) sync thread has to be owned by ledger storage, otherwise we get the antipattern described at the start of the comment.

          So how about this...

          We morph Checkpointer into the following interface

          interface CheckpointListener {
              public CheckPoint newCheckpoint();
              public void checkpointOccurred(Checkpoint checkpoint);
          }
          

          Journal implements CheckpointListener. LedgerStorage takes a Checkpoint listener on construction. SyncThread is broken out from Bookie, and LedgerStorage constructs and owns it, but the implementation is separate from any particular LedgerStorage implementation.

          I won't start coding this until I get some agreement that this is an acceptable way forward.

          Show
          Ivan Kelly added a comment - I was thinking about this some more last night. There's no way of having the checkpointer outside of LedgerStorage without having a circular dependency. To demonstrate, if I wanted to implement a checkpointer, I would have to do something like the following. class MyCheckpointer extends CheckPointer { public CheckPoint newCheckpoint() { return generateCheckpointSomehow(); } public void startCheckpoint(CheckPoint checkpoint) { ledgerStorage.checkpoint(checkpoint); } } LedgerStorage ledgerStorage = new InterleavedLedgerStorage(conf, dirs, new MyCheckpointer()); There's a circular dependency here and there's no way to get rid of it. This circular dependency also exists in the Sijie's patch from 9 Apr, though it's hidden behind inner classes. This is just bad software design, and I'm going to -1 any patch that I find with such antipatterns. Whats more, any solution that has the syncthread and ledgerstorage as side by side objects rather than ledgerstorage owning the syncthread will run into this. However, we've been arguing this for weeks now, and we need to find some middle ground. I realise my patch isn't going to go in as is either, due to the points made earlier. Specifically, referring to Sijie's comments on the 23rd, for 1) this can go either way, I prefer LogMark in ledger storage, or at least out of Journal, but I can live with it. However, for 2) sync thread has to be owned by ledger storage, otherwise we get the antipattern described at the start of the comment. So how about this... We morph Checkpointer into the following interface interface CheckpointListener { public CheckPoint newCheckpoint(); public void checkpointOccurred(Checkpoint checkpoint); } Journal implements CheckpointListener. LedgerStorage takes a Checkpoint listener on construction. SyncThread is broken out from Bookie, and LedgerStorage constructs and owns it, but the implementation is separate from any particular LedgerStorage implementation. I won't start coding this until I get some agreement that this is an acceptable way forward.
          Hide
          Rakesh R added a comment -

          Ivan Kelly, thanks for pointing out the cyclic dependency. Also, I agree to keep the checkpointing inside the LedgerStorage, as he is owning the entrylogger.

          Journal implements CheckpointListener

          Are you thinking like, EntryLogListener#onRotateEntryLog will trigger the checkpointing and after the successful completion will notify the Journal about the flushedLogMark, which inturn moves the "lastmark" pointer.

          Will have the following interfaces to generalize the checkpointing algo.

          InterLeavedStorage implements EntryLogListener#onRotateEntryLog() to do the checkpointing.
          Journal implements CheckpointListener#checkpointOccurred(CheckPoint) which moves the "lastmark" pointer.

          Am I missing anything?

          Also, it would be good to move EntryLogListener outside, presently its with EntryLogger.

          Show
          Rakesh R added a comment - Ivan Kelly , thanks for pointing out the cyclic dependency. Also, I agree to keep the checkpointing inside the LedgerStorage, as he is owning the entrylogger. Journal implements CheckpointListener Are you thinking like, EntryLogListener#onRotateEntryLog will trigger the checkpointing and after the successful completion will notify the Journal about the flushedLogMark, which inturn moves the "lastmark" pointer. Will have the following interfaces to generalize the checkpointing algo. InterLeavedStorage implements EntryLogListener#onRotateEntryLog() to do the checkpointing. Journal implements CheckpointListener#checkpointOccurred(CheckPoint) which moves the "lastmark" pointer. Am I missing anything? Also, it would be good to move EntryLogListener outside, presently its with EntryLogger.
          Hide
          Ivan Kelly added a comment -

          InterLeavedStorage implements EntryLogListener#onRotateEntryLog() to do the checkpointing.
          Journal implements CheckpointListener#checkpointOccurred(CheckPoint) which moves the "lastmark" pointer.

          Am I missing anything?

          This is it exactly.

          Show
          Ivan Kelly added a comment - InterLeavedStorage implements EntryLogListener#onRotateEntryLog() to do the checkpointing. Journal implements CheckpointListener#checkpointOccurred(CheckPoint) which moves the "lastmark" pointer. Am I missing anything? This is it exactly.
          Hide
          Rakesh R added a comment -

          I feel its good idea and +1 for this approach.

          Show
          Rakesh R added a comment - I feel its good idea and +1 for this approach.
          Hide
          Sijie Guo added a comment -

          a new patch to remove circular dependency by make the ledger storage interface as CheckPoint LedgerStorage#checkpoint(CheckPoint). The return value indicates the actual point that the ledger storage already synced all the data. keep the original sync thread.

          the benefit:

          1) we don't need to rewrite sync thread, if we provide a new ledger storage.
          2) syncing when entry logger rotated is a tradeoff between recovery time and performance. if in future, we want to reduce recovery time and keep performance, it is easy to do it.

          Show
          Sijie Guo added a comment - a new patch to remove circular dependency by make the ledger storage interface as CheckPoint LedgerStorage#checkpoint(CheckPoint). The return value indicates the actual point that the ledger storage already synced all the data. keep the original sync thread. the benefit: 1) we don't need to rewrite sync thread, if we provide a new ledger storage. 2) syncing when entry logger rotated is a tradeoff between recovery time and performance. if in future, we want to reduce recovery time and keep performance, it is easy to do it.
          Hide
          Sijie Guo added a comment - - edited

          Journal implements CheckpointListener. LedgerStorage takes a Checkpoint listener on construction. SyncThread is broken out from Bookie, and LedgerStorage constructs and owns it, but the implementation is separate from any particular LedgerStorage implementation.

          Journal implements CheckpointListener#checkpointOccurred(CheckPoint) which moves the "lastmark" pointer.

          it doesn't make any sense to make ledger storage control journal to move the mark. the checkpoint flow is not simple a responsibility of a single component. A bookie's data has two parts, one is persisted indexed data, the other is in-memory indexed data + persisted txns. And a checkpoint is the boundary of these two parts. The SyncThread is the role to change the data from one state (in-memory + txns) to the other state (permanent indexable): 1) ledger storage persist its in-memory data, 2) journal move its pointer and gc journals if necessary, 3) if any is failed due to disk issue, it means that such kind of data conversion could not proceed any more, the bookie should be turned to readonly at this moment (otherwise, it would be OOM). when you moved sync thread to ledger storage, you are moving the 2) and 3) part implicitly to ledger storage, which make ledger storage too ambitious and the flow unclear.

          Show
          Sijie Guo added a comment - - edited Journal implements CheckpointListener. LedgerStorage takes a Checkpoint listener on construction. SyncThread is broken out from Bookie, and LedgerStorage constructs and owns it, but the implementation is separate from any particular LedgerStorage implementation. Journal implements CheckpointListener#checkpointOccurred(CheckPoint) which moves the "lastmark" pointer. it doesn't make any sense to make ledger storage control journal to move the mark. the checkpoint flow is not simple a responsibility of a single component. A bookie's data has two parts, one is persisted indexed data, the other is in-memory indexed data + persisted txns. And a checkpoint is the boundary of these two parts. The SyncThread is the role to change the data from one state (in-memory + txns) to the other state (permanent indexable): 1) ledger storage persist its in-memory data, 2) journal move its pointer and gc journals if necessary, 3) if any is failed due to disk issue, it means that such kind of data conversion could not proceed any more, the bookie should be turned to readonly at this moment (otherwise, it would be OOM). when you moved sync thread to ledger storage, you are moving the 2) and 3) part implicitly to ledger storage, which make ledger storage too ambitious and the flow unclear.
          Hide
          Sijie Guo added a comment -

          This should be better. The journal should just be constructed with Journal(conf). LastSyncedLogMark should only come into play for Journal#replay(JournalScanner) which should become Journal#replay(LogMark from, JournalScanner).

          it changes nothing. either you passed the point in constructor or in replay. journal still can't replay anything that already gc'ed.

          Show
          Sijie Guo added a comment - This should be better. The journal should just be constructed with Journal(conf). LastSyncedLogMark should only come into play for Journal#replay(JournalScanner) which should become Journal#replay(LogMark from, JournalScanner). it changes nothing. either you passed the point in constructor or in replay. journal still can't replay anything that already gc'ed.
          Hide
          Hadoop QA added a comment -

          Testing JIRA BOOKKEEPER-564

          Patch BOOKKEEPER-564.patch downloaded at Fri Apr 26 01:47:57 UTC 2013

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

          +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 contains 1 line(s) longer than 120 characters
          . +1 the patch does adds/modifies 6 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: 833
          +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/325/

          Show
          Hadoop QA added a comment - Testing JIRA BOOKKEEPER-564 Patch BOOKKEEPER-564.patch downloaded at Fri Apr 26 01:47:57 UTC 2013 ---------------------------- +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 contains 1 line(s) longer than 120 characters . +1 the patch does adds/modifies 6 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: 833 +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/325/
          Hide
          Ivan Kelly added a comment -

          a new patch to remove circular dependency by make the ledger storage interface as CheckPoint LedgerStorage#checkpoint(CheckPoint). The return value indicates the actual point that the ledger storage already synced all the data. keep the original sync thread.

          The patch does not remove the circular dependency. It's still there in the same form. There's no way to implement Checkpointer, without something like

          Checkpointer cpkt = Checkpointer();
          LedgerStorage ledgerStorage = new InterleavedLedgerStorage(cpkt);
          cpkt.setLedgerStorage(ledgerStorage);
          

          This patch doesn't explicitly do this, because SyncThread is an innerclass of Bookie, but implicitly it's the exact same thing. Try making SyncThread a static inner class and see what happens. Whats more, you don't even need the circular dependency in your patch. SyncThread#newCheckpoint calls journal#requestCheckpoint. You could simply have Journal implement Checkpointer. But by now, the name Checkpointer doesn't make sense anymore, as you aren't able to trigger a checkpoint action through the interface, so maybe rename it to CheckpointSource. And if Journal is implementing this, you may as well add checkpointComlete(Checkpoint) to CheckpointSource. And remove Checkpoint#checkpointComplete(). So the SyncThread can call journal.checkpointComplete(checkpoint), and it's clearer where the operation is occurring. Checkpoint#checkpointComplete() looks a lot like action at a distance.

          So -1 For this patch.

          1) we don't need to rewrite sync thread, if we provide a new ledger storage.
          2) syncing when entry logger rotated is a tradeoff between recovery time and performance. if in future, we want to reduce recovery time and keep performance, it is easy to do it.

          This can be achieved by breaking SyncThread into it's own class, and having the ledger storage own it.

          Journal implements CheckpointListener#checkpointOccurred(CheckPoint) which moves the "lastmark" pointer.

          it doesn't make any sense to make ledger storage control journal to move the mark.

          Agreed, this is what I've been arguing for weeks. But I would also go further and say it makes no sense for the journal to move the mark. There's are 3 components in checkpointing, the ledger storage, the journal and the checkpoint mark. The checkpoint mark is relates to both the ledger storage and the journal. Marks are created by the journal, and refer to the journal position, but the decision of which mark is the checkpoint mark is taken by the ledger storage. This is why I argued that it should not be stored in the journal but in the ledger storage. Equally it could be separate component completely. But since you argued it should be in the journal, and since it's a fact that the ledger storage decides which mark is the checkpoint mark, I added this interface.

          Show
          Ivan Kelly added a comment - a new patch to remove circular dependency by make the ledger storage interface as CheckPoint LedgerStorage#checkpoint(CheckPoint). The return value indicates the actual point that the ledger storage already synced all the data. keep the original sync thread. The patch does not remove the circular dependency. It's still there in the same form. There's no way to implement Checkpointer, without something like Checkpointer cpkt = Checkpointer(); LedgerStorage ledgerStorage = new InterleavedLedgerStorage(cpkt); cpkt.setLedgerStorage(ledgerStorage); This patch doesn't explicitly do this, because SyncThread is an innerclass of Bookie, but implicitly it's the exact same thing. Try making SyncThread a static inner class and see what happens. Whats more, you don't even need the circular dependency in your patch. SyncThread#newCheckpoint calls journal#requestCheckpoint. You could simply have Journal implement Checkpointer. But by now, the name Checkpointer doesn't make sense anymore, as you aren't able to trigger a checkpoint action through the interface, so maybe rename it to CheckpointSource. And if Journal is implementing this, you may as well add checkpointComlete(Checkpoint) to CheckpointSource. And remove Checkpoint#checkpointComplete(). So the SyncThread can call journal.checkpointComplete(checkpoint), and it's clearer where the operation is occurring. Checkpoint#checkpointComplete() looks a lot like action at a distance. So -1 For this patch. 1) we don't need to rewrite sync thread, if we provide a new ledger storage. 2) syncing when entry logger rotated is a tradeoff between recovery time and performance. if in future, we want to reduce recovery time and keep performance, it is easy to do it. This can be achieved by breaking SyncThread into it's own class, and having the ledger storage own it. Journal implements CheckpointListener#checkpointOccurred(CheckPoint) which moves the "lastmark" pointer. it doesn't make any sense to make ledger storage control journal to move the mark. Agreed, this is what I've been arguing for weeks. But I would also go further and say it makes no sense for the journal to move the mark. There's are 3 components in checkpointing, the ledger storage, the journal and the checkpoint mark. The checkpoint mark is relates to both the ledger storage and the journal. Marks are created by the journal, and refer to the journal position, but the decision of which mark is the checkpoint mark is taken by the ledger storage. This is why I argued that it should not be stored in the journal but in the ledger storage. Equally it could be separate component completely. But since you argued it should be in the journal, and since it's a fact that the ledger storage decides which mark is the checkpoint mark, I added this interface.
          Hide
          Sijie Guo added a comment - - edited
          This patch doesn't explicitly do this, because SyncThread is an innerclass of Bookie, but implicitly it's the exact same thing. Try making SyncThread a static inner class and see what happens. Whats more, you don't even need the circular dependency in your patch. SyncThread#newCheckpoint calls journal#requestCheckpoint. You could simply have Journal implement Checkpointer. But by now, the name Checkpointer doesn't make sense anymore, as you aren't able to trigger a checkpoint action through the interface, so maybe rename it to CheckpointSource. And if Journal is implementing this, you may as well add checkpointComlete(Checkpoint) to CheckpointSource. And remove Checkpoint#checkpointComplete(). So the SyncThread can call journal.checkpointComplete(checkpoint), and it's clearer where the operation is occurring. Checkpoint#checkpointComplete() looks a lot like action at a distance.
          

          ok. provide the CheckPointSource, which provide the method, newCheckpoint. Journal implmented this interface. SyncThread doesn't change. so ledger storage could get checkpoint from checkpointsource. If you agreed on it, I made the change.

          If you still don't like it, I would suggest keeping current SyncThread unchanged. I would break the entrylogger preallocation into a separated jira.

          Show
          Sijie Guo added a comment - - edited This patch doesn't explicitly do this , because SyncThread is an innerclass of Bookie, but implicitly it's the exact same thing. Try making SyncThread a static inner class and see what happens. Whats more, you don't even need the circular dependency in your patch. SyncThread#newCheckpoint calls journal#requestCheckpoint. You could simply have Journal implement Checkpointer. But by now, the name Checkpointer doesn't make sense anymore, as you aren't able to trigger a checkpoint action through the interface , so maybe rename it to CheckpointSource. And if Journal is implementing this , you may as well add checkpointComlete(Checkpoint) to CheckpointSource. And remove Checkpoint#checkpointComplete(). So the SyncThread can call journal.checkpointComplete(checkpoint), and it's clearer where the operation is occurring. Checkpoint#checkpointComplete() looks a lot like action at a distance. ok. provide the CheckPointSource, which provide the method, newCheckpoint. Journal implmented this interface. SyncThread doesn't change. so ledger storage could get checkpoint from checkpointsource. If you agreed on it, I made the change. If you still don't like it, I would suggest keeping current SyncThread unchanged. I would break the entrylogger preallocation into a separated jira.
          Hide
          Sijie Guo added a comment -

          This can be achieved by breaking SyncThread into it's own class, and having the ledger storage own it.

          you can't. as I commented in previous comment, syncing flow is comprised of 3 steps, which is related to bookie's state. if you break it into its own class, and having ledger storage own it, you introduced the circular dependency from ledger storage to bookie.

          Show
          Sijie Guo added a comment - This can be achieved by breaking SyncThread into it's own class, and having the ledger storage own it. you can't. as I commented in previous comment, syncing flow is comprised of 3 steps, which is related to bookie's state. if you break it into its own class, and having ledger storage own it, you introduced the circular dependency from ledger storage to bookie.
          Hide
          Ivan Kelly added a comment -

          I'm modified your patch to remove the prealloc stuff and make Checkpointer into CheckpointSource.

          There's a couple of other mods I'd like to do, like make SyncThread usable by tests, and cleaning up the LogMark stuff (there's far too many classes managing what amounts to 2 longs), but that can go in followup jiras.

          Show
          Ivan Kelly added a comment - I'm modified your patch to remove the prealloc stuff and make Checkpointer into CheckpointSource. There's a couple of other mods I'd like to do, like make SyncThread usable by tests, and cleaning up the LogMark stuff (there's far too many classes managing what amounts to 2 longs), but that can go in followup jiras.
          Hide
          Hadoop QA added a comment -

          Testing JIRA BOOKKEEPER-564

          Patch 0001-BOOKKEEPER-584-CheckpointSource.patch downloaded at Mon Apr 29 14:21:33 UTC 2013

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

          +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 3 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 - patch does not compile, cannot run testcases
          +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/327/

          Show
          Hadoop QA added a comment - Testing JIRA BOOKKEEPER-564 Patch 0001-BOOKKEEPER-584-CheckpointSource.patch downloaded at Mon Apr 29 14:21:33 UTC 2013 ---------------------------- +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 3 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 - patch does not compile, cannot run testcases +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/327/
          Hide
          Hadoop QA added a comment -

          Testing JIRA BOOKKEEPER-564

          Patch 0001-BOOKKEEPER-584-CheckpointSource.patch downloaded at Mon Apr 29 15:40:39 UTC 2013

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

          +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 3 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: 834
          +1 DISTRO
          . +1 distro tarball builds with the patch

          ----------------------------
          +1 Overall result, good!, no -1s

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

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

          Show
          Hadoop QA added a comment - Testing JIRA BOOKKEEPER-564 Patch 0001-BOOKKEEPER-584-CheckpointSource.patch downloaded at Mon Apr 29 15:40:39 UTC 2013 ---------------------------- +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 3 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: 834 +1 DISTRO . +1 distro tarball builds with the patch ---------------------------- +1 Overall result, good!, no -1s The full output of the test-patch run is available at . https://builds.apache.org/job/bookkeeper-trunk-precommit-build/328/
          Hide
          Rakesh R added a comment -

          Thanks Ivan Kelly, Sijie Guo for the patch. Just few comments.

          • Following are not used anywhere, can we remove these?
            public static final Checkpoint MIN = new Checkpoint() {
            static final public LogMark MAX_VALUE = new LogMark(Long.MAX_VALUE, Long.MAX_VALUE);
            
          • Just make the LogMarkCheckpoint to 'private'
            private static class LogMarkCheckpoint implements Checkpoint {
          • I think, journal.checkpointComplete() can mark the checkpoint completion only when the
            journal.newCheckpoint().compareTo(lastcheckpoint) > 0
          • One general thought. Is it required to do checkpointing, once bookie marked as readonly?
          Show
          Rakesh R added a comment - Thanks Ivan Kelly , Sijie Guo for the patch. Just few comments. Following are not used anywhere, can we remove these? public static final Checkpoint MIN = new Checkpoint() { static final public LogMark MAX_VALUE = new LogMark( Long .MAX_VALUE, Long .MAX_VALUE); Just make the LogMarkCheckpoint to 'private' private static class LogMarkCheckpoint implements Checkpoint { I think, journal.checkpointComplete() can mark the checkpoint completion only when the journal.newCheckpoint().compareTo(lastcheckpoint) > 0 One general thought. Is it required to do checkpointing, once bookie marked as readonly?
          Hide
          Sijie Guo added a comment -

          +1 for the patch.

          Show
          Sijie Guo added a comment - +1 for the patch.
          Hide
          Ivan Kelly added a comment -

          Rakesh R new patch addresses some of the comments, for the rest..

          public static final Checkpoint MIN = new Checkpoint() {

          This is used.

          I think, journal.checkpointComplete() can mark the checkpoint completion only when the
          journal.newCheckpoint().compareTo(lastcheckpoint) > 0

          I'd prefer to do this in the separate jira to clean up log marks.

          One general thought. Is it required to do checkpointing, once bookie marked as readonly?

          Bookie can go RO when journal disk fails but ledger disk is still fine. There may be entries still unsynced to disk, so it's ok to have the sync thread continue running.

          Show
          Ivan Kelly added a comment - Rakesh R new patch addresses some of the comments, for the rest.. public static final Checkpoint MIN = new Checkpoint() { This is used. I think, journal.checkpointComplete() can mark the checkpoint completion only when the journal.newCheckpoint().compareTo(lastcheckpoint) > 0 I'd prefer to do this in the separate jira to clean up log marks. One general thought. Is it required to do checkpointing, once bookie marked as readonly? Bookie can go RO when journal disk fails but ledger disk is still fine. There may be entries still unsynced to disk, so it's ok to have the sync thread continue running.
          Hide
          Hadoop QA added a comment -

          Testing JIRA BOOKKEEPER-564

          Patch 0001-BOOKKEEPER-564-CheckpointSource.patch downloaded at Tue Apr 30 11:11:32 UTC 2013

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

          +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 3 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: 834
          +1 DISTRO
          . +1 distro tarball builds with the patch

          ----------------------------
          +1 Overall result, good!, no -1s

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

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

          Show
          Hadoop QA added a comment - Testing JIRA BOOKKEEPER-564 Patch 0001-BOOKKEEPER-564-CheckpointSource.patch downloaded at Tue Apr 30 11:11:32 UTC 2013 ---------------------------- +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 3 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: 834 +1 DISTRO . +1 distro tarball builds with the patch ---------------------------- +1 Overall result, good!, no -1s The full output of the test-patch run is available at . https://builds.apache.org/job/bookkeeper-trunk-precommit-build/329/
          Hide
          Rakesh R added a comment -
          public static final Checkpoint MIN = new Checkpoint() {
          

          is used only in the following checks in LogMarkCheckpoint.java. But I couldn't see any checkpoint object is getting assigned with Checkpoint.MIN, and its an unreachable/unused code.

          else if (o == Checkpoint.MIN) {
                 return 1;
          }
          
          
          
          Show
          Rakesh R added a comment - public static final Checkpoint MIN = new Checkpoint() { is used only in the following checks in LogMarkCheckpoint.java. But I couldn't see any checkpoint object is getting assigned with Checkpoint.MIN, and its an unreachable/unused code. else if (o == Checkpoint.MIN) { return 1; }
          Hide
          Ivan Kelly added a comment -

          new patch removes MIN.

          Show
          Ivan Kelly added a comment - new patch removes MIN.
          Hide
          Hadoop QA added a comment -

          Testing JIRA BOOKKEEPER-564

          Patch 0001-BOOKKEEPER-564-CheckpointSource.patch downloaded at Thu May 2 08:21:31 UTC 2013

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

          +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 3 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: 834
          +1 DISTRO
          . +1 distro tarball builds with the patch

          ----------------------------
          +1 Overall result, good!, no -1s

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

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

          Show
          Hadoop QA added a comment - Testing JIRA BOOKKEEPER-564 Patch 0001-BOOKKEEPER-564-CheckpointSource.patch downloaded at Thu May 2 08:21:31 UTC 2013 ---------------------------- +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 3 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: 834 +1 DISTRO . +1 distro tarball builds with the patch ---------------------------- +1 Overall result, good!, no -1s The full output of the test-patch run is available at . https://builds.apache.org/job/bookkeeper-trunk-precommit-build/330/
          Hide
          Rakesh R added a comment -

          Thanks Ivan Kelly and Sijie Guo. +1 latest patch looks nice.

          Show
          Rakesh R added a comment - Thanks Ivan Kelly and Sijie Guo . +1 latest patch looks nice.
          Hide
          Ivan Kelly added a comment -

          Cancelling patch. I'm getting a pretty consistent failure with the patch on ReadOnlyBookieTest

          2013-05-02 17:56:57,944 - INFO  - [pool-7-thread-1:PerChannelBookieClient@115] - Connecting to bookie: /192.168.122.1:15001
          2013-05-02 17:56:57,965 - INFO  - [pool-7-thread-1:PerChannelBookieClient@115] - Connecting to bookie: /192.168.122.1:15003
          2013-05-02 17:56:57,968 - INFO  - [New I/O client worker #1-1:PerChannelBookieClient$1@135] - Successfully connected to bookie: /192.168.122.1:15001
          2013-05-02 17:56:57,971 - INFO  - [New I/O client worker #1-2:PerChannelBookieClient$1@135] - Successfully connected to bookie: /192.168.122.1:15003
          2013-05-02 17:56:58,091 - WARN  - [main:LedgerDirsManager@98] - /tmp/bookie5716829601644809116test/current is out of space. Adding it to filled dirs list
          2013-05-02 17:56:58,170 - ERROR - [SyncThread:Bookie$SyncThread@366] - Exception in SyncThread
          java.lang.NullPointerException
          	at org.apache.bookkeeper.bookie.FileInfo.moveToNewLocation(FileInfo.java:287)
          	at org.apache.bookkeeper.bookie.LedgerCacheImpl.moveLedgerIndexFile(LedgerCacheImpl.java:408)
          	at org.apache.bookkeeper.bookie.LedgerCacheImpl.flushLedger(LedgerCacheImpl.java:365)
          	at org.apache.bookkeeper.bookie.InterleavedLedgerStorage.flushOrCheckpoint(InterleavedLedgerStorage.java:169)
          	at org.apache.bookkeeper.bookie.InterleavedLedgerStorage.checkpoint(InterleavedLedgerStorage.java:207)
          	at org.apache.bookkeeper.bookie.Bookie$SyncThread.checkpoint(Bookie.java:280)
          	at org.apache.bookkeeper.bookie.Bookie$SyncThread.run(Bookie.java:361)
          2013-05-02 17:56:58,175 - INFO  - [ProcessThread(sid:0 cport:-1)::PrepRequestProcessor@466] - Processed session termination for sessionid: 0x13e65f5dec00004
          
          Show
          Ivan Kelly added a comment - Cancelling patch. I'm getting a pretty consistent failure with the patch on ReadOnlyBookieTest 2013-05-02 17:56:57,944 - INFO - [pool-7-thread-1:PerChannelBookieClient@115] - Connecting to bookie: /192.168.122.1:15001 2013-05-02 17:56:57,965 - INFO - [pool-7-thread-1:PerChannelBookieClient@115] - Connecting to bookie: /192.168.122.1:15003 2013-05-02 17:56:57,968 - INFO - [New I/O client worker #1-1:PerChannelBookieClient$1@135] - Successfully connected to bookie: /192.168.122.1:15001 2013-05-02 17:56:57,971 - INFO - [New I/O client worker #1-2:PerChannelBookieClient$1@135] - Successfully connected to bookie: /192.168.122.1:15003 2013-05-02 17:56:58,091 - WARN - [main:LedgerDirsManager@98] - /tmp/bookie5716829601644809116test/current is out of space. Adding it to filled dirs list 2013-05-02 17:56:58,170 - ERROR - [SyncThread:Bookie$SyncThread@366] - Exception in SyncThread java.lang.NullPointerException at org.apache.bookkeeper.bookie.FileInfo.moveToNewLocation(FileInfo.java:287) at org.apache.bookkeeper.bookie.LedgerCacheImpl.moveLedgerIndexFile(LedgerCacheImpl.java:408) at org.apache.bookkeeper.bookie.LedgerCacheImpl.flushLedger(LedgerCacheImpl.java:365) at org.apache.bookkeeper.bookie.InterleavedLedgerStorage.flushOrCheckpoint(InterleavedLedgerStorage.java:169) at org.apache.bookkeeper.bookie.InterleavedLedgerStorage.checkpoint(InterleavedLedgerStorage.java:207) at org.apache.bookkeeper.bookie.Bookie$SyncThread.checkpoint(Bookie.java:280) at org.apache.bookkeeper.bookie.Bookie$SyncThread.run(Bookie.java:361) 2013-05-02 17:56:58,175 - INFO - [ProcessThread(sid:0 cport:-1)::PrepRequestProcessor@466] - Processed session termination for sessionid: 0x13e65f5dec00004
          Hide
          Ivan Kelly added a comment -

          Issue was in FileInfo. checkOpen(false) being called but fc not checked for not null. Fixed.

          Show
          Ivan Kelly added a comment - Issue was in FileInfo. checkOpen(false) being called but fc not checked for not null. Fixed.
          Hide
          Rakesh R added a comment -

          Ivan Kelly good catch. As per the logic, moveToNewLocation() requires a null check. I'd like to know when can 'fc' become null?
          Is this a situation like, just created an empty index file and at the same time flushOrCheckpoint() has occured ?

          Show
          Rakesh R added a comment - Ivan Kelly good catch. As per the logic, moveToNewLocation() requires a null check. I'd like to know when can 'fc' become null? Is this a situation like, just created an empty index file and at the same time flushOrCheckpoint() has occured ?
          Hide
          Ivan Kelly added a comment -

          when a FileInfo is created fc is null. it only gets set when checkOpen(true) is called. So if a FileInfo has been created but no operation has been called on it that would call checkOpen(true), then fc is null and a call to moveToNewLocation will give a NPE.

          Show
          Ivan Kelly added a comment - when a FileInfo is created fc is null. it only gets set when checkOpen(true) is called. So if a FileInfo has been created but no operation has been called on it that would call checkOpen(true), then fc is null and a call to moveToNewLocation will give a NPE.
          Hide
          Rakesh R added a comment -

          +1 ready to go in. pls trigger build...

          Show
          Rakesh R added a comment - +1 ready to go in. pls trigger build...
          Hide
          Hadoop QA added a comment -

          Testing JIRA BOOKKEEPER-564

          Patch 0001-BOOKKEEPER-564-CheckpointSource.patch downloaded at Sat May 4 11:48:13 UTC 2013

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

          +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 3 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: 834
          +1 DISTRO
          . +1 distro tarball builds with the patch

          ----------------------------
          +1 Overall result, good!, no -1s

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

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

          Show
          Hadoop QA added a comment - Testing JIRA BOOKKEEPER-564 Patch 0001-BOOKKEEPER-564-CheckpointSource.patch downloaded at Sat May 4 11:48:13 UTC 2013 ---------------------------- +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 3 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: 834 +1 DISTRO . +1 distro tarball builds with the patch ---------------------------- +1 Overall result, good!, no -1s The full output of the test-patch run is available at . https://builds.apache.org/job/bookkeeper-trunk-precommit-build/331/
          Hide
          Ivan Kelly added a comment -

          Committed r1479085. Thanks for all the work to get this change in guys. Its a great first step towards getting more efficient ledger storage into trunk.

          Show
          Ivan Kelly added a comment - Committed r1479085. Thanks for all the work to get this change in guys. Its a great first step towards getting more efficient ledger storage into trunk.
          Hide
          Hudson added a comment -

          Integrated in bookkeeper-trunk #195 (See https://builds.apache.org/job/bookkeeper-trunk/195/)
          BOOKKEEPER-564: Better checkpoint mechanism (sijie & ivank) (Revision 1479085)

          Result = SUCCESS
          ivank :
          Files :

          • /zookeeper/bookkeeper/trunk/CHANGES.txt
          • /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java
          • /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java
          • /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/CheckpointSource.java
          • /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java
          • /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfo.java
          • /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java
          • /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java
          • /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java
          • /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java
          • /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LogMark.java
          • /zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieAccessor.java
          • /zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java
          • /zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/LedgerDeleteTest.java
          Show
          Hudson added a comment - Integrated in bookkeeper-trunk #195 (See https://builds.apache.org/job/bookkeeper-trunk/195/ ) BOOKKEEPER-564 : Better checkpoint mechanism (sijie & ivank) (Revision 1479085) Result = SUCCESS ivank : Files : /zookeeper/bookkeeper/trunk/CHANGES.txt /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/CheckpointSource.java /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfo.java /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LogMark.java /zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieAccessor.java /zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java /zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/LedgerDeleteTest.java

            People

            • Assignee:
              Sijie Guo
              Reporter:
              Sijie Guo
            • Votes:
              0 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development