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

Issue 1476: LedgerEntry is recycled twice at ReadLastConfirmedAndEntryOp #1509

Closed
wants to merge 6 commits into from

Conversation

infodog
Copy link

@infodog infodog commented Jun 10, 2018

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


In order to uphold a high standard for quality for code contributions, Apache BookKeeper runs various precommit
checks for pull requests. A pull request can only be merged when it passes precommit checks. However running all
the precommit checks can take a long time, some trivial changes don't need to run all the precommit checks. You
can check following list to skip the tests that don't need to run for your pull request. Leave them unchecked if
you are not sure, committers will help you:

  • [skip bookkeeper-server bookie tests]: skip testing org.apache.bookkeeper.bookie in bookkeeper-server module.
  • [skip bookkeeper-server client tests]: skip testing org.apache.bookkeeper.client in bookkeeper-server module.
  • [skip bookkeeper-server replication tests]: skip testing org.apache.bookkeeper.replication in bookkeeper-server module.
  • [skip bookkeeper-server tls tests]: skip testing org.apache.bookkeeper.tls in bookkeeper-server module.
  • [skip bookkeeper-server remaining tests]: skip testing all other tests in bookkeeper-server module.
  • [skip integration tests]: skip docker based integration tests. if you make java code changes, you shouldn't skip integration tests.
  • [skip build java8]: skip build on java8. ONLY skip this when ONLY changing files under documentation under site.
  • [skip build java9]: skip build on java9. ONLY skip this when ONLY changing files under documentation under site.


Be sure to do all of the following to help us incorporate your contribution
quickly and easily:

If this PR is a BookKeeper Proposal (BP):

  • Make sure the PR title is formatted like:
    <BP-#>: Description of bookkeeper proposal
    e.g. BP-1: 64 bits ledger is support
  • Attach the master issue link in the description of this PR.
  • Attach the google doc link if the BP is written in Google Doc.

Otherwise:

  • Make sure the PR title is formatted like:
    <Issue #>: Description of pull request
    e.g. Issue 123: Description ...
  • Make sure tests pass via mvn clean apache-rat:check install spotbugs:check.
  • Replace <Issue #> in the title with the actual Issue number.

@eolivelli
Copy link
Contributor

Thank you @infodog for looking into this.

How can you prove the fix addresses the problem?
Is there any test case we can add in order ensure wr won't fall into this problem in the future?

@infodog
Copy link
Author

infodog commented Jun 11, 2018

@eolivelli
I setup a bookkeeper cluster with 3 bookie server, and then setup two clients read the logs , the client code is as following:

void downloadLog() throws IOException {
        long nextTxId = getLocalLastTxId();
        long lastShardTxId = 0;
        try {
            lastShardTxId = dlm.getLastTxId();

        } catch (Throwable t) {
            //当日志为空的时候,dlm.getLastTxId会出异常,所以需要try住
            //do nothing
        }
        LogReader reader = dlm.getInputStream(nextTxId);

        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()) {
                            updateLocalLog(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();
            }
        }
    }

when I start a writer write to the bookkeeper server, after serveral thoundsand writes, the client will throw the exception saying that object recycled , then the client will not receive any new data. In fact I start 2 clients in the same time to read from the server,and always one client fails, randomly.

After apply the fix, my client will run and always get the updated data. So I think the fix addressed the problem.

Maybe we can create a test case like this? But I dont know how to do that. I think to repoduce the problem the key point is multiple bookies. The problem will arise when the response from the bookie with data comes after the response of the bookie without data.

By the way, When I debug the problem, I add some log statements to the code, and the logs shows there is still some problem hiding. Still shows something I can't understand. Maybe I am not look carefully enough :

the following is my log,you can see there is still error, but i think it's another problem and this will not cause data lose, so I think the problem is addressed.

