-
Notifications
You must be signed in to change notification settings - Fork 3.6k
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
NPE in broker: EntryImpl.getLength() #10433
Comments
@dinghram Thanks for reporting. Did I understand the problem report correctly, that the NPE is happening on EntryImpl.java, line 132? pulsar/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryImpl.java Lines 130 to 133 in 8ea4a39
Called from EntryCacheImpl.java, line 115: pulsar/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryCacheImpl.java Lines 112 to 115 in 8ea4a39
Please share the Pulsar version and deployment type (plain/docker/k8s helm). The target Java JVM version would also be useful if you aren't deploying with official k8s helm chart and Docker images. One experiment that could be useful in tracking down the problem would be to add For example, In the k8s Helm chart deployment, an environment variable can be set for the broker by adding it under
The Would it be possible for you to do a test run with |
Though it won't disable the call to recycle, in where we're resetting the fields to nulls. |
@merlimat I guess the pulsar/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryCacheImpl.java Lines 112 to 115 in 8ea4a39
pulsar/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryImpl.java Lines 81 to 90 in 8ea4a39
The reported NPE is "impossible". :) |
the only explanation would be for the pulsar/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryCacheImpl.java Line 86 in 8ea4a39
|
But that too doesn’t look to be possible.... |
I wonder if something in this direction could be happening: Let's say we have 2 threads, T1 and T2 and a single object instance O1.
I've been recently watching Aleksey Shipilëv's Java Memory Model Unlearning Experience presentation. In the presentation, at about 21 minutes 50 seconds, there's an interesting example: |
@lhotari The pool here we are using is a thread-local pool, and always use the the same thread for a managed ledger to insert entry to the cache pulsar/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java Line 188 in 6704f12
And if passed here pulsar/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryCacheImpl.java Line 112 in 8ea4a39
|
@codelipenghui Thanks for pointing to the OpAddEntry code location. I initially didn't realize that the code had both pulsar/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java Lines 162 to 176 in 8ea4a39
Even with this updated information, the problem is "impossible"... :) This seems to be a nice Java Concurrency puzzler. |
@codelipenghui it seems that Netty's Recycler is thread-local for lookup and creation, but an object can be returned to the pool in any thread. The multi-thread access seems to lead to visibility problems which are possible in the JMM. |
Pulsar version: 2.7.0 + cherry-picked fixes |
There's a Netty bug that has been fixed in 4.1.60.Final by this commit. Pulsar 2.7.2 will be released with Netty 4.1.60.Final . Upgrade to Netty 4.1.60.Final in Pulsar was made as part of #10073 . @dinghram Would you be able to re-test with a build of Pulsar from branch-2.7 since that would include Netty 4.1.60.Final ? |
Another NPE bug that is related to the Netty Recycler: #5727 . |
Since the reported problem could be related to Netty Recycler usage, I'd recommend doing the testing with Netty Recycler completely turned off (this was already proposed in the first comment). You can find positive reports about disabling the Netty Recycler at netty/netty#5968 (comment) and netty/netty#5968 (comment) . Elasticsearch also disabled Netty Recycler completely because of the problems it causes. @dinghram Would you be able to run your CHO (Continuous Hours of Operation) tests with Pulsar configured with |
#2993 seems related. |
UPDATE: The following description of a race condition doesn't seem to be possible since a counter in the base class of EntryImpl protects against it. (AbstractCASReferenceCounted.release0)
The possible race condition: Code in EntryCacheImpl, asyncReadEntry0 method (code is for branch-2.7): pulsar/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryCacheImpl.java Lines 272 to 283 in dcaa1d3
Code for pulsar/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/RangeCache.java Lines 108 to 122 in 45ee784
explanation of a possible race condition:
|
this comment "// Value was already destroyed between get() and retain() " is very bad. |
When looking more at the possible race condition, the previous explained scenario seems to be possible with a slight variation to it. It seems that invalidation while the entry is in use is possible, but the remaining steps are different. It seems that is would require 2 concurrent invalidations so that the problem would manifest itself. It might sound like a very unlikely situation, but it seems that the problem of this issue is also very rare. With enough operations, a very unlikely call path will eventually get executed. |
I created a minimal fix without any new tests in #10480 . |
@lhotari Let me poke into this theory: Initial refCnt of the entry in cache == 1
A: value.retain() succeeds: refCnt++ == 2
A: refCnt-- == 1 "the EntryImpl instance gets returned to the Recycler object pool" does not happen in case A. Did I miss something there? |
@lhotari there are other players there: |
@dlg99 yes, you are right that it's not possible in the way that I first explained it. There's a more recent comment about the races in invalidating the entries which could lead to a somewhat similar situation. The referenced PR contains more discussion about the possible explanations. |
@lhotari Thanks for the feedback. We tested with the -Dio.netty.recycler.maxCapacityPerThread=0 option and saw no negative performance impact. In fact, the CPU usage was lower with this option set. However, this was a single test and not definitive of how Pulsar would behave long term with this setting. As such, it seems we can run with this option set if a customer begins seeing the NPE occur (we have only seen it once). |
This problem might be fixed by #13328 . Netty Recycler was rewritten in 4.1.71.Final with PR netty/netty#11858 . Pulsar uses Netty Recycler extensively and the improvements to Netty Recycler might fix various thread-safety problems. |
The issue had no activity for 30 days, mark with Stale label. |
This problem fixed by #15707. |
@mattisonchao as a committer you should be able to close this issue as completed :) |
@tisonkun Got it. thanks! |
During a CHO test, one broker crashed with a NPE. We lost the logs due to a cleanup of files, but we have the general stack trace:
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryImpl.java: 132: return data.readableBytes();
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryCacheImpl.java: 115: manager.entryAdded(entry.getLength());
The text was updated successfully, but these errors were encountered: