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
[Bug] parseMessageMetadata error when broker entry metadata enable with high loading #22601
Comments
@semistone Just wondering if this could be related to apache/bookkeeper#4196? |
we still try to compare what's the different between our producer and perf tool |
@lhotari
and producer setting (small batch message and big max bytes)
payload then it will show that error and publish throughput isn't good. but if we change to
and filter all data bigger than 15K bytes so we decide to create we still don't known why I don't how which batch producer configuration could fix this errors. and we also publish in multi thread programs, we also tried to reproduce by perf tool but it didn't always happen. thanks |
I tried to upgrade to bookkeeper 4.17.0
|
@semistone Please share a way how to reproduce it. It's not a problem if it's not always consistent. Fixing this issue will be a lot easier if there's at least some way to reproduce. |
@semistone Thanks for testing this. |
I will try to reproduce in perf tool. |
@semistone since you have some way to reproduce this in your own tests, would you be able to test if this can be reproduced with Lines 435 to 436 in 80d4675
It impacts this code: Lines 659 to 681 in 188355b
|
I almost could reproduce by perf tool I guess in batch mode, payload size have some restriction. let me confirm again tomorrow to make sure I didn't make any stupid mistake during my test. |
Hi @lhotari it only include one commit which modify PerformanceProducer.java to include my consumer command is
and producer command is
that error happen when Batch builder is KEY_BASE in my test when it happen it will have WARN message in pulsar-broker.log
unfortunately I can't preproduce in docker, I guess docker standalone is different from my pulsar cluster. Please help to check it, if have any problem to reproduce this issue in your environment, Thanks |
I tested, still the same |
@semistone Thanks for testing. That tells that it's not related to switching the thread in Lines 659 to 681 in 188355b
|
@lhotari and during run(), I try to compare and print bytebuf
it show
the bytebuf object haven't changed, but the data in bytebuf have changed. and it's PooledSlicedByteBuf and PooledUnsafeDirectByteBuf do you have any idea how to find who change the data inside bytebuf ? |
I also test again I also checked the data again and it not related to batch mode |
There's some thought about this in #22110. That's a WIP PR and it might not make sense in the end. Since then, I have found a pattern used in Netty to detect issues. I'll look that up. |
@semistone do you also use Pulsar Proxy? |
@lhotari no, I didn't |
This is a useful detail. When messageKeyGenerationMode is random and BatcherBuilder.KEY_BASED is used, each batch will be size of 1. This could hint that the problem is related to #16605 changes, main difference here: pulsar/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageContainerImpl.java Lines 252 to 286 in 6b29382
|
@semistone would it be possible to share your broker.conf customizations? That could help reproduce the issue. I noticed |
Yes, I enabled it and after debugging, I found
then that issue seem disappear. but not sure is there any side affect or not. here is our broker.conf and I remove all of the password |
I also debug in I don't see the same bytebuf object been reused during OpAddEntry.createNoRetainBuffer and OpAddEntry.run |
Interesting detail. does the problem also go away with |
I test it and It seems also work I will repeat success/failure test later to confirm it again. |
I've been trying to reproduce the issue with local microk8s cluster by deploying Pulsar with Apache Pulsar Helm chart using this values file: https://github.com/lhotari/pulsar-playground/blob/master/test-env/issue22601.yaml |
How many brokers and bookies do you have in the cluster where it reproduces? |
I have 6 bookkeeper in 3 different data center and I left only one broker running for debug and it run on physical server and CentOS Linux release 7.9.2009 |
I add big payload options |
I was using What gets logged when the issue reproduces? |
do you happen to run with debug logging level when the issue reproduces? pulsar/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java Lines 1799 to 1801 in a66ff17
pulsar/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java Lines 1859 to 1872 in a66ff17
|
|
when I print bytebuf.toString() , ridx seem didn't change I didn't turn on debug log |
consumer
producer
I'm using the scripts in https://github.com/lhotari/pulsar-playground/tree/master/issues/issue22601 and those have Which JVM version and parameters are you using when you are able to reproduce? |
since change readonly could fix it |
which JVM args do you run with? For example heap size etc. |
It's also possible that nothing touches it, but it's due to a multithreading issue. one useful experiment would be to make the Another experiment could be to temporarily disable the use of the |
I was able to reproduce a few issues with the test setup. However, these might be different issues. logs are at https://gist.github.com/lhotari/8302131cde5a0f0999e39f8fbd391f09 .
also
The way I reproduced this was that I used the scripts in https://github.com/lhotari/pulsar-playground/tree/master/issues/issue22601 . I modified |
also got this type of exception
|
This
I can reproduce consistently:
This might be related to the |
my JVM option is
my server have I feel it may related to cpu speed. I will debug a little deeper to see anything strange. |
I couldn't reproduce with Pulsar Standalone, but I have a way with a local Microk8s cluster where I could also attach a debugger. With break points in java.lang.IllegalArgumentException and java.nio.BufferUnderflowException, I can see the problem. This issue happens when pulsar/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/ByteBufPair.java Line 149 in 82237d3
There's a feature in Netty that
In Netty, the SslHandler will access the underlying ByteBuffer instances directly. This leads to a similar multi-threading problem as the use of I think that the problem is now clear where it happens, but the solution to fix this isn't yet known. |
Looking up in Netty issue tracker. Found these issues that provide a lot of context:
It seems that this is a long time issue and it has been partially fixed. However, it's not fixed for many locations in the Netty code base and it's not safe to share ByteBuf instances in all cases. We need to find a workaround for the Pulsar use cases. |
This is a great finding |
Thanks for helping check this issue , but at least it could reproduce in your side... and I repeat double check that
this two behavior. Our cluster is for BCP, so the bookkeepers are spread across three data centers, and the network latency is about 10ms. |
@semistone I have created a PR #22760 to fix the problem. It's currently in draft state since I'm currently testing the solution to verify that it mitigates the problem. |
seem change to read only actually change the behavior... :( I will check and test it tomorrow |
I have tested #22760 with the repro case and scripts that are based on the modified pulsar-perf and other details provided by @semistone. The issue no longer reproduces in my microk8s test setup. |
|
I tested with that patch.. I may try to recreate my cluster this week as standalone server to check it again |
@semistone It's better to base on 3.2.3 since there are also other issues where which could result in the same symptoms. I'm also hitting other issues when running the repro scripts. @semistone would you be able to share more details of your repro sequence? I shared complete details of my setup and repro steps in https://github.com/lhotari/pulsar-playground/tree/master/issues/issue22601 so that others could build upon it if they wish. You could decide to share what you wish, but obviously having more details will be helpful. |
I will do it later this week. |
It's likely that this is a Bookkeeper client issue. The PR #22760 might be fixing a different issue. When using TLS between Brokers and Bookies, Bookkeeper will use V3 bookkeeper protocol. This implementation is different from the V2 protocol which is used by default in Pulsar. That is one additional reason why certain bugs appear only when TLS is enabled. |
I finally found the root cause in Bookkeeper client. The client had several buffer leaks since the ByteBufList lifecycle was incorrectly handled and write promises were dropped and ignored when authentication was in progress. The fix is in apache/bookkeeper#4293. |
I write repproduce steps and some investigate history in I tested from install step and confirmed it only happen when bookkeeper TLS enable I will test that patch next week |
Search before asking
Read release policy
Version
Minimal reproduce step
publish event in about 6k QPS and 100Mbits/sec
with metaData
BatcherBuilder.KEY_BASED mode
and producer and send message by high concurrent/parallel producer process.
it happens only in almost real time consumer (almost zero backlog)
What did you expect to see?
no lost event
What did you see instead?
could see error log in broker and show
Failed to peek sticky key from the message metadata
it look like thread safe issue, because it happen randomly.
in 1M events, it only happen few times but the consumer will lose few events
Anything else?
the error similar to
#10967 but I think it's different issue.
the data in bookkeeper is correct.
I can download the event from bookkeeper and parse it successfully.
or consume the same event few minutes later and it could consume successfully.
but all subscriptions will get the same error in the same event in real time consumer(zero backlog).
I have traced source code.
it happens in
PersistentDispatcherMultipleConsumers.readEntriesComplete -> AbstractBaseDispatcher.filterEntriesForConsumer
-> Commands.peekAndCopyMessageMetadata
and I also print the ByteBuf contents,
it's I could clearly see the data isn't the same in bookkeeper
in normal event , the hex code usually start by 010e (magicCrc32c)
in one of our error event, the bytebuf have about 48 bytes strange data, then continue with normal data
this is just an example, but sometimes the first few bytes are correct and something wrong after few bytes later.
I am still trying to debug when and how the ByteBuf returns incorrect data, and why it only happens during stress testing. It is still not easy to reproduce using the perf tool, but we can 100% reproduce it in our producer code.
Does anyone have any idea what could be causing this issue, and any suggestions on which library or class may have potential issues? Additionally, any suggestions on how to debug this issue or if I need to print any specific information to help identify the root cause would be appreciated. Thank you.
Are you willing to submit a PR?
The text was updated successfully, but these errors were encountered: