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

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

                Author: ASF GitHub Bot
            Created on: 12/Apr/19 23:02
            Start Date: 12/Apr/19 23:02
    Worklog Time Spent: 10m 
      Work Description: robinyqiu commented on pull request #8273: [BEAM-4461] 
A transform to perform binary joins of PCollections with schemas
URL: https://github.com/apache/beam/pull/8273#discussion_r275078312
 
 

 ##########
 File path: 
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Join.java
 ##########
 @@ -0,0 +1,237 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.schemas.transforms;
+
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.schemas.FieldAccessDescriptor;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.Row;
+
+/**
+ * A transform that performs equijoins across two schema {@link PCollection}s.
+ *
+ * <p>This transform allows joins between two input PCollections simply by 
specifying the fields to
+ * join on. The resulting {@code PCollection<Row>} will have two fields named 
"lhs" and "rhs"
+ * respectively, each with the schema of the corresponding input PCollection.
+ *
+ * <p>For example, the following demonstrates joining two PCollections using a 
natural join on the
+ * "user" and "country" fields, where both the left-hand and the right-hand 
PCollections have fields
+ * with these names.
+ *
+ * <pre>
+ * {@code PCollection<Row> joined = 
pCollection1.apply(Join.innerJoin(pCollection2).using("user", "country"));
+ * }</pre>
+ *
+ * <p>If the right-hand PCollection contains fields with different names to 
join against, you can
+ * specify them as follows:
+ *
+ * <pre>{@code PCollection<Row> joined = 
pCollection1.apply(Join.innerJoin(pCollection2)
+ *       .on(FieldsEqual.left("user", "country").right("otherUser", 
"otherCountry")));
+ * }</pre>
+ *
+ * <p>Full outer joins, left outer joins, and right outer joins are also 
supported.
+ */
+public class Join {
+  public static final String LHS_TAG = "lhs";
+  public static final String RHS_TAG = "rhs";
+
+  /** Predicate object to specify fields to compare when doing an equi-join. */
+  public static class FieldsEqual {
+    public static Inner left(String... fieldNames) {
+      return new Inner(
+          FieldAccessDescriptor.withFieldNames(fieldNames), 
FieldAccessDescriptor.create());
+    }
+
+    public static Inner left(Integer... fieldIds) {
+      return new Inner(
+          FieldAccessDescriptor.withFieldIds(fieldIds), 
FieldAccessDescriptor.create());
+    }
+
+    public static Inner left(FieldAccessDescriptor fieldAccessDescriptor) {
+      return new Inner(fieldAccessDescriptor, FieldAccessDescriptor.create());
+    }
+
+    public Inner right(String... fieldNames) {
+      return new Inner(
+          FieldAccessDescriptor.create(), 
FieldAccessDescriptor.withFieldNames(fieldNames));
+    }
+
+    public Inner right(Integer... fieldIds) {
+      return new Inner(
+          FieldAccessDescriptor.create(), 
FieldAccessDescriptor.withFieldIds(fieldIds));
+    }
+
+    public Inner right(FieldAccessDescriptor fieldAccessDescriptor) {
+      return new Inner(FieldAccessDescriptor.create(), fieldAccessDescriptor);
+    }
+
+    /** Implementation class for FieldsEqual. */
+    public static class Inner {
+      private FieldAccessDescriptor lhs;
+      private FieldAccessDescriptor rhs;
+
+      private Inner(FieldAccessDescriptor lhs, FieldAccessDescriptor rhs) {
+        this.lhs = lhs;
+        this.rhs = rhs;
+      }
+
+      public Inner left(String... fieldNames) {
+        return new Inner(FieldAccessDescriptor.withFieldNames(fieldNames), 
rhs);
+      }
+
+      public Inner left(Integer... fieldIds) {
+        return new Inner(FieldAccessDescriptor.withFieldIds(fieldIds), rhs);
+      }
+
+      public Inner left(FieldAccessDescriptor fieldAccessDescriptor) {
+        return new Inner(fieldAccessDescriptor, rhs);
+      }
+
+      public Inner right(String... fieldNames) {
+        return new Inner(lhs, 
FieldAccessDescriptor.withFieldNames(fieldNames));
+      }
+
+      public Inner right(Integer... fieldIds) {
+        return new Inner(lhs, FieldAccessDescriptor.withFieldIds(fieldIds));
+      }
+
+      public Inner right(FieldAccessDescriptor fieldAccessDescriptor) {
+        return new Inner(lhs, fieldAccessDescriptor);
+      }
+
+      private Inner resolve(Schema lhsSchema, Schema rhsSchema) {
+        return new Inner(lhs.resolve(lhsSchema), rhs.resolve(rhsSchema));
+      }
+    }
+  }
+
+  /** Perform an inner join. */
+  public static <LhsT, RhsT> Inner<LhsT, RhsT> innerJoin(PCollection<RhsT> 
rhs) {
+    return new Inner<>(JoinType.INNER, rhs);
+  }
+
+  /** Perform a full outer join. */
+  public static <LhsT, RhsT> Inner<LhsT, RhsT> fullOuterJoin(PCollection<RhsT> 
rhs) {
+    return new Inner<>(JoinType.OUTER, rhs);
+  }
+
+  /** Perform a left outer join. */
+  public static <LhsT, RhsT> Inner<LhsT, RhsT> leftOuterJoin(PCollection<RhsT> 
rhs) {
+    return new Inner<>(JoinType.LEFT_OUTER, rhs);
+  }
+
+  /** Perform a right outer join. */
+  public static <LhsT, RhsT> Inner<LhsT, RhsT> 
rightOuterJoin(PCollection<RhsT> rhs) {
+    return new Inner<>(JoinType.RIGHT_OUTER, rhs);
+  };
+
+  private enum JoinType {
+    INNER,
+    OUTER,
+    LEFT_OUTER,
+    RIGHT_OUTER
+  };
+
+  /** Implementation class . */
+  public static class Inner<LhsT, RhsT> extends PTransform<PCollection<LhsT>, 
PCollection<Row>> {
+    private final JoinType joinType;
+    private final PCollection<RhsT> rhs;
+    @Nullable private final FieldsEqual.Inner predicate;
+
+    private Inner(JoinType joinType, PCollection<RhsT> rhs) {
+      this(joinType, rhs, null);
+    }
+
+    private Inner(JoinType joinType, PCollection<RhsT> rhs, FieldsEqual.Inner 
predicate) {
+      this.joinType = joinType;
+      this.rhs = rhs;
+      this.predicate = predicate;
+    }
+
+    /**
+     * Perform a natural join between the PCollections. The fields are 
expected to exist in both
+     * PCollections
+     */
+    public Inner<LhsT, RhsT> using(String... fieldNames) {
+      return new Inner<>(joinType, rhs, 
FieldsEqual.left(fieldNames).right(fieldNames));
+    }
+
+    /**
+     * Perform a natural join between the PCollections. The fields are 
expected to exist in both
+     * PCollections
+     */
+    public Inner<LhsT, RhsT> using(Integer... fieldIds) {
+      return new Inner<>(joinType, rhs, 
FieldsEqual.left(fieldIds).right(fieldIds));
+    }
+
+    /**
+     * Perform a natural join between the PCollections. The fields are 
expected to exist in both
+     * PCollections
+     */
+    public Inner<LhsT, RhsT> using(FieldAccessDescriptor 
fieldAccessDescriptor) {
+      return new Inner<>(
+          joinType, rhs, 
FieldsEqual.left(fieldAccessDescriptor).right(fieldAccessDescriptor));
+    }
+
+    /** Join the PCollections using the provided predicate. */
+    public Inner<LhsT, RhsT> on(FieldsEqual.Inner predicate) {
+      return new Inner<>(joinType, rhs, predicate);
+    }
+
+    @Override
+    public PCollection<Row> expand(PCollection lhs) {
 
 Review comment:
   Why don't use the more specific type here: `PCollection<LhsT>`
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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: 227003)
    Time Spent: 25h 40m  (was: 25.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: 25h 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