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/04/05 09:13:18 UTC

[GitHub] [pulsar] leizhiyuan opened a new pull request, #14641: fix: fix ArrayIndexOutOfBoundsException when concurrent

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

   
   ### Motivation
   
   
   ```
   WARN ] 2022-03-10 17:27:26.976 [pulsar-io-36-20] ServerCnx - [/127.0.0.1:55561] Got exception java.lang.ArrayIndexOutOfBoundsException: 1
   	at java.util.concurrent.CopyOnWriteArrayList.get(CopyOnWriteArrayList.java:388)
   	at java.util.concurrent.CopyOnWriteArrayList.get(CopyOnWriteArrayList.java:397)
   	at org.apache.pulsar.broker.service.AbstractDispatcherMultipleConsumers.getNextConsumer(AbstractDispatcherMultipleConsumers.java:121)
   	at org.apache.pulsar.broker.service.nonpersistent.NonPersistentDispatcherMultipleConsumers.sendMessages(NonPersistentDispatcherMultipleConsumers.java:192)
   	at org.apache.pulsar.broker.service.nonpersistent.NonPersistentTopic.lambda$publishMessage$1(NonPersistentTopic.java:182)
   	at org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap$Section.forEach(ConcurrentOpenHashMap.java:387)
   	at org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap.forEach(ConcurrentOpenHashMap.java:159)
   	at org.apache.pulsar.broker.service.nonpersistent.NonPersistentTopic.publishMessage(NonPersistentTopic.java:176)
   	at org.apache.pulsar.broker.service.Producer.publishMessageToTopic(Producer.java:210)
   	at org.apache.pulsar.broker.service.Producer.publishMessage(Producer.java:149)
   	at org.apache.pulsar.broker.service.ServerCnx.handleSend(ServerCnx.java:1321)
   	at org.apache.pulsar.common.protocol.PulsarDecoder.channelRead(PulsarDecoder.java:232)
   	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
   	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
   	at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)
   	at io.netty.handler.flow.FlowControlHandler.dequeue(FlowControlHandler.java:200)
   ```
   
   ### Modifications
   
    assume consumerList is 1, if we use non persistent topic, and two producers publish messages, there will be two threads (pulsar-io)invoke this method, if one changed to 1, the other will faild.
   
   currentConsumerRoundRobinIndex will be 1 and 
   ```
           int currentRoundRobinConsumerPriority = consumerList.get(currentConsumerRoundRobinIndex).getPriorityLevel();
   
   ```
   will   throw ArrayIndexOutOfBoundsException
   
   reproduce:
   
   
   ```
   /**
    * Licensed to the Apache Software Foundation (ASF) under one
    * or more contributor license agreements.  See the NOTICE file
    * distributed with this work for additional information
    * regarding copyright ownership.  The ASF licenses this file
    * to you under the Apache License, Version 2.0 (the
    * "License"); you may not use this file except in compliance
    * with the License.  You may obtain a copy of the License at
    *
    * http://www.apache.org/licenses/LICENSE-2.0
    *
    * Unless required by applicable law or agreed to in writing,
    * software distributed under the License is distributed on an
    * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    * KIND, either express or implied.  See the License for the
    * specific language governing permissions and limitations
    * under the License.
    */
   
   package org.apache.pulsar.broker.service.nonpersistent;
   
   import java.nio.charset.StandardCharsets;
   import java.util.UUID;
   import java.util.concurrent.TimeUnit;
   import lombok.Cleanup;
   import lombok.SneakyThrows;
   import org.apache.pulsar.broker.service.BrokerTestBase;
   import org.apache.pulsar.client.api.Consumer;
   import org.apache.pulsar.client.api.Message;
   import org.apache.pulsar.client.api.Producer;
   import org.apache.pulsar.client.api.PulsarClient;
   import org.apache.pulsar.client.api.SubscriptionType;
   import org.testng.annotations.AfterMethod;
   import org.testng.annotations.BeforeMethod;
   import org.testng.annotations.Test;
   
   @Test(groups = "broker")
   public class NonPersistentTopicCocurrentTest extends BrokerTestBase {
   
       @BeforeMethod(alwaysRun = true)
       @Override
       protected void setup() throws Exception {
           super.baseSetup();
       }
   
       @AfterMethod(alwaysRun = true)
       @Override
       protected void cleanup() throws Exception {
           super.internalCleanup();
       }
   
       @SneakyThrows
       @Test
       public void test() {
           final String topicName = "non-persistent://prop/ns-abc/topic" + UUID.randomUUID();
           final String subName = "non-persistent";
           PulsarClient pulsarClient1 = newPulsarClient(lookupUrl.toString(), 0);
           @Cleanup
           Consumer<byte[]> consumer = pulsarClient1.newConsumer().topic(topicName)
                   .subscriptionName(subName).subscriptionType(SubscriptionType.Shared).subscribe();
           PulsarClient pulsarClient2 = newPulsarClient(lookupUrl.toString(), 0);
           @Cleanup
           Producer<byte[]> producer1 = pulsarClient2.newProducer().topic(topicName).create();
   
           PulsarClient pulsarClient3 = newPulsarClient(lookupUrl.toString(), 0);
           @Cleanup
           Producer<byte[]> producer2 = pulsarClient3.newProducer().topic(topicName).create();
   
           Thread t1 = new Thread(new Runnable() {
               @Override
               public void run() {
                   for (int i = 0; i < 10000; i++) {
                       producer1.sendAsync("msg".getBytes(StandardCharsets.UTF_8));
                   }
               }
           });
   
           Thread t2 = new Thread(new Runnable() {
               @Override
               public void run() {
                   for (int i = 0; i < 10000; i++) {
                       producer2.sendAsync("msg".getBytes(StandardCharsets.UTF_8));
                   }
               }
           });
           t1.start();
           t2.start();
   
           while (true) {
               Message<byte[]> message = consumer.receive(1, TimeUnit.SECONDS);
               if (message == null) {
                   break;
               } else {
                   System.out.println(message);
               }
           }
       }
   }
   
   
   ```
   
   run sometimes, you will see in console
   
   <img width="1642" alt="image" src="https://user-images.githubusercontent.com/2684384/158001771-1e2303fb-13e9-4f72-a5f7-af72ddb0c26d.png">
   
   
   
   
   
   
   
   


-- 
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 pull request #14641: fix: fix ArrayIndexOutOfBoundsException when concurrent

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

   @eolivelli Please help review this PR again


-- 
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 pull request #14641: fix: fix ArrayIndexOutOfBoundsException when concurrent

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

   @eolivelli PTAL. This PR is blocked for a long time.


-- 
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] wolfstudy commented on pull request #14641: fix: fix ArrayIndexOutOfBoundsException when concurrent

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

   /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 #14641: fix: fix ArrayIndexOutOfBoundsException when concurrent

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

   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] BewareMyPower commented on a diff in pull request #14641: fix: fix ArrayIndexOutOfBoundsException when concurrent

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherMultipleConsumers.java:
##########
@@ -119,17 +119,13 @@ public Consumer getNextConsumer() {
             return null;
         }
 
-        if (currentConsumerRoundRobinIndex >= consumerList.size()) {

Review Comment:
   @eolivelli Could you take a look again? I see this PR only adds a `synchronized` keyword now.



-- 
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] leizhiyuan commented on pull request #14641: fix: fix ArrayIndexOutOfBoundsException when concurrent

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

   > LGTM, but i'm not sure if we add `synchronized` will affect the performance?
   
   we have tested whith load test in our produciton env。


-- 
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] leizhiyuan commented on pull request #14641: fix: fix ArrayIndexOutOfBoundsException when concurrent

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

   /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] wolfstudy commented on pull request #14641: fix: fix ArrayIndexOutOfBoundsException when concurrent

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

   ping @eolivelli PTAL again, 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] congbobo184 commented on pull request #14641: [fix][broker] Fix ArrayIndexOutOfBoundsException due to race condition

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

   @leizhiyuan   hi, I move this PR to release/2.9.5, if you have any questions, please ping me. 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] michaeljmarshall commented on pull request #14641: [fix][broker] Fix ArrayIndexOutOfBoundsException due to race condition

Posted by "michaeljmarshall (via GitHub)" <gi...@apache.org>.
michaeljmarshall commented on PR #14641:
URL: https://github.com/apache/pulsar/pull/14641#issuecomment-1610272197

   As discussed on the mailing list https://lists.apache.org/thread/w4jzk27qhtosgsz7l9bmhf1t7o9mxjhp, there is no plan to release 2.9.6, so I am going to remove the release/2.9.6 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] tisonkun commented on pull request #14641: [fix][broker] Fix ArrayIndexOutOfBoundsException due to race condition

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

   /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 #14641: fix: fix ArrayIndexOutOfBoundsException when concurrent

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

   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] leizhiyuan closed pull request #14641: fix: fix ArrayIndexOutOfBoundsException when concurrent

Posted by GitBox <gi...@apache.org>.
leizhiyuan closed pull request #14641: fix: fix ArrayIndexOutOfBoundsException when concurrent
URL: https://github.com/apache/pulsar/pull/14641


-- 
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 #14641: fix: fix ArrayIndexOutOfBoundsException when concurrent

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

   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