[
https://issues.apache.org/jira/browse/BEAM-9825?focusedWorklogId=433595&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-433595
]
ASF GitHub Bot logged work on BEAM-9825:
----------------------------------------
Author: ASF GitHub Bot
Created on: 15/May/20 08:21
Start Date: 15/May/20 08:21
Worklog Time Spent: 10m
Work Description: darshanj commented on a change in pull request #11610:
URL: https://github.com/apache/beam/pull/11610#discussion_r425642444
##########
File path:
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SetFns.java
##########
@@ -0,0 +1,618 @@
+/*
+ * 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.checkArgument;
+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.coders.Coder;
+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.transforms.windowing.Trigger;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+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 {
+
+ /**
+ * Returns a new {@code PTransform} transform that follows SET DISTINCT
semantics to 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} will all distinct
elements that present in
+ * both pipeline is constructed and provided {@link PCollection}.
+ *
+ * <p>Note that this transform requires that the {@code Coder} of the all
{@code PCollection<T>}
+ * to be deterministic (see {@link Coder#verifyDeterministic()}). If the
collection {@code Coder}
+ * is not deterministic, an exception is thrown at pipeline construction
time.
+ *
+ * <p>All inputs must have equal {@link WindowFn}s and compatible triggers
(see {@link
+ * Trigger#isCompatible(Trigger)}).
+ *
+ * <p>By default, the output {@code PCollection<T>} encodes its elements
using the same {@code
+ * Coder} that of {@code PCollection<T>}
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<String> left = p.apply(Create.of("1", "2", "3", "3","4",
"5"));
+ * PCollection<String> right = p.apply(Create.of("1", "3", "4","4", "6"));
+ *
+ * PCollection<String> results =
+ * left.apply(SetFns.intersectDistinct(right)); // results will be
PCollection<String> containing: "1","3","4"
+ *
+ * }</pre>
+ *
+ * @param <T> the type of the elements in the input and output {@code
PCollection<T>}s.
+ */
+ public static <T> SetImpl<T> intersectDistinct(PCollection<T>
rightCollection) {
+ checkNotNull(rightCollection, "rightCollection argument is null");
+ return new SetImpl<>(rightCollection, intersectDistinct());
+ }
+
+ /**
+ * Returns a {@code PTransform} that takes a {@code
PCollectionList<PCollection<T>>} and returns a
+ * {@code PCollection<T>} containing intersection of collections done in
order for all collections
+ * in {@code PCollectionList<T>}.
+ *
+ * <p>Returns a new {@code PTransform} transform that follows SET DISTINCT
semantics which takes a
+ * {@code PCollectionList<PCollection<T>>} and returns a {@code
PCollection<T>} containing
+ * intersection of collections done in order for all collections in {@code
PCollectionList<T>}.
+ *
+ * <p>The elements of the output {@link PCollection} will have all distinct
elements that are
+ * present in both pipeline is constructed and next {@link PCollection} in
the list and applied to
+ * all collections in order.
+ *
+ * <p>Note that this transform requires that the {@code Coder} of the all
{@code PCollection<T>}
+ * to be deterministic (see {@link Coder#verifyDeterministic()}). If the
collection {@code Coder}
+ * is not deterministic, an exception is thrown at pipeline construction
time.
+ *
+ * <p>All inputs must have equal {@link WindowFn}s and compatible triggers
(see {@link
+ * Trigger#isCompatible(Trigger)}).
+ *
+ * <p>By default, the output {@code PCollection<T>} encodes its elements
using the same {@code
+ * Coder} that of first in {@code PCollectionList<T>}
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<String> first = p.apply(Create.of("1", "2", "3", "3","4",
"5"));
+ * PCollection<String> second = p.apply(Create.of("1", "3", "4","4", "6"));
+ * PCollection<String> third = p.apply(Create.of("3", "4","4"));
+ *
+ * // Following example will perform (first intersect second) intersect
third.
+ * PCollection<String> results =
+ * PCollectionList.of(first).and(second).and(third)
+ * .apply(SetFns.intersectDistinct()); // results will be
PCollection<String> containing: "3","4"
+ *
+ * }</pre>
+ *
+ * @param <T> the type of the elements in the input {@code
PCollectionList<T>} and output {@code
+ * PCollection<T>}s.
+ */
+ public static <T> SetImplCollections<T> intersectDistinct() {
+ SerializableBiFunction<Long, Long, Long> intersectFn =
+ (numberOfElementsinLeft, numberOfElementsinRight) ->
+ (numberOfElementsinLeft > 0 && numberOfElementsinRight > 0) ? 1L :
0L;
+ return new SetImplCollections<>(intersectFn);
+ }
+
+ /**
+ * Returns a new {@code PTransform} transform that follows SET ALL semantics
to 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} which will follow
INTESECT_ALL Semantics as
+ * follows: Given there are m elements on pipeline which is constructed
{@link PCollection} (left)
+ * and n elements on in provided {@link PCollection} (right): - it will
output MIN(m - n, 0)
+ * elements of left for all elements which are present in both left and
right.
+ *
+ * <p>Note that this transform requires that the {@code Coder} of the all
{@code PCollection<T>}
+ * to be deterministic (see {@link Coder#verifyDeterministic()}). If the
collection {@code Coder}
+ * is not deterministic, an exception is thrown at pipeline construction
time.
+ *
+ * <p>All inputs must have equal {@link WindowFn}s and compatible triggers
(see {@link
+ * Trigger#isCompatible(Trigger)}).
+ *
+ * <p>By default, the output {@code PCollection<T>} encodes its elements
using the same {@code
+ * Coder} that of {@code PCollection<T>}
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<String> left = p.apply(Create.of("1","1","1", "2", "3",
"3","4", "5"));
+ * PCollection<String> right = p.apply(Create.of("1","1", "3", "4","4",
"6"));
+ *
+ * PCollection<String> results =
+ * left.apply(SetFns.intersectAll(right)); // results will be
PCollection<String> containing: "1","1","3","4"
+ * }</pre>
+ *
+ * @param <T> the type of the elements in the input and output {@code
PCollection<T>}s.
+ */
+ public static <T> SetImpl<T> intersectAll(PCollection<T> rightCollection) {
+ checkNotNull(rightCollection, "rightCollection argument is null");
+ return new SetImpl<>(rightCollection, intersectAll());
+ }
+
+ /**
+ * Returns a new {@code PTransform} transform that follows SET ALL semantics
which takes a {@code
+ * PCollectionList<PCollection<T>>} and returns a {@code PCollection<T>}
containing intersection
+ * all of collections done in order for all collections in {@code
PCollectionList<T>}.
+ *
+ * <p>The elements of the output {@link PCollection} which will follow
INTERSECT_ALL semantics.
+ * Output is calculated as follows: Given there are m elements on pipeline
which is constructed
+ * {@link PCollection} (left) and n elements on in provided {@link
PCollection} (right): - it will
+ * output MIN(m - n, 0) elements of left for all elements which are present
in both left and
+ * right.
+ *
+ * <p>Note that this transform requires that the {@code Coder} of the all
{@code PCollection<T>}
+ * to be deterministic (see {@link Coder#verifyDeterministic()}). If the
collection {@code Coder}
+ * is not deterministic, an exception is thrown at pipeline construction
time.
+ *
+ * <p>All inputs must have equal {@link WindowFn}s and compatible triggers
(see {@link
+ * Trigger#isCompatible(Trigger)}).
+ *
+ * <p>By default, the output {@code PCollection<T>} encodes its elements
using the same {@code
+ * Coder} that of first in {@code PCollectionList<T>}
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ * PCollection<String> first = p.apply(Create.of("1","1","1", "2", "3",
"3","4", "5"));
+ * PCollection<String> second = p.apply(Create.of("1","1", "3", "4","4",
"6"));
+ * PCollection<String> third = p.apply(Create.of("1", "5"));
+ *
+ * // Following example will perform (first intersect second) intersect
third.
+ * PCollection<String> results =
+ * PCollectionList.of(first).and(second).and(third)
+ * .apply(SetFns.intersectAll()); // results will be PCollection<String>
containing: "1"
+ *
+ * }</pre>
+ *
+ * @param <T> the type of the elements in the input {@code
PCollectionList<T>} and output {@code
+ * PCollection<T>}s.
+ */
+ public static <T> SetImplCollections<T> intersectAll() {
+ return new SetImplCollections<>(Math::min);
+ }
+
+ /**
+ * Returns a new {@code PTransform} transform that follows SET DISTINCT
semantics to 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} will all distinct
elements that present in
+ * pipeline is constructed but not present in provided {@link PCollection}.
+ *
+ * <p>Note that this transform requires that the {@code Coder} of the all
{@code PCollection<T>}
+ * to be deterministic (see {@link Coder#verifyDeterministic()}). If the
collection {@code Coder}
+ * is not deterministic, an exception is thrown at pipeline construction
time.
+ *
+ * <p>All inputs must have equal {@link WindowFn}s and compatible triggers
(see {@link
+ * Trigger#isCompatible(Trigger)}).
+ *
+ * <p>By default, the output {@code PCollection<T>} encodes its elements
using the same {@code
+ * Coder} that of {@code PCollection<T>}
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<String> left = p.apply(Create.of("1","1","1", "2", "3",
"3","4", "5"));
+ * PCollection<String> right = p.apply(Create.of("1","1", "3", "4","4",
"6"));
+ *
+ * PCollection<String> results =
+ * left.apply(SetFns.exceptDistinct(right)); // results will be
PCollection<String> containing: "2","5"
+ * }</pre>
+ *
+ * @param <T> the type of the elements in the input and output {@code
PCollection<T>}s.
+ */
+ public static <T> SetImpl<T> exceptDistinct(PCollection<T> rightCollection) {
+ checkNotNull(rightCollection, "rightCollection argument is null");
+ return new SetImpl<>(rightCollection, exceptDistinct());
+ }
+
+ /**
+ * Returns a {@code PTransform} that takes a {@code
PCollectionList<PCollection<T>>} and returns a
+ * {@code PCollection<T>} containing difference (except) of collections done
in order for all
+ * collections in {@code PCollectionList<T>}.
+ *
+ * <p>Returns a new {@code PTransform} transform that follows SET DISTINCT
semantics which takes a
+ * {@code PCollectionList<PCollection<T>>} and returns a {@code
PCollection<T>} containing
+ * difference (except) of collections done in order for all collections in
{@code
+ * PCollectionList<T>}.
+ *
+ * <p>The elements of the output {@link PCollection} will have all distinct
elements that are
+ * present in pipeline is constructed but not present in next {@link
PCollection} in the list and
+ * applied to all collections in order.
+ *
+ * <p>Note that this transform requires that the {@code Coder} of the all
{@code PCollection<T>}
+ * to be deterministic (see {@link Coder#verifyDeterministic()}). If the
collection {@code Coder}
+ * is not deterministic, an exception is thrown at pipeline construction
time.
+ *
+ * <p>All inputs must have equal {@link WindowFn}s and compatible triggers
(see {@link
+ * Trigger#isCompatible(Trigger)}).
+ *
+ * <p>By default, the output {@code PCollection<T>} encodes its elements
using the same {@code
+ * Coder} that of first in {@code PCollectionList<T>}
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ * PCollection<String> first = p.apply(Create.of("1","1","1", "2", "3",
"3","4", "5"));
+ * PCollection<String> second = p.apply(Create.of("1","1", "3", "4","4",
"6"));
+ *
+ * PCollection<String> third = p.apply(Create.of("1", "2","2"));
+ *
+ * // Following example will perform (first intersect second) intersect
third.
+ * PCollection<String> results =
+ * PCollectionList.of(first).and(second).and(third)
+ * .apply(SetFns.exceptDistinct()); // results will be
PCollection<String> containing: "5"
+ *
+ * }</pre>
+ *
+ * @param <T> the type of the elements in the input {@code
PCollectionList<T>} and output {@code
+ * PCollection<T>}s.
+ */
+ public static <T> SetImplCollections<T> exceptDistinct() {
+ SerializableBiFunction<Long, Long, Long> exceptFn =
+ (numberOfElementsinLeft, numberOfElementsinRight) ->
+ numberOfElementsinLeft > 0 && numberOfElementsinRight == 0 ? 1L :
0L;
+ return new SetImplCollections<>(exceptFn);
+ }
+
+ /**
+ * Returns a new {@code PTransform} transform that follows SET ALL semantics
to 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} which will follow
EXCEPT_ALL Semantics as
+ * follows: Given there are m elements on pipeline which is constructed
{@link PCollection} (left)
+ * and n elements on in provided {@link PCollection} (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.
+ *
+ * <p>Note that this transform requires that the {@code Coder} of the all
{@code PCollection<T>}
+ * to be deterministic (see {@link Coder#verifyDeterministic()}). If the
collection {@code Coder}
+ * is not deterministic, an exception is thrown at pipeline construction
time.
+ *
+ * <p>All inputs must have equal {@link WindowFn}s and compatible triggers
(see {@link
+ * Trigger#isCompatible(Trigger)}).
+ *
+ * <p>By default, the output {@code PCollection<T>} encodes its elements
using the same {@code
+ * Coder} that of {@code PCollection<T>}
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<String> left = p.apply(Create.of("1","1","1", "2", "3",
"3","3","4", "5"));
+ * PCollection<String> right = p.apply(Create.of("1", "3", "4","4", "6"));
+ *
+ * PCollection<String> results =
+ * left.apply(SetFns.exceptAll(right)); // results will be
PCollection<String> containing: "1","1","2","3","3","5"
+ * }</pre>
+ *
+ * @param <T> the type of the elements in the input and output {@code
PCollection<T>}s.
+ */
+ public static <T> SetImpl<T> exceptAll(PCollection<T> rightCollection) {
Review comment:
Yes. It is to avoid mutation and create correct abstraction. As this
Transform doesn't have any builder like pattern, does taking out a variable
makes any sense?
----------------------------------------------------------------
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: 433595)
Remaining Estimate: 87h 50m (was: 88h)
Time Spent: 8h 10m (was: 8h)
> 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: 8h 10m
> Remaining Estimate: 87h 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)