Re: Question regarding Checkpoint logic in SortedLedgerStorage

2017-10-24 Thread Charan Reddy G
Sure Sijie and Ivan. It sounds appropriate.

Thanks,
Charan

On Tue, Oct 24, 2017 at 1:00 AM, Sijie Guo  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  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

2017-10-17 Thread Charan Reddy G
@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  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  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" 
> 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  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" 
> > > 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

2017-10-13 Thread Charan Reddy G
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  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" 
> 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
> >
>


Question regarding Checkpoint logic in SortedLedgerStorage

2017-10-12 Thread Charan Reddy G
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 Synchronization in InterleavedLedgerStorage

2017-07-18 Thread Charan Reddy G
te:

>
>
> On Fri, Jul 14, 2017 at 6:00 PM, Sijie Guo  wrote:
>
>>
>>
>> On Sat, Jul 15, 2017 at 8:06 AM, Charan Reddy G 
>> wrote:
>>
>>> Hey,
>>>
>>> In InterleavedLedgerStorage, since the initial version of it (
>>> https://github.com/apache/bookkeeper/commit/4a94ce1d8184f5f
>>> 38def015d80777a8113b96690 and https://github.com/apache/book
>>> keeper/commit/d175ada58dcaf78f0a70b0ebebf489255ae67b5f), addEntry and
>>> processEntry methods are synchronized. If it is synchronized then I dont
>>> get what is the point in having 'writeThreadPool' in
>>> BookieRequestProcessor, if anyhow they are going to be executed
>>> sequentially because of synchronized addEntry method in
>>> InterleavedLedgerStorage.
>>>
>>
>> When InterleavedLedgerStore is used in the context of SortedLedgerStore,
>> the addEntry and processEntry are only called when flushing happened. The
>> flushing happens in background thread, which is effectively running
>> sequentially. But adding to the memtable happens concurrently.
>>
>> The reason of having 'writeThreadPool' is more on separating writes and
>> reads into different thread pools. so writes will not be affected by reads.
>> In the context of SortedLedgerStore, the 'writeThreadPool' adds the
>> concurrency.
>>
>>
>>>
>>> If we look at the implementation of addEntry and processEntry method,
>>> 'somethingWritten' can be made threadsafe by using AtomicBoolean,
>>> ledgerCache.updateLastAddConfirmed and entryLogger.addEntry methods are
>>> inherently threadsafe.
>>>
>>> I'm not sure about semantics of ledgerCache.putEntryOffset method here.
>>> I'm not confident enough to say if LedgerCacheImpl and IndexInMemPageMgr
>>> (and probably IndexPersistenceMgr) are thread-safe classes.
>>>
>>
>> LedgerCacheImpl and IndexInMemPageMgr are thread-safe classes. You can
>> confirm this from SortedLedgerStorage.
>>
>>
>>>
>>> As far as I understood, if ledgerCache.putEntryOffset is thread safe,
>>> then I dont see the need of synchronization for those methods. In any case,
>>> if they are not thread-safe can you please say why it is not thread-safe
>>> and how we can do more granular synchronization at LedgerCacheImpl level,
>>> so that we can remove the need of synchrnoization at
>>> InterleavedLedgerStorage level.
>>>
>>
>> I don't see any reason why we can't remove the synchronization.
>>
>>
>>>
>>> I'm currently working on Multiple Entrylogs -
>>> https://issues.apache.org/jira/browse/BOOKKEEPER-1041.
>>>
>>
>> I am wondering if your multiple-entrylogs approach is making things
>> complicated. I have been thinking there can be a simpler approach achieving
>> the same goal: for example, having a ledger storage comprised of N
>> interleaved/sorted ledger storages, which they share same LedgerCache, but
>> having different memtables (for sortedledgerstore) and different entry log
>> files.
>>
>
> This is more cleaner approach. @charan can you comment?
>
> JV
>
>
>>
>>
>>> To reap the benefits of multipleentrylogs feature from performance
>>> perspective, this synchrnoization should be taken care or atleast bring it
>>> down to more granular synchronization (if possible).
>>>
>>> @Override
>>> synchronized public long addEntry(ByteBuffer entry) throws
>>> IOException {
>>> long ledgerId = entry.getLong();
>>> long entryId = entry.getLong();
>>> long lac = entry.getLong();
>>> entry.rewind();
>>> processEntry(ledgerId, entryId, entry);
>>> ledgerCache.updateLastAddConfirmed(ledgerId, lac);
>>> return entryId;
>>> }
>>>
>>> synchronized protected void processEntry(long ledgerId, long
>>> entryId, ByteBuffer entry, boolean rollLog)
>>> throws IOException {
>>> somethingWritten = true;
>>> long pos = entryLogger.addEntry(ledgerId, entry, rollLog);
>>> ledgerCache.putEntryOffset(ledgerId, entryId, pos);
>>> }
>>>
>>> Thanks,
>>> Charan
>>>
>>
>>
>
>
> --
> Jvrao
> ---
> First they ignore you, then they laugh at you, then they fight you, then
> you win. - Mahatma Gandhi
>
>
>


Re: BookieInitializationTest.testWithDiskFullAndAbilityToCreateNewIndexFile testcase broken

2017-07-18 Thread Charan Reddy G
+1 LGTM.

THanks,
CHaran

On Tue, Jul 18, 2017 at 10:43 AM, Sijie Guo  wrote:

> Charan,
>
> Here is the pull request to address the problem: https://github.com/
> apache/bookkeeper/pull/257
>
> - Sijie
>
> On Wed, Jul 19, 2017 at 12:49 AM, Sijie Guo  wrote:
>
>> Sure.
>>
>> I was looking into that last night as well. I will work on that.
>>
>> - Sijie
>>
>> On Tue, Jul 18, 2017 at 4:23 PM, Charan Reddy G 
>> wrote:
>>
>>> Hey Sijie,
>>>
>>> It looks like one of your commit -https://github.com/reddychara
>>> n/bookkeeper/commit/c49621bacaa960d240afbbee83a9703b4cbc3ec2 broke
>>> BookieInitializationTest.testWithDiskFullAndAbilityToCreateNewIndexFile 
>>> testcase.
>>> Can you please check that.
>>>
>>> Thanks,
>>> Charan
>>>
>>
>>
>


BookieInitializationTest.testWithDiskFullAndAbilityToCreateNewIndexFile testcase broken

2017-07-18 Thread Charan Reddy G
Hey Sijie,

It looks like one of your commit -
https://github.com/reddycharan/bookkeeper/commit/c49621bacaa960d240afbbee83a9703b4cbc3ec2
broke BookieInitializationTest.
testWithDiskFullAndAbilityToCreateNewIndexFile testcase. Can you please
check that.

Thanks,
Charan


Question regarding Synchronization in InterleavedLedgerStorage

2017-07-14 Thread Charan Reddy G
Hey,

