Skip to content

Commit

Permalink
Plumb Redistribute "allow duplicates" property to Dataflow (#31490)
Browse files Browse the repository at this point in the history
 - Since Dataflow only translates primitives, added a runner-specific DataflowGroupByKey primitive
 - Added override for RedistributeByKey that replaces GBK with Dataflow GBK, to plumb property
  • Loading branch information
celeste-zeng authored Jun 17, 2024
1 parent 403ad56 commit 3a53181
Show file tree
Hide file tree
Showing 8 changed files with 359 additions and 25 deletions.
Original file line number Diff line number Diff line change
@@ -1,5 +1,6 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run",
"https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test",
"https://github.com/apache/beam/pull/31268": "noting that PR #31268 should run this test"
"https://github.com/apache/beam/pull/31268": "noting that PR #31268 should run this test",
"https://github.com/apache/beam/pull/31490": "noting that PR #31490 should run this test"
}
Original file line number Diff line number Diff line change
@@ -1,5 +1,6 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run",
"https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test",
"https://github.com/apache/beam/pull/31268": "noting that PR #31268 should run this test"
"https://github.com/apache/beam/pull/31268": "noting that PR #31268 should run this test",
"https://github.com/apache/beam/pull/31490": "noting that PR #31490 should run this test"
}
Original file line number Diff line number Diff line change
@@ -1,5 +1,6 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run",
"https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test",
"https://github.com/apache/beam/pull/31268": "noting that PR #31268 should run this test"
"https://github.com/apache/beam/pull/31268": "noting that PR #31268 should run this test",
"https://github.com/apache/beam/pull/31490": "noting that PR #31490 should run this test"
}
Original file line number Diff line number Diff line change
@@ -1,5 +1,6 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run",
"https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test",
"https://github.com/apache/beam/pull/31268": "noting that PR #31268 should run this test"
"https://github.com/apache/beam/pull/31268": "noting that PR #31268 should run this test",
"https://github.com/apache/beam/pull/31490": "noting that PR #31490 should run this test"
}
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,7 @@
import org.apache.beam.runners.dataflow.PrimitiveParDoSingleFactory.ParDoSingle;
import org.apache.beam.runners.dataflow.TransformTranslator.StepTranslationContext;
import org.apache.beam.runners.dataflow.TransformTranslator.TranslationContext;
import org.apache.beam.runners.dataflow.internal.DataflowGroupByKey;
import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
import org.apache.beam.runners.dataflow.util.CloudObject;
import org.apache.beam.runners.dataflow.util.CloudObjects;
Expand All @@ -80,15 +81,13 @@
import org.apache.beam.sdk.transforms.Materializations;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.transforms.Redistribute.RedistributeByKey;
import org.apache.beam.sdk.transforms.View;
import org.apache.beam.sdk.transforms.display.DisplayData;
import org.apache.beam.sdk.transforms.display.HasDisplayData;
import org.apache.beam.sdk.transforms.reflect.DoFnSignatures;
import org.apache.beam.sdk.transforms.resourcehints.ResourceHint;
import org.apache.beam.sdk.transforms.resourcehints.ResourceHints;
import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
import org.apache.beam.sdk.transforms.windowing.ReshuffleTrigger;
import org.apache.beam.sdk.transforms.windowing.Window;
import org.apache.beam.sdk.util.AppliedCombineFn;
import org.apache.beam.sdk.util.CoderUtils;
Expand Down Expand Up @@ -920,37 +919,30 @@ private <K1, K2, V> void groupByKeyAndSortValuesHelper(
});

