-
Notifications
You must be signed in to change notification settings - Fork 3.7k
[improve][monitor]Add prometheusRawMetricsProvider support #17531
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Changes from all commits
ff6ca6e
a883290
2c10276
6cf8f62
8704521
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,138 @@ | ||
| /** | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, | ||
| * software distributed under the License is distributed on an | ||
| * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
| * KIND, either express or implied. See the License for the | ||
| * specific language governing permissions and limitations | ||
| * under the License. | ||
| */ | ||
| package org.apache.pulsar.broker.stats.prometheus.metrics; | ||
|
|
||
| import com.google.common.base.Joiner; | ||
| import io.prometheus.client.Collector; | ||
| import java.util.Map; | ||
| import java.util.TreeMap; | ||
| import java.util.concurrent.ConcurrentHashMap; | ||
| import java.util.concurrent.ConcurrentMap; | ||
| import org.apache.bookkeeper.stats.Counter; | ||
| import org.apache.bookkeeper.stats.Gauge; | ||
| import org.apache.bookkeeper.stats.OpStatsLogger; | ||
| import org.apache.bookkeeper.stats.StatsLogger; | ||
|
|
||
| /** | ||
| * A {@code StatsLogger} that caches the stats objects created by other {@code StatsLogger}. | ||
| */ | ||
| public class CachingStatsLogger implements StatsLogger { | ||
|
|
||
| protected final StatsLogger underlying; | ||
| protected final ConcurrentMap<ScopeContext, Counter> counters; | ||
| protected final ConcurrentMap<ScopeContext, OpStatsLogger> opStatsLoggers; | ||
| protected final ConcurrentMap<ScopeContext, StatsLogger> scopeStatsLoggers; | ||
| protected final ConcurrentMap<ScopeContext, StatsLogger> scopeLabelStatsLoggers; | ||
|
|
||
| private final String scope; | ||
| private final Map<String, String> labels; | ||
|
|
||
| public CachingStatsLogger(String scope, StatsLogger statsLogger, Map<String, String> labels) { | ||
| this.scope = scope; | ||
| this.labels = labels; | ||
| this.underlying = statsLogger; | ||
| this.counters = new ConcurrentHashMap<>(); | ||
| this.opStatsLoggers = new ConcurrentHashMap<>(); | ||
| this.scopeStatsLoggers = new ConcurrentHashMap<>(); | ||
| this.scopeLabelStatsLoggers = new ConcurrentHashMap<>(); | ||
| } | ||
|
|
||
| @Override | ||
| public int hashCode() { | ||
| return underlying.hashCode(); | ||
| } | ||
|
|
||
| @Override | ||
| public boolean equals(Object obj) { | ||
| if (!(obj instanceof CachingStatsLogger)) { | ||
| return false; | ||
| } | ||
| CachingStatsLogger another = (CachingStatsLogger) obj; | ||
| return underlying.equals(another.underlying); | ||
| } | ||
|
|
||
| @Override | ||
| public String toString() { | ||
| return underlying.toString(); | ||
| } | ||
|
|
||
| @Override | ||
| public OpStatsLogger getOpStatsLogger(String name) { | ||
| return opStatsLoggers.computeIfAbsent(scopeContext(name), x -> underlying.getOpStatsLogger(name)); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why would you cache it under key = |
||
| } | ||
|
|
||
| @Override | ||
| public Counter getCounter(String name) { | ||
| return counters.computeIfAbsent(scopeContext(name), x -> underlying.getCounter(name)); | ||
| } | ||
|
|
||
| @Override | ||
| public <T extends Number> void registerGauge(String name, Gauge<T> gauge) { | ||
| underlying.registerGauge(name, gauge); | ||
| } | ||
|
|
||
| @Override | ||
| public <T extends Number> void unregisterGauge(String name, Gauge<T> gauge) { | ||
| underlying.unregisterGauge(name, gauge); | ||
| } | ||
|
|
||
| @Override | ||
| public StatsLogger scope(String name) { | ||
| return scopeStatsLoggers.computeIfAbsent(scopeContext(name), | ||
| x -> new CachingStatsLogger(scope, underlying.scope(name), labels)); | ||
| } | ||
|
|
||
| @Override | ||
| public StatsLogger scopeLabel(String labelName, String labelValue) { | ||
| Map<String, String> newLabels = new TreeMap<>(labels); | ||
| newLabels.put(labelName, labelValue); | ||
| return scopeLabelStatsLoggers.computeIfAbsent(new ScopeContext(completeName(""), newLabels), | ||
| x -> new CachingStatsLogger(scope, underlying.scopeLabel(labelName, labelValue), newLabels)); | ||
| } | ||
|
|
||
|
|
||
| @Override | ||
| public void removeScope(String name, StatsLogger statsLogger) { | ||
| scopeStatsLoggers.remove(scopeContext(name), statsLogger); | ||
| } | ||
|
|
||
| /** | ||
| Thread-scoped stats not currently supported. | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why is it not supported? |
||
| */ | ||
| @Override | ||
| public OpStatsLogger getThreadScopedOpStatsLogger(String name) { | ||
| return getOpStatsLogger(name); | ||
| } | ||
|
|
||
| /** | ||
| Thread-scoped stats not currently supported. | ||
| */ | ||
| @Override | ||
| public Counter getThreadScopedCounter(String name) { | ||
| return getCounter(name); | ||
| } | ||
|
|
||
| private ScopeContext scopeContext(String name) { | ||
| return new ScopeContext(completeName(name), labels); | ||
| } | ||
|
|
||
| private String completeName(String name) { | ||
| return Collector.sanitizeMetricName(scope.isEmpty() ? name : Joiner.on('_').join(scope, name)); | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,64 @@ | ||
| /** | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, | ||
| * software distributed under the License is distributed on an | ||
| * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
| * KIND, either express or implied. See the License for the | ||
| * specific language governing permissions and limitations | ||
| * under the License. | ||
| */ | ||
| package org.apache.pulsar.broker.stats.prometheus.metrics; | ||
|
|
||
| import java.util.Collections; | ||
| import java.util.concurrent.ConcurrentHashMap; | ||
| import java.util.concurrent.ConcurrentMap; | ||
| import org.apache.bookkeeper.stats.StatsLogger; | ||
| import org.apache.bookkeeper.stats.StatsProvider; | ||
| import org.apache.commons.configuration.Configuration; | ||
|
|
||
| /** | ||
| * A {@code CachingStatsProvider} adds the caching functionality to an existing {@code StatsProvider}. | ||
| * | ||
| * <p>The stats provider will cache the stats objects created by the other {@code StatsProvider} to allow | ||
| * the reusability of stats objects and avoid creating a lot of stats objects. | ||
| */ | ||
| public class CachingStatsProvider implements StatsProvider { | ||
|
|
||
| protected final StatsProvider underlying; | ||
| protected final ConcurrentMap<String, StatsLogger> statsLoggers; | ||
|
|
||
| public CachingStatsProvider(StatsProvider provider) { | ||
| this.underlying = provider; | ||
| this.statsLoggers = new ConcurrentHashMap<>(); | ||
| } | ||
|
|
||
| @Override | ||
| public void start(Configuration conf) { | ||
| this.underlying.start(conf); | ||
| } | ||
|
|
||
| @Override | ||
| public void stop() { | ||
| this.underlying.stop(); | ||
| } | ||
|
|
||
| @Override | ||
| public StatsLogger getStatsLogger(String scope) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Now that we have labels, should
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. updated. |
||
| return statsLoggers.computeIfAbsent(scope, | ||
| x -> new CachingStatsLogger(scope, underlying.getStatsLogger(scope), Collections.emptyMap())); | ||
| } | ||
|
|
||
| @Override | ||
| public String getStatsName(String... statsComponents) { | ||
| return underlying.getStatsName(statsComponents); | ||
| } | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It's weird to have a cast in software you wrote - you usually see this when you use third-party software. Why can't you change the original variable type?