fieldAccessDescriptor = FieldAccessDescriptor.withAllFields(); } else {
@Override public PCollection<Row> expand(PCollection<T> input) { Schema inputSchema = input.getSchema(); verifyCompatibility(inputSchema); return 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 input, OutputReceiver<Row> r) { Row output = castRow(input, inputSchema, outputSchema()); r.output(output); } })) .setRowSchema(outputSchema()); }
@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 public void testAllFields() { FieldAccessDescriptor fieldAccessDescriptor = FieldAccessDescriptor.withAllFields(); assertTrue(fieldAccessDescriptor.resolve(SIMPLE_SCHEMA).allFields()); }
@Test public void testNestedFieldById() { FieldAccessDescriptor fieldAccessDescriptor = FieldAccessDescriptor.withFieldNames("field1") .withNestedField(1, FieldAccessDescriptor.withAllFields()); fieldAccessDescriptor = fieldAccessDescriptor.resolve(NESTED_SCHEMA2); assertTrue(fieldAccessDescriptor.fieldIdsAccessed().isEmpty()); assertEquals(1, fieldAccessDescriptor.nestedFields().size()); FieldAccessDescriptor nestedAccess = fieldAccessDescriptor.nestedFields().get(1); assertTrue(nestedAccess.allFields()); }
@Test public void testNestedFieldByName() { FieldAccessDescriptor fieldAccessDescriptor = FieldAccessDescriptor.withFieldNames("field1") .withNestedField("field1", FieldAccessDescriptor.withAllFields()); fieldAccessDescriptor = fieldAccessDescriptor.resolve(NESTED_SCHEMA2); assertTrue(fieldAccessDescriptor.fieldIdsAccessed().isEmpty()); assertEquals(1, fieldAccessDescriptor.nestedFields().size()); FieldAccessDescriptor nestedAccess = fieldAccessDescriptor.nestedFields().get(1); assertTrue(nestedAccess.allFields()); }
fieldAccessDescriptor = FieldAccessDescriptor.withAllFields(); } else {
@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 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({ValidatesRunner.class, UsesSchema.class}) public void testFieldAccessSchemaPipeline() { List<MyPojo> pojoList = Lists.newArrayList(new MyPojo("a", 1), new MyPojo("b", 2), new MyPojo("c", 3)); Schema schema = Schema.builder().addStringField("string_field").addInt32Field("integer_field").build(); PCollection<String> output = pipeline .apply( Create.of(pojoList) .withSchema( schema, o -> Row.withSchema(schema).addValues(o.stringField, o.integerField).build(), r -> new MyPojo(r.getString("string_field"), r.getInt32("integer_field")))) .apply( ParDo.of( new DoFn<MyPojo, String>() { @FieldAccess("foo") final FieldAccessDescriptor fieldAccess = FieldAccessDescriptor.withAllFields(); @ProcessElement public void process(@FieldAccess("foo") Row row, OutputReceiver<String> r) { r.output(row.getString(0) + ":" + row.getInt32(1)); } })); PAssert.that(output).containsInAnyOrder("a:1", "b:2", "c:3"); pipeline.run(); }