2018-06-10 09:25:26 [ BookKeeperClientWorker-OrderedExecutor-0-0:49040066 ] - [ WARN ] --- ledgerId=206,entryId=2083, length=-1, handler=395656841,entryImpl.hashCode=609480430 org.apache.bookkeeper.client.impl.LedgerEntryImpl.close(LedgerEntryImpl.java:167)
2018-06-10 09:25:26 [ BookKeeperClientWorker-OrderedExecutor-0-0:49040066 ] - [ WARN ] ReadLastConfirmedAndEntryOp ledgerId=206,preEntryId=2082 org.apache.bookkeeper.client.ReadLastConfirmedAndEntryOp.(ReadLastConfirmedAndEntryOp.java:461)
2018-06-10 09:25:26 [ BookKeeperClientWorker-OrderedExecutor-0-0:49040066 ] - [ WARN ] +++ledgerId=206, entryId=2083, handler=395656841,entryImpl.hashCode=609480430 org.apache.bookkeeper.client.impl.LedgerEntryImpl.create(LedgerEntryImpl.java:55)
2018-06-10 09:25:26 [ BookKeeperClientWorker-OrderedExecutor-0-0:49040066 ] - [ WARN ] initiate request.hashCode=926536875,ledgerId=206,preEntryId=2082,parallelRead=false org.apache.bookkeeper.client.ReadLastConfirmedAndEntryOp.initiate(ReadLastConfirmedAndEntryOp.java:500)
2018-06-10 09:25:26 [ BookKeeperClientWorker-OrderedExecutor-0-0:49040066 ] - [ WARN ] txid->cb,txid=97404, cb.hashCode=940171436 org.apache.bookkeeper.proto.PerChannelBookieClient.readEntryInternal(PerChannelBookieClient.java:744)
2018-06-10 09:25:27 [ DL-io-2:49040901 ] - [ WARN ] readV3Response get ledgerId=50,entryId=-1,txnid=97403 org.apache.bookkeeper.proto.PerChannelBookieClient.readV3Response(PerChannelBookieClient.java:1243)
2018-06-10 09:25:27 [ DL-io-2:49040901 ] - [ WARN ] readV3Response remove ledgerId=50,entryId=-1,txnid=97403 org.apache.bookkeeper.proto.PerChannelBookieClient.readV3Response(PerChannelBookieClient.java:1264)
2018-06-10 09:25:27 [ BookKeeperClientWorker-OrderedExecutor-0-0:49040901 ] - [ WARN ] calling submitCallback,completeRequest hasValidResponse,request=617824970,ledgerId=50,entryId=49 org.apache.bookkeeper.client.ReadLastConfirmedAndEntryOp.completeRequest(ReadLastConfirmedAndEntryOp.java:640)
2018-06-10 09:25:27 [ BookKeeperClientWorker-OrderedExecutor-0-0:49040902 ] - [ WARN ] request close request.hash=617824970request.entryImpl handle=2003401814 , ledgerId=50,entryId=49 org.apache.bookkeeper.client.ReadLastConfirmedAndEntryOp.submitCallback(ReadLastConfirmedAndEntryOp.java:549)
2018-06-10 09:25:27 [ BookKeeperClientWorker-OrderedExecutor-0-0:49040902 ] - [ WARN ] --- ledgerId=50,entryId=49, length=-1, handler=1827103622,entryImpl.hashCode=2003401814 org.apache.bookkeeper.client.impl.LedgerEntryImpl.close(LedgerEntryImpl.java:167)
2018-06-10 09:25:27 [ DL-io-1:49041072 ] - [ WARN ] readV3Response get ledgerId=206,entryId=-1,txnid=97404 org.apache.bookkeeper.proto.PerChannelBookieClient.readV3Response(PerChannelBookieClient.java:1243)
2018-06-10 09:25:44 [ DL-io-1:49057519 ] - [ ERROR ] readV3Response remove v==null, txnid=97404, op=READ_ENTRY org.apache.bookkeeper.proto.PerChannelBookieClient.readV3Response(PerChannelBookieClient.java:1261)
2018-06-10 09:25:44 [ BookKeeperClientWorker-OrderedExecutor-0-0:49057519 ] - [ WARN ] ReadLastConfirmedAndEntryOp ledgerId=50,preEntryId=48 org.apache.bookkeeper.client.ReadLastConfirmedAndEntryOp.(ReadLastConfirmedAndEntryOp.java:461)
2018-06

@eolivelli
Copy link
Contributor

@infodog I see your description.
It is better to fix the first known issue with your current work, then you will create a follow up issue if this is no enough.

@sijie I don't know DL codebase very much and I am not using long-poll so I can't be very helpful here.
Don't we have some "tailing reader" test case ?
Or maybe existing tests are too narrow that does not make the problem happen.

@sijie
Copy link
Member

sijie commented Jun 11, 2018

@eolivelli I think the problem only happened when speculations happen. also it is not just on long poll reads, I think it will impact normal reads. those tests around speculations might end before real problems show up.

@eolivelli
Copy link
Contributor

So @sijie you are saying that we have some code path not covered by test cases.

It would be good to have minimal coverage of this change, just by using mockito.
How does it sounds to you?

@sijie
Copy link
Member

sijie commented Jun 11, 2018

@eolivelli - sure, we need to add test cases for this. however I don't think the fix address the root cause. still looking ...

@eolivelli
Copy link
Contributor

@sijie sure, go ahead.

*Problem*

There are two flags on checking whether a request is completed. That is being misued for two different branches,
one is when the request is completed with advanced lac but no entry piggybacked, the other one is when the request
is completed with adavanced lac with an entry piggybacked. When this happen, it will cause a request being completed
twice and the entry buffer is recycled twice.

*Solution*

Remove direct usage of submitCallback and use completeRequest instead.

Add a unit test for reproduce the issue and ensure the fix address the problem.
@sijie
Copy link
Member

sijie commented Jun 12, 2018

@infodog :

your fix is correct. however, it can be simplified with changing using submitCallback to use completeRequest. this would guarantee a request only being completed once.

I created a PR to your branch to improve it and also added a unit test to reproduce the problem and ensure your changes fix the problem: infodog#1 if you merged it, the changes will be applied to your branch and showed up here.

then we should be ready to merge your fix.

@sijie
Copy link
Member

sijie commented Jun 12, 2018

@eolivelli - changes in infodog#1 include the unit test.

@jiazhai @yzang please review this PR as well, since it is dlog related.

@infodog
Copy link
Author

infodog commented Jun 12, 2018

@sijie I think the fix of infodog#1 may have problem.

because
request.complete(rCtx.getBookieIndex(), bookie, buffer, entryId) will be called even the request is already closed.

In request.complete

entryImpl.setLength(buffer.getLong(DigestManager.METADATA_LENGTH - 8));
entryImpl.setEntryBuf(content);

the entryImpl may already recycled and owned by another request, so this will ruin another request's data?

Maybe should move

writeSet.recycle();
orderedEnsemble.recycle();

from request.complete to request.close, and when we found thre requested is closed, we dont call request.complete?

@sijie
Copy link
Member

sijie commented Jun 12, 2018

@infodog - https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOp.java#L127 use an atomic boolean to guarantee it only executed once. so the problem you described won't happen.

@infodog
Copy link
Author

infodog commented Jun 12, 2018

@sijie but in https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOp.java#L615 use requestComplete to guarantee request is only close once, while https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOp.java#L127 use complete to guarantee ,they are two diff variables. So it's possible after the request is closed at the same time entryImpl is recycled, the https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOp.java#L127 code will be executed, and the data maybe ruined.

@sijie
Copy link
Member

sijie commented Jun 12, 2018

@infodog I see your point now. let me try to explain:

  1. yes. there are two flags, the completed flag in request is to checking if the request itself is completed or not, the requestComplete flag is in the ReadLacOp is to checking if the op itself is completed or not. in some sense, we need these two different flags, since one op can potentially contain multiple requests. it might be a bit strange in this class, since in this case, one operation only have one request.

  2. are there any race conditions between these two flags? the answer is no. because the request callback can only happen on one thread. so if a request successfully completed with a valid entry, request.complete returns true and then it will submit the callback, which will set requestComplete to true. so any future callback will not try to complete the request or submit callback again, this prevents the race condition you described.

  3. just for your information, in bookkeeper client, the operations/callbacks for same ledger will be executed in same thread.

Hope this explains why the change in infodog#1 work.

@infodog
Copy link
Author

infodog commented Jun 12, 2018

@sijie it's possible that compleRequest() come before request.complte(), such as https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOp.java#L589, this will happen when the response without piggyback data comes before the response with data.

In this case, the request will be closed in completeRequest() by the submitCallback() in it, the entryImpl will be recycled. Because completeRequest() does not set comlete to true, later when request.complete being called,

writeSet.recycle();
orderedEnsemble.recycle(); 

will be executed,and although this time request.close will not be called. Since entryImpl is already recycled , if entryImpl is resued by other request, then data will be ruined even the operation is executed in the same thread.

@sijie
Copy link
Member

sijie commented Jun 12, 2018

@infodog you are correct! I will revert infodog#1 to your original fix, I will update the unit test to ensure verify the case your described.

- add validations in the unit test to ensure the recycled entry will not be mutated
@sijie
Copy link
Member

sijie commented Jun 12, 2018

@infodog I've updated infodog#1 with your original fix and updated the unit test to add validations to make sure recycled entry will not be mutated by any subsequent callbacks. please take a look and let me know if it makes sense to you.

@infodog
Copy link
Author

infodog commented Jun 13, 2018

@sijie I still have questions,

  1. Although the fix will solve the object recycle problem, but the response with valid data is ignored. Will this case cause problem to the application using dl as a replication service? How the application using the dl client shoud do to handle this situation?
    Is there any retry mechanism embeded by the bookkeeper client? I think as a quick fix, if this happed the client should throw a exception to the application, so that the application can implement its own retry mechanism. Maybe we should not treat this situation ( the response with valid data come after response withoud data situation) as an error one.When we receive an empty reponse, we should not close the request? Or we should impletement a retry mechanism after this situation happend?

  2. back to the fix detail, if request.complete is not called, then on https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOp.java#L127

writeSet.recycle();
orderedEnsemble.recycle(); 

is not executed.
will this cause resource leak?

  1. if requestComplete already set the buffer.retain(); here is redundant

@sijie
Copy link
Member

sijie commented Jun 13, 2018

@infodog good questions, comments inline.

Will this case cause problem to the application using dl as a replication service?

This will not cause any problems to the application using DL. Here we completed the request only when lac (last add confirmed) is advanced. it is better if there is an entry piggybacked, if there is no entry piggybacked, it is still good, because we received a valid advanced lac.

you might think we dropped a valid response. however the thought behind is to tell the client caller (which is DL library at this case) as soon as LAC is advanced, it can read entries till the new LAC. DL will react to the callback and know LAC is advanced and issue corresponding read requests.

How the application using the dl client shoud do to handle this situation?

You don't need to handle this situation. Since DL handles that.

Is there any retry mechanism embeded by the bookkeeper client?

The conversation between normal reads and long poll reads and knowing LAC advance are all happending in the dl library. applications don't have to worry about that.

will this cause resource leak?

This will not cause resource leak. The recycle there means putting back the objects to be reused. if recycle is not called, it doesn't mean resource leak, it only means the objects will not be put back to the object pool to be reused.

the recycle objects are different from bytebuf where releasing bytebuf should be done when the bytebuf is not reused anymore, otherwise it will causing memory can't be released.

if requestComplete already set the buffer.retain(), here is redundant

I am not sure what exactly you are asking. but I am guessing you are asking the boolean flag in the request. that boolean flag is needed, it is used for flagging whether the request is completed or not. the requestComplete is used for checking if the operation is completed (or callback is submitted or not). Those booleans have different meanings. It is not easy to tell the difference from first glance.

Hope this clarifies your questions.

@sijie
Copy link
Member

sijie commented Jun 13, 2018

@infodog if the comments make sense to you, it would be great if you can merge infodog#1 to your branch, so it can be showed up here. then we can merge this change once it passes other committers's review and CI checks.

 Issue 1476: LedgerEntry is recycled twice at ReadLastConfirmedAndEntryOp
@infodog
Copy link
Author

infodog commented Jun 13, 2018

@sijie Ok thanks very much for the explanation, I already merged to my branch.

@sijie
Copy link
Member

sijie commented Jun 13, 2018

@infodog thank you.

@jiazhai @yzang @eolivelli can you guys review this change?

Copy link
Member

@jiazhai jiazhai left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 with [commit] (b2b4f22)

@sijie sijie changed the title solve #1476 issue. Issue 1476: LedgerEntry is recycled twice at ReadLastConfirmedAndEntryOp Jun 13, 2018
@sijie
Copy link
Member

sijie commented Jun 13, 2018

run bookkeeper-server bookie tests

@sijie
Copy link
Member

sijie commented Jun 13, 2018

rebuild java8


// readEntryComplete above will release the entry impl back to the object pools.
// we want to make sure after the entry is recycled, it will not be mutated by any future callbacks.
LedgerEntryImpl entry = LedgerEntryImpl.create(LEDGERID, Long.MAX_VALUE);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@sijie @infodog
are you assuming that here we are getting the same entry as above ?

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@eolivelli
Yes I assuming that.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is there any way to perform an assertion about this assumption ?
if that assumption is broken the test is not really useful.

I don't have a suggestion on how to capture the LedgerEntryImpl.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@eolivelli It is hard to capture the original entry. I mean it is possible, however that's going to make things a bit complicated, which I don't think it is worth doing that.

Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

overall ok, just a question on the test

@sijie
Copy link
Member

sijie commented Jun 13, 2018

run bookkeeper-server bookie tests

@sijie
Copy link
Member

sijie commented Jun 13, 2018

rubuild java8

Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it is not worth to complicate the test. I assume that without the fix the test is failing.

+1 LGTM
awesome work @infodog and @sijie !

@sijie
Copy link
Member

sijie commented Jun 13, 2018

run bookkeeper-server bookie tests

1 similar comment
@sijie
Copy link
Member

sijie commented Jun 13, 2018

run bookkeeper-server bookie tests

@sijie
Copy link
Member

sijie commented Jun 14, 2018

retest this please

@sijie
Copy link
Member

sijie commented Jun 14, 2018

(somehow the jenkins precommit checks are not running after rebased. trying to close and reopen)

@sijie sijie closed this Jun 14, 2018
@sijie sijie reopened this Jun 14, 2018
@sijie
Copy link
Member

sijie commented Jun 14, 2018

run bookkeeper-server bookie tests

@sijie
Copy link
Member

sijie commented Jun 14, 2018

run bookkeeper-server bookie tests
run bookkeeper-server tls tests
run pr validation

@sijie
Copy link
Member

sijie commented Jun 14, 2018

I think "bookie tests" precommit check is flaky due to #1516 .

there was one successful run - https://builds.apache.org/job/bookkeeper_precommit_bookie_tests/61/

so going to ignore CI to merge this bug fix.

@sijie
Copy link
Member

sijie commented Jun 14, 2018

IGNORE CI

@sijie sijie closed this in 6476fc3 Jun 14, 2018
sijie added a commit that referenced this pull request 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>
@sijie
Copy link
Member

sijie commented Jun 14, 2018

merged the changes. thank you @infodog !

reddycharan pushed a commit to reddycharan/bookkeeper that referenced this pull request 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
Projects
None yet
Development

Successfully merging this pull request may close these issues.

recycle already.
5 participants