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/16 06:52:18 UTC

[GitHub] [pulsar] congbobo184 opened a new pull request #14709: [Transaction] Fix transaction buffer recover BrokerMetadataException close topic

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


   ### Motivation
   When TopicTransactionBuffer recover fail throw BrokerMetadataException, we should close this topic, if we don't close the topic, we can't send message because TopicTransactionBuffer recover fail
   
   ![image](https://user-images.githubusercontent.com/39078850/158532983-993c0303-4051-4e56-90e1-c6ce89fa3775.png)
   
   ### Modifications
   When recover fail by BrokerMetadataException, close topic
   ### 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] github-actions[bot] commented on pull request #14709: [Transaction] Fix transaction buffer recover BrokerMetadataException close topic

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #14709:
URL: https://github.com/apache/pulsar/pull/14709#issuecomment-1068809088


   @congbobo184:Thanks for your contribution. For this PR, do we need to update docs?
   (The [PR template contains info about doc](https://github.com/apache/pulsar/blob/master/.github/PULL_REQUEST_TEMPLATE.md#documentation), which helps others know more about the changes. Can you provide doc-related info in this and future PR descriptions? Thanks)


-- 
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 #14709: [Transaction] Fix transaction buffer recover BrokerMetadataException close topic

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



##########
File path: pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java
##########
@@ -802,7 +802,7 @@ public void testCancelTxnTimeout() throws Exception{
     public void testNotChangeMaxReadPositionAndAddAbortTimesWhenCheckIfNoSnapshot() throws Exception {
         PersistentTopic persistentTopic = (PersistentTopic) getPulsarServiceList().get(0)
                 .getBrokerService()
-                .getTopic(NAMESPACE1 + "/test", true)
+                .getTopic(NAMESPACE1 + "/changeMaxReadPositionAndAddAbortTimes", true)

Review comment:
       you can add some suffix (UUID.randomUUID) in order to not reuse the same topic in case of test that is re-run due to a failure

##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java
##########
@@ -179,6 +179,9 @@ public void handleTxnEntry(Entry entry) {
 
                     @Override
                     public void recoverExceptionally(Exception e) {
+                        if (e instanceof PulsarClientException.BrokerMetadataException) {

Review comment:
       can we add a log line here ? like "Closing topic {} due to {} while recovering the transaction buffer"




-- 
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 #14709: [Transaction] Fix transaction buffer recover BrokerMetadataException close topic

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


   


-- 
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] lhotari commented on pull request #14709: [fix] [txn]: fix transaction buffer recover BrokerMetadataException close topic

Posted by GitBox <gi...@apache.org>.
lhotari commented on pull request #14709:
URL: https://github.com/apache/pulsar/pull/14709#issuecomment-1077350430


   This PR introduced a very flaky test that blocks CI . Master branch is basically broken at the moment.
   @congbobo184 Please fix #14844 asap so that CI gets unblocked.


-- 
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 #14709: [Transaction] Fix transaction buffer recover BrokerMetadataException close topic

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


   


-- 
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] gaoran10 commented on a change in pull request #14709: [Transaction] Fix transaction buffer recover BrokerMetadataException close topic

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



##########
File path: pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java
##########
@@ -442,4 +450,47 @@ private void checkSnapshotCount(TopicName topicName, boolean hasSnapshot,
         reader.close();
     }
 
+
+    @Test

Review comment:
       Please add a timeout param.




-- 
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] github-actions[bot] commented on pull request #14709: [Transaction] Fix transaction buffer recover BrokerMetadataException close topic

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #14709:
URL: https://github.com/apache/pulsar/pull/14709#issuecomment-1068857119


   @congbobo184:Thanks for providing doc info!


-- 
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] gaoran10 commented on a change in pull request #14709: [Transaction] Fix transaction buffer recover BrokerMetadataException close topic

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



##########
File path: pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java
##########
@@ -442,4 +450,47 @@ private void checkSnapshotCount(TopicName topicName, boolean hasSnapshot,
         reader.close();
     }
 
+
+    @Test
+    public void testTransactionBufferRecoverThrowBrokerMetadataException() throws Exception {
+        String topic = NAMESPACE1 + "/testTransactionBufferRecoverThrowBrokerMetadataException";
+        @Cleanup
+        Producer<byte[]> producer = pulsarClient
+                .newProducer()
+                .topic(topic)
+                .sendTimeout(0, TimeUnit.SECONDS)
+                .create();
+
+        Transaction txn = pulsarClient.newTransaction()
+                .withTransactionTimeout(5, TimeUnit.SECONDS)
+                .build().get();
+
+        producer.newMessage(txn).value("test".getBytes()).sendAsync();
+        producer.newMessage(txn).value("test".getBytes()).sendAsync();
+        txn.commit().get();
+
+        // take snapshot
+        PersistentTopic originalTopic = (PersistentTopic) getPulsarServiceList().get(0)
+                .getBrokerService().getTopic(TopicName.get(topic).toString(), false).get().get();
+        TransactionBufferSnapshotService transactionBufferSnapshotService =
+                mock(TransactionBufferSnapshotService.class);
+        SystemTopicClient.Reader<TransactionBufferSnapshot> reader = mock(SystemTopicClient.Reader.class);
+        // mock reader can't read snapshot fail
+        doThrow(new PulsarClientException.BrokerMetadataException("")).when(reader).hasMoreEvents();
+        doReturn(CompletableFuture.completedFuture(reader)).when(transactionBufferSnapshotService).createReader(any());
+
+        Field field = PulsarService.class.getDeclaredField("transactionBufferSnapshotService");
+        field.setAccessible(true);
+        field.set(getPulsarServiceList().get(0), transactionBufferSnapshotService);
+
+        // recover again will throw BrokerMetadataException then close topic
+        new TopicTransactionBuffer(originalTopic);
+        Awaitility.await().untilAsserted(() -> {
+            // isFenced means closed
+            Field close = AbstractTopic.class.getDeclaredField("isFenced");
+            close.setAccessible(true);
+            assertTrue((boolean) close.get(originalTopic));
+        });

Review comment:
       Could we send a transaction message at last? It could make sure the topic could recover successfully.

##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java
##########
@@ -179,6 +179,9 @@ public void handleTxnEntry(Entry entry) {
 
                     @Override
                     public void recoverExceptionally(Exception e) {
+                        if (e instanceof PulsarClientException.BrokerMetadataException) {

Review comment:
       Other exceptions don't need to close the topic, right?




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