You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@bookkeeper.apache.org by Charan Reddy G <re...@gmail.com> on 2017/10/13 06:34:12 UTC

Question regarding Checkpoint logic in SortedLedgerStorage

Hey Sijie/IvanK,

With
https://github.com/apache/bookkeeper/commit/d175ada58dcaf78f0a70b0ebebf489255ae67b5f
you introduced Bookkeeper-564 : Better checkpoint mechanism -  Scheduling
checkpoint only when rotating an entry log file.

I'm trying to understand how it would work in the following scenario
- using SortedLedgerStorage
- since it is SortedLedgerStorage entries would be in EntryMemtable
- GarbageCollectorThread.EntryLogScanner.process method calls entryLogger
.addEntry(ledgerId, entry)
- in EntryLogger.addEntry method, lets say it comes to know it has reached
EntryLogLimit and creates NewLog
-  since current active entrylog is rotated,
EntryLogListener.onRotateEntryLog is called
- which sets the currentMark of journal to checkpointHolder. Point to note,
that all the entries added to the Bookie are not added to entryLog yet,
there are entries still in entrymemtable
- lets say SyncThread tries to checkpoint at this instant

now the concern is, in SortedLedgerStorage.checkpoint method, before
calling super.checkpoint(checkpoint), it does memTable.flush(this,
checkpoint); But memTable.flush would just add entries to the current
active entrylog (BufferedLogChannel) and it doesn't guarantee persistence.
super(InterLeavedLedgerStorage).checkpoint will only flushRotatedLogs
(persists) and finally mark the checkpointcomplete with 'lastcheckpoint',
but the 'lastCheckpoint' in the checkpointHolder would also include the
entries which were in Entrymemtable and are not actually persisted in the
whole process. Is there issue in SortedLedgerStorage checkpoint logic?

        @Override
    public Checkpoint checkpoint(final Checkpoint checkpoint) throws
IOException {
        Checkpoint lastCheckpoint = checkpointHolder.getLastCheckpoint();
        // if checkpoint is less than last checkpoint, we don't need to do
checkpoint again.
        if (lastCheckpoint.compareTo(checkpoint) > 0) {
            return lastCheckpoint;
        }
        memTable.flush(this, checkpoint);
        return super.checkpoint(checkpoint);
    }

Thanks,
Charan

Re: Question regarding Checkpoint logic in SortedLedgerStorage

Posted by Sijie Guo <gu...@gmail.com>.
On Oct 25, 2017 12:53 AM, "Enrico Olivelli - Diennea" <
enrico.olivelli@diennea.com> wrote:

Il giorno mar, 24/10/2017 alle 15.41 -0700, Sijie Guo ha scritto:

I filed an issue for the problem discussed here:
https://github.com/apache/bookkeeper/issues/659


Sijie, Charan, thank you for catching this
IMHO we should take options 1 o 2 as Sijie proposed


I think we agreed on option 1.


Sijie,
Do you have an approximate idea about affected versions 4.3, 4.4, 4.5 ?


I think it impacts all those versions at sorted ledger storage, when
compaction happens to advance the checkpoint holder.

Sijie



Enrico




- Sijie

On Tue, Oct 24, 2017 at 2:06 PM, Charan Reddy G <reddycharan18@gmail.com<
mailto:reddycharan18@gmail.com>>
wrote:



Sure Sijie and Ivan. It sounds appropriate.

Thanks,
Charan

On Tue, Oct 24, 2017 at 1:00 AM, Sijie Guo <guosijie@gmail.com<mailto:guo
sijie@gmail.com>> wrote:



Yes, we should fix this ASAP.

Charan, what is your opinion on the fix?

- Sijie

On Tue, Oct 24, 2017 at 12:12 AM, Ivan Kelly <ivank@apache.org<mailto:ivank
@apache.org>> wrote:





Use the approach that I used before (as in the old commits at twitter’s
branch). Ledger storage is responsible for instantiating the


checkpoints.
I would go with this approach. It was probably me that asked for the
changes in the other direction before, but I can't remember why I
asked for them, so the view held strongly then isn't held strongly
now.

It would be good to get this change in soon, so that when the yahoo
DbLedgerStorage goes in, we can verify that checkpointing works as
expected.

-Ivan










--

Enrico Olivelli Software Development Manager @Diennea Tel.: (+39) 0546
066100 - Int. 925 Viale G.Marconi 30/14 - 48018 Faenza (RA) MagNews -
E-mail Marketing Solutions http://www.magnews.it Diennea - Digital
Marketing Solutions http://www.diennea.com

________________________________

Iscriviti alla nostra newsletter per rimanere aggiornato su digital ed
email marketing! http://www.magnews.it/newsletter/

The information in this email is confidential and may be legally
privileged. If you are not the intended recipient please notify the sender
immediately and destroy this email. Any unauthorized, direct or indirect,
disclosure, copying, storage, distribution or other use is strictly
forbidden.

Re: Question regarding Checkpoint logic in SortedLedgerStorage

Posted by Enrico Olivelli - Diennea <en...@diennea.com>.
Il giorno mar, 24/10/2017 alle 15.41 -0700, Sijie Guo ha scritto:

I filed an issue for the problem discussed here:
https://github.com/apache/bookkeeper/issues/659


Sijie, Charan, thank you for catching this
IMHO we should take options 1 o 2 as Sijie proposed

Sijie,
Do you have an approximate idea about affected versions 4.3, 4.4, 4.5 ?

Enrico




- Sijie

On Tue, Oct 24, 2017 at 2:06 PM, Charan Reddy G <re...@gmail.com>>
wrote:



Sure Sijie and Ivan. It sounds appropriate.

Thanks,
Charan

On Tue, Oct 24, 2017 at 1:00 AM, Sijie Guo <gu...@gmail.com>> wrote:



Yes, we should fix this ASAP.

Charan, what is your opinion on the fix?

- Sijie

On Tue, Oct 24, 2017 at 12:12 AM, Ivan Kelly <iv...@apache.org>> wrote:





Use the approach that I used before (as in the old commits at twitter’s
branch). Ledger storage is responsible for instantiating the


checkpoints.
I would go with this approach. It was probably me that asked for the
changes in the other direction before, but I can't remember why I
asked for them, so the view held strongly then isn't held strongly
now.

It would be good to get this change in soon, so that when the yahoo
DbLedgerStorage goes in, we can verify that checkpointing works as
expected.

-Ivan










--

Enrico Olivelli Software Development Manager @Diennea Tel.: (+39) 0546 066100 - Int. 925 Viale G.Marconi 30/14 - 48018 Faenza (RA) MagNews - E-mail Marketing Solutions http://www.magnews.it Diennea - Digital Marketing Solutions http://www.diennea.com

________________________________

Iscriviti alla nostra newsletter per rimanere aggiornato su digital ed email marketing! http://www.magnews.it/newsletter/

The information in this email is confidential and may be legally privileged. If you are not the intended recipient please notify the sender immediately and destroy this email. Any unauthorized, direct or indirect, disclosure, copying, storage, distribution or other use is strictly forbidden.

Re: Question regarding Checkpoint logic in SortedLedgerStorage

Posted by Sijie Guo <gu...@gmail.com>.
FYI. I created a test to reproduce the behavior.
https://github.com/apache/bookkeeper/pull/677

- Sijie

On Tue, Oct 24, 2017 at 3:41 PM, Sijie Guo <gu...@gmail.com> wrote:

> I filed an issue for the problem discussed here: https://github.com/
> apache/bookkeeper/issues/659
>
> - Sijie
>
> On Tue, Oct 24, 2017 at 2:06 PM, Charan Reddy G <re...@gmail.com>
> wrote:
>
>> Sure Sijie and Ivan. It sounds appropriate.
>>
>> Thanks,
>> Charan
>>
>> On Tue, Oct 24, 2017 at 1:00 AM, Sijie Guo <gu...@gmail.com> wrote:
>>
>>> Yes, we should fix this ASAP.
>>>
>>> Charan, what is your opinion on the fix?
>>>
>>> - Sijie
>>>
>>> On Tue, Oct 24, 2017 at 12:12 AM, Ivan Kelly <iv...@apache.org> wrote:
>>>
>>>> > Use the approach that I used before (as in the old commits at
>>>> twitter’s
>>>> > branch). Ledger storage is responsible for instantiating the
>>>> checkpoints.
>>>> I would go with this approach. It was probably me that asked for the
>>>> changes in the other direction before, but I can't remember why I
>>>> asked for them, so the view held strongly then isn't held strongly
>>>> now.
>>>>
>>>> It would be good to get this change in soon, so that when the yahoo
>>>> DbLedgerStorage goes in, we can verify that checkpointing works as
>>>> expected.
>>>>
>>>> -Ivan
>>>>
>>>
>>>
>>
>

Re: Question regarding Checkpoint logic in SortedLedgerStorage

Posted by Sijie Guo <gu...@gmail.com>.
I filed an issue for the problem discussed here:
https://github.com/apache/bookkeeper/issues/659

- Sijie

On Tue, Oct 24, 2017 at 2:06 PM, Charan Reddy G <re...@gmail.com>
wrote:

> Sure Sijie and Ivan. It sounds appropriate.
>
> Thanks,
> Charan
>
> On Tue, Oct 24, 2017 at 1:00 AM, Sijie Guo <gu...@gmail.com> wrote:
>
>> Yes, we should fix this ASAP.
>>
>> Charan, what is your opinion on the fix?
>>
>> - Sijie
>>
>> On Tue, Oct 24, 2017 at 12:12 AM, Ivan Kelly <iv...@apache.org> wrote:
>>
>>> > Use the approach that I used before (as in the old commits at twitter’s
>>> > branch). Ledger storage is responsible for instantiating the
>>> checkpoints.
>>> I would go with this approach. It was probably me that asked for the
>>> changes in the other direction before, but I can't remember why I
>>> asked for them, so the view held strongly then isn't held strongly
>>> now.
>>>
>>> It would be good to get this change in soon, so that when the yahoo
>>> DbLedgerStorage goes in, we can verify that checkpointing works as
>>> expected.
>>>
>>> -Ivan
>>>
>>
>>
>

Re: Question regarding Checkpoint logic in SortedLedgerStorage

Posted by Charan Reddy G <re...@gmail.com>.
Sure Sijie and Ivan. It sounds appropriate.

Thanks,
Charan

On Tue, Oct 24, 2017 at 1:00 AM, Sijie Guo <gu...@gmail.com> wrote:

> Yes, we should fix this ASAP.
>
> Charan, what is your opinion on the fix?
>
> - Sijie
>
> On Tue, Oct 24, 2017 at 12:12 AM, Ivan Kelly <iv...@apache.org> wrote:
>
>> > Use the approach that I used before (as in the old commits at twitter’s
>> > branch). Ledger storage is responsible for instantiating the
>> checkpoints.
>> I would go with this approach. It was probably me that asked for the
>> changes in the other direction before, but I can't remember why I
>> asked for them, so the view held strongly then isn't held strongly
>> now.
>>
>> It would be good to get this change in soon, so that when the yahoo
>> DbLedgerStorage goes in, we can verify that checkpointing works as
>> expected.
>>
>> -Ivan
>>
>
>

Re: Question regarding Checkpoint logic in SortedLedgerStorage

Posted by Sijie Guo <gu...@gmail.com>.
Yes, we should fix this ASAP.

Charan, what is your opinion on the fix?

- Sijie

On Tue, Oct 24, 2017 at 12:12 AM, Ivan Kelly <iv...@apache.org> wrote:

> > Use the approach that I used before (as in the old commits at twitter’s
> > branch). Ledger storage is responsible for instantiating the checkpoints.
> I would go with this approach. It was probably me that asked for the
> changes in the other direction before, but I can't remember why I
> asked for them, so the view held strongly then isn't held strongly
> now.
>
> It would be good to get this change in soon, so that when the yahoo
> DbLedgerStorage goes in, we can verify that checkpointing works as
> expected.
>
> -Ivan
>

Re: Question regarding Checkpoint logic in SortedLedgerStorage

