[ 
https://issues.apache.org/jira/browse/BEAM-4461?focusedWorklogId=205965&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-205965
 ]

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_r261341623
 
 

 ##########
 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. */
 
 Review comment:
   fixed
 
----------------------------------------------------------------
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: 205965)
    Time Spent: 22h 40m  (was: 22.5h)

> 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: 22h 40m
>  Remaining Estimate: 0h
>
> e.g. JoinBy(fields). Project, Filter, etc.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to