-
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
[Bug] Producers getting timed out intermittently #23920
Comments
This detail hints into the direction that there could be a deadlock on the broker side when the problem occurs. Would you be able to capture a thread dump with In Pulsar 4.0.1 there's a known deadlock bug that has been fixed in 4.0.2 with #23854. That applies to Key_Shared subscriptions. In this case, please share (an possibly sanitize before sharing) the topic stats Commands for creating stats and stats-internal for a partitioned topic: pulsar-admin topics partitioned-stats --per-partition persistent://public/default/picks-teams-stream > topic_stats_picks-teams-stream_$(date +%s).json
pulsar-admin topics partitioned-stats-internal persistent://public/default/picks-teams-stream > topic_stats_internal_picks-teams-stream_$(date +%s).json You might want to sanitize IP addresses in stats etc. before sharing. gh gist create topic_stats_picks-teams-stream_*.json topic_stats_picks-teams-stream_*.json After creating a gist, you can check the content and then share the link in this issue. |
Now looking back at the datadog metrics when Currently we are using Failover subscription mode. I have captured some threaddumps and a heapdump from one Pulsar broker, will try to share it. Regarding stats of the partitioned topic during the consumer not getting records issue, we have captured using Non-zero backlog metric when a console consume is started from pulsar cli. Messages are delivered after 10 minutes and backlog goes to 0. Will try to capture the partitioned level stats and internal stats when issue occurs again. We suspect both producer and consumer issues are inter-related as observed them happening at the same. |
I filed issue #23921 . A heap dump could be used to check if the issue applies. This can be done by searching |
Some observations from heap which was captured on the broker of Pulsar cluster exhibiting this issue.
We observed high negative value for tokens for the PublishRateLimiterImpl object which have non-zero value for throttledProducersCount
We will try to capture some more heap dumps once issue occurs again and confirm this above observations. Thanks @lhotari for assisting in heap dump analysis. |
Search before asking
Read release policy
Version
Setup summary:
Also tried with 4.0.2 version
Pulsar Deployment - VM based, we are running on EC2 machines.
As far as configurations are concerned, we had following settings enabled
flow_or_qps_equally_divide
We updated the load balancer configurations to use the Pulsar defaults, i.e. Modular Load balancer, ThresholdShedder and
range_equally_divide
for bundle split strategy. Reason for this change was because we observed errors in broker logs related to topic shedding.Then we also disabled tiered storage.
In all cases, the reported issue is observed.
Minimal reproduce step
This issue is continuation of Apache Pulsar's slack thread - https://apache-pulsar.slack.com/archives/C5Z4T36F7/p1738158630009249
To and fro discussion with @lhotari can be found there.
We are observing an intermittent issue where Pulsar producer client is unable to produce message to a partitioned topic, it is failing with org.apache.pulsar.client.api.PulsarClientException$TimeoutException after 30 seconds.
Pulsar brokers, bookies and client instances are running in the same AWS VPC, so chances of network outage are quite low. Brokers are configured to use advertisedAddress , advertisedListeners is not configured.
In the client application, the Pulsar client and Producer object is initialized once and then messages are produced in sporadic intervals.
There is no other producer or consumer running on the cluster and there is more than sufficient resources available to cater to produce request.
Pulsar topics are partitioned.
We also have applied publish rate and dispatch rate policies of 1 MBps on the said topic (per partition)
But the throughput of messages being produced is far less than applied quota, for example, 1-5 messages in 2-5 minutes.
We did try to remove the publish and dispatch rates on the topic and issue was resolved.
Although we could not conclusively say that issue got resolved after removing those policies.
We also applied those policies back and did not observe issue for a day and when issue occurred again, we unload the namespace bundles and assigned to different brokers.
We did this several times so unloading namespace bundles seems to quick fix for the issue.
We also enabled debug logs on client and the server side.
Observed a "Setting auto read to false" false in broker logs (see below)
We also observed that
pulsar.pulsar_publish_rate_limit_times
for the topics with this issue and found that they had non-zero values for this metric.Another observation when we encounter this issue is that no messages are delivered to subscriptions. We tried consuming data from topic using Pulsar's CLI client from earliest position but no records were delivered.
We enabled
unblockStuckSubscriptionEnabled=true
in broker.conf but still observed this issue.Currently we are removing the topicPolicies - publish rate and dispatch rate and will observe if the issue recurs.
As per our tests, these policies are working fine when there is sufficient records being produced and consumed from pulsar topics.
But when producer load is sparse, we are encountering the issue where producer is unable to produce message.
Let me know if any other details are required around this.
What did you expect to see?
Records getting produced successfully
What did you see instead?
Records not being produced or consumed at sporadic intervals.
The issue happens intermittently and gets some solved on it's own, we haven't been able to nail the scenarios which deterministically leads to this issue.
The client patterns would be sparse producer and non zero values of
pulsar.pulsar_publish_rate_limit_times
during the duration of issue.Anything else?
No response
Are you willing to submit a PR?
The text was updated successfully, but these errors were encountered: