001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.metrics.impl;
019
020import java.util.Arrays;
021import java.util.concurrent.atomic.AtomicLong;
022import java.util.concurrent.atomic.LongAdder;
023import java.util.stream.Stream;
024
025import org.apache.hadoop.hbase.metrics.Snapshot;
026import org.apache.hadoop.hbase.util.AtomicUtils;
027import org.apache.yetus.audience.InterfaceAudience;
028import org.apache.yetus.audience.InterfaceStability;
029
030/**
031 * FastLongHistogram is a thread-safe class that estimate distribution of data and computes the
032 * quantiles.
033 */
034@InterfaceAudience.Private
035@InterfaceStability.Evolving
036public class FastLongHistogram {
037
038  /**
039   * Default number of bins.
040   */
041  public static final int DEFAULT_NBINS = 255;
042
043  public static final double[] DEFAULT_QUANTILES =
044      new double[]{0.25, 0.5, 0.75, 0.90, 0.95, 0.98, 0.99, 0.999};
045
046  /**
047   * Bins is a class containing a list of buckets(or bins) for estimation histogram of some data.
048   */
049  private static class Bins {
050    private final LongAdder[] counts;
051    // inclusive
052    private final long binsMin;
053    // exclusive
054    private final long binsMax;
055    private final long bins10XMax;
056    private final AtomicLong min = new AtomicLong(Long.MAX_VALUE);
057    private final AtomicLong max = new AtomicLong(0L);
058
059    private final LongAdder count = new LongAdder();
060    private final LongAdder total = new LongAdder();
061
062    // set to true when any of data has been inserted to the Bins. It is set after the counts are
063    // updated.
064    private volatile boolean hasData = false;
065
066    /**
067     * The constructor for creating a Bins without any prior data.
068     */
069    public Bins(int numBins) {
070      counts = createCounters(numBins);
071      this.binsMin = 1L;
072
073      // These two numbers are total guesses
074      // and should be treated as highly suspect.
075      this.binsMax = 1000;
076      this.bins10XMax = binsMax * 10;
077    }
078
079    /**
080     * The constructor for creating a Bins with last Bins.
081     */
082    public Bins(Bins last, int numBins, double minQ, double maxQ) {
083      long[] values = last.getQuantiles(new double[] { minQ, maxQ });
084      long wd = values[1] - values[0] + 1;
085      // expand minQ and maxQ in two ends back assuming uniform distribution
086      this.binsMin = Math.max(0L, (long) (values[0] - wd * minQ));
087      long binsMax = (long) (values[1] + wd * (1 - maxQ)) + 1;
088      // make sure each of bins is at least of width 1
089      this.binsMax = Math.max(binsMax, this.binsMin + numBins);
090      this.bins10XMax = Math.max((long) (values[1] + (binsMax - 1) * 9), this.binsMax + 1);
091
092      this.counts = createCounters(numBins);
093    }
094
095    private LongAdder[] createCounters(int numBins) {
096      return Stream.generate(LongAdder::new).limit(numBins + 3).toArray(LongAdder[]::new);
097    }
098
099    private int getIndex(long value) {
100      if (value < this.binsMin) {
101        return 0;
102      } else if (value > this.bins10XMax) {
103        return this.counts.length - 1;
104      } else if (value >= this.binsMax) {
105        return this.counts.length - 2;
106      }
107      // compute the position
108      return 1 + (int) ((value - this.binsMin) * (this.counts.length - 3) /
109          (this.binsMax - this.binsMin));
110
111    }
112
113    /**
114     * Adds a value to the histogram.
115     */
116    public void add(long value, long count) {
117      if (value < 0) {
118        // The whole computation is completely thrown off if there are negative numbers
119        //
120        // Normally we would throw an IllegalArgumentException however this is the metrics
121        // system and it should be completely safe at all times.
122        // So silently throw it away.
123        return;
124      }
125      AtomicUtils.updateMin(min, value);
126      AtomicUtils.updateMax(max, value);
127
128      this.count.add(count);
129      this.total.add(value * count);
130
131      int pos = getIndex(value);
132      this.counts[pos].add(count);
133
134      // hasData needs to be updated as last
135      this.hasData = true;
136    }
137
138    /**
139     * Computes the quantiles give the ratios.
140     */
141    public long[] getQuantiles(double[] quantiles) {
142      if (!hasData) {
143        // No data yet.
144        return new long[quantiles.length];
145      }
146
147      // Make a snapshot of lowerCounter, higherCounter and bins.counts to counts.
148      // This is not synchronized, but since the counter are accumulating, the result is a good
149      // estimation of a snapshot.
150      long[] counts = new long[this.counts.length];
151      long total = 0L;
152      for (int i = 0; i < this.counts.length; i++) {
153        counts[i] = this.counts[i].sum();
154        total += counts[i];
155      }
156
157      int rIndex = 0;
158      double qCount = total * quantiles[0];
159      long cum = 0L;
160
161      long[] res = new long[quantiles.length];
162      countsLoop: for (int i = 0; i < counts.length; i++) {
163        // mn and mx define a value range
164        long mn, mx;
165        if (i == 0) {
166          mn = this.min.get();
167          mx = this.binsMin;
168        } else if (i == counts.length - 1) {
169          mn = this.bins10XMax;
170          mx = this.max.get();
171        } else if (i == counts.length - 2) {
172          mn = this.binsMax;
173          mx = this.bins10XMax;
174        } else {
175          mn = this.binsMin + (i - 1) * (this.binsMax - this.binsMin) / (this.counts.length - 3);
176          mx = this.binsMin + i * (this.binsMax - this.binsMin) / (this.counts.length - 3);
177        }
178
179        if (mx < this.min.get()) {
180          continue;
181        }
182        if (mn > this.max.get()) {
183          break;
184        }
185        mn = Math.max(mn, this.min.get());
186        mx = Math.min(mx, this.max.get());
187
188        // lastCum/cum are the corresponding counts to mn/mx
189        double lastCum = cum;
190        cum += counts[i];
191
192        // fill the results for qCount is within current range.
193        while (qCount <= cum) {
194          if (cum == lastCum) {
195            res[rIndex] = mn;
196          } else {
197            res[rIndex] = (long) ((qCount - lastCum) * (mx - mn) / (cum - lastCum) + mn);
198          }
199
200          // move to next quantile
201          rIndex++;
202          if (rIndex >= quantiles.length) {
203            break countsLoop;
204          }
205          qCount = total * quantiles[rIndex];
206        }
207      }
208      // In case quantiles contains values >= 100%
209      for (; rIndex < quantiles.length; rIndex++) {
210        res[rIndex] = this.max.get();
211      }
212
213      return res;
214    }
215
216    long getNumAtOrBelow(long val) {
217      return Arrays.stream(counts).mapToLong(c -> c.sum()).limit(getIndex(val) + 1).sum();
218    }
219
220    public long getMin() {
221      long min = this.min.get();
222      return min == Long.MAX_VALUE ? 0 : min; // in case it is not initialized
223    }
224
225    public long getMean() {
226      long count = this.count.sum();
227      long total = this.total.sum();
228      if (count == 0) {
229        return 0;
230      }
231      return total / count;
232    }
233  }
234
235  // The bins counting values. It is replaced with a new one in calling of reset().
236  private volatile Bins bins;
237
238  /**
239   * Constructor.
240   */
241  public FastLongHistogram() {
242    this(DEFAULT_NBINS);
243  }
244
245  /**
246   * Constructor.
247   * @param numOfBins the number of bins for the histogram. A larger value results in more precise
248   *          results but with lower efficiency, and vice versus.
249   */
250  public FastLongHistogram(int numOfBins) {
251    this.bins = new Bins(numOfBins);
252  }
253
254  /**
255   * Constructor setting the bins assuming a uniform distribution within a range.
256   * @param numOfBins the number of bins for the histogram. A larger value results in more precise
257   *          results but with lower efficiency, and vice versus.
258   * @param min lower bound of the region, inclusive.
259   * @param max higher bound of the region, inclusive.
260   */
261  public FastLongHistogram(int numOfBins, long min, long max) {
262    this(numOfBins);
263    Bins bins = new Bins(numOfBins);
264    bins.add(min, 1);
265    bins.add(max, 1);
266    this.bins = new Bins(bins, numOfBins, 0.01, 0.999);
267  }
268
269  private FastLongHistogram(Bins bins) {
270    this.bins = bins;
271  }
272
273  /**
274   * Adds a value to the histogram.
275   */
276  public void add(long value, long count) {
277    this.bins.add(value, count);
278  }
279
280  /**
281   * Computes the quantiles give the ratios.
282   */
283  public long[] getQuantiles(double[] quantiles) {
284    return this.bins.getQuantiles(quantiles);
285  }
286
287  public long[] getQuantiles() {
288    return this.bins.getQuantiles(DEFAULT_QUANTILES);
289  }
290
291  public long getMin() {
292    return this.bins.getMin();
293  }
294
295  public long getMax() {
296    return this.bins.max.get();
297  }
298
299  public long getCount() {
300    return this.bins.count.sum();
301  }
302
303  public long getMean() {
304    return this.bins.getMean();
305  }
306
307  public long getNumAtOrBelow(long value) {
308    return this.bins.getNumAtOrBelow(value);
309  }
310
311  /**
312   * Resets the histogram for new counting.
313   */
314  public Snapshot snapshotAndReset() {
315    final Bins oldBins = this.bins;
316    this.bins = new Bins(this.bins, this.bins.counts.length - 3, 0.01, 0.99);
317    final long[] percentiles = oldBins.getQuantiles(DEFAULT_QUANTILES);
318    final long count = oldBins.count.sum();
319
320    return new Snapshot() {
321      @Override
322      public long[] getQuantiles(double[] quantiles) {
323        return oldBins.getQuantiles(quantiles);
324      }
325
326      @Override
327      public long[] getQuantiles() {
328        return percentiles;
329      }
330
331      @Override
332      public long getCount() {
333        return count;
334      }
335
336      @Override
337      public long getCountAtOrBelow(long val) {
338        return oldBins.getNumAtOrBelow(val);
339      }
340
341      @Override
342      public long get25thPercentile() {
343        return percentiles[0];
344      }
345
346      @Override
347      public long get75thPercentile() {
348        return percentiles[2];
349      }
350
351      @Override
352      public long get90thPercentile() {
353        return percentiles[3];
354      }
355
356      @Override
357      public long get95thPercentile() {
358        return percentiles[4];
359      }
360
361      @Override
362      public long get98thPercentile() {
363        return percentiles[5];
364      }
365
366      @Override
367      public long get99thPercentile() {
368        return percentiles[6];
369      }
370
371      @Override
372      public long get999thPercentile() {
373        return percentiles[7];
374      }
375
376      @Override
377      public long getMedian() {
378        return percentiles[1];
379      }
380
381      @Override
382      public long getMax() {
383        return oldBins.max.get();
384      }
385
386      @Override
387      public long getMean() {
388        return oldBins.getMean();
389      }
390
391      @Override
392      public long getMin() {
393        return oldBins.getMin();
394      }
395    };
396  }
397}