1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.metrics2.util;
20
21 import java.io.IOException;
22 import java.util.Arrays;
23 import java.util.HashMap;
24 import java.util.LinkedList;
25 import java.util.ListIterator;
26 import java.util.Map;
27
28 import org.apache.hadoop.hbase.classification.InterfaceAudience;
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46 @InterfaceAudience.Private
47 public class MetricSampleQuantiles {
48
49
50
51
52 private long count = 0;
53
54
55
56
57 private LinkedList<SampleItem> samples;
58
59
60
61
62
63
64 private long[] buffer = new long[500];
65 private int bufferCount = 0;
66
67
68
69
70 private final MetricQuantile quantiles[];
71
72 public MetricSampleQuantiles(MetricQuantile[] quantiles) {
73 this.quantiles = Arrays.copyOf(quantiles, quantiles.length);
74 this.samples = new LinkedList<SampleItem>();
75 }
76
77
78
79
80
81
82
83
84
85
86
87 private double allowableError(int rank) {
88 int size = samples.size();
89 double minError = size + 1;
90 for (MetricQuantile q : quantiles) {
91 double error;
92 if (rank <= q.quantile * size) {
93 error = (2.0 * q.error * (size - rank)) / (1.0 - q.quantile);
94 } else {
95 error = (2.0 * q.error * rank) / q.quantile;
96 }
97 if (error < minError) {
98 minError = error;
99 }
100 }
101
102 return minError;
103 }
104
105
106
107
108
109
110 synchronized public void insert(long v) {
111 buffer[bufferCount] = v;
112 bufferCount++;
113
114 count++;
115
116 if (bufferCount == buffer.length) {
117 insertBatch();
118 compress();
119 }
120 }
121
122
123
124
125
126 private void insertBatch() {
127 if (bufferCount == 0) {
128 return;
129 }
130
131 Arrays.sort(buffer, 0, bufferCount);
132
133
134 int start = 0;
135 if (samples.size() == 0) {
136 SampleItem newItem = new SampleItem(buffer[0], 1, 0);
137 samples.add(newItem);
138 start++;
139 }
140
141 ListIterator<SampleItem> it = samples.listIterator();
142 SampleItem item = it.next();
143 for (int i = start; i < bufferCount; i++) {
144 long v = buffer[i];
145 while (it.nextIndex() < samples.size() && item.value < v) {
146 item = it.next();
147 }
148
149 if (item.value > v) {
150 it.previous();
151 }
152
153
154 int delta;
155 if (it.previousIndex() == 0 || it.nextIndex() == samples.size()) {
156 delta = 0;
157 } else {
158 delta = ((int) Math.floor(allowableError(it.nextIndex()))) - 1;
159 }
160 SampleItem newItem = new SampleItem(v, 1, delta);
161 it.add(newItem);
162 item = newItem;
163 }
164
165 bufferCount = 0;
166 }
167
168
169
170
171
172
173 private void compress() {
174 if (samples.size() < 2) {
175 return;
176 }
177
178 ListIterator<SampleItem> it = samples.listIterator();
179 SampleItem prev = null;
180 SampleItem next = it.next();
181
182 while (it.hasNext()) {
183 prev = next;
184 next = it.next();
185 if (prev.g + next.g + next.delta <= allowableError(it.previousIndex())) {
186 next.g += prev.g;
187
188 it.previous();
189 it.previous();
190 it.remove();
191
192 it.next();
193 }
194 }
195 }
196
197
198
199
200
201
202
203 private long query(double quantile) throws IOException {
204 if (samples.size() == 0) {
205 throw new IOException("No samples present");
206 }
207
208 int rankMin = 0;
209 int desired = (int) (quantile * count);
210
211 for (int i = 1; i < samples.size(); i++) {
212 SampleItem prev = samples.get(i - 1);
213 SampleItem cur = samples.get(i);
214
215 rankMin += prev.g;
216
217 if (rankMin + cur.g + cur.delta > desired + (allowableError(i) / 2)) {
218 return prev.value;
219 }
220 }
221
222
223 return samples.get(samples.size() - 1).value;
224 }
225
226
227
228
229
230
231
232
233 synchronized public Map<MetricQuantile, Long> snapshot() throws IOException {
234
235 insertBatch();
236 Map<MetricQuantile, Long> values = new HashMap<MetricQuantile, Long>(quantiles.length);
237 for (int i = 0; i < quantiles.length; i++) {
238 values.put(quantiles[i], query(quantiles[i].quantile));
239 }
240
241 return values;
242 }
243
244
245
246
247
248
249 synchronized public long getCount() {
250 return count;
251 }
252
253
254
255
256
257
258 synchronized public int getSampleCount() {
259 return samples.size();
260 }
261
262
263
264
265 synchronized public void clear() {
266 count = 0;
267 bufferCount = 0;
268 samples.clear();
269 }
270
271
272
273
274
275 private static class SampleItem {
276
277
278
279
280 public final long value;
281
282
283
284
285
286
287
288 public int g;
289
290
291
292
293
294 public final int delta;
295
296 public SampleItem(long value, int lowerDelta, int delta) {
297 this.value = value;
298 this.g = lowerDelta;
299 this.delta = delta;
300 }
301
302 @Override
303 public String toString() {
304 return String.format("%d, %d, %d", value, g, delta);
305 }
306 }
307 }