In InterleavedLedgerStorage, since the initial version of it (
https://github.com/apache/bookkeeper/commit/4a94ce1d8184f5f38def015d80777a8113b96690
and
https://github.com/apache/bookkeeper/commit/d175ada58dcaf78f0a70b0ebebf489255ae67b5f),
addEntry and processEntry methods are synchronized. If it is synchronized
then I dont get what is the point in having 'writeThreadPool' in
BookieRequestProcessor, if anyhow they are going to be executed
sequentially because of synchronized addEntry method in
InterleavedLedgerStorage.

If we look at the implementation of addEntry and processEntry method,
'somethingWritten' can be made threadsafe by using AtomicBoolean,
ledgerCache.updateLastAddConfirmed and entryLogger.addEntry methods are
inherently threadsafe.

I'm not sure about semantics of ledgerCache.putEntryOffset method here. I'm
not confident enough to say if LedgerCacheImpl and IndexInMemPageMgr (and
probably IndexPersistenceMgr) are thread-safe classes.

As far as I understood, if ledgerCache.putEntryOffset is thread safe, then
I dont see the need of synchronization for those methods. In any case, if
they are not thread-safe can you please say why it is not thread-safe and
how we can do more granular synchronization at LedgerCacheImpl level, so
that we can remove the need of synchrnoization at InterleavedLedgerStorage
level.

I'm currently working on Multiple Entrylogs -
https://issues.apache.org/jira/browse/BOOKKEEPER-1041. To reap the benefits
of multipleentrylogs feature from performance perspective, this
synchrnoization should be taken care or atleast bring it down to more
granular synchronization (if possible).

@Override
synchronized public long addEntry(ByteBuffer entry) throws IOException {
long ledgerId = entry.getLong();
long entryId = entry.getLong();
long lac = entry.getLong();
entry.rewind();
processEntry(ledgerId, entryId, entry);
ledgerCache.updateLastAddConfirmed(ledgerId, lac);
return entryId;
}

synchronized protected void processEntry(long ledgerId, long entryId,
ByteBuffer entry, boolean rollLog)
throws IOException {
somethingWritten = true;
long pos = entryLogger.addEntry(ledgerId, entry, rollLog);
ledgerCache.putEntryOffset(ledgerId, entryId, pos);
}

Thanks,
Charan


Re: [ANNOUNCE] JV Jujjuri joins the Apache BookKeeper PMC

2017-07-06 Thread Charan Reddy G
Congrats JV!

On Thu, Jul 6, 2017 at 9:29 AM, Enrico Olivelli  wrote:

> Congrats JV !
>
> Il gio 6 lug 2017, 18:24 Sijie Guo  ha scritto:
>
> > On behalf of the Apache BookKeeper PMC I am pleased to announce that JV
> > Jujjuri has accepted our invitation to become a PMC member on the Apache
> > BookKeeper project.
> >
> > JV has been an active contributor in many areas, including driving
> > community meetings, reviewing patches, helping with triaging bugs and
> > organizing the bookkeeper meetups. He has the ability and passion to move
> > the development of bookkeeper forward and has been speaking of Apache
> > BookKeeper in different conferences and meetups.
> >
> > Please join me in thanking JV for his contributions to date and
> > anticipation of many more contributions.
> >
> > Welcome to the PMC, JV!
> >
> > - Sijie
> >
> --
>
>
> -- Enrico Olivelli
>


CompactionTest tests are broken because of BOOKKEEPER-1088 change

2017-06-09 Thread Charan Reddy G
Hey Sijie,

It seems one of your commit
https://github.com/apache/bookkeeper/commit/5fe86525a9c823f79b3e97fd82ea4aa1c75c79eb
has broken CompactionTests tests. For me locally consistently some of the
tests in CompactionTest.java are failing with test timeout exceptions. Can
you please look into the failures.

Thanks,
Charan


Issue with 'preallocation' feature in EntryLogger

2017-05-31 Thread Charan Reddy G
Hey,

With BOOKKEEPER-643
(https://github.com/apache/bookkeeper/commit/694568b0ff0d048c284c8d5db0c9455d30dfa3ce)
feature, 'entryLogFilePreallocationEnabled' is introduced. But by the
way it is handled,  it looks like it can never be used.

In EntryLoggerAllocator.createNewLog, even if
entryLogPreAllocationEnabled is configured to true, else block is
unreachable because ‘preallocation’ will always be null and it would
end up with ‘if’ block. So effectively
‘entryLogFilePreallocationEnabled’ logic is broken.

synchronized BufferedLogChannel createNewLog() throws IOException {
BufferedLogChannel bc;
if (!entryLogPreAllocationEnabled || null == preallocation) {
// initialization time to create a new log
bc = allocateNewLog();
} else {
// has a preallocated entry log
..

..
preallocation = allocatorExecutor.submit(new
Callable() {   < this is the only place
where 'preallocation' is set and it is not possible to get into the
else block in this method  >

@Override
public BufferedLogChannel call() throws IOException {
return allocateNewLog();
}
});
}
LOG.info("Created new entry logger {}.", bc.getLogId());
return bc;
}


Thanks,
Charan


Issue in entryLogFilePreallocationEnabled logic in EntryLogger.java

2017-05-17 Thread Charan Reddy G
As part of BOOKKEEPER-643 (
https://github.com/apache/bookkeeper/commit/694568b0ff0d048c284c8d5db0c9455d30dfa3ce),
"entryLogFilePreallocationEnabled" logic is introduced in EntryLogger.java.

Because of this logic, preallocated entryLogChannel will be used as a newly
created logchannel even though the ledgerDir, in which it is created, is
full.

Consider the following example -

1) We have configured Bookies with two LedgerDirs - 'ledgerDir1',
'ledgerDir2'
2) lets say current active channel in EntryLogger is 'logChannel1' which is
created in 'ledgerDir1'
3) since 'entryLogFilePreallocationEnabled' is enabled by default, when we
set 'logChannel1' as the current active channel, we create async task to
create a new logChannel for future purpose (check
allocatorExecutor.submit(allocateNewLog task))
4) currently both the ledgerDirs are in writable state, so for precreation
of new logchannel it might pick any ledgerdir
5) lets say it chose 'ledgerDir1' again for the creation of new logchannel
- 'logChannel2'
6) now lets assume ledgerDir1 is full, in that case shouldCreateNewEntryLog
will be set to true
7) if a new entry is added to entrylogger, then since
shouldCreateNewEntryLog is set to true, it will close the current active
logchannel - 'logChannel1' and add it to logChannelsToFlush list.
8) For new logChannel it will get the preallocated log - 'logChannel2' and
will continue to use it as current active channel, though 'ledgerDir1' is
not writable anymore.

relevant code snippets in EntryLogger.java

