codelipenghui opened a new pull request #11737:
URL: https://github.com/apache/pulsar/pull/11737


   Here is the log when the issue happens. The producer continues to reconnect 
to the broker, but the fenced state of the topic is always true.
   
   ```
   19:01:42.351 [pulsar-io-4-1] INFO  
org.apache.pulsar.broker.service.ServerCnx - 
[/10.24.34.151:48052][persistent://public/default/test-8] Creating producer. 
producerId=8
   19:01:42.352 [Thread-174681] INFO  
org.apache.pulsar.broker.service.ServerCnx - [/10.24.34.151:48052] 
persistent://public/default/test-8 configured with schema false
   19:01:42.352 [Thread-174681] WARN  
org.apache.pulsar.broker.service.AbstractTopic - 
[persistent://public/default/test-8] Attempting to add producer to a fenced 
topic
   19:01:42.352 [Thread-174681] ERROR 
org.apache.pulsar.broker.service.ServerCnx - [/10.24.34.151:48052] Failed to 
add producer to topic persistent://public/default/test-8: producerId=8, 
org.apache.pulsar.broker.service.BrokerServiceException$TopicFencedException: 
Topic is temporarily unavailable
   ```
   
   After check the heap dump of the broker, the `pendingWriteOps` is 5, this is 
the reason why the topic can not recover from the fenced state.
   
   
![image](https://user-images.githubusercontent.com/12592133/130329537-e3c1d71e-169c-4cfe-bda9-b3821bffa1e4.png)
   
   
   The topic will change to unfenced only the `pendingWriteOps` is 0, details 
can find at 
[PersistentTopic.decrementPendingWriteOpsAndCheck()](https://github.com/apache/pulsar/blob/794aa20d9f2a4e668cc36465362d22e042e6e536/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java#L463)
   
   But after checking the ML state of the topic, it shows the 
`pendingAddEntries` is 0 which not equals to `pendingWriteOps` of the topic.
   
   
![image](https://user-images.githubusercontent.com/12592133/130329559-23a7c91b-6973-4ce5-adbf-537e5305c8f9.png)
   
   The root cause is we are polling add entry op from the `pendingAddEntries` 
in multiple threads, one is the the ZK callback thread when complete the ledger 
creating 
(https://github.com/apache/pulsar/blob/794aa20d9f2a4e668cc36465362d22e042e6e536/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java#L1406,
 
https://github.com/apache/pulsar/blob/794aa20d9f2a4e668cc36465362d22e042e6e536/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java#L1669),
   another one is the ML worker thread when complete the add entry op 
(https://github.com/apache/pulsar/blob/794aa20d9f2a4e668cc36465362d22e042e6e536/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java#L181)
   
   After the ongoing add entry op complete, but the corresponding op might been 
polled by the `clearPendingAddEntries` method. So it will poll another one, but 
due to
   not equals to the current op, the polled op will not get a chance to be 
failed, so that the `pendingWriteOps` will not change to 0.
   
   I have attached the complete logs for the topic: 
[downloaded-logs-20210820-125853.csv](https://github.com/apache/pulsar/files/7025898/downloaded-logs-20210820-125853.csv)
   
   The fix is to complete the add entry op with ManagedLedgerException if the 
polled op is not equals to the current op.
   
   


-- 
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