public BeamPCollectionTable(PCollection<InputT> upstream) { super(upstream.getSchema()); if (!upstream.hasSchema()) { throw new RuntimeException("SQL can only run over PCollections that have schemas."); } this.upstream = upstream; }
@Override public PCollection<T> expand(PCollection<T> input) { Schema schema = input.getSchema(); for (String fieldName : Sets.union(
@Override public PCollection<KV<Row, Row>> expand(PCollection<InputT> input) { SchemaAggregateFn.Inner<InputT> fn = schemaAggregateFn.withSchema(input.getSchema(), input.getToRowFunction()); return input.apply(byFields).apply(Combine.groupedValues(fn)); } }
@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); } }
@Override public PCollection<Row> expand(PCollection<InputT> input) { SchemaAggregateFn.Inner<InputT> fn = schemaAggregateFn.withSchema(input.getSchema(), input.getToRowFunction()); return input.apply(Combine.globally(fn)).setRowSchema(fn.getOutputSchema()); } }
@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; }
TupleTag<?> tag = entry.getKey(); PCollection<?> pc = entry.getValue(); Schema schema = pc.getSchema(); componentSchemas.put(tag.getId(), schema); TupleTag<Row> rowTag = new TupleTag<>(tag.getId() + "_ROW");
(SchemaCoder<OutputT>) SchemaCoder.of( input.getSchema(), SerializableFunctions.identity(), SerializableFunctions.identity()); if (!outputSchemaCoder.getSchema().assignableToIgnoreNullable(input.getSchema())) { throw new RuntimeException( "Cannot convert between types that don't have equivalent schemas." + " input schema: " + input.getSchema() + " output schema: " + outputSchemaCoder.getSchema());
@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()); } }
@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 testSimpleUnnesting() { 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_SCHEMA).addValues(r, r).build()) .collect(Collectors.toList()); PCollection<Row> unnested = pipeline.apply(Create.of(rows).withRowSchema(NESTED_SCHEMA)).apply(Unnest.create()); assertEquals(UNNESTED_SCHEMA, unnested.getSchema()); List<Row> expected = bottomRow .stream() .map( r -> Row.withSchema(UNNESTED_SCHEMA) .addValues(r.getValue(0), r.getValue(1), r.getValue(0), r.getValue(1)) .build()) .collect(Collectors.toList()); ; PAssert.that(unnested).containsInAnyOrder(expected); pipeline.run(); }
@Test @Category(NeedsRunner.class) public void testOutputCoders() { Schema keySchema = Schema.builder().addStringField("field1").build(); PCollection<KV<Row, Iterable<POJO>>> grouped = pipeline .apply(Create.of(new POJO("key1", 1, "value1"))) .apply(Group.byFieldNames("field1")); // Make sure that the key has the right schema. PCollection<Row> keys = grouped.apply(Keys.create()); assertTrue(keys.getSchema().equivalent(keySchema)); // Make sure that the value has the right schema. PCollection<POJO> values = grouped.apply(Values.create()).apply(Flatten.iterables()); assertTrue(values.getSchema().equivalent(POJO_SCHEMA)); pipeline.run(); }
@Test public void testAvgGroupByNullable() { String sql = "SELECT AVG(f_int1), f_int2 FROM PCOLLECTION GROUP BY f_int2"; PCollection<Row> out = boundedInput.apply(SqlTransform.query(sql)); Schema schema = out.getSchema(); PAssert.that(out) .containsInAnyOrder( Row.withSchema(schema).addValues(null, null).build(), Row.withSchema(schema).addValues(2, 1).build(), Row.withSchema(schema).addValues(1, 5).build(), Row.withSchema(schema).addValues(3, 2).build()); pipeline.run(); }
@Test public void testCountGroupByNullable() { String sql = "SELECT COUNT(f_int1) as c, f_int2 FROM PCOLLECTION GROUP BY f_int2"; PCollection<Row> out = boundedInput.apply(SqlTransform.query(sql)); Schema schema = out.getSchema(); PAssert.that(out) .containsInAnyOrder( Row.withSchema(schema).addValues(0L, null).build(), Row.withSchema(schema).addValues(1L, 1).build(), Row.withSchema(schema).addValues(1L, 5).build(), Row.withSchema(schema).addValues(1L, 2).build()); assertEquals( Schema.builder() // COUNT() is never nullable, and calcite knows it .addInt64Field("c") .addNullableField("f_int2", Schema.FieldType.INT32) .build(), schema); pipeline.run(); }