1 | |
|
2 | |
|
3 | |
|
4 | |
|
5 | |
|
6 | |
|
7 | |
|
8 | |
|
9 | |
|
10 | |
|
11 | |
|
12 | |
|
13 | |
|
14 | |
|
15 | |
package datafu.pig.stats; |
16 | |
|
17 | |
import java.io.IOException; |
18 | |
import java.util.ArrayList; |
19 | |
import java.util.Collections; |
20 | |
import java.util.HashMap; |
21 | |
import java.util.List; |
22 | |
|
23 | |
import org.apache.pig.Accumulator; |
24 | |
import org.apache.pig.data.DataBag; |
25 | |
import org.apache.pig.data.DataType; |
26 | |
import org.apache.pig.data.Tuple; |
27 | |
import org.apache.pig.data.TupleFactory; |
28 | |
import org.apache.pig.impl.logicalLayer.schema.Schema; |
29 | |
|
30 | |
import com.google.common.collect.Lists; |
31 | |
|
32 | |
import datafu.pig.util.SimpleEvalFunc; |
33 | |
|
34 | |
|
35 | |
|
36 | |
|
37 | |
|
38 | |
|
39 | |
|
40 | |
|
41 | |
|
42 | |
|
43 | |
|
44 | |
|
45 | |
|
46 | |
|
47 | |
|
48 | |
|
49 | |
|
50 | |
|
51 | |
|
52 | |
|
53 | |
|
54 | |
|
55 | |
|
56 | |
|
57 | |
|
58 | |
|
59 | |
|
60 | |
|
61 | |
|
62 | |
|
63 | |
|
64 | |
|
65 | |
|
66 | |
|
67 | |
|
68 | |
|
69 | |
|
70 | |
|
71 | |
|
72 | |
|
73 | |
|
74 | |
|
75 | |
|
76 | |
|
77 | |
|
78 | |
|
79 | |
|
80 | |
|
81 | |
|
82 | |
|
83 | |
|
84 | |
|
85 | |
|
86 | |
|
87 | |
|
88 | |
|
89 | |
|
90 | |
|
91 | |
|
92 | |
|
93 | |
|
94 | |
|
95 | |
|
96 | |
|
97 | |
|
98 | |
|
99 | |
|
100 | |
|
101 | |
|
102 | |
|
103 | |
|
104 | |
|
105 | |
|
106 | |
|
107 | |
|
108 | |
|
109 | |
|
110 | |
|
111 | |
|
112 | |
|
113 | |
|
114 | |
|
115 | |
|
116 | |
|
117 | |
|
118 | |
|
119 | |
|
120 | 5 | public class StreamingQuantile extends SimpleEvalFunc<Tuple> implements Accumulator<Tuple> { |
121 | |
|
122 | |
private final int numQuantiles; |
123 | |
private final QuantileEstimator estimator; |
124 | |
private List<Double> quantiles; |
125 | |
|
126 | |
public StreamingQuantile(String... k) |
127 | 2395 | { |
128 | 2395 | this.quantiles = QuantileUtil.getQuantilesFromParams(k); |
129 | 2395 | this.numQuantiles = getNumQuantiles(this.quantiles); |
130 | 2395 | this.estimator = new QuantileEstimator(this.numQuantiles); |
131 | 2395 | } |
132 | |
|
133 | |
private static int getNumQuantiles(List<Double> quantiles) |
134 | |
{ |
135 | 2395 | quantiles = new ArrayList<Double>(quantiles); |
136 | 2395 | Collections.sort(quantiles); |
137 | 2395 | int start = 0; |
138 | 2395 | int end = quantiles.size()-1; |
139 | 3265 | while (quantiles.get(start) == 0.0) start++; |
140 | 3788 | while (quantiles.get(end) == 1.0) end--; |
141 | 2395 | double gcd = 1.0; |
142 | 9663 | for (int i=end; i>=start; i--) |
143 | |
{ |
144 | 7268 | gcd = gcd(gcd,quantiles.get(i)); |
145 | |
} |
146 | 2395 | int numQuantiles = (int)(1/gcd) + 1; |
147 | 2395 | return numQuantiles; |
148 | |
} |
149 | |
|
150 | |
private static double gcd(double a, double b) |
151 | |
{ |
152 | 7268 | if (round(a) == 0.0) |
153 | |
{ |
154 | 0 | throw new IllegalArgumentException("Quantiles are smaller than the allowed precision"); |
155 | |
} |
156 | 7268 | if (round(b) == 0.0) |
157 | |
{ |
158 | 0 | throw new IllegalArgumentException("Quantiles are smaller than the allowed precision"); |
159 | |
} |
160 | 26330 | while (round(b) != 0.0) |
161 | |
{ |
162 | 19062 | double t = b; |
163 | 19062 | b = a % b; |
164 | 19062 | a = t; |
165 | 19062 | } |
166 | 7268 | return round(a); |
167 | |
} |
168 | |
|
169 | |
private static double round(double d) |
170 | |
{ |
171 | 58270 | return Math.round(d*100000.0)/100000.0; |
172 | |
} |
173 | |
|
174 | |
@Override |
175 | |
public void accumulate(Tuple b) throws IOException |
176 | |
{ |
177 | 10 | DataBag bag = (DataBag) b.get(0); |
178 | 10 | if (bag == null || bag.size() == 0) |
179 | 0 | return; |
180 | |
|
181 | 10 | for (Tuple t : bag) { |
182 | 101032 | Object o = t.get(0); |
183 | 101032 | if (!(o instanceof Number)) { |
184 | 0 | throw new IllegalStateException("bag must have numerical values (and be non-null)"); |
185 | |
} |
186 | 101032 | estimator.add(((Number) o).doubleValue()); |
187 | 101032 | } |
188 | 10 | } |
189 | |
|
190 | |
@Override |
191 | |
public void cleanup() |
192 | |
{ |
193 | 10 | estimator.clear(); |
194 | 10 | } |
195 | |
|
196 | |
@Override |
197 | |
public Tuple getValue() |
198 | |
{ |
199 | 5 | Tuple t = TupleFactory.getInstance().newTuple(this.quantiles.size()); |
200 | |
try { |
201 | 5 | HashMap<Double,Double> quantileValues = new HashMap<Double,Double>(this.quantiles.size()); |
202 | 5 | double quantileKey = 0.0; |
203 | 5 | for (double quantileValue : estimator.getQuantiles()) { |
204 | 10115 | quantileValues.put(round(quantileKey), quantileValue); |
205 | 10115 | quantileKey += 1.0/(this.numQuantiles-1); |
206 | |
} |
207 | 5 | int j = 0; |
208 | 5 | for (double d : this.quantiles) |
209 | |
{ |
210 | 21 | Double quantileValue = quantileValues.get(round(d)); |
211 | 21 | t.set(j, quantileValue); |
212 | 21 | j++; |
213 | 21 | } |
214 | 0 | } catch (IOException e) { |
215 | 0 | return null; |
216 | 5 | } |
217 | 5 | return t; |
218 | |
} |
219 | |
|
220 | |
public Tuple call(DataBag b) throws IOException |
221 | |
{ |
222 | 0 | accumulate(TupleFactory.getInstance().newTuple(b)); |
223 | 0 | Tuple ret = getValue(); |
224 | 0 | cleanup(); |
225 | 0 | return ret; |
226 | |
} |
227 | |
|
228 | |
@Override |
229 | |
public Schema outputSchema(Schema input) |
230 | |
{ |
231 | 1240 | Schema tupleSchema = new Schema(); |
232 | 2289760 | for (int i = 0; i < numQuantiles; i++) { |
233 | 2288520 | tupleSchema.add(new Schema.FieldSchema("quantile_" + i, DataType.DOUBLE)); |
234 | |
} |
235 | 1240 | return tupleSchema; |
236 | |
} |
237 | |
|
238 | |
static class QuantileEstimator |
239 | |
{ |
240 | |
private static final long MAX_TOT_ELEMS = 1024L * 1024L * 1024L * 1024L; |
241 | |
|
242 | 2395 | private final List<List<Double>> buffer = Lists.newArrayList(); |
243 | |
private final int numQuantiles; |
244 | |
private final int maxElementsPerBuffer; |
245 | |
private int totalElements; |
246 | |
private double min; |
247 | |
private double max; |
248 | |
|
249 | |
public QuantileEstimator(int numQuantiles) |
250 | 2395 | { |
251 | 2395 | this.numQuantiles = numQuantiles; |
252 | 2395 | this.maxElementsPerBuffer = computeMaxElementsPerBuffer(); |
253 | 2395 | } |
254 | |
|
255 | |
private int computeMaxElementsPerBuffer() |
256 | |
{ |
257 | 2395 | double epsilon = 1.0 / (numQuantiles - 1.0); |
258 | 2395 | int b = 2; |
259 | 75731 | while ((b - 2) * (0x1L << (b - 2)) + 0.5 <= epsilon * MAX_TOT_ELEMS) { |
260 | 73336 | ++b; |
261 | |
} |
262 | 2395 | return (int) (MAX_TOT_ELEMS / (0x1L << (b - 1))); |
263 | |
} |
264 | |
|
265 | |
private void ensureBuffer(int level) |
266 | |
{ |
267 | 202074 | while (buffer.size() < level + 1) { |
268 | 10 | buffer.add(null); |
269 | |
} |
270 | 202064 | if (buffer.get(level) == null) { |
271 | 10 | buffer.set(level, Lists.<Double>newArrayList()); |
272 | |
} |
273 | 202064 | } |
274 | |
|
275 | |
private void collapse(List<Double> a, List<Double> b, List<Double> out) |
276 | |
{ |
277 | 0 | int indexA = 0, indexB = 0, count = 0; |
278 | 0 | Double smaller = null; |
279 | 0 | while (indexA < maxElementsPerBuffer || indexB < maxElementsPerBuffer) { |
280 | 0 | if (indexA >= maxElementsPerBuffer || |
281 | |
(indexB < maxElementsPerBuffer && a.get(indexA) >= b.get(indexB))) { |
282 | 0 | smaller = b.get(indexB++); |
283 | |
} else { |
284 | 0 | smaller = a.get(indexA++); |
285 | |
} |
286 | |
|
287 | 0 | if (count++ % 2 == 0) { |
288 | 0 | out.add(smaller); |
289 | |
} |
290 | |
} |
291 | 0 | a.clear(); |
292 | 0 | b.clear(); |
293 | 0 | } |
294 | |
|
295 | |
private void recursiveCollapse(List<Double> buf, int level) |
296 | |
{ |
297 | 0 | ensureBuffer(level + 1); |
298 | |
|
299 | |
List<Double> merged; |
300 | 0 | if (buffer.get(level + 1).isEmpty()) { |
301 | 0 | merged = buffer.get(level + 1); |
302 | |
} else { |
303 | 0 | merged = Lists.newArrayListWithCapacity(maxElementsPerBuffer); |
304 | |
} |
305 | |
|
306 | 0 | collapse(buffer.get(level), buf, merged); |
307 | 0 | if (buffer.get(level + 1) != merged) { |
308 | 0 | recursiveCollapse(merged, level + 1); |
309 | |
} |
310 | 0 | } |
311 | |
|
312 | |
public void add(double elem) |
313 | |
{ |
314 | 101032 | if (totalElements == 0 || elem < min) { |
315 | 101004 | min = elem; |
316 | |
} |
317 | 101032 | if (totalElements == 0 || max < elem) { |
318 | 23 | max = elem; |
319 | |
} |
320 | |
|
321 | 101032 | if (totalElements > 0 && totalElements % (2 * maxElementsPerBuffer) == 0) { |
322 | 0 | Collections.sort(buffer.get(0)); |
323 | 0 | Collections.sort(buffer.get(1)); |
324 | 0 | recursiveCollapse(buffer.get(0), 1); |
325 | |
} |
326 | |
|
327 | 101032 | ensureBuffer(0); |
328 | 101032 | ensureBuffer(1); |
329 | 101032 | int index = buffer.get(0).size() < maxElementsPerBuffer ? 0 : 1; |
330 | 101032 | buffer.get(index).add(elem); |
331 | 101032 | totalElements++; |
332 | 101032 | } |
333 | |
|
334 | |
public void clear() |
335 | |
{ |
336 | 10 | buffer.clear(); |
337 | 10 | totalElements = 0; |
338 | 10 | } |
339 | |
|
340 | |
public List<Double> getQuantiles() |
341 | |
{ |
342 | 5 | List<Double> quantiles = Lists.newArrayList(); |
343 | 5 | quantiles.add(min); |
344 | |
|
345 | 5 | if (buffer.get(0) != null) { |
346 | 5 | Collections.sort(buffer.get(0)); |
347 | |
} |
348 | 5 | if (buffer.get(1) != null) { |
349 | 5 | Collections.sort(buffer.get(1)); |
350 | |
} |
351 | |
|
352 | 5 | int[] index = new int[buffer.size()]; |
353 | 5 | long S = 0; |
354 | 10110 | for (int i = 1; i <= numQuantiles - 2; i++) { |
355 | 10105 | long targetS = (long) Math.ceil(i * (totalElements / (numQuantiles - 1.0))); |
356 | |
|
357 | |
while (true) { |
358 | 111103 | double smallest = max; |
359 | 111103 | int minBufferId = -1; |
360 | 333309 | for (int j = 0; j < buffer.size(); j++) { |
361 | 222206 | if (buffer.get(j) != null && index[j] < buffer.get(j).size()) { |
362 | 149014 | if (!(smallest < buffer.get(j).get(index[j]))) { |
363 | 149014 | smallest = buffer.get(j).get(index[j]); |
364 | 149014 | minBufferId = j; |
365 | |
} |
366 | |
} |
367 | |
} |
368 | |
|
369 | 111103 | long incrementS = minBufferId <= 1 ? 1L : (0x1L << (minBufferId - 1)); |
370 | 111103 | if (S + incrementS >= targetS) { |
371 | 10105 | quantiles.add(smallest); |
372 | 10105 | break; |
373 | |
} else { |
374 | 100998 | index[minBufferId]++; |
375 | 100998 | S += incrementS; |
376 | |
} |
377 | 100998 | } |
378 | |
} |
379 | |
|
380 | 5 | quantiles.add(max); |
381 | 5 | return quantiles; |
382 | |
} |
383 | |
} |
384 | |
} |