soumya-ghosh opened a new issue, #23920:
URL: https://github.com/apache/pulsar/issues/23920

   ### Search before asking
   
   - [x] I searched in the [issues](https://github.com/apache/pulsar/issues) 
and found nothing similar.
   
   
   ### Read release policy
   
   - [x] I understand that unsupported versions don't get bug fixes. I will 
attempt to reproduce the issue on a supported version of Pulsar client and 
Pulsar broker.
   
   
   ### Version
   
   Setup summary:
    * Pulsar 4.0.1, 3 brokers, 5 bookies, 3 ZKs, JVM - 17.0.13
    * Produce and Consume through Pulsar Java client (4.0.1) - Clients are 
using Java 11
   
   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
    * Extensible load balnacer and TransferShedder 
    * Bundle split algorithm - `flow_or_qps_equally_divide`
    * Tiered storage to S3
   
   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.
   
   ```java
   // Pulsar client
   this.pulsarClient =
             PulsarClient.builder()
                 .serviceUrl(connectionUrl)
                 .memoryLimit(0, SizeUnit.MEGA_BYTES)
                 .build();
   
   // Producer configurations
   ProducerBuilder<T> builder =
           pulsarClient
               .newProducer(schema)
               .producerName(producerName)
               .topic(topicName)
               .compressionType(CompressionType.LZ4)
               .enableBatching(true)
               .batchingMaxPublishDelay(30, TimeUnit.MILLISECONDS)
               .blockIfQueueFull(true);
   ```
   
   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.
   
   
![Image](https://github.com/user-attachments/assets/c8f6a1d7-ebe0-4d8b-849a-319f1fb4ea60)
   
   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)
   
   ```
   2025-02-02T16:49:45,932+0000 [BookKeeperClientWorker-OrderedExecutor-4-0] 
DEBUG org.apache.pulsar.broker.service.AbstractTopic - 
[persistent://public/default/default_shivam_stream-partition-8] 
Producer352f00f6-a898-4ed4-9469-8eb62c2ed0bf Got request to create producer 
   2025-02-02T16:49:45,932+0000 [BookKeeperClientWorker-OrderedExecutor-4-0] 
DEBUG org.apache.pulsar.broker.service.AbstractTopic - 
[persistent://public/default/default_shivam_stream-partition-8] 
[Producer352f00f6-a898-4ed4-9469-8eb62c2ed0bf] Added producer -- count: 1
   2025-02-02T16:49:45,932+0000 [BookKeeperClientWorker-OrderedExecutor-4-0] 
DEBUG org.apache.pulsar.broker.service.schema.SchemaRegistryServiceImpl - 
[public/default/default_shivam_stream] Put schema finished
   2025-02-02T16:49:45,933+0000 [broker-topic-workers-OrderedExecutor-5-0] INFO 
 org.apache.pulsar.broker.service.ServerCnx - [/10.22.2.15:39103] Created new 
producer: 
Producer{topic=PersistentTopic{topic=persistent://public/default/default_shivam_stream-partition-8},
 client=[id: 0x8eacecff, L:/10.10.17.40:6650 - R:/10.22.2.15:39103] 
[SR:10.22.2.15, state:Connected], 
producerName=Producer352f00f6-a898-4ed4-9469-8eb62c2ed0bf, producerId=8}, role: 
null
   2025-02-02T16:49:45,933+0000 [BookKeeperClientWorker-OrderedExecutor-4-0] 
DEBUG org.apache.pulsar.broker.service.schema.SchemaRegistryServiceImpl - 
[public/default/default_shivam_stream] Schema is already exists
   2025-02-02T16:49:45,933+0000 [BookKeeperClientWorker-OrderedExecutor-4-0] 
DEBUG org.apache.pulsar.broker.service.AbstractTopic - 
[persistent://public/default/default_shivam_stream-partition-9] 
Producer352f00f6-a898-4ed4-9469-8eb62c2ed0bf Got request to create producer 
   2025-02-02T16:49:45,933+0000 [BookKeeperClientWorker-OrderedExecutor-4-0] 
DEBUG org.apache.pulsar.broker.service.AbstractTopic - 
[persistent://public/default/default_shivam_stream-partition-9] 
[Producer352f00f6-a898-4ed4-9469-8eb62c2ed0bf] Added producer -- count: 1
   2025-02-02T16:49:45,933+0000 [broker-topic-workers-OrderedExecutor-5-0] INFO 
 org.apache.pulsar.broker.service.ServerCnx - [/10.22.2.15:39103] Created new 
producer: 
Producer{topic=PersistentTopic{topic=persistent://public/default/default_shivam_stream-partition-9},
 client=[id: 0x8eacecff, L:/10.10.17.40:6650 - R:/10.22.2.15:39103] 
[SR:10.22.2.15, state:Connected], 
producerName=Producer352f00f6-a898-4ed4-9469-8eb62c2ed0bf, producerId=9}, role: 
null
   2025-02-02T16:49:45,933+0000 [BookKeeperClientWorker-OrderedExecutor-4-0] 
DEBUG org.apache.pulsar.broker.service.schema.SchemaRegistryServiceImpl - 
[public/default/default_shivam_stream] Put schema finished
   2025-02-02T16:49:46,175+0000 [pulsar-io-3-12] DEBUG 
org.apache.pulsar.broker.service.ServerCnx - [/10.22.2.15:39103] Received send 
message request. producer: 9:0 Producer352f00f6-a898-4ed4-9469-8eb62c2ed0bf:0 
size: 133, partition key is: 1, ordering key is null, uncompressedSize is 45
   2025-02-02T16:49:46,175+0000 [pulsar-io-3-12] DEBUG 
org.apache.pulsar.broker.service.ServerCnxThrottleTracker - [[id: 0x8eacecff, 
L:/10.10.17.40:6650 - R:/10.22.2.15:39103] [SR:10.22.2.15, state:Connected]] 
Setting auto read to false
   2025-02-02T16:49:46,180+0000 [BookKeeperClientWorker-OrderedExecutor-4-0] 
DEBUG org.apache.pulsar.broker.service.Producer - 
[PersistentTopic{topic=persistent://public/default/default_shivam_stream-partition-9}]
 [Producer352f00f6-a898-4ed4-9469-8eb62c2ed0bf] [9] triggered send callback. 
cnx /10.22.2.15:39103, sequenceId 0
   2025-02-02T16:49:46,180+0000 [pulsar-io-3-12] DEBUG 
org.apache.pulsar.broker.service.Producer - 
[PersistentTopic{topic=persistent://public/default/default_shivam_stream-partition-9}]
 [Producer352f00f6-a898-4ed4-9469-8eb62c2ed0bf] [9] Persisted message. cnx [id: 
0x8eacecff, L:/10.10.17.40:6650 - R:/10.22.2.15:39103] [SR:10.22.2.15, 
state:Connected], sequenceId 0
   2025-02-02T16:49:47,082+0000 [pulsar-io-3-13] DEBUG 
org.apache.pulsar.broker.service.ServerCnx - New connection from 
/127.0.0.1:57030
   2025-02-02T16:49:47,083+0000 [pulsar-io-3-13] INFO  
org.apache.pulsar.broker.service.ServerCnx - Closed connection from 
/127.0.0.1:57030
   2025-02-02T16:49:51,001+0000 
[pulsar-backlog-quota-checker-OrderedScheduler-0-0] DEBUG 
org.apache.pulsar.broker.service.persistent.PersistentTopic - 
[persistent://public/default/default_shivam_stream-partition-6] No backlog. 
Update old position info is null
   2025-02-02T16:49:51,001+0000 
[pulsar-backlog-quota-checker-OrderedScheduler-0-0] DEBUG 
org.apache.pulsar.broker.service.persistent.PersistentTopic - 
[persistent://public/default/default_shivam_stream-partition-6] Time backlog 
quota = [-1]. Checking if exceeded.
   2025-02-02T16:49:51,001+0000 
[pulsar-backlog-quota-checker-OrderedScheduler-0-0] DEBUG 
org.apache.pulsar.broker.service.BrokerService - quota not exceeded for 
[persistent://public/default/default_shivam_stream-partition-6]
   2025-02-02T16:49:51,001+0000 
[pulsar-backlog-quota-checker-OrderedScheduler-0-0] DEBUG 
org.apache.pulsar.broker.service.persistent.PersistentTopic - 
[persistent://public/default/default_shivam_stream-partition-7] No backlog. 
Update old position info is null
   2025-02-02T16:49:51,001+0000 
[pulsar-backlog-quota-checker-OrderedScheduler-0-0] DEBUG 
org.apache.pulsar.broker.service.persistent.PersistentTopic - 
[persistent://public/default/default_shivam_stream-partition-7] Time backlog 
quota = [-1]. Checking if exceeded.
   2025-02-02T16:49:51,001+0000 
[pulsar-backlog-quota-checker-OrderedScheduler-0-0] DEBUG 
org.apache.pulsar.broker.service.BrokerService - quota not exceeded for 
[persistent://public/default/default_shivam_stream-partition-7]
   --
   2025-02-02T16:52:39,369+0000 [BookKeeperClientWorker-OrderedExecutor-1-0] 
DEBUG org.apache.pulsar.broker.service.AbstractTopic - 
[persistent://public/default/default_shivam_stream-partition-7] 
Producer3e526aac-10f4-4a5e-8c51-2dbda8545bbf Got request to create producer 
   2025-02-02T16:52:39,369+0000 [BookKeeperClientWorker-OrderedExecutor-1-0] 
DEBUG org.apache.pulsar.broker.service.AbstractTopic - 
[persistent://public/default/default_shivam_stream-partition-7] 
[Producer3e526aac-10f4-4a5e-8c51-2dbda8545bbf] Added producer -- count: 1
   2025-02-02T16:52:39,369+0000 [BookKeeperClientWorker-OrderedExecutor-1-0] 
DEBUG org.apache.pulsar.broker.service.schema.SchemaRegistryServiceImpl - 
[public/default/default_shivam_stream] Put schema finished
   2025-02-02T16:52:39,369+0000 [broker-topic-workers-OrderedExecutor-4-0] INFO 
 org.apache.pulsar.broker.service.ServerCnx - [/10.22.2.15:54474] Created new 
producer: 
Producer{topic=PersistentTopic{topic=persistent://public/default/default_shivam_stream-partition-7},
 client=[id: 0x83526df1, L:/10.10.17.40:6650 - R:/10.22.2.15:54474] 
[SR:10.22.2.15, state:Connected], 
producerName=Producer3e526aac-10f4-4a5e-8c51-2dbda8545bbf, producerId=7}, role: 
null
   2025-02-02T16:52:39,369+0000 [BookKeeperClientWorker-OrderedExecutor-1-0] 
DEBUG org.apache.pulsar.broker.service.schema.SchemaRegistryServiceImpl - 
[public/default/default_shivam_stream] Schema is already exists
   2025-02-02T16:52:39,369+0000 [BookKeeperClientWorker-OrderedExecutor-1-0] 
DEBUG org.apache.pulsar.broker.service.AbstractTopic - 
[persistent://public/default/default_shivam_stream-partition-4] 
Producer3e526aac-10f4-4a5e-8c51-2dbda8545bbf Got request to create producer 
   2025-02-02T16:52:39,369+0000 [BookKeeperClientWorker-OrderedExecutor-1-0] 
DEBUG org.apache.pulsar.broker.service.AbstractTopic - 
[persistent://public/default/default_shivam_stream-partition-4] 
[Producer3e526aac-10f4-4a5e-8c51-2dbda8545bbf] Added producer -- count: 1
   2025-02-02T16:52:39,369+0000 [BookKeeperClientWorker-OrderedExecutor-1-0] 
DEBUG org.apache.pulsar.broker.service.schema.SchemaRegistryServiceImpl - 
[public/default/default_shivam_stream] Put schema finished
   2025-02-02T16:52:39,369+0000 [broker-topic-workers-OrderedExecutor-3-0] INFO 
 org.apache.pulsar.broker.service.ServerCnx - [/10.22.2.15:54474] Created new 
producer: 
Producer{topic=PersistentTopic{topic=persistent://public/default/default_shivam_stream-partition-4},
 client=[id: 0x83526df1, L:/10.10.17.40:6650 - R:/10.22.2.15:54474] 
[SR:10.22.2.15, state:Connected], 
producerName=Producer3e526aac-10f4-4a5e-8c51-2dbda8545bbf, producerId=4}, role: 
null
   2025-02-02T16:52:39,596+0000 [pulsar-io-3-7] DEBUG 
org.apache.pulsar.broker.service.ServerCnx - [/10.22.2.15:54474] Received send 
message request. producer: 9:0 Producer3e526aac-10f4-4a5e-8c51-2dbda8545bbf:0 
size: 133, partition key is: 1, ordering key is null, uncompressedSize is 45
   2025-02-02T16:52:39,596+0000 [pulsar-io-3-7] DEBUG 
org.apache.pulsar.broker.service.ServerCnxThrottleTracker - [[id: 0x83526df1, 
L:/10.10.17.40:6650 - R:/10.22.2.15:54474] [SR:10.22.2.15, state:Connected]] 
Setting auto read to false
   2025-02-02T16:52:39,601+0000 [BookKeeperClientWorker-OrderedExecutor-4-0] 
DEBUG org.apache.pulsar.broker.service.Producer - 
[PersistentTopic{topic=persistent://public/default/default_shivam_stream-partition-9}]
 [Producer3e526aac-10f4-4a5e-8c51-2dbda8545bbf] [9] triggered send callback. 
cnx /10.22.2.15:54474, sequenceId 0
   2025-02-02T16:52:39,601+0000 [pulsar-io-3-7] DEBUG 
org.apache.pulsar.broker.service.Producer - 
[PersistentTopic{topic=persistent://public/default/default_shivam_stream-partition-9}]
 [Producer3e526aac-10f4-4a5e-8c51-2dbda8545bbf] [9] Persisted message. cnx [id: 
0x83526df1, L:/10.10.17.40:6650 - R:/10.22.2.15:54474] [SR:10.22.2.15, 
state:Connected], sequenceId 0
   2025-02-02T16:52:51,001+0000 
[pulsar-backlog-quota-checker-OrderedScheduler-0-0] DEBUG 
org.apache.pulsar.broker.service.persistent.PersistentTopic - 
[persistent://public/default/default_shivam_stream-partition-6] No backlog. 
Update old position info is null
   2025-02-02T16:52:51,001+0000 
[pulsar-backlog-quota-checker-OrderedScheduler-0-0] DEBUG 
org.apache.pulsar.broker.service.persistent.PersistentTopic - 
[persistent://public/default/default_shivam_stream-partition-6] Time backlog 
quota = [-1]. Checking if exceeded.
   2025-02-02T16:52:51,001+0000 
[pulsar-backlog-quota-checker-OrderedScheduler-0-0] DEBUG 
org.apache.pulsar.broker.service.BrokerService - quota not exceeded for 
[persistent://public/default/default_shivam_stream-partition-6]
   2025-02-02T16:52:51,001+0000 
[pulsar-backlog-quota-checker-OrderedScheduler-0-0] DEBUG 
org.apache.pulsar.broker.service.persistent.PersistentTopic - 
[persistent://public/default/default_shivam_stream-partition-7] No backlog. 
Update old position info is null
   2025-02-02T16:52:51,001+0000 
[pulsar-backlog-quota-checker-OrderedScheduler-0-0] DEBUG 
org.apache.pulsar.broker.service.persistent.PersistentTopic - 
[persistent://public/default/default_shivam_stream-partition-7] Time backlog 
quota = [-1]. Checking if exceeded.
   2025-02-02T16:52:51,001+0000 
[pulsar-backlog-quota-checker-OrderedScheduler-0-0] DEBUG 
org.apache.pulsar.broker.service.BrokerService - quota not exceeded for 
[persistent://public/default/default_shivam_stream-partition-7]
   2025-02-02T16:52:51,001+0000 
[pulsar-backlog-quota-checker-OrderedScheduler-0-0] DEBUG 
org.apache.pulsar.broker.service.persistent.PersistentTopic - 
[persistent://public/default/default_shivam_stream-partition-8] No backlog. 
Update old position info is null
   2025-02-02T16:52:51,001+0000 
[pulsar-backlog-quota-checker-OrderedScheduler-0-0] DEBUG 
org.apache.pulsar.broker.service.persistent.PersistentTopic - 
[persistent://public/default/default_shivam_stream-partition-8] Time backlog 
quota = [-1]. Checking if exceeded.
   --
   2025-02-02T16:54:29,252+0000 [BookKeeperClientWorker-OrderedExecutor-1-0] 
DEBUG org.apache.pulsar.broker.service.AbstractTopic - 
[persistent://public/default/default_shivam_stream-partition-4] 
Producerbc584de6-13d3-46c3-9d7f-9f5c4f7619b2 Got request to create producer 
   2025-02-02T16:54:29,252+0000 [BookKeeperClientWorker-OrderedExecutor-1-0] 
DEBUG org.apache.pulsar.broker.service.AbstractTopic - 
[persistent://public/default/default_shivam_stream-partition-4] 
[Producerbc584de6-13d3-46c3-9d7f-9f5c4f7619b2] Added producer -- count: 1
   2025-02-02T16:54:29,252+0000 [BookKeeperClientWorker-OrderedExecutor-1-0] 
DEBUG org.apache.pulsar.broker.service.schema.SchemaRegistryServiceImpl - 
[public/default/default_shivam_stream] Put schema finished
   2025-02-02T16:54:29,252+0000 [broker-topic-workers-OrderedExecutor-3-0] INFO 
 org.apache.pulsar.broker.service.ServerCnx - [/10.22.2.15:55876] Created new 
producer: 
Producer{topic=PersistentTopic{topic=persistent://public/default/default_shivam_stream-partition-4},
 client=[id: 0x4b78fe29, L:/10.10.17.40:6650 - R:/10.22.2.15:55876] 
[SR:10.22.2.15, state:Connected], 
producerName=Producerbc584de6-13d3-46c3-9d7f-9f5c4f7619b2, producerId=4}, role: 
null
   2025-02-02T16:54:29,253+0000 [BookKeeperClientWorker-OrderedExecutor-1-0] 
DEBUG org.apache.pulsar.broker.service.schema.SchemaRegistryServiceImpl - 
[public/default/default_shivam_stream] Schema is already exists
   20...
   ```
   
   
   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.
   
   <img width="1678" alt="Image" 
src="https://github.com/user-attachments/assets/8adda537-9d4c-4484-873b-38baa4e07cea";
 />
   
   We checked this metric for a topic in our load environment setup where these 
publish rate limit is applied.
   
   <img width="1678" alt="Image" 
src="https://github.com/user-attachments/assets/ac18e13c-3c33-4f60-adb6-4711f91fbf6b";
 />
   
   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?
   
   - [x] I'm willing to submit a PR!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to