Posted by Ivan Kelly <iv...@apache.org>.
> Use the approach that I used before (as in the old commits at twitter’s
> branch). Ledger storage is responsible for instantiating the checkpoints.
I would go with this approach. It was probably me that asked for the
changes in the other direction before, but I can't remember why I
asked for them, so the view held strongly then isn't held strongly
now.

It would be good to get this change in soon, so that when the yahoo
DbLedgerStorage goes in, we can verify that checkpointing works as
expected.

-Ivan

Re: Question regarding Checkpoint logic in SortedLedgerStorage

Posted by Sijie Guo <gu...@gmail.com>.
Hi all,

Sorry for late response. I apologize for being late. I have spent my time
over the weekend going through all the history about the checkpoint
changes, because the implementation in current master doesn’t match what I
had in my mind. Now I think I have all the details ready for explain.

The current checkpoint logic was introduced in BOOKKEEPER-564
<https://issues.apache.org/jira/browse/BOOKKEEPER-564> by me, to avoid
fsync on current memtable and active entry log file and avoid latency
impacts. The idea behind of this checkpoint mechanism:

   - #1: all the entries before a checkpoint marker should be added into
   ledger storage on entry log file before checkpointing.
   - #2: a checkpoint marker can only be advanced on disk only after a
   successfully checkpoint (making ledger storage durably persisted)

The original implementation of this idea was:

“*Checkpoints are instantiated by ledger storage (either interleaved
storage or ledger storage), because ledger storage has the knowledge about
what entries have been persisted in entry log files.*”

I found the original implementation was *CORRECT* for both interleaved
storage and sorted storage by looking into the history of this
implementation in twitter’s branch. That was what I had in my mind when I
first time read Charan's email.

   - in interleaved storage, it starts a checkpoint when rotating an entry
   log file :
   https://github.com/twitter/bookkeeper/blob/59c2e072fe6e8725dfd380445e621d48550c0337/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java#L337
   It guarantees all the entries added before this checkpoint has been added
   into entry log files.
   - In sorted ledger storage, it DOES NOT use the checkpoint in
   interleaved storage. It uses the checkpoint marker in memtable for doing
   checkpoints. So it guarantees all the entries added before a checkpoint has
   been successfully written into entry log files, no matter how compaction is
   doing.
   https://github.com/twitter/bookkeeper/blob/59c2e072fe6e8725dfd380445e621d48550c0337/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java#L215

Now, Let’s get back to current master. The current master is taking a
different approach when I tried to contribute BOOKKEEPER-564 back to the
community due to concerns on which component should be responsible for
instantiating checkpoints. That causes the problem on sorted storage (as
what Charan described).

So the current master logic works, if you are using interleaved storage
only, so all the entries are added to entry log files before adding to
journal files, so when the entry logger rotates the files (no matter it is
from normal adds or compaction adds), it always guarantees that the entries
before a requested checkpoint marker are already added in the entry log
files.

*The logic doesn’t work for sorted ledger storage when compaction can
advance the checkpoint marker while sorted ledger manager isn’t aware of.*

I can see a few ways to address it:

   1. Use the approach that I used before (as in the old commits at
   twitter’s branch). Ledger storage is responsible for instantiating the
   checkpoints.
   2. Sorted ledger storage maintains a separate checkpoint marker (can be
   using the checkpoint markers in memtables).
   3. Compaction avoids advancing checkpoint holder in entry log files.
   This can be covered by
   https://cwiki.apache.org/confluence/display/BOOKKEEPER/BP-6+-+Use+separate+log+for+compaction

Although, I think a solution like 1 and 2 are better. The checkpoints
should be managed for a specific ledger storage implementation.


Let me know if you have any comments or questions about this. I think it
should cover all the history around checkpoint mechanism.

- Sijie

On Tue, Oct 17, 2017 at 1:09 PM, Sijie Guo <gu...@gmail.com> wrote:

>
>
> On Tue, Oct 17, 2017 at 11:46 AM, Charan Reddy G <re...@gmail.com>
> wrote:
>
>> @Sijie..Did you get chance to go through the scenario/code path.
>>
>
> Just come back to US today. I will check and get back to you.
>
>
>>
>> @JV..will create a bug, once I get clarity.
>>
>> Thanks,
>> Charan
>>
>> On Fri, Oct 13, 2017 at 5:19 PM, Venkateswara Rao Jujjuri <
>> jujjuri@gmail.com> wrote:
>>
>>> Charan this looks like an issue to me. Do we have a defect/issue opened?
>>>
>>> On Fri, Oct 13, 2017 at 4:18 PM, Sijie Guo <gu...@gmail.com> wrote:
>>>
>>> > Charan,
>>> >
>>> > Didn't mean to say the logic is correct. I was just trying to point out
>>> > the points that I remembered for checkpoint.
>>> >
>>> > I am currently traveling, so I don't have code available to check the
>>> > sequence. I can check the logic when I am close to the laptop.
>>> >
>>> > Sijie
>>> >
>>> >
>>> > On Oct 14, 2017 6:11 AM, "Charan Reddy G" <re...@gmail.com>
>>> wrote:
>>> >
>>> > Hey Sijie,
>>> >
>>> > I'm not questioning the semantics of checkpoint or the optimization
>>> which
>>> > was added with Bookkeeper-564. But my concern is are we sure,
>>> checkpoint
>>> > logic/code is correct and "marker is only updated when all the entries
>>> > added before are persisted.", in the case of SortedLedgerStorage. Can
>>> you
>>> > please go through the scenario I mentioned in my email. From what I
>>> > understood, if entryLog is rotated because of addEntry request from GC,
>>> > then we set the 'currentMark' of journal as 'lastCheckpoint' in
>>> > checkpointHolder. The entries added before this 'lastCheckpoint' are
>>> still
>>> > in EntryMemTable. When next checkpoint happens, we are not actually
>>> > persisting entries which were in EntryMemTable but we are marking
>>> > 'lastCheckpoint' as LogMarkCheckpoint (in checkpointComplete method).
>>> >
>>> > Thanks,
>>> > Charan
>>> >
>>> > On Fri, Oct 13, 2017 at 2:46 PM, Sijie Guo <gu...@gmail.com> wrote:
>>> >
>>> > > The core of the checkpoint is:
>>> > >
>>> > > - marker is only updated when all the entries added before are
>>> persisted.
>>> > > That means it doesn't affect correctness if entries added after are
>>> > > flushed.
>>> > >
>>> > > - the flush in entry log files is just writing data to filesystem.
>>> The
>>> > real
>>> > > fsync happens after checkpoint. The separate is for performance
>>> > > consideration.
>>> > >
>>> > >
>>> > >
>>> > > On Oct 12, 2017 11:34 PM, "Charan Reddy G" <re...@gmail.com>
>>> > > wrote:
>>> > >
>>> > > > Hey Sijie/IvanK,
>>> > > >
>>> > > > With
>>> > > > https://github.com/apache/bookkeeper/commit/
>>> > > d175ada58dcaf78f0a70b0ebebf489
>>> > > > 255ae67b5f
>>> > > > you introduced Bookkeeper-564 : Better checkpoint mechanism -
>>> > Scheduling
>>> > > > checkpoint only when rotating an entry log file.
>>> > > >
>>> > > > I'm trying to understand how it would work in the following
>>> scenario
>>> > > > - using SortedLedgerStorage
>>> > > > - since it is SortedLedgerStorage entries would be in EntryMemtable
>>> > > > - GarbageCollectorThread.EntryLogScanner.process method calls
>>> > > entryLogger
>>> > > > .addEntry(ledgerId, entry)
>>> > > > - in EntryLogger.addEntry method, lets say it comes to know it has
>>> > > reached
>>> > > > EntryLogLimit and creates NewLog
>>> > > > -  since current active entrylog is rotated,
>>> > > > EntryLogListener.onRotateEntryLog is called
>>> > > > - which sets the currentMark of journal to checkpointHolder. Point
>>> to
>>> > > note,
>>> > > > that all the entries added to the Bookie are not added to entryLog
>>> yet,
>>> > > > there are entries still in entrymemtable
>>> > > > - lets say SyncThread tries to checkpoint at this instant
>>> > > >
>>> > > > now the concern is, in SortedLedgerStorage.checkpoint method,
>>> before
>>> > > > calling super.checkpoint(checkpoint), it does memTable.flush(this,
>>> > > > checkpoint); But memTable.flush would just add entries to the
>>> current
>>> > > > active entrylog (BufferedLogChannel) and it doesn't guarantee
>>> > > persistence.
>>> > > > super(InterLeavedLedgerStorage).checkpoint will only
>>> flushRotatedLogs
>>> > > > (persists) and finally mark the checkpointcomplete with
>>> > 'lastcheckpoint',
>>> > > > but the 'lastCheckpoint' in the checkpointHolder would also
>>> include the
>>> > > > entries which were in Entrymemtable and are not actually persisted
>>> in
>>> > the
>>> > > > whole process. Is there issue in SortedLedgerStorage checkpoint
>>> logic?
>>> > > >
>>> > > >         @Override
>>> > > >     public Checkpoint checkpoint(final Checkpoint checkpoint)
>>> throws
>>> > > > IOException {
>>> > > >         Checkpoint lastCheckpoint = checkpointHolder.
>>> > > getLastCheckpoint();
>>> > > >         // if checkpoint is less than last checkpoint, we don't
>>> need to
>>> > > do
>>> > > > checkpoint again.
>>> > > >         if (lastCheckpoint.compareTo(checkpoint) > 0) {
>>> > > >             return lastCheckpoint;
>>> > > >         }
>>> > > >         memTable.flush(this, checkpoint);
>>> > > >         return super.checkpoint(checkpoint);
>>> > > >     }
>>> > > >
>>> > > > Thanks,
>>> > > > Charan
>>> > > >
>>> > >
>>> >
>>> >
>>> >
>>>
>>>
>>> --
>>> Jvrao
>>> ---
>>> First they ignore you, then they laugh at you, then they fight you, then
>>> you win. - Mahatma Gandhi
>>>
>>
>>
>

Re: Question regarding Checkpoint logic in SortedLedgerStorage

Posted by Sijie Guo <gu...@gmail.com>.
On Tue, Oct 17, 2017 at 11:46 AM, Charan Reddy G <re...@gmail.com>
wrote:

> @Sijie..Did you get chance to go through the scenario/code path.
>

Just come back to US today. I will check and get back to you.


>
> @JV..will create a bug, once I get clarity.
>
> Thanks,
> Charan
>
> On Fri, Oct 13, 2017 at 5:19 PM, Venkateswara Rao Jujjuri <
> jujjuri@gmail.com> wrote:
>
>> Charan this looks like an issue to me. Do we have a defect/issue opened?
>>
>> On Fri, Oct 13, 2017 at 4:18 PM, Sijie Guo <gu...@gmail.com> wrote:
>>
>> > Charan,
>> >
>> > Didn't mean to say the logic is correct. I was just trying to point out
>> > the points that I remembered for checkpoint.
>> >
>> > I am currently traveling, so I don't have code available to check the
>> > sequence. I can check the logic when I am close to the laptop.
>> >
>> > Sijie
>> >
>> >
>> > On Oct 14, 2017 6:11 AM, "Charan Reddy G" <re...@gmail.com>
>> wrote:
>> >
>> > Hey Sijie,
>> >
>> > I'm not questioning the semantics of checkpoint or the optimization
>> which
>> > was added with Bookkeeper-564. But my concern is are we sure, checkpoint
>> > logic/code is correct and "marker is only updated when all the entries
>> > added before are persisted.", in the case of SortedLedgerStorage. Can
>> you
>> > please go through the scenario I mentioned in my email. From what I
>> > understood, if entryLog is rotated because of addEntry request from GC,
>> > then we set the 'currentMark' of journal as 'lastCheckpoint' in
>> > checkpointHolder. The entries added before this 'lastCheckpoint' are
>> still
>> > in EntryMemTable. When next checkpoint happens, we are not actually
>> > persisting entries which were in EntryMemTable but we are marking
>> > 'lastCheckpoint' as LogMarkCheckpoint (in checkpointComplete method).
>> >
>> > Thanks,
>> > Charan
>> >
>> > On Fri, Oct 13, 2017 at 2:46 PM, Sijie Guo <gu...@gmail.com> wrote:
>> >
>> > > The core of the checkpoint is:
>> > >
>> > > - marker is only updated when all the entries added before are
>> persisted.
>> > > That means it doesn't affect correctness if entries added after are
>> > > flushed.
>> > >
>> > > - the flush in entry log files is just writing data to filesystem. The
>> > real
>> > > fsync happens after checkpoint. The separate is for performance
>> > > consideration.
>> > >
>> > >
>> > >
>> > > On Oct 12, 2017 11:34 PM, "Charan Reddy G" <re...@gmail.com>
>> > > wrote:
>> > >
>> > > > Hey Sijie/IvanK,
>> > > >
>> > > > With
>> > > > https://github.com/apache/bookkeeper/commit/
>> > > d175ada58dcaf78f0a70b0ebebf489
>> > > > 255ae67b5f
>> > > > you introduced Bookkeeper-564 : Better checkpoint mechanism -
>> > Scheduling
>> > > > checkpoint only when rotating an entry log file.
>> > > >
>> > > > I'm trying to understand how it would work in the following scenario
>> > > > - using SortedLedgerStorage
>> > > > - since it is SortedLedgerStorage entries would be in EntryMemtable
>> > > > - GarbageCollectorThread.EntryLogScanner.process method calls
>> > > entryLogger
>> > > > .addEntry(ledgerId, entry)
>> > > > - in EntryLogger.addEntry method, lets say it comes to know it has
>> > > reached
>> > > > EntryLogLimit and creates NewLog
>> > > > -  since current active entrylog is rotated,
>> > > > EntryLogListener.onRotateEntryLog is called
>> > > > - which sets the currentMark of journal to checkpointHolder. Point
>> to
>> > > note,
>> > > > that all the entries added to the Bookie are not added to entryLog
>> yet,
>> > > > there are entries still in entrymemtable
>> > > > - lets say SyncThread tries to checkpoint at this instant
>> > > >
>> > > > now the concern is, in SortedLedgerStorage.checkpoint method, before
>> > > > calling super.checkpoint(checkpoint), it does memTable.flush(this,
>> > > > checkpoint); But memTable.flush would just add entries to the
>> current
>> > > > active entrylog (BufferedLogChannel) and it doesn't guarantee
>> > > persistence.
>> > > > super(InterLeavedLedgerStorage).checkpoint will only
>> flushRotatedLogs
>> > > > (persists) and finally mark the checkpointcomplete with
>> > 'lastcheckpoint',
>> > > > but the 'lastCheckpoint' in the checkpointHolder would also include
>> the
>> > > > entries which were in Entrymemtable and are not actually persisted
>> in
>> > the
>> > > > whole process. Is there issue in SortedLedgerStorage checkpoint
>> logic?
>> > > >
>> > > >         @Override
>> > > >     public Checkpoint checkpoint(final Checkpoint checkpoint) throws
>> > > > IOException {
>> > > >         Checkpoint lastCheckpoint = checkpointHolder.
>> > > getLastCheckpoint();
>> > > >         // if checkpoint is less than last checkpoint, we don't
>> need to
>> > > do
>> > > > checkpoint again.
>> > > >         if (lastCheckpoint.compareTo(checkpoint) > 0) {
>> > > >             return lastCheckpoint;
>> > > >         }
>> > > >         memTable.flush(this, checkpoint);
>> > > >         return super.checkpoint(checkpoint);
>> > > >     }
>> > > >
>> > > > Thanks,
>> > > > Charan
>> > > >
>> > >
>> >
>> >
>> >
>>
>>
>> --
>> Jvrao
>> ---
>> First they ignore you, then they laugh at you, then they fight you, then
>> you win. - Mahatma Gandhi
>>
>
>

Re: Question regarding Checkpoint logic in SortedLedgerStorage

Posted by Charan Reddy G <re...@gmail.com>.
@Sijie..Did you get chance to go through the scenario/code path.

@JV..will create a bug, once I get clarity.

Thanks,
Charan

On Fri, Oct 13, 2017 at 5:19 PM, Venkateswara Rao Jujjuri <jujjuri@gmail.com
> wrote:

> Charan this looks like an issue to me. Do we have a defect/issue opened?
>
> On Fri, Oct 13, 2017 at 4:18 PM, Sijie Guo <gu...@gmail.com> wrote:
>
> > Charan,
> >
> > Didn't mean to say the logic is correct. I was just trying to point out
> > the points that I remembered for checkpoint.
> >
> > I am currently traveling, so I don't have code available to check the
> > sequence. I can check the logic when I am close to the laptop.
> >
> > Sijie
> >
> >
> > On Oct 14, 2017 6:11 AM, "Charan Reddy G" <re...@gmail.com>
> wrote:
> >
> > Hey Sijie,
> >
> > I'm not questioning the semantics of checkpoint or the optimization which
> > was added with Bookkeeper-564. But my concern is are we sure, checkpoint
> > logic/code is correct and "marker is only updated when all the entries
> > added before are persisted.", in the case of SortedLedgerStorage. Can you
> > please go through the scenario I mentioned in my email. From what I
> > understood, if entryLog is rotated because of addEntry request from GC,
> > then we set the 'currentMark' of journal as 'lastCheckpoint' in
> > checkpointHolder. The entries added before this 'lastCheckpoint' are
> still
> > in EntryMemTable. When next checkpoint happens, we are not actually
> > persisting entries which were in EntryMemTable but we are marking
> > 'lastCheckpoint' as LogMarkCheckpoint (in checkpointComplete method).
> >
> > Thanks,
> > Charan
> >
> > On Fri, Oct 13, 2017 at 2:46 PM, Sijie Guo <gu...@gmail.com> wrote:
> >
> > > The core of the checkpoint is:
> > >
> > > - marker is only updated when all the entries added before are
> persisted.
> > > That means it doesn't affect correctness if entries added after are
> > > flushed.
> > >
> > > - the flush in entry log files is just writing data to filesystem. The
> > real
> > > fsync happens after checkpoint. The separate is for performance
> > > consideration.
> > >
> > >
> > >
> > > On Oct 12, 2017 11:34 PM, "Charan Reddy G" <re...@gmail.com>
> > > wrote:
> > >
> > > > Hey Sijie/IvanK,
> > > >
> > > > With
> > > > https://github.com/apache/bookkeeper/commit/
> > > d175ada58dcaf78f0a70b0ebebf489
> > > > 255ae67b5f
> > > > you introduced Bookkeeper-564 : Better checkpoint mechanism -
> > Scheduling
> > > > checkpoint only when rotating an entry log file.
> > > >
> > > > I'm trying to understand how it would work in the following scenario
> > > > - using SortedLedgerStorage
> > > > - since it is SortedLedgerStorage entries would be in EntryMemtable
> > > > - GarbageCollectorThread.EntryLogScanner.process method calls
> > > entryLogger
> > > > .addEntry(ledgerId, entry)
> > > > - in EntryLogger.addEntry method, lets say it comes to know it has
> > > reached
> > > > EntryLogLimit and creates NewLog
> > > > -  since current active entrylog is rotated,
> > > > EntryLogListener.onRotateEntryLog is called
> > > > - which sets the currentMark of journal to checkpointHolder. Point to
> > > note,
> > > > that all the entries added to the Bookie are not added to entryLog
> yet,
> > > > there are entries still in entrymemtable
> > > > - lets say SyncThread tries to checkpoint at this instant
> > > >
> > > > now the concern is, in SortedLedgerStorage.checkpoint method, before
> > > > calling super.checkpoint(checkpoint), it does memTable.flush(this,
> > > > checkpoint); But memTable.flush would just add entries to the current
> > > > active entrylog (BufferedLogChannel) and it doesn't guarantee
> > > persistence.
> > > > super(InterLeavedLedgerStorage).checkpoint will only
> flushRotatedLogs
> > > > (persists) and finally mark the checkpointcomplete with
> > 'lastcheckpoint',
> > > > but the 'lastCheckpoint' in the checkpointHolder would also include
> the
> > > > entries which were in Entrymemtable and are not actually persisted in
> > the
> > > > whole process. Is there issue in SortedLedgerStorage checkpoint
> logic?
> > > >
> > > >         @Override
> > > >     public Checkpoint checkpoint(final Checkpoint checkpoint) throws
> > > > IOException {
> > > >         Checkpoint lastCheckpoint = checkpointHolder.
> > > getLastCheckpoint();
> > > >         // if checkpoint is less than last checkpoint, we don't need
> to
> > > do
> > > > checkpoint again.
> > > >         if (lastCheckpoint.compareTo(checkpoint) > 0) {
> > > >             return lastCheckpoint;
> > > >         }
> > > >         memTable.flush(this, checkpoint);
> > > >         return super.checkpoint(checkpoint);
> > > >     }
> > > >
> > > > Thanks,
> > > > Charan
> > > >
> > >
> >
> >
> >
>
>
> --
> Jvrao
> ---
> First they ignore you, then they laugh at you, then they fight you, then
> you win. - Mahatma Gandhi
>

Re: Question regarding Checkpoint logic in SortedLedgerStorage

Posted by Venkateswara Rao Jujjuri <ju...@gmail.com>.
Charan this looks like an issue to me. Do we have a defect/issue opened?

On Fri, Oct 13, 2017 at 4:18 PM, Sijie Guo <gu...@gmail.com> wrote:

> Charan,
>
> Didn't mean to say the logic is correct. I was just trying to point out
> the points that I remembered for checkpoint.
>
> I am currently traveling, so I don't have code available to check the
> sequence. I can check the logic when I am close to the laptop.
>
> Sijie
>
>
> On Oct 14, 2017 6:11 AM, "Charan Reddy G" <re...@gmail.com> wrote:
>
> Hey Sijie,
>
> I'm not questioning the semantics of checkpoint or the optimization which
> was added with Bookkeeper-564. But my concern is are we sure, checkpoint
> logic/code is correct and "marker is only updated when all the entries
> added before are persisted.", in the case of SortedLedgerStorage. Can you
> please go through the scenario I mentioned in my email. From what I
> understood, if entryLog is rotated because of addEntry request from GC,
> then we set the 'currentMark' of journal as 'lastCheckpoint' in
> checkpointHolder. The entries added before this 'lastCheckpoint' are still
> in EntryMemTable. When next checkpoint happens, we are not actually
> persisting entries which were in EntryMemTable but we are marking
> 'lastCheckpoint' as LogMarkCheckpoint (in checkpointComplete method).
>
> Thanks,
> Charan
>
> On Fri, Oct 13, 2017 at 2:46 PM, Sijie Guo <gu...@gmail.com> wrote:
>
> > The core of the checkpoint is:
> >
> > - marker is only updated when all the entries added before are persisted.
> > That means it doesn't affect correctness if entries added after are
> > flushed.
> >
> > - the flush in entry log files is just writing data to filesystem. The
> real
> > fsync happens after checkpoint. The separate is for performance
> > consideration.
> >
> >
> >
> > On Oct 12, 2017 11:34 PM, "Charan Reddy G" <re...@gmail.com>
> > wrote:
> >
> > > Hey Sijie/IvanK,
> > >
> > > With
> > > https://github.com/apache/bookkeeper/commit/
> > d175ada58dcaf78f0a70b0ebebf489
> > > 255ae67b5f
> > > you introduced Bookkeeper-564 : Better checkpoint mechanism -
> Scheduling
> > > checkpoint only when rotating an entry log file.
> > >
> > > I'm trying to understand how it would work in the following scenario
> > > - using SortedLedgerStorage
> > > - since it is SortedLedgerStorage entries would be in EntryMemtable
> > > - GarbageCollectorThread.EntryLogScanner.process method calls
> > entryLogger
> > > .addEntry(ledgerId, entry)
> > > - in EntryLogger.addEntry method, lets say it comes to know it has
> > reached
> > > EntryLogLimit and creates NewLog
> > > -  since current active entrylog is rotated,
> > > EntryLogListener.onRotateEntryLog is called
> > > - which sets the currentMark of journal to checkpointHolder. Point to
> > note,
> > > that all the entries added to the Bookie are not added to entryLog yet,
> > > there are entries still in entrymemtable
> > > - lets say SyncThread tries to checkpoint at this instant
> > >
> > > now the concern is, in SortedLedgerStorage.checkpoint method, before
> > > calling super.checkpoint(checkpoint), it does memTable.flush(this,
> > > checkpoint); But memTable.flush would just add entries to the current
> > > active entrylog (BufferedLogChannel) and it doesn't guarantee
> > persistence.
> > > super(InterLeavedLedgerStorage).checkpoint will only flushRotatedLogs
> > > (persists) and finally mark the checkpointcomplete with
> 'lastcheckpoint',
> > > but the 'lastCheckpoint' in the checkpointHolder would also include the
> > > entries which were in Entrymemtable and are not actually persisted in
> the
> > > whole process. Is there issue in SortedLedgerStorage checkpoint logic?
> > >
> > >         @Override
> > >     public Checkpoint checkpoint(final Checkpoint checkpoint) throws
> > > IOException {
> > >         Checkpoint lastCheckpoint = checkpointHolder.
> > getLastCheckpoint();
> > >         // if checkpoint is less than last checkpoint, we don't need to
> > do
> > > checkpoint again.
> > >         if (lastCheckpoint.compareTo(checkpoint) > 0) {
> > >             return lastCheckpoint;
> > >         }
> > >         memTable.flush(this, checkpoint);
> > >         return super.checkpoint(checkpoint);
> > >     }
> > >
> > > Thanks,
> > > Charan
> > >
> >
>
>
>


-- 
Jvrao
---
First they ignore you, then they laugh at you, then they fight you, then
you win. - Mahatma Gandhi

Re: Question regarding Checkpoint logic in SortedLedgerStorage

Posted by Sijie Guo <gu...@gmail.com>.
Charan,

Didn't mean to say the logic is correct. I was just trying to point out the
points that I remembered for checkpoint.

I am currently traveling, so I don't have code available to check the
sequence. I can check the logic when I am close to the laptop.

Sijie


On Oct 14, 2017 6:11 AM, "Charan Reddy G" <re...@gmail.com> wrote:

Hey Sijie,

I'm not questioning the semantics of checkpoint or the optimization which
was added with Bookkeeper-564. But my concern is are we sure, checkpoint
logic/code is correct and "marker is only updated when all the entries
added before are persisted.", in the case of SortedLedgerStorage. Can you
please go through the scenario I mentioned in my email. From what I
understood, if entryLog is rotated because of addEntry request from GC,
then we set the 'currentMark' of journal as 'lastCheckpoint' in
checkpointHolder. The entries added before this 'lastCheckpoint' are still
in EntryMemTable. When next checkpoint happens, we are not actually
persisting entries which were in EntryMemTable but we are marking
'lastCheckpoint' as LogMarkCheckpoint (in checkpointComplete method).

Thanks,
Charan

On Fri, Oct 13, 2017 at 2:46 PM, Sijie Guo <gu...@gmail.com> wrote:

> The core of the checkpoint is:
>
> - marker is only updated when all the entries added before are persisted.
> That means it doesn't affect correctness if entries added after are
> flushed.
>
> - the flush in entry log files is just writing data to filesystem. The
real
> fsync happens after checkpoint. The separate is for performance
> consideration.
>
>
>
> On Oct 12, 2017 11:34 PM, "Charan Reddy G" <re...@gmail.com>
> wrote:
>
> > Hey Sijie/IvanK,
> >
> > With
> > https://github.com/apache/bookkeeper/commit/
> d175ada58dcaf78f0a70b0ebebf489
> > 255ae67b5f
> > you introduced Bookkeeper-564 : Better checkpoint mechanism -
Scheduling
> > checkpoint only when rotating an entry log file.
> >
> > I'm trying to understand how it would work in the following scenario
> > - using SortedLedgerStorage
> > - since it is SortedLedgerStorage entries would be in EntryMemtable
> > - GarbageCollectorThread.EntryLogScanner.process method calls
> entryLogger
> > .addEntry(ledgerId, entry)
> > - in EntryLogger.addEntry method, lets say it comes to know it has
> reached
> > EntryLogLimit and creates NewLog
> > -  since current active entrylog is rotated,
> > EntryLogListener.onRotateEntryLog is called
> > - which sets the currentMark of journal to checkpointHolder. Point to
> note,
> > that all the entries added to the Bookie are not added to entryLog yet,
> > there are entries still in entrymemtable
> > - lets say SyncThread tries to checkpoint at this instant
> >
> > now the concern is, in SortedLedgerStorage.checkpoint method, before
> > calling super.checkpoint(checkpoint), it does memTable.flush(this,
> > checkpoint); But memTable.flush would just add entries to the current
> > active entrylog (BufferedLogChannel) and it doesn't guarantee
> persistence.
> > super(InterLeavedLedgerStorage).checkpoint will only flushRotatedLogs
> > (persists) and finally mark the checkpointcomplete with
'lastcheckpoint',
> > but the 'lastCheckpoint' in the checkpointHolder would also include the
> > entries which were in Entrymemtable and are not actually persisted in
the
> > whole process. Is there issue in SortedLedgerStorage checkpoint logic?
> >
> >         @Override
> >     public Checkpoint checkpoint(final Checkpoint checkpoint) throws
> > IOException {
> >         Checkpoint lastCheckpoint = checkpointHolder.
> getLastCheckpoint();
> >         // if checkpoint is less than last checkpoint, we don't need to
> do
> > checkpoint again.
> >         if (lastCheckpoint.compareTo(checkpoint) > 0) {
> >             return lastCheckpoint;
> >         }
> >         memTable.flush(this, checkpoint);
> >         return super.checkpoint(checkpoint);
> >     }
> >
> > Thanks,
> > Charan
> >
>

Re: Question regarding Checkpoint logic in SortedLedgerStorage

Posted by Charan Reddy G <re...@gmail.com>.
Hey Sijie,

I'm not questioning the semantics of checkpoint or the optimization which
was added with Bookkeeper-564. But my concern is are we sure, checkpoint
logic/code is correct and "marker is only updated when all the entries
added before are persisted.", in the case of SortedLedgerStorage. Can you
please go through the scenario I mentioned in my email. From what I
understood, if entryLog is rotated because of addEntry request from GC,
then we set the 'currentMark' of journal as 'lastCheckpoint' in
checkpointHolder. The entries added before this 'lastCheckpoint' are still
in EntryMemTable. When next checkpoint happens, we are not actually
persisting entries which were in EntryMemTable but we are marking
'lastCheckpoint' as LogMarkCheckpoint (in checkpointComplete method).

Thanks,
Charan

On Fri, Oct 13, 2017 at 2:46 PM, Sijie Guo <gu...@gmail.com> wrote:

> The core of the checkpoint is:
>
> - marker is only updated when all the entries added before are persisted.
> That means it doesn't affect correctness if entries added after are
> flushed.
>
> - the flush in entry log files is just writing data to filesystem. The real
> fsync happens after checkpoint. The separate is for performance
> consideration.
>
>
>
> On Oct 12, 2017 11:34 PM, "Charan Reddy G" <re...@gmail.com>
> wrote:
>
> > Hey Sijie/IvanK,
> >
> > With
> > https://github.com/apache/bookkeeper/commit/
> d175ada58dcaf78f0a70b0ebebf489
> > 255ae67b5f
> > you introduced Bookkeeper-564 : Better checkpoint mechanism -  Scheduling
> > checkpoint only when rotating an entry log file.
> >
> > I'm trying to understand how it would work in the following scenario
> > - using SortedLedgerStorage
> > - since it is SortedLedgerStorage entries would be in EntryMemtable
> > - GarbageCollectorThread.EntryLogScanner.process method calls
> entryLogger
> > .addEntry(ledgerId, entry)
> > - in EntryLogger.addEntry method, lets say it comes to know it has
> reached
> > EntryLogLimit and creates NewLog
> > -  since current active entrylog is rotated,
> > EntryLogListener.onRotateEntryLog is called
> > - which sets the currentMark of journal to checkpointHolder. Point to
> note,
> > that all the entries added to the Bookie are not added to entryLog yet,
> > there are entries still in entrymemtable
> > - lets say SyncThread tries to checkpoint at this instant
> >
> > now the concern is, in SortedLedgerStorage.checkpoint method, before
> > calling super.checkpoint(checkpoint), it does memTable.flush(this,
> > checkpoint); But memTable.flush would just add entries to the current
> > active entrylog (BufferedLogChannel) and it doesn't guarantee
> persistence.
> > super(InterLeavedLedgerStorage).checkpoint will only flushRotatedLogs
> > (persists) and finally mark the checkpointcomplete with 'lastcheckpoint',
> > but the 'lastCheckpoint' in the checkpointHolder would also include the
> > entries which were in Entrymemtable and are not actually persisted in the
> > whole process. Is there issue in SortedLedgerStorage checkpoint logic?
> >
> >         @Override
> >     public Checkpoint checkpoint(final Checkpoint checkpoint) throws
> > IOException {
> >         Checkpoint lastCheckpoint = checkpointHolder.
> getLastCheckpoint();
> >         // if checkpoint is less than last checkpoint, we don't need to
> do
> > checkpoint again.
> >         if (lastCheckpoint.compareTo(checkpoint) > 0) {
> >             return lastCheckpoint;
> >         }
> >         memTable.flush(this, checkpoint);
> >         return super.checkpoint(checkpoint);
> >     }
> >
> > Thanks,
> > Charan
> >
>

Re: Question regarding Checkpoint logic in SortedLedgerStorage

Posted by Sijie Guo <gu...@gmail.com>.
The core of the checkpoint is:

- marker is only updated when all the entries added before are persisted.
That means it doesn't affect correctness if entries added after are flushed.

- the flush in entry log files is just writing data to filesystem. The real
fsync happens after checkpoint. The separate is for performance
consideration.



On Oct 12, 2017 11:34 PM, "Charan Reddy G" <re...@gmail.com> wrote:

> Hey Sijie/IvanK,
>
> With
> https://github.com/apache/bookkeeper/commit/d175ada58dcaf78f0a70b0ebebf489
> 255ae67b5f
> you introduced Bookkeeper-564 : Better checkpoint mechanism -  Scheduling
> checkpoint only when rotating an entry log file.
>
> I'm trying to understand how it would work in the following scenario
> - using SortedLedgerStorage
> - since it is SortedLedgerStorage entries would be in EntryMemtable
> - GarbageCollectorThread.EntryLogScanner.process method calls entryLogger
> .addEntry(ledgerId, entry)
> - in EntryLogger.addEntry method, lets say it comes to know it has reached
> EntryLogLimit and creates NewLog
> -  since current active entrylog is rotated,
> EntryLogListener.onRotateEntryLog is called
> - which sets the currentMark of journal to checkpointHolder. Point to note,
> that all the entries added to the Bookie are not added to entryLog yet,
> there are entries still in entrymemtable
> - lets say SyncThread tries to checkpoint at this instant
>
> now the concern is, in SortedLedgerStorage.checkpoint method, before
> calling super.checkpoint(checkpoint), it does memTable.flush(this,
> checkpoint); But memTable.flush would just add entries to the current
> active entrylog (BufferedLogChannel) and it doesn't guarantee persistence.
> super(InterLeavedLedgerStorage).checkpoint will only flushRotatedLogs
> (persists) and finally mark the checkpointcomplete with 'lastcheckpoint',
> but the 'lastCheckpoint' in the checkpointHolder would also include the
> entries which were in Entrymemtable and are not actually persisted in the
> whole process. Is there issue in SortedLedgerStorage checkpoint logic?
>
>         @Override
>     public Checkpoint checkpoint(final Checkpoint checkpoint) throws
> IOException {
>         Checkpoint lastCheckpoint = checkpointHolder.getLastCheckpoint();
>         // if checkpoint is less than last checkpoint, we don't need to do
> checkpoint again.
>         if (lastCheckpoint.compareTo(checkpoint) > 0) {
>             return lastCheckpoint;
>         }
>         memTable.flush(this, checkpoint);
>         return super.checkpoint(checkpoint);
>     }
>
> Thanks,
> Charan
>