Skip to content

Commit

Permalink
Expose lastConsumedTimestamp and lastAckedTimestamp to consumer stats
Browse files Browse the repository at this point in the history
---

Master Issue: apache#6046

*Motivation*

Make people can use the timestamp to tell if acknowledge and consumption
are happening.

*Modifications*

- Add lastConsumedTimestamp and lastAckedTimestamp to consume stats

*Verify this change*

- Pass the test `testConsumerStatsLastTimestamp`
  • Loading branch information
zymap committed Jan 14, 2020
1 parent d1f122f commit d22242b
Show file tree
Hide file tree
Showing 3 changed files with 95 additions and 0 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -79,6 +79,9 @@ public class Consumer {
private final Rate msgOut;
private final Rate msgRedeliver;

private long lastConsumedTimestamp;
private long lastAckedTimestamp;

// Represents how many messages we can safely send to the consumer without
// overflowing its receiving queue. The consumer will use Flow commands to
// increase its availability
Expand Down Expand Up @@ -188,6 +191,7 @@ public boolean readCompacted() {
*/
public ChannelPromise sendMessages(final List<Entry> entries, EntryBatchSizes batchSizes, int totalMessages,
long totalBytes, RedeliveryTracker redeliveryTracker) {
this.lastConsumedTimestamp = System.currentTimeMillis();
final ChannelHandlerContext ctx = cnx.ctx();
final ChannelPromise writePromise = ctx.newPromise();

Expand Down Expand Up @@ -335,6 +339,7 @@ void doUnsubscribe(final long requestId) {
}

void messageAcked(CommandAck ack) {
this.lastAckedTimestamp = System.currentTimeMillis();
Map<String,Long> properties = Collections.emptyMap();
if (ack.getPropertiesCount() > 0) {
properties = ack.getPropertiesList().stream()
Expand Down Expand Up @@ -450,6 +455,8 @@ public void updateRates() {
}

public ConsumerStats getStats() {
stats.lastAckedTimestamp = lastAckedTimestamp;
stats.lastConsumedTimestamp = lastConsumedTimestamp;
stats.availablePermits = getAvailablePermits();
stats.unackedMessages = unackedMessages;
stats.blockedConsumerOnUnackedMsgs = blockedConsumerOnUnackedMsgs;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import static org.apache.commons.lang3.StringUtils.isBlank;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertFalse;
import static org.testng.Assert.assertNotEquals;
import static org.testng.Assert.assertNotNull;
import static org.testng.Assert.assertNull;
import static org.testng.Assert.assertTrue;
Expand All @@ -31,6 +32,7 @@
import com.google.common.collect.Sets;

import java.net.URL;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
Expand Down Expand Up @@ -60,6 +62,8 @@
import org.apache.pulsar.client.api.MessageRoutingMode;
import org.apache.pulsar.client.api.Producer;
import org.apache.pulsar.client.api.PulsarClient;
import org.apache.pulsar.client.api.PulsarClientException;
import org.apache.pulsar.client.api.SubscriptionInitialPosition;
import org.apache.pulsar.client.api.SubscriptionType;
import org.apache.pulsar.client.impl.MessageIdImpl;
import org.apache.pulsar.common.naming.TopicDomain;
Expand Down Expand Up @@ -948,4 +952,85 @@ public void testCreateNamespaceWithNoClusters() throws PulsarAdminException {
assertEquals(admin.namespaces().getNamespaceReplicationClusters(namespace),
Collections.singletonList(localCluster));
}

@Test(timeOut = 30000)
public void testConsumerStatsLastTimestamp() throws PulsarClientException, PulsarAdminException, InterruptedException {
long timestamp = System.currentTimeMillis();
final String topicName = "consumer-stats-" + timestamp;
final String subscribeName = topicName + "-test-stats-sub";
final String topic = "persistent://prop-xyz/ns1/" + topicName;
final String producerName = "producer-" + topicName;

@Cleanup
PulsarClient client = PulsarClient.builder().serviceUrl(pulsar.getWebServiceAddress()).build();
Producer<byte[]> producer = client.newProducer().topic(topic)
.enableBatching(false)
.producerName(producerName)
.create();

// a. Send a message to the topic.
producer.send("message-1".getBytes(StandardCharsets.UTF_8));

// b. Create a consumer, because there was a message in the topic, the consumer will receive the message pushed
// by the broker, the lastConsumedTimestamp will as the consume subscribe time.
Consumer<byte[]> consumer = client.newConsumer().topic(topic)
.subscriptionName(subscribeName)
.subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
.subscribe();

// Get the consumer stats.
TopicStats topicStats = admin.topics().getStats(topic);
SubscriptionStats subscriptionStats = topicStats.subscriptions.get(subscribeName);
ConsumerStats consumerStats = subscriptionStats.consumers.get(0);
long startConsumedTimestamp = consumerStats.lastConsumedTimestamp;
long startAckedTimestamp = consumerStats.lastAckedTimestamp;

// Because the message was pushed by the broker, the consumedTimestamp should not as 0.
assertNotEquals(0, startConsumedTimestamp);
// There is no consumer ack the message, so the lastAckedTimestamp still as 0.
assertEquals(0, startAckedTimestamp);


// c. The Consumer receives the message and acks the message.
Message<byte[]> message = consumer.receive();
consumer.acknowledge(message);
// Waiting for the ack command send to the broker.
TimeUnit.SECONDS.sleep(5);

// Get the consumer stats.
topicStats = admin.topics().getStats(topic);
subscriptionStats = topicStats.subscriptions.get(subscribeName);
consumerStats = subscriptionStats.consumers.get(0);
long consumedTimestamp = consumerStats.lastConsumedTimestamp;
long ackedTimestamp = consumerStats.lastAckedTimestamp;

// The lastConsumedTimestamp should same as the last time because the broker does not push any messages and the
// consumer does not pull any messages.
assertEquals(startConsumedTimestamp, consumedTimestamp);
assertTrue(startAckedTimestamp < ackedTimestamp);

// d. Send another messages. The lastConsumedTimestamp should be updated.
producer.send("message-2".getBytes(StandardCharsets.UTF_8));

// e. Receive the message and ack it.
message = consumer.receive();
consumer.acknowledge(message);
// Waiting for the ack command send to the broker.
TimeUnit.SECONDS.sleep(5);

// Get the consumer stats again.
topicStats = admin.topics().getStats(topic);
subscriptionStats = topicStats.subscriptions.get(subscribeName);
consumerStats = subscriptionStats.consumers.get(0);
long lastConsumedTimestamp = consumerStats.lastConsumedTimestamp;
long lastAckedTimestamp = consumerStats.lastAckedTimestamp;

assertTrue(consumedTimestamp < lastConsumedTimestamp);
assertTrue(ackedTimestamp < lastAckedTimestamp);
assertTrue(startConsumedTimestamp < lastConsumedTimestamp);
assertTrue(startAckedTimestamp < lastAckedTimestamp);

consumer.close();
producer.close();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,9 @@ public class ConsumerStats {
private int clientVersionOffset = -1;
private int clientVersionLength;

public long lastAckedTimestamp;
public long lastConsumedTimestamp;

/** Metadata (key/value strings) associated with this consumer. */
public Map<String, String> metadata;

Expand Down

0 comments on commit d22242b

Please sign in to comment.