On Fri, Mar 16, 2018 at 1:24 AM, Enrico Olivelli <eolive...@gmail.com> wrote:
> 2018-03-15 12:02 GMT+01:00 Enrico Olivelli <eolive...@gmail.com>: > > > > > > > 2018-03-15 11:13 GMT+01:00 Ivan Kelly <iv...@apache.org>: > > > >> > 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-49 LEAK: 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) > > 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) > > > > > > > > > > Thank you > > > > Enrico > > > > > >> > >> i tried running my attempted repro in java9, but no cigar. > >> > >> > >> -Ivan > >> > > > > >