[
https://issues.apache.org/jira/browse/BEAM-9825?focusedWorklogId=431057&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-431057
]
ASF GitHub Bot logged work on BEAM-9825:
----------------------------------------
Author: ASF GitHub Bot
Created on: 06/May/20 07:15
Start Date: 06/May/20 07:15
Worklog Time Spent: 10m
Work Description: darshanj commented on a change in pull request #11610:
URL: https://github.com/apache/beam/pull/11610#discussion_r420587907
##########
File path:
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SetFns.java
##########
@@ -0,0 +1,261 @@
+/*
+ * 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 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.TupleTag;
+import
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
+
+public class SetFns {
+
+ /**
+ * Returns a new {@code SetFns.SetImpl<T>} transform that compute the
intersection with provided
+ * {@code PCollection<T>}.
+ *
+ * <p>The argument should not be modified after this is called.
+ *
+ * <p>The elements of the output {@link PCollection<T>} will all distinct
elements that present in
+ * both pipeline is constructed and provided {@link PCollection<T>}.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<String> left = p.apply(Create.of("1", "2", "3", "4", "5"));
+ * PCollection<String> right = p.apply(Create.of("1", "3", "4", "6"));
+ *
+ * PCollection<String> results =
+ * left.apply(SetFns.intersect(right));
+ * }</pre>
+ */
+ public static <T> SetImpl<T> intersect(PCollection<T> rightCollection) {
+ checkNotNull(rightCollection, "rightCollection argument is null");
+ SerializableBiFunction<Long, Long, Long> intersectFn =
+ (numberOfElementsinLeft, numberOfElementsinRight) ->
(numberOfElementsinLeft > 0 && numberOfElementsinRight > 0) ? 1L : 0L;
+ return new SetImpl<>(rightCollection, intersectFn);
+ }
+
+ /**
+ * Returns a new {@code SetFns.SetImpl<T>} transform that compute the
intersection all with
+ * provided {@code PCollection<T>}.
+ *
+ * <p>The argument should not be modified after this is called.
+ *
+ * <p>The elements of the output {@link PCollection<T>} which will follow
EXCEPT_ALL Semantics as
+ * follows: Given there are m elements on pipeline which is constructed
{@link PCollection<T>}
+ * (left) and n elements on in provided {@link PCollection<T>} (right): - it
will output MIN(m -
+ * n, 0) elements of left for all elements which are present in both left
and right.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<String> left = p.apply(Create.of("1", "2", "3", "4", "5"));
+ * PCollection<String> right = p.apply(Create.of("1", "3", "4", "6"));
+ *
+ * PCollection<String> results =
+ * left.apply(SetFns.intersectAll(right));
+ * }</pre>
+ */
+ public static <T> SetImpl<T> intersectAll(PCollection<T> rightCollection) {
+ checkNotNull(rightCollection, "rightCollection argument is null");
+ SerializableBiFunction<Long, Long, Long> intersectFn =
+ (numberOfElementsinLeft, numberOfElementsinRight) ->
(numberOfElementsinLeft > 0 && numberOfElementsinRight > 0) ?
Math.min(numberOfElementsinLeft, numberOfElementsinRight) : 0L;
+ return new SetImpl<>(rightCollection, intersectFn);
+ }
+
+ /**
+ * Returns a new {@code SetFns.SetImpl<T>} transform that compute the
difference (except) with
+ * provided {@code PCollection<T>}.
+ *
+ * <p>The argument should not be modified after this is called.
+ *
+ * <p>The elements of the output {@link PCollection<T>} will all distinct
elements that present in
+ * pipeline is constructed {@link PCollection<T>} but not present in
provided {@link
+ * PCollection<T>}.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<String> left = p.apply(Create.of("1", "2", "3", "4", "5"));
+ * PCollection<String> right = p.apply(Create.of("1", "3", "4", "6"));
+ *
+ * PCollection<String> results =
+ * left.apply(SetFns.except(right));
+ * }</pre>
+ */
+ public static <T> SetImpl<T> except(PCollection<T> rightCollection) {
+ checkNotNull(rightCollection, "rightCollection argument is null");
+ SerializableBiFunction<Long, Long, Long> exceptFn =
+ (numberOfElementsinLeft, numberOfElementsinRight) ->
numberOfElementsinLeft > 0 && numberOfElementsinRight == 0 ? 1L : 0L;
+ return new SetImpl<>(rightCollection, exceptFn);
+ }
+
+ /**
+ * Returns a new {@code SetFns.SetImpl<T>} transform that compute the
difference all (exceptAll)
+ * with provided {@code PCollection<T>}.
+ *
+ * <p>The argument should not be modified after this is called.
+ *
+ * <p>The elements of the output {@link PCollection<T>} which will follow
EXCEPT_ALL Semantics as
+ * follows: Given there are m elements on pipeline which is constructed
{@link PCollection<T>}
+ * (left) and n elements on in provided {@link PCollection<T>} (right): - it
will output m
+ * elements of left for all elements which are present in left but not in
right. - it will output
+ * MAX(m - n, 0) elements of left for all elements which are present in both
left and right.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<String> left = p.apply(Create.of("1", "2", "3", "4", "5"));
+ * PCollection<String> right = p.apply(Create.of("1", "3", "4", "6"));
+ *
+ * PCollection<String> results =
+ * left.apply(SetFns.exceptAll(right));
+ * }</pre>
+ */
+ public static <T> SetImpl<T> exceptAll(PCollection<T> rightCollection) {
+ checkNotNull(rightCollection, "rightCollection argument is null");
+ SerializableBiFunction<Long, Long, Long> exceptFn =
+ (numberOfElementsinLeft, numberOfElementsinRight) -> {
+ if (numberOfElementsinLeft > 0 && numberOfElementsinRight == 0) {
+ return numberOfElementsinLeft;
+ } else if (numberOfElementsinLeft > 0 && numberOfElementsinRight >
0) {
+ return Math.max(numberOfElementsinLeft - numberOfElementsinRight,
0L);
+ }
+ return 0L;
+ };
+ return new SetImpl<>(rightCollection, exceptFn);
+ }
+
+ /**
+ * Returns a new {@code SetFns.SetImpl<T>} transform that compute the union
with provided {@code
+ * PCollection<T>}.
+ *
+ * <p>The argument should not be modified after this is called.
+ *
+ * <p>The elements of the output {@link PCollection<T>} will all distinct
elements that present in
+ * pipeline is constructed {@link PCollection<T>} and {@link PCollection<T>}.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<String> left = p.apply(Create.of("1", "2", "3", "4", "5"));
+ * PCollection<String> right = p.apply(Create.of("1", "3", "4", "6"));
+ *
+ * PCollection<String> results =
+ * left.apply(SetFns.union(right));
+ * }</pre>
+ */
+ public static <T> SetImpl<T> union(PCollection<T> rightCollection) {
+ checkNotNull(rightCollection, "rightCollection argument is null");
+ SerializableBiFunction<Long, Long, Long> unionFn =
(numberOfElementsinLeft, numberOfElementsinRight) -> 1L;
+ return new SetImpl<>(rightCollection, unionFn);
+ }
+
+ /**
+ * Returns a new {@code SetFns.SetImpl<T>} transform that compute the
unionAll with provided
Review comment:
Thanks very valuable. I have fixed this.
----------------------------------------------------------------
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: 431057)
Remaining Estimate: 93h 50m (was: 94h)
Time Spent: 2h 10m (was: 2h)
> 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: 2h 10m
> Remaining Estimate: 93h 50m
>
> 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)