/** Returns the result of unnesting the given schema with the given naming policy. */ static Schema getUnnestedSchema(Schema schema, SerializableFunction<List<String>, String> fn) { List<String> nameComponents = Lists.newArrayList(); return getUnnestedSchema(schema, nameComponents, fn); }
/** Returns the result of unnesting the given schema. The default naming policy is used. */ static Schema getUnnestedSchema(Schema schema) { List<String> nameComponents = Lists.newArrayList(); return getUnnestedSchema(schema, nameComponents, CONCAT_FIELD_NAMES); } /** Returns the result of unnesting the given schema with the given naming policy. */
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; }
@Override public PCollection<Row> expand(PCollection<T> input) { Schema inputSchema = input.getSchema(); Schema outputSchema = getUnnestedSchema(inputSchema, getFieldNameFunction()); return input .apply( ParDo.of( new DoFn<T, Row>() { @ProcessElement public void processElement(@Element Row row, OutputReceiver<Row> o) { o.output(unnestRow(row, outputSchema)); } })) .setRowSchema(outputSchema); } }
private static Schema getUnnestedSchema( Schema schema, List<String> nameComponents, SerializableFunction<List<String>, String> fn) { Schema.Builder builder = Schema.builder(); for (Field field : schema.getFields()) { nameComponents.add(field.getName()); if (field.getType().getTypeName().isCompositeType()) { Schema nestedSchema = getUnnestedSchema(field.getType().getRowSchema(), nameComponents, fn); for (Field nestedField : nestedSchema.getFields()) { builder.addField(nestedField); } } else { String name = fn.apply(nameComponents); Field newField = field.toBuilder().setName(name).build(); builder.addField(newField); } nameComponents.remove(nameComponents.size() - 1); } return builder.build(); } /** Unnest a row. */