.apply( Group.<OuterAggregate>globally() .aggregateFields(field1Selector, Sum.ofLongs(), "field1_sum") .aggregateFields(field3Selector, Sum.ofIntegers(), "field3_sum") .aggregateFields(field1Selector, Top.largestLongsFn(1), "field1_top"));
@Test @Category(NeedsRunner.class) public void testAggregateByMultipleFields() { Collection<AggregatePojos> elements = ImmutableList.of( new AggregatePojos(1, 1, 2), new AggregatePojos(2, 1, 3), new AggregatePojos(3, 2, 4), new AggregatePojos(4, 2, 5)); List<String> fieldNames = Lists.newArrayList("field1", "field2"); PCollection<Row> aggregate = pipeline .apply(Create.of(elements)) .apply( Group.<AggregatePojos>globally() .aggregateFields(fieldNames, new MultipleFieldCombineFn(), "field1+field2")); Schema outputSchema = Schema.builder().addInt64Field("field1+field2").build(); Row expectedRow = Row.withSchema(outputSchema).addValues(16L).build(); PAssert.that(aggregate).containsInAnyOrder(expectedRow); 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> CombineFieldsGlobally<InputT> aggregateFields( List<String> inputFieldNames, CombineFn<CombineInputT, AccumT, CombineOutputT> fn, String outputFieldName) { return aggregateFields( FieldAccessDescriptor.withFieldNames(inputFieldNames), fn, outputFieldName); }
/** * 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> CombineFieldsGlobally<InputT> aggregateFields( List<String> inputFieldNames, CombineFn<CombineInputT, AccumT, CombineOutputT> fn, Field outputField) { return aggregateFields( FieldAccessDescriptor.withFieldNames(inputFieldNames), fn, outputField); }
/** The same as {@link #aggregateFields} but with field ids. */ public <CombineInputT, AccumT, CombineOutputT> CombineFieldsGlobally<InputT> aggregateFieldsById( List<Integer> inputFieldIds, CombineFn<CombineInputT, AccumT, CombineOutputT> fn, String outputFieldName) { return aggregateFields( FieldAccessDescriptor.withFieldIds(inputFieldIds), fn, outputFieldName); }
public <CombineInputT, AccumT, CombineOutputT> CombineFieldsGlobally<InputT> aggregateFieldsById( List<Integer> inputFieldIds, CombineFn<CombineInputT, AccumT, CombineOutputT> fn, Field outputField) { return aggregateFields(FieldAccessDescriptor.withFieldIds(inputFieldIds), fn, outputField); }