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/08/10 12:38:50 UTC

[GitHub] [pulsar] tjiuming opened a new pull request, #17041: [monitoring][broker][metadata] add metadata store metrics

tjiuming opened a new pull request, #17041:
URL: https://github.com/apache/pulsar/pull/17041

   ### Motivation
   
   there is no direct metrics for MetadataStore, the PR is supposed to fix it.
   
   ### Documentation
   
   Check the box below or label this PR directly.
   
   Need to update docs? 
   
   - [x] `doc-required` 
   (Your PR needs to update docs and you will update later)
     
   - [ ] `doc-not-needed` 
   (Please explain why)
     
   - [ ] `doc` 
   (Your PR contains doc changes)
   
   - [ ] `doc-complete`
   (Docs have been already added)


-- 
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


[GitHub] [pulsar] codelipenghui merged pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
codelipenghui merged PR #17041:
URL: https://github.com/apache/pulsar/pull/17041


-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r962823290


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java:
##########
@@ -1522,6 +1524,81 @@ public void testSplitTopicAndPartitionLabel() throws Exception {
         consumer2.close();
     }
 
+
+    @Test
+    public void testMetadataStoreStats() throws Exception {

Review Comment:
   I add test here because I want to ensure that all the metrics outputs as expect



-- 
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


[GitHub] [pulsar] tjiuming commented on pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#issuecomment-1248006786

   @codelipenghui @asafm I've added `metadataStoreName` to MetadataStore, PTAL


-- 
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


[GitHub] [pulsar] asafm commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
asafm commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r963470784


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java:
##########
@@ -1557,49 +1557,40 @@ public void testMetadataStoreStats() throws Exception {
         String metricsStr = output.toString();
         Multimap<String, Metric> metricsMap = parseMetrics(metricsStr);
 
-        Collection<Metric> getOpsFailed = metricsMap.get("pulsar_metadata_store_get_failed" + "_total");
-        Collection<Metric> delOpsFailed = metricsMap.get("pulsar_metadata_store_del_failed" + "_total");
-        Collection<Metric> putOpsFailed = metricsMap.get("pulsar_metadata_store_put_failed" + "_total");
-
-        Collection<Metric> getOpsLatency = metricsMap.get("pulsar_metadata_store_get_latency_ms" + "_sum");
-        Collection<Metric> delOpsLatency = metricsMap.get("pulsar_metadata_store_del_latency_ms" + "_sum");
-        Collection<Metric> putOpsLatency = metricsMap.get("pulsar_metadata_store_put_latency_ms" + "_sum");
-
+        Collection<Metric> opsFailed = metricsMap.get("pulsar_metadata_store_ops_failed" + "_total");
+        Collection<Metric> opsLatency = metricsMap.get("pulsar_metadata_store_ops_latency_ms" + "_sum");
         Collection<Metric> putBytes = metricsMap.get("pulsar_metadata_store_put_bytes" + "_total");
 
-        Assert.assertTrue(getOpsFailed.size() > 1);
-        Assert.assertTrue(delOpsFailed.size() > 1);
-        Assert.assertTrue(putOpsFailed.size() > 1);
-        Assert.assertTrue(getOpsLatency.size() > 1);
-        Assert.assertTrue(delOpsLatency.size() > 1);
-        Assert.assertTrue(putOpsLatency.size() > 1);
+        Assert.assertTrue(opsFailed.size() > 1);
+        Assert.assertTrue(opsLatency.size() > 1);
         Assert.assertTrue(putBytes.size() > 1);
 
-        for (Metric m : getOpsFailed) {
-            Assert.assertEquals(m.tags.get("cluster"), "test");
-            Assert.assertTrue(m.tags.get("name").startsWith("metadata-store"));
-        }
-        for (Metric m : delOpsFailed) {
-            Assert.assertEquals(m.tags.get("cluster"), "test");
-            Assert.assertTrue(m.tags.get("name").startsWith("metadata-store"));
-        }
-        for (Metric m : putOpsFailed) {
-            Assert.assertEquals(m.tags.get("cluster"), "test");
-            Assert.assertTrue(m.tags.get("name").startsWith("metadata-store"));
-        }
-        for (Metric m : getOpsLatency) {
-            Assert.assertEquals(m.tags.get("cluster"), "test");
-            Assert.assertTrue(m.tags.get("name").startsWith("metadata-store"));
-            Assert.assertTrue(m.value > 0);
-        }
-        for (Metric m : delOpsLatency) {
+        for (Metric m : opsFailed) {
             Assert.assertEquals(m.tags.get("cluster"), "test");
             Assert.assertTrue(m.tags.get("name").startsWith("metadata-store"));
+            if (m.tags.get("type").equals("get")) {
+                Assert.assertTrue(m.value >= 0);

Review Comment:
   ![image](https://user-images.githubusercontent.com/989425/188597706-b625e38e-8122-4c1b-beb6-ed74ae8c8549.png)
   You don't get that auto-complete as the image I snapshot from my IntelliJ?



-- 
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


[GitHub] [pulsar] codelipenghui commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r945700331


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -24,41 +24,42 @@
 
 public final class MetadataStoreStats implements AutoCloseable {
     private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
-    private static final String LABEL_NAME = "metadata_store_name";
+    private static final String LABEL_NAME = "name";
+    protected static final String PREFIX = "pulsar_metadata_store_";
 
     private static final Histogram GET_OPS_SUCCEED = Histogram
-            .build("pulsar_metadata_store_get_ops_latency", "-")
+            .build(PREFIX + "get_latency", "-")
             .unit("ms")
             .buckets(BUCKETS)
             .labelNames(LABEL_NAME)
             .register();
     private static final Histogram DEL_OPS_SUCCEED = Histogram
-            .build("pulsar_metadata_store_del_ops_latency", "-")
+            .build(PREFIX + "del_latency", "-")

Review Comment:
   I'm not very sure about this part. Just the first impression, using labels here will reduce the metrics name.
   Documentation will also be more concise.



##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -24,41 +24,42 @@
 
 public final class MetadataStoreStats implements AutoCloseable {
     private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
-    private static final String LABEL_NAME = "metadata_store_name";
+    private static final String LABEL_NAME = "name";
+    protected static final String PREFIX = "pulsar_metadata_store_";
 
     private static final Histogram GET_OPS_SUCCEED = Histogram
-            .build("pulsar_metadata_store_get_ops_latency", "-")
+            .build(PREFIX + "get_latency", "-")
             .unit("ms")
             .buckets(BUCKETS)
             .labelNames(LABEL_NAME)
             .register();
     private static final Histogram DEL_OPS_SUCCEED = Histogram
-            .build("pulsar_metadata_store_del_ops_latency", "-")
+            .build(PREFIX + "del_latency", "-")

Review Comment:
   I'm thinking how about adding operation type as a label?
   
   It looks like 
   
   pulsar_metadata_store_op_latency_ms{type="put/get/del"}
   pulsar_metadata_store_op_failed_total{type="put/get/del"}



##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -24,41 +24,42 @@
 
 public final class MetadataStoreStats implements AutoCloseable {
     private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
-    private static final String LABEL_NAME = "metadata_store_name";
+    private static final String LABEL_NAME = "name";
+    protected static final String PREFIX = "pulsar_metadata_store_";
 
     private static final Histogram GET_OPS_SUCCEED = Histogram
-            .build("pulsar_metadata_store_get_ops_latency", "-")
+            .build(PREFIX + "get_latency", "-")
             .unit("ms")
             .buckets(BUCKETS)
             .labelNames(LABEL_NAME)
             .register();
     private static final Histogram DEL_OPS_SUCCEED = Histogram
-            .build("pulsar_metadata_store_del_ops_latency", "-")
+            .build(PREFIX + "del_latency", "-")

Review Comment:
   I think unit "ms" will append _ms automatically?



-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r943693887


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java:
##########
@@ -235,10 +239,20 @@ public <T> MetadataCache<T> getMetadataCache(MetadataSerde<T> serde) {
 
     @Override
     public CompletableFuture<Optional<GetResult>> get(String path) {
+        long start = System.currentTimeMillis();
         if (!isValidPath(path)) {
-            return FutureUtil.failedFuture(new MetadataStoreException.InvalidPathException(path));
+            stats.recordGetOpsFailed();
+            return FutureUtil
+                    .failedFuture(new MetadataStoreException.InvalidPathException(path));
         }
-        return storeGet(path);
+        return storeGet(path)
+                .whenComplete((v, t) -> {
+                    if (t != null) {
+                        stats.recordGetOpsFailed();
+                    } else {
+                        stats.recordGetOpsLatency(System.currentTimeMillis() - start);

Review Comment:
   latency record ops's latency, but `recordGetOpsFailed ` the times of failed ops. I think no need to keep consistent.



-- 
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


[GitHub] [pulsar] codelipenghui commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r944063001


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java:
##########
@@ -1522,6 +1517,83 @@ public void testSplitTopicAndPartitionLabel() throws Exception {
         consumer2.close();
     }
 
+
+    @Test
+    public void testMetadataStoreStats() throws Exception {
+        String ns = "prop/ns-abc1";
+        admin.namespaces().createNamespace(ns);
+
+        String topic = "persistent://prop/ns-abc1/metadata-store-" + UUID.randomUUID();
+        String subName = "my-sub1";
+
+        @Cleanup
+        Producer<String> producer = pulsarClient.newProducer(Schema.STRING)
+                .topic(topic).create();
+        @Cleanup
+        Consumer<String> consumer = pulsarClient.newConsumer(Schema.STRING)
+                .topic(topic).subscriptionName(subName).subscribe();
+
+        for (int i = 0; i < 100; i++) {
+            producer.newMessage().value(UUID.randomUUID().toString()).send();
+        }
+
+        for (;;) {
+            Message<String> message = consumer.receive(10, TimeUnit.SECONDS);
+            if (message == null) {
+                break;
+            }
+            consumer.acknowledge(message);
+        }
+
+        ByteArrayOutputStream output = new ByteArrayOutputStream();
+        PrometheusMetricsGenerator.generate(pulsar, false, false, false, false, output);
+        String metricsStr = output.toString();
+        Multimap<String, Metric> metricsMap = parseMetrics(metricsStr);
+
+        Collection<Metric> getOpsFailed = metricsMap.get("pulsar_metadata_store_get_ops_failed" + "_total");
+        Collection<Metric> delOpsFailed = metricsMap.get("pulsar_metadata_store_del_ops_failed" + "_total");
+        Collection<Metric> putOpsFailed = metricsMap.get("pulsar_metadata_store_put_ops_failed" + "_total");
+
+        Collection<Metric> getOpsLatency = metricsMap.get("pulsar_metadata_store_get_ops_latency" + "_sum");
+        Collection<Metric> delOpsLatency = metricsMap.get("pulsar_metadata_store_del_ops_latency" + "_sum");
+        Collection<Metric> putOpsLatency = metricsMap.get("pulsar_metadata_store_put_ops_latency" + "_sum");
+
+        Collection<Metric> putBytes = metricsMap.get("pulsar_metadata_store_put_bytes" + "_total");
+
+        Assert.assertTrue(getOpsFailed.size() > 0);

Review Comment:
   I think we can update the checkstyle like this https://lightrun.com/answers/checkstyle-checkstyle-illegalimport-does-not-warn-about-static-imports
   
   There are many places are using `Assert.assert`



-- 
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


[GitHub] [pulsar] tjiuming commented on pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#issuecomment-1239598295

   > @tjiuming
   > 
   > > Broker metadata store initialized before configuration metadata store, so, metadata-store-0 is broker metadata store and metadata-store-1 is configuration metadata store
   > 
   > IMO, we should improve this part. Instead of adding docs to tell users which one is configuration store, and which one is metadata store, we should add the type to the metrics like `type=configuration/metadata`. I also consider adding the metadataStoreUrl, but it can be a long string value.
   > 
   > And we should also add the `cluster` label. Users can have one Prometheus service for multiple clusters, e.g. https://github.com/apache/pulsar/wiki/PIP-8:-Pulsar-beyond-1M-topics
   
   @codelipenghui `cluster` label will be added automatically in `PrometheusMetricsGeneratorUtils.generateSystemMetrics(...)`
   for metadata store name, if I change `protected AbstractMetadataStore()` to `protected AbstractMetadataStore(String name)`, there will be toooo many places need to fix, WDYT?


-- 
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


[GitHub] [pulsar] tjiuming commented on pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#issuecomment-1244880716

   > I think it's the 100% right thing to do. If we don't have this change, we might need to explain to many users what the name exactly means. And I think that should be an important point that we missed before, if you are troubleshooting problems with the heap dump, it is hard to determine which one is the configuration store and which one is the metastore.
   
   @codelipenghui I've tried to change `public AbstractMetadataStore()` to `public AbstractMetadataStore(String name)`, there will be faaaar beyond 100 places need to change, I think we should consider it carefully
   


-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r948756814


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -24,41 +24,42 @@
 
 public final class MetadataStoreStats implements AutoCloseable {
     private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
-    private static final String LABEL_NAME = "metadata_store_name";
+    private static final String LABEL_NAME = "name";
+    protected static final String PREFIX = "pulsar_metadata_store_";
 
     private static final Histogram GET_OPS_SUCCEED = Histogram
-            .build("pulsar_metadata_store_get_ops_latency", "-")
+            .build(PREFIX + "get_latency", "-")
             .unit("ms")
             .buckets(BUCKETS)
             .labelNames(LABEL_NAME)
             .register();
     private static final Histogram DEL_OPS_SUCCEED = Histogram
-            .build("pulsar_metadata_store_del_ops_latency", "-")
+            .build(PREFIX + "del_latency", "-")

Review Comment:
   @codelipenghui   https://prometheus.io/docs/practices/naming/   you're right, Prometheus recommend using `label` to distinguish



-- 
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


[GitHub] [pulsar] asafm commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
asafm commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r947596578


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -24,41 +24,42 @@
 
 public final class MetadataStoreStats implements AutoCloseable {
     private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
-    private static final String LABEL_NAME = "metadata_store_name";
+    private static final String LABEL_NAME = "name";
+    protected static final String PREFIX = "pulsar_metadata_store_";
 
     private static final Histogram GET_OPS_SUCCEED = Histogram
-            .build("pulsar_metadata_store_get_ops_latency", "-")
+            .build(PREFIX + "get_latency", "-")
             .unit("ms")
             .buckets(BUCKETS)
             .labelNames(LABEL_NAME)
             .register();
     private static final Histogram DEL_OPS_SUCCEED = Histogram
-            .build("pulsar_metadata_store_del_ops_latency", "-")
+            .build(PREFIX + "del_latency", "-")

Review Comment:
   The advantage you get is mostly when you want to look at those metrics together - like give me an average of all operations. Is it something we'll need or always look at per action?



-- 
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


[GitHub] [pulsar] tjiuming commented on pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#issuecomment-1238831630

   @asafm Yes, you are right and I understand your concern, but actually, we can know that which metadata store is configuration store and which stores broker metadata, just takes a little skill:
   
   In PulsarService.java, code as below:
   ```
               localMetadataStore = createLocalMetadataStore(localMetadataSynchronizer);
               localMetadataStore.registerSessionListener(this::handleMetadataSessionEvent);
   
               coordinationService = new CoordinationServiceImpl(localMetadataStore);
   
               if (config.isConfigurationStoreSeparated()) {
                   configMetadataSynchronizer = StringUtils.isNotBlank(config.getConfigurationMetadataSyncEventTopic())
                           ? new PulsarMetadataEventSynchronizer(this, config.getConfigurationMetadataSyncEventTopic())
                           : null;
                   configurationMetadataStore = createConfigurationMetadataStore(configMetadataSynchronizer);
                   shouldShutdownConfigurationMetadataStore = true;
               } else {
                   configurationMetadataStore = localMetadataStore;
                   shouldShutdownConfigurationMetadataStore = false;
               }
   ```
   Broker metadata store initialized before configuration metadata store, so, `metadata-store-0` is broker metadata store and `metadata-store-1` is configuration metadata store


-- 
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


[GitHub] [pulsar] asafm commented on pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
asafm commented on PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#issuecomment-1237878215

   >for 1, if I change protected AbstractMetadataStore() to protected AbstractMetadataStore(String metadataStoreName),
   
   Maybe I misunderstand. You have 5 classes that extend `AbstractMetadataStore`, which are not abstract, right? If I understand correctly, you only need to modify them, no?


-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r960851204


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -24,48 +24,32 @@
 
 public final class MetadataStoreStats implements AutoCloseable {
     private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
-    private static final String LABEL_NAME = "name";
+    private static final String OPS_TYPE_LABEL_NAME = "type";
+    private static final String METADATA_STORE_LABEL_NAME = "name";
+    private static final String OPS_TYPE_GET = "get";
+    private static final String OPS_TYPE_DEL = "del";
+    private static final String OPS_TYPE_PUT = "put";
     protected static final String PREFIX = "pulsar_metadata_store_";
 
-    private static final Histogram GET_OPS_SUCCEED = Histogram
-            .build(PREFIX + "get_latency", "-")
+    private static final Histogram OPS_SUCCEED = Histogram
+            .build(PREFIX + "ops_latency", "-")
             .unit("ms")
             .buckets(BUCKETS)
-            .labelNames(LABEL_NAME)
+            .labelNames(METADATA_STORE_LABEL_NAME, OPS_TYPE_LABEL_NAME)

Review Comment:
   there are more than 1 Metadata store, one for broker metadata, one for dynamic config.
   you can see `metadataStoreUrl` and `configurationMetadataStoreUrl` in https://github.com/apache/pulsar/blob/master/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java



-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r947395766


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -24,41 +24,42 @@
 
 public final class MetadataStoreStats implements AutoCloseable {
     private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
-    private static final String LABEL_NAME = "metadata_store_name";
+    private static final String LABEL_NAME = "name";
+    protected static final String PREFIX = "pulsar_metadata_store_";
 
     private static final Histogram GET_OPS_SUCCEED = Histogram
-            .build("pulsar_metadata_store_get_ops_latency", "-")
+            .build(PREFIX + "get_latency", "-")
             .unit("ms")
             .buckets(BUCKETS)
             .labelNames(LABEL_NAME)
             .register();
     private static final Histogram DEL_OPS_SUCCEED = Histogram
-            .build("pulsar_metadata_store_del_ops_latency", "-")
+            .build(PREFIX + "del_latency", "-")

Review Comment:
   I don't think so, it just changes
   ```
   pulsar_metadata_store_get_latency_ms{name = 'xx'}
   pulsar_metadata_store_del_latency_ms{name = 'xx'}
   pulsar_metadata_store_put_latency_ms{name = 'xx'}
   ```
   to
   ```
   pulsar_metadata_store_latency_ms{name = 'xx', type = 'get'}
   pulsar_metadata_store_latency_ms{name = 'xx', type = 'del'}
   pulsar_metadata_store_latency_ms{name = 'xx', type = 'put'}
   ```
   It comes no benefits and makes metrics data longer.



-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r962826755


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -0,0 +1,109 @@
+/**
+ * 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.metadata.impl.stats;
+
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public final class MetadataStoreStats implements AutoCloseable {
+    private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
+    private static final String OPS_TYPE_LABEL_NAME = "type";
+    private static final String METADATA_STORE_LABEL_NAME = "name";
+    private static final String OPS_TYPE_GET = "get";
+    private static final String OPS_TYPE_DEL = "del";
+    private static final String OPS_TYPE_PUT = "put";
+    protected static final String PREFIX = "pulsar_metadata_store_";
+
+    private static final Histogram OPS_SUCCEED = Histogram
+            .build(PREFIX + "ops_latency", "-")
+            .unit("ms")
+            .buckets(BUCKETS)
+            .labelNames(METADATA_STORE_LABEL_NAME, OPS_TYPE_LABEL_NAME)
+            .register();
+    private static final Counter OPS_FAILED = Counter
+            .build(PREFIX + "ops_failed", "-")
+            .labelNames(METADATA_STORE_LABEL_NAME, OPS_TYPE_LABEL_NAME)
+            .register();
+    private static final Counter PUT_BYTES = Counter
+            .build(PREFIX + "put_", "-")
+            .unit("bytes")
+            .labelNames(METADATA_STORE_LABEL_NAME)
+            .register();
+
+    private final Histogram.Child getOpsSucceedChild;
+    private final Histogram.Child delOpsSucceedChild;
+    private final Histogram.Child putOpsSucceedChild;
+    private final Counter.Child getOpsFailedChild;
+    private final Counter.Child delOpsFailedChild;
+    private final Counter.Child putOpsFailedChild;
+    private final Counter.Child putBytesChild;
+    private final String metadataStoreName;
+    private final AtomicBoolean closed = new AtomicBoolean(false);
+
+    public MetadataStoreStats(String metadataStoreName) {
+        this.metadataStoreName = metadataStoreName;
+
+        this.getOpsSucceedChild = OPS_SUCCEED.labels(metadataStoreName, OPS_TYPE_GET);
+        this.delOpsSucceedChild = OPS_SUCCEED.labels(metadataStoreName, OPS_TYPE_DEL);
+        this.putOpsSucceedChild = OPS_SUCCEED.labels(metadataStoreName, OPS_TYPE_PUT);
+        this.getOpsFailedChild = OPS_FAILED.labels(metadataStoreName, OPS_TYPE_GET);
+        this.delOpsFailedChild = OPS_FAILED.labels(metadataStoreName, OPS_TYPE_DEL);
+        this.putOpsFailedChild = OPS_FAILED.labels(metadataStoreName, OPS_TYPE_PUT);
+        this.putBytesChild = PUT_BYTES.labels(metadataStoreName);
+    }
+
+    public void recordGetOpsLatency(long millis) {
+        this.getOpsSucceedChild.observe(millis);
+    }
+
+    public void recordDelOpsLatency(long millis) {
+        this.delOpsSucceedChild.observe(millis);
+    }
+
+    public void recordPutOpsLatency(long millis, int bytes) {
+        this.putOpsSucceedChild.observe(millis);
+        this.putBytesChild.inc(bytes);
+    }
+
+    public void recordGetOpsFailed() {
+        this.getOpsFailedChild.inc();
+    }
+
+    public void recordDelOpsFailed() {
+        this.delOpsFailedChild.inc();
+    }
+
+    public void recordPutOpsFailed() {
+        this.putOpsFailedChild.inc();
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (this.closed.compareAndSet(false, true)) {

Review Comment:
   There are many similar uses in Pulsar, would you change them all?



-- 
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


[GitHub] [pulsar] asafm commented on pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
asafm commented on PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#issuecomment-1211901902

   @codelipenghui done


-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r960851204


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -24,48 +24,32 @@
 
 public final class MetadataStoreStats implements AutoCloseable {
     private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
-    private static final String LABEL_NAME = "name";
+    private static final String OPS_TYPE_LABEL_NAME = "type";
+    private static final String METADATA_STORE_LABEL_NAME = "name";
+    private static final String OPS_TYPE_GET = "get";
+    private static final String OPS_TYPE_DEL = "del";
+    private static final String OPS_TYPE_PUT = "put";
     protected static final String PREFIX = "pulsar_metadata_store_";
 
-    private static final Histogram GET_OPS_SUCCEED = Histogram
-            .build(PREFIX + "get_latency", "-")
+    private static final Histogram OPS_SUCCEED = Histogram
+            .build(PREFIX + "ops_latency", "-")
             .unit("ms")
             .buckets(BUCKETS)
-            .labelNames(LABEL_NAME)
+            .labelNames(METADATA_STORE_LABEL_NAME, OPS_TYPE_LABEL_NAME)

Review Comment:
   there will be more than 1 Metadata store, one for broker metadata, one for dynamic config.



-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r960850416


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -24,48 +24,32 @@
 
 public final class MetadataStoreStats implements AutoCloseable {
     private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
-    private static final String LABEL_NAME = "name";
+    private static final String OPS_TYPE_LABEL_NAME = "type";
+    private static final String METADATA_STORE_LABEL_NAME = "name";
+    private static final String OPS_TYPE_GET = "get";
+    private static final String OPS_TYPE_DEL = "del";
+    private static final String OPS_TYPE_PUT = "put";
     protected static final String PREFIX = "pulsar_metadata_store_";
 
-    private static final Histogram GET_OPS_SUCCEED = Histogram
-            .build(PREFIX + "get_latency", "-")
+    private static final Histogram OPS_SUCCEED = Histogram

Review Comment:
   I remember here is a codestyle check, a static final field must be named as `AA_BB_CC`



-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r960864452


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java:
##########
@@ -126,6 +131,9 @@ public CompletableFuture<Boolean> asyncReload(String key, Boolean oldValue,
                         }
                     }
                 });
+
+        this.metadataStoreName = poolName + "-" + ID.getAndIncrement();

Review Comment:
   1. there are more than 1 Metadata store, one for broker metadata, one for dynamic config.
   you can see metadataStoreUrl and configurationMetadataStoreUrl in https://github.com/apache/pulsar/blob/master/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java
   2. the metadata store name generated by the same rule with ThreadPool, keep align with ThreadPool
   3. keep align with ThreadPool name



-- 
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


[GitHub] [pulsar] tjiuming commented on pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#issuecomment-1237135618

   > I commented @tjiuming Aside from style notations, there are 3 main things I would improve:
   > 
   > 1. Metadata store name is not indicative enough - see comment
   > 2. Add latency for failed ops - see comment
   > 3. Move tests out of `PrometheusMetricsTest` - see comment
   
   for 1, if I change `  protected AbstractMetadataStore()` to `  protected AbstractMetadataStore(String metadataStoreName)`, there will be about 100+ places need to fix, I don't want to make such a huge change.
   for 2, resolved
   for 3, as I commented above, I added metadata store test here to ensure all metrics outputs as expected.


-- 
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


[GitHub] [pulsar] asafm commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
asafm commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r962303008


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java:
##########
@@ -1557,49 +1557,40 @@ public void testMetadataStoreStats() throws Exception {
         String metricsStr = output.toString();
         Multimap<String, Metric> metricsMap = parseMetrics(metricsStr);
 
-        Collection<Metric> getOpsFailed = metricsMap.get("pulsar_metadata_store_get_failed" + "_total");
-        Collection<Metric> delOpsFailed = metricsMap.get("pulsar_metadata_store_del_failed" + "_total");
-        Collection<Metric> putOpsFailed = metricsMap.get("pulsar_metadata_store_put_failed" + "_total");
-
-        Collection<Metric> getOpsLatency = metricsMap.get("pulsar_metadata_store_get_latency_ms" + "_sum");
-        Collection<Metric> delOpsLatency = metricsMap.get("pulsar_metadata_store_del_latency_ms" + "_sum");
-        Collection<Metric> putOpsLatency = metricsMap.get("pulsar_metadata_store_put_latency_ms" + "_sum");
-
+        Collection<Metric> opsFailed = metricsMap.get("pulsar_metadata_store_ops_failed" + "_total");
+        Collection<Metric> opsLatency = metricsMap.get("pulsar_metadata_store_ops_latency_ms" + "_sum");
         Collection<Metric> putBytes = metricsMap.get("pulsar_metadata_store_put_bytes" + "_total");
 
-        Assert.assertTrue(getOpsFailed.size() > 1);
-        Assert.assertTrue(delOpsFailed.size() > 1);
-        Assert.assertTrue(putOpsFailed.size() > 1);
-        Assert.assertTrue(getOpsLatency.size() > 1);
-        Assert.assertTrue(delOpsLatency.size() > 1);
-        Assert.assertTrue(putOpsLatency.size() > 1);
+        Assert.assertTrue(opsFailed.size() > 1);
+        Assert.assertTrue(opsLatency.size() > 1);
         Assert.assertTrue(putBytes.size() > 1);
 
-        for (Metric m : getOpsFailed) {
-            Assert.assertEquals(m.tags.get("cluster"), "test");
-            Assert.assertTrue(m.tags.get("name").startsWith("metadata-store"));
-        }
-        for (Metric m : delOpsFailed) {
-            Assert.assertEquals(m.tags.get("cluster"), "test");
-            Assert.assertTrue(m.tags.get("name").startsWith("metadata-store"));
-        }
-        for (Metric m : putOpsFailed) {
-            Assert.assertEquals(m.tags.get("cluster"), "test");
-            Assert.assertTrue(m.tags.get("name").startsWith("metadata-store"));
-        }
-        for (Metric m : getOpsLatency) {
-            Assert.assertEquals(m.tags.get("cluster"), "test");
-            Assert.assertTrue(m.tags.get("name").startsWith("metadata-store"));
-            Assert.assertTrue(m.value > 0);
-        }
-        for (Metric m : delOpsLatency) {
+        for (Metric m : opsFailed) {
             Assert.assertEquals(m.tags.get("cluster"), "test");
             Assert.assertTrue(m.tags.get("name").startsWith("metadata-store"));
+            if (m.tags.get("type").equals("get")) {
+                Assert.assertTrue(m.value >= 0);

Review Comment:
   Out of curiosity: Why? You are a word which adds not value
   
   If I read `assertTrue(m.value >=0)`, it's much easier that `Assert.assertThat(m.value >0)` since the word assert is already present in the method name - it's like saying "Assert assert value > 0" - doesn't sound right.
   



-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r963503249


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -0,0 +1,109 @@
+/**
+ * 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.metadata.impl.stats;
+
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public final class MetadataStoreStats implements AutoCloseable {
+    private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
+    private static final String OPS_TYPE_LABEL_NAME = "type";
+    private static final String METADATA_STORE_LABEL_NAME = "name";
+    private static final String OPS_TYPE_GET = "get";
+    private static final String OPS_TYPE_DEL = "del";
+    private static final String OPS_TYPE_PUT = "put";
+    protected static final String PREFIX = "pulsar_metadata_store_";
+
+    private static final Histogram OPS_SUCCEED = Histogram
+            .build(PREFIX + "ops_latency", "-")
+            .unit("ms")
+            .buckets(BUCKETS)
+            .labelNames(METADATA_STORE_LABEL_NAME, OPS_TYPE_LABEL_NAME)
+            .register();
+    private static final Counter OPS_FAILED = Counter
+            .build(PREFIX + "ops_failed", "-")
+            .labelNames(METADATA_STORE_LABEL_NAME, OPS_TYPE_LABEL_NAME)
+            .register();
+    private static final Counter PUT_BYTES = Counter
+            .build(PREFIX + "put_", "-")
+            .unit("bytes")
+            .labelNames(METADATA_STORE_LABEL_NAME)
+            .register();
+
+    private final Histogram.Child getOpsSucceedChild;
+    private final Histogram.Child delOpsSucceedChild;
+    private final Histogram.Child putOpsSucceedChild;
+    private final Counter.Child getOpsFailedChild;
+    private final Counter.Child delOpsFailedChild;
+    private final Counter.Child putOpsFailedChild;
+    private final Counter.Child putBytesChild;
+    private final String metadataStoreName;
+    private final AtomicBoolean closed = new AtomicBoolean(false);
+
+    public MetadataStoreStats(String metadataStoreName) {
+        this.metadataStoreName = metadataStoreName;
+
+        this.getOpsSucceedChild = OPS_SUCCEED.labels(metadataStoreName, OPS_TYPE_GET);
+        this.delOpsSucceedChild = OPS_SUCCEED.labels(metadataStoreName, OPS_TYPE_DEL);
+        this.putOpsSucceedChild = OPS_SUCCEED.labels(metadataStoreName, OPS_TYPE_PUT);
+        this.getOpsFailedChild = OPS_FAILED.labels(metadataStoreName, OPS_TYPE_GET);
+        this.delOpsFailedChild = OPS_FAILED.labels(metadataStoreName, OPS_TYPE_DEL);
+        this.putOpsFailedChild = OPS_FAILED.labels(metadataStoreName, OPS_TYPE_PUT);
+        this.putBytesChild = PUT_BYTES.labels(metadataStoreName);
+    }
+
+    public void recordGetOpsLatency(long millis) {
+        this.getOpsSucceedChild.observe(millis);
+    }
+
+    public void recordDelOpsLatency(long millis) {
+        this.delOpsSucceedChild.observe(millis);
+    }
+
+    public void recordPutOpsLatency(long millis, int bytes) {
+        this.putOpsSucceedChild.observe(millis);
+        this.putBytesChild.inc(bytes);
+    }
+
+    public void recordGetOpsFailed() {
+        this.getOpsFailedChild.inc();
+    }
+
+    public void recordDelOpsFailed() {
+        this.delOpsFailedChild.inc();
+    }
+
+    public void recordPutOpsFailed() {
+        this.putOpsFailedChild.inc();
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (this.closed.compareAndSet(false, true)) {

Review Comment:
   > there are many tests that call close() twice, which will cause your build to fail on many tests?
   
   right, when I develop offloader metrics, it happened, so I have to add the logic



-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r947666897


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -24,41 +24,42 @@
 
 public final class MetadataStoreStats implements AutoCloseable {
     private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
-    private static final String LABEL_NAME = "metadata_store_name";
+    private static final String LABEL_NAME = "name";
+    protected static final String PREFIX = "pulsar_metadata_store_";
 
     private static final Histogram GET_OPS_SUCCEED = Histogram
-            .build("pulsar_metadata_store_get_ops_latency", "-")
+            .build(PREFIX + "get_latency", "-")
             .unit("ms")
             .buckets(BUCKETS)
             .labelNames(LABEL_NAME)
             .register();
     private static final Histogram DEL_OPS_SUCCEED = Histogram
-            .build("pulsar_metadata_store_del_ops_latency", "-")
+            .build(PREFIX + "del_latency", "-")

Review Comment:
   I don't think that `an average of all operations` is meaningful, I'll ask prometheus community which is better.
    



-- 
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


[GitHub] [pulsar] codelipenghui commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r952670129


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java:
##########
@@ -81,8 +86,8 @@ public abstract class AbstractMetadataStore implements MetadataStoreExtended, Co
     protected abstract CompletableFuture<Boolean> existsFromStore(String path);
 
     protected AbstractMetadataStore() {
-        this.executor = Executors
-                .newSingleThreadScheduledExecutor(new DefaultThreadFactory("metadata-store"));
+        final String poolName = "metadata-store";
+        this.executor = new ScheduledThreadPoolExecutor(1, new DefaultThreadFactory(poolName));

Review Comment:
   Does this change required?



-- 
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


[GitHub] [pulsar] asafm commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
asafm commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r959335404


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java:
##########
@@ -1522,6 +1524,81 @@ public void testSplitTopicAndPartitionLabel() throws Exception {
         consumer2.close();
     }
 
+
+    @Test
+    public void testMetadataStoreStats() throws Exception {

Review Comment:
   I wouldn't add tests to this file. It's an all-include bucket files which can easily wind up 10k lines.
   I recommend moving those tests to `MetadataStoreStatsTest`



##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java:
##########
@@ -126,6 +128,8 @@ public CompletableFuture<Boolean> asyncReload(String key, Boolean oldValue,
                         }
                     }
                 });
+
+        this.stats = MetadataStoreStats.create();

Review Comment:
   Same as `Assert`, `this.` doesn't add any meaning here, so it gets in the way of reading the code.
   
   @codelipenghui What's the process for changing the `checkstyle` plugin to add it?



-- 
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


[GitHub] [pulsar] asafm commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
asafm commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r959328014


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java:
##########
@@ -1522,6 +1517,83 @@ public void testSplitTopicAndPartitionLabel() throws Exception {
         consumer2.close();
     }
 
+
+    @Test
+    public void testMetadataStoreStats() throws Exception {
+        String ns = "prop/ns-abc1";
+        admin.namespaces().createNamespace(ns);
+
+        String topic = "persistent://prop/ns-abc1/metadata-store-" + UUID.randomUUID();
+        String subName = "my-sub1";
+
+        @Cleanup
+        Producer<String> producer = pulsarClient.newProducer(Schema.STRING)
+                .topic(topic).create();
+        @Cleanup
+        Consumer<String> consumer = pulsarClient.newConsumer(Schema.STRING)
+                .topic(topic).subscriptionName(subName).subscribe();
+
+        for (int i = 0; i < 100; i++) {
+            producer.newMessage().value(UUID.randomUUID().toString()).send();
+        }
+
+        for (;;) {
+            Message<String> message = consumer.receive(10, TimeUnit.SECONDS);
+            if (message == null) {
+                break;
+            }
+            consumer.acknowledge(message);
+        }
+
+        ByteArrayOutputStream output = new ByteArrayOutputStream();
+        PrometheusMetricsGenerator.generate(pulsar, false, false, false, false, output);
+        String metricsStr = output.toString();
+        Multimap<String, Metric> metricsMap = parseMetrics(metricsStr);
+
+        Collection<Metric> getOpsFailed = metricsMap.get("pulsar_metadata_store_get_ops_failed" + "_total");
+        Collection<Metric> delOpsFailed = metricsMap.get("pulsar_metadata_store_del_ops_failed" + "_total");
+        Collection<Metric> putOpsFailed = metricsMap.get("pulsar_metadata_store_put_ops_failed" + "_total");
+
+        Collection<Metric> getOpsLatency = metricsMap.get("pulsar_metadata_store_get_ops_latency" + "_sum");
+        Collection<Metric> delOpsLatency = metricsMap.get("pulsar_metadata_store_del_ops_latency" + "_sum");
+        Collection<Metric> putOpsLatency = metricsMap.get("pulsar_metadata_store_put_ops_latency" + "_sum");
+
+        Collection<Metric> putBytes = metricsMap.get("pulsar_metadata_store_put_bytes" + "_total");
+
+        Assert.assertTrue(getOpsFailed.size() > 0);

Review Comment:
   I don't understand, "It's not an issue"?
   
   IMO, the word `Assert.` doesn't add any substantial meaning and just gets in the way of reading the code. 
   



-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r960869706


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java:
##########
@@ -126,6 +131,9 @@ public CompletableFuture<Boolean> asyncReload(String key, Boolean oldValue,
                         }
                     }
                 });
+
+        this.metadataStoreName = poolName + "-" + ID.getAndIncrement();
+        this.metadataStoreStats = new MetadataStoreStats(metadataStoreName);

Review Comment:
   personal code style. and Pulsar checkstyle doesn't requires we can't or we must add a `this.` prefix



-- 
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


[GitHub] [pulsar] codelipenghui commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r944405940


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -0,0 +1,124 @@
+/**
+ * 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.metadata.impl.stats;
+
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public final class MetadataStoreStats implements AutoCloseable {
+    private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
+    private static final String LABEL_NAME = "metadata_store_name";
+
+    private static final Histogram GET_OPS_SUCCEED = Histogram
+            .build("pulsar_metadata_store_get_ops_latency", "-")

Review Comment:
   > It's better to create a prefix pulsar_metadata_store_,
   
   @tjiuming Could you please confirm if this one looks good to you? I don't see the update and comment here.



-- 
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


[GitHub] [pulsar] codelipenghui commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r944061144


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java:
##########
@@ -126,6 +128,8 @@ public CompletableFuture<Boolean> asyncReload(String key, Boolean oldValue,
                         }
                     }
                 });
+
+        this.stats = MetadataStoreStats.create();

Review Comment:
   For this case, does the checkstyle plugin can help? Some use `this.`, and some are not. 
   



-- 
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


[GitHub] [pulsar] asafm commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
asafm commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r973681673


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStoreConfig.java:
##########
@@ -29,6 +29,9 @@
 @Getter
 @ToString
 public class MetadataStoreConfig {
+    public static final String LOCAL_METADATA_STORE = "local-metadata-store";
+    public static final String STATE_METADATA_STORE = "state-metadata-store";

Review Comment:
   >And I would suggest changing "local-metadata-store" -> "metadata-store", just to keep consistent with the configuration name that we have in the `broker.conf`.
   I agree with that. Today the user knows this from configuration: `metadataStoreUrl` and `configurationMetadataStoreUrl` so when they see a label name, they need to be able to match it with what they configured, no?



##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java:
##########
@@ -80,9 +85,9 @@ public abstract class AbstractMetadataStore implements MetadataStoreExtended, Co
 
     protected abstract CompletableFuture<Boolean> existsFromStore(String path);
 
-    protected AbstractMetadataStore() {
-        this.executor = Executors
-                .newSingleThreadScheduledExecutor(new DefaultThreadFactory("metadata-store"));
+    protected AbstractMetadataStore(String metadataStoreName) {
+        final String poolName = "metadata-store";
+        this.executor = new ScheduledThreadPoolExecutor(1, new DefaultThreadFactory(poolName));

Review Comment:
   Shouldn't the poolName be as metadataStore name?



##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java:
##########
@@ -1522,6 +1524,81 @@ public void testSplitTopicAndPartitionLabel() throws Exception {
         consumer2.close();
     }
 
+
+    @Test
+    public void testMetadataStoreStats() throws Exception {

Review Comment:
   You resolved without answering 



##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -0,0 +1,110 @@
+/**
+ * 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.metadata.impl.stats;
+
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public final class MetadataStoreStats implements AutoCloseable {
+    private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
+    private static final String OPS_TYPE_LABEL_NAME = "type";
+    private static final String METADATA_STORE_LABEL_NAME = "name";
+    private static final String STATUS = "status";
+
+    private static final String OPS_TYPE_GET = "get";
+    private static final String OPS_TYPE_DEL = "del";
+    private static final String OPS_TYPE_PUT = "put";
+    private static final String STATUS_SUCCESS = "success";
+    private static final String STATUS_FAIL = "fail";
+
+    protected static final String PREFIX = "pulsar_metadata_store_";
+
+    private static final Histogram OPS_LATENCY = Histogram
+            .build(PREFIX + "ops_latency", "-")
+            .unit("ms")
+            .buckets(BUCKETS)
+            .labelNames(METADATA_STORE_LABEL_NAME, OPS_TYPE_LABEL_NAME, STATUS)
+            .register();
+    private static final Counter PUT_BYTES = Counter
+            .build(PREFIX + "put", "-")
+            .unit("bytes")
+            .labelNames(METADATA_STORE_LABEL_NAME)
+            .register();
+
+    private final Histogram.Child getOpsSucceedChild;
+    private final Histogram.Child delOpsSucceedChild;
+    private final Histogram.Child putOpsSucceedChild;
+    private final Histogram.Child getOpsFailedChild;
+    private final Histogram.Child delOpsFailedChild;
+    private final Histogram.Child putOpsFailedChild;
+    private final Counter.Child putBytesChild;
+    private final String metadataStoreName;
+    private final AtomicBoolean closed = new AtomicBoolean(false);
+
+    public MetadataStoreStats(String metadataStoreName) {
+        this.metadataStoreName = metadataStoreName;
+
+        this.getOpsSucceedChild = OPS_LATENCY.labels(metadataStoreName, OPS_TYPE_GET, STATUS_SUCCESS);
+        this.delOpsSucceedChild = OPS_LATENCY.labels(metadataStoreName, OPS_TYPE_DEL, STATUS_SUCCESS);
+        this.putOpsSucceedChild = OPS_LATENCY.labels(metadataStoreName, OPS_TYPE_PUT, STATUS_SUCCESS);
+        this.getOpsFailedChild = OPS_LATENCY.labels(metadataStoreName, OPS_TYPE_GET, STATUS_FAIL);
+        this.delOpsFailedChild = OPS_LATENCY.labels(metadataStoreName, OPS_TYPE_DEL, STATUS_FAIL);
+        this.putOpsFailedChild = OPS_LATENCY.labels(metadataStoreName, OPS_TYPE_PUT, STATUS_FAIL);
+        this.putBytesChild = PUT_BYTES.labels(metadataStoreName);
+    }
+
+    public void recordGetOpsLatency(long millis) {

Review Comment:
   I know we had this discussion before, but I'm just adding a small note here before this PR gets merged: don't you prefer for consistency's sake to have `recordGetOpsSucceeded` which matches with `recordGetOpsFailed`? 



-- 
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


[GitHub] [pulsar] codelipenghui commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r972893211


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java:
##########
@@ -58,18 +59,22 @@
 import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended;
 import org.apache.pulsar.metadata.api.extended.SessionEvent;
 import org.apache.pulsar.metadata.cache.impl.MetadataCacheImpl;
+import org.apache.pulsar.metadata.impl.stats.MetadataStoreStats;
 
 @Slf4j
 public abstract class AbstractMetadataStore implements MetadataStoreExtended, Consumer<Notification> {
 
+    private static final AtomicInteger ID = new AtomicInteger();

Review Comment:
   We don't need this one any more?



##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStoreConfig.java:
##########
@@ -29,6 +29,9 @@
 @Getter
 @ToString
 public class MetadataStoreConfig {
+    public static final String LOCAL_METADATA_STORE = "local-metadata-store";
+    public static final String STATE_METADATA_STORE = "state-metadata-store";

Review Comment:
   Interesting, I don't know we have a zookeeper-based function state store
   
   I think we don't need to have a separate name for this one? because it will use the same zookeeper with the metadata store right?
   
   And I would suggest changing "local-metadata-store" -> "metadata-store", just to keep consistent with the configuration name that we have in the broker.conf.
   



##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java:
##########
@@ -80,9 +85,9 @@ public abstract class AbstractMetadataStore implements MetadataStoreExtended, Co
 
     protected abstract CompletableFuture<Boolean> existsFromStore(String path);
 
-    protected AbstractMetadataStore() {
-        this.executor = Executors
-                .newSingleThreadScheduledExecutor(new DefaultThreadFactory("metadata-store"));
+    protected AbstractMetadataStore(String metadataStoreName) {
+        final String poolName = "metadata-store";
+        this.executor = new ScheduledThreadPoolExecutor(1, new DefaultThreadFactory(poolName));

Review Comment:
   We don't need this change?



-- 
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


[GitHub] [pulsar] asafm commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
asafm commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r959406647


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java:
##########
@@ -235,10 +239,20 @@ public <T> MetadataCache<T> getMetadataCache(MetadataSerde<T> serde) {
 
     @Override
     public CompletableFuture<Optional<GetResult>> get(String path) {
+        long start = System.currentTimeMillis();
         if (!isValidPath(path)) {
-            return FutureUtil.failedFuture(new MetadataStoreException.InvalidPathException(path));
+            stats.recordGetOpsFailed();
+            return FutureUtil
+                    .failedFuture(new MetadataStoreException.InvalidPathException(path));
         }
-        return storeGet(path);
+        return storeGet(path)
+                .whenComplete((v, t) -> {
+                    if (t != null) {
+                        stats.recordGetOpsFailed();
+                    } else {
+                        stats.recordGetOpsLatency(System.currentTimeMillis() - start);

Review Comment:
   I introduced a new comment iterating "2. You want latency also for failed ops - sometimes they are the root cause - seeing spike in latency will explain problems."



-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r960876919


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -0,0 +1,109 @@
+/**
+ * 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.metadata.impl.stats;
+
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public final class MetadataStoreStats implements AutoCloseable {
+    private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
+    private static final String OPS_TYPE_LABEL_NAME = "type";
+    private static final String METADATA_STORE_LABEL_NAME = "name";
+    private static final String OPS_TYPE_GET = "get";
+    private static final String OPS_TYPE_DEL = "del";
+    private static final String OPS_TYPE_PUT = "put";
+    protected static final String PREFIX = "pulsar_metadata_store_";
+
+    private static final Histogram OPS_SUCCEED = Histogram
+            .build(PREFIX + "ops_latency", "-")
+            .unit("ms")
+            .buckets(BUCKETS)
+            .labelNames(METADATA_STORE_LABEL_NAME, OPS_TYPE_LABEL_NAME)
+            .register();
+    private static final Counter OPS_FAILED = Counter
+            .build(PREFIX + "ops_failed", "-")
+            .labelNames(METADATA_STORE_LABEL_NAME, OPS_TYPE_LABEL_NAME)
+            .register();
+    private static final Counter PUT_BYTES = Counter
+            .build(PREFIX + "put_", "-")
+            .unit("bytes")
+            .labelNames(METADATA_STORE_LABEL_NAME)
+            .register();
+
+    private final Histogram.Child getOpsSucceedChild;
+    private final Histogram.Child delOpsSucceedChild;
+    private final Histogram.Child putOpsSucceedChild;
+    private final Counter.Child getOpsFailedChild;
+    private final Counter.Child delOpsFailedChild;
+    private final Counter.Child putOpsFailedChild;
+    private final Counter.Child putBytesChild;
+    private final String metadataStoreName;
+    private final AtomicBoolean closed = new AtomicBoolean(false);
+
+    public MetadataStoreStats(String metadataStoreName) {
+        this.metadataStoreName = metadataStoreName;
+
+        this.getOpsSucceedChild = OPS_SUCCEED.labels(metadataStoreName, OPS_TYPE_GET);
+        this.delOpsSucceedChild = OPS_SUCCEED.labels(metadataStoreName, OPS_TYPE_DEL);
+        this.putOpsSucceedChild = OPS_SUCCEED.labels(metadataStoreName, OPS_TYPE_PUT);
+        this.getOpsFailedChild = OPS_FAILED.labels(metadataStoreName, OPS_TYPE_GET);
+        this.delOpsFailedChild = OPS_FAILED.labels(metadataStoreName, OPS_TYPE_DEL);
+        this.putOpsFailedChild = OPS_FAILED.labels(metadataStoreName, OPS_TYPE_PUT);
+        this.putBytesChild = PUT_BYTES.labels(metadataStoreName);
+    }
+
+    public void recordGetOpsLatency(long millis) {
+        this.getOpsSucceedChild.observe(millis);
+    }
+
+    public void recordDelOpsLatency(long millis) {
+        this.delOpsSucceedChild.observe(millis);
+    }
+
+    public void recordPutOpsLatency(long millis, int bytes) {
+        this.putOpsSucceedChild.observe(millis);
+        this.putBytesChild.inc(bytes);
+    }
+
+    public void recordGetOpsFailed() {
+        this.getOpsFailedChild.inc();
+    }
+
+    public void recordDelOpsFailed() {
+        this.delOpsFailedChild.inc();
+    }
+
+    public void recordPutOpsFailed() {
+        this.putOpsFailedChild.inc();
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (this.closed.compareAndSet(false, true)) {

Review Comment:
   of cause it's unnecessary in non-test code, but according to my experience, everything happens in tests, this is to prevent ci tests failure



-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r960861522


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -76,12 +60,12 @@ public final class MetadataStoreStats implements AutoCloseable {
     public MetadataStoreStats(String metadataStoreName) {
         this.metadataStoreName = metadataStoreName;
 
-        this.getOpsSucceedChild = GET_OPS_SUCCEED.labels(metadataStoreName);
-        this.delOpsSucceedChild = DEL_OPS_SUCCEED.labels(metadataStoreName);
-        this.puttOpsSucceedChild = PUT_OPS_SUCCEED.labels(metadataStoreName);
-        this.getOpsFailedChild = GET_OPS_FAILED.labels(metadataStoreName);
-        this.delOpsFailedChild = DEL_OPS_FAILED.labels(metadataStoreName);
-        this.putOpsFailedChild = PUT_OPS_FAILED.labels(metadataStoreName);
+        this.getOpsSucceedChild = OPS_SUCCEED.labels(metadataStoreName, OPS_TYPE_GET);

Review Comment:
   personal code style



-- 
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


[GitHub] [pulsar] asafm commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
asafm commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r962308022


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java:
##########
@@ -1522,6 +1524,81 @@ public void testSplitTopicAndPartitionLabel() throws Exception {
         consumer2.close();
     }
 
+
+    @Test
+    public void testMetadataStoreStats() throws Exception {

Review Comment:
   The file `PrometheusMetricsTest.java` is already very large.
   I believe making it was a mistake, since creating a class that test, so many things leads to huge unreadable class.
   
   I recommend moving those tests to `MetadataStoreStatsTest`
   



-- 
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


[GitHub] [pulsar] asafm commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
asafm commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r962302834


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java:
##########
@@ -1557,49 +1557,40 @@ public void testMetadataStoreStats() throws Exception {
         String metricsStr = output.toString();
         Multimap<String, Metric> metricsMap = parseMetrics(metricsStr);
 
-        Collection<Metric> getOpsFailed = metricsMap.get("pulsar_metadata_store_get_failed" + "_total");
-        Collection<Metric> delOpsFailed = metricsMap.get("pulsar_metadata_store_del_failed" + "_total");
-        Collection<Metric> putOpsFailed = metricsMap.get("pulsar_metadata_store_put_failed" + "_total");
-
-        Collection<Metric> getOpsLatency = metricsMap.get("pulsar_metadata_store_get_latency_ms" + "_sum");
-        Collection<Metric> delOpsLatency = metricsMap.get("pulsar_metadata_store_del_latency_ms" + "_sum");
-        Collection<Metric> putOpsLatency = metricsMap.get("pulsar_metadata_store_put_latency_ms" + "_sum");
-
+        Collection<Metric> opsFailed = metricsMap.get("pulsar_metadata_store_ops_failed" + "_total");
+        Collection<Metric> opsLatency = metricsMap.get("pulsar_metadata_store_ops_latency_ms" + "_sum");
         Collection<Metric> putBytes = metricsMap.get("pulsar_metadata_store_put_bytes" + "_total");
 
-        Assert.assertTrue(getOpsFailed.size() > 1);
-        Assert.assertTrue(delOpsFailed.size() > 1);
-        Assert.assertTrue(putOpsFailed.size() > 1);
-        Assert.assertTrue(getOpsLatency.size() > 1);
-        Assert.assertTrue(delOpsLatency.size() > 1);
-        Assert.assertTrue(putOpsLatency.size() > 1);
+        Assert.assertTrue(opsFailed.size() > 1);

Review Comment:
   I don't understand. You can introduce new dependency if you want, no?



-- 
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


[GitHub] [pulsar] asafm commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
asafm commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r973681673


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStoreConfig.java:
##########
@@ -29,6 +29,9 @@
 @Getter
 @ToString
 public class MetadataStoreConfig {
+    public static final String LOCAL_METADATA_STORE = "local-metadata-store";
+    public static final String STATE_METADATA_STORE = "state-metadata-store";

Review Comment:
   >And I would suggest changing "local-metadata-store" -> "metadata-store", just to keep consistent with the configuration name that we have in the `broker.conf`.
   
   I agree with that. Today the user knows this from configuration: `metadataStoreUrl` and `configurationMetadataStoreUrl` so when they see a label name, they need to be able to match it with what they configured, no?



-- 
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


[GitHub] [pulsar] asafm commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
asafm commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r963426402


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -0,0 +1,109 @@
+/**
+ * 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.metadata.impl.stats;
+
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public final class MetadataStoreStats implements AutoCloseable {
+    private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
+    private static final String OPS_TYPE_LABEL_NAME = "type";
+    private static final String METADATA_STORE_LABEL_NAME = "name";
+    private static final String OPS_TYPE_GET = "get";
+    private static final String OPS_TYPE_DEL = "del";
+    private static final String OPS_TYPE_PUT = "put";
+    protected static final String PREFIX = "pulsar_metadata_store_";
+
+    private static final Histogram OPS_SUCCEED = Histogram
+            .build(PREFIX + "ops_latency", "-")
+            .unit("ms")
+            .buckets(BUCKETS)
+            .labelNames(METADATA_STORE_LABEL_NAME, OPS_TYPE_LABEL_NAME)
+            .register();
+    private static final Counter OPS_FAILED = Counter
+            .build(PREFIX + "ops_failed", "-")
+            .labelNames(METADATA_STORE_LABEL_NAME, OPS_TYPE_LABEL_NAME)
+            .register();
+    private static final Counter PUT_BYTES = Counter
+            .build(PREFIX + "put_", "-")
+            .unit("bytes")
+            .labelNames(METADATA_STORE_LABEL_NAME)
+            .register();
+
+    private final Histogram.Child getOpsSucceedChild;
+    private final Histogram.Child delOpsSucceedChild;
+    private final Histogram.Child putOpsSucceedChild;
+    private final Counter.Child getOpsFailedChild;
+    private final Counter.Child delOpsFailedChild;
+    private final Counter.Child putOpsFailedChild;
+    private final Counter.Child putBytesChild;
+    private final String metadataStoreName;
+    private final AtomicBoolean closed = new AtomicBoolean(false);
+
+    public MetadataStoreStats(String metadataStoreName) {
+        this.metadataStoreName = metadataStoreName;
+
+        this.getOpsSucceedChild = OPS_SUCCEED.labels(metadataStoreName, OPS_TYPE_GET);
+        this.delOpsSucceedChild = OPS_SUCCEED.labels(metadataStoreName, OPS_TYPE_DEL);
+        this.putOpsSucceedChild = OPS_SUCCEED.labels(metadataStoreName, OPS_TYPE_PUT);
+        this.getOpsFailedChild = OPS_FAILED.labels(metadataStoreName, OPS_TYPE_GET);
+        this.delOpsFailedChild = OPS_FAILED.labels(metadataStoreName, OPS_TYPE_DEL);
+        this.putOpsFailedChild = OPS_FAILED.labels(metadataStoreName, OPS_TYPE_PUT);
+        this.putBytesChild = PUT_BYTES.labels(metadataStoreName);
+    }
+
+    public void recordGetOpsLatency(long millis) {
+        this.getOpsSucceedChild.observe(millis);
+    }
+
+    public void recordDelOpsLatency(long millis) {
+        this.delOpsSucceedChild.observe(millis);
+    }
+
+    public void recordPutOpsLatency(long millis, int bytes) {
+        this.putOpsSucceedChild.observe(millis);
+        this.putBytesChild.inc(bytes);
+    }
+
+    public void recordGetOpsFailed() {
+        this.getOpsFailedChild.inc();
+    }
+
+    public void recordDelOpsFailed() {
+        this.delOpsFailedChild.inc();
+    }
+
+    public void recordPutOpsFailed() {
+        this.putOpsFailedChild.inc();
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (this.closed.compareAndSet(false, true)) {

Review Comment:
   If you throw `RuntimeException`, you are not required to change, no? Or do you mean, there are many tests that call close() twice, which will cause your build to fail on many tests?



-- 
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


[GitHub] [pulsar] asafm commented on pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
asafm commented on PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#issuecomment-1238253019

   > > > for 1, if I change protected AbstractMetadataStore() to protected AbstractMetadataStore(String metadataStoreName),
   > > 
   > > 
   > > Maybe I misunderstand. You have 5 classes that extend `AbstractMetadataStore`, which are not abstract, right? If I understand correctly, you only need to modify them, no?
   > 
   > not only them, `Factory` and related tests also need to fix. Toooooo many tests referenced it
   
   Ok, I took a closer look at the code, and you are correct. You fix the factory, and give the proper names (apparently instances of metadata stores are created in numerous places, tests excluded), but you end with at least 67 places in the tests that instantiate metadata using the factory and those needs to be fixed.
   
   But, it got me thinking: What is the purpose of this PR? We want to know the operations count and latency on the metadata stores. How do we want them broken down? 
   * Purpose? Pulsar Metdata Store, Pulsar Configuration Store, BK metadata store, etc...
   * Type? ZK, etcD?
   * component: metadata store, global configuration store.
   
   What will help the operator understand and pinpoint? 
   
   In the current implementation in this PR, you don't know which metadata store it refers to: Is the metadata store or configuration store? Is it the metadata store used by the BK client, or something else?
   
   What you see is "metadata-store-0" which tells you nothing.
   If it reveals nothing, maybe we don't need it at all - the name that is.
   
   So that's why I'm asking the questions above. 
   I think those questions are important.
   
   
   
   


-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r943692642


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java:
##########
@@ -126,6 +128,8 @@ public CompletableFuture<Boolean> asyncReload(String key, Boolean oldValue,
                         }
                     }
                 });
+
+        this.stats = MetadataStoreStats.create();

Review Comment:
   it's not an issue



-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r944509577


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -0,0 +1,124 @@
+/**
+ * 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.metadata.impl.stats;
+
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public final class MetadataStoreStats implements AutoCloseable {
+    private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
+    private static final String LABEL_NAME = "metadata_store_name";
+
+    private static final Histogram GET_OPS_SUCCEED = Histogram
+            .build("pulsar_metadata_store_get_ops_latency", "-")

Review Comment:
   @codelipenghui I think it doesn't matter. do you mean like this ?
   ```
   final String prefix = "pulsar_metadata_store_";
   Histogram GET_OPS_SUCCEED = Histogram
               .build( prefix + "get_latency", "-");
   Histogram DEL_OPS_SUCCEED = Histogram
               .build( prefix + "del_latency", "-");
   ```



-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r943863052


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -0,0 +1,92 @@
+/**
+ * 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.metadata.impl.stats;
+
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public final class MetadataStoreStats {
+    private static final AtomicBoolean INITIALIZED = new AtomicBoolean(false);
+    private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
+
+    private final Histogram getOpsLatency;
+    private final Histogram delOpsLatency;
+    private final Histogram putOpsLatency;
+    private final Counter getFailedCounter;
+    private final Counter delFailedCounter;
+    private final Counter putFailedCounter;
+    private final Counter putBytesCounter;
+
+    private MetadataStoreStats() {
+        getOpsLatency = Histogram.build("pulsar_metadata_store_get_ops_latency", "-")

Review Comment:
   sorry, I need to set unit = ms



-- 
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


[GitHub] [pulsar] asafm commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
asafm commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r943389948


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java:
##########
@@ -126,6 +128,8 @@ public CompletableFuture<Boolean> asyncReload(String key, Boolean oldValue,
                         }
                     }
                 });
+
+        this.stats = MetadataStoreStats.create();

Review Comment:
   why `this.`? why note `stats = ` No other `stats` argument here.
   



##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -0,0 +1,92 @@
+/**
+ * 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.metadata.impl.stats;
+
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public final class MetadataStoreStats {
+    private static final AtomicBoolean INITIALIZED = new AtomicBoolean(false);
+    private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
+
+    private final Histogram getOpsLatency;
+    private final Histogram delOpsLatency;
+    private final Histogram putOpsLatency;
+    private final Counter getFailedCounter;
+    private final Counter delFailedCounter;
+    private final Counter putFailedCounter;
+    private final Counter putBytesCounter;
+
+    private MetadataStoreStats() {
+        getOpsLatency = Histogram.build("pulsar_metadata_store_get_ops_latency", "-")

Review Comment:
   You're missing units in name `pulsar_metadata_store_get_ops_latency` --> `pulsar_metadata_store_get_ops_latency_ms`



##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java:
##########
@@ -1522,6 +1517,83 @@ public void testSplitTopicAndPartitionLabel() throws Exception {
         consumer2.close();
     }
 
+
+    @Test
+    public void testMetadataStoreStats() throws Exception {
+        String ns = "prop/ns-abc1";
+        admin.namespaces().createNamespace(ns);
+
+        String topic = "persistent://prop/ns-abc1/metadata-store-" + UUID.randomUUID();
+        String subName = "my-sub1";
+
+        @Cleanup
+        Producer<String> producer = pulsarClient.newProducer(Schema.STRING)
+                .topic(topic).create();
+        @Cleanup
+        Consumer<String> consumer = pulsarClient.newConsumer(Schema.STRING)
+                .topic(topic).subscriptionName(subName).subscribe();
+
+        for (int i = 0; i < 100; i++) {
+            producer.newMessage().value(UUID.randomUUID().toString()).send();
+        }
+
+        for (;;) {
+            Message<String> message = consumer.receive(10, TimeUnit.SECONDS);
+            if (message == null) {
+                break;
+            }
+            consumer.acknowledge(message);
+        }
+
+        ByteArrayOutputStream output = new ByteArrayOutputStream();
+        PrometheusMetricsGenerator.generate(pulsar, false, false, false, false, output);
+        String metricsStr = output.toString();
+        Multimap<String, Metric> metricsMap = parseMetrics(metricsStr);
+
+        Collection<Metric> getOpsFailed = metricsMap.get("pulsar_metadata_store_get_ops_failed" + "_total");
+        Collection<Metric> delOpsFailed = metricsMap.get("pulsar_metadata_store_del_ops_failed" + "_total");
+        Collection<Metric> putOpsFailed = metricsMap.get("pulsar_metadata_store_put_ops_failed" + "_total");
+
+        Collection<Metric> getOpsLatency = metricsMap.get("pulsar_metadata_store_get_ops_latency" + "_sum");
+        Collection<Metric> delOpsLatency = metricsMap.get("pulsar_metadata_store_del_ops_latency" + "_sum");
+        Collection<Metric> putOpsLatency = metricsMap.get("pulsar_metadata_store_put_ops_latency" + "_sum");
+
+        Collection<Metric> putBytes = metricsMap.get("pulsar_metadata_store_put_bytes" + "_total");
+
+        Assert.assertTrue(getOpsFailed.size() > 0);
+        Assert.assertTrue(delOpsFailed.size() > 0);
+        Assert.assertTrue(putOpsFailed.size() > 0);
+        Assert.assertTrue(getOpsLatency.size() > 0);
+        Assert.assertTrue(delOpsLatency.size() > 0);
+        Assert.assertTrue(putOpsLatency.size() > 0);
+        Assert.assertTrue(putBytes.size() > 0);
+
+        for (Metric m : getOpsFailed) {
+            Assert.assertEquals(m.tags.get("cluster"), "test");

Review Comment:
   Why are you not checking the value of this sample?



##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java:
##########
@@ -235,10 +239,20 @@ public <T> MetadataCache<T> getMetadataCache(MetadataSerde<T> serde) {
 
     @Override
     public CompletableFuture<Optional<GetResult>> get(String path) {
+        long start = System.currentTimeMillis();
         if (!isValidPath(path)) {
-            return FutureUtil.failedFuture(new MetadataStoreException.InvalidPathException(path));
+            stats.recordGetOpsFailed();
+            return FutureUtil
+                    .failedFuture(new MetadataStoreException.InvalidPathException(path));
         }
-        return storeGet(path);
+        return storeGet(path)
+                .whenComplete((v, t) -> {
+                    if (t != null) {
+                        stats.recordGetOpsFailed();
+                    } else {
+                        stats.recordGetOpsLatency(System.currentTimeMillis() - start);

Review Comment:
   This method name is inconsistent.
   I suggest
   ```
   stats.recordGetOpsFailed(opLatency);
   status.recordGetOpsSucceeded(opLatency)
   ```
   
   1. You want consistent naming across failure and success
   2. You want latency also for failed ops - sometimes they are the root cause - seeing spike in latency will explain problems.
   



##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -0,0 +1,92 @@
+/**
+ * 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.metadata.impl.stats;
+
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public final class MetadataStoreStats {
+    private static final AtomicBoolean INITIALIZED = new AtomicBoolean(false);
+    private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
+
+    private final Histogram getOpsLatency;
+    private final Histogram delOpsLatency;
+    private final Histogram putOpsLatency;
+    private final Counter getFailedCounter;
+    private final Counter delFailedCounter;
+    private final Counter putFailedCounter;
+    private final Counter putBytesCounter;
+
+    private MetadataStoreStats() {
+        getOpsLatency = Histogram.build("pulsar_metadata_store_get_ops_latency", "-")
+                .buckets(BUCKETS)
+                .register();
+        delOpsLatency = Histogram.build("pulsar_metadata_store_del_ops_latency", "-")
+                .buckets(BUCKETS)
+                .register();
+        putOpsLatency = Histogram.build("pulsar_metadata_store_put_ops_latency", "-")
+                .buckets(BUCKETS)
+                .register();
+
+        getFailedCounter = Counter.build("pulsar_metadata_store_get_ops_failed", "-")
+                .register();
+        delFailedCounter = Counter.build("pulsar_metadata_store_del_ops_failed", "-")
+                .register();
+        putFailedCounter = Counter.build("pulsar_metadata_store_put_ops_failed", "-")
+                .register();
+        putBytesCounter = Counter.build("pulsar_metadata_store_put_bytes", "-")
+                .register();
+    }
+
+    public void recordGetOpsLatency(long millis) {
+        this.getOpsLatency.observe(millis);

Review Comment:
   ```suggestion
          getOpsLatency.observe(millis);
   ```



##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java:
##########
@@ -1522,6 +1517,83 @@ public void testSplitTopicAndPartitionLabel() throws Exception {
         consumer2.close();
     }
 
+
+    @Test
+    public void testMetadataStoreStats() throws Exception {
+        String ns = "prop/ns-abc1";
+        admin.namespaces().createNamespace(ns);
+
+        String topic = "persistent://prop/ns-abc1/metadata-store-" + UUID.randomUUID();
+        String subName = "my-sub1";
+
+        @Cleanup
+        Producer<String> producer = pulsarClient.newProducer(Schema.STRING)
+                .topic(topic).create();
+        @Cleanup
+        Consumer<String> consumer = pulsarClient.newConsumer(Schema.STRING)
+                .topic(topic).subscriptionName(subName).subscribe();
+
+        for (int i = 0; i < 100; i++) {
+            producer.newMessage().value(UUID.randomUUID().toString()).send();
+        }
+
+        for (;;) {
+            Message<String> message = consumer.receive(10, TimeUnit.SECONDS);
+            if (message == null) {
+                break;
+            }
+            consumer.acknowledge(message);
+        }
+
+        ByteArrayOutputStream output = new ByteArrayOutputStream();
+        PrometheusMetricsGenerator.generate(pulsar, false, false, false, false, output);
+        String metricsStr = output.toString();
+        Multimap<String, Metric> metricsMap = parseMetrics(metricsStr);
+
+        Collection<Metric> getOpsFailed = metricsMap.get("pulsar_metadata_store_get_ops_failed" + "_total");
+        Collection<Metric> delOpsFailed = metricsMap.get("pulsar_metadata_store_del_ops_failed" + "_total");
+        Collection<Metric> putOpsFailed = metricsMap.get("pulsar_metadata_store_put_ops_failed" + "_total");
+
+        Collection<Metric> getOpsLatency = metricsMap.get("pulsar_metadata_store_get_ops_latency" + "_sum");
+        Collection<Metric> delOpsLatency = metricsMap.get("pulsar_metadata_store_del_ops_latency" + "_sum");
+        Collection<Metric> putOpsLatency = metricsMap.get("pulsar_metadata_store_put_ops_latency" + "_sum");
+
+        Collection<Metric> putBytes = metricsMap.get("pulsar_metadata_store_put_bytes" + "_total");
+
+        Assert.assertTrue(getOpsFailed.size() > 0);

Review Comment:
   import static Assert



##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -0,0 +1,92 @@
+/**
+ * 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.metadata.impl.stats;
+
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public final class MetadataStoreStats {
+    private static final AtomicBoolean INITIALIZED = new AtomicBoolean(false);
+    private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
+
+    private final Histogram getOpsLatency;
+    private final Histogram delOpsLatency;
+    private final Histogram putOpsLatency;
+    private final Counter getFailedCounter;
+    private final Counter delFailedCounter;
+    private final Counter putFailedCounter;
+    private final Counter putBytesCounter;
+
+    private MetadataStoreStats() {
+        getOpsLatency = Histogram.build("pulsar_metadata_store_get_ops_latency", "-")
+                .buckets(BUCKETS)
+                .register();
+        delOpsLatency = Histogram.build("pulsar_metadata_store_del_ops_latency", "-")
+                .buckets(BUCKETS)
+                .register();
+        putOpsLatency = Histogram.build("pulsar_metadata_store_put_ops_latency", "-")
+                .buckets(BUCKETS)
+                .register();
+
+        getFailedCounter = Counter.build("pulsar_metadata_store_get_ops_failed", "-")
+                .register();
+        delFailedCounter = Counter.build("pulsar_metadata_store_del_ops_failed", "-")
+                .register();
+        putFailedCounter = Counter.build("pulsar_metadata_store_put_ops_failed", "-")
+                .register();
+        putBytesCounter = Counter.build("pulsar_metadata_store_put_bytes", "-")
+                .register();
+    }
+
+    public void recordGetOpsLatency(long millis) {
+        this.getOpsLatency.observe(millis);
+    }
+
+    public void recordDelOpsLatency(long millis) {
+        this.delOpsLatency.observe(millis);
+    }
+
+    public void recordPutOpsLatency(long millis, int bytes) {
+        this.putOpsLatency.observe(millis);
+        this.putBytesCounter.inc(bytes);
+    }
+
+    public void recordGetOpsFailed() {
+        this.getFailedCounter.inc();
+    }
+
+    public void recordDelOpsFailed() {
+        this.delFailedCounter.inc();
+    }
+
+    public void recordPutOpsFailed() {
+        this.putFailedCounter.inc();
+    }
+
+    private static MetadataStoreStats instance;

Review Comment:
   1. static member should be located at the beginning of the class.
   2. static variables should be in uppercase.
   3. There is an easier approach - eager init:
   ```java
   private static MetadataStoreStats INSTANCE = new MetadataStoreStats();
   
   //
   //...
   //
   
   public static MetadataStoreStats getInstance() {
       return INSTANCE;
   }
   ```



-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r944509577


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -0,0 +1,124 @@
+/**
+ * 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.metadata.impl.stats;
+
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public final class MetadataStoreStats implements AutoCloseable {
+    private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
+    private static final String LABEL_NAME = "metadata_store_name";
+
+    private static final Histogram GET_OPS_SUCCEED = Histogram
+            .build("pulsar_metadata_store_get_ops_latency", "-")

Review Comment:
   @codelipenghui I think it's doesn't matter. do you mean like this ?
   ```
   final String prefix = "pulsar_metadata_store_";
   Histogram GET_OPS_SUCCEED = Histogram
               .build( prefix + "get_latency", "-");
   Histogram DEL_OPS_SUCCEED = Histogram
               .build( prefix + "del_latency", "-");
   ```



-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r952674026


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java:
##########
@@ -81,8 +86,8 @@ public abstract class AbstractMetadataStore implements MetadataStoreExtended, Co
     protected abstract CompletableFuture<Boolean> existsFromStore(String path);
 
     protected AbstractMetadataStore() {
-        this.executor = Executors
-                .newSingleThreadScheduledExecutor(new DefaultThreadFactory("metadata-store"));
+        final String poolName = "metadata-store";
+        this.executor = new ScheduledThreadPoolExecutor(1, new DefaultThreadFactory(poolName));

Review Comment:
   yes, it's useful to https://github.com/apache/pulsar/pull/17072, I want to monitor how many tasks pending in the Task Queue.



-- 
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


[GitHub] [pulsar] asafm commented on pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
asafm commented on PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#issuecomment-1232745502

   > @asafm Please help review this PR
   
   @tjiuming done
   


-- 
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


[GitHub] [pulsar] asafm commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
asafm commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r963701217


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java:
##########
@@ -1522,6 +1524,81 @@ public void testSplitTopicAndPartitionLabel() throws Exception {
         consumer2.close();
     }
 
+
+    @Test
+    public void testMetadataStoreStats() throws Exception {

Review Comment:
   Why moving `testMetadataStoreStats` to a new class called `MetadataStoreStatsTest` will prevent you from ensuring all metrics outputs as expected?



-- 
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


[GitHub] [pulsar] asafm commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
asafm commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r963468360


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java:
##########
@@ -1557,49 +1557,40 @@ public void testMetadataStoreStats() throws Exception {
         String metricsStr = output.toString();
         Multimap<String, Metric> metricsMap = parseMetrics(metricsStr);
 
-        Collection<Metric> getOpsFailed = metricsMap.get("pulsar_metadata_store_get_failed" + "_total");
-        Collection<Metric> delOpsFailed = metricsMap.get("pulsar_metadata_store_del_failed" + "_total");
-        Collection<Metric> putOpsFailed = metricsMap.get("pulsar_metadata_store_put_failed" + "_total");
-
-        Collection<Metric> getOpsLatency = metricsMap.get("pulsar_metadata_store_get_latency_ms" + "_sum");
-        Collection<Metric> delOpsLatency = metricsMap.get("pulsar_metadata_store_del_latency_ms" + "_sum");
-        Collection<Metric> putOpsLatency = metricsMap.get("pulsar_metadata_store_put_latency_ms" + "_sum");
-
+        Collection<Metric> opsFailed = metricsMap.get("pulsar_metadata_store_ops_failed" + "_total");
+        Collection<Metric> opsLatency = metricsMap.get("pulsar_metadata_store_ops_latency_ms" + "_sum");
         Collection<Metric> putBytes = metricsMap.get("pulsar_metadata_store_put_bytes" + "_total");
 
-        Assert.assertTrue(getOpsFailed.size() > 1);
-        Assert.assertTrue(delOpsFailed.size() > 1);
-        Assert.assertTrue(putOpsFailed.size() > 1);
-        Assert.assertTrue(getOpsLatency.size() > 1);
-        Assert.assertTrue(delOpsLatency.size() > 1);
-        Assert.assertTrue(putOpsLatency.size() > 1);
+        Assert.assertTrue(opsFailed.size() > 1);

Review Comment:
   I don't mean you will change other tests. If I were to write a new test in Pulsar, I would add `AssertJ` dependency and use it only in my test.



-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r962819868


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java:
##########
@@ -1557,49 +1557,40 @@ public void testMetadataStoreStats() throws Exception {
         String metricsStr = output.toString();
         Multimap<String, Metric> metricsMap = parseMetrics(metricsStr);
 
-        Collection<Metric> getOpsFailed = metricsMap.get("pulsar_metadata_store_get_failed" + "_total");
-        Collection<Metric> delOpsFailed = metricsMap.get("pulsar_metadata_store_del_failed" + "_total");
-        Collection<Metric> putOpsFailed = metricsMap.get("pulsar_metadata_store_put_failed" + "_total");
-
-        Collection<Metric> getOpsLatency = metricsMap.get("pulsar_metadata_store_get_latency_ms" + "_sum");
-        Collection<Metric> delOpsLatency = metricsMap.get("pulsar_metadata_store_del_latency_ms" + "_sum");
-        Collection<Metric> putOpsLatency = metricsMap.get("pulsar_metadata_store_put_latency_ms" + "_sum");
-
+        Collection<Metric> opsFailed = metricsMap.get("pulsar_metadata_store_ops_failed" + "_total");
+        Collection<Metric> opsLatency = metricsMap.get("pulsar_metadata_store_ops_latency_ms" + "_sum");
         Collection<Metric> putBytes = metricsMap.get("pulsar_metadata_store_put_bytes" + "_total");
 
-        Assert.assertTrue(getOpsFailed.size() > 1);
-        Assert.assertTrue(delOpsFailed.size() > 1);
-        Assert.assertTrue(putOpsFailed.size() > 1);
-        Assert.assertTrue(getOpsLatency.size() > 1);
-        Assert.assertTrue(delOpsLatency.size() > 1);
-        Assert.assertTrue(putOpsLatency.size() > 1);
+        Assert.assertTrue(opsFailed.size() > 1);

Review Comment:
   I don't want to make big changes to other stable tests, WDYT?



-- 
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


[GitHub] [pulsar] codelipenghui commented on pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#issuecomment-1249221845

   @asafm Please help review again.


-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r960849226


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java:
##########
@@ -1557,49 +1557,40 @@ public void testMetadataStoreStats() throws Exception {
         String metricsStr = output.toString();
         Multimap<String, Metric> metricsMap = parseMetrics(metricsStr);
 
-        Collection<Metric> getOpsFailed = metricsMap.get("pulsar_metadata_store_get_failed" + "_total");
-        Collection<Metric> delOpsFailed = metricsMap.get("pulsar_metadata_store_del_failed" + "_total");
-        Collection<Metric> putOpsFailed = metricsMap.get("pulsar_metadata_store_put_failed" + "_total");
-
-        Collection<Metric> getOpsLatency = metricsMap.get("pulsar_metadata_store_get_latency_ms" + "_sum");
-        Collection<Metric> delOpsLatency = metricsMap.get("pulsar_metadata_store_del_latency_ms" + "_sum");
-        Collection<Metric> putOpsLatency = metricsMap.get("pulsar_metadata_store_put_latency_ms" + "_sum");
-
+        Collection<Metric> opsFailed = metricsMap.get("pulsar_metadata_store_ops_failed" + "_total");
+        Collection<Metric> opsLatency = metricsMap.get("pulsar_metadata_store_ops_latency_ms" + "_sum");
         Collection<Metric> putBytes = metricsMap.get("pulsar_metadata_store_put_bytes" + "_total");
 
-        Assert.assertTrue(getOpsFailed.size() > 1);
-        Assert.assertTrue(delOpsFailed.size() > 1);
-        Assert.assertTrue(putOpsFailed.size() > 1);
-        Assert.assertTrue(getOpsLatency.size() > 1);
-        Assert.assertTrue(delOpsLatency.size() > 1);
-        Assert.assertTrue(putOpsLatency.size() > 1);
+        Assert.assertTrue(opsFailed.size() > 1);
+        Assert.assertTrue(opsLatency.size() > 1);
         Assert.assertTrue(putBytes.size() > 1);
 
-        for (Metric m : getOpsFailed) {
-            Assert.assertEquals(m.tags.get("cluster"), "test");
-            Assert.assertTrue(m.tags.get("name").startsWith("metadata-store"));
-        }
-        for (Metric m : delOpsFailed) {
-            Assert.assertEquals(m.tags.get("cluster"), "test");
-            Assert.assertTrue(m.tags.get("name").startsWith("metadata-store"));
-        }
-        for (Metric m : putOpsFailed) {
-            Assert.assertEquals(m.tags.get("cluster"), "test");
-            Assert.assertTrue(m.tags.get("name").startsWith("metadata-store"));
-        }
-        for (Metric m : getOpsLatency) {
-            Assert.assertEquals(m.tags.get("cluster"), "test");
-            Assert.assertTrue(m.tags.get("name").startsWith("metadata-store"));
-            Assert.assertTrue(m.value > 0);
-        }
-        for (Metric m : delOpsLatency) {
+        for (Metric m : opsFailed) {
             Assert.assertEquals(m.tags.get("cluster"), "test");
             Assert.assertTrue(m.tags.get("name").startsWith("metadata-store"));
+            if (m.tags.get("type").equals("get")) {
+                Assert.assertTrue(m.value >= 0);

Review Comment:
   personal code style, I prefer `Assert.xxxx`



-- 
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


[GitHub] [pulsar] asafm commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
asafm commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r962306304


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java:
##########
@@ -126,6 +131,9 @@ public CompletableFuture<Boolean> asyncReload(String key, Boolean oldValue,
                         }
                     }
                 });
+
+        this.metadataStoreName = poolName + "-" + ID.getAndIncrement();

Review Comment:
   I guess my question is this: If we have 2 types of metadata stores, one instance of each, why not have the metadata store name be just the type: "metadata-store" and "configuration-metadata-store" - why keep regenerating the name using an auto incremented ID?
   



-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r943695953


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -0,0 +1,92 @@
+/**
+ * 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.metadata.impl.stats;
+
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public final class MetadataStoreStats {
+    private static final AtomicBoolean INITIALIZED = new AtomicBoolean(false);
+    private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
+
+    private final Histogram getOpsLatency;
+    private final Histogram delOpsLatency;
+    private final Histogram putOpsLatency;
+    private final Counter getFailedCounter;
+    private final Counter delFailedCounter;
+    private final Counter putFailedCounter;
+    private final Counter putBytesCounter;
+
+    private MetadataStoreStats() {
+        getOpsLatency = Histogram.build("pulsar_metadata_store_get_ops_latency", "-")
+                .buckets(BUCKETS)
+                .register();
+        delOpsLatency = Histogram.build("pulsar_metadata_store_del_ops_latency", "-")
+                .buckets(BUCKETS)
+                .register();
+        putOpsLatency = Histogram.build("pulsar_metadata_store_put_ops_latency", "-")
+                .buckets(BUCKETS)
+                .register();
+
+        getFailedCounter = Counter.build("pulsar_metadata_store_get_ops_failed", "-")
+                .register();
+        delFailedCounter = Counter.build("pulsar_metadata_store_del_ops_failed", "-")
+                .register();
+        putFailedCounter = Counter.build("pulsar_metadata_store_put_ops_failed", "-")
+                .register();
+        putBytesCounter = Counter.build("pulsar_metadata_store_put_bytes", "-")
+                .register();
+    }
+
+    public void recordGetOpsLatency(long millis) {
+        this.getOpsLatency.observe(millis);
+    }
+
+    public void recordDelOpsLatency(long millis) {
+        this.delOpsLatency.observe(millis);
+    }
+
+    public void recordPutOpsLatency(long millis, int bytes) {
+        this.putOpsLatency.observe(millis);
+        this.putBytesCounter.inc(bytes);
+    }
+
+    public void recordGetOpsFailed() {
+        this.getFailedCounter.inc();
+    }
+
+    public void recordDelOpsFailed() {
+        this.delFailedCounter.inc();
+    }
+
+    public void recordPutOpsFailed() {
+        this.putFailedCounter.inc();
+    }
+
+    private static MetadataStoreStats instance;

Review Comment:
   it's not a static final field, don't need to uppercase, but, it can be static final field



-- 
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


[GitHub] [pulsar] tjiuming commented on pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#issuecomment-1212415160

   @asafm @codelipenghui  there will be more than 1 metadataStore in a broker,  when I develop https://github.com/apache/pulsar/pull/17072, I think record its metrics separately is better, so I've updated the PR. PTAL


-- 
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


[GitHub] [pulsar] codelipenghui commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r944063327


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java:
##########
@@ -65,12 +65,7 @@
 import org.apache.pulsar.broker.service.persistent.PersistentSubscription;
 import org.apache.pulsar.broker.service.persistent.PersistentTopic;
 import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator;
-import org.apache.pulsar.client.api.Consumer;
-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.SubscriptionType;
+import org.apache.pulsar.client.api.*;

Review Comment:
   Please avoid the star import



##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -0,0 +1,124 @@
+/**
+ * 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.metadata.impl.stats;
+
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public final class MetadataStoreStats implements AutoCloseable {
+    private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
+    private static final String LABEL_NAME = "metadata_store_name";

Review Comment:
   ```suggestion
       private static final String LABEL_NAME = "name";
   ```
   
   Can we just use name?  Because the indicator name already have `pulsar_metadata_store` prefix



##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -0,0 +1,124 @@
+/**
+ * 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.metadata.impl.stats;
+
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public final class MetadataStoreStats implements AutoCloseable {
+    private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
+    private static final String LABEL_NAME = "metadata_store_name";
+
+    private static final Histogram GET_OPS_SUCCEED = Histogram
+            .build("pulsar_metadata_store_get_ops_latency", "-")

Review Comment:
   It's better to create a prefix `pulsar_metadata_store_`,
   
   and can we remove "_ops_"? Just keep the name simple. 



##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -0,0 +1,124 @@
+/**
+ * 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.metadata.impl.stats;
+
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public final class MetadataStoreStats implements AutoCloseable {
+    private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
+    private static final String LABEL_NAME = "metadata_store_name";
+
+    private static final Histogram GET_OPS_SUCCEED = Histogram
+            .build("pulsar_metadata_store_get_ops_latency", "-")
+            .unit("ms")
+            .buckets(BUCKETS)
+            .labelNames(LABEL_NAME)
+            .register();
+    private static final Histogram DEL_OPS_SUCCEED = Histogram
+            .build("pulsar_metadata_store_del_ops_latency", "-")
+            .unit("ms")
+            .buckets(BUCKETS)
+            .labelNames(LABEL_NAME)
+            .register();
+    private static final Histogram PUT_OPS_SUCCEED = Histogram
+            .build("pulsar_metadata_store_put_ops_latency", "-")
+            .unit("ms")
+            .buckets(BUCKETS)
+            .labelNames(LABEL_NAME)
+            .register();
+    private static final Counter GET_OPS_FAILED = Counter
+            .build("pulsar_metadata_store_get_ops_failed", "-")
+            .labelNames(LABEL_NAME)
+            .register();
+    private static final Counter DEL_OPS_FAILED = Counter
+            .build("pulsar_metadata_store_del_ops_failed", "-")
+            .labelNames(LABEL_NAME)
+            .register();
+    private static final Counter PUT_OPS_FAILED = Counter
+            .build("pulsar_metadata_store_put_ops_failed", "-")
+            .labelNames(LABEL_NAME)
+            .register();
+    private static final Counter PUT_BYTES = Counter
+            .build("pulsar_metadata_store_put", "-")
+            .unit("byte")

Review Comment:
   Should be `bytes`?



-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r973726806


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java:
##########
@@ -1522,6 +1524,81 @@ public void testSplitTopicAndPartitionLabel() throws Exception {
         consumer2.close();
     }
 
+
+    @Test
+    public void testMetadataStoreStats() throws Exception {

Review Comment:
   sorry I missed ur comment



##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java:
##########
@@ -1522,6 +1524,81 @@ public void testSplitTopicAndPartitionLabel() throws Exception {
         consumer2.close();
     }
 
+
+    @Test
+    public void testMetadataStoreStats() throws Exception {

Review Comment:
   @asafm sorry I missed ur comment



-- 
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


[GitHub] [pulsar] codelipenghui commented on pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#issuecomment-1240795071

   @tjiuming 
   
   > cluster label will be added automatically in PrometheusMetricsGeneratorUtils.generateSystemMetrics(...)
   for metadata store name
   
   Oh, I see. Good to know.
   
   > if I change protected AbstractMetadataStore() to protected AbstractMetadataStore(String name), there will be toooo many places need to fix, WDYT?
   
   I think it's the 100% right thing to do. If we don't have this change, we might need to explain to many users what the name exactly means. And I think that should be an important point that we missed before, if you are troubleshooting problems with the heap dump, it is hard to determine which one is the configuration store and which one is the metastore.


-- 
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


[GitHub] [pulsar] asafm commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
asafm commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r963463126


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java:
##########
@@ -1557,39 +1557,37 @@ public void testMetadataStoreStats() throws Exception {
         String metricsStr = output.toString();
         Multimap<String, Metric> metricsMap = parseMetrics(metricsStr);
 
-        Collection<Metric> opsFailed = metricsMap.get("pulsar_metadata_store_ops_failed" + "_total");
         Collection<Metric> opsLatency = metricsMap.get("pulsar_metadata_store_ops_latency_ms" + "_sum");
         Collection<Metric> putBytes = metricsMap.get("pulsar_metadata_store_put_bytes" + "_total");
 
-        Assert.assertTrue(opsFailed.size() > 1);
         Assert.assertTrue(opsLatency.size() > 1);
         Assert.assertTrue(putBytes.size() > 1);
 
-        for (Metric m : opsFailed) {
-            Assert.assertEquals(m.tags.get("cluster"), "test");
-            Assert.assertTrue(m.tags.get("name").startsWith("metadata-store"));
-            if (m.tags.get("type").equals("get")) {
-                Assert.assertTrue(m.value >= 0);
-            } else if (m.tags.get("type").equals("del")) {
-                Assert.assertTrue(m.value >= 0);
-            } else if (m.tags.get("type").equals("put")) {
-                Assert.assertTrue(m.value >= 0);
-            } else {
-                Assert.fail();
-            }
-        }
-
         for (Metric m : opsLatency) {
             Assert.assertEquals(m.tags.get("cluster"), "test");
             Assert.assertTrue(m.tags.get("name").startsWith("metadata-store"));
-            if (m.tags.get("type").equals("get")) {
-                Assert.assertTrue(m.value > 0);
-            } else if (m.tags.get("type").equals("del")) {
-                Assert.assertTrue(m.value > 0);
-            } else if (m.tags.get("type").equals("put")) {
-                Assert.assertTrue(m.value > 0);
+            Assert.assertNotNull(m.tags.get("status"));
+
+            if (m.tags.get("staus").equals("success")) {

Review Comment:
   `staus` -> `status`



-- 
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


[GitHub] [pulsar] tjiuming commented on pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#issuecomment-1237927278

   > > for 1, if I change protected AbstractMetadataStore() to protected AbstractMetadataStore(String metadataStoreName),
   > 
   > Maybe I misunderstand. You have 5 classes that extend `AbstractMetadataStore`, which are not abstract, right? If I understand correctly, you only need to modify them, no?
   
   not only them, `Factory` and related tests also need to fix. Toooooo many tests referenced it


-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r962485629


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java:
##########
@@ -1557,49 +1557,40 @@ public void testMetadataStoreStats() throws Exception {
         String metricsStr = output.toString();
         Multimap<String, Metric> metricsMap = parseMetrics(metricsStr);
 
-        Collection<Metric> getOpsFailed = metricsMap.get("pulsar_metadata_store_get_failed" + "_total");
-        Collection<Metric> delOpsFailed = metricsMap.get("pulsar_metadata_store_del_failed" + "_total");
-        Collection<Metric> putOpsFailed = metricsMap.get("pulsar_metadata_store_put_failed" + "_total");
-
-        Collection<Metric> getOpsLatency = metricsMap.get("pulsar_metadata_store_get_latency_ms" + "_sum");
-        Collection<Metric> delOpsLatency = metricsMap.get("pulsar_metadata_store_del_latency_ms" + "_sum");
-        Collection<Metric> putOpsLatency = metricsMap.get("pulsar_metadata_store_put_latency_ms" + "_sum");
-
+        Collection<Metric> opsFailed = metricsMap.get("pulsar_metadata_store_ops_failed" + "_total");
+        Collection<Metric> opsLatency = metricsMap.get("pulsar_metadata_store_ops_latency_ms" + "_sum");
         Collection<Metric> putBytes = metricsMap.get("pulsar_metadata_store_put_bytes" + "_total");
 
-        Assert.assertTrue(getOpsFailed.size() > 1);
-        Assert.assertTrue(delOpsFailed.size() > 1);
-        Assert.assertTrue(putOpsFailed.size() > 1);
-        Assert.assertTrue(getOpsLatency.size() > 1);
-        Assert.assertTrue(delOpsLatency.size() > 1);
-        Assert.assertTrue(putOpsLatency.size() > 1);
+        Assert.assertTrue(opsFailed.size() > 1);
+        Assert.assertTrue(opsLatency.size() > 1);
         Assert.assertTrue(putBytes.size() > 1);
 
-        for (Metric m : getOpsFailed) {
-            Assert.assertEquals(m.tags.get("cluster"), "test");
-            Assert.assertTrue(m.tags.get("name").startsWith("metadata-store"));
-        }
-        for (Metric m : delOpsFailed) {
-            Assert.assertEquals(m.tags.get("cluster"), "test");
-            Assert.assertTrue(m.tags.get("name").startsWith("metadata-store"));
-        }
-        for (Metric m : putOpsFailed) {
-            Assert.assertEquals(m.tags.get("cluster"), "test");
-            Assert.assertTrue(m.tags.get("name").startsWith("metadata-store"));
-        }
-        for (Metric m : getOpsLatency) {
-            Assert.assertEquals(m.tags.get("cluster"), "test");
-            Assert.assertTrue(m.tags.get("name").startsWith("metadata-store"));
-            Assert.assertTrue(m.value > 0);
-        }
-        for (Metric m : delOpsLatency) {
+        for (Metric m : opsFailed) {
             Assert.assertEquals(m.tags.get("cluster"), "test");
             Assert.assertTrue(m.tags.get("name").startsWith("metadata-store"));
+            if (m.tags.get("type").equals("get")) {
+                Assert.assertTrue(m.value >= 0);

Review Comment:
   when I input `Assert`, IDEA will prompt the methods, I just need to select one



-- 
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


[GitHub] [pulsar] codelipenghui commented on pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#issuecomment-1246588526

   @tjiuming I think most of them are from tests? After I checked the source code, I think we can just add a new field `name` to `MetadataStoreConfig`, just keep the empty name by default. So that we can only modify the method `createConfigurationMetadataStore` and `PulsarMetadataStateStoreProviderImpl.init`?


-- 
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


[GitHub] [pulsar] codelipenghui commented on pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#issuecomment-1211542023

   @asafm Please help review this PR.


-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r945648413


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -0,0 +1,124 @@
+/**
+ * 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.metadata.impl.stats;
+
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public final class MetadataStoreStats implements AutoCloseable {
+    private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
+    private static final String LABEL_NAME = "metadata_store_name";
+
+    private static final Histogram GET_OPS_SUCCEED = Histogram
+            .build("pulsar_metadata_store_get_ops_latency", "-")

Review Comment:
   @codelipenghui resolved, PTAL



-- 
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


[GitHub] [pulsar] codelipenghui commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r945483295


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -0,0 +1,124 @@
+/**
+ * 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.metadata.impl.stats;
+
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public final class MetadataStoreStats implements AutoCloseable {
+    private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
+    private static final String LABEL_NAME = "metadata_store_name";
+
+    private static final Histogram GET_OPS_SUCCEED = Histogram
+            .build("pulsar_metadata_store_get_ops_latency", "-")

Review Comment:
   Yes, this looks very intuitive that all the metadata-related metrics start with `pulsar_metadata_store_`



-- 
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


[GitHub] [pulsar] tjiuming commented on pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#issuecomment-1231851770

   @asafm Please help review this PR


-- 
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


[GitHub] [pulsar] asafm commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
asafm commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r959266056


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java:
##########
@@ -1557,49 +1557,40 @@ public void testMetadataStoreStats() throws Exception {
         String metricsStr = output.toString();
         Multimap<String, Metric> metricsMap = parseMetrics(metricsStr);
 
-        Collection<Metric> getOpsFailed = metricsMap.get("pulsar_metadata_store_get_failed" + "_total");
-        Collection<Metric> delOpsFailed = metricsMap.get("pulsar_metadata_store_del_failed" + "_total");
-        Collection<Metric> putOpsFailed = metricsMap.get("pulsar_metadata_store_put_failed" + "_total");
-
-        Collection<Metric> getOpsLatency = metricsMap.get("pulsar_metadata_store_get_latency_ms" + "_sum");
-        Collection<Metric> delOpsLatency = metricsMap.get("pulsar_metadata_store_del_latency_ms" + "_sum");
-        Collection<Metric> putOpsLatency = metricsMap.get("pulsar_metadata_store_put_latency_ms" + "_sum");
-
+        Collection<Metric> opsFailed = metricsMap.get("pulsar_metadata_store_ops_failed" + "_total");
+        Collection<Metric> opsLatency = metricsMap.get("pulsar_metadata_store_ops_latency_ms" + "_sum");
         Collection<Metric> putBytes = metricsMap.get("pulsar_metadata_store_put_bytes" + "_total");
 
-        Assert.assertTrue(getOpsFailed.size() > 1);
-        Assert.assertTrue(delOpsFailed.size() > 1);
-        Assert.assertTrue(putOpsFailed.size() > 1);
-        Assert.assertTrue(getOpsLatency.size() > 1);
-        Assert.assertTrue(delOpsLatency.size() > 1);
-        Assert.assertTrue(putOpsLatency.size() > 1);
+        Assert.assertTrue(opsFailed.size() > 1);

Review Comment:
   I personally like [AssertJ](https://assertj.github.io/): `assertThat(opsFailed.size()).isGreaterThen(1);`
   When the test fails, you see the value of `size()`



##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java:
##########
@@ -1557,49 +1557,40 @@ public void testMetadataStoreStats() throws Exception {
         String metricsStr = output.toString();
         Multimap<String, Metric> metricsMap = parseMetrics(metricsStr);
 
-        Collection<Metric> getOpsFailed = metricsMap.get("pulsar_metadata_store_get_failed" + "_total");
-        Collection<Metric> delOpsFailed = metricsMap.get("pulsar_metadata_store_del_failed" + "_total");
-        Collection<Metric> putOpsFailed = metricsMap.get("pulsar_metadata_store_put_failed" + "_total");
-
-        Collection<Metric> getOpsLatency = metricsMap.get("pulsar_metadata_store_get_latency_ms" + "_sum");
-        Collection<Metric> delOpsLatency = metricsMap.get("pulsar_metadata_store_del_latency_ms" + "_sum");
-        Collection<Metric> putOpsLatency = metricsMap.get("pulsar_metadata_store_put_latency_ms" + "_sum");
-
+        Collection<Metric> opsFailed = metricsMap.get("pulsar_metadata_store_ops_failed" + "_total");
+        Collection<Metric> opsLatency = metricsMap.get("pulsar_metadata_store_ops_latency_ms" + "_sum");
         Collection<Metric> putBytes = metricsMap.get("pulsar_metadata_store_put_bytes" + "_total");
 
-        Assert.assertTrue(getOpsFailed.size() > 1);
-        Assert.assertTrue(delOpsFailed.size() > 1);
-        Assert.assertTrue(putOpsFailed.size() > 1);
-        Assert.assertTrue(getOpsLatency.size() > 1);
-        Assert.assertTrue(delOpsLatency.size() > 1);
-        Assert.assertTrue(putOpsLatency.size() > 1);
+        Assert.assertTrue(opsFailed.size() > 1);
+        Assert.assertTrue(opsLatency.size() > 1);
         Assert.assertTrue(putBytes.size() > 1);
 
-        for (Metric m : getOpsFailed) {
-            Assert.assertEquals(m.tags.get("cluster"), "test");
-            Assert.assertTrue(m.tags.get("name").startsWith("metadata-store"));
-        }
-        for (Metric m : delOpsFailed) {
-            Assert.assertEquals(m.tags.get("cluster"), "test");
-            Assert.assertTrue(m.tags.get("name").startsWith("metadata-store"));
-        }
-        for (Metric m : putOpsFailed) {
-            Assert.assertEquals(m.tags.get("cluster"), "test");
-            Assert.assertTrue(m.tags.get("name").startsWith("metadata-store"));
-        }
-        for (Metric m : getOpsLatency) {
-            Assert.assertEquals(m.tags.get("cluster"), "test");
-            Assert.assertTrue(m.tags.get("name").startsWith("metadata-store"));
-            Assert.assertTrue(m.value > 0);
-        }
-        for (Metric m : delOpsLatency) {
+        for (Metric m : opsFailed) {
             Assert.assertEquals(m.tags.get("cluster"), "test");
             Assert.assertTrue(m.tags.get("name").startsWith("metadata-store"));
+            if (m.tags.get("type").equals("get")) {
+                Assert.assertTrue(m.value >= 0);

Review Comment:
   If you're not using assertJ, I would at least import static `Assert`



##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -0,0 +1,109 @@
+/**
+ * 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.metadata.impl.stats;
+
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public final class MetadataStoreStats implements AutoCloseable {
+    private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
+    private static final String OPS_TYPE_LABEL_NAME = "type";
+    private static final String METADATA_STORE_LABEL_NAME = "name";
+    private static final String OPS_TYPE_GET = "get";
+    private static final String OPS_TYPE_DEL = "del";
+    private static final String OPS_TYPE_PUT = "put";
+    protected static final String PREFIX = "pulsar_metadata_store_";
+
+    private static final Histogram OPS_SUCCEED = Histogram
+            .build(PREFIX + "ops_latency", "-")
+            .unit("ms")
+            .buckets(BUCKETS)
+            .labelNames(METADATA_STORE_LABEL_NAME, OPS_TYPE_LABEL_NAME)
+            .register();
+    private static final Counter OPS_FAILED = Counter
+            .build(PREFIX + "ops_failed", "-")
+            .labelNames(METADATA_STORE_LABEL_NAME, OPS_TYPE_LABEL_NAME)
+            .register();
+    private static final Counter PUT_BYTES = Counter
+            .build(PREFIX + "put_", "-")

Review Comment:
   Based on Prometheus code you need to change `put_` to `put`:
   ```java
       unit = b.unit;
       if (!unit.isEmpty() && !name.endsWith("_" + unit)) {
         name += "_" + unit;
       }
   ```



##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -24,48 +24,32 @@
 
 public final class MetadataStoreStats implements AutoCloseable {
     private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
-    private static final String LABEL_NAME = "name";
+    private static final String OPS_TYPE_LABEL_NAME = "type";
+    private static final String METADATA_STORE_LABEL_NAME = "name";
+    private static final String OPS_TYPE_GET = "get";
+    private static final String OPS_TYPE_DEL = "del";
+    private static final String OPS_TYPE_PUT = "put";
     protected static final String PREFIX = "pulsar_metadata_store_";
 
-    private static final Histogram GET_OPS_SUCCEED = Histogram
-            .build(PREFIX + "get_latency", "-")
+    private static final Histogram OPS_SUCCEED = Histogram

Review Comment:
   I would camelCase this - although it's static, it's not a constant, it's an actual object, so `opsSucceededHistogram`



##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java:
##########
@@ -126,6 +131,9 @@ public CompletableFuture<Boolean> asyncReload(String key, Boolean oldValue,
                         }
                     }
                 });
+
+        this.metadataStoreName = poolName + "-" + ID.getAndIncrement();
+        this.metadataStoreStats = new MetadataStoreStats(metadataStoreName);

Review Comment:
   `this.` seem redundant here. It's relevant to all places in this PR.
   



##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java:
##########
@@ -126,6 +131,9 @@ public CompletableFuture<Boolean> asyncReload(String key, Boolean oldValue,
                         }
                     }
                 });
+
+        this.metadataStoreName = poolName + "-" + ID.getAndIncrement();

Review Comment:
   I don't get this at all.
   1. Do we have more than one metadata store in Pulsar Broker?
   2. Why a name is derived from a thread pool name?
   3. Why the name is actually an auto-generated name?



##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -0,0 +1,109 @@
+/**
+ * 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.metadata.impl.stats;
+
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public final class MetadataStoreStats implements AutoCloseable {
+    private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
+    private static final String OPS_TYPE_LABEL_NAME = "type";
+    private static final String METADATA_STORE_LABEL_NAME = "name";
+    private static final String OPS_TYPE_GET = "get";
+    private static final String OPS_TYPE_DEL = "del";
+    private static final String OPS_TYPE_PUT = "put";
+    protected static final String PREFIX = "pulsar_metadata_store_";
+
+    private static final Histogram OPS_SUCCEED = Histogram
+            .build(PREFIX + "ops_latency", "-")
+            .unit("ms")
+            .buckets(BUCKETS)
+            .labelNames(METADATA_STORE_LABEL_NAME, OPS_TYPE_LABEL_NAME)
+            .register();
+    private static final Counter OPS_FAILED = Counter
+            .build(PREFIX + "ops_failed", "-")
+            .labelNames(METADATA_STORE_LABEL_NAME, OPS_TYPE_LABEL_NAME)
+            .register();
+    private static final Counter PUT_BYTES = Counter
+            .build(PREFIX + "put_", "-")
+            .unit("bytes")
+            .labelNames(METADATA_STORE_LABEL_NAME)
+            .register();
+
+    private final Histogram.Child getOpsSucceedChild;
+    private final Histogram.Child delOpsSucceedChild;
+    private final Histogram.Child putOpsSucceedChild;
+    private final Counter.Child getOpsFailedChild;
+    private final Counter.Child delOpsFailedChild;
+    private final Counter.Child putOpsFailedChild;
+    private final Counter.Child putBytesChild;
+    private final String metadataStoreName;
+    private final AtomicBoolean closed = new AtomicBoolean(false);
+
+    public MetadataStoreStats(String metadataStoreName) {
+        this.metadataStoreName = metadataStoreName;
+
+        this.getOpsSucceedChild = OPS_SUCCEED.labels(metadataStoreName, OPS_TYPE_GET);
+        this.delOpsSucceedChild = OPS_SUCCEED.labels(metadataStoreName, OPS_TYPE_DEL);
+        this.putOpsSucceedChild = OPS_SUCCEED.labels(metadataStoreName, OPS_TYPE_PUT);
+        this.getOpsFailedChild = OPS_FAILED.labels(metadataStoreName, OPS_TYPE_GET);
+        this.delOpsFailedChild = OPS_FAILED.labels(metadataStoreName, OPS_TYPE_DEL);
+        this.putOpsFailedChild = OPS_FAILED.labels(metadataStoreName, OPS_TYPE_PUT);
+        this.putBytesChild = PUT_BYTES.labels(metadataStoreName);
+    }
+
+    public void recordGetOpsLatency(long millis) {
+        this.getOpsSucceedChild.observe(millis);
+    }
+
+    public void recordDelOpsLatency(long millis) {
+        this.delOpsSucceedChild.observe(millis);
+    }
+
+    public void recordPutOpsLatency(long millis, int bytes) {
+        this.putOpsSucceedChild.observe(millis);
+        this.putBytesChild.inc(bytes);
+    }
+
+    public void recordGetOpsFailed() {
+        this.getOpsFailedChild.inc();
+    }
+
+    public void recordDelOpsFailed() {
+        this.delOpsFailedChild.inc();
+    }
+
+    public void recordPutOpsFailed() {
+        this.putOpsFailedChild.inc();
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (this.closed.compareAndSet(false, true)) {

Review Comment:
   I wouldn't bother with this check - it's a developer bug to call close twice



##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -76,12 +60,12 @@ public final class MetadataStoreStats implements AutoCloseable {
     public MetadataStoreStats(String metadataStoreName) {
         this.metadataStoreName = metadataStoreName;
 
-        this.getOpsSucceedChild = GET_OPS_SUCCEED.labels(metadataStoreName);
-        this.delOpsSucceedChild = DEL_OPS_SUCCEED.labels(metadataStoreName);
-        this.puttOpsSucceedChild = PUT_OPS_SUCCEED.labels(metadataStoreName);
-        this.getOpsFailedChild = GET_OPS_FAILED.labels(metadataStoreName);
-        this.delOpsFailedChild = DEL_OPS_FAILED.labels(metadataStoreName);
-        this.putOpsFailedChild = PUT_OPS_FAILED.labels(metadataStoreName);
+        this.getOpsSucceedChild = OPS_SUCCEED.labels(metadataStoreName, OPS_TYPE_GET);

Review Comment:
   why `this.` prefix? There is no collision in constructor arguments.



##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -0,0 +1,109 @@
+/**
+ * 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.metadata.impl.stats;
+
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public final class MetadataStoreStats implements AutoCloseable {
+    private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
+    private static final String OPS_TYPE_LABEL_NAME = "type";
+    private static final String METADATA_STORE_LABEL_NAME = "name";
+    private static final String OPS_TYPE_GET = "get";
+    private static final String OPS_TYPE_DEL = "del";
+    private static final String OPS_TYPE_PUT = "put";
+    protected static final String PREFIX = "pulsar_metadata_store_";
+
+    private static final Histogram OPS_SUCCEED = Histogram
+            .build(PREFIX + "ops_latency", "-")
+            .unit("ms")
+            .buckets(BUCKETS)
+            .labelNames(METADATA_STORE_LABEL_NAME, OPS_TYPE_LABEL_NAME)
+            .register();
+    private static final Counter OPS_FAILED = Counter

Review Comment:
   I think we're missing a key stat here - latency for failed ops. From my experience, this is sometimes key to figuring out production issues. I would record latency for this and not just counter



##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -24,48 +24,32 @@
 
 public final class MetadataStoreStats implements AutoCloseable {
     private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
-    private static final String LABEL_NAME = "name";
+    private static final String OPS_TYPE_LABEL_NAME = "type";
+    private static final String METADATA_STORE_LABEL_NAME = "name";
+    private static final String OPS_TYPE_GET = "get";
+    private static final String OPS_TYPE_DEL = "del";
+    private static final String OPS_TYPE_PUT = "put";
     protected static final String PREFIX = "pulsar_metadata_store_";
 
-    private static final Histogram GET_OPS_SUCCEED = Histogram
-            .build(PREFIX + "get_latency", "-")
+    private static final Histogram OPS_SUCCEED = Histogram
+            .build(PREFIX + "ops_latency", "-")
             .unit("ms")
             .buckets(BUCKETS)
-            .labelNames(LABEL_NAME)
+            .labelNames(METADATA_STORE_LABEL_NAME, OPS_TYPE_LABEL_NAME)

Review Comment:
   Can you explain the meaning of the metadata store name? I haven't seen in the implementation of metadata stores any name.



-- 
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


[GitHub] [pulsar] asafm commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
asafm commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r962307406


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -0,0 +1,109 @@
+/**
+ * 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.metadata.impl.stats;
+
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public final class MetadataStoreStats implements AutoCloseable {
+    private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
+    private static final String OPS_TYPE_LABEL_NAME = "type";
+    private static final String METADATA_STORE_LABEL_NAME = "name";
+    private static final String OPS_TYPE_GET = "get";
+    private static final String OPS_TYPE_DEL = "del";
+    private static final String OPS_TYPE_PUT = "put";
+    protected static final String PREFIX = "pulsar_metadata_store_";
+
+    private static final Histogram OPS_SUCCEED = Histogram
+            .build(PREFIX + "ops_latency", "-")
+            .unit("ms")
+            .buckets(BUCKETS)
+            .labelNames(METADATA_STORE_LABEL_NAME, OPS_TYPE_LABEL_NAME)
+            .register();
+    private static final Counter OPS_FAILED = Counter

Review Comment:
   I disagree. I have resolved many production incidence, last was of M3 (Uber TSDB) which it literally saved and helped resolve a very big production issue.
   



-- 
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


[GitHub] [pulsar] asafm commented on pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
asafm commented on PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#issuecomment-1239343850

   > @asafm Yes, you are right and I understand your concern, but actually, we can know that which metadata store is configuration store and which stores broker metadata, just takes a little skill:
   > 
   > In PulsarService.java, code as below:
   > 
   > ```
   >             localMetadataStore = createLocalMetadataStore(localMetadataSynchronizer);
   >             localMetadataStore.registerSessionListener(this::handleMetadataSessionEvent);
   > 
   >             coordinationService = new CoordinationServiceImpl(localMetadataStore);
   > 
   >             if (config.isConfigurationStoreSeparated()) {
   >                 configMetadataSynchronizer = StringUtils.isNotBlank(config.getConfigurationMetadataSyncEventTopic())
   >                         ? new PulsarMetadataEventSynchronizer(this, config.getConfigurationMetadataSyncEventTopic())
   >                         : null;
   >                 configurationMetadataStore = createConfigurationMetadataStore(configMetadataSynchronizer);
   >                 shouldShutdownConfigurationMetadataStore = true;
   >             } else {
   >                 configurationMetadataStore = localMetadataStore;
   >                 shouldShutdownConfigurationMetadataStore = false;
   >             }
   > ```
   > 
   > Broker metadata store initialized before configuration metadata store, so, `metadata-store-0` is broker metadata store and `metadata-store-1` is configuration metadata store
   
   1. Do you find that a reasonable user experience for a Pulsar user? IMO, they will get quite frustrated and mad when they read that, where all they wanted to do was to know the metrics of metadata store access. 
   2. Reiterating my question in the original comment - how of the options I listed above, what is your goal when breaking down the metadata store metrics? (see the question in the previous comment)


-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r962821373


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -0,0 +1,109 @@
+/**
+ * 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.metadata.impl.stats;
+
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public final class MetadataStoreStats implements AutoCloseable {
+    private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
+    private static final String OPS_TYPE_LABEL_NAME = "type";
+    private static final String METADATA_STORE_LABEL_NAME = "name";
+    private static final String OPS_TYPE_GET = "get";
+    private static final String OPS_TYPE_DEL = "del";
+    private static final String OPS_TYPE_PUT = "put";
+    protected static final String PREFIX = "pulsar_metadata_store_";
+
+    private static final Histogram OPS_SUCCEED = Histogram
+            .build(PREFIX + "ops_latency", "-")
+            .unit("ms")
+            .buckets(BUCKETS)
+            .labelNames(METADATA_STORE_LABEL_NAME, OPS_TYPE_LABEL_NAME)
+            .register();
+    private static final Counter OPS_FAILED = Counter

Review Comment:
   OK, I'll add the latency if you insist



-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r962820305


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -24,48 +24,32 @@
 
 public final class MetadataStoreStats implements AutoCloseable {
     private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
-    private static final String LABEL_NAME = "name";
+    private static final String OPS_TYPE_LABEL_NAME = "type";
+    private static final String METADATA_STORE_LABEL_NAME = "name";
+    private static final String OPS_TYPE_GET = "get";
+    private static final String OPS_TYPE_DEL = "del";
+    private static final String OPS_TYPE_PUT = "put";
     protected static final String PREFIX = "pulsar_metadata_store_";
 
-    private static final Histogram GET_OPS_SUCCEED = Histogram
-            .build(PREFIX + "get_latency", "-")
+    private static final Histogram OPS_SUCCEED = Histogram

Review Comment:
   it's not a blocker



-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r949375203


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java:
##########
@@ -1550,15 +1557,15 @@ public void testMetadataStoreStats() throws Exception {
         String metricsStr = output.toString();
         Multimap<String, Metric> metricsMap = parseMetrics(metricsStr);
 
-        Collection<Metric> getOpsFailed = metricsMap.get("pulsar_metadata_store_get_ops_failed" + "_total");
-        Collection<Metric> delOpsFailed = metricsMap.get("pulsar_metadata_store_del_ops_failed" + "_total");
-        Collection<Metric> putOpsFailed = metricsMap.get("pulsar_metadata_store_put_ops_failed" + "_total");
+        Collection<Metric> getOpsFailed = metricsMap.get("pulsar_metadata_store_get_failed" + "_total");

Review Comment:
   `_total` added by Prometheus, use `+` to distinguish



-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r943692396


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java:
##########
@@ -1522,6 +1517,83 @@ public void testSplitTopicAndPartitionLabel() throws Exception {
         consumer2.close();
     }
 
+
+    @Test
+    public void testMetadataStoreStats() throws Exception {
+        String ns = "prop/ns-abc1";
+        admin.namespaces().createNamespace(ns);
+
+        String topic = "persistent://prop/ns-abc1/metadata-store-" + UUID.randomUUID();
+        String subName = "my-sub1";
+
+        @Cleanup
+        Producer<String> producer = pulsarClient.newProducer(Schema.STRING)
+                .topic(topic).create();
+        @Cleanup
+        Consumer<String> consumer = pulsarClient.newConsumer(Schema.STRING)
+                .topic(topic).subscriptionName(subName).subscribe();
+
+        for (int i = 0; i < 100; i++) {
+            producer.newMessage().value(UUID.randomUUID().toString()).send();
+        }
+
+        for (;;) {
+            Message<String> message = consumer.receive(10, TimeUnit.SECONDS);
+            if (message == null) {
+                break;
+            }
+            consumer.acknowledge(message);
+        }
+
+        ByteArrayOutputStream output = new ByteArrayOutputStream();
+        PrometheusMetricsGenerator.generate(pulsar, false, false, false, false, output);
+        String metricsStr = output.toString();
+        Multimap<String, Metric> metricsMap = parseMetrics(metricsStr);
+
+        Collection<Metric> getOpsFailed = metricsMap.get("pulsar_metadata_store_get_ops_failed" + "_total");
+        Collection<Metric> delOpsFailed = metricsMap.get("pulsar_metadata_store_del_ops_failed" + "_total");
+        Collection<Metric> putOpsFailed = metricsMap.get("pulsar_metadata_store_put_ops_failed" + "_total");
+
+        Collection<Metric> getOpsLatency = metricsMap.get("pulsar_metadata_store_get_ops_latency" + "_sum");
+        Collection<Metric> delOpsLatency = metricsMap.get("pulsar_metadata_store_del_ops_latency" + "_sum");
+        Collection<Metric> putOpsLatency = metricsMap.get("pulsar_metadata_store_put_ops_latency" + "_sum");
+
+        Collection<Metric> putBytes = metricsMap.get("pulsar_metadata_store_put_bytes" + "_total");
+
+        Assert.assertTrue(getOpsFailed.size() > 0);

Review Comment:
   it's not an issue



##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java:
##########
@@ -1522,6 +1517,83 @@ public void testSplitTopicAndPartitionLabel() throws Exception {
         consumer2.close();
     }
 
+
+    @Test
+    public void testMetadataStoreStats() throws Exception {
+        String ns = "prop/ns-abc1";
+        admin.namespaces().createNamespace(ns);
+
+        String topic = "persistent://prop/ns-abc1/metadata-store-" + UUID.randomUUID();
+        String subName = "my-sub1";
+
+        @Cleanup
+        Producer<String> producer = pulsarClient.newProducer(Schema.STRING)
+                .topic(topic).create();
+        @Cleanup
+        Consumer<String> consumer = pulsarClient.newConsumer(Schema.STRING)
+                .topic(topic).subscriptionName(subName).subscribe();
+
+        for (int i = 0; i < 100; i++) {
+            producer.newMessage().value(UUID.randomUUID().toString()).send();
+        }
+
+        for (;;) {
+            Message<String> message = consumer.receive(10, TimeUnit.SECONDS);
+            if (message == null) {
+                break;
+            }
+            consumer.acknowledge(message);
+        }
+
+        ByteArrayOutputStream output = new ByteArrayOutputStream();
+        PrometheusMetricsGenerator.generate(pulsar, false, false, false, false, output);
+        String metricsStr = output.toString();
+        Multimap<String, Metric> metricsMap = parseMetrics(metricsStr);
+
+        Collection<Metric> getOpsFailed = metricsMap.get("pulsar_metadata_store_get_ops_failed" + "_total");
+        Collection<Metric> delOpsFailed = metricsMap.get("pulsar_metadata_store_del_ops_failed" + "_total");
+        Collection<Metric> putOpsFailed = metricsMap.get("pulsar_metadata_store_put_ops_failed" + "_total");
+
+        Collection<Metric> getOpsLatency = metricsMap.get("pulsar_metadata_store_get_ops_latency" + "_sum");
+        Collection<Metric> delOpsLatency = metricsMap.get("pulsar_metadata_store_del_ops_latency" + "_sum");
+        Collection<Metric> putOpsLatency = metricsMap.get("pulsar_metadata_store_put_ops_latency" + "_sum");
+
+        Collection<Metric> putBytes = metricsMap.get("pulsar_metadata_store_put_bytes" + "_total");
+
+        Assert.assertTrue(getOpsFailed.size() > 0);
+        Assert.assertTrue(delOpsFailed.size() > 0);
+        Assert.assertTrue(putOpsFailed.size() > 0);
+        Assert.assertTrue(getOpsLatency.size() > 0);
+        Assert.assertTrue(delOpsLatency.size() > 0);
+        Assert.assertTrue(putOpsLatency.size() > 0);
+        Assert.assertTrue(putBytes.size() > 0);
+
+        for (Metric m : getOpsFailed) {
+            Assert.assertEquals(m.tags.get("cluster"), "test");

Review Comment:
   make sure the `cluster` label being added atomically



-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r943695129


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -0,0 +1,92 @@
+/**
+ * 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.metadata.impl.stats;
+
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public final class MetadataStoreStats {
+    private static final AtomicBoolean INITIALIZED = new AtomicBoolean(false);
+    private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
+
+    private final Histogram getOpsLatency;
+    private final Histogram delOpsLatency;
+    private final Histogram putOpsLatency;
+    private final Counter getFailedCounter;
+    private final Counter delFailedCounter;
+    private final Counter putFailedCounter;
+    private final Counter putBytesCounter;
+
+    private MetadataStoreStats() {
+        getOpsLatency = Histogram.build("pulsar_metadata_store_get_ops_latency", "-")

Review Comment:
   do we need to add unit at the end of metrics' name? then the old code may have to keep consistent with it too, it may comes broken change



-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r952674026


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java:
##########
@@ -81,8 +86,8 @@ public abstract class AbstractMetadataStore implements MetadataStoreExtended, Co
     protected abstract CompletableFuture<Boolean> existsFromStore(String path);
 
     protected AbstractMetadataStore() {
-        this.executor = Executors
-                .newSingleThreadScheduledExecutor(new DefaultThreadFactory("metadata-store"));
+        final String poolName = "metadata-store";
+        this.executor = new ScheduledThreadPoolExecutor(1, new DefaultThreadFactory(poolName));

Review Comment:
   yes, it's useful for https://github.com/apache/pulsar/pull/17072, I want to monitor how many tasks pending in the Task Queue.



-- 
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


[GitHub] [pulsar] asafm commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
asafm commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r959331016


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java:
##########
@@ -1522,6 +1517,83 @@ public void testSplitTopicAndPartitionLabel() throws Exception {
         consumer2.close();
     }
 
+
+    @Test
+    public void testMetadataStoreStats() throws Exception {
+        String ns = "prop/ns-abc1";
+        admin.namespaces().createNamespace(ns);
+
+        String topic = "persistent://prop/ns-abc1/metadata-store-" + UUID.randomUUID();
+        String subName = "my-sub1";
+
+        @Cleanup
+        Producer<String> producer = pulsarClient.newProducer(Schema.STRING)
+                .topic(topic).create();
+        @Cleanup
+        Consumer<String> consumer = pulsarClient.newConsumer(Schema.STRING)
+                .topic(topic).subscriptionName(subName).subscribe();
+
+        for (int i = 0; i < 100; i++) {
+            producer.newMessage().value(UUID.randomUUID().toString()).send();
+        }
+
+        for (;;) {
+            Message<String> message = consumer.receive(10, TimeUnit.SECONDS);
+            if (message == null) {
+                break;
+            }
+            consumer.acknowledge(message);
+        }
+
+        ByteArrayOutputStream output = new ByteArrayOutputStream();
+        PrometheusMetricsGenerator.generate(pulsar, false, false, false, false, output);
+        String metricsStr = output.toString();
+        Multimap<String, Metric> metricsMap = parseMetrics(metricsStr);
+
+        Collection<Metric> getOpsFailed = metricsMap.get("pulsar_metadata_store_get_ops_failed" + "_total");
+        Collection<Metric> delOpsFailed = metricsMap.get("pulsar_metadata_store_del_ops_failed" + "_total");
+        Collection<Metric> putOpsFailed = metricsMap.get("pulsar_metadata_store_put_ops_failed" + "_total");
+
+        Collection<Metric> getOpsLatency = metricsMap.get("pulsar_metadata_store_get_ops_latency" + "_sum");
+        Collection<Metric> delOpsLatency = metricsMap.get("pulsar_metadata_store_del_ops_latency" + "_sum");
+        Collection<Metric> putOpsLatency = metricsMap.get("pulsar_metadata_store_put_ops_latency" + "_sum");
+
+        Collection<Metric> putBytes = metricsMap.get("pulsar_metadata_store_put_bytes" + "_total");
+
+        Assert.assertTrue(getOpsFailed.size() > 0);

Review Comment:
   One more small thing: It's customary for the person making the comment to resolve the comment, since he is the one needs to check that it was implemented - I treat it like a to-do list when I revisit a PR, and resolve when I'm done



-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r960870489


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -0,0 +1,109 @@
+/**
+ * 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.metadata.impl.stats;
+
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public final class MetadataStoreStats implements AutoCloseable {
+    private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
+    private static final String OPS_TYPE_LABEL_NAME = "type";
+    private static final String METADATA_STORE_LABEL_NAME = "name";
+    private static final String OPS_TYPE_GET = "get";
+    private static final String OPS_TYPE_DEL = "del";
+    private static final String OPS_TYPE_PUT = "put";
+    protected static final String PREFIX = "pulsar_metadata_store_";
+
+    private static final Histogram OPS_SUCCEED = Histogram
+            .build(PREFIX + "ops_latency", "-")
+            .unit("ms")
+            .buckets(BUCKETS)
+            .labelNames(METADATA_STORE_LABEL_NAME, OPS_TYPE_LABEL_NAME)
+            .register();
+    private static final Counter OPS_FAILED = Counter

Review Comment:
   I think record the latency of failed ops is meaningless



-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r960873532


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -0,0 +1,109 @@
+/**
+ * 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.metadata.impl.stats;
+
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public final class MetadataStoreStats implements AutoCloseable {
+    private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
+    private static final String OPS_TYPE_LABEL_NAME = "type";
+    private static final String METADATA_STORE_LABEL_NAME = "name";
+    private static final String OPS_TYPE_GET = "get";
+    private static final String OPS_TYPE_DEL = "del";
+    private static final String OPS_TYPE_PUT = "put";
+    protected static final String PREFIX = "pulsar_metadata_store_";
+
+    private static final Histogram OPS_SUCCEED = Histogram
+            .build(PREFIX + "ops_latency", "-")
+            .unit("ms")
+            .buckets(BUCKETS)
+            .labelNames(METADATA_STORE_LABEL_NAME, OPS_TYPE_LABEL_NAME)
+            .register();
+    private static final Counter OPS_FAILED = Counter
+            .build(PREFIX + "ops_failed", "-")
+            .labelNames(METADATA_STORE_LABEL_NAME, OPS_TYPE_LABEL_NAME)
+            .register();
+    private static final Counter PUT_BYTES = Counter
+            .build(PREFIX + "put_", "-")

Review Comment:
   it's not a blocker, I won't fix here due to it's unnecessary



-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r960878699


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java:
##########
@@ -1522,6 +1524,81 @@ public void testSplitTopicAndPartitionLabel() throws Exception {
         consumer2.close();
     }
 
+
+    @Test
+    public void testMetadataStoreStats() throws Exception {

Review Comment:
   don't worry about it, I excluded topic/consumer/producer stats:
   ```
           PrometheusMetricsGenerator.generate(pulsar, false, false, false, false, output);
   ```



-- 
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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r960848693


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java:
##########
@@ -1557,49 +1557,40 @@ public void testMetadataStoreStats() throws Exception {
         String metricsStr = output.toString();
         Multimap<String, Metric> metricsMap = parseMetrics(metricsStr);
 
-        Collection<Metric> getOpsFailed = metricsMap.get("pulsar_metadata_store_get_failed" + "_total");
-        Collection<Metric> delOpsFailed = metricsMap.get("pulsar_metadata_store_del_failed" + "_total");
-        Collection<Metric> putOpsFailed = metricsMap.get("pulsar_metadata_store_put_failed" + "_total");
-
-        Collection<Metric> getOpsLatency = metricsMap.get("pulsar_metadata_store_get_latency_ms" + "_sum");
-        Collection<Metric> delOpsLatency = metricsMap.get("pulsar_metadata_store_del_latency_ms" + "_sum");
-        Collection<Metric> putOpsLatency = metricsMap.get("pulsar_metadata_store_put_latency_ms" + "_sum");
-
+        Collection<Metric> opsFailed = metricsMap.get("pulsar_metadata_store_ops_failed" + "_total");
+        Collection<Metric> opsLatency = metricsMap.get("pulsar_metadata_store_ops_latency_ms" + "_sum");
         Collection<Metric> putBytes = metricsMap.get("pulsar_metadata_store_put_bytes" + "_total");
 
-        Assert.assertTrue(getOpsFailed.size() > 1);
-        Assert.assertTrue(delOpsFailed.size() > 1);
-        Assert.assertTrue(putOpsFailed.size() > 1);
-        Assert.assertTrue(getOpsLatency.size() > 1);
-        Assert.assertTrue(delOpsLatency.size() > 1);
-        Assert.assertTrue(putOpsLatency.size() > 1);
+        Assert.assertTrue(opsFailed.size() > 1);

Review Comment:
   this class imports org.testng.Assert



-- 
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


[GitHub] [pulsar] asafm commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
asafm commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r962303603


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -24,48 +24,32 @@
 
 public final class MetadataStoreStats implements AutoCloseable {
     private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
-    private static final String LABEL_NAME = "name";
+    private static final String OPS_TYPE_LABEL_NAME = "type";
+    private static final String METADATA_STORE_LABEL_NAME = "name";
+    private static final String OPS_TYPE_GET = "get";
+    private static final String OPS_TYPE_DEL = "del";
+    private static final String OPS_TYPE_PUT = "put";
     protected static final String PREFIX = "pulsar_metadata_store_";
 
-    private static final Histogram GET_OPS_SUCCEED = Histogram
-            .build(PREFIX + "get_latency", "-")
+    private static final Histogram OPS_SUCCEED = Histogram

Review Comment:
   The rule only applies to public field, not private:
   ```
           <module name="ConstantNameCheck">
               <!-- Validates non-private, static, final fields against the supplied
               public/package final fields "^[A-Z][A-Z0-9]*(_[A-Z0-9]+)*$". -->
               <metadata name="altname" value="ConstantName"/>
               <property name="applyToPublic" value="true"/>
               <property name="applyToProtected" value="true"/>
               <property name="applyToPackage" value="true"/>
               <property name="applyToPrivate" value="false"/>
               <property name="format" value="^([A-Z][A-Za-z0-9_]*|FLAG_.*)$"/>
               <message key="name.invalidPattern"
                        value="Variable ''{0}'' should be in ALL_CAPS (if it is a constant) or be private (otherwise)."/>
               <property name="severity" value="error"/>
           </module>
   
   ```



-- 
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


[GitHub] [pulsar] asafm commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
asafm commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r962307669


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -0,0 +1,109 @@
+/**
+ * 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.metadata.impl.stats;
+
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public final class MetadataStoreStats implements AutoCloseable {
+    private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
+    private static final String OPS_TYPE_LABEL_NAME = "type";
+    private static final String METADATA_STORE_LABEL_NAME = "name";
+    private static final String OPS_TYPE_GET = "get";
+    private static final String OPS_TYPE_DEL = "del";
+    private static final String OPS_TYPE_PUT = "put";
+    protected static final String PREFIX = "pulsar_metadata_store_";
+
+    private static final Histogram OPS_SUCCEED = Histogram
+            .build(PREFIX + "ops_latency", "-")
+            .unit("ms")
+            .buckets(BUCKETS)
+            .labelNames(METADATA_STORE_LABEL_NAME, OPS_TYPE_LABEL_NAME)
+            .register();
+    private static final Counter OPS_FAILED = Counter
+            .build(PREFIX + "ops_failed", "-")
+            .labelNames(METADATA_STORE_LABEL_NAME, OPS_TYPE_LABEL_NAME)
+            .register();
+    private static final Counter PUT_BYTES = Counter
+            .build(PREFIX + "put_", "-")
+            .unit("bytes")
+            .labelNames(METADATA_STORE_LABEL_NAME)
+            .register();
+
+    private final Histogram.Child getOpsSucceedChild;
+    private final Histogram.Child delOpsSucceedChild;
+    private final Histogram.Child putOpsSucceedChild;
+    private final Counter.Child getOpsFailedChild;
+    private final Counter.Child delOpsFailedChild;
+    private final Counter.Child putOpsFailedChild;
+    private final Counter.Child putBytesChild;
+    private final String metadataStoreName;
+    private final AtomicBoolean closed = new AtomicBoolean(false);
+
+    public MetadataStoreStats(String metadataStoreName) {
+        this.metadataStoreName = metadataStoreName;
+
+        this.getOpsSucceedChild = OPS_SUCCEED.labels(metadataStoreName, OPS_TYPE_GET);
+        this.delOpsSucceedChild = OPS_SUCCEED.labels(metadataStoreName, OPS_TYPE_DEL);
+        this.putOpsSucceedChild = OPS_SUCCEED.labels(metadataStoreName, OPS_TYPE_PUT);
+        this.getOpsFailedChild = OPS_FAILED.labels(metadataStoreName, OPS_TYPE_GET);
+        this.delOpsFailedChild = OPS_FAILED.labels(metadataStoreName, OPS_TYPE_DEL);
+        this.putOpsFailedChild = OPS_FAILED.labels(metadataStoreName, OPS_TYPE_PUT);
+        this.putBytesChild = PUT_BYTES.labels(metadataStoreName);
+    }
+
+    public void recordGetOpsLatency(long millis) {
+        this.getOpsSucceedChild.observe(millis);
+    }
+
+    public void recordDelOpsLatency(long millis) {
+        this.delOpsSucceedChild.observe(millis);
+    }
+
+    public void recordPutOpsLatency(long millis, int bytes) {
+        this.putOpsSucceedChild.observe(millis);
+        this.putBytesChild.inc(bytes);
+    }
+
+    public void recordGetOpsFailed() {
+        this.getOpsFailedChild.inc();
+    }
+
+    public void recordDelOpsFailed() {
+        this.delOpsFailedChild.inc();
+    }
+
+    public void recordPutOpsFailed() {
+        this.putOpsFailedChild.inc();
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (this.closed.compareAndSet(false, true)) {

Review Comment:
   Then let us find the bug in test when it happens: Throw an exception if we called close, and it's already closed.
   



-- 
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


[GitHub] [pulsar] asafm commented on pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
asafm commented on PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#issuecomment-1236334066

   I commented @tjiuming 
   Aside from style notations, there are 3 main things I would improve:
   1. Metadata store name is not indicative enough - see comment
   2. Add latency for failed ops - see comment
   3. Move tests out of `PrometheusMetricsTest` - see comment


-- 
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


[GitHub] [pulsar] asafm commented on a diff in pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
asafm commented on code in PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#discussion_r945656745


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java:
##########
@@ -24,41 +24,42 @@
 
 public final class MetadataStoreStats implements AutoCloseable {
     private static final double[] BUCKETS = new double[]{1, 3, 5, 10, 20, 50, 100, 200, 500, 1000, 2000, 5000};
-    private static final String LABEL_NAME = "metadata_store_name";
+    private static final String LABEL_NAME = "name";
+    protected static final String PREFIX = "pulsar_metadata_store_";
 
     private static final Histogram GET_OPS_SUCCEED = Histogram
-            .build("pulsar_metadata_store_get_ops_latency", "-")
+            .build(PREFIX + "get_latency", "-")
             .unit("ms")
             .buckets(BUCKETS)
             .labelNames(LABEL_NAME)
             .register();
     private static final Histogram DEL_OPS_SUCCEED = Histogram
-            .build("pulsar_metadata_store_del_ops_latency", "-")
+            .build(PREFIX + "del_latency", "-")

Review Comment:
   `del_latency` --> `del_latency_ms` (according to [this](https://prometheus.io/docs/practices/naming/))



##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java:
##########
@@ -1550,15 +1557,15 @@ public void testMetadataStoreStats() throws Exception {
         String metricsStr = output.toString();
         Multimap<String, Metric> metricsMap = parseMetrics(metricsStr);
 
-        Collection<Metric> getOpsFailed = metricsMap.get("pulsar_metadata_store_get_ops_failed" + "_total");
-        Collection<Metric> delOpsFailed = metricsMap.get("pulsar_metadata_store_del_ops_failed" + "_total");
-        Collection<Metric> putOpsFailed = metricsMap.get("pulsar_metadata_store_put_ops_failed" + "_total");
+        Collection<Metric> getOpsFailed = metricsMap.get("pulsar_metadata_store_get_failed" + "_total");

Review Comment:
   Out of curiosity, while the `+` in the middle? Usually when you have the metric name you CMD-F and search. That plus sign will only get in the way :)
   



-- 
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


[GitHub] [pulsar] codelipenghui commented on pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#issuecomment-1239381247

   @tjiuming 
   
   > Broker metadata store initialized before configuration metadata store, so, metadata-store-0 is broker metadata store and metadata-store-1 is configuration metadata store
   
   IMO, we should improve this part. Instead of adding docs to tell users which one is configuration store, and which one is metadata store, we should add the type to the metrics like `type=configuration/metadata`. I also consider adding the metadataStoreUrl, but it can be a long string value.
   
   And we should also add the `cluster` label. Users can have one Prometheus service for multiple clusters, e.g. https://github.com/apache/pulsar/wiki/PIP-8:-Pulsar-beyond-1M-topics


-- 
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


[GitHub] [pulsar] tjiuming commented on pull request #17041: [monitoring][broker][metadata] add metadata store metrics

Posted by GitBox <gi...@apache.org>.
tjiuming commented on PR #17041:
URL: https://github.com/apache/pulsar/pull/17041#issuecomment-1249122187

   /pulsarbot run-failure-checks


-- 
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