Re: Help with bad errors on 4.6.1

2018-03-27 Thread Enrico Olivelli
End of this story
With this patch the problem does not occur anymore

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

The patch does not address directly the problem, the root source is still
unknown, this is very bad. But with that change no error is reported
anymore, so actually it is enough to go to production with java9 (10) and
my application.

I will start release process for 4.6.2 soon.

Thanks to everyone who helped

Enrico

Il ven 16 mar 2018, 12:30 Enrico Olivelli  ha scritto:

> resending (GMAIL webmail messed up prev email)
>
>
>
> 2018-03-16 10:34 GMT+01:00 Sijie Guo :
>
>> On Fri, Mar 16, 2018 at 2:26 AM, Enrico Olivelli 
>> wrote:
>>
>> > Thank you Sijie,
>> > I have already applied a similar patch to my local code based on 4.6.1
>> but
>> > the problem remains.
>> >
>>
>> What do you mean "the problem" here?
>>
>
> missing buf.release(), that you fixed on 4.7 and will cherry pick to 4.6
> branch
>
>
>
>> Do you mean the corruption problem or the leaking problem? The change I
>> pointed out only address the leaking problem. I don't think it is the
>> problem of corruption.
>>
>>
> Actually we are talking about a wire protocol level corruption.
> The only good "fix" that I have at the moment is to use use
> UnpooledByteBufAllocator.DEFAULT in encoder methods for v3
>
>
> Enrico
>
>
>
>
>
>>
>> >
>> > I am looking into Netty allocateUnitializedArray, which is used for
>> Pooled
>> > Heap Buffers.
>> > You all are more aware of BK code than me, is there any point in which
>> we
>> > assume that the buffer is full of zeroes ?
>> >
>> >
>> > Enrico
>> >
>> >
>> >
>> >
>> > 2018-03-16 10:22 GMT+01:00 Ivan Kelly :
>> >
>> > > > With "paranoid" log in Netty I found this that is very interesting,
>> but
>> > > it
>> > > > happens even on Java 8.
>> > > I don't think leaks are the problem here though. This seems to be more
>> > > like a doublefree issue.
>> > >
>> > > -Ivan
>> > >
>> >
>>
> --


-- Enrico Olivelli


Re: Help with bad errors on 4.6.1

2018-03-16 Thread Enrico Olivelli
resending (GMAIL webmail messed up prev email)



2018-03-16 10:34 GMT+01:00 Sijie Guo :

> On Fri, Mar 16, 2018 at 2:26 AM, Enrico Olivelli 
> wrote:
>
> > Thank you Sijie,
> > I have already applied a similar patch to my local code based on 4.6.1
> but
> > the problem remains.
> >
>
> What do you mean "the problem" here?
>


missing buf.release(), that you fixed on 4.7 and will cherry pick to 4.6
branch



>
> Do you mean the corruption problem or the leaking problem? The change I
> pointed out only address the leaking problem. I don't think it is the
> problem of corruption.
>
>

Actually we are talking about a wire protocol level corruption.
The only good "fix" that I have at the moment is to use use
UnpooledByteBufAllocator.DEFAULT in encoder methods for v3


Enrico





>
> >
> > I am looking into Netty allocateUnitializedArray, which is used for
> Pooled
> > Heap Buffers.
> > You all are more aware of BK code than me, is there any point in which we
> > assume that the buffer is full of zeroes ?
> >
> >
> > Enrico
> >
> >
> >
> >
> > 2018-03-16 10:22 GMT+01:00 Ivan Kelly :
> >
> > > > With "paranoid" log in Netty I found this that is very interesting,
> but
> > > it
> > > > happens even on Java 8.
> > > I don't think leaks are the problem here though. This seems to be more
> > > like a doublefree issue.
> > >
> > > -Ivan
> > >
> >
>


Re: Help with bad errors on 4.6.1

2018-03-16 Thread Enrico Olivelli
2018-03-16 10:34 GMT+01:00 Sijie Guo :

> On Fri, Mar 16, 2018 at 2:26 AM, Enrico Olivelli 
> wrote:
>
> > Thank you Sijie,
> > I have already applied a similar patch to my local code based on 4.6.1
> but
> > the problem remains.
> >
>
> What do you mean "the problem" here?
>

missing buf.release(), that you fixed on 4.7 and will cherry pick to 4.6
branch


>
> Do you mean the corruption problem or the leaking problem? The change I
> pointed out only address the leaking problem. I don't think it is the
> problem of corruption.
>

Actually we are talking about a wire protocol level corruption.
The only good "fix" that I have at the moment is to use use
UnpooledByteBufAllocator.DEFAULT here for client side

https://github.com/apache/bookkeeper/blob/b53c0c3370c42d753afb852824ea12d2dbce4f2e/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieProtoEncoding.java#L325


and for bookie side

https://github.com/apache/bookkeeper/blob/b53c0c3370c42d753afb852824ea12d2dbce4f2e/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieProtoEncoding.java#L349













I have pointed the leak in order to answer to Ivan because it was the only
error report from Netty using Ivan's suggestion to enable "paranoid" debug
level for ByteBufs leaks


Enrico




> >
> > I am looking into Netty allocateUnitializedArray, which is used for
> Pooled
> > Heap Buffers.
> > You all are more aware of BK code than me, is there any point in which we
> > assume that the buffer is full of zeroes ?
> >
> >
> > Enrico
> >
> >
> >
> >
> > 2018-03-16 10:22 GMT+01:00 Ivan Kelly :
> >
> > > > With "paranoid" log in Netty I found this that is very interesting,
> but
> > > it
> > > > happens even on Java 8.
> > > I don't think leaks are the problem here though. This seems to be more
> > > like a doublefree issue.
> > >
> > > -Ivan
> > >
> >
>


Re: Help with bad errors on 4.6.1

2018-03-16 Thread Sijie Guo
On Fri, Mar 16, 2018 at 2:26 AM, Enrico Olivelli 
wrote:

> Thank you Sijie,
> I have already applied a similar patch to my local code based on 4.6.1 but
> the problem remains.
>

What do you mean "the problem" here?

Do you mean the corruption problem or the leaking problem? The change I
pointed out only address the leaking problem. I don't think it is the
problem of corruption.


>
> I am looking into Netty allocateUnitializedArray, which is used for Pooled
> Heap Buffers.
> You all are more aware of BK code than me, is there any point in which we
> assume that the buffer is full of zeroes ?
>
>
> Enrico
>
>
>
>
> 2018-03-16 10:22 GMT+01:00 Ivan Kelly :
>
> > > With "paranoid" log in Netty I found this that is very interesting, but
> > it
> > > happens even on Java 8.
> > I don't think leaks are the problem here though. This seems to be more
> > like a doublefree issue.
> >
> > -Ivan
> >
>


Re: Help with bad errors on 4.6.1

2018-03-16 Thread Enrico Olivelli
Thank you Sijie,
I have already applied a similar patch to my local code based on 4.6.1 but
the problem remains.

I am looking into Netty allocateUnitializedArray, which is used for Pooled
Heap Buffers.
You all are more aware of BK code than me, is there any point in which we
assume that the buffer is full of zeroes ?


Enrico




2018-03-16 10:22 GMT+01:00 Ivan Kelly :

> > With "paranoid" log in Netty I found this that is very interesting, but
> it
> > happens even on Java 8.
> I don't think leaks are the problem here though. This seems to be more
> like a doublefree issue.
>
> -Ivan
>


Re: Help with bad errors on 4.6.1

2018-03-16 Thread Ivan Kelly
> With "paranoid" log in Netty I found this that is very interesting, but it
> happens even on Java 8.
I don't think leaks are the problem here though. This seems to be more
like a doublefree issue.

-Ivan


Re: Help with bad errors on 4.6.1

2018-03-16 Thread Sijie Guo
On Fri, Mar 16, 2018 at 1:24 AM, Enrico Olivelli 
wrote:

> 2018-03-15 12:02 GMT+01:00 Enrico Olivelli :
>
> >
> >
> > 2018-03-15 11:13 GMT+01:00 Ivan Kelly :
> >
> >> > What is the difference in Channel#write/ByteBuf pooling.in Java 9
> ?
> >> Sounds like it could be an issue in netty itself. Java 9 removed a
> >> bunch of stuff around Unsafe, which I'm pretty sure netty was using
> >> for ByteBuf. Have you tried setting the pool debugging to paranoid?
> >>
> >> -Dio.netty.leakDetection.level=paranoid
> >>
> >
> >
> > only with 'advanced' , sorry. I will try again with paranoid
> >
> > I have tried to attach my laptop to the same BK cluster and create a
> > reproducer client, no results !! Bookies do not break!
> > it seems that the error is only between the machines of that group
> > (network)
> >
> > It seems something very weird, maybe a mix of message size/network
> > settings/Java GC (pooled ByteBuf heap bufs are released on finalize() as
> > far as I have understood from Netty logs)
> >
> > I hope that with 'paranoid' I will have some more stacktrace to have code
> > references
> >
>
>
> I am playing now with io.netty.uninitializedArrayAllocationThreshold which
> is a new feature in Java 9 and it deals with byte[] which are not set to
> zeros (so can be compatilble with unexpected contents)
>
> With "paranoid" log in Netty I found this that is very interesting, but it
> happens even on Java 8.
>





>
> Enrico
>
>
> 8-03-16-09-13-49LEAK: ByteBuf.release() was not called before it's
> garbage-collected. See http://netty.io/wiki/reference-counted-objects.html
> for more information.
> WARNING: 1 leak records were discarded because the leak record count is
> limited to 4. Use system property io.netty.leakDetection.maxRecords to
> increase the limit.
> Recent access records: 5
> #5:
>
> io.netty.buffer.AdvancedLeakAwareByteBuf.readLong(
> AdvancedLeakAwareByteBuf.java:444)
>
> org.apache.bookkeeper.bookie.InterleavedLedgerStorage.getLastAddConfirmed(
> InterleavedLedgerStorage.java:245)
>


I think that's a leaking bug. I fixed here:

https://github.com/apache/bookkeeper/commit/8b8c5515e041347e0fdd791534fa0a694c96c6ae#diff-55705379048304b4fd800432e743487eR250

I will cherry-pick this fix into 4.6.


>
> org.apache.bookkeeper.bookie.LedgerDescriptorImpl.getLastAddConfirmed(
> LedgerDescriptorImpl.java:154)
>
> org.apache.bookkeeper.bookie.Bookie.readLastAddConfirmed(Bookie.java:1432)
>
> org.apache.bookkeeper.proto.ReadEntryProcessorV3.readEntry(
> ReadEntryProcessorV3.java:182)
>
> org.apache.bookkeeper.proto.ReadEntryProcessorV3.readEntry(
> ReadEntryProcessorV3.java:152)
>
> org.apache.bookkeeper.proto.ReadEntryProcessorV3.getReadResponse(
> ReadEntryProcessorV3.java:215)
>
> org.apache.bookkeeper.proto.ReadEntryProcessorV3.executeOp(
> ReadEntryProcessorV3.java:258)
>
> org.apache.bookkeeper.proto.ReadEntryProcessorV3.safeRun(
> ReadEntryProcessorV3.java:254)
> org.apache.bookkeeper.common.util.SafeRunnable.run(
> SafeRunnable.java:36)
> java.util.concurrent.Executors$RunnableAdapter.
> call(Executors.java:511)
> java.util.concurrent.FutureTask.run(FutureTask.java:266)
>
> java.util.concurrent.ScheduledThreadPoolExecutor$
> ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
>
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(
> ScheduledThreadPoolExecutor.java:293)
>
> java.util.concurrent.ThreadPoolExecutor.runWorker(
> ThreadPoolExecutor.java:1149)
>
> java.util.concurrent.ThreadPoolExecutor$Worker.run(
> ThreadPoolExecutor.java:624)
> java.lang.Thread.run(Thread.java:748)
>
>
> Created at:
> io.netty.util.ResourceLeakDetector.track(
> ResourceLeakDetector.java:237)
>
> io.netty.buffer.PooledByteBufAllocator.newDirectBuffer(
> PooledByteBufAllocator.java:331)
>
> io.netty.buffer.AbstractByteBufAllocator.directBuffer(
> AbstractByteBufAllocator.java:181)
> org.apache.bookkeeper.bookie.EntryLogger.readEntry(
> EntryLogger.java:990)
>
> org.apache.bookkeeper.bookie.InterleavedLedgerStorage.getEntry(
> InterleavedLedgerStorage.java:303)
>
> org.apache.bookkeeper.bookie.SortedLedgerStorage.getLastEntryId(
> SortedLedgerStorage.java:140)
>
> org.apache.bookkeeper.bookie.SortedLedgerStorage.getEntry(
> SortedLedgerStorage.java:146)
>
> org.apache.bookkeeper.bookie.InterleavedLedgerStorage.getLastAddConfirmed(
> InterleavedLedgerStorage.java:239)
>
> org.apache.bookkeeper.bookie.LedgerDescriptorImpl.getLastAddConfirmed(
> LedgerDescriptorImpl.java:154)
>
> org.apache.bookkeeper.bookie.Bookie.readLastAddConfirmed(Bookie.java:1432)
>
> org.apache.bookkeeper.proto.ReadEntryProcessorV3.readEntry(
> ReadEntryProcessorV3.java:182)
>
> org.apache.bookkeeper.proto.ReadEntryProcessorV3.readEntry(
> ReadEntryProcessorV3.java:152)
>
> org.apache.bookkeeper.proto.ReadEntryProcessorV3.getReadResponse(
> ReadEntryProcessorV3.java:215)
>
> 

Re: Help with bad errors on 4.6.1

2018-03-16 Thread Enrico Olivelli
2018-03-15 12:02 GMT+01:00 Enrico Olivelli :

>
>
> 2018-03-15 11:13 GMT+01:00 Ivan Kelly :
>
>> > What is the difference in Channel#write/ByteBuf pooling.in Java 9 ?
>> Sounds like it could be an issue in netty itself. Java 9 removed a
>> bunch of stuff around Unsafe, which I'm pretty sure netty was using
>> for ByteBuf. Have you tried setting the pool debugging to paranoid?
>>
>> -Dio.netty.leakDetection.level=paranoid
>>
>
>
> only with 'advanced' , sorry. I will try again with paranoid
>
> I have tried to attach my laptop to the same BK cluster and create a
> reproducer client, no results !! Bookies do not break!
> it seems that the error is only between the machines of that group
> (network)
>
> It seems something very weird, maybe a mix of message size/network
> settings/Java GC (pooled ByteBuf heap bufs are released on finalize() as
> far as I have understood from Netty logs)
>
> I hope that with 'paranoid' I will have some more stacktrace to have code
> references
>


I am playing now with io.netty.uninitializedArrayAllocationThreshold which
is a new feature in Java 9 and it deals with byte[] which are not set to
zeros (so can be compatilble with unexpected contents)

With "paranoid" log in Netty I found this that is very interesting, but it
happens even on Java 8.

Enrico


8-03-16-09-13-49LEAK: ByteBuf.release() was not called before it's
garbage-collected. See http://netty.io/wiki/reference-counted-objects.html
for more information.
WARNING: 1 leak records were discarded because the leak record count is
limited to 4. Use system property io.netty.leakDetection.maxRecords to
increase the limit.
Recent access records: 5
#5:

io.netty.buffer.AdvancedLeakAwareByteBuf.readLong(AdvancedLeakAwareByteBuf.java:444)

org.apache.bookkeeper.bookie.InterleavedLedgerStorage.getLastAddConfirmed(InterleavedLedgerStorage.java:245)

org.apache.bookkeeper.bookie.LedgerDescriptorImpl.getLastAddConfirmed(LedgerDescriptorImpl.java:154)

org.apache.bookkeeper.bookie.Bookie.readLastAddConfirmed(Bookie.java:1432)

org.apache.bookkeeper.proto.ReadEntryProcessorV3.readEntry(ReadEntryProcessorV3.java:182)

org.apache.bookkeeper.proto.ReadEntryProcessorV3.readEntry(ReadEntryProcessorV3.java:152)

org.apache.bookkeeper.proto.ReadEntryProcessorV3.getReadResponse(ReadEntryProcessorV3.java:215)

org.apache.bookkeeper.proto.ReadEntryProcessorV3.executeOp(ReadEntryProcessorV3.java:258)

org.apache.bookkeeper.proto.ReadEntryProcessorV3.safeRun(ReadEntryProcessorV3.java:254)
org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36)
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
java.util.concurrent.FutureTask.run(FutureTask.java:266)

java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)

java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)

java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)

java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
java.lang.Thread.run(Thread.java:748)


Created at:
io.netty.util.ResourceLeakDetector.track(ResourceLeakDetector.java:237)

io.netty.buffer.PooledByteBufAllocator.newDirectBuffer(PooledByteBufAllocator.java:331)

io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:181)
org.apache.bookkeeper.bookie.EntryLogger.readEntry(EntryLogger.java:990)

org.apache.bookkeeper.bookie.InterleavedLedgerStorage.getEntry(InterleavedLedgerStorage.java:303)

org.apache.bookkeeper.bookie.SortedLedgerStorage.getLastEntryId(SortedLedgerStorage.java:140)

org.apache.bookkeeper.bookie.SortedLedgerStorage.getEntry(SortedLedgerStorage.java:146)

org.apache.bookkeeper.bookie.InterleavedLedgerStorage.getLastAddConfirmed(InterleavedLedgerStorage.java:239)

org.apache.bookkeeper.bookie.LedgerDescriptorImpl.getLastAddConfirmed(LedgerDescriptorImpl.java:154)

org.apache.bookkeeper.bookie.Bookie.readLastAddConfirmed(Bookie.java:1432)

org.apache.bookkeeper.proto.ReadEntryProcessorV3.readEntry(ReadEntryProcessorV3.java:182)

org.apache.bookkeeper.proto.ReadEntryProcessorV3.readEntry(ReadEntryProcessorV3.java:152)

org.apache.bookkeeper.proto.ReadEntryProcessorV3.getReadResponse(ReadEntryProcessorV3.java:215)

org.apache.bookkeeper.proto.ReadEntryProcessorV3.executeOp(ReadEntryProcessorV3.java:258)

org.apache.bookkeeper.proto.ReadEntryProcessorV3.safeRun(ReadEntryProcessorV3.java:254)
org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36)
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
java.util.concurrent.FutureTask.run(FutureTask.java:266)

java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)


Re: Help with bad errors on 4.6.1

2018-03-15 Thread Enrico Olivelli
2018-03-15 11:13 GMT+01:00 Ivan Kelly :

> > What is the difference in Channel#write/ByteBuf pooling.in Java 9 ?
> Sounds like it could be an issue in netty itself. Java 9 removed a
> bunch of stuff around Unsafe, which I'm pretty sure netty was using
> for ByteBuf. Have you tried setting the pool debugging to paranoid?
>
> -Dio.netty.leakDetection.level=paranoid
>


only with 'advanced' , sorry. I will try again with paranoid

I have tried to attach my laptop to the same BK cluster and create a
reproducer client, no results !! Bookies do not break!
it seems that the error is only between the machines of that group (network)

It seems something very weird, maybe a mix of message size/network
settings/Java GC (pooled ByteBuf heap bufs are released on finalize() as
far as I have understood from Netty logs)

I hope that with 'paranoid' I will have some more stacktrace to have code
references

Thank you

Enrico


>
> i tried running my attempted repro in java9, but no cigar.
>
>
> -Ivan
>


Re: Help with bad errors on 4.6.1

2018-03-15 Thread Ivan Kelly
> What is the difference in Channel#write/ByteBuf pooling.in Java 9 ?
Sounds like it could be an issue in netty itself. Java 9 removed a
bunch of stuff around Unsafe, which I'm pretty sure netty was using
for ByteBuf. Have you tried setting the pool debugging to paranoid?

-Dio.netty.leakDetection.level=paranoid


i tried running my attempted repro in java9, but no cigar.


-Ivan


Re: Help with bad errors on 4.6.1

2018-03-15 Thread Enrico Olivelli
Very latest news:
I have narrowed the problem to ResponseEnDecoderV3#encode, using
UnpooledByteBufAllocator.DEFAULT instead of the allocator from the channel
the error disappear.

So the problem is about the encoding of the responses, using Java 9 and
Pooled Byte Bufs.
This is compatible with the errors on the client side about corrupted
responses in case of Client on Java8 and Server on Java9.

I am now doing tests with Bookie on Java 8 and Clients on Java 9 and the
problem seems the same, I receive corrupted messages on Bookie.

Does any ring bell ?

What is the difference in Channel#write/ByteBuf pooling.in Java 9 ?

Enrico







2018-03-15 5:21 GMT+01:00 Enrico Olivelli :

> Latest findings, some good news, and some very bad.
>
> Good news:
> I was wrong, I did not switch back the system to Java 8 correcly.
>
> The problem is on Bookie side and occours only if the bookie in on Java 9.
>
> Bad news:
> I have a fix. The fix to use Unpooled ByteBufs in serializeProtobuf:
>
> private static ByteBuf serializeProtobuf(MessageLite msg, ByteBufAllocator
> allocator) {
> int size = msg.getSerializedSize();
> ByteBuf buf = Unpooled.buffer(size, size);
> ...
>
> I will continue to track down to the cause, I think it is on the read-path
> (not sure).
>
> On client side we have a flag to not use pooled ByteBufs on Channel
> Allocator, the most trivial fix at the moment is to make the same on Bookie
> side as an hotfix for branch 4.6.
>
> Before jumping to this extreme hotfix solution I will dig into the issue,
> now that I know that the problem is ONLY on Java 9 and on the Bookie it
> will be simpler to find a reproducer.
>
> It remains the point that in other systems I have and in test cases there
> is no failure.
>
> Honestly I have no Java 9 bookie in production, only Java 8 bookies, maybe
> this is the motivation of the fact that no one ever reported this problem
> from production
>
> Enrico
>
>
>
>
> 2018-03-14 17:27 GMT+01:00 Ivan Kelly :
>
>> >> > @Ivan
>> >> > I wonder if some tests on Jepsen with bookie restarts may find this
>> kind
>> >> of
>> >> > issues, given that it is not a network/SO problem
>> >> If jepsen can catch then normal integration test can.
>>
>> I attempted a repro for this using the integration test stuff.
>> Running for 2-3 hours in a loop, no bug hit. Perhaps I'm not doing
>> exactly what you are doing.
>>
>> https://github.com/ivankelly/bookkeeper/blob/enrico-bug/test
>> s/integration/enrico-bug/src/test/java/org/apache/bookkeepe
>> r/tests/integration/TestEnricoBug.java
>>
>> -Ivan
>>
>
>


Re: Help with bad errors on 4.6.1

2018-03-14 Thread Enrico Olivelli
Latest findings, some good news, and some very bad.

Good news:
I was wrong, I did not switch back the system to Java 8 correcly.

The problem is on Bookie side and occours only if the bookie in on Java 9.

Bad news:
I have a fix. The fix to use Unpooled ByteBufs in serializeProtobuf:

private static ByteBuf serializeProtobuf(MessageLite msg, ByteBufAllocator
allocator) {
int size = msg.getSerializedSize();
ByteBuf buf = Unpooled.buffer(size, size);
...

I will continue to track down to the cause, I think it is on the read-path
(not sure).

On client side we have a flag to not use pooled ByteBufs on Channel
Allocator, the most trivial fix at the moment is to make the same on Bookie
side as an hotfix for branch 4.6.

Before jumping to this extreme hotfix solution I will dig into the issue,
now that I know that the problem is ONLY on Java 9 and on the Bookie it
will be simpler to find a reproducer.

It remains the point that in other systems I have and in test cases there
is no failure.

Honestly I have no Java 9 bookie in production, only Java 8 bookies, maybe
this is the motivation of the fact that no one ever reported this problem
from production

Enrico




2018-03-14 17:27 GMT+01:00 Ivan Kelly :

> >> > @Ivan
> >> > I wonder if some tests on Jepsen with bookie restarts may find this
> kind
> >> of
> >> > issues, given that it is not a network/SO problem
> >> If jepsen can catch then normal integration test can.
>
> I attempted a repro for this using the integration test stuff.
> Running for 2-3 hours in a loop, no bug hit. Perhaps I'm not doing
> exactly what you are doing.
>
> https://github.com/ivankelly/bookkeeper/blob/enrico-bug/
> tests/integration/enrico-bug/src/test/java/org/apache/
> bookkeeper/tests/integration/TestEnricoBug.java
>
> -Ivan
>


Re: Help with bad errors on 4.6.1

2018-03-14 Thread Ivan Kelly
>> > @Ivan
>> > I wonder if some tests on Jepsen with bookie restarts may find this kind
>> of
>> > issues, given that it is not a network/SO problem
>> If jepsen can catch then normal integration test can.

I attempted a repro for this using the integration test stuff.
Running for 2-3 hours in a loop, no bug hit. Perhaps I'm not doing
exactly what you are doing.

https://github.com/ivankelly/bookkeeper/blob/enrico-bug/tests/integration/enrico-bug/src/test/java/org/apache/bookkeeper/tests/integration/TestEnricoBug.java

-Ivan


Re: Help with bad errors on 4.6.1

2018-03-13 Thread Enrico Olivelli
2018-03-13 17:19 GMT+01:00 Ivan Kelly :

> > @Ivan
> > I wonder if some tests on Jepsen with bookie restarts may find this kind
> of
> > issues, given that it is not a network/SO problem
> If jepsen can catch then normal integration test can. The readers in
> question, are they tailing with long poll, or just calling
> readLastAddConfirmed in a loop? What is the configuration in terms of
> ensemble/write/ack?
>

readLastAddConfirmed in a loop, see this code, it is mostly like the
tutorial

https://github.com/diennea/majordodo/blob/1487dc85a79e64ac0624a320729f2ad425fe15dd/majordodo-core/src/main/java/majordodo/replication/ReplicatedCommitLog.java#L975





>
> I can try to put together a repro too, using the integ test framework.
>

thank you


>
> -Ivan
>


Re: Help with bad errors on 4.6.1

2018-03-13 Thread Ivan Kelly
> @Ivan
> I wonder if some tests on Jepsen with bookie restarts may find this kind of
> issues, given that it is not a network/SO problem
If jepsen can catch then normal integration test can. The readers in
question, are they tailing with long poll, or just calling
readLastAddConfirmed in a loop? What is the configuration in terms of
ensemble/write/ack?

I can try to put together a repro too, using the integ test framework.

-Ivan


Re: Help with bad errors on 4.6.1

2018-03-13 Thread Enrico Olivelli
Findings of today:
A - the system fails even with BK 4.6.0
B - we have moved all the clients and the bookies to different machines
(keeping the same ZK cluster), same problem
C - I have copies of the application which are running on other similar
machines (on the same Blade/VMWare system)
D - I have tried to disable Netty polls on client side (Sijie's
suggestion): no effect
E - with ensemblesize = 1 the problem on readers does not occour, but the
writer seems not to be able to recover from a restart of the only bookie
(seems stuck at writing on logger PendingAddOp "Failed to write entry 
Bookie Operation Timed Out")
F - ZK cluster is working perfeclty as it is serving a lot of other
services of the application (Kafka, Majordodo, BlazingCache, HBase)
without errors
G - all of the other distributed components are running without issues
(Kafka,HDFS see the list above about ZK) and other database connections
too (the application connects to serveral external machines)
H - bookkeper bookiesanity is running OKAY on every bookie
I - my collegues checked networking and VMWARE and OS, we were suspecting
about problems on lookback interfaces but the problem still occours moving
each part on a dedicated machine
L - I have tested with 4.6.2-SNAPSHOT...same as above
M - the problem starts when a bookie restarts and then joins the cluster
again (not when you kill it)

given all of these facts:
1) It may be a problem of network/SO (given points F and G I doubt)
2) it may be a bug on BK
3) it is not a regression on 4.6.1 but 4.6.2 has no fix
4) I will intrument BK code in order to have better debug of the error
5) I will create a reproducer without the full application (which is huge)

