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 2021/10/28 12:21:45 UTC

[GitHub] [pulsar] liangyepianzhou opened a new pull request #12521: Txn client check timeout

liangyepianzhou opened a new pull request #12521:
URL: https://github.com/apache/pulsar/pull/12521


   ### Motivation
   Optimize the logic on the Transaction Client side.
   Avoid sending and acking messages with timeout  transactions.
   
   ### Modifications
   
   * TransactionImp
   
        *  Add a tool field for CAS to replace State : STATE_UPDATE.
   **When committing and aborted, only the successful cas operation will make subsequent judgments, otherwise it will return a failure future**
        *   Implement TimerTasker to execute tasks that replace the state of the transaction as Aborted.
   * TransactionBuildImpl
        * In the callback of the build method, call the timer of PulsarClient to start a Timeout. Pass in the corresponding transactionImpl (TimeTasker has been implemented)
   
   
   ### Verifying this change
   Add a test in TransactionEndToEndTest.
   
   ### 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): (yes / no)
     - The public API: (yes / no)
     - The schema: (yes / no / don't know)
     - The default values of configurations: (yes / no)
     - The wire protocol: (yes / no)
     - The rest endpoints: (yes / no)
     - The admin cli options: (yes / no)
     - Anything that affects deployment: (yes / no / don't know)
   
   ### Documentation  
   - [ ] `no-need-doc` 
     Optimize feature.
   
   


-- 
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] liangyepianzhou commented on pull request #12521: [Transaction]Txn client check timeout

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


   /pulsarbot run-failure-checks


-- 
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 #12521: Txn client check timeout

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


   @liangyepianzhou: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] liangyepianzhou removed a comment on pull request #12521: [Transaction]Txn client check timeout

Posted by GitBox <gi...@apache.org>.
liangyepianzhou removed a comment on pull request #12521:
URL: https://github.com/apache/pulsar/pull/12521#issuecomment-963116477


   /pulsarbot run-failure-checks


-- 
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] liangyepianzhou removed a comment on pull request #12521: [Transaction]Txn client check timeout

Posted by GitBox <gi...@apache.org>.
liangyepianzhou removed a comment on pull request #12521:
URL: https://github.com/apache/pulsar/pull/12521#issuecomment-964655264






-- 
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] liangyepianzhou commented on pull request #12521: [Transaction]Txn client check timeout

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


   /pulsarbot run-failure-checks


-- 
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] liangyepianzhou removed a comment on pull request #12521: [Transaction]Txn client check timeout

Posted by GitBox <gi...@apache.org>.
liangyepianzhou removed a comment on pull request #12521:
URL: https://github.com/apache/pulsar/pull/12521#issuecomment-996647820


   /pulsarbot run-failure-checks


-- 
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] liangyepianzhou commented on pull request #12521: [Transaction]Txn client check timeout

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


   /pulsarbot run-failure-checks


-- 
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 #12521: [Transaction]Txn client check timeout

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


   


-- 
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] liangyepianzhou commented on pull request #12521: [Transaction]Txn client check timeout

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


   /pulsarbot run-failure-checks


-- 
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] liangyepianzhou commented on pull request #12521: [Transaction]Txn client check timeout

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


   /pulsarbot run-failure-checks


-- 
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 #12521: [Transaction]Txn client check timeout

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



##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/PartitionedProducerImpl.java
##########
@@ -191,6 +193,20 @@ private void start(List<Integer> indexList) {
 
     @Override
     CompletableFuture<MessageId> internalSendWithTxnAsync(Message<?> message, Transaction txn) {
+        if (txn != null && ((TransactionImpl)txn).getState() != TransactionImpl.State.OPEN) {

Review comment:
       Can we reuse the code from newly added in ProducerImpl?




-- 
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] liangyepianzhou commented on pull request #12521: [Transaction]Txn client check timeout

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


   /pulsarbot run-failure-checks


-- 
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] liangyepianzhou removed a comment on pull request #12521: [Transaction]Txn client check timeout

Posted by GitBox <gi...@apache.org>.
liangyepianzhou removed a comment on pull request #12521:
URL: https://github.com/apache/pulsar/pull/12521#issuecomment-963073338


   /pulsarbot run-failure-checks


-- 
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 pull request #12521: [Transaction]Txn client check timeout

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


   @codelipenghui @gaoran10 @eolivelli please help review this PR, 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] liangyepianzhou commented on pull request #12521: [Transaction]Txn client check timeout

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


   /pulsarbot run-failure-checks


-- 
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] liangyepianzhou commented on pull request #12521: [Transaction]Txn client check timeout

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


   /pulsarbot run-failure-checks


-- 
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 #12521: [Transaction]Txn client check timeout

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



##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/PartitionedProducerImpl.java
##########
@@ -191,6 +193,20 @@ private void start(List<Integer> indexList) {
 
     @Override
     CompletableFuture<MessageId> internalSendWithTxnAsync(Message<?> message, Transaction txn) {
+        if (txn != null && ((TransactionImpl)txn).getState() != TransactionImpl.State.OPEN) {

Review comment:
       @liangyepianzhou LGTM, please check the above comment to avoid the duplicated code.




-- 
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 #12521: [Transaction]Txn client check timeout

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



##########
File path: pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java
##########
@@ -960,4 +960,36 @@ public void oneTransactionOneTopicWithMultiSubTest() throws Exception {
         }
         assertTrue(flag);
     }
+
+    @Test
+    public void testTxnTimeOutInClient() throws Exception{
+        String topic = NAMESPACE1 + "/testTxnTimeOutInClient";
+        Producer producer = pulsarClient.newProducer(Schema.STRING).producerName("testTxnTimeOut_producer")
+                .topic(topic).sendTimeout(0, TimeUnit.SECONDS).enableBatching(false).create();
+        Consumer consumer = pulsarClient.newConsumer(Schema.STRING).consumerName("testTxnTimeOut_consumer")
+                .topic(topic).subscriptionName("testTxnTimeOut_sub").subscribe();
+
+        Transaction transaction = pulsarClient.newTransaction().withTransactionTimeout(1, TimeUnit.SECONDS)
+                .build().get();
+        producer.newMessage().send();
+        Awaitility.await().untilAsserted(() -> {
+            Assert.assertEquals(((TransactionImpl)transaction).getState(), TransactionImpl.State.ABORTED);
+        });
+
+        try {
+            producer.newMessage(transaction).send();
+            Assert.fail();
+        } catch (Exception e) {
+            Assert.assertTrue(e.getCause().getCause() instanceof TransactionCoordinatorClientException

Review comment:
       This will confuse users because they did not abort the transaction, we should using a TransactionTimeoutException.

##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java
##########
@@ -490,6 +491,15 @@ public void reconsumeLaterCumulative(Message<?> message, long delayTime, TimeUni
         if (null != txn) {
             checkArgument(txn instanceof TransactionImpl);
             txnImpl = (TransactionImpl) txn;
+           if (txnImpl.getState() != TransactionImpl.State.OPEN) {
+               CompletableFuture<Void> completableFuture = new CompletableFuture<>();
+               completableFuture
+                       .completeExceptionally(new TransactionCoordinatorClientException
+                               .InvalidTxnStatusException("["+ txn.getTxnID().toString() +"] with unexpected state : "

Review comment:
       It's better to create the template error message for InvalidTxnStatusException, looks like using a String formater, so that you can reuse the formater.




-- 
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] liangyepianzhou commented on pull request #12521: [Transaction]Txn client check timeout

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


   /pulsarbot run-failure-checks


-- 
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 #12521: [Transaction]Txn client check timeout

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



##########
File path: pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java
##########
@@ -757,12 +758,31 @@ public void produceAndConsumeCloseStateTxnTest() throws Exception {
             timeoutTxn.commit().get();
             fail();
         } catch (Exception e) {
-            assertTrue(e.getCause() instanceof TransactionNotFoundException);
+            assertTrue(e.getCause() instanceof TransactionCoordinatorClientException.InvalidTxnStatusException);
         }
         Field field = TransactionImpl.class.getDeclaredField("state");
         field.setAccessible(true);
         TransactionImpl.State state = (TransactionImpl.State) field.get(timeoutTxn);
-        assertEquals(state, TransactionImpl.State.ERROR);
+        assertEquals(state, TransactionImpl.State.TIMEOUT);

Review comment:
       please don't change this test, we should check the TC timeout, so return State.ERROR

##########
File path: pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConsumerAckResponseTest.java
##########
@@ -50,6 +51,8 @@ public void setup() throws Exception {
         super.producerBaseSetup();
         doReturn(1L).when(transaction).getTxnIdLeastBits();
         doReturn(1L).when(transaction).getTxnIdMostBits();
+        doReturn(new TxnID(1L, 1L)).when(transaction).getTxnID();

Review comment:
       why add this logic




-- 
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] liangyepianzhou commented on pull request #12521: [Transaction]Txn client check timeout

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


   /pulsarbot run-failure-checks


-- 
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] liangyepianzhou commented on pull request #12521: [Transaction]Txn client check timeout

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


   @congbobo184 @gaoran10 @codelipenghui Please help review. THX!


-- 
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] liangyepianzhou removed a comment on pull request #12521: [Transaction]Txn client check timeout

Posted by GitBox <gi...@apache.org>.
liangyepianzhou removed a comment on pull request #12521:
URL: https://github.com/apache/pulsar/pull/12521#issuecomment-957022093


   /pulsarbot run-failure-checks


-- 
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] liangyepianzhou commented on pull request #12521: [Transaction]Txn client check timeout

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


   /pulsarbot run-failure-checks


-- 
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] liangyepianzhou commented on pull request #12521: [Transaction]Txn client check timeout

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


   /pulsarbot run-failure-checks


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