Details
-
Bug
-
Status: Resolved
-
Critical
-
Resolution: Fixed
-
2.18.0
-
None
-
Unknown
Description
It looks like CAMEL-9040 fix introduced a double release of a netty buffer
that leads to IllegalReferenceCountException under load:
2016-10-22 10:20:15.442 WARN 6853 --- [ClientTCPWorker] io.netty.util.ReferenceCountUtil : Failed to release a message: AdvancedLeakAwareByteBuf(PooledUnsafeDirectByteBuf(freed)) io.netty.util.IllegalReferenceCountException: refCnt: 0, decrement: 1 at io.netty.buffer.AbstractReferenceCountedByteBuf.release(AbstractReferenceCountedByteBuf.java:111) ~[netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.buffer.WrappedByteBuf.release(WrappedByteBuf.java:1029) ~[netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.buffer.AdvancedLeakAwareByteBuf.release(AdvancedLeakAwareByteBuf.java:951) ~[netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.util.ReferenceCountUtil.release(ReferenceCountUtil.java:84) ~[netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.util.ReferenceCountUtil.safeRelease(ReferenceCountUtil.java:109) ~[netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.channel.ChannelOutboundBuffer.remove0(ChannelOutboundBuffer.java:296) [netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.channel.ChannelOutboundBuffer.failFlushed(ChannelOutboundBuffer.java:621) [netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.channel.AbstractChannel$AbstractUnsafe.flush0(AbstractChannel.java:869) [netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.channel.nio.AbstractNioChannel$AbstractNioUnsafe.flush0(AbstractNioChannel.java:362) [netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.channel.AbstractChannel$AbstractUnsafe.flush(AbstractChannel.java:823) [netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.channel.DefaultChannelPipeline$HeadContext.flush(DefaultChannelPipeline.java:1296) [netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.channel.AbstractChannelHandlerContext.invokeFlush0(AbstractChannelHandlerContext.java:786) [netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.channel.AbstractChannelHandlerContext.invokeFlush(AbstractChannelHandlerContext.java:778) [netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.channel.AbstractChannelHandlerContext.flush(AbstractChannelHandlerContext.java:759) [netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.channel.CombinedChannelDuplexHandler$DelegatingChannelHandlerContext.flush(CombinedChannelDuplexHandler.java:530) [netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.channel.ChannelOutboundHandlerAdapter.flush(ChannelOutboundHandlerAdapter.java:115) [netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.channel.CombinedChannelDuplexHandler.flush(CombinedChannelDuplexHandler.java:355) [netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.channel.AbstractChannelHandlerContext.invokeFlush0(AbstractChannelHandlerContext.java:786) [netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.channel.AbstractChannelHandlerContext.invokeWriteAndFlush(AbstractChannelHandlerContext.java:812) [netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.channel.AbstractChannelHandlerContext.write(AbstractChannelHandlerContext.java:824) [netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.channel.AbstractChannelHandlerContext.writeAndFlush(AbstractChannelHandlerContext.java:804) [netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.channel.AbstractChannelHandlerContext.writeAndFlush(AbstractChannelHandlerContext.java:841) [netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.channel.DefaultChannelPipeline.writeAndFlush(DefaultChannelPipeline.java:1032) [netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.channel.AbstractChannel.writeAndFlush(AbstractChannel.java:296) [netty-all-4.1.5.Final.jar:4.1.5.Final] at org.apache.camel.component.netty4.NettyHelper.writeBodyAsync(NettyHelper.java:105) [camel-netty4-2.18.0.jar:2.18.0] at org.apache.camel.component.netty4.NettyProducer.processWithConnectedChannel(NettyProducer.java:306) [camel-netty4-2.18.0.jar:2.18.0] at org.apache.camel.component.netty4.NettyProducer$ChannelConnectedListener.operationComplete(NettyProducer.java:642) [camel-netty4-2.18.0.jar:2.18.0] at org.apache.camel.component.netty4.NettyProducer$ChannelConnectedListener.operationComplete(NettyProducer.java:619) [camel-netty4-2.18.0.jar:2.18.0] at io.netty.util.concurrent.DefaultPromise.notifyListener0(DefaultPromise.java:514) [netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.util.concurrent.DefaultPromise.notifyListeners0(DefaultPromise.java:507) [netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.util.concurrent.DefaultPromise.notifyListenersNow(DefaultPromise.java:486) [netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.util.concurrent.DefaultPromise.notifyListeners(DefaultPromise.java:427) [netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.util.concurrent.DefaultPromise.trySuccess(DefaultPromise.java:111) [netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.channel.DefaultChannelPromise.trySuccess(DefaultChannelPromise.java:82) [netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.channel.nio.AbstractNioChannel$AbstractNioUnsafe.fulfillConnectPromise(AbstractNioChannel.java:306) [netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.channel.nio.AbstractNioChannel$AbstractNioUnsafe.finishConnect(AbstractNioChannel.java:341) [netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:627) [netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:551) [netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:465) [netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:437) [netty-all-4.1.5.Final.jar:4.1.5.Final] at io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:873) [netty-all-4.1.5.Final.jar:4.1.5.Final] at java.lang.Thread.run(Thread.java:745) [na:1.8.0_101]
The buffer is release at
at org.apache.camel.component.netty4.http.NettyHttpProducer$1.onDone(NettyHttpProducer.java:85) at org.apache.camel.support.SynchronizationAdapter.onComplete(SynchronizationAdapter.java:35) at org.apache.camel.util.UnitOfWorkHelper.doneSynchronizations(UnitOfWorkHelper.java:104) at org.apache.camel.impl.DefaultUnitOfWork.done(DefaultUnitOfWork.java:230) at org.apache.camel.util.UnitOfWorkHelper.doneUow(UnitOfWorkHelper.java:65) at org.apache.camel.impl.DefaultConsumer.doneUoW(DefaultConsumer.java:107) at org.apache.camel.component.netty4.handlers.ServerChannelHandler$1.done(ServerChannelHandler.java:149) at org.apache.camel.processor.CamelInternalProcessor$InternalCallback.done(CamelInternalProcessor.java:257) at org.apache.camel.processor.Pipeline$1.done(Pipeline.java:147) at org.apache.camel.processor.CamelInternalProcessor$InternalCallback.done(CamelInternalProcessor.java:257) at org.apache.camel.processor.RedeliveryErrorHandler$2.done(RedeliveryErrorHandler.java:554) at org.apache.camel.management.InstrumentationProcessor$1.done(InstrumentationProcessor.java:86) at org.apache.camel.processor.SendProcessor$1.done(SendProcessor.java:155) at org.apache.camel.component.netty4.http.NettyHttpProducer$NettyHttpProducerCallback.done(NettyHttpProducer.java:149) at org.apache.camel.component.netty4.NettyProducer$NettyProducerCallback.done(NettyProducer.java:548) at org.apache.camel.component.netty4.handlers.ClientChannelHandler.channelRead0(ClientChannelHandler.java:205) at io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:372) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:358) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:350) at io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:102) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:372) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:358) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:350) at io.netty.channel.CombinedChannelDuplexHandler$DelegatingChannelHandlerContext.fireChannelRead(CombinedChannelDuplexHandler.java:435) at io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:293) at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:267) at io.netty.channel.CombinedChannelDuplexHandler.channelRead(CombinedChannelDuplexHandler.java:250) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:372) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:358) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:350) at io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1334) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:372) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:358) at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:926) at io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:129) at io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:610) at io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:551) at io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:465) at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:437) at io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:873) at java.lang.Thread.run(Thread.java:745)
and
at org.apache.camel.component.netty4.NettyHelper.writeBodyAsync(NettyHelper.java:105) at org.apache.camel.component.netty4.NettyProducer.processWithConnectedChannel(NettyProducer.java:306) at org.apache.camel.component.netty4.NettyProducer$ChannelConnectedListener.operationComplete(NettyProducer.java:642) at org.apache.camel.component.netty4.NettyProducer$ChannelConnectedListener.operationComplete(NettyProducer.java:619) at io.netty.util.concurrent.DefaultPromise.notifyListener0(DefaultPromise.java:514) at io.netty.util.concurrent.DefaultPromise.notifyListeners0(DefaultPromise.java:507) at io.netty.util.concurrent.DefaultPromise.notifyListenersNow(DefaultPromise.java:486) at io.netty.util.concurrent.DefaultPromise.notifyListeners(DefaultPromise.java:427) at io.netty.util.concurrent.DefaultPromise.trySuccess(DefaultPromise.java:111) at io.netty.channel.DefaultChannelPromise.trySuccess(DefaultChannelPromise.java:82) at io.netty.channel.nio.AbstractNioChannel$AbstractNioUnsafe.fulfillConnectPromise(AbstractNioChannel.java:306) at io.netty.channel.nio.AbstractNioChannel$AbstractNioUnsafe.finishConnect(AbstractNioChannel.java:341) at io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:627) at io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:551) at io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:465) at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:437) at io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:873) at java.lang.Thread.run(Thread.java:745)
Note that refCnt check do not work if buffer is being used by another thread, it's just hiding a problem if it works. I will try to look into it deeper, but I am not sure if I fully understand the buffer allocation / freeing lifecycle in netty yet
My test routes are:
from("timer:load?period=100") .to("log:client?groupInterval=5000") .to("netty4-http:http://127.0.0.1:8081/main-service"); from("netty4-http:http://0.0.0.0:8081/main-service") .to("log:main-service?groupInterval=5000") .to("netty4-http:http://127.0.0.1:8082/childService1") .to("netty4-http:http://127.0.0.1:8083/childService2"); from("netty4-http:http://0.0.0.0:8082/childService1") .to("log:childService1?groupInterval=5000") .delayer(100); from("netty4-http:http://0.0.0.0:8083/childService2") .to("log:childService2?groupInterval=5000") .delayer(100);
Attachments
Issue Links
- links to