private LedgerDirsListener getLedgerDirsListener() {
return new LedgerDirsListener() {
@Override
public void diskFull(File disk) {
// If the current entry log disk is full, then create new
entry
// log.
if (currentDir != null && currentDir.equals(disk)) {
shouldCreateNewEntryLog.set(true);
}
}

@Override
public void diskAlmostFull(File disk) {
// If the current entry log disk is almost full, then
create new entry
// log.
if (currentDir != null && currentDir.equals(disk)) {
shouldCreateNewEntryLog.set(true);
}
}

}

synchronized long addEntry(long ledger, ByteBuffer entry, boolean
rollLog) throws IOException {
.
boolean createNewLog = shouldCreateNewEntryLog.get();
if (createNewLog || reachEntryLogLimit) {
if (doRegularFlushes) {
flushCurrentLog();
}
createNewLog();
// Reset the flag
if (createNewLog) {
shouldCreateNewEntryLog.set(false);
}
}
.
}


synchronized BufferedLogChannel createNewLog() throws IOException {
BufferedLogChannel bc;
if (!entryLogPreAllocationEnabled || null == preallocation) {
// initialization time to create a new log
bc = allocateNewLog();
} else {
// has a preallocated entry log
try {
bc = preallocation.get();
...
preallocation = allocatorExecutor.submit(new
Callable() {
@Override
public BufferedLogChannel call() throws IOException {
return allocateNewLog();
}
});
}
LOG.info("Created new entry logger {}.", bc.getLogId());
return bc;
}

Thanks,
Charan


Re: Issue in GarbageCollectorThread compaction logic

2017-05-10 Thread Charan Reddy G
Hey Matteo,

I agree with what you said for JournalChannel

JournalChannel.preAllocIfNeeded

void preAllocIfNeeded(long size) throws IOException {
if (bc.position() + size > nextPrealloc) {
nextPrealloc += preAllocSize;
zeros.clear();
fc.write(zeros, nextPrealloc - journalAlignSize);
}
}

But the scenario in question is for EntryLogger.logChannel and
in GarbageCollectorThread.doCompactEntryLogs codepath. Here we just keep
adding entries to the EntryLogger.logChannel and flush it before deleting
the compacted entrylog.

Thanks,
Charan

On Wed, May 10, 2017 at 11:35 AM, Matteo Merli 
wrote:

> > On the other hand, a change to the file size (st_size, as made by say
> ftruncate(2)), would require a metadata flush."
>
> This is why the Journal is pre-allocating the blocks instead of just
> appending entries at the end of the file.
>
> That way the metadata is only synced each time you expand a block (eg:
> every 16 MB)
>
>
>
> On Wed, May 10, 2017 at 11:28 AM Charan Reddy G 
> wrote:
>
>>  @JV and @Sijie I think Android java source is not the appropriate one to
>> look into. For FileChannelImpl we need to look into jdk code
>>
>> http://grepcode.com/file/repository.grepcode.com/java/
>> root/jdk/openjdk/8u40-b25/sun/nio/ch/FileChannelImpl.java#
>> FileChannelImpl.force%28boolean%29
>>
>>
>> public void force(boolean metaData) throws IOException {
>> ensureOpen();
>> int rv = -1;
>> int ti = -1;
>> try {
>> begin();
>> ti = threads.add();
>> if (!isOpen())
>> return;
>> do {
>> rv = nd.force(fd, metaData);
>> } while ((rv == IOStatus.INTERRUPTED) && isOpen());
>> } finally {
>> threads.remove(ti);
>> end(rv > -1);
>> assert IOStatus.check(rv);
>> }
>> }
>>
>> here 'nd' is of type FileDispatcherImpl. I tried to look into its code
>> but since it is native code, we wont be able to read that
>>
>> http://grepcode.com/file/repository.grepcode.com/java/
>> root/jdk/openjdk/8u40-b25/sun/nio/ch/FileDispatcherImpl.
>> java#FileDispatcherImpl
>>
>> So it is not clear if it is just difference of fdatasync() and fsync().
>> Even if we assume that it is the difference of fdatasync() and fsync(),
>> man pages of fdatasync() says
>>
>> "fdatasync() is similar to fsync(), but does not flush modified metadata
>> unless that metadata is needed in order to allow a subsequent data
>> retrieval to be correctly handled.   For  example, changes  to  st_atime
>> or st_mtime (respectively, time of last access and time of last
>> modification; see stat(2)) do not require flushing because they are not
>> necessary for a subsequent data read to be handled correctly.  *On the
>> other hand, a change to the file size (st_size, as made by say
>> ftruncate(2)), would require a metadata flush.**"*
>>
>> Also, when we look into java API for force method -
>> https://docs.oracle.com/javase/7/docs/api/java/nio/
>> channels/FileChannel.html#force(boolean), it says the implementation is
>> unspecified and system-dependent. So I don't think it is safe to make any
>> assumptions here.
>>
>> "The metaData parameter can be used to limit the number of I/O
>> operations that this method is required to perform. Passing false for
>> this parameter indicates that only updates to the file's content need be
>> written to storage; passing true indicates that updates to both the
>> file's content and metadata must be written, which generally requires at
>> least one more I/O operation. Whether this parameter actually has any
>> effect is dependent upon the underlying operating system and is therefore
>> unspecified. Invoking this method may cause an I/O operation to occur
>> even if the channel was only opened for reading. Some operating systems,
>> for example, maintain a last-access time as part of a file's metadata, and
>> this time is updated whenever the file is read. Whether or not this is
>> actually done is system-dependent and is therefore unspecified."
>>
>> Thanks,
>> Charan
>>
>> On Wed, May 10, 2017 at 4:07 AM, Sijie Guo  wrote:
>>
>>>
>>>
>>> On May 10, 2017 12:42 PM, "Venkateswara Rao Jujjuri" 
>>> wrote:
>>>
>>> Looked at the source and it is fdatasync()
>>> https://android.googlesource.com/platform/

Re: Issue in GarbageCollectorThread compaction logic

2017-05-10 Thread Charan Reddy G
 @JV and @Sijie I think Android java source is not the appropriate one to
look into. For FileChannelImpl we need to look into jdk code

http://grepcode.com/file/repository.grepcode.com/java/root/jdk/openjdk/8u40-b25/sun/nio/ch/FileChannelImpl.java#FileChannelImpl.force%28boolean%29


public void force(boolean metaData) throws IOException {
ensureOpen();
int rv = -1;
int ti = -1;
try {
begin();
ti = threads.add();
if (!isOpen())
return;
do {
rv = nd.force(fd, metaData);
} while ((rv == IOStatus.INTERRUPTED) && isOpen());
} finally {
threads.remove(ti);
end(rv > -1);
assert IOStatus.check(rv);
}
}

here 'nd' is of type FileDispatcherImpl. I tried to look into its code but
since it is native code, we wont be able to read that

http://grepcode.com/file/repository.grepcode.com/java/root/jdk/openjdk/8u40-b25/sun/nio/ch/FileDispatcherImpl.java#FileDispatcherImpl

So it is not clear if it is just difference of fdatasync() and fsync().
Even if we assume that it is the difference of fdatasync() and fsync(), man
pages of fdatasync() says

"fdatasync() is similar to fsync(), but does not flush modified metadata
unless that metadata is needed in order to allow a subsequent data
retrieval to be correctly handled.   For  example, changes  to  st_atime or
st_mtime (respectively, time of last access and time of last modification;
see stat(2)) do not require flushing because they are not necessary for a
subsequent data read to be handled correctly.  *On the other hand, a change
to the file size (st_size, as made by say ftruncate(2)), would require a
metadata flush.**"*

Also, when we look into java API for force method -
https://docs.oracle.com/javase/7/docs/api/java/nio/channels/FileChannel.html#force(boolean),
it says the implementation is unspecified and system-dependent. So I don't
think it is safe to make any assumptions here.

"The metaData parameter can be used to limit the number of I/O operations
that this method is required to perform. Passing false for this parameter
indicates that only updates to the file's content need be written to
storage; passing true indicates that updates to both the file's content and
metadata must be written, which generally requires at least one more I/O
operation. Whether this parameter actually has any effect is dependent upon
the underlying operating system and is therefore unspecified. Invoking this
method may cause an I/O operation to occur even if the channel was only
opened for reading. Some operating systems, for example, maintain a
last-access time as part of a file's metadata, and this time is updated
whenever the file is read. Whether or not this is actually done is
system-dependent and is therefore unspecified."

Thanks,
Charan

On Wed, May 10, 2017 at 4:07 AM, Sijie Guo  wrote:

>
>
> On May 10, 2017 12:42 PM, "Venkateswara Rao Jujjuri" 
> wrote:
>
> Looked at the source and it is fdatasync()
> https://android.googlesource.com/platform/libcore/+/2496a68/
> luni/src/main/java/java/nio/FileChannelImpl.java
> if (metadata) {
> Libcore.os.fsync(fd);
> } else {
>  Libcore.os.fdatasync(fd);
> }
>
> So, agreed. fdatasync() fluesh metadata when the file size etc changes,
> and avoids metadata flush if no data related changes but
> but that tells me there may be no situation for us to call force(true) in
> the code.
> But our code calls it with true in some code path, should we change
> everything to false for consistency
>
>
> I don't remember that we called force(true) on file channel. Do you mean
> the buffered channel?
>
>
> JV
>
> On Tue, May 9, 2017 at 4:40 PM, Sijie Guo  wrote:
>
>> We don't force flush metadata in bookies. Because:
>>
>> - flush metadata requires more I/O operations.
>> - flush metadata usually updates file metadata information (e.g. access
>> permissions, last modification/access time).
>>
>> The difference of force(false) and force(true) is the difference between
>> fdatasync() and fsync(). The former one update last modification time,
>> while the latter one doesn't. We don't care about the metadata information
>> like access/modification time, that's the reason we only called
>> force(false).
>>
>> - Sijie
>>
>>
>> On Sat, May 6, 2017 at 9:10 AM, Charan Reddy G 
>> wrote:
>>
>>> By metadata I mean metadata of file
>>>
>>> in BufferedChannel.java
>>>
>>> public void flush(boolean shouldForceWrite) throws IOException {
>>> synchronized(this) {
>>> flushInternal();
>>> }
>&g

Question regarding synchronization in EntryLogger

2017-05-05 Thread Charan Reddy G
Hey,

In EntryLogger, I'm wondering what is the need of making flushCurrentLog
method synchronized? (if we make logChannel variable volatile or of type
AtomicReference and bytesWrittenSinceLastFlush of type AtomicLong). Anyhow
in BufferedChannel.flush, flushInternal() is called from synchrnoized block
of BufferedChannel.

EntryLogger.flushCurrentLog
synchronized void flushCurrentLog() throws IOException {
if (logChannel != null) {
logChannel.flush(true);
bytesWrittenSinceLastFlush = 0;
LOG.debug("Flush and sync current entry logger {}.",
logChannel.getLogId());
}
}

BufferedChannel.flush
public void flush(boolean shouldForceWrite) throws IOException {
synchronized(this) {
flushInternal();
}
if (shouldForceWrite) {
forceWrite(false);
}
}


I'm currently working on "(BOOKKEEPER-1041) Multiple active entrylogs". For
this feature I need to make changes to checkpoint logic. Currently with
BOOKKEEPER-564 change, we are scheduling checkpoint only when current
entrylog file is rotated. So we dont call 'flushCurrentLog' when we
checkpoint. But for BOOKKEEPER-1041 feature, since there are going to be
multiple active entrylogs, scheduling checkpoint when entrylog file is
rotated, is not an option. So I need to call flushCurrentLogs when
checkpoint is made for every 'flushinterval' period. Here it would be
optimal if flushCurrentLog is not in synchronized block, also I don't see a
reason for why it has to be in synchrnoized block to begin with. In
BufferedChannel.flush method 'forceWrite' method (which is not in
synchronized block of BufferedChannel) is gonna take considerable time,
since it flush/forceWrites file to disk. So if EntryLogger's lock is not
held during the forceWrite of BufferedChannel, then progress can be made in
EntryLogger.addEntry.

JV and I had gone through this code mutiple times and we couldn't find a
reason for it to be synchrnoized. Please let us know if we are missing
something here.

Thanks,
Charan


Re: Issue in GarbageCollectorThread compaction logic

2017-05-05 Thread Charan Reddy G
By metadata I mean metadata of file

in BufferedChannel.java

public void flush(boolean shouldForceWrite) throws IOException {
synchronized(this) {
flushInternal();
}
if (shouldForceWrite) {
forceWrite(false);  <- false here
}
}

public long forceWrite(boolean forceMetadata) throws IOException {
// This is the point up to which we had flushed to the file system
page cache
// before issuing this force write hence is guaranteed to be made
durable by
// the force write, any flush that happens after this may or may
// not be flushed
long positionForceWrite = writeBufferStartPosition.get();
fileChannel.force(forceMetadata);<--- forcemetadata
false
return positionForceWrite;
}

On Fri, May 5, 2017 at 5:56 PM, Charan Reddy G 
wrote:

> Hey,
>
> In GarbageCollectorThread.doCompactEntryLogs {
>
> ...
>
> try {
> compactEntryLog(scannerFactory, meta);
> scannerFactory.flush();<--- this
> will eventually call entrylogger.flushCurrentLog and it force writes the
> content of the BufferedChannel but not the metadata of the file
>
> LOG.info("Removing entry log {} after compaction",
> meta.getEntryLogId());
> removeEntryLog(meta.getEntryLogId());  <--- this
> will delete the compacted entrylog
>
> }
> ...
>
>
> in doCompactEntryLogs, we first write the non-deleted entries of the
> entryLog which is getting compacted to the currentLog in entryLogger, then
> we flush the entrylogger before deleting the compacted entrylog. But when
> currentLog is flushed by the entrylogger, it flushes only the content of
> the file but not the metadata. After flush is completed the compacted
> entrylog is deleted. It is not ok to not to force flush the metadata of the
> currentLog for the persisted (checkpointed) data. The filesystem behaviour
> is unexpected in this case and there is possibility of data loss if the
> Bookie crashes before closing that logchannel.
>
> Thanks,
> Charan
>


Issue in GarbageCollectorThread compaction logic

2017-05-05 Thread Charan Reddy G
Hey,

In GarbageCollectorThread.doCompactEntryLogs {

...

try {
compactEntryLog(scannerFactory, meta);
scannerFactory.flush();<--- this
will eventually call entrylogger.flushCurrentLog and it force writes the
content of the BufferedChannel but not the metadata of the file

LOG.info("Removing entry log {} after compaction",
meta.getEntryLogId());
removeEntryLog(meta.getEntryLogId());  <--- this
will delete the compacted entrylog

}
...


in doCompactEntryLogs, we first write the non-deleted entries of the
entryLog which is getting compacted to the currentLog in entryLogger, then
we flush the entrylogger before deleting the compacted entrylog. But when
currentLog is flushed by the entrylogger, it flushes only the content of
the file but not the metadata. After flush is completed the compacted
entrylog is deleted. It is not ok to not to force flush the metadata of the
currentLog for the persisted (checkpointed) data. The filesystem behaviour
is unexpected in this case and there is possibility of data loss if the
Bookie crashes before closing that logchannel.

Thanks,
Charan


Re: Issue with entryLogger.leastUnflushedLogId, when preallocatedLogId rolls over Int.MAX

2017-05-02 Thread Charan Reddy G
"The question is - is this really an issue? When roll over happen, it might
cause an entry log file won't be garbage collected or compacted in this
lifecycle of the bookie. But it doesn't actually impact correctness, right?"

Yes it is not correctness issue. But the chances of those entrylogs getting
garbage collected are very slim (even after Bookie restart).
extractMetaFromEntryLogs
will extract metadata from those entrylogs next time when
entryLogger.getLeastUnflushedLogId()
reaches close to Int.MAX but not rolled over again.

Thanks,
Charan


On Fri, Apr 28, 2017 at 12:11 AM, Enrico Olivelli 
wrote:

> Il ven 28 apr 2017, 09:05 Sijie Guo  ha scritto:
>
> > Hi Enrico,
> >
> > Let's try to figure things out in the email thread before create a JIRA.
> >
>
> Got it, sorry
> Enrico
>
> >
> > - Sijie
> >
> > On Thu, Apr 27, 2017 at 11:43 PM, Enrico Olivelli 
> > wrote:
> >
> >> Thank you Charan,
> >> Can you create a JIRA?
> >> Do you already have a fix?
> >>
> >> Il ven 28 apr 2017, 00:27 Charan Reddy G  ha
> >> scritto:
> >>
> >>> Hey,
> >>>
> >>> With Bookkeeper-833 Bug (
> >>>
> >>> https://github.com/apache/bookkeeper/commit/
> da1a2fa6b19ddcdba68834147bf6afbe5bf90cbf
> >>> ),
> >>> entryLogId in EntryLogger is capped at Int.MAX, so preallocatedLogId
> >>> rolls
> >>> over to 0 after reaching Int.MAX. In EntryLogger.flushRotatedLogs we
> set
> >>> "leastUnflushedLogId = flushedLogId + 1;", so it makes
> >>> leastUnflushedLogId
> >>> also to roll over. But this affects the
> >>> GarbageCollectorThread.extractMetaFromEntryLogs logic. This method
> >>> extracts
> >>> EntryLogMetadata from the newly rotated entrylogs, but when rollover
> >>> happens this logic seems to be broken
> >>>
> >>> in GarbageCollectorThread.java
> >>>
> >>> protected Map
> >>> extractMetaFromEntryLogs(Map entryLogMetaMap)
> {
> >>> // Extract it for every entry log except for the current one.
> >>> // Entry Log ID's are just a long value that starts at 0 and
> >>> increments
> >>> // by 1 when the log fills up and we roll to a new one.
> >>> long curLogId = entryLogger.getLeastUnflushedLogId();
> >>><--when rollover happens, this will start from 1
> >>> boolean hasExceptionWhenScan = false;
> >>> for (long entryLogId = scannedLogId; entryLogId < curLogId;
> >>> entryLogId++) {<--- because of "entryLogId < curLogId"
> condition
> >>> it
> >>> will skip the newly rotated logs
> >>>
> >>> Thanks,
> >>> Charan
> >>>
> >> --
> >>
> >>
> >> -- Enrico Olivelli
> >>
> >
> > --
>
>
> -- Enrico Olivelli
>


Issue with entryLogger.leastUnflushedLogId, when preallocatedLogId rolls over Int.MAX

2017-04-27 Thread Charan Reddy G
Hey,

With Bookkeeper-833 Bug (
https://github.com/apache/bookkeeper/commit/da1a2fa6b19ddcdba68834147bf6afbe5bf90cbf),
entryLogId in EntryLogger is capped at Int.MAX, so preallocatedLogId rolls
over to 0 after reaching Int.MAX. In EntryLogger.flushRotatedLogs we set
"leastUnflushedLogId = flushedLogId + 1;", so it makes leastUnflushedLogId
also to roll over. But this affects the
GarbageCollectorThread.extractMetaFromEntryLogs logic. This method extracts
EntryLogMetadata from the newly rotated entrylogs, but when rollover
happens this logic seems to be broken

in GarbageCollectorThread.java

protected Map
extractMetaFromEntryLogs(Map entryLogMetaMap) {
// Extract it for every entry log except for the current one.
// Entry Log ID's are just a long value that starts at 0 and
increments
// by 1 when the log fills up and we roll to a new one.
long curLogId = entryLogger.getLeastUnflushedLogId();
   <--when rollover happens, this will start from 1
boolean hasExceptionWhenScan = false;
for (long entryLogId = scannedLogId; entryLogId < curLogId;
entryLogId++) {<--- because of "entryLogId < curLogId" condition it
will skip the newly rotated logs

Thanks,
Charan


Re: Question regarding Checkpoint/Flush codePath

2017-04-21 Thread Charan Reddy G
+Sijie, Matteo, Enrico

On Thu, Apr 20, 2017 at 10:55 PM, Charan Reddy G 
wrote:

> Hey,
>
> SyncThread.checkpoint(Checkpoint checkpoint) (which is called
> periodically by SyncThread's executor for every flushInterval) ultimately
> calls EntryLogger.flushRotatedLogs.
>
> In EntryLogger.flushRotatedLogs, first we set 'logChannelsToFlush' to null
> and then we try to flush and close individual file. Now, if IOException
> happens while trying to flush/close the logchannel, then exception is
> thrown as it is and it get propagates back upto SyncThread.checkpoint. Here
> we catch that IOException, log it and return without calling the
> checkpointComplete. But by now we lost reference of 'logChannelsToFlush'
> (rolled logs which are yet to be closed), because it is set to null before
> we try to flush/close individually rolledlogs. The next execution of
> 'checkpoint' (after flushinterval) wouldn't be knowing about the rolledlogs
> it failed to flush/close the previous time and it would flush the newly
> rolledlogs. So the failure of flush/close of the previous rolledlogs goes
> unnoticed completely.
>
> in EntryLogger.java
> void flushRotatedLogs() throws IOException {
> List channels = null;
> long flushedLogId = INVALID_LID;
> synchronized (this) {
> channels = logChannelsToFlush;
> logChannelsToFlush = null;   <- here we
> set 'logChannelsToFlush' to null before it tries to flush/close rolledlogs
> }
> if (null == channels) {
> return;
> }
> for (BufferedLogChannel channel : channels) {
> channel.flush(true);
>  <IOEXception can happen here or in the following
> closeFileChannel call
> // since this channel is only used for writing, after flushing
> the channel,
> // we had to close the underlying file channel. Otherwise, we
> might end up
> // leaking fds which cause the disk spaces could not be
> reclaimed.
> closeFileChannel(channel);
> if (channel.getLogId() > flushedLogId) {
> flushedLogId = channel.getLogId();
> }
> LOG.info("Synced entry logger {} to disk.",
> channel.getLogId());
> }
> // move the leastUnflushedLogId ptr
> leastUnflushedLogId = flushedLogId + 1;
> }
>
> in SyncThread.java
> public void checkpoint(Checkpoint checkpoint) {
> try {
> checkpoint = ledgerStorage.checkpoint(checkpoint);
> } catch (NoWritableLedgerDirException e) {
> LOG.error("No writeable ledger directories", e);
> dirsListener.allDisksFull();
> return;
> } catch (IOException e) {
> LOG.error("Exception flushing ledgers", e); <-that IOExc
> gets propagated to this method and here it is caught and not dealt
> appropriately
> return;
> }
>
> try {
> checkpointSource.checkpointComplete(checkpoint, true);
> } catch (IOException e) {
> LOG.error("Exception marking checkpoint as complete", e);
> dirsListener.allDisksFull();
> }
> }
>
> Thanks,
> Charan
>


Question regarding Checkpoint/Flush codePath

2017-04-20 Thread Charan Reddy G
Hey,

SyncThread.checkpoint(Checkpoint checkpoint) (which is called periodically
by SyncThread's executor for every flushInterval) ultimately calls
EntryLogger.flushRotatedLogs.

In EntryLogger.flushRotatedLogs, first we set 'logChannelsToFlush' to null
and then we try to flush and close individual file. Now, if IOException
happens while trying to flush/close the logchannel, then exception is
thrown as it is and it get propagates back upto SyncThread.checkpoint. Here
we catch that IOException, log it and return without calling the
checkpointComplete. But by now we lost reference of 'logChannelsToFlush'
(rolled logs which are yet to be closed), because it is set to null before
we try to flush/close individually rolledlogs. The next execution of
'checkpoint' (after flushinterval) wouldn't be knowing about the rolledlogs
it failed to flush/close the previous time and it would flush the newly
rolledlogs. So the failure of flush/close of the previous rolledlogs goes
unnoticed completely.

in EntryLogger.java
void flushRotatedLogs() throws IOException {
List channels = null;
long flushedLogId = INVALID_LID;
synchronized (this) {
channels = logChannelsToFlush;
logChannelsToFlush = null;   <- here we set
'logChannelsToFlush' to null before it tries to flush/close rolledlogs
}
if (null == channels) {
return;
}
for (BufferedLogChannel channel : channels) {
channel.flush(true);
  flushedLogId) {
flushedLogId = channel.getLogId();
}
LOG.info("Synced entry logger {} to disk.", channel.getLogId());
}
// move the leastUnflushedLogId ptr
leastUnflushedLogId = flushedLogId + 1;
}

in SyncThread.java
public void checkpoint(Checkpoint checkpoint) {
try {
checkpoint = ledgerStorage.checkpoint(checkpoint);
} catch (NoWritableLedgerDirException e) {
LOG.error("No writeable ledger directories", e);
dirsListener.allDisksFull();
return;
} catch (IOException e) {
LOG.error("Exception flushing ledgers", e); <-that IOExc
gets propagated to this method and here it is caught and not dealt
appropriately
return;
}

try {
checkpointSource.checkpointComplete(checkpoint, true);
} catch (IOException e) {
LOG.error("Exception marking checkpoint as complete", e);
dirsListener.allDisksFull();
}
}

Thanks,
Charan


Re: [ANNOUNCE] New BookKeeper Committer: Enrico Olivelli & Charan Reddy

2017-04-11 Thread Charan Reddy G
Congrats Enrico!

Thank You Sijie and JV.

On Tue, Apr 11, 2017 at 11:19 AM, Enrico Olivelli 
wrote:

> Thank you!
> Congrats Charan!
>
> Il mar 11 apr 2017, 18:01 Sijie Guo  ha scritto:
>
> > The Apache BookKeeper PMC recently extended committer karma to Enrico
> > Olivelli & Charan Reddy and they have accepted.
> >
> > Charan Reddy and Enrico Olivelli have been contributing bookkeeper
> changes
> > to the community for a while. They are active in code contributions,
> > discussion and code reviews. We are looking forward to more contributions
> > from them.
> >
> > Congratulations and welcome onboard, Enrico Olivelli & Charan Reddy!
> >
> > Sijie on behave of the BookKeeper PMC
> >
> --
>
>
> -- Enrico Olivelli
>


Re: Issues in ZK event handling logic for BookiesChanged event

2017-03-07 Thread Charan Reddy G
*Issue 1:*

*Matteo's Response:*

*"The  way I understand this is that you might get less notifications than*







*the number of changes on the z-node, but when you set the watch, it
shouldbe tied to the z-node version that you have already read, so you
should beget notified for any change after that.Resuming, you might not see
all intermediate states, but you shouldguaranteed to converge to final
state.Probably Flavio can give more insight on this."*

>From what I understood and my local testing, it would trigger watcher only
for the change which happened after setting the watcher but not for the
changes which happen after the last read state. So my question is, aren't
we missing the change (if any happened) during the transition period (after
receiving the watcher event and before setting the next watcher). Need ZK
experts here.

*Issue 2:*

*Sijie's Response:*





*"If the zookeeper client is configured with retries, the getChildren will
beretried with transient failures before throwing any exceptions. I
believethe zookeeper client at bookie server is configured with infinite
retriesby default, no?"*

I checked the code, in AutoRecoveryMain(ServerConfiguration conf,
StatsLogger statsLogger) constructor, RetryPolicy policy is not set. If no
retry policy is set, "connectRetryPolicy" of ZooKeeperClient is set
to BoundExponentialBackoffRetryPolicy with Integer.MAX_VALUE retries and
"operationRetryPolicy" of ZooKeeperClient is set to
BoundExponentialBackoffRetryPolicy
with 0 retries. I'm not sure how each one of them works. Have to look into
the details.

Thanks,
Charan

On Tue, Mar 7, 2017 at 10:51 AM, Sijie Guo  wrote:

> On Tue, Mar 7, 2017 at 10:10 AM, Charan Reddy G 
> wrote:
>
> > Hey,
> >
> > *Issue 1:*
> >
> > I was going through the code to understand how BookiesChanged event is
> > handled for AuditTask (in Auditor, BookkeeperAdmin and BookieWatcher
> > classes). In BookieWatcher.notifyBookiesChanged they are making
> > getChildren
> > call on "bookieRegistrationPath" and setting watcher (which listens for
> > NodeChildrenChanged event). Now as we know, Zookeeper watchers are one
> time
> > triggers, and hence in our code on consuming the watch event we reset
> > another watch to get notified of future changes.
> >
> > But due to the limitation of ZK (which is well documented thing in ZK
> > https://zookeeper.apache.org/doc/r3.4.5/zookeeperProgrammers.html#ch_
> > zkWatches), "Because watches are one time triggers and there is latency
> > between getting the event and sending a new request to get a watch you
> > cannot reliably see every change that happens to a node in ZooKeeper. Be
> > prepared to handle the case where the znode changes multiple times
> between
> > getting the event and setting the watch again."
> >
> > So now in the implementation - (Auditor.notifyBookiesChanged,
> > Auditor.availableBookiesChanged,
> > BookKeeperAdmin.notifyBookiesChanged and BookieWatcher.
> > notifyBookiesChanged)
> > there is a possibility that if multiple bookies go off at a time, we
> might
> > miss handling BookiesChanged event in Auditor, which is a serious risk.
> >
> > Disussed with my teammember Kishore briefly, that to overcome this issue
> we
> > should compare return value of getChildren("bookieRegistrationPath") (in
> > BookieWatcher.notifyBookiesChanged) with previous known state to
> recognize
> > the changes, if any happenend, during the transition period.
> >
> > Am I missing something here? otherwise I think it is a serious flaw in
> the
> > implmentation.
> >
> > *Issue 2:* Also in Auditor class, if any exception happens in
> >  notifyBookieChanges(), then we are just logging error and proceeding.
> > Which basically means that we may not successfully able to set watcher
> but
> > we are still proceeding. In this case aren't we missing BookiesChanged
> > event completely from then on?
> >
>
> If the zookeeper client is configured with retries, the getChildren will be
> retried with transient failures before throwing any exceptions. I believe
> the zookeeper client at bookie server is configured with infinite retries
> by default, no?
>
> >
> > @Override
> > public void availableBookiesChanged() {
> > // since a watch is triggered, we need to watch again on the
> > bookies
> > try {
> > notifyBookieChanges();
> > } *catch (BKException bke) {*
> > *LOG.error("Exception while registering for a bookie change
> > notification", bke);*
> > *}*
> > submitAuditTask();
> > }
> >
> > Thanks,
> > Charan
> >
>


Issues in ZK event handling logic for BookiesChanged event

2017-03-07 Thread Charan Reddy G
Hey,

*Issue 1:*

I was going through the code to understand how BookiesChanged event is
handled for AuditTask (in Auditor, BookkeeperAdmin and BookieWatcher
classes). In BookieWatcher.notifyBookiesChanged they are making getChildren
call on "bookieRegistrationPath" and setting watcher (which listens for
NodeChildrenChanged event). Now as we know, Zookeeper watchers are one time
triggers, and hence in our code on consuming the watch event we reset
another watch to get notified of future changes.

But due to the limitation of ZK (which is well documented thing in ZK
https://zookeeper.apache.org/doc/r3.4.5/zookeeperProgrammers.html#ch_
zkWatches), "Because watches are one time triggers and there is latency
between getting the event and sending a new request to get a watch you
cannot reliably see every change that happens to a node in ZooKeeper. Be
prepared to handle the case where the znode changes multiple times between
getting the event and setting the watch again."

So now in the implementation - (Auditor.notifyBookiesChanged,
Auditor.availableBookiesChanged,
BookKeeperAdmin.notifyBookiesChanged and BookieWatcher.notifyBookiesChanged)
there is a possibility that if multiple bookies go off at a time, we might
miss handling BookiesChanged event in Auditor, which is a serious risk.

Disussed with my teammember Kishore briefly, that to overcome this issue we
should compare return value of getChildren("bookieRegistrationPath") (in
BookieWatcher.notifyBookiesChanged) with previous known state to recognize
the changes, if any happenend, during the transition period.

Am I missing something here? otherwise I think it is a serious flaw in the
implmentation.

*Issue 2:* Also in Auditor class, if any exception happens in
 notifyBookieChanges(), then we are just logging error and proceeding.
Which basically means that we may not successfully able to set watcher but
we are still proceeding. In this case aren't we missing BookiesChanged
event completely from then on?

@Override
public void availableBookiesChanged() {
// since a watch is triggered, we need to watch again on the bookies
try {
notifyBookieChanges();
} *catch (BKException bke) {*
*LOG.error("Exception while registering for a bookie change
notification", bke);*
*}*
submitAuditTask();
}

Thanks,
Charan


Re: Question regarding force compaction/GC logic in GarbageCollectorThread.java

2017-02-07 Thread Charan Reddy G
Thanks Sijie, for the clarification. So going by Twitter's code, you intend
to run force Major/Minor compaction only once when enableForceGC is called.
Can you please confirm that (which is different from current community code
behavior. Not sure if it was bug in the current community code or it was
intended to be like that.) Also can you please confirm why it is having
different behavior in current community code?

Twitter code has diverged quite a bit from community code, any plans of
merging changes to the community?

Thanks,
Charan

On Tue, Feb 7, 2017 at 10:27 PM, Sijie Guo  wrote:

> The forceGarbageCollection was contributed by Twitter before. I remembered
> we wrapped that into a try-finally block. I just checked Twitter's branch.
> We did wrap that in a try-finally block. https://github.com/
> twitter/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/
> bookkeeper/bookie/GarbageCollectorThread.java#L563
>
> I need to merge that back.
>
> - Sijie
>
> On Tue, Feb 7, 2017 at 10:22 PM, Charan Reddy G 
> wrote:
>
>> Hey Sijie,
>>
>> Thanks for the response, but didn't get which finally block you are
>> referring to. Are you saying that before executing 'continue' statement in
>> major compaction 'if' block,  forceGarbageCollection should be set to
>> false, so that force Major/Minor compaction would be run only once when
>> enableForceGC is called?
>>
>> Thanks,
>> Charan
>>
>> On Feb 7, 2017 10:08 PM, "Sijie Guo"  wrote:
>>
>>> I think forceGarabageCollection should be set in a final block. Can't
>>> remember why it wasn't in current master branch.
>>>
>>> - Sijie
>>>
>>> On Tue, Feb 7, 2017 at 7:01 PM, Charan Reddy G 
>>> wrote:
>>>
>>>> Hi,
>>>>
>>>> I'm trying to understand the reason behind "continue;" statement in
>>>> line 352 of GarbageCollectorThread.java (https://github.com/apache/boo
>>>> kkeeper/blob/master/bookkeeper-server/src/main/java/org/apac
>>>> he/bookkeeper/bookie/GarbageCollectorThread.java). It is
>>>> understandable that if we have done majorcompaction then minor compaction
>>>> is not required and it can be skipped, but aren't we missing "
>>>> forceGarbageCollection.set(false);" in line 362? Or is it supposed to
>>>> be like that?
>>>>
>>>> Basically I'm trying to findout when enableForceGC() is called, is it
>>>> supposed to trigger and do force GC/Compaction just once or untill it gets
>>>> disabled by calling disableForceGC()?
>>>>
>>>> Thanks,
>>>> Charan
>>>>
>>>
>>>
>


Re: Question regarding force compaction/GC logic in GarbageCollectorThread.java

2017-02-07 Thread Charan Reddy G
Hey Sijie,

Thanks for the response, but didn't get which finally block you are
referring to. Are you saying that before executing 'continue' statement in
major compaction 'if' block,  forceGarbageCollection should be set to
false, so that force Major/Minor compaction would be run only once when
enableForceGC is called?

Thanks,
Charan

On Feb 7, 2017 10:08 PM, "Sijie Guo"  wrote:

> I think forceGarabageCollection should be set in a final block. Can't
> remember why it wasn't in current master branch.
>
> - Sijie
>
> On Tue, Feb 7, 2017 at 7:01 PM, Charan Reddy G 
> wrote:
>
>> Hi,
>>
>> I'm trying to understand the reason behind "continue;" statement in line
>> 352 of GarbageCollectorThread.java (https://github.com/apache/boo
>> kkeeper/blob/master/bookkeeper-server/src/main/java/org/
>> apache/bookkeeper/bookie/GarbageCollectorThread.java). It is
>> understandable that if we have done majorcompaction then minor compaction
>> is not required and it can be skipped, but aren't we missing "
>> forceGarbageCollection.set(false);" in line 362? Or is it supposed to be
>> like that?
>>
>> Basically I'm trying to findout when enableForceGC() is called, is it
>> supposed to trigger and do force GC/Compaction just once or untill it gets
>> disabled by calling disableForceGC()?
>>
>> Thanks,
>> Charan
>>
>
>


Question regarding force compaction/GC logic in GarbageCollectorThread.java

2017-02-07 Thread Charan Reddy G
Hi,

I'm trying to understand the reason behind "continue;" statement in line
352 of GarbageCollectorThread.java (
https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java).
It is understandable that if we have done majorcompaction then minor
compaction is not required and it can be skipped, but aren't we missing "
forceGarbageCollection.set(false);" in line 362? Or is it supposed to be
like that?

Basically I'm trying to findout when enableForceGC() is called, is it
supposed to trigger and do force GC/Compaction just once or untill it gets
disabled by calling disableForceGC()?

Thanks,
Charan


Re: [jira] [Commented] (BOOKKEEPER-913) Fix flakiness in TestBackwardCompat

2017-01-30 Thread Charan Reddy G
Hey JV,

I don't see option to change the status (reopen) of the closed bug.

Thanks,
Charan

On Sat, Jan 28, 2017 at 11:27 AM, Venkateswara Rao Jujjuri <
jujj...@gmail.com> wrote:

> Please go ahead and reopen.
>
> On Fri, Jan 27, 2017 at 10:25 AM, Charan Reddy Guttapalem (JIRA) <
> j...@apache.org> wrote:
>
> >
> > [ https://issues.apache.org/jira/browse/BOOKKEEPER-913?
> > page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&
> > focusedCommentId=15843294#comment-15843294 ]
> >
> > Charan Reddy Guttapalem commented on BOOKKEEPER-913:
> > 
> >
> > I think this bug needs to be reopened.
> >
> > > Fix flakiness in TestBackwardCompat
> > > ---
> > >
> > > Key: BOOKKEEPER-913
> > > URL: https://issues.apache.org/
> > jira/browse/BOOKKEEPER-913
> > > Project: Bookkeeper
> > >  Issue Type: Bug
> > >Reporter: Matteo Merli
> > >Assignee: Matteo Merli
> > >Priority: Trivial
> > > Fix For: 4.4.0
> > >
> > >
> > > This test is intermittently failing. It could be related to a timing
> > issue, given that it fails more frequently in Jenkins compared to running
> > the test on the laptop.
> > > {noformat}
> > > Failed tests:   testCompat410(org.apache.bookkeeper.test.
> TestBackwardCompat):
> > Shouldn't be able to write
> > > {noformat}
> >
> >
> >
> > --
> > This message was sent by Atlassian JIRA
> > (v6.3.4#6332)
> >
>
>
>
> --
> Jvrao
> ---
> First they ignore you, then they laugh at you, then they fight you, then
> you win. - Mahatma Gandhi
>


Re: Build failed in Jenkins: bookkeeper-master #1638

2017-01-26 Thread Charan Reddy G
This failure (java warning made to build error) is because of my recent
commit -
https://github.com/apache/bookkeeper/commit/0fbb98b3629df18518ab6b2298bdd6fe5f19af7e

I fixed it in one of my pending pull request -
https://github.com/apache/bookkeeper/pull/99

@Sijie - this is the findbugs error you are referring to, I fixed it in the
pending pull request.

Thanks,
Charan

On Wed, Jan 25, 2017 at 5:11 AM, Apache Jenkins Server <
jenk...@builds.apache.org> wrote:

> See 
>
> --
> [...truncated 452 lines...]
> Tests run: 12, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 3.735 sec
> - in org.apache.bookkeeper.client.BookieWriteLedgersWithDifferen
> tDigestsTest
> Running org.apache.bookkeeper.client.LocalBookKeeperTest
> Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 11.911 sec
> - in org.apache.bookkeeper.client.LocalBookKeeperTest
> Running org.apache.bookkeeper.client.TestFencing
> Tests run: 14, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 14.484
> sec - in org.apache.bookkeeper.client.TestFencing
> Running org.apache.bookkeeper.client.TestLedgerChecker
> Tests run: 11, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 4.937 sec
> - in org.apache.bookkeeper.client.TestLedgerChecker
> Running org.apache.bookkeeper.client.BookieWriteLedgerTest
> Tests run: 60, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 21.078
> sec - in org.apache.bookkeeper.client.BookieWriteLedgerTest
> Running org.apache.bookkeeper.client.UpdateLedgerCmdTest
> Tests run: 1, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 9.381 sec
> - in org.apache.bookkeeper.client.UpdateLedgerCmdTest
> Running org.apache.bookkeeper.client.BookieRecoveryTest
> Tests run: 72, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 34.598
> sec - in org.apache.bookkeeper.client.BookieRecoveryTest
> Running org.apache.bookkeeper.client.LedgerRecoveryTest
> Tests run: 18, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 7.073 sec
> - in org.apache.bookkeeper.client.LedgerRecoveryTest
> Running org.apache.bookkeeper.client.TestReadTimeout
> Tests run: 1, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 16.586 sec
> - in org.apache.bookkeeper.client.TestReadTimeout
> Running org.apache.bookkeeper.client.TestBookieWatcher
> Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 15.695 sec
> - in org.apache.bookkeeper.client.TestBookieWatcher
> Running org.apache.bookkeeper.client.RoundRobinDistributionScheduleTest
> Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.171 sec
> - in org.apache.bookkeeper.client.RoundRobinDistributionScheduleTest
> Running org.apache.bookkeeper.client.TestTryReadLastConfirmed
> Tests run: 3, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 8.766 sec
> - in org.apache.bookkeeper.client.TestTryReadLastConfirmed
> Running org.apache.bookkeeper.client.TestBookieHealthCheck
> Tests run: 3, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 16.001 sec
> - in org.apache.bookkeeper.client.TestBookieHealthCheck
> Running org.apache.bookkeeper.client.ListLedgersTest
> Tests run: 8, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 3.104 sec
> - in org.apache.bookkeeper.client.ListLedgersTest
> Running org.apache.bookkeeper.client.BookKeeperTest
> Tests run: 16, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 37.858
> sec - in org.apache.bookkeeper.client.BookKeeperTest
> Running org.apache.bookkeeper.client.TestLedgerFragmentReplication
> Tests run: 4, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 2.286 sec
> - in org.apache.bookkeeper.client.TestLedgerFragmentReplication
> Running org.apache.bookkeeper.client.TestRackawareEnsemblePlacementPolicy
> Tests run: 13, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 5.829 sec
> - in org.apache.bookkeeper.client.TestRackawareEnsemblePlacementPolicy
> Running org.apache.bookkeeper.client.TestRackawareEnsemblePlacement
> PolicyUsingScript
> Tests run: 7, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.542 sec
> - in org.apache.bookkeeper.client.TestRackawareEnsemblePlacement
> PolicyUsingScript
> Running org.apache.bookkeeper.client.LedgerCloseTest
> Tests run: 4, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 20.094 sec
> - in org.apache.bookkeeper.client.LedgerCloseTest
> Running org.apache.bookkeeper.client.SlowBookieTest
> Tests run: 3, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 25.879 sec
> - in org.apache.bookkeeper.client.SlowBookieTest
> Running org.apache.bookkeeper.client.TestWatchEnsembleChange
> Tests run: 6, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 18.967 sec
> - in org.apache.bookkeeper.client.TestWatchEnsembleChange
> Running org.apache.bookkeeper.client.BookKeeperCloseTest
> Tests run: 10, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 5.355 sec
> - in org.apache.bookkeeper.client.BookKeeperCloseTest
> Running org.apache.bookkeeper.client.TestRegionAwareEnsemblePlaceme
> ntPolicy
> Tests run: 30, Failur

Build failure because of Java 8 features in BOOKKEEPER-964 commit

2016-12-12 Thread Charan Reddy G
Hey Matteo,

I think in this commit - BOOKKEEPER-964
https://github.com/apache/bookkeeper/commit/ecbb053e6e873859507e247cae727f4bc8b9f7fa
, you brought Java 8 features and API but the older versions of Maven shade
plugin (2.1) doesn't support Java 8 and it would cause maven build failure
with the following exception. For more details check
https://jira.nuxeo.com/browse/NXP-18655. In Bookkeeper-Server pom.xml you
need to upgrade maven-shade-plugin version from 2.1 to 2.4.2


[ERROR] Failed to execute goal
org.apache.maven.plugins:maven-shade-plugin:2.1:shade (default) on project
bookkeeper-server: Error creating shaded jar: 52264 -> [Help 1]
org.apache.maven.lifecycle.LifecycleExecutionException: Failed to execute
goal org.apache.maven.plugins:maven-shade-plugin:2.1:shade (default) on
project bookkeeper-server: Error creating shaded jar: 52264
at
org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:217)
at
org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:153)
at
org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:145)
at
org.apache.maven.lifecycle.internal.LifecycleModuleBuilder.buildProject(LifecycleModuleBuilder.java:84)


Thanks,
Charan


BookieClientTest.testWriteGaps testcase is failing consistently since BOOKKEEPER-965 ( Long Polling Part I) commit

2016-11-29 Thread Charan Reddy G
Hey,

BookieClientTest.testWriteGaps testcase is failing consistently since
BOOKKEEPER-965 commit (
https://github.com/apache/bookkeeper/commit/9359d682a1598e30364eca6021d976f911e055b2)
. Is anyone looking into this issue

Tests run: 2, Failures: 1, Errors: 0, Skipped: 0, Time elapsed: 3.596 sec
<<< FAILURE! - in org.apache.bookkeeper.test.BookieClientTest
testWriteGaps(org.apache.bookkeeper.test.BookieClientTest)  Time elapsed:
2.147 sec  <<< FAILURE!
java.lang.AssertionError: expected:<0> but was:<-13>
at org.apache.bookkeeper.test.BookieClientTest.testWriteGaps(
BookieClientTest.java:147)


Thanks,
Charan


Re: [Discuss] 64 bits ledger id

2016-10-28 Thread Charan Reddy G
In our last meeting, we briefly discussed about options of handling
negative long ledgerids in LongeHierarchicalLedgerManager. Mainly by using
the unsigned long value (20 digits) for determining the ledger zNode path.
I see couple of problems with that

(signed long) -> (unsigned long)
-9223372036854775808  ->  "09223372036854775808"
 9223372036854775807  ->  "09223372036854775807"

now both these ledger znodes would be under the same parent zNode
0922/3372/0368/5477

gc method in ScanAndCompareGarbageCollector, for LedgerRange
(0922/3372/0368/5477) the following logic would break. Because we would get
mix of positive and negative ledgerids in this ledgerrange and
scanandcompare logic with bkActiveLedgers (ledgerStorage.getActiveLedgers)
will not work.


while(ledgerRangeIterator.hasNext()) {
LedgerRange lRange = ledgerRangeIterator.next();

Long start = lastEnd + 1;
Long end = lRange.end();
if (!ledgerRangeIterator.hasNext()) {
end = Long.MAX_VALUE;
}

Iterable subBkActiveLedgers =
bkActiveLedgers.subSet(start, true, end, true);

and also with ledgerrange with all negative ledgerids we need to do some
tweaking for the 'start' and the 'end'.

So in summary, I feel its risky to use unsigned long for determining the
ledger zNode path, since we used signed long for ledgerids all across the
codebase and this inconsistency can cause issue in some other areas as well.

So I'm more convinced to use the signed long for determining the ledger
zNode path and let the LongHierarchicalLedgerManager/Iterator deal with
negative ledgerids accordingly (for eg. deal with znode with '-' while
doing comparison and iteration). So it makes scope of the change limited to
LongHierarchicalLedgerManager rather than applying patches to other areas.

(ledger id)  ->  (znode path)
-9223372036854775808  ->  "-922/3372/0368/5477/5808"
 9223372036854775807  ->  "0922/3372/0368/5477/5807"

Thanks,
Charan


On Wed, Oct 26, 2016 at 10:55 PM, Sijie Guo  wrote:

> On Wed, Oct 26, 2016 at 3:49 PM, Matteo Merli  wrote:
>
> > On Wed, Oct 26, 2016 at 11:45 AM Venkateswara Rao Jujjuri <
> > jujj...@gmail.com>
> > wrote:
> >
> > > - Ledgers are unique across multiple clusters. Useful if storage tiers
> > with
> > > different stores are employed.
> > >
> >
> > For this you could combine the ledgerId with another 64bit id, that could
> > encode the rest of the required informations ( storage tier, cluster, ..
> )
> >
>
> +1 on this idea
>
>
> >
> >
> > > - No centralized id creation - Allows client to give the name instead
> of
> > > server generating name on create.
> > >
> >
> > This should be already possible with your changes in 4.4, right?
> >
> > Wouldn't be enough to combine the 64bit ledgerId with an additional id
> that
> > doesn't need to flow through BK ?
> >
>


Re: [Discuss] 64 bits ledger id

2016-10-26 Thread Charan Reddy G
*Can anyone from your team describe how did you guys extend the
ledgermanager? I am interested in how did you guys handle backward
compatibilityfor Hierarchical Ledger Manager.*

We created LongHierarchicalLedgerManager, which would work for *positive
long* ledgerids (so technically only 63 bits for ledgerid).
This LongHierarchicalLedgerManager extends HierarchicalLedgerManager and
its logic is similar to HierarchicalLedgerManager. But instead of using
2-level hierarchical znodes (2-4-4 split), we use 4-level hierarchical
znode with 3-4-4-4-4 split. We didn't plan for backward compatibility for
HierarchicalLedgerManager, since we started the cluster with
LongHierarchicalLedgerManager, we were ok with it.

Thanks,
Charan

On Wed, Oct 26, 2016 at 3:49 PM, Matteo Merli  wrote:

> On Wed, Oct 26, 2016 at 11:45 AM Venkateswara Rao Jujjuri <
> jujj...@gmail.com>
> wrote:
>
> > - Ledgers are unique across multiple clusters. Useful if storage tiers
> with
> > different stores are employed.
> >
>
> For this you could combine the ledgerId with another 64bit id, that could
> encode the rest of the required informations ( storage tier, cluster, .. )
>
>
> > - No centralized id creation - Allows client to give the name instead of
> > server generating name on create.
> >
>
> This should be already possible with your changes in 4.4, right?
>
> Wouldn't be enough to combine the 64bit ledgerId with an additional id that
> doesn't need to flow through BK ?
>


Question regarding exit strategy when we reach threshold space usage (0.95)

2016-08-04 Thread Charan Reddy G
Hi,

in LedgerDirsListener of InterleavedLedgerStorage, if the configured value
of isForceGCAllowWhenNoSpace is false, then when all disks are full, we are
suspending both Major and Minor GC. I'm wondering what is the exit strategy
from here? how will we come out of Readonly mode? Since we suspended both
Major and Minor GC even deleting Ledgers wouldn't help

@Override

*public* *void* allDisksFull() {

*if* (gcThread.isForceGCAllowWhenNoSpace) {

gcThread.enableForceGC();

} *else* {

gcThread.suspendMajorGC();

gcThread.suspendMinorGC();

}

}
Thanks,
Charan


Re: [ANNOUNCE] New BookKeeper Committer: JV Jujjuri

2016-06-14 Thread Charan Reddy G
Congrats JV!

Thanks,
Charan

On Tuesday, June 14, 2016, Sijie Guo  wrote:

> The Apache BookKeeper PMC recently extended committer karma to JV Jujjuri
> and he has accepted. JV is the main driver of bookkeeper production usage
> at Salesforce. He is active in the community, involving not only code
> contribution but also discussion and code reviews. We are looking forward
> to more contributions from him.
>
> Congratulations and welcome onboard, JV!
>


Re: Question/Issue regarding formatEntry in BookieShell

2016-03-21 Thread Charan Reddy G
Thanks Sijie. I was thinking much in the lines of possibility of
identifying the DigestType looking at the persisted entry data. But
providing option to mention DigestType is good enough solution.

Thanks,
Charan.

On Mon, Mar 21, 2016 at 12:02 PM, Sijie Guo  wrote:

> unfortunately don't know... we used crc32 at Twitter.
>
> - Sijie
>
> On Mon, Mar 21, 2016 at 10:32 AM, Venkateswara Rao Jujjuri <
> jujj...@gmail.com> wrote:
>
> > Right that works, but most of the folks stick to crc32. :)
> >
> > Sijie do you know if anyone in production uses mac?
> >
> > On Mon, Mar 21, 2016 at 10:28 AM, Sijie Guo  wrote:
> >
> > > How about passing a flag to those commands to specify the digest types?
> > By
> > > default, use crc32.
> > >
> > > - Sijie
> > >
> > > On Thu, Mar 17, 2016 at 5:10 PM, Charan Reddy G <
> reddychara...@gmail.com
> > >
> > > wrote:
> > >
> > > > Currently in BookieShell for formatting Entry (in formatEntry
> method),
> > to
> > > > skip digest, position of recBuff is set to (32+8)
> (rebuff.position(32 +
> > > > 8);). This works fine if the Digest is of type CRC32Digest where the
> > > > MacCodeLength is 8, but it wouldn’t work well for MacDigest since its
> > > > MacCodeLength is 20 or for any other future DigestManager if its
> > > > MacCodeLength != 8.
> > > >
> > > > Hence when we execute ReadLedgerCmd or ReadjournalCmd, for CRC32 it
> > > prints
> > > > data correctly for e.g.: “EntryNData”. But for MacDigest it would
> print
> > > > something like the following - "N<..}.`rEntryNData”, here first
> 12
> > > > bytes is part of Digest of that Entry.
> > > >
> > > > I tried to see if there is easy way to fix it but couldn’t find any.
> > > > Basically I’m trying to find out if there is any easy way to find the
> > > > DigestType or the MacCodeLength of that entry so that we can skip
> those
> > > > bytes. The entryLength we receive as part of 'recBuff’ is not actual
> > > length
> > > > of the entry but the accumulative length of the data until this
> entry,
> > so
> > > > it won’t be helpful to us. Any help would be appreciated, to print
> the
> > > > actual entry data without digest noise.
> > > >
> > > > Thanks,
> > > >
> > > > Charan.
> > > >
> > >
> >
> >
> >
> > --
> > Jvrao
> > ---
> > First they ignore you, then they laugh at you, then they fight you, then
> > you win. - Mahatma Gandhi
> >
>


Question/Issue regarding formatEntry in BookieShell

2016-03-19 Thread Charan Reddy G
Currently in BookieShell for formatting Entry (in formatEntry method), to
skip digest, position of recBuff is set to (32+8) (rebuff.position(32 +
8);). This works fine if the Digest is of type CRC32Digest where the
MacCodeLength is 8, but it wouldn’t work well for MacDigest since its
MacCodeLength is 20 or for any other future DigestManager if its
MacCodeLength != 8.

Hence when we execute ReadLedgerCmd or ReadjournalCmd, for CRC32 it prints
data correctly for e.g.: “EntryNData”. But for MacDigest it would print
something like the following - "N<..}.`rEntryNData”, here first 12
bytes is part of Digest of that Entry.

I tried to see if there is easy way to fix it but couldn’t find any.
Basically I’m trying to find out if there is any easy way to find the
DigestType or the MacCodeLength of that entry so that we can skip those
bytes. The entryLength we receive as part of 'recBuff’ is not actual length
of the entry but the accumulative length of the data until this entry, so
it won’t be helpful to us. Any help would be appreciated, to print the
actual entry data without digest noise.

Thanks,

Charan.


Re: Few questions regarding internals of BookKeeper

2016-03-09 Thread Charan Reddy G
Added inline responses

On Tue, Mar 8, 2016 at 10:04 PM, Sijie Guo  wrote:

> On Tue, Mar 8, 2016 at 6:51 PM, Venkateswara Rao Jujjuri <
> jujj...@gmail.com>
> wrote:
>
> > Hi Charan, please see my response below.
> >
> > Sijie/Matteo/others can you please chime in too?
> >
> > On Wed, Mar 2, 2016 at 11:40 AM, Charan Reddy G  >
> > wrote:
> >
> > > Hi,
> > >
> > > I've few questions related to BookKeeper internals and I think some of
> > them
> > > are issues. Can someone please clarify them
> > >
> > > 1) SortedLedgerStorage: in SortedLedgerStorage in getEntry method, why
> > does
> > > it first try to read from EntryLogger (persistence storage) instead of
> > > EntryMemTable, isn't it optimal to first attempt to read from memory
> > > instead of from persistence storage? Is it empirical decision to go
> other
> > > way round?
> > >
> >
> > Last time I brought this up with Sijie and he thinks this is not the
> case.
> > Charan,
> > can you please provide call flow here?
> >
>
> I need to check this to confirm.
> Charan: Please check SortedLedgerStorage.getEntry() method
>
> >
> >
> > > 2) Cookie: why IndexDirs is not part of Cookie? In generateCookie
> method
> > of
> > > Cookie class, we added JournalDir and LedgerDirs but not IndexDirs, is
> it
> > > ok to not to include IndexDirs in it?
> > >
> >
> > Index dirs doesn't contain actual data, we need to treat it like
> metadata.
> > So, I believe we must not include index files.
> >
>
> I think the indexDirs were added later after we introduced Cookie. so we
> forgot to add them to the Cookie.
>
> This should be added though.
> Charan: so new layoutVersion of Cookie? Should we create a new
> Bug/Feature/Story for it?
>
> >
> >
> > > 3) MetaFormat BookieShell Command: On executing 'metaformat'
> BookieShell
> > > command it is observed that it does everything as expected but it
> doesn't
> > > deletes UnderReplicatedLedgers list from the ZooKeeper. It looks like
> an
> > > issue to me.
> >
>
> Could you file a jira for it?
> Charan: created jira Bug -
> https://issues.apache.org/jira/browse/BOOKKEEPER-903
>
> > >
> > > 4) length in LedgerEntry: when we read LedgerEntry using LedgerHandle
> > > readEntries method, the length of LedgerEntry instance is not
> > representing
> > > the actual length of the entry instead it represents the cumulative
> > length
> > > of the entry in Ledger (entrylength of n = actual length of entry n +
> > > entrylength of (n-1)). Is this what is it supposed to be? Am I doing
> > > something wrong here?
> > >
> >
> > That is what it is; Entry length = header length + payload (the actual
> > payload from client)
> >
>
Charan: JV you misunderstood what I mentioned. I meant, I'm observing that
Entry length (of entry n) = HeaderLength (of entry n) + Payload (of entry
n) + EntryLength (of entry (n-1)).
Sijie, can you please comment on this.

>
> > > 5) AutoRecoveryMain: it seems it is configurable to enable/disable
> > > AutoRecoveryDaemon and it runs as daemon thread in Bookie/BookieServer
> > > process only if it is enabled and the other option option is to run
> > > separate AutoRecoveryMain Daemon service process in each node from CLI
> > > (from bookkeeper shell script). Are there pro and cons to go one way or
> > the
> > > other?
>
>
> We run AutoRecoveryMain as separated service in our mesos scheduler.
> Because it is a stateless service.
> Charan: Thanks Sijie for clarification. I presume you are running 
> AutoRecoveryMain
> service in each Bookie node.
>
> > Is it ok to not to run AutoRecoveryMain daemon in any of the
> > > bookies?
> > >
> >
> > This is preferred to be enabled. Disable is only for backward
> > compatibility. We even talked
> > about it last time when we met Sijie and Matteo at dinner. We can change
> > config file to enable it by default.
> > I believe this must be enabled on all bookies.
> >
> >
> > >
> > > 6) updatecookie command and updateledgers command: It is observed after
> > > calling updatecookie command it will update the cookies both in ZK and
> > > local filesystem directories according to the new BookieAddress, but if
> > we
> > > try listbookies command after updatecookie command it still shows old
> > > bookie address, is it not

Re: [VOTE] Remove Hedwig from source tree

2016-03-08 Thread Charan Reddy G
+1

On Tue, Mar 8, 2016 at 10:54 AM, Arun M. Krishnakumar 
wrote:

> +1
>
> On Tue, Mar 8, 2016 at 10:34 AM, Matteo Merli  wrote:
>
> > +1
> >
> > On Tue, Mar 8, 2016 at 10:23 AM Robin Dhamankar 
> > wrote:
> >
> > > +1
> > > On Mar 8, 2016 10:22 AM, "Uma gangumalla" 
> wrote:
> > >
> > > > +1
> > > >
> > > > Regards,
> > > > Uma
> > > >
> > > > On Mon, Mar 7, 2016 at 9:57 PM, Sijie Guo  wrote:
> > > >
> > > > > Since Hedwig has been inactive for a while, there was a proposal to
> > > > remove
> > > > > it from the code base. I'd raise a vote on removing hedwig from the
> > > code
> > > > > base.
> > > > >
> > > > > Here is a ticket for the discussion :
> > > > > https://issues.apache.org/jira/browse/BOOKKEEPER-769
> > > > >
> > > > > - Sijie
> > > > >
> > > >
> > >
> >
>


Few questions regarding internals of BookKeeper

2016-03-02 Thread Charan Reddy G
Hi,

I've few questions related to BookKeeper internals and I think some of them
are issues. Can someone please clarify them

1) SortedLedgerStorage: in SortedLedgerStorage in getEntry method, why does
it first try to read from EntryLogger (persistence storage) instead of
EntryMemTable, isn't it optimal to first attempt to read from memory
instead of from persistence storage? Is it empirical decision to go other
way round?

2) Cookie: why IndexDirs is not part of Cookie? In generateCookie method of
Cookie class, we added JournalDir and LedgerDirs but not IndexDirs, is it
ok to not to include IndexDirs in it?

3) MetaFormat BookieShell Command: On executing 'metaformat' BookieShell
command it is observed that it does everything as expected but it doesn't
deletes UnderReplicatedLedgers list from the ZooKeeper. It looks like an
issue to me.

4) length in LedgerEntry: when we read LedgerEntry using LedgerHandle
readEntries method, the length of LedgerEntry instance is not representing
the actual length of the entry instead it represents the cumulative length
of the entry in Ledger (entrylength of n = actual length of entry n +
entrylength of (n-1)). Is this what is it supposed to be? Am I doing
something wrong here?

5) AutoRecoveryMain: it seems it is configurable to enable/disable
AutoRecoveryDaemon and it runs as daemon thread in Bookie/BookieServer
process only if it is enabled and the other option option is to run
separate AutoRecoveryMain Daemon service process in each node from CLI
(from bookkeeper shell script). Are there pro and cons to go one way or the
other? Is it ok to not to run AutoRecoveryMain daemon in any of the bookies?

6) updatecookie command and updateledgers command: It is observed after
calling updatecookie command it will update the cookies both in ZK and
local filesystem directories according to the new BookieAddress, but if we
try listbookies command after updatecookie command it still shows old
bookie address, is it not supposed to get updated as part of updatecookie
command? will it not cause discrepancies? Similarly after updateledgers
command listbookies command still show older bookieaddress.

Thanks,
Charan Reddy