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

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

                Author: ASF GitHub Bot
            Created on: 01/Nov/18 19:18
            Start Date: 01/Nov/18 19:18
    Worklog Time Spent: 10m 
      Work Description: akedin commented on a change in pull request #6832: 
[BEAM-4461] CoGroup transforms for schemas.
URL: https://github.com/apache/beam/pull/6832#discussion_r230143949
 
 

 ##########
 File path: 
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/CoGroup.java
 ##########
 @@ -0,0 +1,326 @@
+/*
+ * 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 com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.schemas.FieldAccessDescriptor;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.join.CoGbkResult;
+import org.apache.beam.sdk.transforms.join.CoGroupByKey;
+import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TupleTag;
+
+/**
+ * A transform that performs equijoins across multiple schema {@link 
PCollection}s.
+ *
+ * <p>This transform has similarites to {@link CoGroupByKey}, however works on 
PCollections that
+ * have schemas. This allows users of the transform to simply specify schema 
fields to join on. The
+ * output type of the transform is a {@literal KV<Row, Row>} where the value 
contains one field for
+ * every input PCollection and the key represents the fields that were joined 
on. By default the
+ * cross product is not expanded, so all fields in the output row are array 
fields.
+ *
+ * <p>For example, the following demonstrates joining three PCollections on 
the "user" and "country"
+ * fields.
+ *
+ * <pre>{@code
+ * TupleTag<Input1Type> input1Tag = new TupleTag<>("input1");
+ * TupleTag<Input2Type> input2Tag = new TupleTag<>("input2");
+ * TupleTag<Input3Type> input3Tag = new TupleTag<>("input3");
+ * PCollection<KV<Row, Row>> joined = PCollectionTuple
+ *     .of(input1Tag, input1)
+ *     .and(input2Tag, input2)
+ *     .and(input3Tag, input3)
+ *   .apply(CoGroup.byFieldNames("user", "country"));
+ * }</pre>
+ *
+ * <p>In the above case, the key schema will contain the two string fields 
"user" and "country"; in
+ * this case, the schemas for Input1, Input2, Input3 must all have fields 
named "user" and
+ * "country". The value schema will contain three array of Row fields named 
"input1" "input2" and
 
 Review comment:
   Also I am not sure it is a right thing to put the values into rows, 
especially if you already use KVs and CoGBK, I would stick to CoGbkResult-style 
or KVs:
   
   ```
   Key:    Row with fields/values of Group fields:
                           { 
                               "user" : xxxx,
                               "country" : yyyy 
                           }
             Value: CoGbkResult with keys corresponding to each input tag, 
values are iterables of Rows for the matching Keys
                           {
                               KV { "input1" =>  [  /* rows from input1 with 
matching key */ ] },
                               KV { "input2" => [ /* rows from input 2 with 
matching key */ ] },
                               KV { "input3" => [ 
                                     // rows from input 3 with matching key, 
e.g.:
                                     Row { "user" : xxxx, "country": yyyy, 
"orderId" : aaaa, "sku" : 1234 },
                                     Row { "user" : xxxx, "country": yyyy, 
"orderId" : bbbb, "sku" : 5678 }
                                   ]
                           }
    ```

----------------------------------------------------------------
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: 161706)
    Time Spent: 16h 40m  (was: 16.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
>          Time Spent: 16h 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