Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Fix locking for ConsumerImpl when creating deadLetterProducer. #9166

Merged
merged 1 commit into from
Jan 18, 2021
Merged
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -166,7 +166,7 @@ public class ConsumerImpl<T> extends ConsumerBase<T> implements ConnectionHandle

private final DeadLetterPolicy deadLetterPolicy;

private Producer<T> deadLetterProducer;
private volatile Producer<T> deadLetterProducer;

private volatile Producer<T> retryLetterProducer;
private final ReadWriteLock createProducerLock = new ReentrantReadWriteLock();
Expand Down Expand Up @@ -702,8 +702,8 @@ protected CompletableFuture<Void> doReconsumeLater(Message<?> message, AckType a
processPossibleToDLQ((MessageIdImpl)messageId);
if (deadLetterProducer == null) {
try {
createProducerLock.writeLock().lock();
if (deadLetterProducer == null) {
createProducerLock.writeLock().lock();
deadLetterProducer = client.newProducer(schema)
.topic(this.deadLetterPolicy
.getDeadLetterTopic())
Expand All @@ -712,9 +712,9 @@ protected CompletableFuture<Void> doReconsumeLater(Message<?> message, AckType a
}
} catch (Exception e) {
log.error("Create dead letter producer exception with topic: {}", deadLetterPolicy.getDeadLetterTopic(), e);
} finally {
} finally {
createProducerLock.writeLock().unlock();
}
}
}
if (deadLetterProducer != null) {
propertiesMap.put(RetryMessageUtil.SYSTEM_PROPERTY_REAL_TOPIC, originTopicNameStr);
Expand Down Expand Up @@ -1816,12 +1816,17 @@ private boolean processPossibleToDLQ(MessageIdImpl messageId) {
if (deadLetterMessages != null) {
if (deadLetterProducer == null) {
try {
deadLetterProducer = client.newProducer(schema)
.topic(this.deadLetterPolicy.getDeadLetterTopic())
.blockIfQueueFull(false)
.create();
createProducerLock.writeLock().lock();
if (deadLetterProducer == null) {
deadLetterProducer = client.newProducer(schema)
.topic(this.deadLetterPolicy.getDeadLetterTopic())
.blockIfQueueFull(false)
.create();
}
} catch (Exception e) {
log.error("Create dead letter producer exception with topic: {}", deadLetterPolicy.getDeadLetterTopic(), e);
} finally {
createProducerLock.writeLock().unlock();
}
}
if (deadLetterProducer != null) {
Expand Down