public class Sets
extends java.lang.Object
PTransform
s that allow to compute different set functions across PCollection
s.
They come in two variants. 1. Between two PCollection
2. Between two or more PCollection
in a PCollectionList
.
Following PTransform
s follows SET DISTINCT semantics: intersectDistinct,
expectDistinct, unionDistinct
Following PTransform
s follows SET ALL semantics: intersectAll, expectAll, unionAll
For example, the following demonstrates intersectDistinct between two collections PCollection
s.
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"
For example, the following demonstrates intersectDistinct between three collections PCollection
s in a PCollectionList
.
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"
Constructor and Description |
---|
Sets() |
Modifier and Type | Method and Description |
---|---|
static <T> PTransform<PCollectionList<T>,PCollection<T>> |
exceptAll()
Returns a new
PTransform transform that follows SET ALL semantics which takes a PCollectionList<PCollection<T>> and returns a PCollection<T> containing the difference
all (exceptAll) of collections done in order for all collections in PCollectionList<T> . |
static <T> PTransform<PCollection<T>,PCollection<T>> |
exceptAll(PCollection<T> rightCollection)
Returns a new
PTransform transform that follows SET ALL semantics to compute the
difference all (exceptAll) with provided PCollection<T> . |
static <T> PTransform<PCollectionList<T>,PCollection<T>> |
exceptDistinct()
Returns a
PTransform that takes a PCollectionList<PCollection<T>> and returns a
PCollection<T> containing the difference (except) of collections done in order for all
collections in PCollectionList<T> . |
static <T> PTransform<PCollection<T>,PCollection<T>> |
exceptDistinct(PCollection<T> rightCollection)
Returns a new
PTransform transform that follows SET DISTINCT semantics to compute the
difference (except) with provided PCollection<T> . |
static <T> PTransform<PCollectionList<T>,PCollection<T>> |
intersectAll()
Returns a new
PTransform transform that follows SET ALL semantics which takes a PCollectionList<PCollection<T>> and returns a PCollection<T> containing the
intersection all of collections done in order for all collections in PCollectionList<T> . |
static <T> PTransform<PCollection<T>,PCollection<T>> |
intersectAll(PCollection<T> rightCollection)
Returns a new
PTransform transform that follows SET ALL semantics to compute the
intersection with provided PCollection<T> . |
static <T> PTransform<PCollectionList<T>,PCollection<T>> |
intersectDistinct()
Returns a
PTransform that takes a PCollectionList<PCollection<T>> and returns a
PCollection<T> containing the intersection of collections done in order for all
collections in PCollectionList<T> . |
static <T> PTransform<PCollection<T>,PCollection<T>> |
intersectDistinct(PCollection<T> rightCollection)
Returns a new
PTransform transform that follows SET DISTINCT semantics to compute the
intersection with provided PCollection<T> . |
static <T> Flatten.PCollections<T> |
unionAll()
Returns a new
PTransform transform that follows SET ALL semantics which takes a PCollectionList<PCollection<T>> and returns a PCollection<T> containing the unionAll
of collections done in order for all collections in PCollectionList<T> . |
static <T> PTransform<PCollection<T>,PCollection<T>> |
unionAll(PCollection<T> rightCollection)
Returns a new
PTransform transform that follows SET ALL semantics to compute the
unionAll with provided PCollection<T> . |
static <T> PTransform<PCollectionList<T>,PCollection<T>> |
unionDistinct()
Returns a new
PTransform transform that follows SET DISTINCT semantics which takes a
PCollectionList<PCollection<T>> and returns a PCollection<T> containing the
union of collections done in order for all collections in PCollectionList<T> . |
static <T> PTransform<PCollection<T>,PCollection<T>> |
unionDistinct(PCollection<T> rightCollection)
Returns a new
PTransform transform that follows SET DISTINCT semantics to compute the
union with provided PCollection<T> . |
public static <T> PTransform<PCollection<T>,PCollection<T>> intersectDistinct(PCollection<T> rightCollection)
PTransform
transform that follows SET DISTINCT semantics to compute the
intersection with provided PCollection<T>
.
The argument should not be modified after this is called.
The elements of the output PCollection
will all distinct elements that present in
both pipeline is constructed and provided PCollection
.
Note that this transform requires that the Coder
of the all PCollection<T>
to be deterministic (see Coder.verifyDeterministic()
). If the collection Coder
is not deterministic, an exception is thrown at pipeline construction time.
All inputs must have equal WindowFn
s and compatible triggers (see Trigger.isCompatible(Trigger)
). Triggers with multiple firings may lead to nondeterministic
results since the this PTransform
is only computed over each individual firing.
By default, the output PCollection<T>
encodes its elements using the same Coder
as that of the input PCollection<T>
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"
T
- the type of the elements in the input and output PCollection<T>
s.public static <T> PTransform<PCollectionList<T>,PCollection<T>> intersectDistinct()
PTransform
that takes a PCollectionList<PCollection<T>>
and returns a
PCollection<T>
containing the intersection of collections done in order for all
collections in PCollectionList<T>
.
Returns a new PTransform
transform that follows SET DISTINCT semantics which takes a
PCollectionList<PCollection<T>>
and returns a PCollection<T>
containing the
intersection of collections done in order for all collections in PCollectionList<T>
.
The elements of the output PCollection
will have all distinct elements that are
present in both pipeline is constructed and next PCollection
in the list and applied to
all collections in order.
Note that this transform requires that the Coder
of the all PCollection<T>
to be deterministic (see Coder.verifyDeterministic()
). If the collection Coder
is not deterministic, an exception is thrown at pipeline construction time.
All inputs must have equal WindowFn
s and compatible triggers (see Trigger.isCompatible(Trigger)
).Triggers with multiple firings may lead to nondeterministic
results since the this PTransform
is only computed over each individual firing.
By default, the output PCollection<T>
encodes its elements using the same Coder
as that of the first PCollection<T>
in PCollectionList<T>
.
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"
T
- the type of the elements in the input PCollectionList<T>
and output PCollection<T>
s.public static <T> PTransform<PCollection<T>,PCollection<T>> intersectAll(PCollection<T> rightCollection)
PTransform
transform that follows SET ALL semantics to compute the
intersection with provided PCollection<T>
.
The argument should not be modified after this is called.
The elements of the output PCollection
which will follow INTESECT_ALL Semantics as
follows: Given there are m elements on pipeline which is constructed PCollection
(left)
and n elements on in provided PCollection
(right): - it will output MIN(m - n, 0)
elements of left for all elements which are present in both left and right.
Note that this transform requires that the Coder
of the all PCollection<T>
to be deterministic (see Coder.verifyDeterministic()
). If the collection Coder
is not deterministic, an exception is thrown at pipeline construction time.
All inputs must have equal WindowFn
s and compatible triggers (see Trigger.isCompatible(Trigger)
).Triggers with multiple firings may lead to nondeterministic
results since the this PTransform
is only computed over each individual firing.
By default, the output PCollection<T>
encodes its elements using the same Coder
as that of the input PCollection<T>
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"
T
- the type of the elements in the input and output PCollection<T>
s.public static <T> PTransform<PCollectionList<T>,PCollection<T>> intersectAll()
PTransform
transform that follows SET ALL semantics which takes a PCollectionList<PCollection<T>>
and returns a PCollection<T>
containing the
intersection all of collections done in order for all collections in PCollectionList<T>
.
The elements of the output PCollection
which will follow INTERSECT_ALL semantics.
Output is calculated as follows: Given there are m elements on pipeline which is constructed
PCollection
(left) and n elements on in provided PCollection
(right): - it will
output MIN(m - n, 0) elements of left for all elements which are present in both left and
right.
Note that this transform requires that the Coder
of the all PCollection<T>
to be deterministic (see Coder.verifyDeterministic()
). If the collection Coder
is not deterministic, an exception is thrown at pipeline construction time.
All inputs must have equal WindowFn
s and compatible triggers (see Trigger.isCompatible(Trigger)
).Triggers with multiple firings may lead to nondeterministic
results since the this PTransform
is only computed over each individual firing.
By default, the output PCollection<T>
encodes its elements using the same Coder
as that of the first PCollection<T>
in PCollectionList<T>
.
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"
T
- the type of the elements in the input PCollectionList<T>
and output PCollection<T>
s.public static <T> PTransform<PCollection<T>,PCollection<T>> exceptDistinct(PCollection<T> rightCollection)
PTransform
transform that follows SET DISTINCT semantics to compute the
difference (except) with provided PCollection<T>
.
The argument should not be modified after this is called.
The elements of the output PCollection
will all distinct elements that present in
pipeline is constructed but not present in provided PCollection
.
Note that this transform requires that the Coder
of the all PCollection<T>
to be deterministic (see Coder.verifyDeterministic()
). If the collection Coder
is not deterministic, an exception is thrown at pipeline construction time.
All inputs must have equal WindowFn
s and compatible triggers (see Trigger.isCompatible(Trigger)
).Triggers with multiple firings may lead to nondeterministic
results since the this PTransform
is only computed over each individual firing.
By default, the output PCollection<T>
encodes its elements using the same Coder
as that of the input PCollection<T>
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"
T
- the type of the elements in the input and output PCollection<T>
s.public static <T> PTransform<PCollectionList<T>,PCollection<T>> exceptDistinct()
PTransform
that takes a PCollectionList<PCollection<T>>
and returns a
PCollection<T>
containing the difference (except) of collections done in order for all
collections in PCollectionList<T>
.
Returns a new PTransform
transform that follows SET DISTINCT semantics which takes a
PCollectionList<PCollection<T>>
and returns a PCollection<T>
containing the
difference (except) of collections done in order for all collections in PCollectionList<T>
.
The elements of the output PCollection
will have all distinct elements that are
present in pipeline is constructed but not present in next PCollection
in the list and
applied to all collections in order.
Note that this transform requires that the Coder
of the all PCollection<T>
to be deterministic (see Coder.verifyDeterministic()
). If the collection Coder
is not deterministic, an exception is thrown at pipeline construction time.
All inputs must have equal WindowFn
s and compatible triggers (see Trigger.isCompatible(Trigger)
).Triggers with multiple firings may lead to nondeterministic
results since the this PTransform
is only computed over each individual firing.
By default, the output PCollection<T>
encodes its elements using the same Coder
as that of the first PCollection<T>
in PCollectionList<T>
.
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"
T
- the type of the elements in the input PCollectionList<T>
and output PCollection<T>
s.public static <T> PTransform<PCollection<T>,PCollection<T>> exceptAll(PCollection<T> rightCollection)
PTransform
transform that follows SET ALL semantics to compute the
difference all (exceptAll) with provided PCollection<T>
.
The argument should not be modified after this is called.
The elements of the output PCollection
which will follow EXCEPT_ALL Semantics as
follows: Given there are m elements on pipeline which is constructed PCollection
(left)
and n elements on in provided 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.
Note that this transform requires that the Coder
of the all PCollection<T>
to be deterministic (see Coder.verifyDeterministic()
). If the collection Coder
is not deterministic, an exception is thrown at pipeline construction time.
All inputs must have equal WindowFn
s and compatible triggers (see Trigger.isCompatible(Trigger)
).Triggers with multiple firings may lead to nondeterministic
results since the this PTransform
is only computed over each individual firing.
By default, the output PCollection<T>
encodes its elements using the same Coder
as that of the input PCollection<T>
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"
T
- the type of the elements in the input and output PCollection<T>
s.public static <T> PTransform<PCollectionList<T>,PCollection<T>> exceptAll()
PTransform
transform that follows SET ALL semantics which takes a PCollectionList<PCollection<T>>
and returns a PCollection<T>
containing the difference
all (exceptAll) of collections done in order for all collections in PCollectionList<T>
.
The elements of the output PCollection
which will follow EXCEPT_ALL semantics.
Output is calculated as follows: Given there are m elements on pipeline which is constructed
PCollection
(left) and n elements on in provided 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.
Note that this transform requires that the Coder
of the all PCollection<T>
to be deterministic (see Coder.verifyDeterministic()
). If the collection Coder
is not deterministic, an exception is thrown at pipeline construction time.
All inputs must have equal WindowFn
s and compatible triggers (see Trigger.isCompatible(Trigger)
).Triggers with multiple firings may lead to nondeterministic
results since the this PTransform
is only computed over each individual firing.
By default, the output PCollection<T>
encodes its elements using the same Coder
as that of the first PCollection<T>
in PCollectionList<T>
.
Pipeline p = ...;
PCollection<String> first = p.apply(Create.of("1", "1", "1", "2", "3", "3", "3", "4", "5"));
PCollection<String> second = p.apply(Create.of("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.exceptAll()); // results will be PCollection<String> containing: "1","2","3","3"
T
- the type of the elements in the input PCollectionList<T>
and output PCollection<T>
s.public static <T> PTransform<PCollection<T>,PCollection<T>> unionDistinct(PCollection<T> rightCollection)
PTransform
transform that follows SET DISTINCT semantics to compute the
union with provided PCollection<T>
.
The argument should not be modified after this is called.
The elements of the output PCollection
will all distinct elements that present in
pipeline is constructed or present in provided PCollection
.
Note that this transform requires that the Coder
of the all PCollection<T>
to be deterministic (see Coder.verifyDeterministic()
). If the collection Coder
is not deterministic, an exception is thrown at pipeline construction time.
All inputs must have equal WindowFn
s and compatible triggers (see Trigger.isCompatible(Trigger)
).Triggers with multiple firings may lead to nondeterministic
results since the this PTransform
is only computed over each individual firing.
By default, the output PCollection<T>
encodes its elements using the same Coder
as that of the input PCollection<T>
Pipeline p = ...;
PCollection<String> left = p.apply(Create.of("1", "1", "2"));
PCollection<String> right = p.apply(Create.of("1", "3", "4", "4"));
PCollection<String> results =
left.apply(SetFns.unionDistinct(right)); // results will be PCollection<String> containing: "1","2","3","4"
T
- the type of the elements in the input and output PCollection<T>
s.public static <T> PTransform<PCollectionList<T>,PCollection<T>> unionDistinct()
PTransform
transform that follows SET DISTINCT semantics which takes a
PCollectionList<PCollection<T>>
and returns a PCollection<T>
containing the
union of collections done in order for all collections in PCollectionList<T>
.
The elements of the output PCollection
will have all distinct elements that are
present in pipeline is constructed or present in next PCollection
in the list and
applied to all collections in order.
Note that this transform requires that the Coder
of the all PCollection<T>
to be deterministic (see Coder.verifyDeterministic()
). If the collection Coder
is not deterministic, an exception is thrown at pipeline construction time.
All inputs must have equal WindowFn
s and compatible triggers (see Trigger.isCompatible(Trigger)
).Triggers with multiple firings may lead to nondeterministic
results since the this PTransform
is only computed over each individual firing.
By default, the output PCollection<T>
encodes its elements using the same Coder
as that of the first PCollection<T>
in PCollectionList<T>
.
Pipeline p = ...;
PCollection<String> first = p.apply(Create.of("1", "1", "2"));
PCollection<String> second = p.apply(Create.of("1", "3", "4", "4"));
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.unionDistinct()); // results will be PCollection<String> containing: "1","2","3","4","5"
T
- the type of the elements in the input PCollectionList<T>
and output PCollection<T>
s.public static <T> PTransform<PCollection<T>,PCollection<T>> unionAll(PCollection<T> rightCollection)
PTransform
transform that follows SET ALL semantics to compute the
unionAll with provided PCollection<T>
.
The argument should not be modified after this is called.
The elements of the output PCollection
which will follow UNION_ALL semantics as
follows: Given there are m elements on pipeline which is constructed PCollection
(left)
and n elements on in provided PCollection
(right): - it will output m elements of left
and m elements of right.
Note that this transform requires that the Coder
of the all PCollection<T>
to be deterministic (see Coder.verifyDeterministic()
). If the collection Coder
is not deterministic, an exception is thrown at pipeline construction time.
All inputs must have equal WindowFn
s and compatible triggers (see Trigger.isCompatible(Trigger)
).Triggers with multiple firings may lead to nondeterministic
results since the this PTransform
is only computed over each individual firing.
By default, the output PCollection<T>
encodes its elements using the same Coder
as that of the input PCollection<T>
Pipeline p = ...;
PCollection<String> left = p.apply(Create.of("1", "1", "2"));
PCollection<String> right = p.apply(Create.of("1", "3", "4", "4"));
PCollection<String> results =
left.apply(SetFns.unionAll(right)); // results will be PCollection<String> containing: "1","1","1","2","3","4","4"
T
- the type of the elements in the input and output PCollection<T>
s.public static <T> Flatten.PCollections<T> unionAll()
PTransform
transform that follows SET ALL semantics which takes a PCollectionList<PCollection<T>>
and returns a PCollection<T>
containing the unionAll
of collections done in order for all collections in PCollectionList<T>
.
The elements of the output PCollection
which will follow UNION_ALL semantics. Output
is calculated as follows: Given there are m elements on pipeline which is constructed PCollection
(left) and n elements on in provided PCollection
(right): - it will output
m elements of left and m elements of right.
Note that this transform requires that the Coder
of the all inputs PCollection<T>
to be deterministic (see Coder.verifyDeterministic()
). If the
collection Coder
is not deterministic, an exception is thrown at pipeline construction
time.
All inputs must have equal WindowFn
s and compatible triggers (see Trigger.isCompatible(Trigger)
).Triggers with multiple firings may lead to nondeterministic
results since the this PTransform
is only computed over each individual firing.
By default, the output PCollection<T>
encodes its elements using the same Coder
as that of the first PCollection<T>
in PCollectionList<T>
.
Pipeline p = ...;
PCollection<String> first = p.apply(Create.of("1", "1", "2"));
PCollection<String> second = p.apply(Create.of("1", "3", "4", "4"));
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.unionAll()); // results will be PCollection<String> containing: "1","1","1","1","2","3","4","4","5"
T
- the type of the elements in the input PCollectionList<T>
and output PCollection<T>
s.