You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2022/06/15 14:01:33 UTC

[GitHub] [pulsar] codelipenghui commented on a diff in pull request #15867: add metric for InMemoryDelayedDeliveryTracker's memory usage

codelipenghui commented on code in PR #15867:
URL: https://github.com/apache/pulsar/pull/15867#discussion_r898013253


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java:
##########
@@ -345,6 +346,9 @@ private static void printNamespaceStats(SimpleTextOutputStream stream, String cl
 
         metric(stream, cluster, namespace, "pulsar_subscription_delayed", stats.msgDelayed);
 
+        metric(stream, cluster, namespace, "pulsar_delayed_delivery_tracker_memory_usage",

Review Comment:
   I think `pulsar_delayed_message_index_size_bytes` is better? The usage is a little confusing which can mean *%
   
   And delayed_delivery_tracker is just the name of our implementation. From the user's perspective, the delayed message index is more easy to understand



##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java:
##########
@@ -308,4 +307,70 @@ public void testAutoCreatePartitionedTopicThatNameIncludePartition() throws Exce
         assertEquals(admin.topics().getList(ns).size(), 0);
         pulsar.getConfiguration().setAllowAutoTopicCreationType("non-partitioned");
     }
+
+
+    @Test
+    public void testNamespaceLevelDelayedDeliveryTrackerMemoryUsageMetric() throws Exception {
+        String topic = "persistent://prop/autoNs/test_delayed_message_metric" + UUID.randomUUID();
+        testDelayedDeliveryTrackerMemoryUsageMetric(topic, false);
+    }
+
+    @Test
+    public void testTopicLevelDelayedDeliveryTrackerMemoryUsageMetric() throws Exception {
+        String topic = "persistent://prop/autoNs/test_delayed_message_metric" + UUID.randomUUID();
+        testDelayedDeliveryTrackerMemoryUsageMetric(topic, true);
+    }
+
+
+    private void testDelayedDeliveryTrackerMemoryUsageMetric(String topic, boolean exposeTopicLevelMetrics) throws Exception {
+        PulsarClient client = pulsar.getClient();
+        String namespace = TopicName.get(topic).getNamespace();
+        admin.namespaces().createNamespace(namespace);
+
+        @Cleanup
+        Producer<String> producer = client.newProducer(Schema.STRING).topic(topic).enableBatching(false).create();
+        @Cleanup
+        Consumer<String> consumer = client.newConsumer(Schema.STRING)
+                .topic(topic)
+                .subscriptionName("test_sub")
+                .subscriptionType(SubscriptionType.Shared)
+                .messageListener((MessageListener<String>) (consumer1, msg) -> {
+                    try {
+                        consumer1.acknowledge(msg);
+                    } catch (PulsarClientException e) {
+                        e.printStackTrace();
+                    }
+                })
+                .subscribe();
+        for (int a = 0; a < 100; a++) {
+            producer.newMessage().value(UUID.randomUUID().toString()).deliverAfter(30, TimeUnit.SECONDS).send();
+        }
+        producer.flush();
+
+        //wait for Dispatcher build delayed message index.
+        for (int a = 0; a < 100; a++) {
+            Thread.sleep(1);
+        }
+
+        ByteArrayOutputStream output = new ByteArrayOutputStream();

Review Comment:
   Close after the test. Or add try resource.



##########
pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/TopicStatsImpl.java:
##########
@@ -131,6 +131,9 @@ public class TopicStatsImpl implements TopicStats {
     /** The serialized size of non-contiguous deleted messages ranges. */
     public int nonContiguousDeletedMessagesRangesSerializedSize;
 
+    /** The size of InMemoryDelayedDeliveryTracer memory usage. */
+    public int delayedTrackerMemoryUsage;

Review Comment:
   ```suggestion
       public int delayedMessageIndexSizeInBytes;
   ```



##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java:
##########
@@ -308,4 +307,70 @@ public void testAutoCreatePartitionedTopicThatNameIncludePartition() throws Exce
         assertEquals(admin.topics().getList(ns).size(), 0);
         pulsar.getConfiguration().setAllowAutoTopicCreationType("non-partitioned");
     }
+
+
+    @Test
+    public void testNamespaceLevelDelayedDeliveryTrackerMemoryUsageMetric() throws Exception {
+        String topic = "persistent://prop/autoNs/test_delayed_message_metric" + UUID.randomUUID();
+        testDelayedDeliveryTrackerMemoryUsageMetric(topic, false);
+    }
+
+    @Test
+    public void testTopicLevelDelayedDeliveryTrackerMemoryUsageMetric() throws Exception {
+        String topic = "persistent://prop/autoNs/test_delayed_message_metric" + UUID.randomUUID();
+        testDelayedDeliveryTrackerMemoryUsageMetric(topic, true);
+    }

Review Comment:
   Can replace with a data provider.



##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java:
##########
@@ -308,4 +307,70 @@ public void testAutoCreatePartitionedTopicThatNameIncludePartition() throws Exce
         assertEquals(admin.topics().getList(ns).size(), 0);
         pulsar.getConfiguration().setAllowAutoTopicCreationType("non-partitioned");
     }
+
+
+    @Test
+    public void testNamespaceLevelDelayedDeliveryTrackerMemoryUsageMetric() throws Exception {
+        String topic = "persistent://prop/autoNs/test_delayed_message_metric" + UUID.randomUUID();
+        testDelayedDeliveryTrackerMemoryUsageMetric(topic, false);
+    }
+
+    @Test
+    public void testTopicLevelDelayedDeliveryTrackerMemoryUsageMetric() throws Exception {
+        String topic = "persistent://prop/autoNs/test_delayed_message_metric" + UUID.randomUUID();
+        testDelayedDeliveryTrackerMemoryUsageMetric(topic, true);
+    }
+
+
+    private void testDelayedDeliveryTrackerMemoryUsageMetric(String topic, boolean exposeTopicLevelMetrics) throws Exception {
+        PulsarClient client = pulsar.getClient();
+        String namespace = TopicName.get(topic).getNamespace();
+        admin.namespaces().createNamespace(namespace);
+
+        @Cleanup
+        Producer<String> producer = client.newProducer(Schema.STRING).topic(topic).enableBatching(false).create();
+        @Cleanup
+        Consumer<String> consumer = client.newConsumer(Schema.STRING)
+                .topic(topic)
+                .subscriptionName("test_sub")
+                .subscriptionType(SubscriptionType.Shared)
+                .messageListener((MessageListener<String>) (consumer1, msg) -> {
+                    try {
+                        consumer1.acknowledge(msg);
+                    } catch (PulsarClientException e) {
+                        e.printStackTrace();
+                    }
+                })
+                .subscribe();
+        for (int a = 0; a < 100; a++) {
+            producer.newMessage().value(UUID.randomUUID().toString()).deliverAfter(30, TimeUnit.SECONDS).send();
+        }
+        producer.flush();
+
+        //wait for Dispatcher build delayed message index.
+        for (int a = 0; a < 100; a++) {
+            Thread.sleep(1);
+        }

Review Comment:
   Here might introduce a flaky test because the delayed message index might not complete within 100ms
   
   It's better to use Awaitibility to check some state for the delayed message tracker to make sure we are able to continue the test.



##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java:
##########
@@ -308,4 +307,70 @@ public void testAutoCreatePartitionedTopicThatNameIncludePartition() throws Exce
         assertEquals(admin.topics().getList(ns).size(), 0);
         pulsar.getConfiguration().setAllowAutoTopicCreationType("non-partitioned");
     }
+
+
+    @Test
+    public void testNamespaceLevelDelayedDeliveryTrackerMemoryUsageMetric() throws Exception {
+        String topic = "persistent://prop/autoNs/test_delayed_message_metric" + UUID.randomUUID();
+        testDelayedDeliveryTrackerMemoryUsageMetric(topic, false);
+    }
+
+    @Test
+    public void testTopicLevelDelayedDeliveryTrackerMemoryUsageMetric() throws Exception {
+        String topic = "persistent://prop/autoNs/test_delayed_message_metric" + UUID.randomUUID();
+        testDelayedDeliveryTrackerMemoryUsageMetric(topic, true);
+    }
+
+
+    private void testDelayedDeliveryTrackerMemoryUsageMetric(String topic, boolean exposeTopicLevelMetrics) throws Exception {
+        PulsarClient client = pulsar.getClient();
+        String namespace = TopicName.get(topic).getNamespace();
+        admin.namespaces().createNamespace(namespace);
+
+        @Cleanup
+        Producer<String> producer = client.newProducer(Schema.STRING).topic(topic).enableBatching(false).create();
+        @Cleanup
+        Consumer<String> consumer = client.newConsumer(Schema.STRING)
+                .topic(topic)
+                .subscriptionName("test_sub")
+                .subscriptionType(SubscriptionType.Shared)
+                .messageListener((MessageListener<String>) (consumer1, msg) -> {
+                    try {
+                        consumer1.acknowledge(msg);
+                    } catch (PulsarClientException e) {
+                        e.printStackTrace();
+                    }
+                })
+                .subscribe();
+        for (int a = 0; a < 100; a++) {
+            producer.newMessage().value(UUID.randomUUID().toString()).deliverAfter(30, TimeUnit.SECONDS).send();
+        }
+        producer.flush();
+
+        //wait for Dispatcher build delayed message index.
+        for (int a = 0; a < 100; a++) {
+            Thread.sleep(1);
+        }
+
+        ByteArrayOutputStream output = new ByteArrayOutputStream();
+        PrometheusMetricsGenerator.generate(pulsar, exposeTopicLevelMetrics, true, true, output);
+        String metricsStr = output.toString(StandardCharsets.UTF_8);
+
+        Multimap<String, PrometheusMetricsTest.Metric> metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr);
+        Collection<PrometheusMetricsTest.Metric> metrics = metricsMap.get("pulsar_delayed_delivery_tracker_memory_usage");
+        Assert.assertTrue(metrics.size() > 0);
+
+        int num = 0;
+        for (PrometheusMetricsTest.Metric metric : metrics) {
+            if (exposeTopicLevelMetrics && metric.tags.get("topic").equals(topic)) {
+                Assert.assertTrue(metric.value > 0);

Review Comment:
   Need to check not namespace level metrics.



##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java:
##########
@@ -308,4 +307,70 @@ public void testAutoCreatePartitionedTopicThatNameIncludePartition() throws Exce
         assertEquals(admin.topics().getList(ns).size(), 0);
         pulsar.getConfiguration().setAllowAutoTopicCreationType("non-partitioned");
     }
+
+
+    @Test
+    public void testNamespaceLevelDelayedDeliveryTrackerMemoryUsageMetric() throws Exception {
+        String topic = "persistent://prop/autoNs/test_delayed_message_metric" + UUID.randomUUID();
+        testDelayedDeliveryTrackerMemoryUsageMetric(topic, false);
+    }
+
+    @Test
+    public void testTopicLevelDelayedDeliveryTrackerMemoryUsageMetric() throws Exception {
+        String topic = "persistent://prop/autoNs/test_delayed_message_metric" + UUID.randomUUID();
+        testDelayedDeliveryTrackerMemoryUsageMetric(topic, true);
+    }
+
+
+    private void testDelayedDeliveryTrackerMemoryUsageMetric(String topic, boolean exposeTopicLevelMetrics) throws Exception {
+        PulsarClient client = pulsar.getClient();
+        String namespace = TopicName.get(topic).getNamespace();
+        admin.namespaces().createNamespace(namespace);
+
+        @Cleanup
+        Producer<String> producer = client.newProducer(Schema.STRING).topic(topic).enableBatching(false).create();
+        @Cleanup
+        Consumer<String> consumer = client.newConsumer(Schema.STRING)
+                .topic(topic)
+                .subscriptionName("test_sub")
+                .subscriptionType(SubscriptionType.Shared)
+                .messageListener((MessageListener<String>) (consumer1, msg) -> {
+                    try {
+                        consumer1.acknowledge(msg);
+                    } catch (PulsarClientException e) {
+                        e.printStackTrace();
+                    }
+                })
+                .subscribe();
+        for (int a = 0; a < 100; a++) {
+            producer.newMessage().value(UUID.randomUUID().toString()).deliverAfter(30, TimeUnit.SECONDS).send();

Review Comment:
   ```suggestion
               producer.newMessage().value(UUID.randomUUID().toString()).deliverAfter(30, TimeUnit.SECONDS).sendAsync();
   ```



##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java:
##########
@@ -308,4 +307,70 @@ public void testAutoCreatePartitionedTopicThatNameIncludePartition() throws Exce
         assertEquals(admin.topics().getList(ns).size(), 0);
         pulsar.getConfiguration().setAllowAutoTopicCreationType("non-partitioned");
     }
+
+
+    @Test
+    public void testNamespaceLevelDelayedDeliveryTrackerMemoryUsageMetric() throws Exception {
+        String topic = "persistent://prop/autoNs/test_delayed_message_metric" + UUID.randomUUID();
+        testDelayedDeliveryTrackerMemoryUsageMetric(topic, false);
+    }
+
+    @Test
+    public void testTopicLevelDelayedDeliveryTrackerMemoryUsageMetric() throws Exception {
+        String topic = "persistent://prop/autoNs/test_delayed_message_metric" + UUID.randomUUID();
+        testDelayedDeliveryTrackerMemoryUsageMetric(topic, true);
+    }
+
+
+    private void testDelayedDeliveryTrackerMemoryUsageMetric(String topic, boolean exposeTopicLevelMetrics) throws Exception {
+        PulsarClient client = pulsar.getClient();
+        String namespace = TopicName.get(topic).getNamespace();
+        admin.namespaces().createNamespace(namespace);
+
+        @Cleanup
+        Producer<String> producer = client.newProducer(Schema.STRING).topic(topic).enableBatching(false).create();
+        @Cleanup
+        Consumer<String> consumer = client.newConsumer(Schema.STRING)
+                .topic(topic)
+                .subscriptionName("test_sub")
+                .subscriptionType(SubscriptionType.Shared)
+                .messageListener((MessageListener<String>) (consumer1, msg) -> {
+                    try {
+                        consumer1.acknowledge(msg);
+                    } catch (PulsarClientException e) {
+                        e.printStackTrace();
+                    }
+                })
+                .subscribe();
+        for (int a = 0; a < 100; a++) {
+            producer.newMessage().value(UUID.randomUUID().toString()).deliverAfter(30, TimeUnit.SECONDS).send();
+        }
+        producer.flush();
+
+        //wait for Dispatcher build delayed message index.
+        for (int a = 0; a < 100; a++) {
+            Thread.sleep(1);
+        }
+
+        ByteArrayOutputStream output = new ByteArrayOutputStream();
+        PrometheusMetricsGenerator.generate(pulsar, exposeTopicLevelMetrics, true, true, output);
+        String metricsStr = output.toString(StandardCharsets.UTF_8);
+
+        Multimap<String, PrometheusMetricsTest.Metric> metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr);
+        Collection<PrometheusMetricsTest.Metric> metrics = metricsMap.get("pulsar_delayed_delivery_tracker_memory_usage");
+        Assert.assertTrue(metrics.size() > 0);
+
+        int num = 0;
+        for (PrometheusMetricsTest.Metric metric : metrics) {
+            if (exposeTopicLevelMetrics && metric.tags.get("topic").equals(topic)) {
+                Assert.assertTrue(metric.value > 0);
+                num++;
+            } else if (!exposeTopicLevelMetrics && metric.tags.get("namespace").equals(namespace)) {
+                Assert.assertTrue(metric.value > 0);
+                num++;

Review Comment:
   Need to check no topic level metrics.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org