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

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

                Author: ASF GitHub Bot
            Created on: 14/May/20 21:09
            Start Date: 14/May/20 21:09
    Worklog Time Spent: 10m 
      Work Description: lukecwik commented on a change in pull request #11610:
URL: https://github.com/apache/beam/pull/11610#discussion_r425432135



##########
File path: 
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SetFns.java
##########
@@ -0,0 +1,528 @@
+/*
+ * 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.transforms;
+
+import static 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.Pipeline;
+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.PCollectionList;
+import org.apache.beam.sdk.values.TupleTag;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
+
+public class SetFns {

Review comment:
       We should mention in the comments that we rely on the deterministic 
encoding of the coder similar to how we do GroupByKey.
   
   Also, this implementation assumes that there will only be a single firing of 
the trigger. If there are multiple then the results are likely undefined.
   
   




----------------------------------------------------------------
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:
[email protected]


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

            Worklog Id:     (was: 433362)
    Remaining Estimate: 89h  (was: 89h 10m)
            Time Spent: 7h  (was: 6h 50m)

> Transforms for Intersect, IntersectAll, Except, ExceptAll, Union, UnionAll
> --------------------------------------------------------------------------
>
>                 Key: BEAM-9825
>                 URL: https://issues.apache.org/jira/browse/BEAM-9825
>             Project: Beam
>          Issue Type: New Feature
>          Components: sdk-java-core
>            Reporter: Darshan Jani
>            Assignee: Darshan Jani
>            Priority: Major
>   Original Estimate: 96h
>          Time Spent: 7h
>  Remaining Estimate: 89h
>
> I'd like to propose following new high-level transforms.
>  * Intersect
> Compute the intersection between elements of two PCollection.
> Given _leftCollection_ and _rightCollection_, this transform returns a 
> collection containing elements that common to both _leftCollection_ and 
> _rightCollection_
>  
>  * Except
> Compute the difference between elements of two PCollection.
> Given _leftCollection_ and _rightCollection_, this transform returns a 
> collection containing elements that are in _leftCollection_ but not in 
> _rightCollection_
>  * Union
> Find the elements that are either of two PCollection.
> Implement IntersetAll, ExceptAll and UnionAll variants of transforms.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to