/** * Select the following fields for the specified PCollection using {@link * FieldAccessDescriptor}. * * <p>Each PCollection in the input must have fields specified for the join key. */ public Inner byFieldAccessDescriptor( TupleTag<?> tag, FieldAccessDescriptor fieldAccessDescriptor) { if (allInputsFieldAccessDescriptor != null) { throw new IllegalStateException("Cannot set both a global and per-tag fields."); } return new Inner( new ImmutableMap.Builder<TupleTag<?>, FieldAccessDescriptor>() .putAll(fieldAccessDescriptorMap) .put(tag, fieldAccessDescriptor) .build()); }
TupleTag<Row> rowTag = new TupleTag<>(tag.getId() + "_ROW"); toRows.put(rowTag.getId(), (SerializableFunction<Object, Row>) pc.getToRowFunction()); FieldAccessDescriptor fieldAccessDescriptor = getFieldAccessDescriptor(tag); if (fieldAccessDescriptor == null) { throw new IllegalStateException("No fields were set for input " + tag); extractKey(pc, schema, keySchema, resolved, tag.getId()); keyedPCollectionTuple = keyedPCollectionTuple.and(rowTag, keyedPCollection);
"CoGroup", CoGroup.byFieldNames(pc1Tag, "user", "country") .byFieldNames(pc2Tag, "user2", "country2") .byFieldNames(pc3Tag, "user3", "country3"));
"CoGroup", CoGroup.byFieldNames(pc1Tag, "user", "country") .byFieldNames(pc2Tag, "user2", "country2")); pipeline.run();
@Test @Category(NeedsRunner.class) public void testMismatchingKeys() { PCollection<Row> pc1 = pipeline .apply( "Create1", Create.of(Row.withSchema(CG_SCHEMA_1).addValues("user1", 1, "us").build())) .setRowSchema(CG_SCHEMA_1); PCollection<Row> pc2 = pipeline .apply( "Create2", Create.of(Row.withSchema(CG_SCHEMA_1).addValues("user1", 9, "us").build())) .setRowSchema(CG_SCHEMA_1); TupleTag<Row> pc1Tag = new TupleTag<>("pc1"); TupleTag<Row> pc2Tag = new TupleTag<>("pc2"); thrown.expect(IllegalStateException.class); PCollection<KV<Row, Row>> joined = PCollectionTuple.of(pc1Tag, pc1) .and(pc2Tag, pc2) .apply("CoGroup", CoGroup.byFieldNames(pc1Tag, "user").byFieldNames(pc2Tag, "count")); pipeline.run(); }
/** * Join by the following {@link FieldAccessDescriptor}. * * <p>The same access descriptor is used in all input PCollections. */ public static Inner byFieldAccessDescriptor(FieldAccessDescriptor fieldAccessDescriptor) { return new Inner(fieldAccessDescriptor); }
/** * Join by the following field names. * * <p>The same field names are used in all input PCollections. */ public Inner byFieldNames(TupleTag<?> tag, String... fieldNames) { return byFieldAccessDescriptor(tag, FieldAccessDescriptor.withFieldNames(fieldNames)); }
/** * Select the following field ids for the specified PCollection. * * <p>Each PCollection in the input must have fields specified for the join key. */ public Inner byFieldIds(TupleTag<?> tag, Integer... fieldIds) { return byFieldAccessDescriptor(tag, FieldAccessDescriptor.withFieldIds(fieldIds)); }
/** * Select the following fields for the specified PCollection using {@link FieldAccessDescriptor}. * * <p>Each PCollection in the input must have fields specified for the join key. */ public static Inner byFieldAccessDescriptor( TupleTag<?> tag, FieldAccessDescriptor fieldAccessDescriptor) { return new Inner().byFieldAccessDescriptor(tag, fieldAccessDescriptor); }