? byFields.aggregateFieldsById(inputs, combineFn, fieldAggregation.outputField) : combined.aggregateFieldsById(inputs, combineFn, fieldAggregation.outputField); } else { ? byFields.aggregateField(inputs.get(0), combineFn, fieldAggregation.outputField) : combined.aggregateField(inputs.get(0), combineFn, fieldAggregation.outputField); combiner = byFields.aggregate(AggregationCombineFnAdapter.createConstantCombineFn());
.apply( Group.<OuterAggregate>byFieldAccessDescriptor(field2Selector) .aggregateFields(field1Selector, Sum.ofLongs(), "field1_sum") .aggregateFields(field3Selector, Sum.ofIntegers(), "field3_sum") .aggregateFields(field1Selector, Top.largestLongsFn(1), "field1_top"));
.apply( Group.<AggregatePojos>byFieldNames("field2") .aggregateField("field1", Sum.ofLongs(), "field1_sum") .aggregateField("field3", Sum.ofIntegers(), "field3_sum") .aggregateField("field1", Top.largestLongsFn(1), "field1_top"));
@Test @Category(NeedsRunner.class) public void testPerKeyAggregation() { Collection<POJO> elements = ImmutableList.of( new POJO("key1", 1, "value1"), new POJO("key1", 1, "value2"), new POJO("key2", 2, "value3"), new POJO("key2", 2, "value4"), new POJO("key2", 2, "value4")); PCollection<KV<Row, Long>> count = pipeline .apply(Create.of(elements)) .apply(Group.<POJO>byFieldNames("field1").aggregate(Count.combineFn())); Schema keySchema = Schema.builder().addStringField("field1").build(); Collection<KV<Row, Long>> expectedCounts = ImmutableList.of( KV.of(Row.withSchema(keySchema).addValue("key1").build(), 2L), KV.of(Row.withSchema(keySchema).addValue("key2").build(), 3L)); PAssert.that(count).containsInAnyOrder(expectedCounts); pipeline.run(); }
/** * Build up an aggregation function over the input elements. * * <p>This method specifies an aggregation over multiple fields of the input. The union of all * calls to aggregateField and aggregateFields will determine the output schema. * * <p>Field types in the output schema will be inferred from the provided combine function. * Sometimes the field type cannot be inferred due to Java's type erasure. In that case, use the * overload that allows setting the output field type explicitly. */ public <CombineInputT, AccumT, CombineOutputT> CombineFieldsByFields<InputT> aggregateFields( List<String> inputFieldNames, CombineFn<CombineInputT, AccumT, CombineOutputT> fn, String outputFieldName) { return aggregateFields( FieldAccessDescriptor.withFieldNames(inputFieldNames), fn, outputFieldName); }
/** Same as {@link #byFieldNames(String...)}. */ public static <T> ByFields<T> byFieldNames(Iterable<String> fieldNames) { return new ByFields<>(FieldAccessDescriptor.withFieldNames(fieldNames)); }
/** * Returns a transform that groups all elements in the input {@link PCollection} keyed by the * fields specified. The output of this transform will be a {@link KV} keyed by a {@link Row} * containing the specified extracted fields. The returned transform contains further builder * methods to control how the grouping is done. */ public static <T> ByFields<T> byFieldAccessDescriptor(FieldAccessDescriptor fieldAccess) { return new ByFields<>(fieldAccess); }
/** * Returns a transform that groups all elements in the input {@link PCollection} keyed by the list * of fields specified. The output of this transform will be a {@link KV} keyed by a {@link Row} * containing the specified extracted fields. The returned transform contains further builder * methods to control how the grouping is done. */ public static <T> ByFields<T> byFieldIds(Integer... fieldIds) { return new ByFields<>(FieldAccessDescriptor.withFieldIds(fieldIds)); }
/** Same as {@link #byFieldIds(Integer...)}. */ public static <T> ByFields<T> byFieldIds(Iterable<Integer> fieldIds) { return new ByFields<>(FieldAccessDescriptor.withFieldIds(fieldIds)); }
public <CombineInputT, AccumT, CombineOutputT> CombineFieldsByFields<InputT> aggregateFieldsById( List<Integer> inputFieldIds, CombineFn<CombineInputT, AccumT, CombineOutputT> fn, String outputFieldName) { return aggregateFields( FieldAccessDescriptor.withFieldIds(inputFieldIds), fn, outputFieldName); }
public <CombineInputT, AccumT, CombineOutputT> CombineFieldsByFields<InputT> aggregateFieldsById( List<Integer> inputFieldIds, CombineFn<CombineInputT, AccumT, CombineOutputT> fn, Field outputField) { return aggregateFields(FieldAccessDescriptor.withFieldIds(inputFieldIds), fn, outputField); }
/** * Build up an aggregation function over the input elements. * * <p>This method specifies an aggregation over multiple fields of the input. The union of all * calls to aggregateField and aggregateFields will determine the output schema. */ public <CombineInputT, AccumT, CombineOutputT> CombineFieldsByFields<InputT> aggregateFields( List<String> inputFieldNames, CombineFn<CombineInputT, AccumT, CombineOutputT> fn, Field outputField) { return aggregateFields( FieldAccessDescriptor.withFieldNames(inputFieldNames), fn, outputField); }
/** * Returns a transform that groups all elements in the input {@link PCollection} keyed by the list * of fields specified. The output of this transform will be a {@link KV} keyed by a {@link Row} * containing the specified extracted fields. The returned transform contains further builder * methods to control how the grouping is done. */ public static <T> ByFields<T> byFieldNames(String... fieldNames) { return new ByFields<>(FieldAccessDescriptor.withFieldNames(fieldNames)); }