registerTransformTranslator(
RedistributeByKey.class,
new TransformTranslator<RedistributeByKey>() {
DataflowGroupByKey.class,
new TransformTranslator<DataflowGroupByKey>() {
@Override
public void translate(RedistributeByKey transform, TranslationContext context) {
redistributeByKeyHelper(transform, context);
public void translate(DataflowGroupByKey transform, TranslationContext context) {
dataflowGroupByKeyHelper(transform, context);
}

private <K, V> void redistributeByKeyHelper(
RedistributeByKey<K, V> transform, TranslationContext context) {
private <K, V> void dataflowGroupByKeyHelper(
DataflowGroupByKey<K, V> transform, TranslationContext context) {
StepTranslationContext stepContext = context.addStep(transform, "GroupByKey");

PCollection<KV<K, V>> input = context.getInput(transform);
stepContext.addInput(PropertyNames.PARALLEL_INPUT, input);
stepContext.addOutput(PropertyNames.OUTPUT, context.getOutput(transform));

// Dataflow worker implements reshuffle by reading GBK with ReshuffleTrigger; that is
// the only part of
// the windowing strategy that should be observed.
WindowingStrategy<?, ?> windowingStrategy =
input.getWindowingStrategy().withTrigger(new ReshuffleTrigger<>());
WindowingStrategy<?, ?> windowingStrategy = input.getWindowingStrategy();
stepContext.addInput(PropertyNames.DISALLOW_COMBINER_LIFTING, true);
stepContext.addInput(
PropertyNames.SERIALIZED_FN,
byteArrayToJsonString(
serializeWindowingStrategy(windowingStrategy, context.getPipelineOptions())));

// Many group by key options do not apply to redistribute but Dataflow doesn't
// understand
// that. We set them here to be sure to avoid any complex codepaths
stepContext.addInput(PropertyNames.DISALLOW_COMBINER_LIFTING, true);
stepContext.addInput(PropertyNames.IS_MERGING_WINDOW_FN, false);
stepContext.addInput(PropertyNames.ALLOW_DUPLICATES, transform.getAllowDuplicates());
stepContext.addInput(
PropertyNames.IS_MERGING_WINDOW_FN,
!windowingStrategy.getWindowFn().isNonMerging());
stepContext.addInput(PropertyNames.ALLOW_DUPLICATES, transform.allowDuplicates());
}
});

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -119,6 +119,7 @@
import org.apache.beam.sdk.transforms.Impulse;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.transforms.Redistribute.RedistributeByKey;
import org.apache.beam.sdk.transforms.Reshuffle;
import org.apache.beam.sdk.transforms.SimpleFunction;
import org.apache.beam.sdk.transforms.View;
Expand Down Expand Up @@ -698,6 +699,11 @@ private List<PTransformOverride> getOverrides(boolean streaming) {
PTransformMatchers.classEqualTo(ParDo.SingleOutput.class),
new PrimitiveParDoSingleFactory()));

overridesBuilder.add(
PTransformOverride.of(
PTransformMatchers.classEqualTo(RedistributeByKey.class),
new RedistributeByKeyOverrideFactory()));

if (streaming) {
// For update compatibility, always use a Read for Create in streaming mode.
overridesBuilder
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,138 @@
/*
* 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.runners.dataflow;

import java.util.Collections;
import org.apache.beam.runners.dataflow.internal.DataflowGroupByKey;
import org.apache.beam.sdk.runners.AppliedPTransform;
import org.apache.beam.sdk.runners.PTransformOverrideFactory;
import org.apache.beam.sdk.runners.PTransformOverrideFactory.PTransformReplacement;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.DoFn.Element;
import org.apache.beam.sdk.transforms.DoFn.OutputReceiver;
import org.apache.beam.sdk.transforms.DoFn.ProcessElement;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.transforms.Redistribute.RedistributeByKey;
import org.apache.beam.sdk.transforms.Reify;
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.apache.beam.sdk.transforms.windowing.ReshuffleTrigger;
import org.apache.beam.sdk.transforms.windowing.TimestampCombiner;
import org.apache.beam.sdk.transforms.windowing.Window;
import org.apache.beam.sdk.util.IdentityWindowFn;
import org.apache.beam.sdk.util.construction.PTransformReplacements;
import org.apache.beam.sdk.util.construction.SingleInputOutputOverrideFactory;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.ValueInSingleWindow;
import org.apache.beam.sdk.values.WindowingStrategy;
import org.joda.time.Duration;

class RedistributeByKeyOverrideFactory<K, V>
extends SingleInputOutputOverrideFactory<
PCollection<KV<K, V>>, PCollection<KV<K, V>>, RedistributeByKey<K, V>> {

@Override
public PTransformReplacement<PCollection<KV<K, V>>, PCollection<KV<K, V>>>
getReplacementTransform(
AppliedPTransform<PCollection<KV<K, V>>, PCollection<KV<K, V>>, RedistributeByKey<K, V>>
transform) {
return PTransformOverrideFactory.PTransformReplacement.of(
PTransformReplacements.getSingletonMainInput(transform),
new DataflowRedistributeByKey<>(transform.getTransform()));
}

/** Specialized implementation of {@link RedistributeByKey} for Dataflow pipelines. */
private static class DataflowRedistributeByKey<K, V>
extends PTransform<PCollection<KV<K, V>>, PCollection<KV<K, V>>> {

private final RedistributeByKey<K, V> originalTransform;

private DataflowRedistributeByKey(RedistributeByKey<K, V> originalTransform) {
this.originalTransform = originalTransform;
}

@Override
public PCollection<KV<K, V>> expand(PCollection<KV<K, V>> input) {
WindowingStrategy<?, ?> originalStrategy = input.getWindowingStrategy();
Window<KV<K, V>> rewindow =
Window.<KV<K, V>>into(
new IdentityWindowFn<>(originalStrategy.getWindowFn().windowCoder()))
.triggering(new ReshuffleTrigger<>())
.discardingFiredPanes()
.withTimestampCombiner(TimestampCombiner.EARLIEST)
.withAllowedLateness(Duration.millis(BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()));

PCollection<KV<K, ValueInSingleWindow<V>>> reified =
input
.apply("SetIdentityWindow", rewindow)
.apply("ReifyOriginalMetadata", Reify.windowsInValue());

PCollection<KV<K, Iterable<ValueInSingleWindow<V>>>> grouped;
if (originalTransform.getAllowDuplicates()) {
grouped = reified.apply(DataflowGroupByKey.createWithAllowDuplicates());
} else {
grouped = reified.apply(DataflowGroupByKey.create());
}

return grouped
.apply(
"ExpandIterable",
ParDo.of(
new DoFn<
KV<K, Iterable<ValueInSingleWindow<V>>>, KV<K, ValueInSingleWindow<V>>>() {
@ProcessElement
public void processElement(
@Element KV<K, Iterable<ValueInSingleWindow<V>>> element,
OutputReceiver<KV<K, ValueInSingleWindow<V>>> r) {
K key = element.getKey();
for (ValueInSingleWindow<V> value : element.getValue()) {
r.output(KV.of(key, value));
}
}
}))
.apply("RestoreMetadata", new RestoreMetadata<>())
.setWindowingStrategyInternal(originalStrategy);
}
}

private static class RestoreMetadata<K, V>
extends PTransform<PCollection<KV<K, ValueInSingleWindow<V>>>, PCollection<KV<K, V>>> {
@Override
public PCollection<KV<K, V>> expand(PCollection<KV<K, ValueInSingleWindow<V>>> input) {
return input.apply(
ParDo.of(
new DoFn<KV<K, ValueInSingleWindow<V>>, KV<K, V>>() {
@Override
public Duration getAllowedTimestampSkew() {
return Duration.millis(Long.MAX_VALUE);
}

@ProcessElement
public void processElement(
@Element KV<K, ValueInSingleWindow<V>> kv, OutputReceiver<KV<K, V>> r) {
r.outputWindowedValue(
KV.of(kv.getKey(), kv.getValue().getValue()),
kv.getValue().getTimestamp(),
Collections.singleton(kv.getValue().getWindow()),
kv.getValue().getPane());
}
}));
}
}
}
Loading

0 comments on commit 3a53181

Please sign in to comment.