I have memory (hprof) dumps of a failing client and a failing bookie if
someone has time to spend, honestly I have already spent some time in order
to find some leak/bad recycler, but without success (not sure this is the
good way to approach this problem)

I have no proof but maybe there is a problem with Pending reads, when the
bookie is down the read remains "pending", then when the channel is active
again (the bookie joins the cluster) that pending "old" read (which is not
needed anymore) reaches the bookie and crash everything.

It is interesting that it seems that "other" bookies break, not the one
which joins the cluster (this is what is seems to me)

@Ivan
I wonder if some tests on Jepsen with bookie restarts may find this kind of
issues, given that it is not a network/SO problem

Regards

Enrico





2018-03-12 20:51 GMT+01:00 Enrico Olivelli :

>
>
> Il lun 12 mar 2018, 20:40 Ivan Kelly  ha scritto:
>
>> > It is interesting that the problems is on 'readers' and it seems that
>> the
>> > PCBC seems corrupted and even writes (if the broker is promoted to
>> > 'leader') are able to go on after the reads broke the client.
>> Are writes coming from the same clients? Or clients in the same process?
>>
>
> Same o.a.b.c.BookKeeper object
>
>>
>> -Ivan
>>
> --
>
>
> -- Enrico Olivelli
>


Re: Help with bad errors on 4.6.1

2018-03-12 Thread Enrico Olivelli
Il lun 12 mar 2018, 20:40 Ivan Kelly  ha scritto:

> > It is interesting that the problems is on 'readers' and it seems that the
> > PCBC seems corrupted and even writes (if the broker is promoted to
> > 'leader') are able to go on after the reads broke the client.
> Are writes coming from the same clients? Or clients in the same process?
>

Same o.a.b.c.BookKeeper object

>
> -Ivan
>
-- 


-- Enrico Olivelli


Re: Help with bad errors on 4.6.1

2018-03-12 Thread Ivan Kelly
> It is interesting that the problems is on 'readers' and it seems that the
> PCBC seems corrupted and even writes (if the broker is promoted to
> 'leader') are able to go on after the reads broke the client.
Are writes coming from the same clients? Or clients in the same process?

-Ivan


Re: Help with bad errors on 4.6.1

2018-03-12 Thread Enrico Olivelli
Il lun 12 mar 2018, 19:37 Sijie Guo  ha scritto:

> Thanks Enrico!
>
> On Mon, Mar 12, 2018 at 4:21 AM, Enrico Olivelli 
> wrote:
>
> > Summary of my findings:
> >
> > The problem is about clients which get messed up and are not able to read
> > and write to bookies after rolling restarts of an application,
> > the problem appears only on a cluster of 6 machines (reduced to 3 in
> order
> > to narrow down the search) of my colleagues which are performing "manual"
> > tests on a new version of an application, no changes at BK client level
> are
> > present in the application
> >
>
> Does the problem only exist in that cluster? Or have you tried other
> machines with same setup?
>

I have only those machines which reproduce the issue. We have three other
similar clusters which do not have the problem. My colleagues are comparing
the machines but they are all on the same vmware subsystem, apparently the
only difference as far as I can know at the moment is that that cluster
have a higher application load, not directly on BK, but CPUs are busier.
My colleagues are excluding hardware issues but currently they are still
investigating.

It is interesting that there is a way to trigger the problem
systematically, that is by restarting one or at most two bookies.

It is interesting that the problems is on 'readers' and it seems that the
PCBC seems corrupted and even writes (if the broker is promoted to
'leader') are able to go on after the reads broke the client.

I was trying to reproduce the issue on my laptop without success.
I will do further investigation on BK code in next days


Stay tuned
Enrico


> As what you said, you didn't see this error in your production. I am
> wondering the corruption is coming from your hardware/os.
>
>
> - Sijie
>
>
>
> >
> > I am able to reproduce the problem quite simply on the those machines, it
> > is enough to restart bookies to mess the cluster, this is my reproducer:
> > 1) there is a client which is continuously writing to BK, on a WAL (same
> > pattern as in the BK tutorial)
> > 2) there are two other clients (other machines) which are 'tailing' the
> WAL
> > (as in the tutorial), these are the 'readers' which have the problem
> > 3) there are 3 bookies, co-located in the same machines as the clients
> > 4) running BK 4.6.1  on both clients and servers, using v3 protocol
> > 5) ledgers are written with WQ=2 and AQ=2 (over a cluster of 3)
> > 6) restart one or two bookies and the readers start being prevented to
> > read: readers start receiving invalid v2 responses (com.google.protobuf.
> > InvalidProtocolBufferException: Protocol message contained an invalid tag
> > (zero))  [before Sijie's patch this was "Invalid opcode..."]
> > 7) on bookie side we have the error below,
> > "com.google.protobuf.InvalidProtocolBufferException
> > $InvalidWireTypeException:
> > Protocol message tag had invalid wire type",
> > BookieProtoEncoding#RequestDecoder falls back to v2 protocol and start
> > answering with wrong encoding
> >
> > I am not able to create a reproducer on BK test suite.
> >
> > Notes:
> > - when I "restart" bookies I issue a kill -9 (I think this could be the
> > reason why I can't reproduce the issue on testcases)
> > - the issue starts when bookie comes up again
> > - when a reader is promoted to leader (writer) it is no more able to
> write
> > entries, all writes time out
> > - I am running on Linux with Epool, no Auth, no TLS
> > - Ivan took a look and some network dumps and found corrupted messages
> > (thank you Ivan !!)
> >
> > Beside Additional (not important) notes from the history of this email
> > thread:
> > - I have never seen this kind of errors in production, and we have been
> > running 4.6.1 in production for a month
> > - I am running with jdk9 but even reverting to jdk8 the problem is the
> same
> > - I was running with Netty 4.1.21Final but I have reverted to 4.1.12Final
> > and the problem is the same
> >
> > I think it may be an issue on PerChannelBookieClient, but to me it is not
> > very clear how "connection resets" are handled, I am digging into code.
> >
> > Bolow there is the stacktrace or the error on bookie side
> >
> > (line numbers may not match 4.6.1 tag because I have added plenty of
> debug,
> > but we are at the point when BookieProtoEncoding#RequestDecoder switches
> > to
> > v2 protocol)
> >
> > Thank you very much to all of you that is helping with this problem
> >
> > Enrico
> >
> >
> > 18-03-12-11-08-34error decoding msg PooledSlicedByteBuf(ridx: 145,
> > widx: 145, cap: 145/145, unwrapped: PooledUnsafeDirectByteBuf(ridx: 149,
> > widx: 298, cap: 65536))
> > 18-03-12-11-08-34
> > com.google.protobuf.InvalidProtocolBufferException
> > $InvalidWireTypeException:
> > Protocol message tag had invalid wire type.
> > com.google.protobuf.InvalidProtocolBufferException
> > $InvalidWireTypeException:
> > Protocol message tag had invalid wire type.
> > at
> > 

Re: Help with bad errors on 4.6.1

2018-03-12 Thread Sijie Guo
Thanks Enrico!

On Mon, Mar 12, 2018 at 4:21 AM, Enrico Olivelli 
wrote:

> Summary of my findings:
>
> The problem is about clients which get messed up and are not able to read
> and write to bookies after rolling restarts of an application,
> the problem appears only on a cluster of 6 machines (reduced to 3 in order
> to narrow down the search) of my colleagues which are performing "manual"
> tests on a new version of an application, no changes at BK client level are
> present in the application
>

Does the problem only exist in that cluster? Or have you tried other
machines with same setup?

As what you said, you didn't see this error in your production. I am
wondering the corruption is coming from your hardware/os.


- Sijie



>
> I am able to reproduce the problem quite simply on the those machines, it
> is enough to restart bookies to mess the cluster, this is my reproducer:
> 1) there is a client which is continuously writing to BK, on a WAL (same
> pattern as in the BK tutorial)
> 2) there are two other clients (other machines) which are 'tailing' the WAL
> (as in the tutorial), these are the 'readers' which have the problem
> 3) there are 3 bookies, co-located in the same machines as the clients
> 4) running BK 4.6.1  on both clients and servers, using v3 protocol
> 5) ledgers are written with WQ=2 and AQ=2 (over a cluster of 3)
> 6) restart one or two bookies and the readers start being prevented to
> read: readers start receiving invalid v2 responses (com.google.protobuf.
> InvalidProtocolBufferException: Protocol message contained an invalid tag
> (zero))  [before Sijie's patch this was "Invalid opcode..."]
> 7) on bookie side we have the error below,
> "com.google.protobuf.InvalidProtocolBufferException
> $InvalidWireTypeException:
> Protocol message tag had invalid wire type",
> BookieProtoEncoding#RequestDecoder falls back to v2 protocol and start
> answering with wrong encoding
>
> I am not able to create a reproducer on BK test suite.
>
> Notes:
> - when I "restart" bookies I issue a kill -9 (I think this could be the
> reason why I can't reproduce the issue on testcases)
> - the issue starts when bookie comes up again
> - when a reader is promoted to leader (writer) it is no more able to write
> entries, all writes time out
> - I am running on Linux with Epool, no Auth, no TLS
> - Ivan took a look and some network dumps and found corrupted messages
> (thank you Ivan !!)
>
> Beside Additional (not important) notes from the history of this email
> thread:
> - I have never seen this kind of errors in production, and we have been
> running 4.6.1 in production for a month
> - I am running with jdk9 but even reverting to jdk8 the problem is the same
> - I was running with Netty 4.1.21Final but I have reverted to 4.1.12Final
> and the problem is the same
>
> I think it may be an issue on PerChannelBookieClient, but to me it is not
> very clear how "connection resets" are handled, I am digging into code.
>
> Bolow there is the stacktrace or the error on bookie side
>
> (line numbers may not match 4.6.1 tag because I have added plenty of debug,
> but we are at the point when BookieProtoEncoding#RequestDecoder switches
> to
> v2 protocol)
>
> Thank you very much to all of you that is helping with this problem
>
> Enrico
>
>
> 18-03-12-11-08-34error decoding msg PooledSlicedByteBuf(ridx: 145,
> widx: 145, cap: 145/145, unwrapped: PooledUnsafeDirectByteBuf(ridx: 149,
> widx: 298, cap: 65536))
> 18-03-12-11-08-34
> com.google.protobuf.InvalidProtocolBufferException
> $InvalidWireTypeException:
> Protocol message tag had invalid wire type.
> com.google.protobuf.InvalidProtocolBufferException
> $InvalidWireTypeException:
> Protocol message tag had invalid wire type.
> at
> com.google.protobuf.InvalidProtocolBufferException.invalidWireType(
> InvalidProtocolBufferException.java:115)
> at
> com.google.protobuf.UnknownFieldSet$Builder.mergeFieldFrom(
> UnknownFieldSet.java:551)
> at
> com.google.protobuf.GeneratedMessageV3.parseUnknownField(
> GeneratedMessageV3.java:293)
> at
> org.apache.bookkeeper.proto.BookkeeperProtocol$AddRequest.
> (BookkeeperProtocol.java:4536)
> at
> org.apache.bookkeeper.proto.BookkeeperProtocol$AddRequest.
> (BookkeeperProtocol.java:4497)
> at
> org.apache.bookkeeper.proto.BookkeeperProtocol$AddRequest$
> 1.parsePartialFrom(BookkeeperProtocol.java:5378)
> at
> org.apache.bookkeeper.proto.BookkeeperProtocol$AddRequest$
> 1.parsePartialFrom(BookkeeperProtocol.java:5373)
> at
> com.google.protobuf.CodedInputStream$StreamDecoder.readMessage(
> CodedInputStream.java:2362)
> at
> org.apache.bookkeeper.proto.BookkeeperProtocol$Request.<
> init>(BookkeeperProtocol.java:1367)
> at
> org.apache.bookkeeper.proto.BookkeeperProtocol$Request.<
> init>(BookkeeperProtocol.java:1296)
> at
> org.apache.bookkeeper.proto.BookkeeperProtocol$Request$1.parsePartialFrom(
> BookkeeperProtocol.java:3265)
> at
> 

Re: Help with bad errors on 4.6.1

2018-03-12 Thread Ivan Kelly
> - when I "restart" bookies I issue a kill -9 (I think this could be the
> reason why I can't reproduce the issue on testcases)
With a clean shutdown of bookies we close the channels, and it should
do the tcp shutdown handshake. -9 will kill the process before it gets
to do any of that, but the kernel will kill the socket. Pretty sure it
doesn't do the fin-ack handshake. I suspect that netty should be
throwing up a different exception in this case.

You can't do a kill -9 with unit tests, but with integration tests
it's possible.

-Ivan


Re: Help with bad errors on 4.6.1

2018-03-12 Thread Enrico Olivelli
Summary of my findings:

The problem is about clients which get messed up and are not able to read
and write to bookies after rolling restarts of an application,
the problem appears only on a cluster of 6 machines (reduced to 3 in order
to narrow down the search) of my colleagues which are performing "manual"
tests on a new version of an application, no changes at BK client level are
present in the application

I am able to reproduce the problem quite simply on the those machines, it
is enough to restart bookies to mess the cluster, this is my reproducer:
1) there is a client which is continuously writing to BK, on a WAL (same
pattern as in the BK tutorial)
2) there are two other clients (other machines) which are 'tailing' the WAL
(as in the tutorial), these are the 'readers' which have the problem
3) there are 3 bookies, co-located in the same machines as the clients
4) running BK 4.6.1  on both clients and servers, using v3 protocol
5) ledgers are written with WQ=2 and AQ=2 (over a cluster of 3)
6) restart one or two bookies and the readers start being prevented to
read: readers start receiving invalid v2 responses (com.google.protobuf.
InvalidProtocolBufferException: Protocol message contained an invalid tag
(zero))  [before Sijie's patch this was "Invalid opcode..."]
7) on bookie side we have the error below,
"com.google.protobuf.InvalidProtocolBufferException$InvalidWireTypeException:
Protocol message tag had invalid wire type",
BookieProtoEncoding#RequestDecoder falls back to v2 protocol and start
answering with wrong encoding

I am not able to create a reproducer on BK test suite.

Notes:
- when I "restart" bookies I issue a kill -9 (I think this could be the
reason why I can't reproduce the issue on testcases)
- the issue starts when bookie comes up again
- when a reader is promoted to leader (writer) it is no more able to write
entries, all writes time out
- I am running on Linux with Epool, no Auth, no TLS
- Ivan took a look and some network dumps and found corrupted messages
(thank you Ivan !!)

Beside Additional (not important) notes from the history of this email
thread:
- I have never seen this kind of errors in production, and we have been
running 4.6.1 in production for a month
- I am running with jdk9 but even reverting to jdk8 the problem is the same
- I was running with Netty 4.1.21Final but I have reverted to 4.1.12Final
and the problem is the same

I think it may be an issue on PerChannelBookieClient, but to me it is not
very clear how "connection resets" are handled, I am digging into code.

Bolow there is the stacktrace or the error on bookie side

(line numbers may not match 4.6.1 tag because I have added plenty of debug,
but we are at the point when BookieProtoEncoding#RequestDecoder switches to
v2 protocol)

Thank you very much to all of you that is helping with this problem

Enrico


18-03-12-11-08-34error decoding msg PooledSlicedByteBuf(ridx: 145,
widx: 145, cap: 145/145, unwrapped: PooledUnsafeDirectByteBuf(ridx: 149,
widx: 298, cap: 65536))
18-03-12-11-08-34
com.google.protobuf.InvalidProtocolBufferException$InvalidWireTypeException:
Protocol message tag had invalid wire type.
com.google.protobuf.InvalidProtocolBufferException$InvalidWireTypeException:
Protocol message tag had invalid wire type.
at
com.google.protobuf.InvalidProtocolBufferException.invalidWireType(InvalidProtocolBufferException.java:115)
at
com.google.protobuf.UnknownFieldSet$Builder.mergeFieldFrom(UnknownFieldSet.java:551)
at
com.google.protobuf.GeneratedMessageV3.parseUnknownField(GeneratedMessageV3.java:293)
at
org.apache.bookkeeper.proto.BookkeeperProtocol$AddRequest.(BookkeeperProtocol.java:4536)
at
org.apache.bookkeeper.proto.BookkeeperProtocol$AddRequest.(BookkeeperProtocol.java:4497)
at
org.apache.bookkeeper.proto.BookkeeperProtocol$AddRequest$1.parsePartialFrom(BookkeeperProtocol.java:5378)
at
org.apache.bookkeeper.proto.BookkeeperProtocol$AddRequest$1.parsePartialFrom(BookkeeperProtocol.java:5373)
at
com.google.protobuf.CodedInputStream$StreamDecoder.readMessage(CodedInputStream.java:2362)
at
org.apache.bookkeeper.proto.BookkeeperProtocol$Request.(BookkeeperProtocol.java:1367)
at
org.apache.bookkeeper.proto.BookkeeperProtocol$Request.(BookkeeperProtocol.java:1296)
at
org.apache.bookkeeper.proto.BookkeeperProtocol$Request$1.parsePartialFrom(BookkeeperProtocol.java:3265)
at
org.apache.bookkeeper.proto.BookkeeperProtocol$Request$1.parsePartialFrom(BookkeeperProtocol.java:3260)
at
com.google.protobuf.AbstractParser.parsePartialFrom(AbstractParser.java:221)
at com.google.protobuf.AbstractParser.parseFrom(AbstractParser.java:239)
at com.google.protobuf.AbstractParser.parseFrom(AbstractParser.java:49)
at
com.google.protobuf.GeneratedMessageV3.parseWithIOException(GeneratedMessageV3.java:320)
at
org.apache.bookkeeper.proto.BookkeeperProtocol$Request.parseFrom(BookkeeperProtocol.java:1905)
at

Re: Help with bad errors on 4.6.1

2018-03-12 Thread Enrico Olivelli
I will send a report soon.
With new debug I have some finding, I am looking into problems during
restarts of bookies. Maybe there is some problem in error handling in PCBC.

Thank you
Enrico

2018-03-12 10:58 GMT+01:00 Ivan Kelly :

> Enrico, could you summarize what the state of things is now? What are
> you running, what problems are you seeing and how are the problems
> manifesting themselves.
>
> Regards,
> Ivan
>
> On Mon, Mar 12, 2018 at 10:15 AM, Enrico Olivelli 
> wrote:
> > Applyed Sijie's fixes and added some debug:
> >
> > Problem is triggered when you restart a bookie (I have a cluster of 3
> > bookies, WQ = 2 and AQ = 2)
> >
> > Below a new error on client side ("tailing" reader)
> >
> > Enrico
> >
> > this is a new error on client side:
> > 18-03-12-09-11-45Unexpected exception caught by bookie client channel
> > handler
> > 18-03-12-09-11-45io.netty.handler.codec.DecoderException:
> > com.google.protobuf.InvalidProtocolBufferException: Protocol message
> > contained an invalid tag (zero).
> > io.netty.handler.codec.DecoderException:
> > com.google.protobuf.InvalidProtocolBufferException: Protocol message
> > contained an invalid tag (zero).
> > at
> > io.netty.handler.codec.MessageToMessageDecoder.channelRead(
> MessageToMessageDecoder.java:98)
> > at
> > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(
> AbstractChannelHandlerContext.java:362)
> > at
> > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(
> AbstractChannelHandlerContext.java:348)
> > at
> > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(
> AbstractChannelHandlerContext.java:340)
> > at
> > io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(
> ByteToMessageDecoder.java:310)
> > at
> > io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(
> ByteToMessageDecoder.java:297)
> > at
> > io.netty.handler.codec.ByteToMessageDecoder.callDecode(
> ByteToMessageDecoder.java:413)
> > at
> > io.netty.handler.codec.ByteToMessageDecoder.channelRead(
> ByteToMessageDecoder.java:265)
> > at
> > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(
> AbstractChannelHandlerContext.java:362)
> > at
> > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(
> AbstractChannelHandlerContext.java:348)
> > at
> > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(
> AbstractChannelHandlerContext.java:340)
> > at
> > io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(
> DefaultChannelPipeline.java:1334)
> > at
> > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(
> AbstractChannelHandlerContext.java:362)
> > at
> > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(
> AbstractChannelHandlerContext.java:348)
> > at
> > io.netty.channel.DefaultChannelPipeline.fireChannelRead(
> DefaultChannelPipeline.java:926)
> > at
> > io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.
> epollInReady(AbstractEpollStreamChannel.java:979)
> > at
> > io.netty.channel.epoll.EpollEventLoop.processReady(
> EpollEventLoop.java:401)
> > at io.netty.channel.epoll.EpollEventLoop.run(
> EpollEventLoop.java:306)
> > at
> > io.netty.util.concurrent.SingleThreadEventExecutor$5.
> run(SingleThreadEventExecutor.java:858)
> > at java.base/java.lang.Thread.run(Thread.java:844)
> > Caused by: com.google.protobuf.InvalidProtocolBufferException: Protocol
> > message contained an invalid tag (zero).
> > at
> > com.google.protobuf.InvalidProtocolBufferException.invalidTag(
> InvalidProtocolBufferException.java:105)
> > at
> > com.google.protobuf.CodedInputStream$StreamDecoder.readTag(
> CodedInputStream.java:2060)
> > at
> > org.apache.bookkeeper.proto.BookkeeperProtocol$ReadResponse.(
> BookkeeperProtocol.java:10009)
> > at
> > org.apache.bookkeeper.proto.BookkeeperProtocol$ReadResponse.(
> BookkeeperProtocol.java:9975)
> > at
> > org.apache.bookkeeper.proto.BookkeeperProtocol$ReadResponse$1.
> parsePartialFrom(BookkeeperProtocol.java:10869)
> > at
> > org.apache.bookkeeper.proto.BookkeeperProtocol$ReadResponse$1.
> parsePartialFrom(BookkeeperProtocol.java:10864)
> > at
> > com.google.protobuf.CodedInputStream$StreamDecoder.readMessage(
> CodedInputStream.java:2362)
> > at
> > org.apache.bookkeeper.proto.BookkeeperProtocol$Response.<
> init>(BookkeeperProtocol.java:7852)
> > at
> > org.apache.bookkeeper.proto.BookkeeperProtocol$Response.<
> init>(BookkeeperProtocol.java:7782)
> > at
> > org.apache.bookkeeper.proto.BookkeeperProtocol$Response$1.
> parsePartialFrom(BookkeeperProtocol.java:9887)
> > at
> > org.apache.bookkeeper.proto.BookkeeperProtocol$Response$1.
> parsePartialFrom(BookkeeperProtocol.java:9882)
> > at
> > com.google.protobuf.AbstractParser.parsePartialFrom(
> AbstractParser.java:221)
> > at com.google.protobuf.AbstractParser.parseFrom(
> 

Re: Help with bad errors on 4.6.1

2018-03-12 Thread Ivan Kelly
Enrico, could you summarize what the state of things is now? What are
you running, what problems are you seeing and how are the problems
manifesting themselves.

Regards,
Ivan

On Mon, Mar 12, 2018 at 10:15 AM, Enrico Olivelli  wrote:
> Applyed Sijie's fixes and added some debug:
>
> Problem is triggered when you restart a bookie (I have a cluster of 3
> bookies, WQ = 2 and AQ = 2)
>
> Below a new error on client side ("tailing" reader)
>
> Enrico
>
> this is a new error on client side:
> 18-03-12-09-11-45Unexpected exception caught by bookie client channel
> handler
> 18-03-12-09-11-45io.netty.handler.codec.DecoderException:
> com.google.protobuf.InvalidProtocolBufferException: Protocol message
> contained an invalid tag (zero).
> io.netty.handler.codec.DecoderException:
> com.google.protobuf.InvalidProtocolBufferException: Protocol message
> contained an invalid tag (zero).
> at
> io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:98)
> at
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362)
> at
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348)
> at
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:340)
> at
> io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:310)
> at
> io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:297)
> at
> io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:413)
> at
> io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:265)
> at
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362)
> at
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348)
> at
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:340)
> at
> io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1334)
> at
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362)
> at
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348)
> at
> io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:926)
> at
> io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.epollInReady(AbstractEpollStreamChannel.java:979)
> at
> io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:401)
> at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:306)
> at
> io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858)
> at java.base/java.lang.Thread.run(Thread.java:844)
> Caused by: com.google.protobuf.InvalidProtocolBufferException: Protocol
> message contained an invalid tag (zero).
> at
> com.google.protobuf.InvalidProtocolBufferException.invalidTag(InvalidProtocolBufferException.java:105)
> at
> com.google.protobuf.CodedInputStream$StreamDecoder.readTag(CodedInputStream.java:2060)
> at
> org.apache.bookkeeper.proto.BookkeeperProtocol$ReadResponse.(BookkeeperProtocol.java:10009)
> at
> org.apache.bookkeeper.proto.BookkeeperProtocol$ReadResponse.(BookkeeperProtocol.java:9975)
> at
> org.apache.bookkeeper.proto.BookkeeperProtocol$ReadResponse$1.parsePartialFrom(BookkeeperProtocol.java:10869)
> at
> org.apache.bookkeeper.proto.BookkeeperProtocol$ReadResponse$1.parsePartialFrom(BookkeeperProtocol.java:10864)
> at
> com.google.protobuf.CodedInputStream$StreamDecoder.readMessage(CodedInputStream.java:2362)
> at
> org.apache.bookkeeper.proto.BookkeeperProtocol$Response.(BookkeeperProtocol.java:7852)
> at
> org.apache.bookkeeper.proto.BookkeeperProtocol$Response.(BookkeeperProtocol.java:7782)
> at
> org.apache.bookkeeper.proto.BookkeeperProtocol$Response$1.parsePartialFrom(BookkeeperProtocol.java:9887)
> at
> org.apache.bookkeeper.proto.BookkeeperProtocol$Response$1.parsePartialFrom(BookkeeperProtocol.java:9882)
> at
> com.google.protobuf.AbstractParser.parsePartialFrom(AbstractParser.java:221)
> at com.google.protobuf.AbstractParser.parseFrom(AbstractParser.java:239)
> at com.google.protobuf.AbstractParser.parseFrom(AbstractParser.java:49)
> at
> com.google.protobuf.GeneratedMessageV3.parseWithIOException(GeneratedMessageV3.java:320)
> at
> org.apache.bookkeeper.proto.BookkeeperProtocol$Response.parseFrom(BookkeeperProtocol.java:8454)
> at
> org.apache.bookkeeper.proto.BookieProtoEncoding$ResponseEnDecoderV3.decode(BookieProtoEncoding.java:329)
> at
> org.apache.bookkeeper.proto.BookieProtoEncoding$ResponseDecoder.decode(BookieProtoEncoding.java:470)
> 

Re: Help with bad errors on 4.6.1

2018-03-12 Thread Enrico Olivelli
Applyed Sijie's fixes and added some debug:

Problem is triggered when you restart a bookie (I have a cluster of 3
bookies, WQ = 2 and AQ = 2)

Below a new error on client side ("tailing" reader)

Enrico

this is a new error on client side:
18-03-12-09-11-45Unexpected exception caught by bookie client channel
handler
18-03-12-09-11-45io.netty.handler.codec.DecoderException:
com.google.protobuf.InvalidProtocolBufferException: Protocol message
contained an invalid tag (zero).
io.netty.handler.codec.DecoderException:
com.google.protobuf.InvalidProtocolBufferException: Protocol message
contained an invalid tag (zero).
at
io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:98)
at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362)
at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348)
at
io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:340)
at
io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:310)
at
io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:297)
at
io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:413)
at
io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:265)
at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362)
at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348)
at
io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:340)
at
io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1334)
at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362)
at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348)
at
io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:926)
at
io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.epollInReady(AbstractEpollStreamChannel.java:979)
at
io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:401)
at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:306)
at
io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858)
at java.base/java.lang.Thread.run(Thread.java:844)
Caused by: com.google.protobuf.InvalidProtocolBufferException: Protocol
message contained an invalid tag (zero).
at
com.google.protobuf.InvalidProtocolBufferException.invalidTag(InvalidProtocolBufferException.java:105)
at
com.google.protobuf.CodedInputStream$StreamDecoder.readTag(CodedInputStream.java:2060)
at
org.apache.bookkeeper.proto.BookkeeperProtocol$ReadResponse.(BookkeeperProtocol.java:10009)
at
org.apache.bookkeeper.proto.BookkeeperProtocol$ReadResponse.(BookkeeperProtocol.java:9975)
at
org.apache.bookkeeper.proto.BookkeeperProtocol$ReadResponse$1.parsePartialFrom(BookkeeperProtocol.java:10869)
at
org.apache.bookkeeper.proto.BookkeeperProtocol$ReadResponse$1.parsePartialFrom(BookkeeperProtocol.java:10864)
at
com.google.protobuf.CodedInputStream$StreamDecoder.readMessage(CodedInputStream.java:2362)
at
org.apache.bookkeeper.proto.BookkeeperProtocol$Response.(BookkeeperProtocol.java:7852)
at
org.apache.bookkeeper.proto.BookkeeperProtocol$Response.(BookkeeperProtocol.java:7782)
at
org.apache.bookkeeper.proto.BookkeeperProtocol$Response$1.parsePartialFrom(BookkeeperProtocol.java:9887)
at
org.apache.bookkeeper.proto.BookkeeperProtocol$Response$1.parsePartialFrom(BookkeeperProtocol.java:9882)
at
com.google.protobuf.AbstractParser.parsePartialFrom(AbstractParser.java:221)
at com.google.protobuf.AbstractParser.parseFrom(AbstractParser.java:239)
at com.google.protobuf.AbstractParser.parseFrom(AbstractParser.java:49)
at
com.google.protobuf.GeneratedMessageV3.parseWithIOException(GeneratedMessageV3.java:320)
at
org.apache.bookkeeper.proto.BookkeeperProtocol$Response.parseFrom(BookkeeperProtocol.java:8454)
at
org.apache.bookkeeper.proto.BookieProtoEncoding$ResponseEnDecoderV3.decode(BookieProtoEncoding.java:329)
at
org.apache.bookkeeper.proto.BookieProtoEncoding$ResponseDecoder.decode(BookieProtoEncoding.java:470)
at
io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:88)
... 19 more


2018-03-09 22:37 GMT+01:00 Sijie Guo :

> Enrico,
>
> I would suggest you applied my fixes and then debug from there. In this
> way, you will have a better sense where the first corruption is from.
>
> Sijie
>
> On Fri, Mar 9, 2018 at 11:48 AM Enrico Olivelli 
> wrote:
>
> > Il ven 9 

Re: Help with bad errors on 4.6.1

2018-03-09 Thread Sijie Guo
Enrico,

I would suggest you applied my fixes and then debug from there. In this
way, you will have a better sense where the first corruption is from.

Sijie

On Fri, Mar 9, 2018 at 11:48 AM Enrico Olivelli  wrote:

> Il ven 9 mar 2018, 19:30 Enrico Olivelli  ha scritto:
>
> > Thank you Ivan!
> > I hope I did not mess up the dump and added ZK ports.  We are not using
> > standard ports and in that 3  machines there is also the 3 nodes zk
> > ensemble which is supporting BK and all the other parts of the
> application
> >
> > So one explanation would be that something is connecting to the bookie
> and
> > this makes the bookie switch in a corrupted state by double releasing a
> > bytebuf?
> >
>
> I did some experiments and it is easy to reproduce the bookie side error
> and the double release with a forged sequence of bytes (just using nc from
> the shell)
> But this seems not enough to break the bookie.
> I guess there is some corruption on client side and the error on the bookie
> is only and effect, as Ivan is saying.
> My colleagues left the system running  with a deep level of debug during
> next weekend, hopefully we will get some other stacktrace
>
> Enrico
>
>
> > Enrico
> >
> >
> > Il ven 9 mar 2018, 18:23 Ivan Kelly  ha scritto:
> >
> >> I need to sign off for the day. I've done some analysis of a tcpdump
> >> enrico sent to me out of band (may contain sensitive info so best not
> >> to post on public forum).
> >>
> >> I've attached a dump of just first bit of the header. Format is
> >>  ()
> >> 
> >>
> >> There are definitely corrupt packets coming from somewhere. Search for
> >> lines with CORRUPT.
> >>
> >> 
> >> 0247 -  req (049546) - 00:00:00:08:ff:ff:ff:fe:00:00:00:0bCORRUPT
> >> 
> >>
> >> It's not clear whether these are originating at a valid client or not.
> >> These trigger corrupt responses from the server, which I guess is the
> >> double free manifesting itself. Strangely the
> >> corrupt message seems to have a lot of data in common with what seems
> >> like an ok message (it's clearer on fixed width font).
> >>
> >> 
> >> 0248 -  resp(049720) -
> >>
> >>
> 00:00:00:54:00:03:00:89:00:00:02:86:00:07:e2:b1:00:00:00:00:00:00:02:86:00:05:e9:76:00:00
> >> 0249 -  resp(049546) -
> >> 00:00:00:10:ff:ff:ff:fe:00:00:02:86:00:07:e2:b1:00:00:00:00CORRUPT
> >> 
> >>
> >> There's also some other weird traffic. Correct BK protobuf traffic
> >> should be <4 bytes len>:00:03:
> >> There seems to be other traffic which is being accepted at the same
> >> port, but looks like ZK traffic.
> >>
> >> Anyhow, I'll dig more on monday.
> >>
> >> -Ivan
> >>
> >>
> >> On Fri, Mar 9, 2018 at 3:27 PM, Ivan Kelly  wrote:
> >> > On Fri, Mar 9, 2018 at 3:20 PM, Enrico Olivelli 
> >> wrote:
> >> >> Bookies
> >> >> 10.168.10.117:1822 -> bad bookie with 4.1.21
> >> >> 10.168.10.116:1822 -> bookie with 4.1.12
> >> >> 10.168.10.118:1281 -> bookie with 4.1.12
> >> >>
> >> >> 10.168.10.117 client machine on which I have 4.1.21 client (different
> >> >> process than the bookie one)
> >> > Oh. This dump won't have the stream we need then, as that will be on
> >> > loopback. Try adding "-i any" to the tcpdump. Sorry, I didn't realize
> >> > your clients and servers are colocated.
> >> >
> >> > -Ivan
> >>
> > --
> >
> >
> > -- Enrico Olivelli
> >
> --
>
>
> -- Enrico Olivelli
>


Re: Help with bad errors on 4.6.1

2018-03-09 Thread Enrico Olivelli
Il ven 9 mar 2018, 19:30 Enrico Olivelli  ha scritto:

> Thank you Ivan!
> I hope I did not mess up the dump and added ZK ports.  We are not using
> standard ports and in that 3  machines there is also the 3 nodes zk
> ensemble which is supporting BK and all the other parts of the application
>
> So one explanation would be that something is connecting to the bookie and
> this makes the bookie switch in a corrupted state by double releasing a
> bytebuf?
>

I did some experiments and it is easy to reproduce the bookie side error
and the double release with a forged sequence of bytes (just using nc from
the shell)
But this seems not enough to break the bookie.
I guess there is some corruption on client side and the error on the bookie
is only and effect, as Ivan is saying.
My colleagues left the system running  with a deep level of debug during
next weekend, hopefully we will get some other stacktrace

Enrico


> Enrico
>
>
> Il ven 9 mar 2018, 18:23 Ivan Kelly  ha scritto:
>
>> I need to sign off for the day. I've done some analysis of a tcpdump
>> enrico sent to me out of band (may contain sensitive info so best not
>> to post on public forum).
>>
>> I've attached a dump of just first bit of the header. Format is
>>  ()
>> 
>>
>> There are definitely corrupt packets coming from somewhere. Search for
>> lines with CORRUPT.
>>
>> 
>> 0247 -  req (049546) - 00:00:00:08:ff:ff:ff:fe:00:00:00:0bCORRUPT
>> 
>>
>> It's not clear whether these are originating at a valid client or not.
>> These trigger corrupt responses from the server, which I guess is the
>> double free manifesting itself. Strangely the
>> corrupt message seems to have a lot of data in common with what seems
>> like an ok message (it's clearer on fixed width font).
>>
>> 
>> 0248 -  resp(049720) -
>>
>> 00:00:00:54:00:03:00:89:00:00:02:86:00:07:e2:b1:00:00:00:00:00:00:02:86:00:05:e9:76:00:00
>> 0249 -  resp(049546) -
>> 00:00:00:10:ff:ff:ff:fe:00:00:02:86:00:07:e2:b1:00:00:00:00CORRUPT
>> 
>>
>> There's also some other weird traffic. Correct BK protobuf traffic
>> should be <4 bytes len>:00:03:
>> There seems to be other traffic which is being accepted at the same
>> port, but looks like ZK traffic.
>>
>> Anyhow, I'll dig more on monday.
>>
>> -Ivan
>>
>>
>> On Fri, Mar 9, 2018 at 3:27 PM, Ivan Kelly  wrote:
>> > On Fri, Mar 9, 2018 at 3:20 PM, Enrico Olivelli 
>> wrote:
>> >> Bookies
>> >> 10.168.10.117:1822 -> bad bookie with 4.1.21
>> >> 10.168.10.116:1822 -> bookie with 4.1.12
>> >> 10.168.10.118:1281 -> bookie with 4.1.12
>> >>
>> >> 10.168.10.117 client machine on which I have 4.1.21 client (different
>> >> process than the bookie one)
>> > Oh. This dump won't have the stream we need then, as that will be on
>> > loopback. Try adding "-i any" to the tcpdump. Sorry, I didn't realize
>> > your clients and servers are colocated.
>> >
>> > -Ivan
>>
> --
>
>
> -- Enrico Olivelli
>
-- 


-- Enrico Olivelli


Re: Help with bad errors on 4.6.1

2018-03-09 Thread Enrico Olivelli
Thank you Ivan!
I hope I did not mess up the dump and added ZK ports.  We are not using
standard ports and in that 3  machines there is also the 3 nodes zk
ensemble which is supporting BK and all the other parts of the application

So one explanation would be that something is connecting to the bookie and
this makes the bookie switch in a corrupted state by double releasing a
bytebuf?

Enrico

Il ven 9 mar 2018, 18:23 Ivan Kelly  ha scritto:

> I need to sign off for the day. I've done some analysis of a tcpdump
> enrico sent to me out of band (may contain sensitive info so best not
> to post on public forum).
>
> I've attached a dump of just first bit of the header. Format is
>  ()
> 
>
> There are definitely corrupt packets coming from somewhere. Search for
> lines with CORRUPT.
>
> 
> 0247 -  req (049546) - 00:00:00:08:ff:ff:ff:fe:00:00:00:0bCORRUPT
> 
>
> It's not clear whether these are originating at a valid client or not.
> These trigger corrupt responses from the server, which I guess is the
> double free manifesting itself. Strangely the
> corrupt message seems to have a lot of data in common with what seems
> like an ok message (it's clearer on fixed width font).
>
> 
> 0248 -  resp(049720) -
>
> 00:00:00:54:00:03:00:89:00:00:02:86:00:07:e2:b1:00:00:00:00:00:00:02:86:00:05:e9:76:00:00
> 0249 -  resp(049546) -
> 00:00:00:10:ff:ff:ff:fe:00:00:02:86:00:07:e2:b1:00:00:00:00CORRUPT
> 
>
> There's also some other weird traffic. Correct BK protobuf traffic
> should be <4 bytes len>:00:03:
> There seems to be other traffic which is being accepted at the same
> port, but looks like ZK traffic.
>
> Anyhow, I'll dig more on monday.
>
> -Ivan
>
>
> On Fri, Mar 9, 2018 at 3:27 PM, Ivan Kelly  wrote:
> > On Fri, Mar 9, 2018 at 3:20 PM, Enrico Olivelli 
> wrote:
> >> Bookies
> >> 10.168.10.117:1822 -> bad bookie with 4.1.21
> >> 10.168.10.116:1822 -> bookie with 4.1.12
> >> 10.168.10.118:1281 -> bookie with 4.1.12
> >>
> >> 10.168.10.117 client machine on which I have 4.1.21 client (different
> >> process than the bookie one)
> > Oh. This dump won't have the stream we need then, as that will be on
> > loopback. Try adding "-i any" to the tcpdump. Sorry, I didn't realize
> > your clients and servers are colocated.
> >
> > -Ivan
>
-- 


-- Enrico Olivelli


Re: Help with bad errors on 4.6.1

2018-03-09 Thread Ivan Kelly
On Fri, Mar 9, 2018 at 3:20 PM, Enrico Olivelli  wrote:
> Bookies
> 10.168.10.117:1822 -> bad bookie with 4.1.21
> 10.168.10.116:1822 -> bookie with 4.1.12
> 10.168.10.118:1281 -> bookie with 4.1.12
>
> 10.168.10.117 client machine on which I have 4.1.21 client (different
> process than the bookie one)
Oh. This dump won't have the stream we need then, as that will be on
loopback. Try adding "-i any" to the tcpdump. Sorry, I didn't realize
your clients and servers are colocated.

-Ivan


Re: Help with bad errors on 4.6.1

2018-03-09 Thread Enrico Olivelli
Bookies
10.168.10.117:1822 -> bad bookie with 4.1.21
10.168.10.116:1822 -> bookie with 4.1.12
10.168.10.118:1281 -> bookie with 4.1.12

10.168.10.117 client machine on which I have 4.1.21 client (different
process than the bookie one)

Thanks
Enrico




2018-03-09 15:16 GMT+01:00 Ivan Kelly :

> On Fri, Mar 9, 2018 at 3:13 PM, Enrico Olivelli 
> wrote:
> > New dump,
> > sequence (simpler)
> >
> > 1) system is running, reader is reading without errors with netty 4.1.21
> > 2) 3 bookies, one is with 4.1.21 and the other ones with 4.1.12
> > 3) kill one bookie with 4.1.12, the reader starts reading from the bookie
> > with 4.1.21
> > 4) client messes up, unrecoverably
>
> What port is bookie running on? And the ip of the client?
>
> -Ivan
>


Re: Help with bad errors on 4.6.1

2018-03-09 Thread Ivan Kelly
Also, do you have the logs of the error occurring on the server side?

-Ivan

On Fri, Mar 9, 2018 at 3:16 PM, Ivan Kelly  wrote:
> On Fri, Mar 9, 2018 at 3:13 PM, Enrico Olivelli  wrote:
>> New dump,
>> sequence (simpler)
>>
>> 1) system is running, reader is reading without errors with netty 4.1.21
>> 2) 3 bookies, one is with 4.1.21 and the other ones with 4.1.12
>> 3) kill one bookie with 4.1.12, the reader starts reading from the bookie
>> with 4.1.21
>> 4) client messes up, unrecoverably
>
> What port is bookie running on? And the ip of the client?
>
> -Ivan


Re: Help with bad errors on 4.6.1

2018-03-09 Thread Ivan Kelly
On Fri, Mar 9, 2018 at 3:13 PM, Enrico Olivelli  wrote:
> New dump,
> sequence (simpler)
>
> 1) system is running, reader is reading without errors with netty 4.1.21
> 2) 3 bookies, one is with 4.1.21 and the other ones with 4.1.12
> 3) kill one bookie with 4.1.12, the reader starts reading from the bookie
> with 4.1.21
> 4) client messes up, unrecoverably

What port is bookie running on? And the ip of the client?

-Ivan


Re: Help with bad errors on 4.6.1

2018-03-09 Thread Enrico Olivelli
New dump,
sequence (simpler)

1) system is running, reader is reading without errors with netty 4.1.21
2) 3 bookies, one is with 4.1.21 and the other ones with 4.1.12
3) kill one bookie with 4.1.12, the reader starts reading from the bookie
with 4.1.21
4) client messes up, unrecoverably

Enrico


2018-03-09 15:04 GMT+01:00 Enrico Olivelli :

> This is the dump
>
> in order to reproduce the issue I did these steps:
> - all bookies with 4.1.12
> - start reader with netty 4.1.21 -> still no issue
> - start on bookie with 4.1.21 -> start issue
> - bookie again with 4.1.12 -> client messed up, unrecoverable
>
> a strange fact is that I cannot find IllegalReferenceCount errors on
> bookies
>
> I will try different combinations in order to create a simpler reproducer
>
> that you for you help
>
> Enrico
>
>
>
> 2018-03-09 14:27 GMT+01:00 Enrico Olivelli :
>
>>
>>
>> Il ven 9 mar 2018, 14:12 Ivan Kelly  ha scritto:
>>
>>> > Any suggestion on the tcpdump config ? (command line example)
>>>
>>> sudo tcpdump -s 200 -w blah.pcap 'tcp port 3181'
>>>
>>> Where are you going to change the netty? client or server or both?
>>>
>>
>> Both, as the application is packaged as a single bundle.
>>  I will try to narrow the changes and change only one component at a time
>>
>> Enrico
>>
>>>
>>> -Ivan
>>>
>> --
>>
>>
>> -- Enrico Olivelli
>>
>
>


now.pcap
Description: application/vnd.tcpdump.pcap


Re: Help with bad errors on 4.6.1

2018-03-09 Thread Ivan Kelly
I've asked enrico to run again, as this dump doesn't span the time
when the issue started occurring.
What I'm looking for is to be able to inspect the first packet which
triggers the version downgrade of the decoders.


On Fri, Mar 9, 2018 at 3:04 PM, Enrico Olivelli  wrote:
> This is the dump
>
> in order to reproduce the issue I did these steps:
> - all bookies with 4.1.12
> - start reader with netty 4.1.21 -> still no issue
> - start on bookie with 4.1.21 -> start issue
> - bookie again with 4.1.12 -> client messed up, unrecoverable
>
> a strange fact is that I cannot find IllegalReferenceCount errors on bookies
>
> I will try different combinations in order to create a simpler reproducer
>
> that you for you help


Re: Help with bad errors on 4.6.1

2018-03-09 Thread Enrico Olivelli
Il ven 9 mar 2018, 14:12 Ivan Kelly  ha scritto:

> > Any suggestion on the tcpdump config ? (command line example)
>
> sudo tcpdump -s 200 -w blah.pcap 'tcp port 3181'
>
> Where are you going to change the netty? client or server or both?
>

Both, as the application is packaged as a single bundle.
 I will try to narrow the changes and change only one component at a time

Enrico

>
> -Ivan
>
-- 


-- Enrico Olivelli


Re: Help with bad errors on 4.6.1

2018-03-09 Thread Ivan Kelly
> Any suggestion on the tcpdump config ? (command line example)

sudo tcpdump -s 200 -w blah.pcap 'tcp port 3181'

Where are you going to change the netty? client or server or both?

-Ivan


Re: Help with bad errors on 4.6.1

2018-03-09 Thread Enrico Olivelli
2018-03-09 13:48 GMT+01:00 Ivan Kelly :

> Great analysis Sijie.
>
> Enrico, are these high traffic machines? Would it be feasible to put
> tcpdump running? You could even truncate each message to 100 bytes or
> so, to avoid storing payloads. It'd be very useful to see what the
> corrupt traffic actually looks like.
>

It is a really small cluster for manual QA tests (3 application machines +
3 bookie machines), very low traffic.
I have restricted the error isolating a single sub-system (the Majordodo
broker) which is actually a replicated-state-machine with only one leader
and two followers in this case.

Any suggestion on the tcpdump config ? (command line example)

After reverting to Netty 4.1.12 the error did not disappear but it is
rarely reproducible.

I will revert to Netty 4.1.21 to run the debug as you are requiring.

We are checking for network issues but all the machines are inside the same
VMWare system and sharing most of the hardward  (but my collegues did not
make a report to me yet)


Enrico




>
> -Ivan
>
> On Fri, Mar 9, 2018 at 10:43 AM, Enrico Olivelli 
> wrote:
> > Reverted to Netty 4.1.12. System is "more" stable but after "some"
> restart
> > we still have errors on client side on tailing readers, rebooting the JMV
> > "resolved" temporary the problem.
> >
> > I have no more errors on the Bookie side
> >
> > My  idea:
> > - client is reading from 2 bookies, there is some bug in this area
> > - once the reader client messes up Netty stops working properly and
> > corrupts ByteBufs
> > - once the reader JMV is promoted to writer (becomes leader and starts
> > writing) its internal BookKeeper client is still messed up and writes are
> > not able to complete
> >
> >
> > Enrico
> >
> >
> >
> >
> > 2018-03-09 9:55 GMT+01:00 Enrico Olivelli :
> >
> >>
> >>
> >> 2018-03-09 8:59 GMT+01:00 Sijie Guo :
> >>
> >>> Sent out a PR for the issues that I observed:
> >>>
> >>> https://github.com/apache/bookkeeper/pull/1240
> >>>
> >>
> >>
> >> Other findings:
> >> - my problem is not related to jdk9, it happens with jdk8 too
> >> - the "tailing reader" is able to make progress and follow the WAL, so
> not
> >> all the reads fail
> >> - the "writer" is able to make progress and write to the WAL, so not all
> >> the write fail
> >>
> >> I have run BK 4.6.1 tests with Netty 4.1.21Final but there is no issue
> >> (quite the OS as the machines with the error, I am on Fedora, machines
> are
> >> on CentOS)
> >>
> >> Enrico
> >>
> >>
> >>>
> >>>
> >>> On Thu, Mar 8, 2018 at 10:47 PM, Sijie Guo  wrote:
> >>>
>  So the problem here is:
> 
>  - a corrupted request failed the V3 request decoder, so bookie
> switched
>  to use v2 request decoder. Once the switch happen, the bookie will
> always
>  use v2 request decoder decoding v3 request. then all your v3 requests
> will
>  be failing with unknown op and trigger the bytebuf double releasing
> issue.
> 
>  - a corrupted response failed the V3 response decoder, so client
>  switched to use v3 response decoder. Once the switch happen, the
> client
>  will always use v2 request decoder decoding v3 response. so all the
>  responses will be failing with " Received unknown response : op code"
> 
>  Although I don't know how the first request/response happened (it can
> be
>  any issue, even network corruption), the problem is once this happen,
>  either client/bookie will be forcing to use v2 request/response
> decoder and
>  never change. so the problem will remain until you restarted. And
> this is
>  the behavior that Enrico is seeing.
> 
>  There are a couple of issues to address here:
> 
>  1) we need to add a flag to disable falling back to use v2
>  request/response coder and make sure it always use v3 protocol. In
> this
>  way, we will guarantee the problem not persist in the channel level.
>  2) we need to throw exception on unknown op code at request decode :
>  https://github.com/apache/bookkeeper/blob/master/bookkeepe
>  r-server/src/main/java/org/apache/bookkeeper/proto/BookiePro
>  toEncoding.java#L195 . As what we did at response decoder :
>  https://github.com/apache/bookkeeper/blob/master/bookkeepe
>  r-server/src/main/java/org/apache/bookkeeper/proto/BookiePro
>  toEncoding.java#L304 in https://github.com/apache/
> bookkeeper/issues/198
> 
> 
>  Details are listed as below:
> 
>  --
> 
>  Details:
> 
>  - The client side stacktrace clearly showed that it is using v2
> decoder
>  on decoding responses. That means client failed to parse v3 response
> and it
>  falls back to use v2 decoder on decoding responses. Because it is a
>  "corrupted" v3 response, so v2 decoder can't
>  find a right op code.  Then it throws illegal state exception.
> 
> 
>  *Caused by: 

Re: Help with bad errors on 4.6.1

2018-03-09 Thread Ivan Kelly
Great analysis Sijie.

Enrico, are these high traffic machines? Would it be feasible to put
tcpdump running? You could even truncate each message to 100 bytes or
so, to avoid storing payloads. It'd be very useful to see what the
corrupt traffic actually looks like.

-Ivan

On Fri, Mar 9, 2018 at 10:43 AM, Enrico Olivelli  wrote:
> Reverted to Netty 4.1.12. System is "more" stable but after "some" restart
> we still have errors on client side on tailing readers, rebooting the JMV
> "resolved" temporary the problem.
>
> I have no more errors on the Bookie side
>
> My  idea:
> - client is reading from 2 bookies, there is some bug in this area
> - once the reader client messes up Netty stops working properly and
> corrupts ByteBufs
> - once the reader JMV is promoted to writer (becomes leader and starts
> writing) its internal BookKeeper client is still messed up and writes are
> not able to complete
>
>
> Enrico
>
>
>
>
> 2018-03-09 9:55 GMT+01:00 Enrico Olivelli :
>
>>
>>
>> 2018-03-09 8:59 GMT+01:00 Sijie Guo :
>>
>>> Sent out a PR for the issues that I observed:
>>>
>>> https://github.com/apache/bookkeeper/pull/1240
>>>
>>
>>
>> Other findings:
>> - my problem is not related to jdk9, it happens with jdk8 too
>> - the "tailing reader" is able to make progress and follow the WAL, so not
>> all the reads fail
>> - the "writer" is able to make progress and write to the WAL, so not all
>> the write fail
>>
>> I have run BK 4.6.1 tests with Netty 4.1.21Final but there is no issue
>> (quite the OS as the machines with the error, I am on Fedora, machines are
>> on CentOS)
>>
>> Enrico
>>
>>
>>>
>>>
>>> On Thu, Mar 8, 2018 at 10:47 PM, Sijie Guo  wrote:
>>>
 So the problem here is:

 - a corrupted request failed the V3 request decoder, so bookie switched
 to use v2 request decoder. Once the switch happen, the bookie will always
 use v2 request decoder decoding v3 request. then all your v3 requests will
 be failing with unknown op and trigger the bytebuf double releasing issue.

 - a corrupted response failed the V3 response decoder, so client
 switched to use v3 response decoder. Once the switch happen, the client
 will always use v2 request decoder decoding v3 response. so all the
 responses will be failing with " Received unknown response : op code"

 Although I don't know how the first request/response happened (it can be
 any issue, even network corruption), the problem is once this happen,
 either client/bookie will be forcing to use v2 request/response decoder and
 never change. so the problem will remain until you restarted. And this is
 the behavior that Enrico is seeing.

 There are a couple of issues to address here:

 1) we need to add a flag to disable falling back to use v2
 request/response coder and make sure it always use v3 protocol. In this
 way, we will guarantee the problem not persist in the channel level.
 2) we need to throw exception on unknown op code at request decode :
 https://github.com/apache/bookkeeper/blob/master/bookkeepe
 r-server/src/main/java/org/apache/bookkeeper/proto/BookiePro
 toEncoding.java#L195 . As what we did at response decoder :
 https://github.com/apache/bookkeeper/blob/master/bookkeepe
 r-server/src/main/java/org/apache/bookkeeper/proto/BookiePro
 toEncoding.java#L304 in https://github.com/apache/bookkeeper/issues/198


 Details are listed as below:

 --

 Details:

 - The client side stacktrace clearly showed that it is using v2 decoder
 on decoding responses. That means client failed to parse v3 response and it
 falls back to use v2 decoder on decoding responses. Because it is a
 "corrupted" v3 response, so v2 decoder can't
 find a right op code.  Then it throws illegal state exception.


 *Caused by: java.lang.IllegalStateException: Received unknown response :
 op code = 6at
 org.apache.bookkeeper.proto.BookieProtoEncoding$ResponseEnDeCoderPreV3.decode(BookieProtoEncoding.java:294)*

 - For the stacktrace at bookie side:

 1. It is clear that BookieRequestHandler#L77 is called. That means the
 message is neither v2 request nor v3 request. It is a ByteBuf.

 https://github.com/apache/bookkeeper/blob/master/bookkeeper-
 server/src/main/java/org/apache/bookkeeper/proto/BookieReque
 stHandler.java#L77

 2. V3 request decoder is using protobuf to decode bytes array. If it is
 not a valid v3 request, it will always throw exceptions. so the code
 mentioned above will never be triggered

 https://github.com/apache/bookkeeper/blob/master/bookkeeper-
 server/src/main/java/org/apache/bookkeeper/proto/BookieProto
 Encoding.java#L344

 3. The only case that BookieRequestHandler#L77 can happen is v2 request
 decoder 

Re: Help with bad errors on 4.6.1

2018-03-09 Thread Enrico Olivelli
Reverted to Netty 4.1.12. System is "more" stable but after "some" restart
we still have errors on client side on tailing readers, rebooting the JMV
"resolved" temporary the problem.

I have no more errors on the Bookie side

My  idea:
- client is reading from 2 bookies, there is some bug in this area
- once the reader client messes up Netty stops working properly and
corrupts ByteBufs
- once the reader JMV is promoted to writer (becomes leader and starts
writing) its internal BookKeeper client is still messed up and writes are
not able to complete


Enrico




2018-03-09 9:55 GMT+01:00 Enrico Olivelli :

>
>
> 2018-03-09 8:59 GMT+01:00 Sijie Guo :
>
>> Sent out a PR for the issues that I observed:
>>
>> https://github.com/apache/bookkeeper/pull/1240
>>
>
>
> Other findings:
> - my problem is not related to jdk9, it happens with jdk8 too
> - the "tailing reader" is able to make progress and follow the WAL, so not
> all the reads fail
> - the "writer" is able to make progress and write to the WAL, so not all
> the write fail
>
> I have run BK 4.6.1 tests with Netty 4.1.21Final but there is no issue
> (quite the OS as the machines with the error, I am on Fedora, machines are
> on CentOS)
>
> Enrico
>
>
>>
>>
>> On Thu, Mar 8, 2018 at 10:47 PM, Sijie Guo  wrote:
>>
>>> So the problem here is:
>>>
>>> - a corrupted request failed the V3 request decoder, so bookie switched
>>> to use v2 request decoder. Once the switch happen, the bookie will always
>>> use v2 request decoder decoding v3 request. then all your v3 requests will
>>> be failing with unknown op and trigger the bytebuf double releasing issue.
>>>
>>> - a corrupted response failed the V3 response decoder, so client
>>> switched to use v3 response decoder. Once the switch happen, the client
>>> will always use v2 request decoder decoding v3 response. so all the
>>> responses will be failing with " Received unknown response : op code"
>>>
>>> Although I don't know how the first request/response happened (it can be
>>> any issue, even network corruption), the problem is once this happen,
>>> either client/bookie will be forcing to use v2 request/response decoder and
>>> never change. so the problem will remain until you restarted. And this is
>>> the behavior that Enrico is seeing.
>>>
>>> There are a couple of issues to address here:
>>>
>>> 1) we need to add a flag to disable falling back to use v2
>>> request/response coder and make sure it always use v3 protocol. In this
>>> way, we will guarantee the problem not persist in the channel level.
>>> 2) we need to throw exception on unknown op code at request decode :
>>> https://github.com/apache/bookkeeper/blob/master/bookkeepe
>>> r-server/src/main/java/org/apache/bookkeeper/proto/BookiePro
>>> toEncoding.java#L195 . As what we did at response decoder :
>>> https://github.com/apache/bookkeeper/blob/master/bookkeepe
>>> r-server/src/main/java/org/apache/bookkeeper/proto/BookiePro
>>> toEncoding.java#L304 in https://github.com/apache/bookkeeper/issues/198
>>>
>>>
>>> Details are listed as below:
>>>
>>> --
>>>
>>> Details:
>>>
>>> - The client side stacktrace clearly showed that it is using v2 decoder
>>> on decoding responses. That means client failed to parse v3 response and it
>>> falls back to use v2 decoder on decoding responses. Because it is a
>>> "corrupted" v3 response, so v2 decoder can't
>>> find a right op code.  Then it throws illegal state exception.
>>>
>>>
>>> *Caused by: java.lang.IllegalStateException: Received unknown response :
>>> op code = 6at
>>> org.apache.bookkeeper.proto.BookieProtoEncoding$ResponseEnDeCoderPreV3.decode(BookieProtoEncoding.java:294)*
>>>
>>> - For the stacktrace at bookie side:
>>>
>>> 1. It is clear that BookieRequestHandler#L77 is called. That means the
>>> message is neither v2 request nor v3 request. It is a ByteBuf.
>>>
>>> https://github.com/apache/bookkeeper/blob/master/bookkeeper-
>>> server/src/main/java/org/apache/bookkeeper/proto/BookieReque
>>> stHandler.java#L77
>>>
>>> 2. V3 request decoder is using protobuf to decode bytes array. If it is
>>> not a valid v3 request, it will always throw exceptions. so the code
>>> mentioned above will never be triggered
>>>
>>> https://github.com/apache/bookkeeper/blob/master/bookkeeper-
>>> server/src/main/java/org/apache/bookkeeper/proto/BookieProto
>>> Encoding.java#L344
>>>
>>> 3. The only case that BookieRequestHandler#L77 can happen is v2 request
>>> decoder doesn't parse the buffer. This leads to the bug in
>>>
>>> https://github.com/apache/bookkeeper/blob/master/bookkeeper-
>>> server/src/main/java/org/apache/bookkeeper/proto/BookieProto
>>> Encoding.java#L194
>>>
>>>
>>> - How this can happen?
>>>
>>> If the client is using v3 protocol, but both client and bookie side
>>> stacktraces show it is using v2 protocol, that means there was a corruption
>>> causing client and bookie switching from v3 protocol to v2 protocol.
>>>
>>> 

Re: Help with bad errors on 4.6.1

2018-03-09 Thread Enrico Olivelli
2018-03-09 8:59 GMT+01:00 Sijie Guo :

> Sent out a PR for the issues that I observed:
>
> https://github.com/apache/bookkeeper/pull/1240
>


Other findings:
- my problem is not related to jdk9, it happens with jdk8 too
- the "tailing reader" is able to make progress and follow the WAL, so not
all the reads fail
- the "writer" is able to make progress and write to the WAL, so not all
the write fail

I have run BK 4.6.1 tests with Netty 4.1.21Final but there is no issue
(quite the OS as the machines with the error, I am on Fedora, machines are
on CentOS)

Enrico


>
>
> On Thu, Mar 8, 2018 at 10:47 PM, Sijie Guo  wrote:
>
>> So the problem here is:
>>
>> - a corrupted request failed the V3 request decoder, so bookie switched
>> to use v2 request decoder. Once the switch happen, the bookie will always
>> use v2 request decoder decoding v3 request. then all your v3 requests will
>> be failing with unknown op and trigger the bytebuf double releasing issue.
>>
>> - a corrupted response failed the V3 response decoder, so client switched
>> to use v3 response decoder. Once the switch happen, the client will always
>> use v2 request decoder decoding v3 response. so all the responses will be
>> failing with " Received unknown response : op code"
>>
>> Although I don't know how the first request/response happened (it can be
>> any issue, even network corruption), the problem is once this happen,
>> either client/bookie will be forcing to use v2 request/response decoder and
>> never change. so the problem will remain until you restarted. And this is
>> the behavior that Enrico is seeing.
>>
>> There are a couple of issues to address here:
>>
>> 1) we need to add a flag to disable falling back to use v2
>> request/response coder and make sure it always use v3 protocol. In this
>> way, we will guarantee the problem not persist in the channel level.
>> 2) we need to throw exception on unknown op code at request decode :
>> https://github.com/apache/bookkeeper/blob/master/bookkeepe
>> r-server/src/main/java/org/apache/bookkeeper/proto/
>> BookieProtoEncoding.java#L195 . As what we did at response decoder :
>> https://github.com/apache/bookkeeper/blob/master/bookkeepe
>> r-server/src/main/java/org/apache/bookkeeper/proto/
>> BookieProtoEncoding.java#L304 in https://github.com/apache/b
>> ookkeeper/issues/198
>>
>>
>> Details are listed as below:
>>
>> --
>>
>> Details:
>>
>> - The client side stacktrace clearly showed that it is using v2 decoder
>> on decoding responses. That means client failed to parse v3 response and it
>> falls back to use v2 decoder on decoding responses. Because it is a
>> "corrupted" v3 response, so v2 decoder can't
>> find a right op code.  Then it throws illegal state exception.
>>
>>
>> *Caused by: java.lang.IllegalStateException: Received unknown response :
>> op code = 6at
>> org.apache.bookkeeper.proto.BookieProtoEncoding$ResponseEnDeCoderPreV3.decode(BookieProtoEncoding.java:294)*
>>
>> - For the stacktrace at bookie side:
>>
>> 1. It is clear that BookieRequestHandler#L77 is called. That means the
>> message is neither v2 request nor v3 request. It is a ByteBuf.
>>
>> https://github.com/apache/bookkeeper/blob/master/bookkeeper-
>> server/src/main/java/org/apache/bookkeeper/proto/
>> BookieRequestHandler.java#L77
>>
>> 2. V3 request decoder is using protobuf to decode bytes array. If it is
>> not a valid v3 request, it will always throw exceptions. so the code
>> mentioned above will never be triggered
>>
>> https://github.com/apache/bookkeeper/blob/master/bookkeeper-
>> server/src/main/java/org/apache/bookkeeper/proto/
>> BookieProtoEncoding.java#L344
>>
>> 3. The only case that BookieRequestHandler#L77 can happen is v2 request
>> decoder doesn't parse the buffer. This leads to the bug in
>>
>> https://github.com/apache/bookkeeper/blob/master/bookkeeper-
>> server/src/main/java/org/apache/bookkeeper/proto/
>> BookieProtoEncoding.java#L194
>>
>>
>> - How this can happen?
>>
>> If the client is using v3 protocol, but both client and bookie side
>> stacktraces show it is using v2 protocol, that means there was a corruption
>> causing client and bookie switching from v3 protocol to v2 protocol.
>>
>> https://github.com/apache/bookkeeper/blob/master/bookkeeper-
>> server/src/main/java/org/apache/bookkeeper/proto/
>> BookieProtoEncoding.java#L502
>> https://github.com/apache/bookkeeper/blob/master/bookkeeper-
>> server/src/main/java/org/apache/bookkeeper/proto/
>> BookieProtoEncoding.java#L432
>>
>>
>>
>>
>>
>> On Thu, Mar 8, 2018 at 12:18 AM, Enrico Olivelli 
>> wrote:
>>
>>> Hi all,
>>> I am seeing this bad errors in some test environments with 4.6.1
>>> The errors appear during rolling restart of the application, the test
>>> env is made of 6 machines:
>>> - 3 bookies
>>> - 3 client machines (with several BK clients, of different sub-systems)
>>> - running with 4.6.1 both client and servers
>>>
>>> *I do not have* reports of 

Re: Help with bad errors on 4.6.1

2018-03-09 Thread Sijie Guo
Sent out a PR for the issues that I observed:

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

On Thu, Mar 8, 2018 at 10:47 PM, Sijie Guo  wrote:

> So the problem here is:
>
> - a corrupted request failed the V3 request decoder, so bookie switched to
> use v2 request decoder. Once the switch happen, the bookie will always use
> v2 request decoder decoding v3 request. then all your v3 requests will be
> failing with unknown op and trigger the bytebuf double releasing issue.
>
> - a corrupted response failed the V3 response decoder, so client switched
> to use v3 response decoder. Once the switch happen, the client will always
> use v2 request decoder decoding v3 response. so all the responses will be
> failing with " Received unknown response : op code"
>
> Although I don't know how the first request/response happened (it can be
> any issue, even network corruption), the problem is once this happen,
> either client/bookie will be forcing to use v2 request/response decoder and
> never change. so the problem will remain until you restarted. And this is
> the behavior that Enrico is seeing.
>
> There are a couple of issues to address here:
>
> 1) we need to add a flag to disable falling back to use v2
> request/response coder and make sure it always use v3 protocol. In this
> way, we will guarantee the problem not persist in the channel level.
> 2) we need to throw exception on unknown op code at request decode :
> https://github.com/apache/bookkeeper/blob/master/
> bookkeeper-server/src/main/java/org/apache/bookkeeper/
> proto/BookieProtoEncoding.java#L195 . As what we did at response decoder
> : https://github.com/apache/bookkeeper/blob/master/
> bookkeeper-server/src/main/java/org/apache/bookkeeper/
> proto/BookieProtoEncoding.java#L304 in https://github.com/apache/
> bookkeeper/issues/198
>
>
> Details are listed as below:
>
> --
>
> Details:
>
> - The client side stacktrace clearly showed that it is using v2 decoder on
> decoding responses. That means client failed to parse v3 response and it
> falls back to use v2 decoder on decoding responses. Because it is a
> "corrupted" v3 response, so v2 decoder can't
> find a right op code.  Then it throws illegal state exception.
>
>
> *Caused by: java.lang.IllegalStateException: Received unknown response :
> op code = 6at
> org.apache.bookkeeper.proto.BookieProtoEncoding$ResponseEnDeCoderPreV3.decode(BookieProtoEncoding.java:294)*
>
> - For the stacktrace at bookie side:
>
> 1. It is clear that BookieRequestHandler#L77 is called. That means the
> message is neither v2 request nor v3 request. It is a ByteBuf.
>
> https://github.com/apache/bookkeeper/blob/master/
> bookkeeper-server/src/main/java/org/apache/bookkeeper/
> proto/BookieRequestHandler.java#L77
>
> 2. V3 request decoder is using protobuf to decode bytes array. If it is
> not a valid v3 request, it will always throw exceptions. so the code
> mentioned above will never be triggered
>
> https://github.com/apache/bookkeeper/blob/master/
> bookkeeper-server/src/main/java/org/apache/bookkeeper/
> proto/BookieProtoEncoding.java#L344
>
> 3. The only case that BookieRequestHandler#L77 can happen is v2 request
> decoder doesn't parse the buffer. This leads to the bug in
>
> https://github.com/apache/bookkeeper/blob/master/
> bookkeeper-server/src/main/java/org/apache/bookkeeper/
> proto/BookieProtoEncoding.java#L194
>
>
> - How this can happen?
>
> If the client is using v3 protocol, but both client and bookie side
> stacktraces show it is using v2 protocol, that means there was a corruption
> causing client and bookie switching from v3 protocol to v2 protocol.
>
> https://github.com/apache/bookkeeper/blob/master/
> bookkeeper-server/src/main/java/org/apache/bookkeeper/
> proto/BookieProtoEncoding.java#L502
> https://github.com/apache/bookkeeper/blob/master/
> bookkeeper-server/src/main/java/org/apache/bookkeeper/
> proto/BookieProtoEncoding.java#L432
>
>
>
>
>
> On Thu, Mar 8, 2018 at 12:18 AM, Enrico Olivelli 
> wrote:
>
>> Hi all,
>> I am seeing this bad errors in some test environments with 4.6.1
>> The errors appear during rolling restart of the application, the test env
>> is made of 6 machines:
>> - 3 bookies
>> - 3 client machines (with several BK clients, of different sub-systems)
>> - running with 4.6.1 both client and servers
>>
>> *I do not have* reports of this errors from production, already running
>> 4.6.1 for the last month
>>
>> But the problem is quite scary
>>
>> This is a sample of relevant errors on clients (in this case Majordodo
>> brokers, with log level = INFO)
>>
>> logs/org.apache.bookkeeper.client.PendingAddOp:
>> 18-03-07-09-51-43   Write of ledger entry to quorum failed: L366634
>> E2557
>> 18-03-07-09-51-43   Write of ledger entry to quorum failed: L366634
>> E2558
>> 18-03-07-09-51-43   Write of ledger entry to quorum failed: L366634
>> E2559
>> 18-03-07-15-59-55   Failed to write entry (366680, 1865): Bookie
>> 

Re: Help with bad errors on 4.6.1

2018-03-08 Thread Enrico Olivelli
(switch to dev@)

@Sijie very good explanation.

I am back to work I we have found errors even on a client reader which  is
performing tailing reads


-03-09-08-34-19io.netty.handler.codec.DecoderException:
java.lang.IllegalStateException: Received unknown response : op code = 9
io.netty.handler.codec.DecoderException: java.lang.IllegalStateException:
Received unknown response : op code = 9
at
io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:98)
at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362)
at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348)
at
io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:340)
at
io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:310)
at
io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:297)
at
io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:413)
at
io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:265)
at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362)
at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348)
at
io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:340)
at
io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1414)
at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362)
at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348)
at
io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:945)
at
io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.epollInReady(AbstractEpollStreamChannel.java:800)
at
io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:404)
at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:304)
at
io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:886)
at java.base/java.lang.Thread.run(Thread.java:844)
Caused by: java.lang.IllegalStateException: Received unknown response : op
code = 9
at
org.apache.bookkeeper.proto.BookieProtoEncoding$ResponseEnDeCoderPreV3.decode(BookieProtoEncoding.java:294)
at
org.apache.bookkeeper.proto.BookieProtoEncoding$ResponseDecoder.decode(BookieProtoEncoding.java:474)
at
io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:88)
... 19 more

I am now running  BK tests using that version of Netty, in parallel we are
reverting Netty to 4.1.12 on QA env

The previous version of the application was running with Netty 4.1.12

Stay tuned

Enrico



2018-03-09 7:47 GMT+01:00 Sijie Guo :

> So the problem here is:
>
> - a corrupted request failed the V3 request decoder, so bookie switched to
> use v2 request decoder. Once the switch happen, the bookie will always use
> v2 request decoder decoding v3 request. then all your v3 requests will be
> failing with unknown op and trigger the bytebuf double releasing issue.
>
> - a corrupted response failed the V3 response decoder, so client switched
> to use v3 response decoder. Once the switch happen, the client will always
> use v2 request decoder decoding v3 response. so all the responses will be
> failing with " Received unknown response : op code"
>
> Although I don't know how the first request/response happened (it can be
> any issue, even network corruption), the problem is once this happen,
> either client/bookie will be forcing to use v2 request/response decoder and
> never change. so the problem will remain until you restarted. And this is
> the behavior that Enrico is seeing.
>
> There are a couple of issues to address here:
>
> 1) we need to add a flag to disable falling back to use v2
> request/response coder and make sure it always use v3 protocol. In this
> way, we will guarantee the problem not persist in the channel level.
> 2) we need to throw exception on unknown op code at request decode :
> https://github.com/apache/bookkeeper/blob/master/
> bookkeeper-server/src/main/java/org/apache/bookkeeper/
> proto/BookieProtoEncoding.java#L195 . As what we did at response decoder
> : https://github.com/apache/bookkeeper/blob/master/
> bookkeeper-server/src/main/java/org/apache/bookkeeper/
> proto/BookieProtoEncoding.java#L304 in https://github.com/apache/
> bookkeeper/issues/198
>
>
> Details are listed as below:
>
> --
>
> Details:
>
> - The client side stacktrace clearly showed that it is using v2 decoder on
> decoding responses. That means client failed to parse v3 response and it
> falls back to