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

ASF GitHub Bot logged work on BEAM-6098:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 27/Nov/18 18:51
            Start Date: 27/Nov/18 18:51
    Worklog Time Spent: 10m 
      Work Description: kennknowles commented on a change in pull request 
#7118: [BEAM-6098] Support lookup join symmetric in left/right inputs
URL: https://github.com/apache/beam/pull/7118#discussion_r236798796
 
 

 ##########
 File path: 
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRel.java
 ##########
 @@ -120,35 +120,91 @@ public Join copy(
 
   @Override
   public List<RelNode> getPCollectionInputs() {
-    if (isSideInputJoin()) {
-      return ImmutableList.of(BeamSqlRelUtils.getBeamRelInput(left));
+    if (isSideInputLookupJoin()) {
+      return ImmutableList.of(
+          
BeamSqlRelUtils.getBeamRelInput(getInputs().get(nonSeekableInputIndex().get())));
+    } else {
+      return BeamRelNode.super.getPCollectionInputs();
     }
-    return BeamRelNode.super.getPCollectionInputs();
   }
 
   @Override
   public PTransform<PCollectionList<Row>, PCollection<Row>> buildPTransform() {
-    return new Transform();
+    if (isSideInputLookupJoin()) {
+      return new SideInputLookupJoin();
+    } else {
+      return new Transform();
+    }
+  }
+
+  private boolean isSideInputLookupJoin() {
+    return seekableInputIndex().isPresent() && 
nonSeekableInputIndex().isPresent();
 
 Review comment:
   It is a good question. I was just copying existing logic for symmetry. If 
they are both seekable it seems still fine to do this sort of join. You would 
want to make the main input the smaller one. But we don't have sizes yet anyhow.

----------------------------------------------------------------
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:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 169955)
    Time Spent: 1.5h  (was: 1h 20m)

> Support side input join on right or left
> ----------------------------------------
>
>                 Key: BEAM-6098
>                 URL: https://issues.apache.org/jira/browse/BEAM-6098
>             Project: Beam
>          Issue Type: Improvement
>          Components: dsl-sql
>            Reporter: Kenneth Knowles
>            Assignee: Kenneth Knowles
>            Priority: Major
>          Time Spent: 1.5h
>  Remaining Estimate: 0h
>
> Currently, side input join is hardcoded as to which of left vs right must be 
> seekable/non-seekable. Making this symmetric would be an incremental 
> improvement.



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

Reply via email to