-
Notifications
You must be signed in to change notification settings - Fork 3.5k
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
[Proto] java.lang.IllegalStateException: Some required fields are missing #14436
Comments
|
I made an attempt to fix this issue in #14556. The pulsar/pulsar-common/src/main/proto/PulsarApi.proto Lines 105 to 107 in db25438
One interesting detail is this code: pulsar/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java Lines 565 to 577 in 5bcdb5e
I guess that's fine (similar logic has been in place since the initial commit). The problem is more likely a thread safety issue or shared thread local instance issue. That's why #14556 could be effective. |
@lhotari hi, I add some log. I found that, when the ByteBuf read from bookie client, the data length is not correct and the data is not this entry data which you want to read. So this problem may only the ByteBuf not clear enough, it may netty bug or bookie problem |
@congbobo184 Perhaps the ByteBuf was already released and this resulted in the memory location to get reused for something else? |
yes, most likely because Bytebuf was not released cleanly and reused |
@congbobo184 @merlimat @codelipenghui @BewareMyPower When digging into the code, I circle back to the fact that OpReadEntry is not thread safe. OpReadEntry.entries mutated and accessed from multiple threads. That is a big problem. I have an attempt to improve the situation with #11387, but that's not ready and won't resolve the issue completely. The description in #11387 contains some explanation of the issue. Do we all agree that OpReadEntry is not thread safe and that is a problem that must be resolved? |
Yes. But it looks like there are some problems with the pin executor solution. Could we solve this problem by changing |
But it looks like that the thread safety issue is not the cause of this issue. |
@BewareMyPower What's the current assumption of the cause of this issue? |
That could work, but I'm not sure about the consequences for performance. It would be worth testing to see what the impact would be. |
I'm not working on this issue currently, just because this thread safety issue has existed from the very beginning. If this issue could be reproduced only by Pulsar 2.10, then the cause might not be related to |
Ok, so you don't have an assumption of the root cause? @congbobo184 Could you please try running the test case with Please be aware that |
Hi @lhotari , I start up standalone with -Dio.netty.recycler.maxCapacity=0 -Dio.netty.recycler.maxCapacityPerThread=0 the problem seem not reproduce, I don’t know is it because of performance degradation, or because this netty configuration makes it unreproduce.I log the buffer length from reading bookie, So I think the problem might not be with the broker |
@congbobo184 Thank you. Based on this experiment we cannot determine the issue, but since disabling Netty Recycler has an impact, there's a possibility that recycling instances is causing issues in a way or another. The next experiment could be to leave the Netty Recycler setting unchanged (enabled) and instead comment out the line That will disable recycling of OpReadEntry instances. If that makes the problem go away, it's again more information about the possible problem. @congbobo184 Would you be able to continue with this experiment? |
I roll back netty to 4.1.68.Final (requires netty-tc-native.version 2.0.42.Final) follow @lhotari advice, the problem can't be reproduce. The last time I only change netty version to 4.1.68.Final and don't change netty-tc-native.version to 2.0.42.Final to 2.0.48.Final, this problem also happen. So the problem may be in netty-tc-native.version 2.0.48.Final. |
Do you happen to use TLS in your setup? If not, it seems odd why the netty-tc-native version would cause the problem to reproduce. btw. A major difference between Netty 4.1.68.Final and 4.1.74.Final is the Netty Recycler rewrite that was made in 4.1.71.Final. There's more info in this comment: #13328 (comment) |
I am not sure the problem in 2.0.48.Final. So I will test again. change netty version to 4.1.68.Final and don't change the netty-tcnative version |
Thanks for the analysis @congbobo184 . Really helpful.
This doesn't mean that we should accept the bug that is caused by thread safety issues. This is a severe issue and can lead to data inconsistency problems. As discussed privately, this problem seems to become more frequent with Netty 4.1.74.Final compare to Netty 4.1.68.Final version. Most likely this behavioral changes is caused by the new Netty Recycler that was introduced in Netty 4.1.71.Final. My assumption is that the new implementation is more efficient and brings thread safety issues to the surface. I'll bring this severe issue up to discussion on the Apache Pulsar and Apache Bookkeeper developer mailing lists. The quick workaround is to disable Netty Recycler completely by setting the system property |
@lhotari do you mean it's a netty issue ? Should we report it to the netty commuinty and revert the netty version ? |
I have tested with congbo, can't actually prove it for now. After downgrading the netty version, the problem still can be reproduced, and regarding the timing of the reproducing, it is uncertain even to use the same netty version. So far, we don't have an exact root cause, since the problem can be reproduced, I think we can find the root cause. |
No it's not a Netty issue. It's a problem caused by our code in Pulsar and Bookkeeper violating the Java Memory Model and thread safety rules. There are 2 possible solutions:
|
I meant to say that after the new Netty Recycler, it just happens to be more frequent. That's not the root cause. The root cause is the violation of the Java Memory Model and thread safety rules with the use of recycled objects across multiple threads. |
In this case, one example of a class that is not thread safe is LedgerEntriesImpl. It has a single field I'm not saying that full synchronization is required. In this case, it is sufficient to apply the "golden rule" that Aleksey Shipilev presents.
I'm thinking of an approach where there would be a base class for recycled objects with methods that should be called after writing fields and before reading fields in different threads. That would be sufficient to ensure thread safety without the need to have full synchronization. It could be implemented with a simple volatile field write and read. |
Usually every concurrent collections provide causality. For example, when a ConcurrentHashMap is used. It's all good when one thread writes fields and puts it to a ConcurrentHashMap and another thread gets it from the ConcurrentHashMap and reads the fields. There is causality and the reads are consistent. The problem comes with other threads that don't use the same "barrier". There's no guarantee of causality in that case. With asynchronous callbacks, multiple threads are used and there isn't a single barrier. which would guarantee causality. |
@lhotari what about opening a discussion on Netty ? |
I was thinking initially that "thread safety" should be achieved for recycled object classes. However that would be very intrusive since "safe publication" is sufficient to reaching thread safety in this case. There aren't cases where there would be races in writes in multiple threads. The object instance is used and passed across threads and as long as "safe publication" is achieved, that is sufficient. That's why I'm actually not so sure about the solution. "Safe publication" is also a mandatory consideration for new objects so it's not actually completely limited to recycled object instance. It's probably the case, that there's already causality for most execution paths. It's just very hard to analyse the code and find the execution paths where this isn't the case.
I don't think that this is a Netty problem.The usages of Netty Recycler within Netty are most likely covered with causality. I'm not sure if the Netty Recycler is used commonly outside of Netty. There are only 8 projects in grep.app that contain "import io.netty.util.Recycler" |
@congbobo184 Since this is related to BookKeeper client, could you please run an experiment with The reason for this request is that in the client, V2 requests don't have unique keys and I'm wondering if that could cause collisions: |
There seem to be cases where causality and "happens-before" isn't sufficient and classes must be thread safe. For example BookKeeper client's LedgerEntriesImpl contains this type of code:
If there are multiple threads that call this method, the change to the entries field wouldn't be visible to the other thread and this could cause to the "double release" problem that might be causing the problem in this issue where the logging added by @congbobo184 proved that a single instance was used in 2 locations simultaneously. @congbobo184 would you be able to make an experiment where you make all methods synchronized in LedgerEntriesImpl and add a synchronized setter to modify the "entries" field? This doesn't mean that it should be the way to resolve the issue. |
I was looking into the BookKeeper client and it seems that apache/bookkeeper#1792 breaks thread safety for read-only ledgers. |
Since the thread safety is broken only for read-only ledgers, it requires a ledger roll over to happen and the Pulsar ledger cache to be empty to reproduce the issue. These conditions seem to be met also in the way what describes how to reproduce the reported issue. |
@lhotari I am working in transaction issues. I'll take a look at this when I'm done with transaction issues |
@congbobo184 I filed apache/bookkeeper#3104 issue in BK for tracking this. |
Have you tried disabling speculative reads? Please see apache/bookkeeper#3106 (review) |
apache/bookkeeper#3110 will fix it |
The issue had no activity for 30 days, mark with Stale label. |
Describe the bug
![image](https://user-images.githubusercontent.com/39078850/155444823-a7283952-bc22-49e7-99b3-8b2df6a8f7f4.png)
parse message metadata throw Exception
To Reproduce
Steps to reproduce the behavior:
transactionCoordinatorEnabled=true
managedLedgerMaxEntriesPerLedger=3
managedLedgerMinLedgerRolloverTimeMinutes=1
Execute the above code multiple times
![image](https://user-images.githubusercontent.com/39078850/155444823-a7283952-bc22-49e7-99b3-8b2df6a8f7f4.png)
Execute
./pulsar-admin namespaces unload public/default
multiple times during execute the above code4. See error
Expected behavior
can't throw exception
Screenshots
If applicable, add screenshots to help explain your problem.
Desktop (please complete the following information):
Additional context
Not a stable reproduction, requires multiple trials
The text was updated successfully, but these errors were encountered: