Skip to content

Conversation

@hangc0276
Copy link
Contributor

Motivation

Pulsar use bookkeeper as distributed log storage, and init a bookie client to read/write data from/to bookkeeper. However the pulsar bookie client use default NullStatsLogger.INSTANCE to expose runtime metric, which doesn't expose to prometheus or other state storage. When tuning pulsar bookie client performance, we doesn't have any bookie metric to measure where is the bottleneck.

Changes

I implement a prometheus state provider, and use it to trace bookie client runtime metric, and expose it to prometheus.

Please help take a look, if it's ok, i will add test case and update the metric document.

@hangc0276
Copy link
Contributor Author

/pulsarbot run-failure-checks

Copy link
Contributor

@codelipenghui codelipenghui left a comment

Choose a reason for hiding this comment

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

The change looks good. Could you please help add the description of the new metric into https://pulsar.apache.org/docs/en/reference-metrics/#broker? This is very helpful for users to understand these new metrics.

Comment on lines 63 to 68
// bookie_journal_JOURNAL_ADD_ENTRY{success="false",quantile="0.5",} NaN
// bookie_journal_JOURNAL_ADD_ENTRY{success="false",quantile="0.75",} NaN
// bookie_journal_JOURNAL_ADD_ENTRY{success="false",quantile="0.95",} NaN
// bookie_journal_JOURNAL_ADD_ENTRY{success="false",quantile="0.99",} NaN
// bookie_journal_JOURNAL_ADD_ENTRY{success="false",quantile="0.999",} NaN
// bookie_journal_JOURNAL_ADD_ENTRY{success="false",quantile="0.9999",} NaN
Copy link
Contributor

Choose a reason for hiding this comment

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

Can we get these metrics at the bookie client side? If can't, I think it's better to use some bookie client metrics in the example.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I have checked the bookie client metric code, some class constructor and functions are package private and can't be accessed in pulsar.

private ManagedLedgerFactoryImpl(BookkeeperFactoryForCustomEnsemblePlacementPolicy bookKeeperGroupFactory, boolean isBookkeeperManaged, ZooKeeper zooKeeper,
ManagedLedgerFactoryConfig config) throws Exception {
Configuration configuration = new ClientConfiguration();
configuration.addProperty("prometheusStatsLatencyRolloverSeconds", config.getPrometheusStatsLatencyRolloverSeconds());
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
configuration.addProperty("prometheusStatsLatencyRolloverSeconds", config.getPrometheusStatsLatencyRolloverSeconds());
configuration.addProperty(PrometheusMetricsProvider.PROMETHEUS_STATS_LATENCY_ROLLOVER_SECONDS, config.getPrometheusStatsLatencyRolloverSeconds());

import java.io.IOException;
import java.io.Writer;
import java.lang.reflect.Field;
import java.util.concurrent.*;
Copy link
Contributor

Choose a reason for hiding this comment

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

Avoid use import xxx.* and please check all.

@codelipenghui codelipenghui added the doc-required Your PR changes impact docs and you will update later. label Apr 25, 2020
@hangc0276 hangc0276 force-pushed the managed_ledger_metric branch from c78374d to 6d7075b Compare April 25, 2020 10:30
@hangc0276
Copy link
Contributor Author

The change looks good. Could you please help add the description of the new metric into https://pulsar.apache.org/docs/en/reference-metrics/#broker? This is very helpful for users to understand these new metrics.

thanks for your feedback. I have update the document.

@hangc0276
Copy link
Contributor Author

/pulsarbot run-failure-checks

@hangc0276
Copy link
Contributor Author

/pulsarbot run-failure-checks

Copy link
Member

@sijie sijie left a comment

Choose a reason for hiding this comment

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

Just out of curiosity, why do you implement a new StatsProvider? BookKeeper already has the prometheus data provider. Why not re-use it?

@hangc0276
Copy link
Contributor Author

Just out of curiosity, why do you implement a new StatsProvider? BookKeeper already has the prometheus data provider. Why not re-use it?

The reasons of implementing a new prometheus statsProvider as follows:

  1. bookkeeper's prometheus statsProvider will open an individual ip:port to provide http service, which can't be integrated into pulsar's now available prometheus http service which is just concat metric string into prometheus format.
  2. some components of bookkeeper's prometheus statsProvider dependent on are package private, which can't be accessed out of org.apache.bookkeeper.stats.prometheus package,
    • PrometheusStatsLogger has package private constructor
    • SimpleGauge has package private getSample method
    • PrometheusTextFormatUtil has package private writeGauge, writeCounter, writeOpStat method
    • DataSketchesOpStatsLogger has package private constructor

whether have other new ways to reuse bookkeeper's prometheus statsProvider?

@hangc0276 hangc0276 requested review from codelipenghui and sijie May 2, 2020 02:56
@hangc0276
Copy link
Contributor Author

/pulsarbot run-failure-checks

@hangc0276
Copy link
Contributor Author

/pulsarbot run-failure-checks

1 similar comment
@hangc0276
Copy link
Contributor Author

/pulsarbot run-failure-checks

@sijie
Copy link
Member

sijie commented May 7, 2020

@hangc0276 sorry for late response. I generally don't think copying the files and maintain another implementation in Pulsar is a good idea.

bookkeeper's prometheus statsProvider will open an individual ip:port

the prometheus stats provider also work well with bookie HTTP server as well. There is a flag to disable the prometheus HTTP server in the stats provider by setting prometheusStatsHttpEnable to false.

https://github.com/apache/bookkeeper/blob/master/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusMetricsProvider.java#L63

some components of bookkeeper's prometheus statsProvider dependent

Why do you need to access those components? Can't you use the bookkeeper stats library? You can always use Stats.loadProvider to load the stats provider, no?

https://github.com/apache/bookkeeper/blob/master/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/Stats.java#L43

@hangc0276
Copy link
Contributor Author

@hangc0276 sorry for late response. I generally don't think copying the files and maintain another implementation in Pulsar is a good idea.

bookkeeper's prometheus statsProvider will open an individual ip:port

the prometheus stats provider also work well with bookie HTTP server as well. There is a flag to disable the prometheus HTTP server in the stats provider by setting prometheusStatsHttpEnable to false.

https://github.com/apache/bookkeeper/blob/master/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusMetricsProvider.java#L63

some components of bookkeeper's prometheus statsProvider dependent

Why do you need to access those components? Can't you use the bookkeeper stats library? You can always use Stats.loadProvider to load the stats provider, no?

https://github.com/apache/bookkeeper/blob/master/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/Stats.java#L43

Sorry, i am not found the flag. I will fix it soon. Thanks for your feedback.

@ckdarby
Copy link

ckdarby commented May 15, 2020

@hangc0276 This would be very valuable to help us debug some throughput & performance issues we're seeing when on EBS and would like to pinpoint our bottlenecks.

Appreciate the work, do you need anything else to continue forward with this?

@hangc0276
Copy link
Contributor Author

@hangc0276 This would be very valuable to help us debug some throughput & performance issues we're seeing when on EBS and would like to pinpoint our bottlenecks.

Appreciate the work, do you need anything else to continue forward with this?

Thanks for your reply. With the bookie client metric, we could easily find the bottlenecks of pulsar. I need to use bookkeeper prometheus lib instead of the copy one, and fix the prometheus test case. Because the pulsar prometheus test case is conflict with bookkeeper prometheus output format. I will fix it soon.

jiazhai
jiazhai previously approved these changes May 30, 2020
@jiazhai jiazhai self-requested a review May 30, 2020 10:00
@jiazhai jiazhai dismissed their stale review May 30, 2020 10:01

will review later

@hangc0276
Copy link
Contributor Author

@sijie I try to use bookkeeper prometheus provider lib the export the bookie client metric, but failed. The exception is

21:49:06.142 [prometheus-stats-43-1] ERROR org.apache.bookkeeper.common.util.SafeRunnable - Unexpected throwable caught
java.lang.ExceptionInInitializerError: null
        at org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsServlet.lambda$doGet$0(PrometheusMetricsServlet.java:70) ~[org.apache.pulsar-pulsar-broker-2.6.0-SNAPSHOT.jar:2.6.0-SNAPSHOT]
        at org.apache.bookkeeper.mledger.util.SafeRun$1.safeRun(SafeRun.java:32) ~[org.apache.pulsar-managed-ledger-2.6.0-SNAPSHOT.jar:2.6.0-SNAPSHOT]
        at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) [org.apache.bookkeeper-bookkeeper-common-4.10.0.jar:4.10.0]
        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [?:1.8.0_92]
        at java.util.concurrent.FutureTask.run(FutureTask.java:266) [?:1.8.0_92]
        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180) [?:1.8.0_92]
        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293) [?:1.8.0_92]
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [?:1.8.0_92]
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [?:1.8.0_92]
        at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) [io.netty-netty-common-4.1.48.Final.jar:4.1.48.Final]
        at java.lang.Thread.run(Thread.java:745) [?:1.8.0_92]
Caused by: java.lang.IllegalArgumentException: Collector already registered that provides name: process_cpu_seconds_total
        at io.prometheus.client.CollectorRegistry.register(CollectorRegistry.java:54) ~[io.prometheus-simpleclient-0.5.0.jar:?]
        at io.prometheus.client.Collector.register(Collector.java:139) ~[io.prometheus-simpleclient-0.5.0.jar:?]
        at io.prometheus.client.Collector.register(Collector.java:132) ~[io.prometheus-simpleclient-0.5.0.jar:?]
        at io.prometheus.client.hotspot.DefaultExports.initialize(DefaultExports.java:23) ~[io.prometheus-simpleclient_hotspot-0.5.0.jar:?]
        at org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator.<clinit>(PrometheusMetricsGenerator.java:61) ~[org.apache.pulsar-pulsar-broker-2.6.0-SNAPSHOT.jar:2.6.0-SNAPSHOT]
        ... 11 more

The reason is bookkeeper lib PrometheusMetricsProvider will export the follow metrics by default and can't turn off.

this.registerMetrics(new StandardExports());
this.registerMetrics(new MemoryPoolsExports());
this.registerMetrics(new GarbageCollectorExports());
this.registerMetrics(new ThreadExports());
this.registerMetrics(Gauge.build("jvm_memory_direct_bytes_used", "-").create().setChild(new Child() {
         public double get() {
                return PrometheusMetricsProvider.directMemoryUsage != null ? (double)PrometheusMetricsProvider.directMemoryUsage.longValue() : 0.0D / 0.0;
            }
        }, new String[0]));
        this.registerMetrics(Gauge.build("jvm_memory_direct_bytes_max", "-").create().setChild(new Child() {
            public double get() {
                return (double)PlatformDependent.maxDirectMemory();
            }
        }, new String[0]));

However the pulsar broker prometheus metric also export those metrics to prometheus, some of the metrics registered twice, which will lead to metric export failed.

Could you give me some ideas ?

@codelipenghui
Copy link
Contributor

move to 2.7.0 first.

@codelipenghui codelipenghui modified the milestones: 2.6.0, 2.7.0 Jun 4, 2020
@codelipenghui codelipenghui modified the milestones: 2.7.0, 2.6.0 Jun 5, 2020
@hangc0276
Copy link
Contributor Author

/pulsarbot run-failure-checks

Copy link
Member

@sijie sijie left a comment

Choose a reason for hiding this comment

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

@hangc0276 we can add this change first. Can you add a configuration setting either in bookkeeper side or pulsar side to allow excluding jvm settings? So you can avoid this situation and eventually use the bookkeeper metrics library.

@codelipenghui codelipenghui merged commit b983abf into apache:master Jun 8, 2020
@hangc0276
Copy link
Contributor Author

@hangc0276 we can add this change first. Can you add a configuration setting either in bookkeeper side or pulsar side to allow excluding jvm settings? So you can avoid this situation and eventually use the bookkeeper metrics library.

@sijie Ok, I think it's better to add a configuration setting in bookkeeper side. As bookkeeper client metric, it should not import the external jvm and other metrics. I will fix it on bookkeeper side, but it depend on the bookkeeper release.

@sijie
Copy link
Member

sijie commented Jun 10, 2020

@hangc0276 sounds good to me.

@Anonymitaet
Copy link
Member

@Anonymitaet Anonymitaet removed the doc-required Your PR changes impact docs and you will update later. label Jun 10, 2020
cdbartholomew pushed a commit to kafkaesque-io/pulsar that referenced this pull request Jul 24, 2020
### Motivation
Pulsar use bookkeeper as distributed log storage, and init a bookie client to read/write data from/to bookkeeper. However the pulsar bookie client use default ` NullStatsLogger.INSTANCE` to expose runtime metric, which doesn't expose to prometheus or other state storage. When tuning pulsar bookie client performance, we doesn't have any bookie metric to measure where is the bottleneck.

### Changes
I implement a prometheus state provider, and use it to trace bookie client runtime metric, and expose it to prometheus.
huangdx0726 pushed a commit to huangdx0726/pulsar that referenced this pull request Aug 24, 2020
### Motivation
Pulsar use bookkeeper as distributed log storage, and init a bookie client to read/write data from/to bookkeeper. However the pulsar bookie client use default ` NullStatsLogger.INSTANCE` to expose runtime metric, which doesn't expose to prometheus or other state storage. When tuning pulsar bookie client performance, we doesn't have any bookie metric to measure where is the bottleneck.

### Changes
I implement a prometheus state provider, and use it to trace bookie client runtime metric, and expose it to prometheus.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants