Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
import java.util.Objects;
import org.apache.beam.runners.core.SystemReduceFn;
import org.apache.beam.runners.spark.SparkPipelineOptions;
import org.apache.beam.runners.spark.coders.CoderHelpers;
Expand Down Expand Up @@ -486,6 +487,9 @@ public void evaluate(
TranslationUtils.getTupleTagCoders(outputs);
all =
all.mapToPair(TranslationUtils.getTupleTagEncodeFunction(coderMap))
.filter(
Objects::nonNull) // skip nulls to save on encoding, nulls are tags that are
// not read
.persist(level)
.mapToPair(TranslationUtils.getTupleTagDecodeFunction(coderMap));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -445,8 +445,14 @@ public static Map<TupleTag<?>, Coder<WindowedValue<?>>> getTupleTagCoders(
return tuple2 -> {
TupleTag<?> tupleTag = tuple2._1;
WindowedValue<?> windowedValue = tuple2._2;
return new Tuple2<>(
tupleTag, ValueAndCoderLazySerializable.of(windowedValue, coderMap.get(tupleTag)));
Coder<WindowedValue<?>> coder = coderMap.get(tupleTag);
if (coder == null) {
// there is no coder as this output is unconsumed and is not read anywhere, so coder is
// pruned
// from coderMap
return null;
}
return new Tuple2<>(tupleTag, ValueAndCoderLazySerializable.of(windowedValue, coder));
};
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import org.apache.beam.runners.core.construction.SerializablePipelineOptions;
import org.apache.beam.runners.spark.coders.CoderHelpers;
import org.apache.beam.runners.spark.metrics.MetricsAccumulator;
Expand Down Expand Up @@ -234,6 +235,9 @@ public void evaluate(
TranslationUtils.getTupleTagCoders(outputs);
all =
all.mapToPair(TranslationUtils.getTupleTagEncodeFunction(coderMap))
.filter(
Objects
::nonNull) // skip nulls to save on encoding, nulls are tags that are not read
.cache()
.mapToPair(TranslationUtils.getTupleTagDecodeFunction(coderMap));

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Queue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.stream.Collectors;
Expand Down Expand Up @@ -593,6 +594,10 @@ public void evaluate(
TranslationUtils.getTupleTagCoders(outputs);
all =
all.mapToPair(TranslationUtils.getTupleTagEncodeFunction(coderMap))
.filter(
Objects
::nonNull) // skip nulls to save on encoding, nulls are tags that are not
// read
.cache()
.mapToPair(TranslationUtils.getTupleTagDecodeFunction(coderMap));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,14 +40,17 @@
import org.apache.beam.sdk.coders.VarIntCoder;
import org.apache.beam.sdk.transforms.Count;
import org.apache.beam.sdk.transforms.Create;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
import org.apache.beam.sdk.transforms.windowing.PaneInfo;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PBegin;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionTuple;
import org.apache.beam.sdk.values.TupleTag;
import org.apache.beam.sdk.values.TupleTagList;
import org.apache.beam.sdk.values.WindowedValue;
import org.apache.beam.sdk.values.WindowedValues;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables;
Expand Down Expand Up @@ -247,4 +250,65 @@ public void testMultipleOutputParDoShouldHaveFilterWhenSideOutputIsConsumed() {
assertTrue(parsed.stream().anyMatch(e -> e.getName().contains(tag.getId())));
}
}

@Test
public void testMultipleOutputParDoWithUnconsumedSideOutputAndSerializationStorageLevel() {
Pipeline p = Pipeline.create();
TupleTag<String> tag1 = new TupleTag<String>("tag1") {};
TupleTag<String> tag2 = new TupleTag<String>("tag2") {};
TupleTag<String> tag3 = new TupleTag<String>("tag3") {};

SparkPipelineOptions options = contextRule.createPipelineOptions();
// Force serialization by setting storage level to MEMORY_AND_DISK_SER
options.setStorageLevel("MEMORY_AND_DISK_SER");

TransformTranslator.Translator translator = new TransformTranslator.Translator();

PTransform<PBegin, PCollection<String>> createTransform = Create.of("foo", "bar");

PCollectionTuple pCollectionTuple =
p.apply("Create Values", createTransform)
.apply(
"Multiple Output ParDo",
ParDo.of(new MultiOutputDoFn(tag1, tag2, tag3))
.withOutputTags(tag1, TupleTagList.of(tag2).and(tag3)));

// consume tag1 and tag2
pCollectionTuple.get(tag1).apply("Count1", Count.globally());
pCollectionTuple.get(tag2).apply("Count2", Count.globally());

p.replaceAll(SparkTransformOverrides.getDefaultOverrides(false));

EvaluationContext ctxt = new EvaluationContext(contextRule.getSparkContext(), p, options);
SparkRunner.initAccumulators(options, ctxt.getSparkContext());
SparkRunner.updateDependentTransforms(p, translator, ctxt);

// This should not throw NullPointerException
p.traverseTopologically(new SparkRunner.Evaluator(translator, ctxt));

// Also trigger some action on the RDD to ensure serialization happens
@SuppressWarnings("unchecked")
BoundedDataset<String> dataset =
(BoundedDataset<String>) ctxt.borrowDataset(pCollectionTuple.get(tag1));
dataset.getRDD().count();
}

private static class MultiOutputDoFn extends DoFn<String, String> {
private final TupleTag<String> tag1;
private final TupleTag<String> tag2;
private final TupleTag<String> tag3;

MultiOutputDoFn(TupleTag<String> tag1, TupleTag<String> tag2, TupleTag<String> tag3) {
this.tag1 = tag1;
this.tag2 = tag2;
this.tag3 = tag3;
}

@ProcessElement
public void process(@Element String input, MultiOutputReceiver outputReceiver) {
outputReceiver.get(tag1).output(input);
outputReceiver.get(tag2).output(input);
outputReceiver.get(tag3).output(input);
}
}
}
Loading