sure, unless some other corner case of Op completion is missing. Doing something along the lines of the code below will add some visibility into that:
void timeoutQuorumWait() { try { if(!completed) final LedgerHandle cached = lh; clientCtx.getMainWorkerPool().executeOrdered(lh.ledgerId, new SafeRunnable() { @Override public void safeRun() { if (lh == null || lh.ledgerId != cached.ledgerId) { log.error("trying to timeout recycled/reused op", .....); // possibly throw } if (completed) { return; } else if (addEntrySuccessBookies.size() >= cached.getLedgerMetadata().getAckQuorumSize()) { // If ackQuorum number of bookies have acknowledged the write but still not complete, indicates // failures due to not having been written to enough fault domains. Increment corresponding // counter. clientCtx.getClientStats().getWriteTimedOutDueToNotEnoughFaultDomains(). inc(); } lh.handleUnrecoverableErrorDuringAdd(BKException.Code. AddEntryQuorumTimeoutException); } @Override public String toString() { return String.format("AddEntryQuorumTimeout(lid=%d, eid=%d)", cached.ledgerId, entryId); } }); } catch (RejectedExecutionException e) { LOG.warn("Timeout add entry quorum wait failed {} entry: {}", cached.ledgerId, entryId); } }} ---------- Andrey Yegorov On Tue, Feb 16, 2021 at 4:58 PM Venkateswara Rao Jujjuri <jujj...@gmail.com> wrote: > But post recycle should not result in this scenario as I mentioned in my > comment. > > On Tue, Feb 16, 2021 at 2:49 PM Andrey Yegorov <andrey.yego...@gmail.com> > wrote: > > > Simple check for null won't prevent timing out wrong (post recycle/reuse) > > handle. > > > > > > On Tue, Feb 16, 2021 at 14:38 Venkateswara Rao Jujjuri < > jujj...@gmail.com> > > wrote: > > > > > Yes; That is exactly how my theory is and the fix is also the same. But > > > still looking into how we could get into it. > > > > > > // If the object is recycled while this task is on the queue lh could > get > > > null. > > > // monitorPendingAddOps() can't schedule after recycling because > > > // time timed out pendingAddOp won't be available on the > > > // pendingAddOps concurrent queue after a successful recycle. > > > //if (completed) { > > > if (lh == null || completed) { > > > > > > > > > > > > On Tue, Feb 16, 2021 at 2:30 PM Andrey Yegorov < > andrey.yego...@gmail.com > > > > > > wrote: > > > > > > > This could happen if PendingAddOp got recycled (and set lh to null) > > > before > > > > the runnable submitted by timeoutQuorumWait had a chance to run. > > > > I don't know how it got into this state but if is easy to miss if > > > recycled > > > > object gets reused quickly - timeout will fail wrong ledger. > > > > > > > > The right thing to do is to figure bout how recycling could happen > > before > > > > timeout runnable had a chance to run. > > > > > > > > As workaround you can simply cache ledger handle before creating > > runnable > > > > and use the cached value in the runnable. > > > > Hope this makes sense. > > > > > > > > On Tue, Feb 16, 2021 at 13:12 Enrico Olivelli <eolive...@gmail.com> > > > wrote: > > > > > > > > > Sorry > > > > > I never seen that error > > > > > > > > > > Enrico > > > > > > > > > > Il giorno mar 16 feb 2021 alle ore 19:50 Venkateswara Rao Jujjuri < > > > > > jujj...@gmail.com> ha scritto: > > > > > > > > > > > Ignore my comment about different thread, I see that > > > > timeoutQuorumWait() > > > > > is > > > > > > run through OSE. > > > > > > But we did NPE in this line > > > > > > < > > > > > > > > > > > > > > > > > > > > > https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java#L173 > > > > > > >. > > > > > > In this run, we have disabled ensemble changes. > > > > > > > > > > > > On Tue, Feb 16, 2021 at 10:28 AM Venkateswara Rao Jujjuri < > > > > > > jujj...@gmail.com> > > > > > > wrote: > > > > > > > > > > > > > We have observed NPE in pendingAddOp in this line > > > > > > > < > > > > > > > > > > > > > > > > > > > > > https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java#L173 > > > > > > >. > > > > > > > > > > > > > > This looks like a race between execution of > > > recyclePendAddOpObject() > > > > in > > > > > > > OSE context and maybeTimeout() in monitorPendingAddOps() > > executing > > > in > > > > > > > bk.scheduler context. > > > > > > > > > > > > > > This looks like an age-old code and wondering why we haven't > seen > > > > this > > > > > so > > > > > > > far. > > > > > > > Has Anyone in the community observed this? > > > > > > > > > > > > > > > > > > > > > -- > > > > > > > Jvrao > > > > > > > --- > > > > > > > First they ignore you, then they laugh at you, then they fight > > you, > > > > > then > > > > > > > you win. - Mahatma Gandhi > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > -- > > > > > > Jvrao > > > > > > --- > > > > > > First they ignore you, then they laugh at you, then they fight > you, > > > > then > > > > > > you win. - Mahatma Gandhi > > > > > > > > > > > > > > > -- > > > > ---------- > > > > Andrey Yegorov > > > > > > > > > > > > > -- > > > Jvrao > > > --- > > > First they ignore you, then they laugh at you, then they fight you, > then > > > you win. - Mahatma Gandhi > > > > > -- > > ---------- > > Andrey Yegorov > > > > > -- > Jvrao > --- > First they ignore you, then they laugh at you, then they fight you, then > you win. - Mahatma Gandhi >