/** * Select a set of fields described in a {@link FieldAccessDescriptor}. * * <p>This allows for nested fields to be selected as well. */ public static <T> Select<T> fieldAccess(FieldAccessDescriptor fieldAccessDescriptor) { return new Select(fieldAccessDescriptor); }
@ProcessElement public void process( @FieldAccess("filterFields") Row row, OutputReceiver<Row> r) { r.output(selectRow(row, resolved, inputSchema, outputSchema)); } }))
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; }
@Test @Category(NeedsRunner.class) public void testSelectAll() { PCollection<POJO1> pojos = pipeline .apply(Create.of(new POJO1())) .apply(Select.fieldAccess(FieldAccessDescriptor.withAllFields())) .apply(Convert.to(POJO1.class)); PAssert.that(pojos).containsInAnyOrder(new POJO1()); pipeline.run(); }
@Test @Category(NeedsRunner.class) public void testSelectMissingFieldName() { thrown.expect(IllegalArgumentException.class); pipeline.apply(Create.of(new POJO1())).apply(Select.fieldNames("missing")); pipeline.run(); }
@Test @Category(NeedsRunner.class) public void testSelectMissingFieldIndex() { thrown.expect(IllegalArgumentException.class); pipeline.apply(Create.of(new POJO1())).apply(Select.fieldIds(42)); pipeline.run(); }
Schema currentKeySchema = Select.getOutputSchema(schema, resolved); if (keySchema == null) { keySchema = currentKeySchema;
@Test @Category(NeedsRunner.class) public void testSelectNestedAll() { PCollection<POJO2NestedAll> pojos = pipeline .apply(Create.of(new POJO2())) .apply( Select.fieldAccess( FieldAccessDescriptor.create() .withNestedField("field2", FieldAccessDescriptor.withAllFields()))) .apply(Convert.to(POJO2NestedAll.class)); PAssert.that(pojos).containsInAnyOrder(new POJO2NestedAll()); pipeline.run(); }
@Test @Category(NeedsRunner.class) public void testSimpleSelect() { PCollection<POJO1Selected> pojos = pipeline .apply(Create.of(new POJO1())) .apply(Select.fieldNames("field1", "field3")) .apply(Convert.to(POJO1Selected.class)); PAssert.that(pojos).containsInAnyOrder(new POJO1Selected()); pipeline.run(); }
@ProcessElement public void process(@Element Row row, OutputReceiver<KV<Row, Row>> o) { o.output(KV.of(Select.selectRow(row, keyFields, schema, keySchema), row)); } }))
@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; }
@Test @Category(NeedsRunner.class) public void testSelectNestedPartial() { PCollection<POJO2NestedPartial> pojos = pipeline .apply(Create.of(new POJO2())) .apply( Select.fieldAccess( FieldAccessDescriptor.create() .withNestedField( "field2", FieldAccessDescriptor.withFieldNames("field1", "field3")))) .apply(Convert.to(POJO2NestedPartial.class)); PAssert.that(pojos).containsInAnyOrder(new POJO2NestedPartial()); pipeline.run(); } }
/** Select a set of top-level field names from the row. */ public static <T> Select<T> fieldNames(String... names) { return new Select(FieldAccessDescriptor.withFieldNames(names)); }
@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)); } }))
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(); }
/** Select a set of top-level field ids from the row. */ public static <T> Select<T> fieldIds(Integer... ids) { return new Select(FieldAccessDescriptor.withFieldIds(ids)); }
@Override public Row apply(T input) { Row row = toRowFunction.apply(input); return Select.selectRow( row, fieldAggregation.fieldsToAggregate, row.getSchema(), fieldAggregation.inputSubSchema); } }
@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()); } }
@Override public OutputT apply(InputT input) { Row row = toRowFunction.apply(input); Row selected = Select.selectRow( row, fieldAggregation.fieldsToAggregate, row.getSchema(), fieldAggregation.inputSubSchema); if (fieldAggregation.needsUnnesting) { selected = Unnest.unnestRow(selected, fieldAggregation.unnestedInputSubSchema); } return selected.getValue(0); } }
static Row selectRow( Row input, FieldAccessDescriptor fieldAccessDescriptor, Schema inputSchema, Schema outputSchema) { if (fieldAccessDescriptor.allFields()) { return input; } else { Row.Builder output = Row.withSchema(outputSchema); for (int fieldId : fieldAccessDescriptor.fieldIdsAccessed()) { output.addValue(input.getValue(fieldId)); } for (Map.Entry<Integer, FieldAccessDescriptor> nested : fieldAccessDescriptor.nestedFields().entrySet()) { String fieldName = inputSchema.nameOf(nested.getKey()); Schema nestedInputSchema = inputSchema.getField(nested.getKey()).getType().getRowSchema(); Schema nestedOutputSchema = outputSchema.getField(fieldName).getType().getRowSchema(); output.addValue( selectRow( input.getValue(fieldName), nested.getValue(), nestedInputSchema, nestedOutputSchema)); } return output.build(); } } }