[
https://issues.apache.org/jira/browse/BEAM-4461?focusedWorklogId=206397&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-206397
]
ASF GitHub Bot logged work on BEAM-4461:
----------------------------------------
Author: ASF GitHub Bot
Created on: 01/Mar/19 11:42
Start Date: 01/Mar/19 11:42
Worklog Time Spent: 10m
Work Description: kanterov commented on pull request #7353: [BEAM-4461]
Support inner and outer style joins in CoGroup.
URL: https://github.com/apache/beam/pull/7353#discussion_r261573017
##########
File path:
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/CoGroup.java
##########
@@ -107,156 +106,251 @@
* those fields match. In this case, fields must be specified for every input
PCollection. For
* example:
*
- * <pre>{@code PCollection<KV<Row, Row>> joined = PCollectionTuple
- * .of(input1Tag, input1)
- * .and(input2Tag, input2)
+ * <pre>{@code PCollection<KV<Row, Row>> joined
+ * = PCollectionTuple.of("input1Tag", input1, "input2Tag", input2)
* .apply(CoGroup
- * .byFieldNames(input1Tag, "referringUser"))
- * .byFieldNames(input2Tag, "user"));
+ * .join("input1Tag", By.fieldNames("referringUser")))
+ * .join("input2Tag", By.fieldNames("user")));
* }</pre>
+ *
+ * <p>Traditional (SQL) joins are cross-product joins. All rows that match the
join condition are
+ * combined into individual rows and returned; in fact any SQL inner joins is
a subset of the
+ * cross-product of two tables. This transform also supports the same
functionality using the {@link
+ * Inner#crossProductJoin()} method.
+ *
+ * <p>For example, consider the SQL join: SELECT * FROM input1 INNER JOIN
input2 ON input1.user =
+ * input2.user
+ *
+ * <p>You could express this with:
+ *
+ * <pre>{@code
+ * PCollection<Row> joined = PCollectionTuple.of("input1", input1, "input2",
input2)
+ * .apply(CoGroup.join(By.fieldNames("user")).crossProductJoin();
+ * }</pre>
+ *
+ * <p>The schema of the output PCollection contains a nested message for each
of input1 and input2.
+ * Like above, you could use the {@link Convert} transform to convert it to
the following POJO:
+ *
+ * <pre>{@code
+ * {@literal @}DefaultSchema(JavaFieldSchema.class)
+ * public class JoinedValue {
+ * public Input1Type input1;
+ * public Input2Type input2;
+ * }
+ * }</pre>
+ *
+ * <p>The {@link Unnest} transform can then be used to flatten all the
subfields into one single
+ * top-level row containing all the fields in both Input1 and Input2; this
will often be combined
+ * with a {@link Select} transform to select out the fields of interest, as
the key fields will be
+ * identical between input1 and input2.
+ *
+ * <p>This transform also supports outer-join semantics. By default, all input
PCollections must
+ * participate fully in the join, providing inner-join semantics. This means
that the join will only
+ * produce values for "Bob" if all inputs have values for "Bob;" if even a
single input does not
+ * have a value for "Bob," an inner-join will produce no value. However, if
you mark that input as
+ * having outer-join participation then the join will contain values for
"Bob," as long as at least
+ * one input has a "Bob" value; null values will be added for inputs that have
no "Bob" values. To
+ * continue the SQL example:
+ *
+ * <p>SELECT * FROM input1 LEFT OUTER JOIN input2 ON input1.user = input2.user
+ *
+ * <p>Is equivalent to:
+ *
+ * <pre>{@code
+ * PCollection<Row> joined = PCollectionTuple.of("input1", input1, "input2",
input2)
+ * .apply(CoGroup.join("input1",
By.fieldNames("user").withOuterJoinParticipation())
+ * .join("input2", By.fieldNames("user"))
+ * .crossProductJoin();
+ * }</pre>
+ *
+ * <p>SELECT * FROM input1 RIGHT OUTER JOIN input2 ON input1.user = input2.user
+ *
+ * <p>Is equivalent to:
+ *
+ * <pre>{@code
+ * PCollection<Row> joined = PCollectionTuple.of("input1", input1, "input2",
input2)
+ * .apply(CoGroup.join("input1", By.fieldNames("user"))
+ * .join("input2",
By.fieldNames("user").withOuterJoinParticipation())
+ * .crossProductJoin();
+ * }</pre>
+ *
+ * <p>and SELECT * FROM input1 FULL OUTER JOIN input2 ON input1.user =
input2.user
+ *
+ * <p>Is equivalent to:
+ *
+ * <pre>{@code
+ * PCollection<Row> joined = PCollectionTuple.of("input1", input1, "input2",
input2)
+ * .apply(CoGroup.join("input1",
By.fieldNames("user").withOuterJoinParticipation())
+ * .join("input2",
By.fieldNames("user").withOuterJoinParticipation())
+ * .crossProductJoin();
+ * }</pre>
+ *
+ * <p>While the above examples use two inputs to mimic SQL's left and right
join semantics, the
+ * {@link CoGroup} transform supports any number of inputs, and outer-join
participation can be
+ * specified on any subset of them.
+ *
+ * <p>Do note that cross-product joins while simpler and easier to program,
can cause
*/
public class CoGroup {
- /**
- * Join by the following field names.
- *
- * <p>The same field names are used in all input PCollections.
- */
- public static Inner byFieldNames(String... fieldNames) {
- return
byFieldAccessDescriptor(FieldAccessDescriptor.withFieldNames(fieldNames));
- }
+ private static final List NULL_LIST;
- /**
- * Join by the following field ids.
- *
- * <p>The same field ids are used in all input PCollections.
- */
- public static Inner byFieldIds(Integer... fieldIds) {
- return
byFieldAccessDescriptor(FieldAccessDescriptor.withFieldIds(fieldIds));
+ static {
+ NULL_LIST = Lists.newArrayList();
+ NULL_LIST.add(null);
Review comment:
I'm not sure it's going to do safe publishing because list is modified after
final value is assigned. What about `NULL_LIST = Arrays.asList((Object) null)`
----------------------------------------------------------------
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: 206397)
Time Spent: 24h 20m (was: 24h 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: 24h 20m
> Remaining Estimate: 0h
>
> e.g. JoinBy(fields). Project, Filter, etc.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)