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/09/26 08:50:57 UTC

[GitHub] [pulsar] liangyepianzhou opened a new pull request, #17836: [Fix][client] moving get sequenceId into the sync code segment

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

   ### Motivation
   When the producer sends messages in multiple threads, the message with the smaller sequence Id can be pushed later than the message with the bigger sequence Id. Then there will be an exception thrown at persistentTopic::publishTxnMessage 
   ### Modification
   Move getting sequenceId in the sync code.
   
   ### reproduce
   This test can reproduce this problem, but this is not guaranteed to recur.
   
   > 
       @Test
       public void testUpdateSequenceIdInSyncCodeSegment() throws Exception {
           String topic = NAMESPACE1 + "/sequenceId";
           int totalMessage = 10;
           int threadSize = 30;
           String topicName = "subscription";
           ExecutorService executorService = Executors.newFixedThreadPool(threadSize);
   
           //build producer/consumer
           Producer<byte[]> producer = pulsarClient.newProducer()
                   .topic(topic)
                   .producerName("producer")
                   .sendTimeout(0, TimeUnit.SECONDS)
                   .create();
   
           Consumer<byte[]> consumer = pulsarClient.newConsumer()
                   .topic(topic)
                   .subscriptionType(SubscriptionType.Exclusive)
                   .subscriptionName(topicName)
                   .subscribe();
           //store the send/ack result  futures
           ArrayList<CompletableFuture<MessageId>> sendFutures = new ArrayList<>();
           ArrayList<CompletableFuture<Void>> ackFutures = new ArrayList<>();
   
           //send and ack messages with transaction
           Transaction transaction = pulsarClient.newTransaction()
                   .withTransactionTimeout(10, TimeUnit.SECONDS)
                   .build()
                   .get();
   
           for (int i = 0; i < totalMessage * threadSize; i++) {
               producer.newMessage().send();
           }
   
           CountDownLatch countDownLatch = new CountDownLatch(threadSize);
           new Thread(() -> {
               for (int i = 0; i < threadSize; i++) {
                   executorService.submit(() -> {
                       try {
                           for (int j = 0; j < totalMessage; j++) {
                               CompletableFuture<MessageId> sendFuture = producer.newMessage(transaction).sendAsync();
                              sendFutures.add(sendFuture);
                               Message<byte[]> message = consumer.receive();
                               CompletableFuture<Void> ackFuture = consumer.acknowledgeAsync(message.getMessageId(),
                                       transaction);
                             ackFutures.add(ackFuture);
                           }
                           countDownLatch.countDown();
                       } catch (Exception e) {
                           log.error("Failed to send/ack messages with transaction.", e);
                       }
                   });
               }
           }).start();
           //wait the all send/ack op is excuted and store its futures in the arraylist.
           countDownLatch.await(10, TimeUnit.SECONDS);
           transaction.commit().get();
   
           for (int i = 0; i < threadSize * totalMessage; i++) {
               consumer.receive();
           }
   
       }
   
   ### Documentation
   
   - [x] `doc-not-needed` 
   (Please explain why)
   
   ### Matching PR in forked repository
   
   PR in forked repository: https://github.com/liangyepianzhou/pulsar/pull/6


-- 
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 diff in pull request #17836: [Fix][client] moving get sequenceId into the sync code segment

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on code in PR #17836:
URL: https://github.com/apache/pulsar/pull/17836#discussion_r990879267


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java:
##########
@@ -1517,4 +1518,57 @@ public void testGetTxnState() throws Exception {
         Transaction abortingTxn = transaction;
         Awaitility.await().until(() -> abortingTxn.getState() == Transaction.State.ABORTING);
     }
+
+    @Test
+    public void testUpdateSequenceIdInSyncCodeSegment() throws Exception {
+        String topic = NAMESPACE1 + "/sequenceId";
+        int totalMessage = 10;
+        int threadSize = 30;
+        String topicName = "subscription";
+        ExecutorService executorService = Executors.newFixedThreadPool(threadSize);
+
+        //build producer/consumer
+        Producer<byte[]> producer = pulsarClient.newProducer()
+                .topic(topic)
+                .producerName("producer")
+                .sendTimeout(0, TimeUnit.SECONDS)
+                .create();
+
+        Consumer<byte[]> consumer = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionType(SubscriptionType.Exclusive)
+                .subscriptionName(topicName)
+                .subscribe();
+
+        //send and ack messages with transaction
+        Transaction transaction = pulsarClient.newTransaction()
+                .withTransactionTimeout(10, TimeUnit.SECONDS)
+                .build()
+                .get();
+
+        for (int i = 0; i < totalMessage * threadSize; i++) {
+            producer.newMessage().send();
+        }
+
+        CountDownLatch countDownLatch = new CountDownLatch(threadSize);
+        for (int i = 0; i < threadSize; i++) {
+            executorService.submit(() -> {
+                try {
+                    for (int j = 0; j < totalMessage; j++) {
+                        producer.newMessage(transaction).sendAsync();
+                        Message<byte[]> message = consumer.receive();
+                        consumer.acknowledgeAsync(message.getMessageId(),
+                                transaction);
+                    }
+                } catch (Exception e) {
+                    log.error("Failed to send/ack messages with transaction.", e);

Review Comment:
   this test seems only check if the log is printed, haven't verify anything



-- 
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 a diff in pull request #17836: [fix][client] moving get sequenceId into the sync code segment

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on code in PR #17836:
URL: https://github.com/apache/pulsar/pull/17836#discussion_r1027571438


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java:
##########
@@ -1615,4 +1615,59 @@ public void testGetTxnState() throws Exception {
         Transaction abortingTxn = transaction;
         Awaitility.await().until(() -> abortingTxn.getState() == Transaction.State.ABORTING);
     }
+
+    @Test
+    public void testUpdateSequenceIdInSyncCodeSegment() throws Exception {
+        String topic = NAMESPACE1 + "/sequenceId";
+        int totalMessage = 10;
+        int threadSize = 30;
+        String topicName = "subscription";
+        ExecutorService executorService = Executors.newFixedThreadPool(threadSize);
+
+        //build producer/consumer
+        Producer<byte[]> producer = pulsarClient.newProducer()
+                .topic(topic)
+                .producerName("producer")
+                .sendTimeout(0, TimeUnit.SECONDS)
+                .create();
+
+        Consumer<byte[]> consumer = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionType(SubscriptionType.Exclusive)
+                .subscriptionName(topicName)
+                .subscribe();
+
+        //send and ack messages with transaction
+        Transaction transaction = pulsarClient.newTransaction()
+                .withTransactionTimeout(5, TimeUnit.SECONDS)
+                .build()
+                .get();
+
+        for (int i = 0; i < totalMessage * threadSize; i++) {
+            producer.newMessage().send();
+        }
+
+        CountDownLatch countDownLatch = new CountDownLatch(threadSize);
+        for (int i = 0; i < threadSize; i++) {
+            executorService.submit(() -> {
+                try {
+                    for (int j = 0; j < totalMessage; j++) {
+                        //The message will be sent with out-of-order sequence ID.
+                        producer.newMessage(transaction).sendAsync();
+                        Message<byte[]> message = consumer.receive();
+                        consumer.acknowledgeAsync(message.getMessageId(),
+                                transaction);
+                    }
+                } catch (Exception e) {
+                    log.error("Failed to send/ack messages with transaction.", e);
+                } finally {
+                    countDownLatch.countDown();
+                }
+            });
+        }
+        //wait the all send/ack op is executed and store its futures in the arraylist.
+        countDownLatch.await(5, TimeUnit.SECONDS);
+        //The transaction will be failed due to timeout.
+        transaction.commit().get();
+    }

Review Comment:
   The unexpected behavior is the transaction will not be committed successfully. This is clarified in the note.



-- 
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 #17836: [fix][client] moving get sequenceId into the sync code segment

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

   @codelipenghui @gaoran10 @poorbarcode /cc


-- 
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] codecov-commenter commented on pull request #17836: [fix][client] moving get sequenceId into the sync code segment

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on PR #17836:
URL: https://github.com/apache/pulsar/pull/17836#issuecomment-1326319761

   # [Codecov](https://codecov.io/gh/apache/pulsar/pull/17836?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#17836](https://codecov.io/gh/apache/pulsar/pull/17836?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (411d121) into [master](https://codecov.io/gh/apache/pulsar/commit/aeb4503be59f9a9450dfd47cf5dfcb375735d064?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (aeb4503) will **decrease** coverage by `3.02%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pulsar/pull/17836/graphs/tree.svg?width=650&height=150&src=pr&token=acYqCpsK9J&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pulsar/pull/17836?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #17836      +/-   ##
   ============================================
   - Coverage     31.39%   28.37%   -3.03%     
   + Complexity     6651     5383    -1268     
   ============================================
     Files           697      618      -79     
     Lines         68015    58568    -9447     
     Branches       7285     6093    -1192     
   ============================================
   - Hits          21353    16618    -4735     
   + Misses        43667    39621    -4046     
   + Partials       2995     2329     -666     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | unittests | `28.37% <0.00%> (-3.03%)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pulsar/pull/17836?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...va/org/apache/pulsar/client/impl/ProducerImpl.java](https://codecov.io/gh/apache/pulsar/pull/17836/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2NsaWVudC9pbXBsL1Byb2R1Y2VySW1wbC5qYXZh) | `17.01% <0.00%> (+1.16%)` | :arrow_up: |
   | [...naming/RangeEquallyDivideBundleSplitAlgorithm.java](https://codecov.io/gh/apache/pulsar/pull/17836/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2NvbW1vbi9uYW1pbmcvUmFuZ2VFcXVhbGx5RGl2aWRlQnVuZGxlU3BsaXRBbGdvcml0aG0uamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...r/common/naming/NamespaceBundleSplitAlgorithm.java](https://codecov.io/gh/apache/pulsar/pull/17836/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2NvbW1vbi9uYW1pbmcvTmFtZXNwYWNlQnVuZGxlU3BsaXRBbGdvcml0aG0uamF2YQ==) | `0.00% <0.00%> (-61.54%)` | :arrow_down: |
   | [...er/service/persistent/PersistentMessageFinder.java](https://codecov.io/gh/apache/pulsar/pull/17836/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci9zZXJ2aWNlL3BlcnNpc3RlbnQvUGVyc2lzdGVudE1lc3NhZ2VGaW5kZXIuamF2YQ==) | `0.00% <0.00%> (-40.91%)` | :arrow_down: |
   | [...apache/pulsar/common/naming/BundleSplitOption.java](https://codecov.io/gh/apache/pulsar/pull/17836/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2NvbW1vbi9uYW1pbmcvQnVuZGxlU3BsaXRPcHRpb24uamF2YQ==) | `0.00% <0.00%> (-40.00%)` | :arrow_down: |
   | [...he/pulsar/broker/service/TopicPoliciesService.java](https://codecov.io/gh/apache/pulsar/pull/17836/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci9zZXJ2aWNlL1RvcGljUG9saWNpZXNTZXJ2aWNlLmphdmE=) | `10.00% <0.00%> (-40.00%)` | :arrow_down: |
   | [.../apache/pulsar/broker/admin/impl/PackagesBase.java](https://codecov.io/gh/apache/pulsar/pull/17836/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci9hZG1pbi9pbXBsL1BhY2thZ2VzQmFzZS5qYXZh) | `0.00% <0.00%> (-25.69%)` | :arrow_down: |
   | [...ar/broker/loadbalance/impl/SimpleResourceUnit.java](https://codecov.io/gh/apache/pulsar/pull/17836/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci9sb2FkYmFsYW5jZS9pbXBsL1NpbXBsZVJlc291cmNlVW5pdC5qYXZh) | `28.57% <0.00%> (-23.81%)` | :arrow_down: |
   | [...ar/broker/loadbalance/impl/BundleSplitterTask.java](https://codecov.io/gh/apache/pulsar/pull/17836/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci9sb2FkYmFsYW5jZS9pbXBsL0J1bmRsZVNwbGl0dGVyVGFzay5qYXZh) | `57.77% <0.00%> (-20.01%)` | :arrow_down: |
   | [...e/pulsar/common/naming/NamespaceBundleFactory.java](https://codecov.io/gh/apache/pulsar/pull/17836/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2NvbW1vbi9uYW1pbmcvTmFtZXNwYWNlQnVuZGxlRmFjdG9yeS5qYXZh) | `34.44% <0.00%> (-18.34%)` | :arrow_down: |
   | ... and [156 more](https://codecov.io/gh/apache/pulsar/pull/17836/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   


-- 
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 #17836: [fix][client] moving get sequenceId into the sync code segment

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

   The pr had no activity for 30 days, mark with Stale label.


-- 
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] labuladong commented on pull request #17836: [fix][client] moving get sequenceId into the sync code segment

Posted by GitBox <gi...@apache.org>.
labuladong commented on PR #17836:
URL: https://github.com/apache/pulsar/pull/17836#issuecomment-1321301674

   > For example we send 4 messages (msg1, msg2, msg3, msg4) to the broker and then the 4 messages may get 4 sequence Id (1, 3, 2, 4)
   
   For your example, is this helpful to reproduce the issue you mentioned?
   
   ```java
           producer.newMessage(transaction).value("msg1".getBytes()).sequenceId(1).send();
           producer.newMessage(transaction).value("msg2".getBytes()).sequenceId(3).send();
           producer.newMessage(transaction).value("msg3".getBytes()).sequenceId(2).send();
           producer.newMessage(transaction).value("msg4".getBytes()).sequenceId(4).send();
   ```
   


-- 
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 diff in pull request #17836: [fix][client] moving get sequenceId into the sync code segment

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on code in PR #17836:
URL: https://github.com/apache/pulsar/pull/17836#discussion_r1031234086


##########
pulsar-broker/src/test/java/org/apache/pulsar/client/api/ClientDeduplicationTest.java:
##########
@@ -372,4 +375,51 @@ public void testKeyBasedBatchingOrder() throws Exception {
         consumer.close();
         producer.close();
     }
+
+    @Test
+    public void testUpdateSequenceIdInSyncCodeSe7gment() throws Exception {
+        final String topic = "persistent://my-property/my-ns/testUpdateSequenceIdInSyncCodeSegment";
+        int totalMessage = 10;
+        int threadSize = 30;

Review Comment:
   it's better to decrease the `threadSize` and increase the `totalMessage`, the test CPU cores size is not large, so `threadSize` can't be set to large, a lot of threads will wait. better to increase the  `totalMessage`.



-- 
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 a diff in pull request #17836: [fix][client] moving get sequenceId into the sync code segment

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on code in PR #17836:
URL: https://github.com/apache/pulsar/pull/17836#discussion_r1027571438


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java:
##########
@@ -1615,4 +1615,59 @@ public void testGetTxnState() throws Exception {
         Transaction abortingTxn = transaction;
         Awaitility.await().until(() -> abortingTxn.getState() == Transaction.State.ABORTING);
     }
+
+    @Test
+    public void testUpdateSequenceIdInSyncCodeSegment() throws Exception {
+        String topic = NAMESPACE1 + "/sequenceId";
+        int totalMessage = 10;
+        int threadSize = 30;
+        String topicName = "subscription";
+        ExecutorService executorService = Executors.newFixedThreadPool(threadSize);
+
+        //build producer/consumer
+        Producer<byte[]> producer = pulsarClient.newProducer()
+                .topic(topic)
+                .producerName("producer")
+                .sendTimeout(0, TimeUnit.SECONDS)
+                .create();
+
+        Consumer<byte[]> consumer = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionType(SubscriptionType.Exclusive)
+                .subscriptionName(topicName)
+                .subscribe();
+
+        //send and ack messages with transaction
+        Transaction transaction = pulsarClient.newTransaction()
+                .withTransactionTimeout(5, TimeUnit.SECONDS)
+                .build()
+                .get();
+
+        for (int i = 0; i < totalMessage * threadSize; i++) {
+            producer.newMessage().send();
+        }
+
+        CountDownLatch countDownLatch = new CountDownLatch(threadSize);
+        for (int i = 0; i < threadSize; i++) {
+            executorService.submit(() -> {
+                try {
+                    for (int j = 0; j < totalMessage; j++) {
+                        //The message will be sent with out-of-order sequence ID.
+                        producer.newMessage(transaction).sendAsync();
+                        Message<byte[]> message = consumer.receive();
+                        consumer.acknowledgeAsync(message.getMessageId(),
+                                transaction);
+                    }
+                } catch (Exception e) {
+                    log.error("Failed to send/ack messages with transaction.", e);
+                } finally {
+                    countDownLatch.countDown();
+                }
+            });
+        }
+        //wait the all send/ack op is executed and store its futures in the arraylist.
+        countDownLatch.await(5, TimeUnit.SECONDS);
+        //The transaction will be failed due to timeout.
+        transaction.commit().get();
+    }

Review Comment:
   The expected behavior is the transaction will be committed successfully. This is clarified in the note.



-- 
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 merged pull request #17836: [fix][client] moving get sequenceId into the sync code segment

Posted by GitBox <gi...@apache.org>.
liangyepianzhou merged PR #17836:
URL: https://github.com/apache/pulsar/pull/17836


-- 
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 a diff in pull request #17836: [Fix][client] moving get sequenceId into the sync code segment

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on code in PR #17836:
URL: https://github.com/apache/pulsar/pull/17836#discussion_r990880468


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java:
##########
@@ -1517,4 +1518,57 @@ public void testGetTxnState() throws Exception {
         Transaction abortingTxn = transaction;
         Awaitility.await().until(() -> abortingTxn.getState() == Transaction.State.ABORTING);
     }
+
+    @Test
+    public void testUpdateSequenceIdInSyncCodeSegment() throws Exception {
+        String topic = NAMESPACE1 + "/sequenceId";
+        int totalMessage = 10;
+        int threadSize = 30;
+        String topicName = "subscription";
+        ExecutorService executorService = Executors.newFixedThreadPool(threadSize);
+
+        //build producer/consumer
+        Producer<byte[]> producer = pulsarClient.newProducer()
+                .topic(topic)
+                .producerName("producer")
+                .sendTimeout(0, TimeUnit.SECONDS)
+                .create();
+
+        Consumer<byte[]> consumer = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionType(SubscriptionType.Exclusive)
+                .subscriptionName(topicName)
+                .subscribe();
+
+        //send and ack messages with transaction
+        Transaction transaction = pulsarClient.newTransaction()
+                .withTransactionTimeout(10, TimeUnit.SECONDS)
+                .build()
+                .get();
+
+        for (int i = 0; i < totalMessage * threadSize; i++) {
+            producer.newMessage().send();
+        }
+
+        CountDownLatch countDownLatch = new CountDownLatch(threadSize);
+        for (int i = 0; i < threadSize; i++) {
+            executorService.submit(() -> {
+                try {
+                    for (int j = 0; j < totalMessage; j++) {
+                        producer.newMessage(transaction).sendAsync();
+                        Message<byte[]> message = consumer.receive();
+                        consumer.acknowledgeAsync(message.getMessageId(),
+                                transaction);
+                    }
+                } catch (Exception e) {
+                    log.error("Failed to send/ack messages with transaction.", e);

Review Comment:
   This test has a relatively high probability of failing on committing the transaction before the code has been modified.



-- 
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 diff in pull request #17836: [fix][client] moving get sequenceId into the sync code segment

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on code in PR #17836:
URL: https://github.com/apache/pulsar/pull/17836#discussion_r1030309753


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java:
##########
@@ -1615,4 +1615,59 @@ public void testGetTxnState() throws Exception {
         Transaction abortingTxn = transaction;
         Awaitility.await().until(() -> abortingTxn.getState() == Transaction.State.ABORTING);
     }
+
+    @Test
+    public void testUpdateSequenceIdInSyncCodeSegment() throws Exception {

Review Comment:
   could you please add the test on `ClientDeduplicationTest` multi-thread send a number of messages that can send success and the consumer can receive the message.  it seems not a bug in the transaction, so don't test it with the transaction



-- 
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] labuladong commented on a diff in pull request #17836: [fix][client] moving get sequenceId into the sync code segment

Posted by GitBox <gi...@apache.org>.
labuladong commented on code in PR #17836:
URL: https://github.com/apache/pulsar/pull/17836#discussion_r1027386615


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java:
##########
@@ -1615,4 +1615,59 @@ public void testGetTxnState() throws Exception {
         Transaction abortingTxn = transaction;
         Awaitility.await().until(() -> abortingTxn.getState() == Transaction.State.ABORTING);
     }
+
+    @Test
+    public void testUpdateSequenceIdInSyncCodeSegment() throws Exception {
+        String topic = NAMESPACE1 + "/sequenceId";
+        int totalMessage = 10;
+        int threadSize = 30;
+        String topicName = "subscription";
+        ExecutorService executorService = Executors.newFixedThreadPool(threadSize);
+
+        //build producer/consumer
+        Producer<byte[]> producer = pulsarClient.newProducer()
+                .topic(topic)
+                .producerName("producer")
+                .sendTimeout(0, TimeUnit.SECONDS)
+                .create();
+
+        Consumer<byte[]> consumer = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionType(SubscriptionType.Exclusive)
+                .subscriptionName(topicName)
+                .subscribe();
+
+        //send and ack messages with transaction
+        Transaction transaction = pulsarClient.newTransaction()
+                .withTransactionTimeout(5, TimeUnit.SECONDS)
+                .build()
+                .get();
+
+        for (int i = 0; i < totalMessage * threadSize; i++) {
+            producer.newMessage().send();
+        }
+
+        CountDownLatch countDownLatch = new CountDownLatch(threadSize);
+        for (int i = 0; i < threadSize; i++) {
+            executorService.submit(() -> {
+                try {
+                    for (int j = 0; j < totalMessage; j++) {
+                        //The message will be sent with out-of-order sequence ID.
+                        producer.newMessage(transaction).sendAsync();
+                        Message<byte[]> message = consumer.receive();
+                        consumer.acknowledgeAsync(message.getMessageId(),
+                                transaction);
+                    }
+                } catch (Exception e) {
+                    log.error("Failed to send/ack messages with transaction.", e);
+                } finally {
+                    countDownLatch.countDown();
+                }
+            });
+        }
+        //wait the all send/ack op is executed and store its futures in the arraylist.
+        countDownLatch.await(5, TimeUnit.SECONDS);
+        //The transaction will be failed due to timeout.
+        transaction.commit().get();
+    }

Review Comment:
   What behavior is expected and what behavior is unexpected for this test? I suggest clarifying that by `assert` explicitly.



-- 
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 #17836: [fix][client] moving get sequenceId into the sync code segment

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

   > > For example we send 4 messages (msg1, msg2, msg3, msg4) to the broker and then the 4 messages may get 4 sequence Id (1, 3, 2, 4)
   > 
   > For your example, is this helpful to reproduce the issue you mentioned?
   > 
   > ```java
   >         producer.newMessage(transaction).value("msg1".getBytes()).sequenceId(1).send();
   >         producer.newMessage(transaction).value("msg2".getBytes()).sequenceId(3).send();
   >         producer.newMessage(transaction).value("msg3".getBytes()).sequenceId(2).send();
   >         producer.newMessage(transaction).value("msg4".getBytes()).sequenceId(4).send();
   > ```
   
   No, it is a sync method.


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