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

PIP-22: Dead Letter Topic #2508

Merged
merged 15 commits into from
Sep 14, 2018
Merged
Show file tree
Hide file tree
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 @@ -265,7 +265,8 @@ public SendMessageInfo sendMessages(final List<Entry> entries, SendListener list
if (i == (entries.size() - 1)) {
promise = writePromise;
}
ctx.write(Commands.newMessage(consumerId, messageId, metadataAndPayload), promise);
int redeliveryCount = subscription.getDispatcher().getRedeliveryTracker().getRedeliveryCount(PositionImpl.get(messageId.getLedgerId(), messageId.getEntryId()));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Instead of keeping a separate redelivery tracker, would it be possible to reuse the pendingAcks maps on the Consumer class?

pendingAcks is a map for which the key is a pair longs and the value is a pair of longs as well.

(ledgerId, entryId) --> (batchSize, none)

As you can see, the second long field in the value is currently unused. We could keep the counter there.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

After i check the pending acks in consumer, i recommend use redeliveryTracker to track the redelivery count:

First, in a subscription only have one redelivery tracker, pending acks is binding to consumer, we don't need to record redelivery count per consumer and if record it in consumer, will get a different behavior ismax redelivery count per consumer, but for now we defined the redelivery count per subscription.

Second, i think we need separate pending acks and redeliveryTracker, it's better to implement another redeliveryTracker.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

First, in a subscription only have one redelivery tracker, pending acks is binding to consumer, we don't need to record redelivery count per consumer and if record it in consumer, will get a different behavior ismax redelivery count per consumer, but for now we defined the redelivery count per subscription.

Sure, what I was thinking was more to store it in consumer pending acks and carrying it over to the new consumer when redelivering.

Second, i think we need separate pending acks and redeliveryTracker, it's better to implement another redeliveryTracker.

Sure, these are separate concepts. Though I think the current name pendingAcks is a bit misleading. What the map really tracks is "message that were pushed to this consumer and not acked".

My main concern with the new map is to have to potentially accumulate lots of items in the concurrent hash map, irrespective of whether the DLQ is being used or not. Especially, this might be mostly not the case in exclusive/failover subscriptions, where data is typically processed in order.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

My main concern with the new map is to have to potentially accumulate lots of items in the concurrent hash map,

I'm understand your concern, i think the tracker map is limited by maxUnackMessages, only put element into tracker map when trigger by redeliverUnackedMessages().

irrespective of whether the DLQ is being used or not.

I think we can optimize it next step, i think we can add a field in proto like enableRedeliveryTracker, broker just enable this mechanism when enableRedeliveryTracker is true.

Especially, this might be mostly not the case in exclusive/failover subscriptions, where data is typically processed in order.

Yes, i'm agree, i think we can only enable the counter tracking only on Shared subscriptions, mean that just in PersistentDispatcherMultipleConsumers.

ctx.write(Commands.newMessage(consumerId, messageId, redeliveryCount, metadataAndPayload), promise);
messageId.recycle();
messageIdBuilder.recycle();
entry.release();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,4 +70,6 @@ public interface Dispatcher {

void addUnAckedMessages(int unAckMessages);

RedeliveryTracker getRedeliveryTracker();

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,58 @@
/**
* 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;

import org.apache.bookkeeper.mledger.Position;

import java.util.List;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicInteger;

public class InMemoryRedeliveryTracker implements RedeliveryTracker {

private ConcurrentHashMap<Position, AtomicInteger> trackerCache = new ConcurrentHashMap<>(16);

@Override
public int incrementAndGetRedeliveryCount(Position position) {
trackerCache.putIfAbsent(position, new AtomicInteger(0));
return trackerCache.get(position).incrementAndGet();
}

@Override
public int getRedeliveryCount(Position position) {
return trackerCache.getOrDefault(position, new AtomicInteger(0)).get();
}

@Override
public void remove(Position position) {
trackerCache.remove(position);
}

@Override
public void removeBatch(List<Position> positions) {
if (positions != null) {
positions.forEach(this::remove);
}
}

@Override
public void clear() {
trackerCache.clear();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,36 @@
/**
* 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;

import org.apache.bookkeeper.mledger.Position;

import java.util.List;

public interface RedeliveryTracker {

int incrementAndGetRedeliveryCount(Position position);

int getRedeliveryCount(Position position);

void remove(Position position);

void removeBatch(List<Position> positions);

void clear();
}
Original file line number Diff line number Diff line change
@@ -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.broker.service;

import org.apache.bookkeeper.mledger.Position;

import java.util.List;

public class RedeliveryTrackerDisabled implements RedeliveryTracker {

public static final RedeliveryTrackerDisabled REDELIVERY_TRACKER_DISABLED = new RedeliveryTrackerDisabled();

private RedeliveryTrackerDisabled() {}

@Override
public int incrementAndGetRedeliveryCount(Position position) {
return 0;
}

@Override
public int getRedeliveryCount(Position position) {
return 0;
}

@Override
public void remove(Position position) {
// no-op
}

@Override
public void removeBatch(List<Position> positions) {
// no-op
}

@Override
public void clear() {
// no-op
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,8 @@
import org.apache.pulsar.broker.service.BrokerServiceException;
import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerBusyException;
import org.apache.pulsar.broker.service.Consumer;
import org.apache.pulsar.broker.service.RedeliveryTracker;
import org.apache.pulsar.broker.service.RedeliveryTrackerDisabled;
import org.apache.pulsar.broker.service.Subscription;
import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;
import org.apache.pulsar.utils.CopyOnWriteArrayList;
Expand All @@ -54,13 +56,15 @@ public class NonPersistentDispatcherMultipleConsumers extends AbstractDispatcher
private volatile int totalAvailablePermits = 0;

private final ServiceConfiguration serviceConfig;
private final RedeliveryTracker redeliveryTracker;

public NonPersistentDispatcherMultipleConsumers(NonPersistentTopic topic, Subscription subscription) {
this.topic = topic;
this.subscription = subscription;
this.name = topic.getName() + " / " + subscription.getName();
this.msgDrop = new Rate();
this.serviceConfig = topic.getBrokerService().pulsar().getConfiguration();
this.redeliveryTracker = RedeliveryTrackerDisabled.REDELIVERY_TRACKER_DISABLED;
}

@Override
Expand Down Expand Up @@ -178,6 +182,11 @@ public SubType getType() {
return SubType.Shared;
}

@Override
public RedeliveryTracker getRedeliveryTracker() {
return redeliveryTracker;
}

@Override
public void sendMessages(List<Entry> entries) {
Consumer consumer = TOTAL_AVAILABLE_PERMITS_UPDATER.get(this) > 0 ? getNextConsumer() : null;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,8 @@
import org.apache.pulsar.broker.admin.AdminResource;
import org.apache.pulsar.broker.service.AbstractDispatcherSingleActiveConsumer;
import org.apache.pulsar.broker.service.Consumer;
import org.apache.pulsar.broker.service.RedeliveryTracker;
import org.apache.pulsar.broker.service.RedeliveryTrackerDisabled;
import org.apache.pulsar.broker.service.Subscription;
import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;
import org.apache.pulsar.common.naming.TopicName;
Expand All @@ -40,6 +42,7 @@ public final class NonPersistentDispatcherSingleActiveConsumer extends AbstractD
private final Rate msgDrop;
private final Subscription subscription;
private final ServiceConfiguration serviceConfig;
private final RedeliveryTracker redeliveryTracker;

public NonPersistentDispatcherSingleActiveConsumer(SubType subscriptionType, int partitionIndex,
NonPersistentTopic topic, Subscription subscription) {
Expand All @@ -48,6 +51,7 @@ public NonPersistentDispatcherSingleActiveConsumer(SubType subscriptionType, int
this.subscription = subscription;
this.msgDrop = new Rate();
this.serviceConfig = topic.getBrokerService().pulsar().getConfiguration();
this.redeliveryTracker = RedeliveryTrackerDisabled.REDELIVERY_TRACKER_DISABLED;
}

@Override
Expand Down Expand Up @@ -117,6 +121,11 @@ public void consumerFlow(Consumer consumer, int additionalNumberOfMessages) {
// No-op
}

@Override
public RedeliveryTracker getRedeliveryTracker() {
return redeliveryTracker;
}

@Override
protected void scheduleReadOnActiveConsumer() {
// No-op
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,8 @@
import org.apache.pulsar.broker.service.Consumer;
import org.apache.pulsar.broker.service.Consumer.SendMessageInfo;
import org.apache.pulsar.broker.service.Dispatcher;
import org.apache.pulsar.broker.service.RedeliveryTracker;
import org.apache.pulsar.broker.service.InMemoryRedeliveryTracker;
import org.apache.pulsar.client.impl.Backoff;
import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;
import org.apache.pulsar.common.naming.TopicName;
Expand All @@ -69,6 +71,7 @@ public class PersistentDispatcherMultipleConsumers extends AbstractDispatcherMu

private CompletableFuture<Void> closeFuture = null;
private ConcurrentLongPairSet messagesToReplay;
private final RedeliveryTracker redeliveryTracker;

private boolean havePendingRead = false;
private boolean havePendingReplayRead = false;
Expand Down Expand Up @@ -97,6 +100,7 @@ public PersistentDispatcherMultipleConsumers(PersistentTopic topic, ManagedCurso
this.name = topic.getName() + " / " + Codec.decode(cursor.getName());
this.topic = topic;
this.messagesToReplay = new ConcurrentLongPairSet(512, 2);
this.redeliveryTracker = new InMemoryRedeliveryTracker();
this.readBatchSize = MaxReadBatchSize;
this.maxUnackedMessages = topic.getBrokerService().pulsar().getConfiguration()
.getMaxUnackedMessagesPerSubscription();
Expand Down Expand Up @@ -556,7 +560,10 @@ 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()));
positions.forEach(position -> {
messagesToReplay.add(position.getLedgerId(), position.getEntryId());
redeliveryTracker.incrementAndGetRedeliveryCount(position);
});
if (log.isDebugEnabled()) {
log.debug("[{}-{}] Redelivering unacknowledged messages for consumer {}", name, consumer, positions);
}
Expand Down Expand Up @@ -624,5 +631,10 @@ public DispatchRateLimiter getDispatchRateLimiter() {
return dispatchRateLimiter;
}

@Override
public RedeliveryTracker getRedeliveryTracker() {
return redeliveryTracker;
}

private static final Logger log = LoggerFactory.getLogger(PersistentDispatcherMultipleConsumers.class);
}
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,8 @@
import org.apache.pulsar.broker.service.AbstractDispatcherSingleActiveConsumer;
import org.apache.pulsar.broker.service.Consumer;
import org.apache.pulsar.broker.service.Dispatcher;
import org.apache.pulsar.broker.service.RedeliveryTracker;
import org.apache.pulsar.broker.service.RedeliveryTrackerDisabled;
import org.apache.pulsar.client.impl.Backoff;
import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;
import org.apache.pulsar.common.naming.TopicName;
Expand All @@ -62,6 +64,8 @@ public final class PersistentDispatcherSingleActiveConsumer extends AbstractDisp
private final ServiceConfiguration serviceConfig;
private ScheduledFuture<?> readOnActiveConsumerTask = null;

private final RedeliveryTracker redeliveryTracker;

public PersistentDispatcherSingleActiveConsumer(ManagedCursor cursor, SubType subscriptionType, int partitionIndex,
PersistentTopic topic) {
super(subscriptionType, partitionIndex, topic.getName());
Expand All @@ -72,6 +76,7 @@ public PersistentDispatcherSingleActiveConsumer(ManagedCursor cursor, SubType su
this.readBatchSize = MaxReadBatchSize;
this.serviceConfig = topic.getBrokerService().pulsar().getConfiguration();
this.dispatchRateLimiter = null;
this.redeliveryTracker = RedeliveryTrackerDisabled.REDELIVERY_TRACKER_DISABLED;
}

protected void scheduleReadOnActiveConsumer() {
Expand Down Expand Up @@ -307,6 +312,7 @@ private synchronized void internalRedeliverUnacknowledgedMessages(Consumer consu
@Override
public void redeliverUnacknowledgedMessages(Consumer consumer, List<PositionImpl> positions) {
// We cannot redeliver single messages to single consumers to preserve ordering.
positions.forEach(redeliveryTracker::incrementAndGetRedeliveryCount);
redeliverUnacknowledgedMessages(consumer);
}

Expand Down Expand Up @@ -485,5 +491,10 @@ public void addUnAckedMessages(int unAckMessages) {
// No-op
}

@Override
public RedeliveryTracker getRedeliveryTracker() {
return redeliveryTracker;
}

private static final Logger log = LoggerFactory.getLogger(PersistentDispatcherSingleActiveConsumer.class);
}
Original file line number Diff line number Diff line change
Expand Up @@ -195,6 +195,7 @@ public void acknowledgeMessage(List<Position> positions, AckType ackType, Map<St
log.debug("[{}][{}] Individual acks on {}", topicName, subName, positions);
}
cursor.asyncDelete(positions, deleteCallback, positions);
dispatcher.getRedeliveryTracker().removeBatch(positions);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This will not clean up the redelivery tracker when doing a "cumulative-ack". It should be also cleaned at line 192, though for cumulative acks there might be a sequence of entries to clean-up and the concurrent hash map might not be a good fit (since we'd need sorted map).

Given that, and can we initially only enable the counter only on Shared subscriptions?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Given that, and can we initially only enable the counter only on Shared subscriptions?

I'm agree with your comment(Especially, this might be mostly not the case in exclusive/failover subscriptions, where data is typically processed in order).

This will not clean up the redelivery tracker when doing a "cumulative-ack". It should be also cleaned at line 192, though for cumulative acks there might be a sequence of entries to clean-up and the concurrent hash map might not be a good fit (since we'd need sorted map).

If we just enable the redelivery tracker in share mode, we don't need a sorted map, from client can't do cumulative-ack in share mode.

}

if (topic.getManagedLedger().isTerminated() && cursor.getNumberOfEntriesInBacklog() == 0) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -178,7 +178,7 @@ public CompletableFuture<Void> closeAsync() {
}

@Override
void messageReceived(MessageIdData messageId, ByteBuf headersAndPayload, ClientCnx cnx) {
void messageReceived(MessageIdData messageId, int redeliveryCount, ByteBuf headersAndPayload, ClientCnx cnx) {
if (log.isDebugEnabled()) {
log.debug("[{}][{}] Received raw message: {}/{}/{}", topic, subscription,
messageId.getEntryId(), messageId.getLedgerId(), messageId.getPartition());
Expand Down
Loading