@Test @Category(NeedsRunner.class) public void testAlternateNamePolicy() { List<Row> bottomRow = IntStream.rangeClosed(0, 2) .mapToObj(i -> Row.withSchema(SIMPLE_SCHEMA).addValues(i, Integer.toString(i)).build()) .collect(Collectors.toList()); List<Row> rows = bottomRow .stream() .map(r -> Row.withSchema(NESTED_SCHEMA2).addValues(r).build()) .collect(Collectors.toList()); PCollection<Row> unnested = pipeline .apply(Create.of(rows).withRowSchema(NESTED_SCHEMA2)) .apply(Unnest.<Row>create().withFieldNameFunction(Unnest.KEEP_NESTED_NAME)); assertEquals(UNNESTED2_SCHEMA_ALTERNATE, unnested.getSchema()); List<Row> expected = bottomRow .stream() .map( r -> Row.withSchema(UNNESTED2_SCHEMA_ALTERNATE) .addValues(r.getValue(0), r.getValue(1)) .build()) .collect(Collectors.toList()); ; PAssert.that(unnested).containsInAnyOrder(expected); pipeline.run(); }
@Test @Category(NeedsRunner.class) public void testClashingNamePolicy() { List<Row> bottomRow = IntStream.rangeClosed(0, 2) .mapToObj(i -> Row.withSchema(SIMPLE_SCHEMA).addValues(i, Integer.toString(i)).build()) .collect(Collectors.toList()); thrown.expect(IllegalArgumentException.class); List<Row> rows = bottomRow .stream() .map(r -> Row.withSchema(NESTED_SCHEMA).addValues(r, r).build()) .collect(Collectors.toList()); PCollection<Row> unnested = pipeline .apply(Create.of(rows).withRowSchema(NESTED_SCHEMA)) .apply(Unnest.<Row>create().withFieldNameFunction(Unnest.KEEP_NESTED_NAME)); pipeline.run(); } }
/** * Sets a policy for naming deeply-nested fields. * * <p>This is needed to prevent name collisions when differently-nested fields have the same * name. The default is to use the {@link #CONCAT_FIELD_NAMES} strategy that concatenates all * names in the path to generate the unnested name. For example, an unnested name might be * field1_field2_field3. In some cases the {@link #KEEP_NESTED_NAME} strategy can be used to * keep only the most-deeply nested name. However if this results in conflicting names (e.g. if * a schema has two subrows that each have the same schema this will happen), the pipeline will * fail at construction time. * * <p>An example of using this function to customize the separator character: * * <pre>{@code * pc.apply(Unnest.<Type>create().withFieldNameFunction(l -> Strings.join("+", l))); * }</pre> */ public Inner<T> withFieldNameFunction(SerializableFunction<List<String>, String> fn) { return toBuilder().setFieldNameFunction(fn).build(); }
@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); } }