Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

recycle already. #1476

Closed
infodog opened this issue Jun 4, 2018 · 8 comments
Closed

recycle already. #1476

infodog opened this issue Jun 4, 2018 · 8 comments

Comments

@infodog
Copy link

infodog commented Jun 4, 2018

bookkeeper 4.7, when using distributed log api, got the following error:

[BookKeeperClientWorker-OrderedExecutor-0-0] ERROR org.apache.bookkeeper.common.util.SafeRunnable - Unexpected throwable caught
java.lang.IllegalStateException: recycled already
at io.netty.util.Recycler$Stack.pushNow(Recycler.java:560)
at io.netty.util.Recycler$Stack.push(Recycler.java:550)
at io.netty.util.Recycler$DefaultHandle.recycle(Recycler.java:209)
at org.apache.bookkeeper.client.impl.LedgerEntryImpl.recycle(LedgerEntryImpl.java:163)
at org.apache.bookkeeper.client.impl.LedgerEntryImpl.close(LedgerEntryImpl.java:154)
at org.apache.bookkeeper.client.ReadLastConfirmedAndEntryOp$ReadLACAndEntryRequest.close(ReadLastConfirmedAndEntryOp.java:94)
at org.apache.bookkeeper.client.ReadLastConfirmedAndEntryOp.submitCallback(ReadLastConfirmedAndEntryOp.java:530)
at org.apache.bookkeeper.client.ReadLastConfirmedAndEntryOp.readEntryComplete(ReadLastConfirmedAndEntryOp.java:568)
at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadCompletion$1.readEntryComplete(PerChannelBookieClient.java:1559)
at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadCompletion.handleReadResponse(PerChannelBookieClient.java:1640)
at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadCompletion.handleV3Response(PerChannelBookieClient.java:1615)
at org.apache.bookkeeper.proto.PerChannelBookieClient$3.safeRun(PerChannelBookieClient.java:1232)
at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138)
at java.lang.Thread.run(Thread.java:745)

@sijie
Copy link
Member

sijie commented Jun 4, 2018

@infodog thank you for reporting this. will take a look at the stack trace and fix it.

@sijie
Copy link
Member

sijie commented Jun 5, 2018

@infodog can you describe more about this issue?

  • when is this exception thrown? during reading or during shutdown. I was looking into the code and couldn't find how this would happen.

  • does this exception impact reader? or just an error message in the log file?

@infodog
Copy link
Author

infodog commented Jun 5, 2018

@sijie my situation is when one process reading from the log, and one process writing to the log, the writer is fast.
when this exception happened, the reader will stop working.

my code of reading is like following:

while (true) {
try {

            while (true) {
                try {
                    LogRecord record = reader.readNext(false);
                    if (record == null) {
                        break;
                    }
                    System.out.println("read record, txid=" + record.getTransactionId() + ", lastShardId=" + lastShardTxId);
                    if (record.getTransactionId() > getLocalLastTxId()) {
                        updateLog(record);
                    }
                    // read next record
                } catch (LogEmptyException t) {
                    //没事
                    System.out.println("no record in log.");
                } catch (Throwable ioe) {
                    // handle the exception
                    nextTxId = getLocalLastTxId();
                    reader = dlm.getInputStream(nextTxId);
                }

            }


            if (isSwitchingToMaster) {
                return;
            }
            synchronized (downloadThreadMonitor) {
                downloadThreadMonitor.wait(100);
            }
            if (isSwitchingToMaster) {
                return;
            }
        } catch (Throwable e) {
            e.printStackTrace();
        }
    }

@infodog
Copy link
Author

infodog commented Jun 5, 2018

@sijie maybe this issue is like #1229 ? Similar reason?

@sijie
Copy link
Member

sijie commented Jun 5, 2018

@infodog - I think it is similar, it looks like a double-release issue. will take a closer look.

@infodog
Copy link
Author

infodog commented Jun 9, 2018

@sijie - I found the problem, in bookkeeper-4.7.0/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOp.java

this function:

public void initiate() {
        if (parallelRead) {
            request = new ParallelReadRequest(lh.metadata.currentEnsemble, lh.ledgerId, prevEntryId + 1);
        } else {
            request = new SequenceReadRequest(lh.metadata.currentEnsemble, lh.ledgerId, prevEntryId + 1);
        }
        request.read();

        if (!parallelRead && lh.bk.getReadLACSpeculativeRequestPolicy().isPresent()) {
             lh.bk.getReadLACSpeculativeRequestPolicy().get().initiateSpeculativeRequest(scheduler, this);
        }
    }

the request is issued twice one is
request.read()
another one is
lh.bk.getReadLACSpeculativeRequestPolicy().get().initiateSpeculativeRequest(scheduler, this);

So in org.apache.bookkeeper.proto.PerChannelBookieClient.readEntryInternal, the request is put in to the completionObjects twice with different txnid.

so the request will be closed twice.

We should use another request object to initiateSpeculativeRequest?

The following is the logs showing the problem,we can see that cb.hashCode=126537557 is added twice with different txid. And the second one is added by SequenceReadRequest.maybeSendSpeculativeRead

2018-06-09 17:17:56 [ BookKeeperClientWorker-OrderedExecutor-0-0:167553 ] - [ WARN ] txid->cb,txid=7534, cb.hashCode=126537557 org.apache.bookkeeper.proto.PerChannelBookieClient.readEntryInternal(PerChannelBookieClient.java:744)
2018-06-09 17:17:57 [ DL-io-2:168017 ] - [ WARN ] readV3Response get ledgerId=50,entryId=-1,txnid=7532 org.apache.bookkeeper.proto.PerChannelBookieClient.readV3Response(PerChannelBookieClient.java:1243)
2018-06-09 17:17:57 [ DL-io-2:168018 ] - [ WARN ] readV3Response remove ledgerId=50,entryId=-1,txnid=7532 org.apache.bookkeeper.proto.PerChannelBookieClient.readV3Response(PerChannelBookieClient.java:1264)
2018-06-09 17:17:57 [ BookKeeperClientWorker-OrderedExecutor-0-0:168018 ] - [ WARN ] calling submitCallback,completeRequest hasValidResponse,request=1495323235,ledgerId=50,entryId=49 org.apache.bookkeeper.client.ReadLastConfirmedAndEntryOp.completeRequest(ReadLastConfirmedAndEntryOp.java:639)
2018-06-09 17:17:57 [ BookKeeperClientWorker-OrderedExecutor-0-0:168018 ] - [ WARN ] request close request.hash=1495323235request.entryImpl handle=401923138 , ledgerId=50,entryId=49 org.apache.bookkeeper.client.ReadLastConfirmedAndEntryOp.submitCallback(ReadLastConfirmedAndEntryOp.java:548)
2018-06-09 17:17:57 [ BookKeeperClientWorker-OrderedExecutor-0-0:168018 ] - [ WARN ] --- ledgerId=50,entryId=49, length=-1, handler=1123506920,entryImpl.hashCode=401923138 org.apache.bookkeeper.client.impl.LedgerEntryImpl.close(LedgerEntryImpl.java:167)
2018-06-09 17:17:57 [ BookKeeperClientWorker-OrderedExecutor-0-0:168018 ] - [ WARN ] ReadLastConfirmedAndEntryOp ledgerId=50,preEntryId=48 org.apache.bookkeeper.client.ReadLastConfirmedAndEntryOp.(ReadLastConfirmedAndEntryOp.java:461)
2018-06-09 17:17:57 [ BookKeeperClientWorker-OrderedExecutor-0-0:168018 ] - [ WARN ] +++ledgerId=50, entryId=49, handler=1123506920,entryImpl.hashCode=401923138 org.apache.bookkeeper.client.impl.LedgerEntryImpl.create(LedgerEntryImpl.java:55)
2018-06-09 17:17:57 [ BookKeeperClientWorker-OrderedExecutor-0-0:168018 ] - [ WARN ] initiate request.hashCode=338063374,ledgerId=50,preEntryId=48,parallelRead=false org.apache.bookkeeper.client.ReadLastConfirmedAndEntryOp.initiate(ReadLastConfirmedAndEntryOp.java:500)
2018-06-09 17:17:57 [ BookKeeperClientWorker-OrderedExecutor-0-0:168019 ] - [ WARN ] txid->cb,txid=7535, cb.hashCode=723857310 org.apache.bookkeeper.proto.PerChannelBookieClient.readEntryInternal(PerChannelBookieClient.java:744)
2018-06-09 17:17:57 [ BookKeeperClientWorker-OrderedExecutor-0-0:168019 ] - [ WARN ] initiateSpeculativeRequest request.hashCode=338063374,ledgerId=50,preEntryId=48,parallelRead=false org.apache.bookkeeper.client.ReadLastConfirmedAndEntryOp.initiate(ReadLastConfirmedAndEntryOp.java:504)
2018-06-09 17:17:57 [ BookKeeperClientWorker-OrderedExecutor-0-0:168054 ] - [ WARN ] maybeSendSpeculativeRead,request.hashCode=1188747747 org.apache.bookkeeper.client.ReadLastConfirmedAndEntryOp$SequenceReadRequest.maybeSendSpeculativeRead(ReadLastConfirmedAndEntryOp.java:359)
2018-06-09 17:17:57 [ BookKeeperClientWorker-OrderedExecutor-0-0:168071 ] - [ WARN ] txid->cb,txid=7536, cb.hashCode=126537557 org.apache.bookkeeper.proto.PerChannelBookieClient.readEntryInternal(PerChannelBookieClient.java:744)
2018-06-09 17:17:57 [ DL-io-0:168521 ] - [ WARN ] readV3Response get ledgerId=50,entryId=-1,txnid=7533 org.apache.bookkeeper.proto.PerChannelBookieClient.readV3Response(PerChannelBookieClient.java:1243)
2018-06-09 17:17:57 [ DL-io-0:168521 ] - [ WARN ] readV3Response remove ledgerId=50,entryId=-1,txnid=7533 org.apache.bookkeeper.proto.PerChannelBookieClient.readV3Response(PerChannelBookieClient.java:1264)
2018-06-09 17:17:57 [ DL-io-2:168558 ] - [ WARN ] readV3Response get ledgerId=206,entryId=-1,txnid=7534 org.apache.bookkeeper.proto.PerChannelBookieClient.readV3Response(PerChannelBookieClient.java:1243)

@sijie
Copy link
Member

sijie commented Jun 9, 2018

@infodog nice catch! I think the problem comes from speculative reads with object recycling, the problem exists both at speculative reads and speculative longpoll reads. the read objects are recycled without cancelling scheduled speculations.

are you interested in providing a fix for that?

@infodog
Copy link
Author

infodog commented Jun 10, 2018

@sijie OK, I interested in providing a fix.

infodog pushed a commit to infodog/bookkeeper that referenced this issue Jun 10, 2018
@sijie sijie closed this as completed in 6476fc3 Jun 14, 2018
sijie added a commit that referenced this issue Jun 14, 2018
…ryOp

Descriptions of the changes in this PR:

The issue #1476 is caused by peculative reads with object recycling, same request object will be added to the CompletionObjects multiple times with different txnid.  In fact the logic of process the request already take this into account, only on place inside `ReadLastConfirmedAndEntryOp.requestComplete` forget to check requestComplete before calling `submitCallback` which in turn call request.close.

### Motivation

to fix #1476

### Changes

check `requestComplete` before `submitCallback` in `ReadLastConfirmedAndEntryOp.requestComplete`

Master Issue: #1476

Author: Sijie Guo <sijie@apache.org>
Author: infodog <infodog@hotmail.com>
Author: zhengxiangyang <zxy@xinshi.net>

Reviewers: Enrico Olivelli <eolivelli@gmail.com>, Jia Zhai <None>

This closes #1509 from infodog/issue1476, closes #1476

(cherry picked from commit 6476fc3)
Signed-off-by: Sijie Guo <sijie@apache.org>
reddycharan pushed a commit to reddycharan/bookkeeper that referenced this issue Oct 17, 2018
…AndEntryOp

Descriptions of the changes in this PR:

The issue apache#1476 is caused by peculative reads with object recycling, same request object will be added to the CompletionObjects multiple times with different txnid.  In fact the logic of process the request already take this into account, only on place inside `ReadLastConfirmedAndEntryOp.requestComplete` forget to check requestComplete before calling `submitCallback` which in turn call request.close.

### Motivation

to fix apache#1476

### Changes

check `requestComplete` before `submitCallback` in `ReadLastConfirmedAndEntryOp.requestComplete`

Master Issue: apache#1476

Author: Sijie Guo <sijie@apache.org>
Author: infodog <infodog@hotmail.com>
Author: zhengxiangyang <zxy@xinshi.net>

Reviewers: Enrico Olivelli <eolivelli@gmail.com>, Jia Zhai <None>

This closes apache#1509 from infodog/issue1476, closes apache#1476

(cherry picked from commit 6476fc3)
Signed-off-by: Sijie Guo <sijie@apache.org>
(cherry picked from commit c7b1610)
Signed-off-by: JV Jujjuri <vjujjuri@salesforce.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment