Skip to content

Commit

Permalink
Expose consumer names after the mark delete position for the Key_Shar…
Browse files Browse the repository at this point in the history
…ed subscription (apache#8545)

### Motivation

1. Expose consumer names after the mark delete position for the Key_Shared subscription.
2. Remove the consumer from the recenlyJoinedConsumer depends on the valid next position of the next position. Previously, we use the position.nextPosition to decide to remove the consumer from the recenlyJoinedConsumer but this will lead to consumers can't be deleted property. For example, if ledger rollover and the mark delete position is the last position of the old ledger and the max read position is the first position of the new ledger, In this situation, we should remove the consumer from the recenlyJoinedConsumer but in fact, it will stay in the recenlyJoinedConsumer because the max read position always greater than the `markDeletePosition.nextPosition`. 

So we should get the valid next position for the mark delete position.

Related to apache#8499
  • Loading branch information
codelipenghui authored and flowchartsman committed Nov 17, 2020
1 parent 5b07eca commit 7b56d4c
Show file tree
Hide file tree
Showing 7 changed files with 153 additions and 17 deletions.
8 changes: 8 additions & 0 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -194,6 +194,7 @@ flexible messaging model and an intuitive client API.</description>
<skyscreamer.version>1.5.0</skyscreamer.version>
<confluent.version>5.2.2</confluent.version>
<objenesis.version>3.1</objenesis.version>
<awaitility.version>4.0.2</awaitility.version>

<!-- Plugin dependencies -->
<protobuf-maven-plugin.version>0.6.1</protobuf-maven-plugin.version>
Expand Down Expand Up @@ -248,6 +249,13 @@ flexible messaging model and an intuitive client API.</description>
</exclusions>
</dependency>

<dependency>
<groupId>org.awaitility</groupId>
<artifactId>awaitility</artifactId>
<version>${awaitility.version}</version>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.mockito</groupId>
<artifactId>mockito-core</artifactId>
Expand Down
6 changes: 6 additions & 0 deletions pulsar-broker/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -169,6 +169,12 @@
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.awaitility</groupId>
<artifactId>awaitility</artifactId>
<scope>test</scope>
</dependency>

<!-- functions related dependencies (end) -->

<dependency>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
import org.apache.bookkeeper.mledger.Entry;
import org.apache.bookkeeper.mledger.ManagedCursor;
import org.apache.bookkeeper.mledger.Position;
import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
import org.apache.bookkeeper.mledger.impl.PositionImpl;
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.broker.service.BrokerServiceException;
Expand Down Expand Up @@ -325,15 +326,18 @@ && removeConsumersFromRecentJoinedConsumers()) {

private boolean removeConsumersFromRecentJoinedConsumers() {
Iterator<Map.Entry<Consumer, PositionImpl>> itr = recentlyJoinedConsumers.entrySet().iterator();
PositionImpl mdp = (PositionImpl) cursor.getMarkDeletedPosition();
boolean hasConsumerRemovedFromTheRecentJoinedConsumers = false;
while (itr.hasNext()) {
Map.Entry<Consumer, PositionImpl> entry = itr.next();
if (entry.getValue().compareTo(mdp) <= 0) {
itr.remove();
hasConsumerRemovedFromTheRecentJoinedConsumers = true;
} else {
break;
PositionImpl mdp = (PositionImpl) cursor.getMarkDeletedPosition();
if (mdp != null) {
PositionImpl nextPositionOfTheMarkDeletePosition = ((ManagedLedgerImpl)cursor.getManagedLedger()).getNextValidPosition(mdp);
while (itr.hasNext()) {
Map.Entry<Consumer, PositionImpl> entry = itr.next();
if (entry.getValue().compareTo(nextPositionOfTheMarkDeletePosition) <= 0) {
itr.remove();
hasConsumerRemovedFromTheRecentJoinedConsumers = true;
} else {
break;
}
}
}
return hasConsumerRemovedFromTheRecentJoinedConsumers;
Expand Down Expand Up @@ -361,6 +365,10 @@ protected Set<? extends Position> asyncReplayEntries(Set<? extends Position> pos
return cursor.asyncReplayEntries(positions, this, ReadType.Replay, true);
}

public LinkedHashMap<Consumer, PositionImpl> getRecentlyJoinedConsumers() {
return recentlyJoinedConsumers;
}

private static final Logger log = LoggerFactory.getLogger(PersistentStickyKeyDispatcherMultipleConsumers.class);

}
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import com.google.common.base.MoreObjects;

import java.util.Collections;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
Expand Down Expand Up @@ -933,6 +934,15 @@ public SubscriptionStats getStats(Boolean getPreciseBacklog) {
subStats.msgRateExpired = expiryMonitor.getMessageExpiryRate();
subStats.isReplicated = isReplicated();
subStats.isDurable = cursor.isDurable();
if (getType() == SubType.Key_Shared && dispatcher instanceof PersistentStickyKeyDispatcherMultipleConsumers) {
LinkedHashMap<Consumer, PositionImpl> recentlyJoinedConsumers =
((PersistentStickyKeyDispatcherMultipleConsumers) dispatcher).getRecentlyJoinedConsumers();
if (recentlyJoinedConsumers != null && recentlyJoinedConsumers.size() > 0) {
recentlyJoinedConsumers.forEach((k, v) -> {
subStats.consumersAfterMarkDeletePosition.put(k.consumerName(), v.toString());
});
}
}
return subStats;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@
import org.apache.pulsar.common.policies.data.TopicStats;
import org.apache.pulsar.common.policies.data.SubscriptionStats;
import org.apache.pulsar.common.util.FutureUtil;
import org.awaitility.Awaitility;
import org.eclipse.jetty.util.BlockingArrayQueue;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -493,15 +494,15 @@ public void testUnackedCountWithRedeliveries() throws Exception {
consumer1.acknowledge(msgId);
}

TopicStats stats = admin.topics().getStats(topicName);

// Unacked messages count should be 0 for both consumers at this point
SubscriptionStats subStats = stats.subscriptions.get(subName);
assertEquals(subStats.msgBacklog, 0);

for (ConsumerStats cs : subStats.consumers) {
assertEquals(cs.unackedMessages, 0);
}
Awaitility.await().atMost(3, TimeUnit.SECONDS).untilAsserted(() -> {
TopicStats stats = admin.topics().getStats(topicName);
// Unacked messages count should be 0 for both consumers at this point
SubscriptionStats subStats = stats.subscriptions.get(subName);
assertEquals(subStats.msgBacklog, 0);
for (ConsumerStats cs : subStats.consumers) {
assertEquals(cs.unackedMessages, 0);
}
});

producer.close();
consumer1.close();
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,94 @@
/**
* 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.stats;

import lombok.Cleanup;
import lombok.extern.slf4j.Slf4j;
import org.apache.pulsar.client.admin.PulsarAdminException;
import org.apache.pulsar.client.api.Consumer;
import org.apache.pulsar.client.api.Producer;
import org.apache.pulsar.client.api.ProducerConsumerBase;
import org.apache.pulsar.client.api.PulsarClientException;
import org.apache.pulsar.client.api.SubscriptionType;
import org.apache.pulsar.common.policies.data.TopicStats;
import org.testng.Assert;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;

import java.util.UUID;

@Slf4j
public class SubscriptionStatsTest extends ProducerConsumerBase {

@BeforeClass
@Override
protected void setup() throws Exception {
super.internalSetup();
super.producerBaseSetup();
}

@AfterClass
@Override
protected void cleanup() throws Exception {
super.internalCleanup();
}

@Test
public void testConsumersAfterMarkDelete() throws PulsarClientException, PulsarAdminException {
final String topicName = "persistent://my-property/my-ns/testConsumersAfterMarkDelete-"
+ UUID.randomUUID().toString();
final String subName = "my-sub";

Consumer<byte[]> consumer1 = pulsarClient.newConsumer()
.topic(topicName)
.receiverQueueSize(10)
.subscriptionName(subName)
.subscriptionType(SubscriptionType.Key_Shared)
.subscribe();

Producer<byte[]> producer = pulsarClient.newProducer()
.topic(topicName)
.create();

final int messages = 100;
for (int i = 0; i < messages; i++) {
producer.send(String.valueOf(i).getBytes());
}

// Receive by do not ack the message, so that the next consumer can added to the recentJoinedConsumer of the dispatcher.
consumer1.receive();

Consumer<byte[]> consumer2 = pulsarClient.newConsumer()
.topic(topicName)
.receiverQueueSize(10)
.subscriptionName(subName)
.subscriptionType(SubscriptionType.Key_Shared)
.subscribe();

TopicStats stats = admin.topics().getStats(topicName);
Assert.assertEquals(stats.subscriptions.size(), 1);
Assert.assertEquals(stats.subscriptions.entrySet().iterator().next().getValue()
.consumersAfterMarkDeletePosition.size(), 1);

consumer1.close();
consumer2.close();
producer.close();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,11 @@
import static com.google.common.base.Preconditions.checkNotNull;

import com.google.common.collect.Lists;

import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;

import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;

/**
Expand Down Expand Up @@ -91,8 +95,12 @@ public class SubscriptionStats {
/** Mark that the subscription state is kept in sync across different regions. */
public boolean isReplicated;

/** This is for Key_Shared subscription to get the recentJoinedConsumers in the Key_Shared subscription. */
public Map<String, String> consumersAfterMarkDeletePosition;

public SubscriptionStats() {
this.consumers = Lists.newArrayList();
this.consumersAfterMarkDeletePosition = new LinkedHashMap<>();
}

public void reset() {
Expand Down Expand Up @@ -134,6 +142,7 @@ public SubscriptionStats add(SubscriptionStats stats) {
this.consumers.get(i).add(stats.consumers.get(i));
}
}
this.consumersAfterMarkDeletePosition.putAll(stats.consumersAfterMarkDeletePosition);
return this;
}
}

0 comments on commit 7b56d4c

Please sign in to comment.