You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2022/03/22 02:26:41 UTC

[GitHub] [pulsar] congbobo184 opened a new pull request #14781: [fix][txn]: fix pending ack is recovering throw CursorAlreadyClosedxception

congbobo184 opened a new pull request #14781:
URL: https://github.com/apache/pulsar/pull/14781


   ### Motivation
   When Transaction PendingAck recover fail throw CursorAlreadyClosedException, we should stop the recover op. the cursor was been closed, the pendingAck was been closed, so we should stop the recover op, in order to release thread resources
   
   ```
   02:03:00.072 [pulsar-transaction-executor-4-1] ERROR org.apache.pulsar.broker.transaction.pendingack.impl.MLPendingAckStore - MLPendingAckStore of topic [public/default/persistent/source-topic-partition-13-test__transaction_pending_ack] stat reply fail!
   org.apache.bookkeeper.mledger.ManagedLedgerException$CursorAlreadyClosedException: Cursor was already closed
   ```
   
   ### Modifications
   When recover fail by CursorAlreadyClosedException, comeplete recover
   ### Verifying this change
   add test for it
   ### Does this pull request potentially affect one of the following parts:
   
   *If `yes` was chosen, please highlight the changes*
   
     - Dependencies (does it add or upgrade a dependency): (no)
     - The public API: (no)
     - The schema: (no)
     - The default values of configurations: (no)
     - The wire protocol: (no)
     - The rest endpoints: (no)
     - The admin cli options: (no)
     - Anything that affects deployment: (no)
   
   ### Documentation
   
     - Does this pull request introduce a new feature? (yes)
     - If yes, how is the feature documented? (not applicable / docs / JavaDocs / not documented)
     - If a feature is not applicable for documentation, explain why?
     - If a feature is not documented yet in this PR, please create a followup issue for adding the documentation
   


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] congbobo184 commented on a change in pull request #14781: [fix][txn]: fix pending ack is recovering throw CursorAlreadyClosedxception

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on a change in pull request #14781:
URL: https://github.com/apache/pulsar/pull/14781#discussion_r831844989



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStore.java
##########
@@ -398,7 +398,7 @@ public Entry get() {
         public void readEntriesFailed(ManagedLedgerException exception, Object ctx) {
             if (managedLedger.getConfig().isAutoSkipNonRecoverableData()
                     && exception instanceof ManagedLedgerException.NonRecoverableLedgerException
-                    || exception instanceof ManagedLedgerException.ManagedLedgerFencedException) {

Review comment:
       > Why remove the `ManagedLedgerFencedException` check?
   
   we can't delete ManagedLedgerFencedException, that's my mistake




-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] codelipenghui commented on a change in pull request #14781: [fix][txn]: fix pending ack is recovering throw CursorAlreadyClosedxception

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on a change in pull request #14781:
URL: https://github.com/apache/pulsar/pull/14781#discussion_r831730765



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStore.java
##########
@@ -398,7 +398,7 @@ public Entry get() {
         public void readEntriesFailed(ManagedLedgerException exception, Object ctx) {
             if (managedLedger.getConfig().isAutoSkipNonRecoverableData()
                     && exception instanceof ManagedLedgerException.NonRecoverableLedgerException
-                    || exception instanceof ManagedLedgerException.ManagedLedgerFencedException) {

Review comment:
       And from the current implementation, if return `isReadable` here which means the pending ack reply will be complete, is it expected behavior? https://github.com/apache/pulsar/pull/14781/files#diff-07a1d142ec4105cb65fac733494182b461214a68d735ae1c3f5104c3eb4f92dbL363

##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStore.java
##########
@@ -398,7 +398,7 @@ public Entry get() {
         public void readEntriesFailed(ManagedLedgerException exception, Object ctx) {
             if (managedLedger.getConfig().isAutoSkipNonRecoverableData()
                     && exception instanceof ManagedLedgerException.NonRecoverableLedgerException
-                    || exception instanceof ManagedLedgerException.ManagedLedgerFencedException) {

Review comment:
       Why remove the `ManagedLedgerFencedException` check?




-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] congbobo184 commented on a change in pull request #14781: [fix][txn]: fix pending ack is recovering throw CursorAlreadyClosedxception

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on a change in pull request #14781:
URL: https://github.com/apache/pulsar/pull/14781#discussion_r831847153



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStore.java
##########
@@ -398,7 +398,7 @@ public Entry get() {
         public void readEntriesFailed(ManagedLedgerException exception, Object ctx) {
             if (managedLedger.getConfig().isAutoSkipNonRecoverableData()
                     && exception instanceof ManagedLedgerException.NonRecoverableLedgerException
-                    || exception instanceof ManagedLedgerException.ManagedLedgerFencedException) {

Review comment:
       > And from the current implementation, if return `isReadable` here which means the pending ack reply will be complete, is it expected behavior? https://github.com/apache/pulsar/pull/14781/files#diff-07a1d142ec4105cb65fac733494182b461214a68d735ae1c3f5104c3eb4f92dbL363
   
   if ManagedLedgerFencedException or CursorAlreadyClosedxception, the pending ack has been closed so we can stop the recover.




-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on a change in pull request #14781: [fix][txn]: fix pending ack is recovering throw CursorAlreadyClosedxception

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #14781:
URL: https://github.com/apache/pulsar/pull/14781#discussion_r831867147



##########
File path: pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java
##########
@@ -620,6 +620,17 @@ public void testEndTPRecoveringWhenManagerLedgerDisReadable() throws Exception{
         PendingAckHandleImpl pendingAckHandle2 = new PendingAckHandleImpl(persistentSubscription);
         Awaitility.await().untilAsserted(() ->
                 assertEquals(pendingAckHandle2.getStats().state, "Ready"));
+
+        doAnswer(invocation -> {
+            AsyncCallbacks.ReadEntriesCallback callback = invocation.getArgument(1);
+            callback.readEntriesFailed(new ManagedLedgerException.CursorAlreadyClosedException("test"), null);
+            return null;
+        }).when(managedCursor).asyncReadEntries(anyInt(), any(), any(), any());
+
+        PendingAckHandleImpl pendingAckHandle3 = new PendingAckHandleImpl(persistentSubscription);
+
+        Awaitility.await().untilAsserted(() ->
+                assertEquals(pendingAckHandle2.getStats().state, "Ready"));

Review comment:
       did you mean `pendingAckHandle3` here ?




-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] congbobo184 merged pull request #14781: [fix][txn]: fix pending ack is recovering throw CursorAlreadyClosedxception

Posted by GitBox <gi...@apache.org>.
congbobo184 merged pull request #14781:
URL: https://github.com/apache/pulsar/pull/14781


   


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org