K
- the type of the keys of the input and output
PCollection
sV
- the type of the values of the input PCollection
and the elements of the Iterable
s in the output
PCollection
public class GroupByKey<K,V> extends PTransform<PCollection<KV<K,V>>,PCollection<KV<K,Iterable<V>>>>
GroupByKey<K, V>
takes a PCollection<KV<K, V>>
,
groups the values by key and windows, and returns a
PCollection<KV<K, Iterable<V>>>
representing a map from
each distinct key and window of the input PCollection
to an
Iterable
over all the values associated with that key in
the input per window. Absent repeatedly-firing
triggering
, each key in the output
PCollection
is unique within each window.
GroupByKey
is analogous to converting a multi-map into
a uni-map, and related to GROUP BY
in SQL. It corresponds
to the "shuffle" step between the Mapper and the Reducer in the
MapReduce framework.
Two keys of type K
are compared for equality
not by regular Java Object.equals(java.lang.Object)
, but instead by
first encoding each of the keys using the Coder
of the
keys of the input PCollection
, and then comparing the
encoded bytes. This admits efficient parallel evaluation. Note that
this requires that the Coder
of the keys be deterministic (see
Coder.verifyDeterministic()
). If the key Coder
is not
deterministic, an exception is thrown at pipeline construction time.
By default, the Coder
of the keys of the output
PCollection
is the same as that of the keys of the input,
and the Coder
of the elements of the Iterable
values of the output PCollection
is the same as the
Coder
of the values of the input.
Example of use:
PCollection<KV<String, Doc>> urlDocPairs = ...;
PCollection<KV<String, Iterable<Doc>>> urlToDocs =
urlDocPairs.apply(GroupByKey.<String, Doc>create());
PCollection<R> results =
urlToDocs.apply(ParDo.of(new DoFn<KV<String, Iterable<Doc>>, R>() {
public void processElement(ProcessContext c) {
String url = c.element().getKey();
Iterable<Doc> docsWithThatUrl = c.element().getValue();
... process all docs having that url ...
}}));
GroupByKey
is a key primitive in data-parallel
processing, since it is the main way to efficiently bring
associated data together into one location. It is also a key
determiner of the performance of a data-parallel pipeline.
See CoGroupByKey
for a way to group multiple input PCollections by a common key at once.
See Combine.PerKey
for a common pattern of
GroupByKey
followed by Combine.GroupedValues
.
When grouping, windows that can be merged according to the WindowFn
of the input PCollection
will be merged together, and a window pane
corresponding to the new, merged window will be created. The items in this pane
will be emitted when a trigger fires. By default this will be when the input
sources estimate there will be no more data for the window. See
AfterWatermark
for details on the estimation.
The timestamp for each emitted pane is determined by the
windowing operation
.
The output PCollection
will have the same WindowFn
as the input.
If the input PCollection
contains late data (see
PubsubIO.Read.Bound.timestampLabel
for an example of how this can occur) or the
requested TriggerFn
can fire before
the watermark, then there may be multiple elements
output by a GroupByKey
that correspond to the same key and window.
If the WindowFn
of the input requires merging, it is not
valid to apply another GroupByKey
without first applying a new
WindowFn
or applying Window.remerge()
.
Modifier and Type | Class and Description |
---|---|
static class |
GroupByKey.GroupAlsoByWindow<K,V>
Helper transform that takes a collection of timestamp-ordered
values associated with each key, groups the values by window,
combines windows as needed, and for each window in each key,
outputs a collection of key/value-list pairs implicitly assigned
to the window and with the timestamp derived from that window.
|
static class |
GroupByKey.GroupByKeyOnly<K,V>
Primitive helper transform that groups by key only, ignoring any
window assignments.
|
static class |
GroupByKey.ReifyTimestampsAndWindows<K,V>
Helper transform that makes timestamps and window assignments
explicit in the value part of each key/value pair.
|
static class |
GroupByKey.SortValuesByTimestamp<K,V>
Helper transform that sorts the values associated with each key
by timestamp.
|
name
Modifier and Type | Method and Description |
---|---|
static void |
applicableTo(PCollection<?> input) |
PCollection<KV<K,Iterable<V>>> |
apply(PCollection<KV<K,V>> input)
Applies this
PTransform on the given InputT , and returns its
Output . |
static <K,V> GroupByKey<K,V> |
create()
Returns a
GroupByKey<K, V> PTransform . |
boolean |
fewKeys()
Returns whether it groups just few keys.
|
protected Coder<KV<K,Iterable<V>>> |
getDefaultOutputCoder(PCollection<KV<K,V>> input)
Returns the default
Coder to use for the output of this
single-output PTransform when applied to the given input. |
static <K,V> Coder<V> |
getInputValueCoder(Coder<KV<K,V>> inputCoder)
Returns the
Coder of the values of the input to this transform. |
com.google.cloud.dataflow.sdk.util.WindowingStrategy<?,?> |
updateWindowingStrategy(com.google.cloud.dataflow.sdk.util.WindowingStrategy<?,?> inputStrategy) |
void |
validate(PCollection<KV<K,V>> input)
Called before invoking apply (which may be intercepted by the runner) to
verify this transform is fully specified and applicable to the specified
input.
|
getDefaultOutputCoder, getDefaultOutputCoder, getKindString, getName, toString
public static <K,V> GroupByKey<K,V> create()
GroupByKey<K, V>
PTransform
.K
- the type of the keys of the input and output
PCollection
sV
- the type of the values of the input PCollection
and the elements of the Iterable
s in the output
PCollection
public boolean fewKeys()
public static void applicableTo(PCollection<?> input)
public void validate(PCollection<KV<K,V>> input)
PTransform
By default, does nothing.
validate
in class PTransform<PCollection<KV<K,V>>,PCollection<KV<K,Iterable<V>>>>
public com.google.cloud.dataflow.sdk.util.WindowingStrategy<?,?> updateWindowingStrategy(com.google.cloud.dataflow.sdk.util.WindowingStrategy<?,?> inputStrategy)
public PCollection<KV<K,Iterable<V>>> apply(PCollection<KV<K,V>> input)
PTransform
PTransform
on the given InputT
, and returns its
Output
.
Composite transforms, which are defined in terms of other transforms, should return the output of one of the composed transforms. Non-composite transforms, which do not apply any transforms internally, should return a new unbound output and register evaluators (via backend-specific registration methods).
The default implementation throws an exception. A derived class must
either implement apply, or else each runner must supply a custom
implementation via
PipelineRunner.apply(com.google.cloud.dataflow.sdk.transforms.PTransform<InputT, OutputT>, InputT)
.
apply
in class PTransform<PCollection<KV<K,V>>,PCollection<KV<K,Iterable<V>>>>
protected Coder<KV<K,Iterable<V>>> getDefaultOutputCoder(PCollection<KV<K,V>> input)
PTransform
Coder
to use for the output of this
single-output PTransform
when applied to the given input.getDefaultOutputCoder
in class PTransform<PCollection<KV<K,V>>,PCollection<KV<K,Iterable<V>>>>