pulsar icon indicating copy to clipboard operation
pulsar copied to clipboard

[Bug] consumers stops receiving new messages due to invalid blockedConsumerOnUnackedMsgs state

Open 180254 opened this issue 1 year ago • 22 comments

Search before asking

  • [X] I searched in the 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

pulsar server: docker image apachepulsar/pulsar:3.0.4 + helm chart pulsar-helm-chart pulsar client: java client org.apache.pulsar:pulsar-client:3.0.4

Minimal reproduce step

After updating Apache Pulsar, we noticed that one of the consumers sometimes stops receiving new messages for some topics. The last fully working version for us is 3.0.1. I have tested all later versions released so far and also built a branch-3.0.

I looked through the commits and determined when our service stops working:

  • Last commit where our service works properly: https://github.com/apache/pulsar/commit/80a8f8d307ac2c023147410e31d567cfce8f17c5
  • Commit which breaks, our service no longer works properly: https://github.com/apache/pulsar/commit/6e5920879996de4f43dccb4809a910b227f931f5

I performed a test using the last commit from branch 3.0 (https://github.com/apache/pulsar/commit/fd823f6cad40ed5a719cec6476563650a24c6986) and reverting the individualAckNormal method to the last version before the "commit which breaks." The change looks as follows: https://github.com/180254/pulsar-issue-22657/commit/6dac4bf7200f332d2a7f7bf9cbfcf43f811f322e. I have no problem with the modified code.

I found nothing in the logs that would inform me about the consumer suspension, etc. There are no unusual logs at all. Restarting the Kubernetes pod with consumers has helped for some time.

What did you expect to see?

consumer retrieves all messages

What did you see instead?

consumers stops receiving new messages for some topics

Anything else?

The configuration we use:

  • broker configuration:
broker:
    managedLedgerDefaultEnsembleSize: "3"
    managedLedgerDefaultWriteQuorum: "3"
    managedLedgerDefaultAckQuorum: "2"
    brokerDeduplicationEnabled: "true"
    bookkeeperClientTimeoutInSeconds: "5"
    bookkeeperClientHealthCheckErrorThresholdPerInterval: "3"
    bookkeeperClientHealthCheckQuarantineTimeInSeconds: "600"
  • namespace configuration:
bin/pulsar-admin --admin-url "${ADMIN_URL}" namespaces create "${TENANT}/service"
bin/pulsar-admin --admin-url "${ADMIN_URL}" namespaces set-max-unacked-messages-per-consumer -c 10 "${TENANT}/service"
bin/pulsar-admin --admin-url "${ADMIN_URL}" namespaces set-max-unacked-messages-per-subscription -c 20 "${TENANT}/service"
  • the topic is persistent:
persistent://public/service/service12_some_topic_someotherpart
  • we use PatternMultiTopicsConsumerImpl
   return pulsarClient
        .newConsumer(Schema.STRING)
        .subscriptionName(pulsarServerBasename)
        .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
        .subscriptionType(SubscriptionType.Shared)
        .topicsPattern(Pattern.compile("persistent://public/service/service12_.+"))
        .negativeAckRedeliveryDelay(1000, TimeUnit.MILLISECONDS)
        .patternAutoDiscoveryPeriod(60, TimeUnit.SECONDS)
        .receiverQueueSize(1)

We can reproduce it on our service. Test scenario: serviced approximately 20 customers (== 20 topics), each with about 20 messages per second. 1 message is processed in approximately 200ms. The problem occurs for a certain number of topics in the test, not for all

When a problem occurs:

  • pulsar_subscription_back_log metric shows that the backlog is growing
  • pulsar_subscription_unacked_messages metric shows 0
  • pulsar_subscription_blocked_on_unacked_messages metric shows 0
  • in the service metrics (consume & process pod), I do not see that any messages for the broken topic are being processed

Are you willing to submit a PR?

  • [ ] I'm willing to submit a PR!

180254 avatar May 06 '24 13:05 180254

Thanks for the great issue report @180254.

@poorbarcode or @Technoboy- do you have a chance to take a look at this issue report?

lhotari avatar May 08 '24 04:05 lhotari

@180254 in your case, can you detect the issue from topic stats? for example, does it tell "blockedSubscriptionOnUnackedMsgs": true?

lhotari avatar May 08 '24 20:05 lhotari

  • pulsar_subscription_blocked_on_unacked_messages metric shows 0

sorry, noticed this now. I guess topics stats wouldn't have "blockedSubscriptionOnUnackedMsgs": true either?

lhotari avatar May 08 '24 20:05 lhotari

Here are the statistics for a "broken topic", collected them after a test.

{
  "msgRateIn" : 0.0,
  "msgThroughputIn" : 0.0,
  "msgRateOut" : 0.0,
  "msgThroughputOut" : 0.0,
  "bytesInCounter" : 943097,
  "msgInCounter" : 6052,
  "bytesOutCounter" : 15027,
  "msgOutCounter" : 109,
  "averageMsgSize" : 0.0,
  "msgChunkPublished" : false,
  "storageSize" : 943097,
  "backlogSize" : 934725,
  "publishRateLimitedTimes" : 0,
  "earliestMsgPublishTimeInBacklogs" : 0,
  "offloadedStorageSize" : 0,
  "lastOffloadLedgerId" : 0,
  "lastOffloadSuccessTimeStamp" : 0,
  "lastOffloadFailureTimeStamp" : 0,
  "ongoingTxnCount" : 0,
  "abortedTxnCount" : 0,
  "committedTxnCount" : 0,
  "publishers" : [ {
    "accessMode" : "Shared",
    "msgRateIn" : 0.0,
    "msgThroughputIn" : 0.0,
    "averageMsgSize" : 0.0,
    "chunkedMessageRate" : 0.0,
    "producerId" : 9,
    "supportsPartialProducer" : false,
    "metadata" : { },
    "address" : "/10.240.0.18:42256",
    "producerName" : "pulsar-3-11",
    "connectedSince" : "2024-05-09T07:11:32.288324634Z",
    "clientVersion" : "Pulsar-Java-v3.0.4"
  }, {
    "accessMode" : "Shared",
    "msgRateIn" : 0.0,
    "msgThroughputIn" : 0.0,
    "averageMsgSize" : 0.0,
    "chunkedMessageRate" : 0.0,
    "producerId" : 14,
    "supportsPartialProducer" : false,
    "metadata" : { },
    "address" : "/10.240.0.131:44020",
    "producerName" : "pulsar-3-13",
    "connectedSince" : "2024-05-09T07:11:32.28602009Z",
    "clientVersion" : "Pulsar-Java-v3.0.4"
  }, {
    "accessMode" : "Shared",
    "msgRateIn" : 0.0,
    "msgThroughputIn" : 0.0,
    "averageMsgSize" : 0.0,
    "chunkedMessageRate" : 0.0,
    "producerId" : 21,
    "supportsPartialProducer" : false,
    "metadata" : { },
    "address" : "/10.240.0.18:37600",
    "producerName" : "pulsar-3-17",
    "connectedSince" : "2024-05-09T07:11:32.289208551Z",
    "clientVersion" : "Pulsar-Java-v3.0.4"
  }, {
    "accessMode" : "Shared",
    "msgRateIn" : 0.0,
    "msgThroughputIn" : 0.0,
    "averageMsgSize" : 0.0,
    "chunkedMessageRate" : 0.0,
    "producerId" : 20,
    "supportsPartialProducer" : false,
    "metadata" : { },
    "address" : "/10.240.0.152:43668",
    "producerName" : "pulsar-3-22",
    "connectedSince" : "2024-05-09T07:11:32.315173653Z",
    "clientVersion" : "Pulsar-Java-v3.0.4"
  } ],
  "waitingPublishers" : 0,
  "subscriptions" : {
    "pulsartestad10" : {
      "msgRateOut" : 0.0,
      "msgThroughputOut" : 0.0,
      "bytesOutCounter" : 15027,
      "msgOutCounter" : 109,
      "msgRateRedeliver" : 0.0,
      "messageAckRate" : 0.0,
      "chunkedMessageRate" : 0,
      "msgBacklog" : 5777,
      "backlogSize" : 934725,
      "earliestMsgPublishTimeInBacklog" : 0,
      "msgBacklogNoDelayed" : 5777,
      "blockedSubscriptionOnUnackedMsgs" : false,
      "msgDelayed" : 0,
      "unackedMessages" : 0,
      "type" : "Shared",
      "msgRateExpired" : 0.0,
      "totalMsgExpired" : 0,
      "lastExpireTimestamp" : 0,
      "lastConsumedFlowTimestamp" : 1715238695970,
      "lastConsumedTimestamp" : 1715238695970,
      "lastAckedTimestamp" : 1715238696259,
      "lastMarkDeleteAdvancedTimestamp" : 1715238696259,
      "consumers" : [ {
        "msgRateOut" : 0.0,
        "msgThroughputOut" : 0.0,
        "bytesOutCounter" : 2346,
        "msgOutCounter" : 19,
        "msgRateRedeliver" : 0.0,
        "messageAckRate" : 0.0,
        "chunkedMessageRate" : 0.0,
        "consumerName" : "9a385",
        "availablePermits" : -17,
        "unackedMessages" : 0,
        "avgMessagesPerEntry" : 3,
        "blockedConsumerOnUnackedMsgs" : true,
        "lastAckedTimestamp" : 1715238694438,
        "lastConsumedTimestamp" : 1715238694116,
        "lastConsumedFlowTimestamp" : 1715238694124,
        "metadata" : { },
        "address" : "/10.240.0.131:44036",
        "connectedSince" : "2024-05-09T07:11:32.290486476Z",
        "clientVersion" : "Pulsar-Java-v3.0.4",
        "lastAckedTime" : "2024-05-09T07:11:34.438Z",
        "lastConsumedTime" : "2024-05-09T07:11:34.116Z"
      }, {
        "msgRateOut" : 0.0,
        "msgThroughputOut" : 0.0,
        "bytesOutCounter" : 1357,
        "msgOutCounter" : 11,
        "msgRateRedeliver" : 0.0,
        "messageAckRate" : 0.0,
        "chunkedMessageRate" : 0.0,
        "consumerName" : "58a5f",
        "availablePermits" : -10,
        "unackedMessages" : 0,
        "avgMessagesPerEntry" : 11,
        "blockedConsumerOnUnackedMsgs" : true,
        "lastAckedTimestamp" : 1715238693873,
        "lastConsumedTimestamp" : 1715238692374,
        "lastConsumedFlowTimestamp" : 1715238692409,
        "metadata" : { },
        "address" : "/10.240.0.131:44042",
        "connectedSince" : "2024-05-09T07:11:32.291014886Z",
        "clientVersion" : "Pulsar-Java-v3.0.4",
        "lastAckedTime" : "2024-05-09T07:11:33.873Z",
        "lastConsumedTime" : "2024-05-09T07:11:32.374Z"
      }, {
        "msgRateOut" : 0.0,
        "msgThroughputOut" : 0.0,
        "bytesOutCounter" : 1476,
        "msgOutCounter" : 12,
        "msgRateRedeliver" : 0.0,
        "messageAckRate" : 0.0,
        "chunkedMessageRate" : 0.0,
        "consumerName" : "c4fba",
        "availablePermits" : -11,
        "unackedMessages" : 0,
        "avgMessagesPerEntry" : 12,
        "blockedConsumerOnUnackedMsgs" : true,
        "lastAckedTimestamp" : 1715238694112,
        "lastConsumedTimestamp" : 1715238692413,
        "lastConsumedFlowTimestamp" : 1715238692543,
        "metadata" : { },
        "address" : "/10.240.0.131:44008",
        "connectedSince" : "2024-05-09T07:11:32.291343692Z",
        "clientVersion" : "Pulsar-Java-v3.0.4",
        "lastAckedTime" : "2024-05-09T07:11:34.112Z",
        "lastConsumedTime" : "2024-05-09T07:11:32.413Z"
      }, {
        "msgRateOut" : 0.0,
        "msgThroughputOut" : 0.0,
        "bytesOutCounter" : 9848,
        "msgOutCounter" : 67,
        "msgRateRedeliver" : 0.0,
        "messageAckRate" : 0.0,
        "chunkedMessageRate" : 0.0,
        "consumerName" : "cff0d",
        "availablePermits" : -5,
        "unackedMessages" : 0,
        "avgMessagesPerEntry" : 2,
        "blockedConsumerOnUnackedMsgs" : true,
        "lastAckedTimestamp" : 1715238696259,
        "lastConsumedTimestamp" : 1715238695970,
        "lastConsumedFlowTimestamp" : 1715238695972,
        "metadata" : { },
        "address" : "/10.240.0.18:42270",
        "connectedSince" : "2024-05-09T07:11:32.291622498Z",
        "clientVersion" : "Pulsar-Java-v3.0.4",
        "lastAckedTime" : "2024-05-09T07:11:36.259Z",
        "lastConsumedTime" : "2024-05-09T07:11:35.97Z"
      } ],
      "isDurable" : true,
      "isReplicated" : false,
      "allowOutOfOrderDelivery" : false,
      "consumersAfterMarkDeletePosition" : { },
      "nonContiguousDeletedMessagesRanges" : 0,
      "nonContiguousDeletedMessagesRangesSerializedSize" : 0,
      "delayedMessageIndexSizeInBytes" : 0,
      "subscriptionProperties" : { },
      "filterProcessedMsgCount" : 0,
      "filterAcceptedMsgCount" : 0,
      "filterRejectedMsgCount" : 0,
      "filterRescheduledMsgCount" : 0,
      "durable" : true,
      "replicated" : false
    }
  },
  "replication" : { },
  "deduplicationStatus" : "Enabled",
  "nonContiguousDeletedMessagesRanges" : 0,
  "nonContiguousDeletedMessagesRangesSerializedSize" : 0,
  "delayedMessageIndexSizeInBytes" : 0,
  "compaction" : {
    "lastCompactionRemovedEventCount" : 0,
    "lastCompactionSucceedTimestamp" : 0,
    "lastCompactionFailedTimestamp" : 0,
    "lastCompactionDurationTimeInMills" : 0
  },
  "ownerBroker" : "pulsar-broker-2.pulsar-broker.default.svc.cluster.local:8080"
}

There are 4 consumers, each of them is "unackedMessages" : 0 and at the same time "blockedConsumerOnUnackedMsgs" : true.

AdrianPedziwiatr-TomTom avatar May 09 '24 08:05 AdrianPedziwiatr-TomTom

There are 4 consumers, each of them is "unackedMessages" : 0 and at the same time "blockedConsumerOnUnackedMsgs" : true.

@AdrianPedziwiatr-TomTom thanks for sharing. This is an interesting detail.

lhotari avatar May 09 '24 08:05 lhotari

A little progress in reproducing the problem in the unit test:

https://github.com/180254/pulsar-issue-22657/commit/5822ab6236fc666b0412d15f4a60e842ae5b93e3 (test22657_1_parameterized)

v3.0.1: fails when [maxUnackedMsgPerConsumer=1,maxUnackedMsgPerSubscription=1], other variants ok v3.0.2: fails when [maxUnackedMsgPerConsumer=1,maxUnackedMsgPerSubscription=any value], other variants ok branch-3.0 (b178084c5a66034a3044a95fe756e16d50558c4b): fails when [maxUnackedMsgPerConsumer=1,maxUnackedMsgPerSubscription=any value], other variants ok

180254 avatar May 11 '24 20:05 180254

I reproduced the problem also for larger values of maxUnackedMsgPerConsumer. The new test well represents the issue we are struggling with.

Please see: 180254@5822ab6 (test22657_3, test22657_3_moreconsumers)

Test results:

  • v3.0.1: fails when [maxUnackedMsgPerConsumer=1,maxUnackedMsgPerSubscription=1], other variants ok
  • v3.0.2: fails in all cases
  • branch-3.0 (2da571e): fails in all cases

Some log from the failed case:

2024-05-14T17:06:13,335 - INFO  - [awaitility-thread:BrokerServiceTest] - ----
2024-05-14T17:06:13,336 - INFO  - [awaitility-thread:BrokerServiceTest] - subscriptionStats: SubscriptionStatsImpl(msgRateOut=0.0, msgThroughputOut=0.0, bytesOutCounter=744, msgOutCounter=40, msgRateRedeliver=0.0, messageAckRate=0.0, chunkedMessageRate=0, msgBacklog=0, backlogSize=-1, earliestMsgPublishTimeInBacklog=0, msgBacklogNoDelayed=0, blockedSubscriptionOnUnackedMsgs=false, msgDelayed=0, unackedMessages=0, type=Shared, activeConsumerName=null, msgRateExpired=0.0, totalMsgExpired=0, lastExpireTimestamp=0, lastConsumedFlowTimestamp=1715699172372, lastConsumedTimestamp=1715699172383, lastAckedTimestamp=1715699172386, lastMarkDeleteAdvancedTimestamp=1715699172386, consumers=[ConsumerStatsImpl(msgRateOut=0.0, msgThroughputOut=0.0, bytesOutCounter=744, msgOutCounter=40, msgRateRedeliver=0.0, messageAckRate=0.0, chunkedMessageRate=0.0, consumerName=2b07e, availablePermits=-39, unackedMessages=0, avgMessagesPerEntry=40, blockedConsumerOnUnackedMsgs=true, readPositionWhenJoining=null, addressOffset=0, addressLength=16, connectedSinceOffset=34, connectedSinceLength=35, clientVersionOffset=16, clientVersionLength=18, lastAckedTimestamp=1715699172386, lastConsumedTimestamp=1715699172383, lastConsumedFlowTimestamp=1715699172386, keyHashRanges=null, metadata={}, stringBuffer=/127.0.0.1:45914Pulsar-Java-v3.0.52024-05-14T17:06:12.371810907+02:00)], isDurable=true, isReplicated=false, allowOutOfOrderDelivery=false, keySharedMode=null, consumersAfterMarkDeletePosition={}, nonContiguousDeletedMessagesRanges=0, nonContiguousDeletedMessagesRangesSerializedSize=0, delayedMessageIndexSizeInBytes=0, bucketDelayedIndexStats={}, subscriptionProperties={}, filterProcessedMsgCount=0, filterAcceptedMsgCount=0, filterRejectedMsgCount=0, filterRescheduledMsgCount=0)
2024-05-14T17:06:13,336 - INFO  - [awaitility-thread:BrokerServiceTest] - currentReceiverQueueSize: 1
2024-05-14T17:06:13,336 - INFO  - [awaitility-thread:BrokerServiceTest] - numMessagesInQueue: 0
2024-05-14T17:06:13,336 - INFO  - [awaitility-thread:BrokerServiceTest] - unackedMessagesSubscription: 0
2024-05-14T17:06:13,336 - INFO  - [awaitility-thread:BrokerServiceTest] - blockedSubscriptionOnUnackedMsgs: false
2024-05-14T17:06:13,336 - INFO  - [awaitility-thread:BrokerServiceTest] - unackedMessagesConsumer: 0
2024-05-14T17:06:13,336 - INFO  - [awaitility-thread:BrokerServiceTest] - blockedConsumerOnUnackedMsgs: true
2024-05-14T17:06:13,336 - INFO  - [awaitility-thread:BrokerServiceTest] - ----

(in summary)

  unackedMessagesSubscription: 0
  blockedSubscriptionOnUnackedMsgs: false
  unackedMessagesConsumer: 0
  blockedConsumerOnUnackedMsgs: true

At https://github.com/apache/pulsar/compare/branch-3.0...180254:pulsar-issue-22657:branch-3.0 you can find all my tests and the restored old version of the individualAckNormal method for testing/comparison.

180254 avatar May 14 '24 15:05 180254

Getting the same issue, consumer stops receiving new messages

{ "msgRateIn" : 0.0, "msgThroughputIn" : 0.0, "msgRateOut" : 0.0, "msgThroughputOut" : 0.0, "bytesInCounter" : 388755, "msgInCounter" : 136, "bytesOutCounter" : 4481887, "msgOutCounter" : 1636, "averageMsgSize" : 0.0, "msgChunkPublished" : false, "storageSize" : 391126, "backlogSize" : 391126, "publishRateLimitedTimes" : 0, "earliestMsgPublishTimeInBacklogs" : 0, "offloadedStorageSize" : 0, "lastOffloadLedgerId" : 0, "lastOffloadSuccessTimeStamp" : 0, "lastOffloadFailureTimeStamp" : 0, "ongoingTxnCount" : 0, "abortedTxnCount" : 39, "committedTxnCount" : 76, "publishers" : [ ], "waitingPublishers" : 0, "subscriptions" : { "my-subscription" : { "msgRateOut" : 0.0, "msgThroughputOut" : 0.0, "bytesOutCounter" : 4481829, "msgOutCounter" : 1635, "msgRateRedeliver" : 0.0, "messageAckRate" : 0.0, "chunkedMessageRate" : 0, "msgBacklog" : 54, "backlogSize" : 359284, "earliestMsgPublishTimeInBacklog" : 0, "msgBacklogNoDelayed" : 54, "blockedSubscriptionOnUnackedMsgs" : false, "msgDelayed" : 0, "unackedMessages" : 56, "type" : "Shared", "msgRateExpired" : 0.0, "totalMsgExpired" : 0, "lastExpireTimestamp" : 0, "lastConsumedFlowTimestamp" : 1716211257096, "lastConsumedTimestamp" : 1716211552131, "lastAckedTimestamp" : 0, "lastMarkDeleteAdvancedTimestamp" : 1716204988009, "consumers" : [ { "msgRateOut" : 0.0, "msgThroughputOut" : 0.0, "bytesOutCounter" : 340432, "msgOutCounter" : 125, "msgRateRedeliver" : 0.0, "messageAckRate" : 0.0, "chunkedMessageRate" : 0.0, "consumerName" : "8c286d32f5", "availablePermits" : 875, "unackedMessages" : 56, "avgMessagesPerEntry" : 1, "blockedConsumerOnUnackedMsgs" : false, "address" : "/10.20.1.13:35118", "connectedSince" : "2024-05-20T13:20:57.088603363Z", "clientVersion" : "Pulsar-CPP-v3.5.1", "lastAckedTimestamp" : 0, "lastConsumedTimestamp" : 1716211552131, "lastConsumedFlowTimestamp" : 1716211257096, "metadata" : { }, "lastAckedTime" : "1970-01-01T00:00:00Z", "lastConsumedTime" : "2024-05-20T13:25:52.131Z" } ], "isDurable" : true, "isReplicated" : false, "allowOutOfOrderDelivery" : false, "consumersAfterMarkDeletePosition" : { }, "nonContiguousDeletedMessagesRanges" : 49, "nonContiguousDeletedMessagesRangesSerializedSize" : 731, "delayedMessageIndexSizeInBytes" : 0, "subscriptionProperties" : { }, "filterProcessedMsgCount" : 0, "filterAcceptedMsgCount" : 0, "filterRejectedMsgCount" : 0, "filterRescheduledMsgCount" : 0, "durable" : true, "replicated" : false }, "cmi-cdr-webhooks-subscription" : { "msgRateOut" : 0.0, "msgThroughputOut" : 0.0, "bytesOutCounter" : 0, "msgOutCounter" : 0, "msgRateRedeliver" : 0.0, "messageAckRate" : 0.0, "chunkedMessageRate" : 0, "msgBacklog" : 221, "backlogSize" : 391126, "earliestMsgPublishTimeInBacklog" : 0, "msgBacklogNoDelayed" : 221, "blockedSubscriptionOnUnackedMsgs" : false, "msgDelayed" : 0, "unackedMessages" : 0, "type" : "Shared", "msgRateExpired" : 0.0, "totalMsgExpired" : 0, "lastExpireTimestamp" : 0, "lastConsumedFlowTimestamp" : 1716115069064, "lastConsumedTimestamp" : 0, "lastAckedTimestamp" : 0, "lastMarkDeleteAdvancedTimestamp" : 0, "consumers" : [ ], "isDurable" : true, "isReplicated" : false, "allowOutOfOrderDelivery" : false, "consumersAfterMarkDeletePosition" : { }, "nonContiguousDeletedMessagesRanges" : 0, "nonContiguousDeletedMessagesRangesSerializedSize" : 0, "delayedMessageIndexSizeInBytes" : 0, "subscriptionProperties" : { }, "filterProcessedMsgCount" : 0, "filterAcceptedMsgCount" : 0, "filterRejectedMsgCount" : 0, "filterRescheduledMsgCount" : 0, "durable" : true, "replicated" : false } }, "replication" : { }, "deduplicationStatus" : "Disabled", "nonContiguousDeletedMessagesRanges" : 49, "nonContiguousDeletedMessagesRangesSerializedSize" : 731, "delayedMessageIndexSizeInBytes" : 0, "compaction" : { "lastCompactionRemovedEventCount" : 0, "lastCompactionSucceedTimestamp" : 0, "lastCompactionFailedTimestamp" : 0, "lastCompactionDurationTimeInMills" : 0 }, "ownerBroker" : "10.0.0.1:8080" }

pulsar version 3.2.1

prasathsekar avatar May 20 '24 13:05 prasathsekar

pulsar version 3.2.1

@prasathsekar You might be facing another bug that is already fixed in 3.2.3 with #22454. Please upgrade to Pulsar 3.2.3 and then comment whether the problem is resolved.

lhotari avatar May 21 '24 05:05 lhotari

Hi @Technoboy-, @poorbarcode Will you have a chance to look at this? You might have the biggest knowledge, because the commit that changed this was yours.

@MichalKoziorowski-TomTom please confirm whether this problem reproduces on 3.2.3 or 3.0.5 .

lhotari avatar Jun 05 '24 08:06 lhotari

Before submitting a ticket, we also checked 3.2.x. The problem with our service also occurred there.

I reran the proposed BrokerServiceTest.java tests that I shared in previous messages. branch-3.0, commit 46b5419224c1a821ec0b9a839d0d7da57ac5f181: doesn't work, results as reported previously master, commit 342d88dd193bb85c0af91c5193b1422808a9c821: doesn't work, results as reported previously

180254 avatar Jun 05 '24 09:06 180254

I checked the code, my first vision is maybe it could have race conditions here. But I didn't dive deeper.

dao-jun avatar Jun 05 '24 10:06 dao-jun

@180254 I experimented with some changes in https://github.com/lhotari/pulsar/pull/192 , I added test cases based on your work. There are multiple inconsistencies in handling the unacked message counts and blocking/unblocking dispatchers. The main gap in the experiment is the handling for negative acknowledgements. The changes I made fixed the test cases, but I didn't run other tests to verify that there aren't regressions caused by the change. Most likely there are because of invalid negative ack handling in the experiment.

lhotari avatar Jun 05 '24 16:06 lhotari

Hi.

Is there any chance someone will look at this race condition? We're trying to figure out some workaround to not see this problem.

Hi.

Is there any chance someone will look at this race condition? We're trying to figure out some workaround to not see this problem.

I'm doing this now. I'm sorry for the long delay.

lhotari avatar Aug 30 '24 06:08 lhotari

I remember seeing that this is a regression caused by #20990.

lhotari avatar Aug 30 '24 06:08 lhotari

It looks like #23072 makes improvements to the part where the regression happened.

lhotari avatar Aug 30 '24 10:08 lhotari

There's #21126 before #23072.

lhotari avatar Aug 30 '24 10:08 lhotari

Rebased the test cases added by @180254 here: https://github.com/lhotari/pulsar/commit/ff0c8a50fb6bf2afdf0f73faa0be090e94efe964 . It looks like the problem persists after #21126 and #23072 .

lhotari avatar Aug 30 '24 11:08 lhotari

Great job by @180254 in doing the troubleshooting, thank you!

lhotari avatar Aug 30 '24 11:08 lhotari

@180254 I experimented with some changes in https://github.com/lhotari/pulsar/pull/192 , I added test cases based on your work. There are multiple inconsistencies in handling the unacked message counts and blocking/unblocking dispatchers. The main gap in the experiment is the handling for negative acknowledgements. The changes I made fixed the test cases, but I didn't run other tests to verify that there aren't regressions caused by the change. Most likely there are because of invalid negative ack handling in the experiment.

Resuming this work would be needed.

lhotari avatar Sep 19 '24 09:09 lhotari

@lhotari I found this issue was introduced by https://github.com/apache/pulsar/pull/20990. I add a pr to fix this, update blockedConsumerOnUnackedMsgs's value after change the unackedMessages.

summeriiii avatar Dec 31 '24 06:12 summeriiii