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][broker] Fix recentlyJoinedConsumers to address the out-of-order issue #20179

Closed
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 @@ -38,6 +38,14 @@ private PositionImplRecyclable(Handle<PositionImplRecyclable> recyclerHandle) {
this.recyclerHandle = recyclerHandle;
}

public void setLedgerId(final long ledgerId) {
this.ledgerId = ledgerId;
}

public void setEntryId(final long entryId) {
this.entryId = entryId;
}

public static PositionImplRecyclable create() {
return RECYCLER.get();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -134,7 +134,7 @@ public class Consumer {

private static final double avgPercent = 0.9;
private boolean preciseDispatcherFlowControl;
private PositionImpl readPositionWhenJoining;
private String lastSentPositionsWhenJoiningString;
private final String clientAddress; // IP address only, no port number included
private final MessageId startMessageId;
private final boolean isAcknowledgmentAtBatchIndexLevelEnabled;
Expand Down Expand Up @@ -867,8 +867,8 @@ public ConsumerStatsImpl getStats() {
stats.unackedMessages = unackedMessages;
stats.blockedConsumerOnUnackedMsgs = blockedConsumerOnUnackedMsgs;
stats.avgMessagesPerEntry = getAvgMessagesPerEntry();
if (readPositionWhenJoining != null) {
stats.readPositionWhenJoining = readPositionWhenJoining.toString();
if (lastSentPositionsWhenJoiningString != null) {
stats.lastSentPositionsWhenJoining = lastSentPositionsWhenJoiningString;
}
return stats;
}
Expand Down Expand Up @@ -1088,8 +1088,8 @@ public boolean isPreciseDispatcherFlowControl() {
return preciseDispatcherFlowControl;
}

public void setReadPositionWhenJoining(PositionImpl readPositionWhenJoining) {
this.readPositionWhenJoining = readPositionWhenJoining;
public void setLastSentPositionsWhenJoiningString(String lastSentPositionsWhenJoiningString) {
this.lastSentPositionsWhenJoiningString = lastSentPositionsWhenJoiningString;
}

public int getMaxUnackedMessages() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1188,7 +1188,11 @@ public ManagedCursor getCursor() {
}

protected int getStickyKeyHash(Entry entry) {
return StickyKeyConsumerSelector.makeStickyKeyHash(peekStickyKey(entry.getDataBuffer()));
return getStickyKeyHash(peekStickyKey(entry.getDataBuffer()));
}

protected int getStickyKeyHash(byte[] stickyKey) {
return StickyKeyConsumerSelector.makeStickyKeyHash(stickyKey);
}

private static final Logger log = LoggerFactory.getLogger(PersistentDispatcherMultipleConsumers.class);
Expand Down

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -1186,11 +1186,19 @@ public SubscriptionStatsImpl getStats(Boolean getPreciseBacklog, boolean subscri
subStats.allowOutOfOrderDelivery = keySharedDispatcher.isAllowOutOfOrderDelivery();
subStats.keySharedMode = keySharedDispatcher.getKeySharedMode().toString();

LinkedHashMap<Consumer, PositionImpl> recentlyJoinedConsumers = keySharedDispatcher
LinkedHashMap<Consumer, PersistentStickyKeyDispatcherMultipleConsumers.LastSentPositions>
Copy link
Contributor

Choose a reason for hiding this comment

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

this is a code smell to refer to this implementation class PersistentStickyKeyDispatcherMultipleConsumers, maybe we should move the LastSentPositions to a top level public class

recentlyJoinedConsumers = keySharedDispatcher
.getRecentlyJoinedConsumers();
if (recentlyJoinedConsumers != null && recentlyJoinedConsumers.size() > 0) {
recentlyJoinedConsumers.forEach((k, v) -> {
subStats.consumersAfterMarkDeletePosition.put(k.consumerName(), v.toString());
// Dispatchers allows same name consumers
final StringBuilder stringBuilder = new StringBuilder();
Copy link
Contributor

Choose a reason for hiding this comment

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

could we use a struct instead of a string ? (you can use a 'record' now, but as this fix should be cherrypick a plain old java object works better)
strings are error prone in code and also less efficient

stringBuilder.append("consumerName=").append(k.consumerName())
.append(", consumerId=").append(k.consumerId());
if (k.cnx() != null) {
stringBuilder.append(", address=").append(k.cnx().clientAddress());
}
subStats.recentlyJoinedConsumers.put(stringBuilder.toString(), v.toPositionSetString());
});
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3286,43 +3286,48 @@ public void testGetTtlDurationDefaultInSeconds() throws Exception {
}

@Test
public void testGetReadPositionWhenJoining() throws Exception {
final String topic = "persistent://prop-xyz/ns1/testGetReadPositionWhenJoining-" + UUID.randomUUID().toString();
public void testGetLastSentPositionsWhenJoining() throws Exception {
final String topic = "persistent://prop-xyz/ns1/testGetLastSentPositionsWhenJoining-" + UUID.randomUUID().toString();
final String subName = "my-sub";
@Cleanup
Producer<byte[]> producer = pulsarClient.newProducer()
.topic(topic)
.enableBatching(false)
.create();
@Cleanup
Consumer<byte[]> consumer1 = pulsarClient.newConsumer()
.topic(topic)
.consumerName("c1")
.subscriptionType(SubscriptionType.Key_Shared)
.subscriptionName(subName)
.subscribe();

final int messages = 10;
MessageIdImpl messageId = null;
for (int i = 0; i < messages; i++) {
messageId = (MessageIdImpl) producer.send(("Hello Pulsar - " + i).getBytes());
}

List<Consumer<byte[]>> consumers = new ArrayList<>();
for (int i = 0; i < 2; i++) {
Consumer<byte[]> consumer = pulsarClient.newConsumer()
.topic(topic)
.subscriptionType(SubscriptionType.Key_Shared)
.subscriptionName(subName)
.subscribe();
consumers.add(consumer);
for (int i = 0; i < messages; i++) {
assertNotNull(consumer1.receive(100, TimeUnit.MILLISECONDS));
}

@Cleanup
Consumer<byte[]> consumer2 = pulsarClient.newConsumer()
.topic(topic)
.consumerName("c2")
.subscriptionType(SubscriptionType.Key_Shared)
.subscriptionName(subName)
.subscribe();

TopicStats stats = admin.topics().getStats(topic);
Assert.assertEquals(stats.getSubscriptions().size(), 1);
SubscriptionStats subStats = stats.getSubscriptions().get(subName);
Assert.assertNotNull(subStats);
Assert.assertEquals(subStats.getConsumers().size(), 2);
ConsumerStats consumerStats = subStats.getConsumers().get(0);
Assert.assertEquals(consumerStats.getReadPositionWhenJoining(),
PositionImpl.get(messageId.getLedgerId(), messageId.getEntryId() + 1).toString());

for (Consumer<byte[]> consumer : consumers) {
consumer.close();
}
ConsumerStats consumer2Stats = subStats.getConsumers().stream().filter(s -> s.getConsumerName().equals(consumer2.getConsumerName())).findFirst().get();
Assert.assertEquals(consumer2Stats.getLastSentPositionsWhenJoining(),
Set.of(PositionImpl.get(messageId.getLedgerId(), messageId.getEntryId())).toString());
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -154,7 +154,9 @@ void shutdown() throws Exception {
executor.shutdownNow();

for (int i = 0; i < BROKER_COUNT; i++) {
pulsarAdmins[i].close();
if (pulsarAdmins[i] != null) {
pulsarAdmins[i].close();
}
if (pulsarServices[i] != null) {
pulsarServices[i].close();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -116,7 +116,7 @@ public void testConsumersAfterMarkDelete() throws PulsarClientException, PulsarA
TopicStats stats = admin.topics().getStats(topicName);
Assert.assertEquals(stats.getSubscriptions().size(), 1);
Assert.assertEquals(stats.getSubscriptions().entrySet().iterator().next().getValue()
.getConsumersAfterMarkDeletePosition().size(), 1);
.getRecentlyJoinedConsumers().size(), 1);

consumer1.close();
consumer2.close();
Expand Down
Loading