nehsyc commented on a change in pull request #13208:
URL: https://github.com/apache/beam/pull/13208#discussion_r513774255
##########
File path:
runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/GroupIntoBatchesOverride.java
##########
@@ -103,43 +156,76 @@ public void process(ProcessContext c) {
}
@Override
- public PTransformReplacement<PCollection<KV<K, V>>, PCollection<KV<K,
Iterable<V>>>>
+ public PTransformReplacement<PCollection<KV<K, V>>,
PCollection<KV<ShardedKey<K>, Iterable<V>>>>
getReplacementTransform(
AppliedPTransform<
- PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>,
GroupIntoBatches<K, V>>
+ PCollection<KV<K, V>>,
+ PCollection<KV<ShardedKey<K>, Iterable<V>>>,
+ GroupIntoBatches<K, V>.WithShardedKey>
transform) {
return PTransformReplacement.of(
PTransformReplacements.getSingletonMainInput(transform),
- new StreamingGroupIntoBatches(runner, transform.getTransform()));
+ new StreamingGroupIntoBatchesWithShardedKey<>(runner,
transform.getTransform()));
}
@Override
public Map<PCollection<?>, ReplacementOutput> mapOutputs(
- Map<TupleTag<?>, PCollection<?>> outputs, PCollection<KV<K,
Iterable<V>>> newOutput) {
+ Map<TupleTag<?>, PCollection<?>> outputs,
+ PCollection<KV<ShardedKey<K>, Iterable<V>>> newOutput) {
return ReplacementOutputs.singleton(outputs, newOutput);
}
}
/**
- * Specialized implementation of {@link GroupIntoBatches} for unbounded
Dataflow pipelines. The
- * override does the same thing as the original transform but additionally
record the input to add
- * corresponding properties during the graph translation.
+ * Specialized implementation of {@link GroupIntoBatches.WithShardedKey} for
unbounded Dataflow
+ * pipelines. The override does the same thing as the original transform but
additionally records
+ * the input of {@code GroupIntoBatchesDoFn} in order to append relevant
step properties during
+ * the graph translation.
*/
- static class StreamingGroupIntoBatches<K, V>
- extends PTransform<PCollection<KV<K, V>>, PCollection<KV<K,
Iterable<V>>>> {
+ static class StreamingGroupIntoBatchesWithShardedKey<K, V>
+ extends PTransform<PCollection<KV<K, V>>, PCollection<KV<ShardedKey<K>,
Iterable<V>>>> {
private final transient DataflowRunner runner;
- private final GroupIntoBatches<K, V> original;
+ private final GroupIntoBatches<K, V>.WithShardedKey original;
- public StreamingGroupIntoBatches(DataflowRunner runner,
GroupIntoBatches<K, V> original) {
+ public StreamingGroupIntoBatchesWithShardedKey(
+ DataflowRunner runner, GroupIntoBatches<K, V>.WithShardedKey original)
{
this.runner = runner;
this.original = original;
}
@Override
- public PCollection<KV<K, Iterable<V>>> expand(PCollection<KV<K, V>> input)
{
- runner.maybeRecordPCollectionWithAutoSharding(input);
- return input.apply(original);
+ public PCollection<KV<ShardedKey<K>, Iterable<V>>>
expand(PCollection<KV<K, V>> input) {
+ PCollection<KV<ShardedKey<K>, V>> intermediate_input = ShardKeys(input);
+
+ runner.maybeRecordPCollectionWithAutoSharding(intermediate_input);
+
+ if (original.getMaxBufferingDuration() != null) {
Review comment:
We need to recognize the GroupIntoBatchesDoFn which is a private member
of the transform. Here we record the input pcoll of GroupIntoBatchesDoFn, which
would be the output of the key-sharding DoFn, so in the translation we can
append autosharding properties for those steps whose input was recorded. This
doesn't scale indeed. Any suggestions?
----------------------------------------------------------------
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]