graysonzeng opened a new issue, #21892: URL: https://github.com/apache/pulsar/issues/21892
### Search before asking - [X] I searched in the [issues](https://github.com/apache/pulsar/issues) and found nothing similar. ### Version pulsar version:3.1.1 ### Minimal reproduce step broker count: 2 bookie count: 5 broker config: managedLedgerDefaultAckQuorum: "2" managedLedgerDefaultEnsembleSize: "4" managedLedgerDefaultWriteQuorum: "3" // Open Deduplication config brokerDeduplicationEnabled: "true" Using pulsar perf the publishing rate is 200000 messages/sec and the total number of messages is 100000000. Consume it at the same time. ### What did you expect to see? Complete the production and consumption of all messages ### What did you see instead? the producer fall into the following error and be stuck because of this error until the broker is restarted. ```java 2024-01-12T16:58:45,601+0800 [pulsar-client-io-17-1] INFO org.apache.pulsar.client.impl.ProducerImpl - [persistent://pulsar/default2/alltables6-partition-4] [alltables610-persistent://pulsar/default2/alltables6-persistent://pulsar/default2/input_test5-partition-8-0] Re-Sending 1 messages to server 2024-01-12T16:58:45,601+0800 [pulsar-client-io-17-1] INFO org.apache.pulsar.client.impl.ProducerImpl - [persistent://pulsar/default2/alltables6-partition-7] [alltables610-persistent://pulsar/default2/alltables6-persistent://pulsar/default2/input_test5-partition-6-0] Re-Sending 4789 messages to server 2024-01-12T16:58:45,602+0800 [pulsar-client-io-17-1] INFO org.apache.pulsar.client.impl.ProducerImpl - [persistent://pulsar/default2/alltables6-partition-4] [alltables610-persistent://pulsar/default2/alltables6-persistent://pulsar/default2/input_test5-partition-6-0] Re-Sending 1 messages to server 2024-01-12T16:58:45,608+0800 [pulsar-client-io-17-1] WARN org.apache.pulsar.client.impl.ClientCnx - [id: 0x7d55f577, L:/9.165.148.28:39306 - R:21.24.16.52/21.24.16.52:6650] Received send error from server: PersistenceError : Cannot determine whether the message is a duplicate at this time 2024-01-12T16:58:45,608+0800 [pulsar-client-io-17-1] WARN org.apache.pulsar.client.impl.ClientCnx - [21.24.16.52/21.24.16.52:6650] Got exception io.netty.channel.StacklessClosedChannelException 2024-01-12T16:58:45,610+0800 [pulsar-client-io-17-1] WARN org.apache.pulsar.client.impl.ClientCnx - [id: 0x7d55f577, L:/9.165.148.28:39306 ! R:21.24.16.52/21.24.16.52:6650] Received send error from server: PersistenceError : Cannot determine whether the message is a duplicate at this time 2024-01-12T16:58:45,610+0800 [pulsar-client-io-17-1] WARN org.apache.pulsar.client.impl.ClientCnx - [id: 0x7d55f577, L:/9.165.148.28:39306 ! R:21.24.16.52/21.24.16.52:6650] Received send error from server: PersistenceError : Cannot determine whether the message is a duplicate at this time 2024-01-12T16:58:45,610+0800 [pulsar-client-io-17-1] INFO org.apache.pulsar.client.impl.ClientCnx - [id: 0x7d55f577, L:/9.165.148.28:39306 ! R:21.24.16.52/21.24.16.52:6650] Disconnected 2024-01-12T16:58:45,610+0800 [pulsar-client-io-17-1] INFO org.apache.pulsar.client.impl.ConnectionHandler - [persistent://pulsar/default2/alltables6-partition-0] [alltables610-persistent://pulsar/default2/alltables6-persistent://pulsar/default2/input_test5-partition-6-0] Closed connection [id: 0x7d55f577, L:/9.165.148.28:39306 ! R:21.24.16.52/21.24.16.52:6650] -- Will try again in 0.1 s 2024-01-12T16:58:45,610+0800 [pulsar-client-io-17-1] INFO org.apache.pulsar.client.impl.ConnectionHandler - [persistent://pulsar/default2/alltables6-partition-2] [alltables610-persistent://pulsar/default2/alltables6-persistent://pulsar/default2/input_test5-partition-7-0] Closed connection [id: 0x7d55f577, L:/9.165.148.28:39306 ! R:21.24.16.52/21.24.16.52:6650] -- Will try again in 0.1 s ``` ### Anything else? After stuck, the heap dump of the broker was generated and something unusual was discovered  the `pendingAddOps` of LedgerHandle is also retains a lot of requests, the first request status in the queue is not completed, and pendingWriteRequests = 0, and addEntrySuccessBookies is empty.  But the second request is completed status. In isDuplicate of MessageDeduplication, the sequenceId is between lastSequenceIdPersisted and highestSequencedPushed, this is the reason why we receive `Cannot determine whether the message is a duplicate at this time` error ``` java if (lastSequenceIdPersisted != null && sequenceId <= lastSequenceIdPersisted) { return MessageDupStatus.Dup; } else { return MessageDupStatus.Unknown; } ``` ### Are you willing to submit a PR? - [ ] 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]
