-
Notifications
You must be signed in to change notification settings - Fork 895
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Collect Prometheus latency stats using DataSketches
The implementation for collecting and estimating the latency quantiles in the Prometheus Java client library is very slow and it is impacting the the bookie performance. I have added a micro-benchmark that tests our various stats providers. These tests are simulating 16 concurrent threads updating the stats. #### Counter increment ``` Benchmark (statsProvider) Mode Cnt Score Error Units StatsLoggerBenchmark.counterIncrement Prometheus thrpt 3 391.882 ± 786.987 ops/us StatsLoggerBenchmark.counterIncrement Codahale thrpt 3 449.341 ± 1337.736 ops/us StatsLoggerBenchmark.counterIncrement Twitter thrpt 3 43.354 ± 9.331 ops/us StatsLoggerBenchmark.counterIncrement Ostrich thrpt 3 43.790 ± 1.332 ops/us ``` Here prometheus is fast, though not as fast as a simple `LongAdder` which can reach ~500M ops/sec. #### Latency quantiles ``` Benchmark (statsProvider) Mode Cnt Score Error Units StatsLoggerBenchmark.recordLatency Prometheus thrpt 3 0.255 ± 0.667 ops/us StatsLoggerBenchmark.recordLatency Codahale thrpt 3 4.963 ± 1.671 ops/us StatsLoggerBenchmark.recordLatency Twitter thrpt 3 4.793 ± 0.766 ops/us StatsLoggerBenchmark.recordLatency Ostrich thrpt 3 2.473 ± 6.394 ops/us ``` Here is where Prometheus is super-slow: 250K ops/second max, mostly due to contention and GC pressure. ## Modification I have re-adapted a stats collector I had done in the Yahoo branch: https://github.com/yahoo/bookkeeper/tree/yahoo-4.3/bookkeeper-stats-providers/datasketches-metrics-provider/src/main/java/org/apache/bokkeeper/stats/datasketches This is based on the [DataSketches](https://datasketches.github.io/) library to have very fast and lightweight quantile estimates (along with a number of other operations), plus some tricks to avoid concurrency issues by using thread local collectors and aggregating when needed in background. After the change, the throughput is 150x the original prometheus collector. ``` Benchmark (statsProvider) Mode Cnt Score Error Units StatsLoggerBenchmark.counterIncrement Prometheus thrpt 3 531.906 ± 129.602 ops/us StatsLoggerBenchmark.recordLatency Prometheus thrpt 3 27.538 ± 5.893 ops/us ``` It is worth noting that the main bottle-neck in the `recordLatency` test is now the `System.nanoTime()` call used to pass different samples to the stat logger. `System.nanoTime()` is not super fast: ``` Benchmark (statsProvider) Mode Cnt Score Error Units StatsLoggerBenchmark.currentTimeMillis N/A thrpt 3 161.502 ± 267.238 ops/us StatsLoggerBenchmark.nanoTime N/A thrpt 3 32.822 ± 2.256 ops/us ``` By removing the `System.nanoTime()` call from the benchmark, the Prometheus+DataSketches collector results in: ``` Benchmark (statsProvider) Mode Cnt Score Error Units StatsLoggerBenchmark.recordLatency Prometheus thrpt 3 108.542 ± 31.848 ops/us ``` Author: Matteo Merli <mmerli@apache.org> Reviewers: Jia Zhai <None>, Sijie Guo <sijie@apache.org> This closes #1245 from merlimat/prometheus-datasketches
- Loading branch information
Showing
17 changed files
with
913 additions
and
231 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
199 changes: 199 additions & 0 deletions
199
...vider/src/main/java/org/apache/bookkeeper/stats/prometheus/DataSketchesOpStatsLogger.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,199 @@ | ||
/** | ||
* 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.bookkeeper.stats.prometheus; | ||
|
||
import com.yahoo.sketches.quantiles.DoublesSketch; | ||
import com.yahoo.sketches.quantiles.DoublesSketchBuilder; | ||
import com.yahoo.sketches.quantiles.DoublesUnion; | ||
import com.yahoo.sketches.quantiles.DoublesUnionBuilder; | ||
|
||
import io.netty.util.concurrent.FastThreadLocal; | ||
|
||
import java.util.Map; | ||
import java.util.concurrent.ConcurrentHashMap; | ||
import java.util.concurrent.TimeUnit; | ||
import java.util.concurrent.atomic.LongAdder; | ||
import java.util.concurrent.locks.StampedLock; | ||
|
||
import org.apache.bookkeeper.stats.OpStatsData; | ||
import org.apache.bookkeeper.stats.OpStatsLogger; | ||
|
||
/** | ||
* OpStatsLogger implementation that uses DataSketches library to calculate the approximated latency quantiles. | ||
*/ | ||
public class DataSketchesOpStatsLogger implements OpStatsLogger { | ||
|
||
/* | ||
* Use 2 rotating thread local accessor so that we can safely swap them. | ||
*/ | ||
private volatile ThreadLocalAccessor current; | ||
private volatile ThreadLocalAccessor replacement; | ||
|
||
/* | ||
* These are the sketches where all the aggregated results are published. | ||
*/ | ||
private volatile DoublesSketch successResult; | ||
private volatile DoublesSketch failResult; | ||
|
||
private final LongAdder successCountAdder = new LongAdder(); | ||
private final LongAdder failCountAdder = new LongAdder(); | ||
|
||
private final LongAdder successSumAdder = new LongAdder(); | ||
private final LongAdder failSumAdder = new LongAdder(); | ||
|
||
DataSketchesOpStatsLogger() { | ||
this.current = new ThreadLocalAccessor(); | ||
this.replacement = new ThreadLocalAccessor(); | ||
} | ||
|
||
@Override | ||
public void registerFailedEvent(long eventLatency, TimeUnit unit) { | ||
double valueMillis = unit.toMicros(eventLatency) / 1000.0; | ||
|
||
failCountAdder.increment(); | ||
failSumAdder.add((long) valueMillis); | ||
|
||
LocalData localData = current.localData.get(); | ||
|
||
long stamp = localData.lock.readLock(); | ||
try { | ||
localData.failSketch.update(valueMillis); | ||
} finally { | ||
localData.lock.unlockRead(stamp); | ||
} | ||
} | ||
|
||
@Override | ||
public void registerSuccessfulEvent(long eventLatency, TimeUnit unit) { | ||
double valueMillis = unit.toMicros(eventLatency) / 1000.0; | ||
|
||
successCountAdder.increment(); | ||
successSumAdder.add((long) valueMillis); | ||
|
||
LocalData localData = current.localData.get(); | ||
|
||
long stamp = localData.lock.readLock(); | ||
try { | ||
localData.successSketch.update(valueMillis); | ||
} finally { | ||
localData.lock.unlockRead(stamp); | ||
} | ||
} | ||
|
||
@Override | ||
public void registerSuccessfulValue(long value) { | ||
successCountAdder.increment(); | ||
successSumAdder.add(value); | ||
|
||
LocalData localData = current.localData.get(); | ||
|
||
long stamp = localData.lock.readLock(); | ||
try { | ||
localData.successSketch.update(value); | ||
} finally { | ||
localData.lock.unlockRead(stamp); | ||
} | ||
} | ||
|
||
@Override | ||
public void registerFailedValue(long value) { | ||
failCountAdder.increment(); | ||
failSumAdder.add(value); | ||
|
||
LocalData localData = current.localData.get(); | ||
|
||
long stamp = localData.lock.readLock(); | ||
try { | ||
localData.failSketch.update(value); | ||
} finally { | ||
localData.lock.unlockRead(stamp); | ||
} | ||
} | ||
|
||
@Override | ||
public OpStatsData toOpStatsData() { | ||
// Not relevant as we don't use JMX here | ||
throw new UnsupportedOperationException(); | ||
} | ||
|
||
@Override | ||
public void clear() { | ||
// Not relevant as we don't use JMX here | ||
throw new UnsupportedOperationException(); | ||
} | ||
|
||
public void rotateLatencyCollection() { | ||
// Swap current with replacement | ||
ThreadLocalAccessor local = current; | ||
current = replacement; | ||
replacement = local; | ||
|
||
final DoublesUnion aggregateSuccesss = new DoublesUnionBuilder().build(); | ||
final DoublesUnion aggregateFail = new DoublesUnionBuilder().build(); | ||
local.map.forEach((localData, b) -> { | ||
long stamp = localData.lock.writeLock(); | ||
try { | ||
aggregateSuccesss.update(localData.successSketch); | ||
localData.successSketch.reset(); | ||
aggregateFail.update(localData.failSketch); | ||
localData.failSketch.reset(); | ||
} finally { | ||
localData.lock.unlockWrite(stamp); | ||
} | ||
}); | ||
|
||
successResult = aggregateSuccesss.getResultAndReset(); | ||
failResult = aggregateFail.getResultAndReset(); | ||
} | ||
|
||
public long getCount(boolean success) { | ||
return success ? successCountAdder.sum() : failCountAdder.sum(); | ||
} | ||
|
||
public long getSum(boolean success) { | ||
return success ? successSumAdder.sum() : failSumAdder.sum(); | ||
} | ||
|
||
public double getQuantileValue(boolean success, double quantile) { | ||
DoublesSketch s = success ? successResult : failResult; | ||
return s != null ? s.getQuantile(quantile) : Double.NaN; | ||
} | ||
|
||
private static class LocalData { | ||
private final DoublesSketch successSketch = new DoublesSketchBuilder().build(); | ||
private final DoublesSketch failSketch = new DoublesSketchBuilder().build(); | ||
private final StampedLock lock = new StampedLock(); | ||
} | ||
|
||
private static class ThreadLocalAccessor { | ||
private final Map<LocalData, Boolean> map = new ConcurrentHashMap<>(); | ||
private final FastThreadLocal<LocalData> localData = new FastThreadLocal<LocalData>() { | ||
|
||
@Override | ||
protected LocalData initialValue() throws Exception { | ||
LocalData localData = new LocalData(); | ||
map.put(localData, Boolean.TRUE); | ||
return localData; | ||
} | ||
|
||
@Override | ||
protected void onRemoval(LocalData value) throws Exception { | ||
map.remove(value); | ||
} | ||
}; | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.