-
Notifications
You must be signed in to change notification settings - Fork 4.2k
/
BeamBuiltinAggregations.java
694 lines (612 loc) · 20.9 KB
/
BeamBuiltinAggregations.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
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
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
/*
* 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.beam.sdk.extensions.sql.impl.transform;
import java.io.Serializable;
import java.math.BigDecimal;
import java.math.MathContext;
import java.math.RoundingMode;
import java.util.Map;
import java.util.function.Function;
import org.apache.beam.sdk.coders.BigDecimalCoder;
import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
import org.apache.beam.sdk.coders.CannotProvideCoderException;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.CoderRegistry;
import org.apache.beam.sdk.coders.KvCoder;
import org.apache.beam.sdk.coders.NullableCoder;
import org.apache.beam.sdk.extensions.sql.impl.transform.agg.CountIf;
import org.apache.beam.sdk.extensions.sql.impl.transform.agg.CovarianceFn;
import org.apache.beam.sdk.extensions.sql.impl.transform.agg.VarianceFn;
import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
import org.apache.beam.sdk.schemas.Schema;
import org.apache.beam.sdk.schemas.Schema.FieldType;
import org.apache.beam.sdk.schemas.Schema.TypeName;
import org.apache.beam.sdk.transforms.Combine;
import org.apache.beam.sdk.transforms.Combine.CombineFn;
import org.apache.beam.sdk.transforms.Count;
import org.apache.beam.sdk.transforms.Max;
import org.apache.beam.sdk.transforms.Min;
import org.apache.beam.sdk.transforms.Sample;
import org.apache.beam.sdk.transforms.Sum;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Predicates;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables;
import org.checkerframework.checker.nullness.qual.Nullable;
/** Built-in aggregations functions for COUNT/MAX/MIN/SUM/AVG/VAR_POP/VAR_SAMP. */
@SuppressWarnings({
"rawtypes", // TODO(https://github.com/apache/beam/issues/20447)
"nullness" // TODO(https://github.com/apache/beam/issues/20497)
})
public class BeamBuiltinAggregations {
public static final Map<String, Function<Schema.FieldType, CombineFn<?, ?, ?>>>
BUILTIN_AGGREGATOR_FACTORIES =
ImmutableMap.<String, Function<Schema.FieldType, CombineFn<?, ?, ?>>>builder()
.put("ANY_VALUE", typeName -> Sample.anyValueCombineFn())
// Drop null elements for these aggregations.
.put("COUNT", typeName -> new DropNullFnWithDefault(Count.combineFn()))
.put("MAX", typeName -> new DropNullFn(BeamBuiltinAggregations.createMax(typeName)))
.put("MIN", typeName -> new DropNullFn(BeamBuiltinAggregations.createMin(typeName)))
.put("SUM", typeName -> new DropNullFn(BeamBuiltinAggregations.createSum(typeName)))
.put(
"$SUM0",
typeName ->
new DropNullFnWithDefault(BeamBuiltinAggregations.createSum0(typeName)))
.put("AVG", typeName -> new DropNullFn(BeamBuiltinAggregations.createAvg(typeName)))
.put(
"BIT_OR",
typeName -> new DropNullFn(BeamBuiltinAggregations.createBitOr(typeName)))
.put(
"BIT_XOR",
typeName -> new DropNullFn(BeamBuiltinAggregations.createBitXOr(typeName)))
// JIRA link:https://issues.apache.org/jira/browse/BEAM-10379
.put(
"BIT_AND",
typeName -> new DropNullFn(BeamBuiltinAggregations.createBitAnd(typeName)))
.put("VAR_POP", t -> VarianceFn.newPopulation(t.getTypeName()))
.put("VAR_SAMP", t -> VarianceFn.newSample(t.getTypeName()))
.put("COVAR_POP", t -> CovarianceFn.newPopulation(t.getTypeName()))
.put("COVAR_SAMP", t -> CovarianceFn.newSample(t.getTypeName()))
.put("COUNTIF", typeName -> CountIf.combineFn())
.build();
private static MathContext mc = new MathContext(10, RoundingMode.HALF_UP);
public static CombineFn<?, ?, ?> create(String functionName, Schema.FieldType fieldType) {
Function<Schema.FieldType, CombineFn<?, ?, ?>> aggregatorFactory =
BUILTIN_AGGREGATOR_FACTORIES.get(functionName);
if (aggregatorFactory != null) {
return aggregatorFactory.apply(fieldType);
}
throw new UnsupportedOperationException(
String.format("Aggregator [%s] is not supported", functionName));
}
/** {@link CombineFn} for MAX based on {@link Max} and {@link Combine.BinaryCombineFn}. */
static CombineFn createMax(FieldType fieldType) {
if (CalciteUtils.isDateTimeType(fieldType)) {
return new CustMax<>();
}
switch (fieldType.getTypeName()) {
case BOOLEAN:
case INT16:
case BYTE:
case FLOAT:
case DATETIME:
case DECIMAL:
case STRING:
return new CustMax<>();
case INT32:
return Max.ofIntegers();
case INT64:
return Max.ofLongs();
case DOUBLE:
return Max.ofDoubles();
default:
throw new UnsupportedOperationException(
String.format("[%s] is not supported in MAX", fieldType));
}
}
/** {@link CombineFn} for MIN based on {@link Min} and {@link Combine.BinaryCombineFn}. */
static CombineFn createMin(Schema.FieldType fieldType) {
if (CalciteUtils.isDateTimeType(fieldType)) {
return new CustMin();
}
switch (fieldType.getTypeName()) {
case BOOLEAN:
case BYTE:
case INT16:
case FLOAT:
case DATETIME:
case DECIMAL:
case STRING:
return new CustMin();
case INT32:
return Min.ofIntegers();
case INT64:
return Min.ofLongs();
case DOUBLE:
return Min.ofDoubles();
default:
throw new UnsupportedOperationException(
String.format("[%s] is not supported in MIN", fieldType));
}
}
/** {@link CombineFn} for Sum based on {@link Sum} and {@link Combine.BinaryCombineFn}. */
static CombineFn createSum(Schema.FieldType fieldType) {
switch (fieldType.getTypeName()) {
case INT32:
return Sum.ofIntegers();
case INT16:
return new ShortSum();
case BYTE:
return new ByteSum();
case INT64:
return new LongSum();
case FLOAT:
return new FloatSum();
case DOUBLE:
return Sum.ofDoubles();
case DECIMAL:
return new BigDecimalSum();
default:
throw new UnsupportedOperationException(
String.format("[%s] is not supported in SUM", fieldType));
}
}
/**
* {@link CombineFn} for Sum0 where sum of null returns 0 based on {@link Sum} and {@link
* Combine.BinaryCombineFn}.
*/
static CombineFn createSum0(Schema.FieldType fieldType) {
switch (fieldType.getTypeName()) {
case INT32:
return new IntegerSum0();
case INT16:
return new ShortSum0();
case BYTE:
return new ByteSum0();
case INT64:
return new LongSum0();
case FLOAT:
return new FloatSum0();
case DOUBLE:
return new DoubleSum0();
case DECIMAL:
return new BigDecimalSum0();
default:
throw new UnsupportedOperationException(
String.format("[%s] is not supported in SUM0", fieldType));
}
}
/** {@link CombineFn} for AVG. */
static CombineFn createAvg(Schema.FieldType fieldType) {
switch (fieldType.getTypeName()) {
case INT32:
return new IntegerAvg();
case INT16:
return new ShortAvg();
case BYTE:
return new ByteAvg();
case INT64:
return new LongAvg();
case FLOAT:
return new FloatAvg();
case DOUBLE:
return new DoubleAvg();
case DECIMAL:
return new BigDecimalAvg();
default:
throw new UnsupportedOperationException(
String.format("[%s] is not supported in AVG", fieldType));
}
}
static CombineFn createBitOr(Schema.FieldType fieldType) {
if (fieldType.getTypeName() == TypeName.INT64) {
return new BitOr();
}
throw new UnsupportedOperationException(
String.format("[%s] is not supported in BIT_OR", fieldType));
}
static CombineFn createBitAnd(Schema.FieldType fieldType) {
if (fieldType.getTypeName() == TypeName.INT64) {
return new BitAnd();
}
throw new UnsupportedOperationException(
String.format("[%s] is not supported in BIT_AND", fieldType));
}
public static CombineFn createBitXOr(Schema.FieldType fieldType) {
if (fieldType.getTypeName() == TypeName.INT64) {
return new BitXOr();
}
throw new UnsupportedOperationException(
String.format("[%s] is not supported in BIT_XOR", fieldType));
}
static class CustMax<T extends Comparable<T>> extends Combine.BinaryCombineFn<T> {
@Override
public T apply(T left, T right) {
return (right == null || right.compareTo(left) < 0) ? left : right;
}
}
static class CustMin<T extends Comparable<T>> extends Combine.BinaryCombineFn<T> {
@Override
public T apply(T left, T right) {
return (left == null || left.compareTo(right) < 0) ? left : right;
}
}
static class IntegerSum extends Combine.BinaryCombineFn<Integer> {
@Override
public Integer apply(Integer left, Integer right) {
return (int) (left + right);
}
}
static class ShortSum extends Combine.BinaryCombineFn<Short> {
@Override
public Short apply(Short left, Short right) {
return (short) (left + right);
}
}
static class ByteSum extends Combine.BinaryCombineFn<Byte> {
@Override
public Byte apply(Byte left, Byte right) {
return (byte) (left + right);
}
}
static class FloatSum extends Combine.BinaryCombineFn<Float> {
@Override
public Float apply(Float left, Float right) {
return left + right;
}
}
static class DoubleSum extends Combine.BinaryCombineFn<Double> {
@Override
public Double apply(Double left, Double right) {
return (double) left + right;
}
}
static class LongSum extends Combine.BinaryCombineFn<Long> {
@Override
public Long apply(Long left, Long right) {
return Math.addExact(left, right);
}
}
static class BigDecimalSum extends Combine.BinaryCombineFn<BigDecimal> {
@Override
public BigDecimal apply(BigDecimal left, BigDecimal right) {
return left.add(right);
}
}
static class IntegerSum0 extends IntegerSum {
@Override
public @Nullable Integer identity() {
return 0;
}
}
static class ShortSum0 extends ShortSum {
@Override
public @Nullable Short identity() {
return 0;
}
}
static class ByteSum0 extends ByteSum {
@Override
public @Nullable Byte identity() {
return 0;
}
}
static class FloatSum0 extends FloatSum {
@Override
public @Nullable Float identity() {
return 0F;
}
}
static class DoubleSum0 extends DoubleSum {
@Override
public @Nullable Double identity() {
return 0D;
}
}
static class LongSum0 extends LongSum {
@Override
public @Nullable Long identity() {
return 0L;
}
}
static class BigDecimalSum0 extends BigDecimalSum {
@Override
public @Nullable BigDecimal identity() {
return BigDecimal.ZERO;
}
}
private static class DropNullFn<InputT, AccumT, OutputT>
extends CombineFn<InputT, AccumT, OutputT> {
protected final CombineFn<InputT, AccumT, OutputT> combineFn;
DropNullFn(CombineFn<InputT, AccumT, OutputT> combineFn) {
this.combineFn = combineFn;
}
@Override
public AccumT createAccumulator() {
return null;
}
@Override
public AccumT addInput(AccumT accumulator, InputT input) {
if (input == null) {
return accumulator;
}
if (accumulator == null) {
accumulator = combineFn.createAccumulator();
}
return combineFn.addInput(accumulator, input);
}
@Override
public AccumT mergeAccumulators(Iterable<AccumT> accumulators) {
// filter out nulls
accumulators = Iterables.filter(accumulators, Predicates.notNull());
// handle only nulls
if (!accumulators.iterator().hasNext()) {
return null;
}
return combineFn.mergeAccumulators(accumulators);
}
@Override
public OutputT extractOutput(AccumT accumulator) {
if (accumulator == null) {
return null;
}
return combineFn.extractOutput(accumulator);
}
@Override
public Coder<AccumT> getAccumulatorCoder(CoderRegistry registry, Coder<InputT> inputCoder)
throws CannotProvideCoderException {
Coder<AccumT> coder = combineFn.getAccumulatorCoder(registry, inputCoder);
if (coder instanceof NullableCoder) {
return coder;
}
return NullableCoder.of(coder);
}
}
private static class DropNullFnWithDefault<InputT, AccumT, OutputT>
extends DropNullFn<InputT, AccumT, OutputT> {
DropNullFnWithDefault(CombineFn<InputT, AccumT, OutputT> combineFn) {
super(combineFn);
}
@Override
public AccumT createAccumulator() {
return combineFn.createAccumulator();
}
}
/** {@link CombineFn} for <em>AVG</em> on {@link Number} types. */
abstract static class Avg<T extends Number> extends CombineFn<T, KV<Integer, BigDecimal>, T> {
@Override
public KV<Integer, BigDecimal> createAccumulator() {
return KV.of(0, BigDecimal.ZERO);
}
@Override
public KV<Integer, BigDecimal> addInput(KV<Integer, BigDecimal> accumulator, T input) {
return KV.of(accumulator.getKey() + 1, accumulator.getValue().add(toBigDecimal(input)));
}
@Override
public KV<Integer, BigDecimal> mergeAccumulators(
Iterable<KV<Integer, BigDecimal>> accumulators) {
int size = 0;
BigDecimal acc = BigDecimal.ZERO;
for (KV<Integer, BigDecimal> ele : accumulators) {
size += ele.getKey();
acc = acc.add(ele.getValue());
}
return KV.of(size, acc);
}
@Override
public Coder<KV<Integer, BigDecimal>> getAccumulatorCoder(
CoderRegistry registry, Coder<T> inputCoder) {
return KvCoder.of(BigEndianIntegerCoder.of(), BigDecimalCoder.of());
}
protected BigDecimal prepareOutput(KV<Integer, BigDecimal> accumulator) {
return accumulator.getValue().divide(new BigDecimal(accumulator.getKey()), mc);
}
@Override
public abstract T extractOutput(KV<Integer, BigDecimal> accumulator);
public abstract BigDecimal toBigDecimal(T record);
}
static class IntegerAvg extends Avg<Integer> {
@Override
public @Nullable Integer extractOutput(KV<Integer, BigDecimal> accumulator) {
return accumulator.getKey() == 0 ? null : prepareOutput(accumulator).intValue();
}
@Override
public BigDecimal toBigDecimal(Integer record) {
return new BigDecimal(record);
}
}
static class LongAvg extends Avg<Long> {
@Override
public @Nullable Long extractOutput(KV<Integer, BigDecimal> accumulator) {
return accumulator.getKey() == 0 ? null : prepareOutput(accumulator).longValue();
}
@Override
public BigDecimal toBigDecimal(Long record) {
return new BigDecimal(record);
}
}
static class ShortAvg extends Avg<Short> {
@Override
public @Nullable Short extractOutput(KV<Integer, BigDecimal> accumulator) {
return accumulator.getKey() == 0 ? null : prepareOutput(accumulator).shortValue();
}
@Override
public BigDecimal toBigDecimal(Short record) {
return new BigDecimal(record);
}
}
static class ByteAvg extends Avg<Byte> {
@Override
public @Nullable Byte extractOutput(KV<Integer, BigDecimal> accumulator) {
return accumulator.getKey() == 0 ? null : prepareOutput(accumulator).byteValue();
}
@Override
public BigDecimal toBigDecimal(Byte record) {
return new BigDecimal(record);
}
}
static class FloatAvg extends Avg<Float> {
@Override
public @Nullable Float extractOutput(KV<Integer, BigDecimal> accumulator) {
return accumulator.getKey() == 0 ? null : prepareOutput(accumulator).floatValue();
}
@Override
public BigDecimal toBigDecimal(Float record) {
return new BigDecimal(record);
}
}
static class DoubleAvg extends Avg<Double> {
@Override
public @Nullable Double extractOutput(KV<Integer, BigDecimal> accumulator) {
return accumulator.getKey() == 0 ? null : prepareOutput(accumulator).doubleValue();
}
@Override
public BigDecimal toBigDecimal(Double record) {
return new BigDecimal(record);
}
}
static class BigDecimalAvg extends Avg<BigDecimal> {
@Override
public @Nullable BigDecimal extractOutput(KV<Integer, BigDecimal> accumulator) {
return accumulator.getKey() == 0 ? null : prepareOutput(accumulator);
}
@Override
public BigDecimal toBigDecimal(BigDecimal record) {
return record;
}
}
static class BitOr<T extends Number> extends CombineFn<T, BitOr.Accum, Long> {
static class Accum implements Serializable {
/** True if no inputs have been seen yet. */
boolean isEmpty = true;
/** The bitwise-or of the inputs seen so far. */
long bitOr = 0L;
}
@Override
public BitOr.Accum createAccumulator() {
return new BitOr.Accum();
}
@Override
public BitOr.Accum addInput(BitOr.Accum accum, T input) {
accum.isEmpty = false;
accum.bitOr |= input.longValue();
return accum;
}
@Override
public BitOr.Accum mergeAccumulators(Iterable<BitOr.Accum> accums) {
BitOr.Accum merged = createAccumulator();
for (BitOr.Accum accum : accums) {
if (accum.isEmpty) {
continue;
}
merged.isEmpty = false;
merged.bitOr |= accum.bitOr;
}
return merged;
}
@Override
public Long extractOutput(BitOr.Accum accum) {
if (accum.isEmpty) {
return null;
}
return accum.bitOr;
}
}
/**
* Bitwise AND function implementation.
*
* <p>Note: null values are ignored when mixed with non-null values.
* (https://issues.apache.org/jira/browse/BEAM-10379)
*/
static class BitAnd<T extends Number> extends CombineFn<T, BitAnd.Accum, Long> {
static class Accum implements Serializable {
/** True if no inputs have been seen yet. */
boolean isEmpty = true;
/** The bitwise-and of the inputs seen so far. */
long bitAnd = -1L;
}
@Override
public BitAnd.Accum createAccumulator() {
return new BitAnd.Accum();
}
@Override
public BitAnd.Accum addInput(BitAnd.Accum accum, T input) {
accum.isEmpty = false;
accum.bitAnd &= input.longValue();
return accum;
}
@Override
public BitAnd.Accum mergeAccumulators(Iterable<BitAnd.Accum> accums) {
BitAnd.Accum merged = createAccumulator();
for (BitAnd.Accum accum : accums) {
if (accum.isEmpty) {
continue;
}
merged.isEmpty = false;
merged.bitAnd &= accum.bitAnd;
}
return merged;
}
@Override
public Long extractOutput(BitAnd.Accum accum) {
if (accum.isEmpty) {
return null;
}
return accum.bitAnd;
}
}
public static class BitXOr<T extends Number> extends CombineFn<T, BitXOr.Accum, Long> {
static class Accum implements Serializable {
/** True if no inputs have been seen yet. */
boolean isEmpty = true;
/** The bitwise-and of the inputs seen so far. */
long bitXOr = 0L;
}
@Override
public BitXOr.Accum createAccumulator() {
return new BitXOr.Accum();
}
@Override
public BitXOr.Accum addInput(Accum mutableAccumulator, T input) {
if (input != null) {
mutableAccumulator.isEmpty = false;
mutableAccumulator.bitXOr ^= input.longValue();
}
return mutableAccumulator;
}
@Override
public BitXOr.Accum mergeAccumulators(Iterable<BitXOr.Accum> accumulators) {
BitXOr.Accum merged = createAccumulator();
for (BitXOr.Accum accum : accumulators) {
if (accum.isEmpty) {
continue;
}
merged.isEmpty = false;
merged.bitXOr ^= accum.bitXOr;
}
return merged;
}
@Override
public Long extractOutput(BitXOr.Accum accumulator) {
if (accumulator.isEmpty) {
return null;
}
return accumulator.bitXOr;
}
}
}