-
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
Consumer often discards received messages #2401
Comments
It seems that this error does not occur if I revert the following change and deploy it to the broker side. However, I don't understand the reason... |
@merlimat Do you have any thoughts about what the cause might be? |
This phenomenon seems to happen when the consumer is receiving a lot of messages. For example, it is prone to happen when the consumer connects to a topic with a large number of messages in the backlog. |
Looks like a bytebuf issue. When reading a bytebuf, we often do things like reading the size from the packet and setting the writeIndex to the current writeIndex + size, so that it can be passed into the protobuf parser. This is why the write index seems so messed up. Do you have a reliable repro, or do you just see it happening regularly in production? Is TLS auth enabled, or just TLS transport? this is going to be a bytebuf pooling issue of some sort with TLs. |
What rate are messages arriving at the topic in question? I'm trying to get a feel for how we could repro this locally. |
In our environment, it can be reproduced by using the OpenMessaging benchmark tool and applying the following load:
It seems that it tends to occur when there are many subscriptions. It also occurred several times in production.
We are not using TLS auth. |
I have a local repro, will look into it more in the morning. |
Sorry, OpenMessaging benchmark tool has not yet supported TLS... |
Yes, it's only a 2 line change to make it work. |
Spent some time on this today, still no root cause, but it looks very much like something is holding on to a bytebuf that has been released back to the pool, and then it's writing to it. Will continue looking. |
The netty SSL handler uses a coalescing buffer queue, which modifies the buffers used to queue the writes so that SSL_write can be given larger chunks, thereby increasing the 'goodput'. If we pass in a retained duplicate as we have been doing until now, then later clients will be passed junk, as SSL will have modified cached entry buffers. This patch introduces a copying ByteBufPair encoder, which is only used with SSL connections.
The netty SSL handler uses a coalescing buffer queue, which modifies the buffers used to queue the writes so that SSL_write can be given larger chunks, thereby increasing the 'goodput'. If we pass in a retained duplicate as we have been doing until now, then later clients will be passed junk, as SSL will have modified cached entry buffers. This patch introduces a copying ByteBufPair encoder, which is only used with SSL connections.
The netty SSL handler uses a coalescing buffer queue, which modifies the buffers used to queue the writes so that SSL_write can be given larger chunks, thereby increasing the 'goodput'. If we pass in a retained duplicate as we have been doing until now, then later clients will be passed junk, as SSL will have modified cached entry buffers. This patch introduces a copying ByteBufPair encoder, which is only used with SSL connections.
The netty SSL handler uses a coalescing buffer queue, which modifies the buffers used to queue the writes so that SSL_write can be given larger chunks, thereby increasing the 'goodput'. If we pass in a retained duplicate as we have been doing until now, then later clients will be passed junk, as SSL will have modified cached entry buffers. This patch introduces a copying ByteBufPair encoder, which is only used with SSL connections.
When TLS is enabled, consumer often discards received messages. The following is the consumer log when a message is discarded.
At this time, an IndexOutOfBoundsException has been thrown at this line.
If receiving the discarded messages again by resetting the cursor, this error may not occur.
When TLS is disabled, this phenomenon does not seem to occur.
System configuration
Pulsar version: 2.1.0-incubating
The text was updated successfully, but these errors were encountered: