diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java index 208292692a413..15f783be2b0cc 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java @@ -58,6 +58,7 @@ import org.apache.pulsar.client.impl.transaction.TransactionImpl; import org.apache.pulsar.client.util.ConsumerName; import org.apache.pulsar.client.util.ExecutorProvider; +import org.apache.pulsar.client.util.NoOpLock; import org.apache.pulsar.common.api.proto.CommandAck.AckType; import org.apache.pulsar.common.api.proto.CommandSubscribe; import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; @@ -96,7 +97,9 @@ public abstract class ConsumerBase extends HandlerState implements Consumer CONSUMER_EPOCH = AtomicLongFieldUpdater.newUpdater(ConsumerBase.class, "consumerEpoch"); @@ -163,6 +166,11 @@ protected ConsumerBase(PulsarClientImpl client, String topic, ConsumerConfigurat } else { this.batchReceivePolicy = BatchReceivePolicy.DEFAULT_POLICY; } + if (getSubType() == CommandSubscribe.SubType.Failover || getSubType() == CommandSubscribe.SubType.Exclusive) { + incomingQueueLock = new ReentrantLock(); + } else { + incomingQueueLock = new NoOpLock(); + } initReceiverQueueSize(); } @@ -840,8 +848,9 @@ protected boolean canEnqueueMessage(Message message) { protected boolean enqueueMessageAndCheckBatchReceive(Message message) { int messageSize = message.size(); - // synchronize redeliverUnacknowledgedMessages() - synchronized (incomingQueueLock) { + // synchronize redeliverUnacknowledgedMessages(). + incomingQueueLock.lock(); + try { if (isValidConsumerEpoch(message) && canEnqueueMessage(message) && incomingMessages.offer(message)) { // After we have enqueued the messages on `incomingMessages` queue, we cannot touch the message // instance anymore, since for pooled messages, this instance was possibly already been released @@ -850,6 +859,8 @@ protected boolean enqueueMessageAndCheckBatchReceive(Message message) { getMemoryLimitController().ifPresent(limiter -> limiter.forceReserveMemory(messageSize)); updateAutoScaleReceiverQueueHint(); } + } finally { + incomingQueueLock.unlock(); } return hasEnoughMessagesForBatchReceive(); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 8c2bf29f2a2bb..5734cff6792bd 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -1881,7 +1881,8 @@ public void redeliverUnacknowledgedMessages() { } int currentSize; - synchronized (incomingQueueLock) { + incomingQueueLock.lock(); + try { // we should increase epoch every time, because MultiTopicsConsumerImpl also increase it, // we need to keep both epochs the same if (conf.getSubscriptionType() == SubscriptionType.Failover @@ -1893,7 +1894,8 @@ public void redeliverUnacknowledgedMessages() { currentSize = incomingMessages.size(); clearIncomingMessages(); unAckedMessageTracker.clear(); - + } finally { + incomingQueueLock.unlock(); } // is channel is connected, we should send redeliver command to broker diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java index 90ddb9a3ea6ad..173886407e87c 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java @@ -666,7 +666,8 @@ private ConsumerConfigurationData getInternalConsumerConfig() { @Override public void redeliverUnacknowledgedMessages() { internalPinnedExecutor.execute(() -> { - synchronized (incomingQueueLock) { + incomingQueueLock.lock(); + try { CONSUMER_EPOCH.incrementAndGet(this); consumers.values().stream().forEach(consumer -> { consumer.redeliverUnacknowledgedMessages(); @@ -674,6 +675,8 @@ public void redeliverUnacknowledgedMessages() { }); clearIncomingMessages(); unAckedMessageTracker.clear(); + } finally { + incomingQueueLock.unlock(); } }); resumeReceivingFromPausedConsumersIfNeeded(); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/util/NoOpLock.java b/pulsar-client/src/main/java/org/apache/pulsar/client/util/NoOpLock.java new file mode 100644 index 0000000000000..ebe8d151d35d8 --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/util/NoOpLock.java @@ -0,0 +1,55 @@ +/** + * 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.client.util; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; + +public final class NoOpLock implements Lock { + public static final NoOpLock INSTANCE = new NoOpLock(); + + public void lock() { + } + + public void lockInterruptibly() throws InterruptedException { + if (Thread.interrupted()) { + throw new InterruptedException(); + } + } + + public boolean tryLock() { + return true; + } + + public boolean tryLock(long l, TimeUnit tu) throws InterruptedException { + if (Thread.interrupted()) { + throw new InterruptedException(); + } else { + return true; + } + } + + public void unlock() { + } + + public Condition newCondition() { + throw new UnsupportedOperationException(); + } +}