sijie commented on a change in pull request #2400: Dead Letter Topic
URL: https://github.com/apache/incubator-pulsar/pull/2400#discussion_r211359011
##########
File path:
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java
##########
@@ -556,13 +618,90 @@ public synchronized void
redeliverUnacknowledgedMessages(Consumer consumer) {
@Override
public synchronized void redeliverUnacknowledgedMessages(Consumer
consumer, List<PositionImpl> positions) {
- positions.forEach(position ->
messagesToReplay.add(position.getLedgerId(), position.getEntryId()));
if (log.isDebugEnabled()) {
log.debug("[{}-{}] Redelivering unacknowledged messages for
consumer {}", name, consumer, positions);
}
+ if (maxRedeliveryCount > 0 && redeliveryTracker != null) {
+ for (PositionImpl position : positions) {
+ if (redeliveryTracker.incrementAndGetRedeliveryCount(position)
<= maxRedeliveryCount) {
+ messagesToReplay.add(position.getLedgerId(),
position.getEntryId());
+ } else {
+ messagesToDeadLetter.add(position);
+ }
+ }
+ if (messagesToDeadLetter.size() > 0) {
+ CountDownLatch latch = new
CountDownLatch(messagesToDeadLetter.size());
+ for (PositionImpl position : messagesToDeadLetter) {
+ cursor.asyncReadEntry(position, new
AsyncCallbacks.ReadEntryCallback() {
+ @Override
+ public void readEntryComplete(Entry entry, Object ctx)
{
+ if (entry == null) {
+ log.error("[{}-{}] Read an null entry from
cursor {}", name, consumer, position);
+ latch.countDown();
+ } else {
+ try {
+ ByteBuf headersAndPayload =
entry.getDataBuffer();
+ MessageImpl<byte[]> msg =
MessageImpl.deserialize(headersAndPayload);
+ headersAndPayload.retain();
+ msg.setReplicatedFrom("DLQ");
+ CompletableFuture<MessageId> future =
deadLetterTopicProducer.sendAsync(msg);
+ future.whenCompleteAsync((messageId,
error) -> {
+ if (error != null) {
+ log.error("[{}-{}] Fail to send
message to dead letter topic {} {} {}",
+ name, consumer,
deadLetterTopic, error.getMessage(), error);
+
messagesToReplay.add(position.getLedgerId(), position.getEntryId());
+ latch.countDown();
+ } else {
+ cursor.asyncDelete(position,
deleteCallback, position);
+ redeliveryTracker.remove(position);
+ latch.countDown();
+ }
+ });
+ } catch (Throwable t) {
+ log.error("[{}-{}] Failed to deserialize
message at {} {} {} {}", name, consumer,
+ entry.getPosition(),
entry.getLedgerId(), t.getMessage(), t);
+ cursor.asyncDelete(position,
deleteCallback, position);
+ redeliveryTracker.remove(position);
+ entry.release();
+ latch.countDown();
+ }
+ }
+ }
+ @Override
+ public void readEntryFailed(ManagedLedgerException
exception, Object ctx) {
+ log.error("[{}-{}] Read entries failed {} {}",
name, consumer, exception.getMessage(), exception);
+ messagesToReplay.add(position.getLedgerId(),
position.getEntryId());
+ latch.countDown();
+ }
+ }, null);
+ }
+ try {
Review comment:
I don't think we need to wait for the results here. because the logic has
been processed in the callbacks. right?
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
With regards,
Apache Git Services