FieldAggregation( FieldAccessDescriptor fieldsToAggregate, Field outputField, CombineFn<FieldT, AccumT, OutputT> fn, TupleTag<Object> combineTag, Schema aggregationSchema, @Nullable Schema inputSchema) { if (inputSchema != null) { this.fieldsToAggregate = fieldsToAggregate.resolve(inputSchema); this.inputSubSchema = Select.getOutputSchema(inputSchema, this.fieldsToAggregate); this.unnestedInputSubSchema = Unnest.getUnnestedSchema(inputSubSchema); this.needsUnnesting = !inputSchema.equals(unnestedInputSubSchema); } else { this.fieldsToAggregate = fieldsToAggregate; this.inputSubSchema = null; this.unnestedInputSubSchema = null; this.needsUnnesting = false; } this.outputField = outputField; this.fn = fn; this.combineTag = combineTag; this.aggregationSchema = aggregationSchema; }
Schema currentKeySchema = Select.getOutputSchema(schema, resolved); if (keySchema == null) { keySchema = currentKeySchema;
@Override public PCollection<Row> expand(PCollection<T> input) { Schema inputSchema = input.getSchema(); FieldAccessDescriptor resolved = fieldAccessDescriptor.resolve(inputSchema); Schema outputSchema = getOutputSchema(inputSchema, resolved); PCollection<Row> selected = input .apply( ParDo.of( new DoFn<T, Row>() { // TODO: This should be the same as resolved so that Beam knows which fields // are being accessed. Currently Beam only supports wildcard descriptors. // Once BEAM-4457 is fixed, fix this. @FieldAccess("filterFields") final FieldAccessDescriptor fieldAccessDescriptor = FieldAccessDescriptor.withAllFields(); @ProcessElement public void process( @FieldAccess("filterFields") Row row, OutputReceiver<Row> r) { r.output(selectRow(row, resolved, inputSchema, outputSchema)); } })) .setRowSchema(outputSchema); return selected; }
static Schema getOutputSchema(Schema inputSchema, FieldAccessDescriptor fieldAccessDescriptor) { if (fieldAccessDescriptor.allFields()) { return inputSchema; } Schema.Builder builder = new Schema.Builder(); for (int fieldId : fieldAccessDescriptor.fieldIdsAccessed()) { builder.addField(inputSchema.getField(fieldId)); } for (Map.Entry<Integer, FieldAccessDescriptor> nested : fieldAccessDescriptor.nestedFields().entrySet()) { Field field = inputSchema.getField(nested.getKey()); FieldAccessDescriptor nestedDescriptor = nested.getValue(); FieldType nestedType = FieldType.row(getOutputSchema(field.getType().getRowSchema(), nestedDescriptor)); if (field.getNullable()) { builder.addNullableField(field.getName(), nestedType); } else { builder.addField(field.getName(), nestedType); } } return builder.build(); }
@Override public PCollection<KV<Row, Iterable<InputT>>> expand(PCollection<InputT> input) { Schema schema = input.getSchema(); FieldAccessDescriptor resolved = fieldAccessDescriptor.resolve(schema); keySchema = Select.getOutputSchema(schema, resolved); return input .apply( "Group by fields", ParDo.of( new DoFn<InputT, KV<Row, InputT>>() { @ProcessElement public void process( @Element InputT element, @Element Row row, OutputReceiver<KV<Row, InputT>> o) { o.output(KV.of(Select.selectRow(row, resolved, schema, keySchema), element)); } })) .setCoder(KvCoder.of(SchemaCoder.of(keySchema), input.getCoder())) .apply(GroupByKey.create()); } }