On Thu, Mar 8, 2018 at 12:33 PM Andrey Yegorov <andrey.yego...@gmail.com>
wrote:

> I am looking more at the PendigAddOp and it looks like, in addition to the
> case that Sijie has fixed, there is another scenario where recycler can get
> triggered.
>
> I.e.:
> first sendWriteRequest() in PendingAddOp.safeRun() fails in PCBC's
> writeAndFlush (channel == null or channel.writeAndFlush throws)
> hasRun is set to true at that point. pendingWriteRequests is still zero
> (not incremented yet)
> PCBC's errorOut triggers callback that is sure that since
> pendingWriteRequests==0 and hasRun it can release PendingAddOp back to
> recycler
>
> now there are still sendWriteRequest's to follow and all bets are off.
> i.e. by the time they are executed the addOp can get reused so they end up
> sending another request to the wrong bookie.
>
> So far my idea of solution for that is to run PCBC's errorOut
> asynchronously / submit it back to orderedSafeExecutor so addOp can
> increment counts properly.
>

Oh PendingAddOp should be executed at safe executor. Otherwise it is not
just recycle problem, it might be a correctness issue.


>
>
> ----------
> Andrey Yegorov
>
> On Thu, Mar 8, 2018 at 11:52 AM, Sijie Guo <guosi...@gmail.com> wrote:
>
>>
>>
>> On Thu, Mar 8, 2018 at 7:42 AM, Enrico Olivelli <eolive...@gmail.com>
>> wrote:
>>
>>>
>>>
>>> 2018-03-08 14:50 GMT+01:00 Ivan Kelly <iv...@apache.org>:
>>>
>>>> It just occurred to me that this could be a problem with the recycler.
>>>> If we recycle a buffer too early, but then keep using it, another user
>>>> could pick it up, and between them they could corrupt the data that
>>>> would cause it to be unhandled.
>>>>
>>>
>>>
>>> Maybe current work from Sijie, Andrey and JV which Sijie is backporting
>>> to branch-4.6 could resolve the issue
>>>
>>
>>
>> I think that is unrelated. The change Andrey, JV and me dealt with is the
>> PendingAddOp. It is on client side not bookie side.
>>
>>
>>>
>>>
>>>
>>>
>>>
>>>>
>>>> -Ivan
>>>>
>>>> On Thu, Mar 8, 2018 at 1:49 PM, Enrico Olivelli <eolive...@gmail.com>
>>>> wrote:
>>>> >
>>>> >
>>>> > 2018-03-08 11:57 GMT+01:00 Ivan Kelly <iv...@apache.org>:
>>>> >>
>>>> >> Hmm, yes, looks messed up. Do you have a reliable repro of this?
>>>> >>
>>>> >> "at
>>>> io.netty.channel.DefaultChannelPipeline.onUnhandledInboundMessage"
>>>> >> <- it seems the message itself wasn't handled. Do you have any more
>>>> >> bookie side logs leading up to this event?
>>>> >
>>>> >
>>>> >
>>>> > The only suspicious logs are for logger named
>>>> > "org.apache.bookkeeper.bookie.EntryLogger"
>>>> >
>>>> >
>>>> > 18-03-08-09-22-40       Created new entry log file
>>>> > /data/qasql/bookie/./bookie_data/current/4b0.log for logId 1200.
>>>> > 18-03-08-09-32-41       Failed to get ledgers map index from: 53.log
>>>> : No
>>>> > ledgers map index found on entryLogId53
>>>> > 18-03-08-09-33-10       Failed to get ledgers map index from: 54.log
>>>> : No
>>>> > ledgers map index found on entryLogId54
>>>> > 18-03-08-09-33-10       Failed to get ledgers map index from: 375.log
>>>> : No
>>>> > ledgers map index found on entryLogId375
>>>> > 18-03-08-09-33-13       Failed to get ledgers map index from: 878.log
>>>> : No
>>>> > ledgers map index found on entryLogId878
>>>> > 18-03-08-09-33-22       Failed to get ledgers map index from:
>>>> 1197.log : No
>>>> > ledgers map index found on entryLogId1197
>>>> > 18-03-08-09-33-57       Failed to get ledgers map index from:
>>>> 1198.log : No
>>>> > ledgers map index found on entryLogId1198
>>>> > 18-03-08-09-34-54       Failed to get ledgers map index from:
>>>> 1199.log : No
>>>> > ledgers map index found on entryLogId1199
>>>>
>>>
>>
>> This is also unrelated. It is a fine logging, since it means it attempts
>> to read ledgers map index, but didn't find one.
>>
>> the ledgers map index is only written when it is enabled at the bookies.
>>
>>
>>> >
>>>> > I will try to run Bookie with level debug, but it produces lots of
>>>> spam
>>>> >
>>>> > Thanks
>>>> > Enrico
>>>> >
>>>> >>
>>>> >>
>>>> >> -Ivan
>>>> >>
>>>> >> On Thu, Mar 8, 2018 at 9:18 AM, Enrico Olivelli <eolive...@gmail.com
>>>> >
>>>> >> 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
>>>> >> > operation timeout
>>>> >> > 18-03-07-15-59-55       Failed to write entry (366680, 1865):
>>>> Bookie
>>>> >> > operation timeout
>>>> >> > 18-03-07-16-00-00       Failed to write entry (366680, 1865):
>>>> Bookie
>>>> >> > operation timeout
>>>> >> > 18-03-07-16-00-00       Failed to write entry (366680, 1865):
>>>> Bookie
>>>> >> > operation timeout
>>>> >> > 18-03-07-16-00-05       Failed to write entry (366680, 1865):
>>>> Bookie
>>>> >> > operation timeout
>>>> >> >
>>>> >> >
>>>> >> > org.apache.bookkeeper.proto.PerChannelBookieClient:
>>>> >> >
>>>> >> > 18-03-07-10-06-44       Unexpected exception caught by bookie
>>>> client
>>>> >> > channel
>>>> >> > handler
>>>> >> > 18-03-07-10-06-44       io.netty.handler.codec.DecoderException:
>>>> >> > java.lang.IllegalStateException: Received unknown response : op
>>>> code = 6
>>>> >> > io.netty.handler.codec.DecoderException:
>>>> >> > java.lang.IllegalStateException:
>>>> >> > Received unknown response : op code = 6
>>>> >> >         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 = 6
>>>> >> >         at
>>>> >> >
>>>> >> > org.apache.bookkeeper.proto.Bo
>>>> okieProtoEncoding$ResponseEnDeCoderPreV3.decode(BookieProtoEncoding.java:294)
>>>> >> >         at
>>>> >> >
>>>> >> > org.apache.bookkeeper.proto.Bo
>>>> okieProtoEncoding$ResponseDecoder.decode(BookieProtoEncoding.java:478)
>>>> >> >         at
>>>> >> >
>>>> >> >
>>>> io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:88)
>>>> >> >
>>>> >> >
>>>> >> >         ... 19 more
>>>> >> >
>>>> >> >
>>>> >> > Errors on Bookie side, with log level = INFO)
>>>> >> >
>>>> >> >
>>>> >> > io.netty.channel.DefaultChannelPipeline:
>>>> >> > 8-03-07-09-51-38       An exceptionCaught() event was fired, and it
>>>> >> > reached
>>>> >> > at the tail of the pipeline. It usually means the last handler in
>>>> the
>>>> >> > pipeline did not handle the exception.
>>>> >> > 18-03-07-09-51-38
>>>>  io.netty.util.IllegalReferenceCountException:
>>>> >> > refCnt: 0, increment: 1
>>>> >> > io.netty.util.IllegalReferenceCountException: refCnt: 0,
>>>> increment: 1
>>>> >> >         at
>>>> >> >
>>>> >> >
>>>> io.netty.buffer.AbstractReferenceCountedByteBuf.release0(AbstractReferenceCountedByteBuf.java:100)
>>>> >> >         at
>>>> >> >
>>>> >> >
>>>> io.netty.buffer.AbstractReferenceCountedByteBuf.release(AbstractReferenceCountedByteBuf.java:84)
>>>> >> >         at
>>>> >> >
>>>> io.netty.util.ReferenceCountUtil.release(ReferenceCountUtil.java:88)
>>>> >> >         at
>>>> >> >
>>>> >> >
>>>> io.netty.channel.DefaultChannelPipeline.onUnhandledInboundMessage(DefaultChannelPipeline.java:1202)
>>>> >> >         at
>>>> >> >
>>>> >> >
>>>> io.netty.channel.DefaultChannelPipeline$TailContext.channelRead(DefaultChannelPipeline.java:1299)
>>>> >> >         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.ChannelInboundHandlerAdapter.channelRead(ChannelInboundHandlerAdapter.java:86)
>>>> >> >         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
>>>> >> >
>>>> >> > org.apache.bookkeeper.proto.Bo
>>>> okieRequestHandler.channelRead(BookieRequestHandler.java:78)
>>>> >> >         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.ChannelInboundHandlerAdapter.channelRead(ChannelInboundHandlerAdapter.java:86)
>>>> >> >         at
>>>> >> >
>>>> >> > org.apache.bookkeeper.proto.Au
>>>> thHandler$ServerSideHandler.channelRead(AuthHandler.java:88)
>>>> >> >         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.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:102)
>>>> >> >         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)
>>>>
>>>
>> questions:
>>
>> 1) which protocol version are you using?
>>
>> 2) is the exception coming from read response or write response? it sound
>> like write response. If it is write response, it would be surprised about
>> the ref count. because we don't send any buffer back in write response.
>>
>>
>>> >> >
>>>> >> >
>>>> >
>>>> >
>>>>
>>>
>>>
>>
>

Reply via email to