Skip to content

Commit

Permalink
Collect Prometheus latency stats using DataSketches
Browse files Browse the repository at this point in the history
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
merlimat authored and sijie committed Mar 15, 2018
1 parent 70b2e4c commit 3bff199
Show file tree
Hide file tree
Showing 17 changed files with 913 additions and 231 deletions.
3 changes: 3 additions & 0 deletions bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt
Original file line number Diff line number Diff line change
Expand Up @@ -271,6 +271,8 @@ Apache Software License, Version 2.
- lib/org.eclipse.jetty-jetty-util-9.4.5.v20170502.jar [34]
- lib/org.rocksdb-rocksdbjni-5.8.6.jar [35]
- lib/com.beust-jcommander-1.48.jar [36]
- lib/com.yahoo.datasketches-memory-0.8.3.jar [37]
- lib/com.yahoo.datasketches-sketches-core-0.8.3.jar [37]

[1] Source available at https://github.com/FasterXML/jackson-annotations/tree/jackson-annotations-2.8.9
[2] Source available at https://github.com/FasterXML/jackson-core/tree/jackson-core-2.8.9
Expand Down Expand Up @@ -307,6 +309,7 @@ Apache Software License, Version 2.
[34] Source available at https://github.com/eclipse/jetty.project/tree/jetty-9.4.5.v20170502
[35] Source available at https://github.com/facebook/rocksdb/tree/v5.8.6
[36] Source available at https://github.com/cbeust/jcommander/tree/jcommander-1.48
[37] Source available at https://github.com/DataSketches/sketches-core/tree/sketches-0.8.3

------------------------------------------------------------------------------------
lib/io.netty-netty-3.10.1.Final.jar contains the extensions to Java Collections Framework which has
Expand Down
3 changes: 3 additions & 0 deletions bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt
Original file line number Diff line number Diff line change
Expand Up @@ -236,6 +236,8 @@ Apache Software License, Version 2.
- lib/org.eclipse.jetty-jetty-util-9.4.5.v20170502.jar [21]
- lib/org.rocksdb-rocksdbjni-5.8.6.jar [22]
- lib/com.beust-jcommander-1.48.jar [23]
- lib/com.yahoo.datasketches-memory-0.8.3.jar [24]
- lib/com.yahoo.datasketches-sketches-core-0.8.3.jar [24]

[1] Source available at https://github.com/FasterXML/jackson-annotations/tree/jackson-annotations-2.8.9
[2] Source available at https://github.com/FasterXML/jackson-core/tree/jackson-core-2.8.9
Expand All @@ -260,6 +262,7 @@ Apache Software License, Version 2.
[21] Source available at https://github.com/eclipse/jetty.project/tree/jetty-9.4.5.v20170502
[22] Source available at https://github.com/facebook/rocksdb/tree/v5.8.6
[23] Source available at https://github.com/cbeust/jcommander/tree/jcommander-1.48
[24] Source available at https://github.com/DataSketches/sketches-core/tree/sketches-0.8.3

------------------------------------------------------------------------------------
lib/io.netty-netty-all-4.1.12.Final.jar bundles some 3rd party dependencies
Expand Down
10 changes: 10 additions & 0 deletions bookkeeper-stats-providers/prometheus-metrics-provider/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,11 @@
<groupId>io.prometheus</groupId>
<artifactId>simpleclient_servlet</artifactId>
</dependency>

<dependency>
<groupId>io.netty</groupId>
<artifactId>netty-all</artifactId>
</dependency>

<dependency>
<groupId>org.eclipse.jetty</groupId>
Expand All @@ -66,6 +71,11 @@
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
</dependency>

<dependency>
<groupId>com.yahoo.datasketches</groupId>
<artifactId>sketches-core</artifactId>
</dependency>

</dependencies>
</project>
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);
}
};
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,46 +16,41 @@
*/
package org.apache.bookkeeper.stats.prometheus;

import io.prometheus.client.Collector;
import io.prometheus.client.CollectorRegistry;
import io.prometheus.client.Gauge;
import java.util.concurrent.atomic.LongAdder;

import org.apache.bookkeeper.stats.Counter;

/**
* A {@link Counter} implementation based on <i>Prometheus</i> metrics library.
* {@link Counter} implementation based on {@link LongAdder}.
*
* <p>LongAdder keeps a counter per-thread and then aggregates to get the result, in order to avoid contention between
* multiple threads.
*/
public class PrometheusCounter implements Counter {

private final Gauge gauge;

public PrometheusCounter(CollectorRegistry registry, String name) {
this.gauge = PrometheusUtil.safeRegister(registry,
Gauge.build().name(Collector.sanitizeMetricName(name)).help("-").create());
}
public class LongAdderCounter implements Counter {
private final LongAdder counter = new LongAdder();

@Override
public void clear() {
gauge.clear();
counter.reset();
}

@Override
public void inc() {
gauge.inc();
counter.increment();
}

@Override
public void dec() {
gauge.dec();
counter.decrement();
}

@Override
public void add(long delta) {
gauge.inc(delta);
counter.add(delta);
}

@Override
public Long get() {
return (long) gauge.get();
return counter.sum();
}

}
Loading

0 comments on commit 3bff199

Please sign in to comment.