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/11/29 11:15:26 UTC

[GitHub] [pulsar] Technoboy- opened a new pull request #13023: Fix consume message order issue when use listener.

Technoboy- opened a new pull request #13023:
URL: https://github.com/apache/pulsar/pull/13023


   ### Motivation
   When ConsumerImpl try to call `tryTriggerListener`,  it's the internal thread(internalPinnedExecutor) executing `triggerListener`.
   
   Then external executor will call `callMessageListener` -> `tryTriggerListener`.  
   
   This will result in the message order issue. We should keep calling `triggerListener` in the internalPinnedExecutor.
   
   ### Modifications
   - Call `triggerListener` always in internalPinnedExecutor.
   
   ### Documentation
   
   - [x] `no-need-doc` 
   
   
   
   


-- 
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] Jason918 commented on a change in pull request #13023: Fix consume message order issue when use listener.

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



##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java
##########
@@ -916,30 +916,32 @@ private void doPendingBatchReceiveTask(Timeout timeout) {
     protected void triggerListener() {
         // Trigger the notification on the message listener in a separate thread to avoid blocking the networking
         // thread while the message processing happens
-        try {
-            // Control executor to call MessageListener one by one.
-            if (executorQueueSize.get() < 1) {
-                final Message<T> msg = internalReceive(0, TimeUnit.MILLISECONDS);
-                if (msg != null) {
-                    executorQueueSize.incrementAndGet();
-                    if (SubscriptionType.Key_Shared == conf.getSubscriptionType()) {
-                        executorProvider.getExecutor(peekMessageKey(msg)).execute(() ->
-                                callMessageListener(msg));
-                    } else {
-                        getExternalExecutor(msg).execute(() -> {
-                            callMessageListener(msg);
-                        });
+        internalPinnedExecutor.execute(() -> {

Review comment:
       Does this ordering  guarantee depends on this `internalPinnedExecutor` to be single thread? 
   Aka, does this `triggerListener` will be called concurrently?
   This `internalPinnedExecutor` can be configured with `ClientConfigurationData#numIoThreads`




-- 
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] Technoboy- commented on a change in pull request #13023: Fix consume message order issue when use listener.

Posted by GitBox <gi...@apache.org>.
Technoboy- commented on a change in pull request #13023:
URL: https://github.com/apache/pulsar/pull/13023#discussion_r767231840



##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java
##########
@@ -916,30 +916,32 @@ private void doPendingBatchReceiveTask(Timeout timeout) {
     protected void triggerListener() {
         // Trigger the notification on the message listener in a separate thread to avoid blocking the networking
         // thread while the message processing happens
-        try {
-            // Control executor to call MessageListener one by one.
-            if (executorQueueSize.get() < 1) {
-                final Message<T> msg = internalReceive(0, TimeUnit.MILLISECONDS);
-                if (msg != null) {
-                    executorQueueSize.incrementAndGet();
-                    if (SubscriptionType.Key_Shared == conf.getSubscriptionType()) {
-                        executorProvider.getExecutor(peekMessageKey(msg)).execute(() ->
-                                callMessageListener(msg));
-                    } else {
-                        getExternalExecutor(msg).execute(() -> {
-                            callMessageListener(msg);
-                        });
+        internalPinnedExecutor.execute(() -> {

Review comment:
       Yes, but every consumer instance will bind to one executor.
   See : internalExecutorService.getExecutor().




-- 
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] michaeljmarshall commented on a change in pull request #13023: Fix consume message order issue when use listener.

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



##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java
##########
@@ -916,30 +916,32 @@ private void doPendingBatchReceiveTask(Timeout timeout) {
     protected void triggerListener() {
         // Trigger the notification on the message listener in a separate thread to avoid blocking the networking
         // thread while the message processing happens
-        try {
-            // Control executor to call MessageListener one by one.
-            if (executorQueueSize.get() < 1) {
-                final Message<T> msg = internalReceive(0, TimeUnit.MILLISECONDS);
-                if (msg != null) {
-                    executorQueueSize.incrementAndGet();
-                    if (SubscriptionType.Key_Shared == conf.getSubscriptionType()) {
-                        executorProvider.getExecutor(peekMessageKey(msg)).execute(() ->
-                                callMessageListener(msg));
-                    } else {
-                        getExternalExecutor(msg).execute(() -> {
-                            callMessageListener(msg);
-                        });
+        internalPinnedExecutor.execute(() -> {
+            try {
+                // Control executor to call MessageListener one by one.
+                if (executorQueueSize.get() < 1) {
+                    final Message<T> msg = internalReceive(0, TimeUnit.MILLISECONDS);
+                    if (msg != null) {
+                        executorQueueSize.incrementAndGet();

Review comment:
       After looking at the client a bit more, we certainly follow this paradigm frequently. It is possible that this `internalPinnedExecutor` might become a bit of a bottle neck. However, I think we can keep the design as is for now. I opened https://github.com/apache/pulsar/pull/13273 to clean up this code block a bit.




-- 
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] michaeljmarshall commented on a change in pull request #13023: Fix consume message order issue when use listener.

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



##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java
##########
@@ -916,30 +916,32 @@ private void doPendingBatchReceiveTask(Timeout timeout) {
     protected void triggerListener() {
         // Trigger the notification on the message listener in a separate thread to avoid blocking the networking
         // thread while the message processing happens
-        try {
-            // Control executor to call MessageListener one by one.
-            if (executorQueueSize.get() < 1) {
-                final Message<T> msg = internalReceive(0, TimeUnit.MILLISECONDS);
-                if (msg != null) {
-                    executorQueueSize.incrementAndGet();
-                    if (SubscriptionType.Key_Shared == conf.getSubscriptionType()) {
-                        executorProvider.getExecutor(peekMessageKey(msg)).execute(() ->
-                                callMessageListener(msg));
-                    } else {
-                        getExternalExecutor(msg).execute(() -> {
-                            callMessageListener(msg);
-                        });
+        internalPinnedExecutor.execute(() -> {
+            try {
+                // Control executor to call MessageListener one by one.
+                if (executorQueueSize.get() < 1) {
+                    final Message<T> msg = internalReceive(0, TimeUnit.MILLISECONDS);
+                    if (msg != null) {
+                        executorQueueSize.incrementAndGet();

Review comment:
       Here is the race condition. Lines 922 and 925. The non-synchronized get and subsequent update would explain messages processed out of order. By putting this on the same thread, `internalPinnedExecutor`, the race is no longer possible. However, I wonder if we want the extra thread switching here and if we want the `internalReceive` method called on the `internalPinnedExecutor` for _all_ method calls. It seems like we should have instead solved this data race.




-- 
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 merged pull request #13023: Fix consume message order issue when use listener.

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


   


-- 
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] dave2wave commented on pull request #13023: Fix consume message order issue when use listener.

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


   While this may have fixed an ordering bug. It has caused degraded performance when the client is consuming a topic at a high rate. The evidence is that the change moved work between threads and under high load `pulsar-client-internal-n-n` threads have increased load and be limited by cpu.
   
   Where are documents that explain the threading architecture?
   
   Also, I would like to understand why you think this fixed the ordering issue ...


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