[Bug] consumers stops receiving new messages due to invalid blockedConsumerOnUnackedMsgs state
180254 opened this issue · comments
Search before asking
- I searched in the issues and found nothing similar.
Read release policy
- 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: 80a8f8d
- Commit which breaks, our service no longer works properly: 6e59208
I performed a test using the last commit from branch 3.0 (fd823f6) and reverting the individualAckNormal method to the last version before the "commit which breaks." The change looks as follows: 180254@6dac4bf. 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!
Thanks for the great issue report @180254.
@poorbarcode or @Technoboy- do you have a chance to take a look at this issue report?
@180254 in your case, can you detect the issue from topic stats? for example, does it tell "blockedSubscriptionOnUnackedMsgs": true
?
- pulsar_subscription_blocked_on_unacked_messages metric shows 0
sorry, noticed this now. I guess topics stats wouldn't have "blockedSubscriptionOnUnackedMsgs": true
either?
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
.
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.
A little progress in reproducing the problem in the unit test:
180254@5822ab6 (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 (b178084): fails when [maxUnackedMsgPerConsumer=1
,maxUnackedMsgPerSubscription=any value
], other variants ok
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 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.