-
Notifications
You must be signed in to change notification settings - Fork 3.6k
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
Changes from 13 commits
9a61230
7c0092a
41fb24a
92a43a9
dde79b7
3c6a0c8
08a012d
793fc1f
50ad8f7
b3a17e7
e5f88cf
5ba2168
903c4a0
8bccab9
df07259
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
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 |
---|---|---|
|
@@ -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); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
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).
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) { | ||
|
There was a problem hiding this comment.
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 theConsumer
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.There was a problem hiding this comment.
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 is
max 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.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sure, what I was thinking was more to store it in consumer pending acks and carrying it over to the new consumer when redelivering.
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.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'm understand your concern, i think the tracker map is limited by maxUnackMessages, only put element into tracker map when trigger by redeliverUnackedMessages().
I think we can optimize it next step, i think we can add a field in proto like
enableRedeliveryTracker
, broker just enable this mechanism whenenableRedeliveryTracker
is true.Yes, i'm agree, i think we can only enable the counter tracking only on Shared subscriptions, mean that just in PersistentDispatcherMultipleConsumers.