[
https://issues.apache.org/jira/browse/BEAM-4461?focusedWorklogId=205969&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-205969
]
ASF GitHub Bot logged work on BEAM-4461:
----------------------------------------
Author: ASF GitHub Bot
Created on: 28/Feb/19 19:10
Start Date: 28/Feb/19 19:10
Worklog Time Spent: 10m
Work Description: reuvenlax commented on pull request #7353: [BEAM-4461]
Support inner and outer style joins in CoGroup.
URL: https://github.com/apache/beam/pull/7353#discussion_r261341687
##########
File path:
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/CoGroup.java
##########
@@ -275,76 +358,263 @@ private FieldAccessDescriptor
getFieldAccessDescriptor(TupleTag<?> tag) {
}
}
+ // Create a new tag for the output.
+ TupleTag randomTag = new TupleTag<>();
+ String keyedTag = tag + "_" + randomTag;
+ tagToTupleTag.put(tag, keyedTag);
PCollection<KV<Row, Row>> keyedPCollection =
- extractKey(pc, schema, keySchema, resolved, tag.getId());
- keyedPCollectionTuple = keyedPCollectionTuple.and(rowTag,
keyedPCollection);
+ extractKey(pc, schema, keySchema, resolved, tag);
+ keyedPCollectionTuple = keyedPCollectionTuple.and(keyedTag,
keyedPCollection);
+ }
+ return new JoinInformation(
+ keyedPCollectionTuple, keySchema, componentSchemas, toRows,
sortedTags, tagToTupleTag);
+ }
+
+ private static <T> PCollection<KV<Row, Row>> extractKey(
+ PCollection<T> pCollection,
+ Schema schema,
+ Schema keySchema,
+ FieldAccessDescriptor keyFields,
+ String tag) {
+ return pCollection
+ .apply(
+ "extractKey" + tag,
+ ParDo.of(
+ new DoFn<T, KV<Row, Row>>() {
+ @ProcessElement
+ public void process(@Element Row row,
OutputReceiver<KV<Row, Row>> o) {
+ o.output(KV.of(Select.selectRow(row, keyFields, schema,
keySchema), row));
+ }
+ }))
+ .setCoder(KvCoder.of(SchemaCoder.of(keySchema),
SchemaCoder.of(schema)));
+ }
+ }
+
+ /** The implementing PTransform. */
+ public static class Inner extends PTransform<PCollectionTuple,
PCollection<KV<Row, Row>>> {
+ private final JoinArguments joinArgs;
+
+ private Inner() {
+ this(new JoinArguments(Collections.emptyMap()));
+ }
+
+ private Inner(JoinArguments joinArgs) {
+ this.joinArgs = joinArgs;
+ }
+
+ /**
+ * Select the following fields for the specified PCollection with the
specified join args.
+ *
+ * <p>Each PCollection in the input must have fields specified for the
join key.
+ */
+ public Inner join(String tag, By clause) {
+ if (joinArgs.allInputsJoinArgs != null) {
+ throw new IllegalStateException("Cannot set both a global and per-tag
fields.");
}
+ return new Inner(joinArgs.with(tag, clause));
+ }
+
+ /** Expand the join into imdividual rows, similar to SQL joins. */
+ public ExpandCrossProduct crossProductJoin() {
+ return new ExpandCrossProduct(joinArgs);
+ }
+ private Schema getOutputSchema(JoinInformation joinInformation) {
// Construct the output schema. It contains one field for each input
PCollection, of type
// ARRAY[ROW].
Schema.Builder joinedSchemaBuilder = Schema.builder();
- for (Map.Entry<String, Schema> entry : componentSchemas.entrySet()) {
+ for (Map.Entry<String, Schema> entry :
joinInformation.componentSchemas.entrySet()) {
joinedSchemaBuilder.addArrayField(entry.getKey(),
FieldType.row(entry.getValue()));
}
- Schema joinedSchema = joinedSchemaBuilder.build();
+ return joinedSchemaBuilder.build();
+ }
+
+ @Override
+ public PCollection<KV<Row, Row>> expand(PCollectionTuple input) {
+ JoinInformation joinInformation =
+ JoinInformation.from(input, joinArgs::getFieldAccessDescriptor);
+
+ Schema joinedSchema = getOutputSchema(joinInformation);
- return keyedPCollectionTuple
+ return joinInformation
+ .keyedPCollectionTuple
.apply("CoGroupByKey", CoGroupByKey.create())
- .apply("ConvertToRow", ParDo.of(new ConvertToRow(sortedTags, toRows,
joinedSchema)))
- .setCoder(KvCoder.of(SchemaCoder.of(keySchema),
SchemaCoder.of(joinedSchema)));
+ .apply(
+ "ConvertToRow",
+ ParDo.of(
+ new ConvertToRow(
+ joinInformation.sortedTags,
+ joinInformation.toRows,
+ joinedSchema,
+ joinInformation.tagToTupleTag)))
+ .setCoder(
+ KvCoder.of(SchemaCoder.of(joinInformation.keySchema),
SchemaCoder.of(joinedSchema)));
}
+ // Used by the unexpanded join to create the output rows.
private static class ConvertToRow extends DoFn<KV<Row, CoGbkResult>,
KV<Row, Row>> {
- List<TupleTag<Row>> sortedTags;
- Map<String, SerializableFunction<Object, Row>> toRows =
Maps.newHashMap();
- Schema joinedSchema;
+ private final List<String> sortedTags;
+ private final Map<String, SerializableFunction<Object, Row>> toRows;
+ private final Map<String, String> tagToTupleTag;
+ private final Schema joinedSchema;
- public ConvertToRow(
- List<TupleTag<Row>> sortedTags,
+ ConvertToRow(
+ List<String> sortedTags,
Map<String, SerializableFunction<Object, Row>> toRows,
- Schema joinedSchema) {
+ Schema joinedSchema,
+ Map<String, String> tagToTupleTag) {
this.sortedTags = sortedTags;
this.toRows = toRows;
this.joinedSchema = joinedSchema;
+ this.tagToTupleTag = tagToTupleTag;
}
@ProcessElement
public void process(@Element KV<Row, CoGbkResult> kv,
OutputReceiver<KV<Row, Row>> o) {
Row key = kv.getKey();
CoGbkResult result = kv.getValue();
List<Object> fields =
Lists.newArrayListWithExpectedSize(sortedTags.size());
- for (TupleTag<?> tag : sortedTags) {
+ for (String tag : sortedTags) {
// TODO: This forces the entire join to materialize in memory. We
should create a
// lazy Row interface on top of the iterable returned by
CoGbkResult. This will
// allow the data to be streamed in.
- SerializableFunction<Object, Row> toRow = toRows.get(tag.getId());
+ SerializableFunction<Object, Row> toRow = toRows.get(tag);
+ String tupleTag = tagToTupleTag.get(tag);
List<Row> joined = Lists.newArrayList();
- for (Object item : result.getAll(tag)) {
+ for (Object item : result.getAll(tupleTag)) {
joined.add(toRow.apply(item));
}
fields.add(joined);
}
o.output(KV.of(key,
Row.withSchema(joinedSchema).addValues(fields).build()));
}
}
+ }
- private static <T> PCollection<KV<Row, Row>> extractKey(
- PCollection<T> pCollection,
- Schema schema,
- Schema keySchema,
- FieldAccessDescriptor keyFields,
- String tag) {
- return pCollection
+ /** A {@link PTransform} that calculates the cross-product join. */
+ public static class ExpandCrossProduct extends PTransform<PCollectionTuple,
PCollection<Row>> {
+ private final JoinArguments joinArgs;
+
+ ExpandCrossProduct(JoinArguments joinArgs) {
+ this.joinArgs = joinArgs;
+ }
+
+ /**
+ * Select the following fields for the specified PCollection with the
specified join args.
+ *
+ * <p>Each PCollection in the input must have fields specified for the
join key.
+ */
+ public ExpandCrossProduct join(String tag, By clause) {
+ if (joinArgs.allInputsJoinArgs != null) {
+ throw new IllegalStateException("Cannot set both a global and per-tag
fields.");
+ }
+ return new ExpandCrossProduct(joinArgs.with(tag, clause));
+ }
+
+ private Schema getOutputSchema(JoinInformation joinInformation) {
+ // Construct the output schema. It contains one field for each input
PCollection, of type
+ // ROW. If a field supports outer-join semantics, then that field will
be nullable in the
+ // schema.
+ Schema.Builder joinedSchemaBuilder = Schema.builder();
+ for (Map.Entry<String, Schema> entry :
joinInformation.componentSchemas.entrySet()) {
+ FieldType fieldType = FieldType.row(entry.getValue());
+ if (joinArgs.getOuterJoinParticipation(entry.getKey())) {
+ fieldType = fieldType.withNullable(true);
+ }
+ joinedSchemaBuilder.addField(entry.getKey(), fieldType);
+ }
+ return joinedSchemaBuilder.build();
+ }
+
+ @Override
+ public PCollection<Row> expand(PCollectionTuple input) {
+ JoinInformation joinInformation =
+ JoinInformation.from(input, joinArgs::getFieldAccessDescriptor);
+
+ Schema joinedSchema = getOutputSchema(joinInformation);
+
+ return joinInformation
+ .keyedPCollectionTuple
+ .apply("CoGroupByKey", CoGroupByKey.create())
+ .apply("Values", Values.create())
.apply(
- "extractKey" + tag,
+ "ExpandToRow",
ParDo.of(
- new DoFn<T, KV<Row, Row>>() {
- @ProcessElement
- public void process(@Element Row row,
OutputReceiver<KV<Row, Row>> o) {
- o.output(KV.of(Select.selectRow(row, keyFields, schema,
keySchema), row));
- }
- }))
- .setCoder(KvCoder.of(SchemaCoder.of(keySchema),
SchemaCoder.of(schema)));
+ new ExpandToRows(
+ joinInformation.sortedTags,
+ joinInformation.toRows,
+ joinedSchema,
+ joinInformation.tagToTupleTag)))
+ .setRowSchema(joinedSchema);
+ }
+
+ /** A DoFn that expands the result of a CoGroupByKey into the cross
product. */
+ private class ExpandToRows extends DoFn<CoGbkResult, Row> {
+ private final List<String> sortedTags;
+ private final Map<String, SerializableFunction<Object, Row>> toRows;
+ private final Schema outputSchema;
+ private final Map<String, String> tagToTupleTag;
+
+ public ExpandToRows(
Review comment:
This is an excellent point. Split this out so we create the iterators ahead
of time. Also changed maps to key off of tag index so that we don't have to
hash the string on every single element.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 205969)
Time Spent: 23h 20m (was: 23h 10m)
> Create a library of useful transforms that use schemas
> ------------------------------------------------------
>
> Key: BEAM-4461
> URL: https://issues.apache.org/jira/browse/BEAM-4461
> Project: Beam
> Issue Type: Sub-task
> Components: sdk-java-core
> Reporter: Reuven Lax
> Assignee: Reuven Lax
> Priority: Major
> Labels: triaged
> Time Spent: 23h 20m
> Remaining Estimate: 0h
>
> e.g. JoinBy(fields). Project, Filter, etc.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)