Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[Spark Dataset runner] Skip unconsumed additional outputs of ParDo.MultiOutput to avoid caching if not necessary #24711

Merged
merged 2 commits into from
Dec 28, 2022
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 @@ -152,6 +152,8 @@ public String name() {
public interface TranslationState extends EncoderProvider {
<T> Dataset<WindowedValue<T>> getDataset(PCollection<T> pCollection);

boolean isLeave(PCollection<?> pCollection);

<T> void putDataset(
PCollection<T> pCollection, Dataset<WindowedValue<T>> dataset, boolean cache);

Expand Down Expand Up @@ -256,6 +258,11 @@ public <T> void putDataset(
}
}

@Override
public boolean isLeave(PCollection<?> pCollection) {
return getResult(pCollection).dependentTransforms.isEmpty();
}

@Override
public <T> Broadcast<SideInputValues<T>> getSideInputBroadcast(
PCollection<T> pCollection, SideInputValues.Loader<T> loader) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -149,6 +149,11 @@ public <T> void putDataset(
state.putDataset(pCollection, dataset, cache);
}

@Override
public boolean isLeave(PCollection<?> pCollection) {
return state.isLeave(pCollection);
}

@Override
public Supplier<PipelineOptions> getOptionsSupplier() {
return state.getOptionsSupplier();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@

import static org.apache.beam.runners.spark.structuredstreaming.translation.utils.ScalaInterop.scalaIterator;
import static org.apache.beam.runners.spark.structuredstreaming.translation.utils.ScalaInterop.tuple;
import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;

import java.io.Serializable;
import java.util.ArrayDeque;
Expand Down Expand Up @@ -61,17 +60,17 @@
*/
abstract class DoFnPartitionIteratorFactory<InT, FnOutT, OutT extends @NonNull Object>
implements Function1<Iterator<WindowedValue<InT>>, Iterator<OutT>>, Serializable {
private final String stepName;
private final DoFn<InT, FnOutT> doFn;
private final DoFnSchemaInformation doFnSchema;
private final Supplier<PipelineOptions> options;
private final Coder<InT> coder;
private final WindowingStrategy<?, ?> windowingStrategy;
private final TupleTag<FnOutT> mainOutput;
private final List<TupleTag<?>> additionalOutputs;
private final Map<TupleTag<?>, Coder<?>> outputCoders;
private final Map<String, PCollectionView<?>> sideInputs;
private final SideInputReader sideInputReader;
protected final String stepName;
protected final DoFn<InT, FnOutT> doFn;
protected final DoFnSchemaInformation doFnSchema;
protected final Supplier<PipelineOptions> options;
protected final Coder<InT> coder;
protected final WindowingStrategy<?, ?> windowingStrategy;
protected final TupleTag<FnOutT> mainOutput;
protected final List<TupleTag<?>> additionalOutputs;
protected final Map<TupleTag<?>, Coder<?>> outputCoders;
protected final Map<String, PCollectionView<?>> sideInputs;
protected final SideInputReader sideInputReader;

private DoFnPartitionIteratorFactory(
AppliedPTransform<PCollection<? extends InT>, ?, MultiOutput<InT, FnOutT>> appliedPT,
Expand Down Expand Up @@ -147,7 +146,9 @@ DoFnRunners.OutputManager outputManager(Deque<WindowedValue<OutT>> buffer) {
return new DoFnRunners.OutputManager() {
@Override
public <T> void output(TupleTag<T> tag, WindowedValue<T> output) {
buffer.add((WindowedValue<OutT>) output);
if (mainOutput.equals(tag)) {
mosche marked this conversation as resolved.
Show resolved Hide resolved
buffer.add((WindowedValue<OutT>) output);
}
}
};
}
Expand Down Expand Up @@ -177,8 +178,10 @@ DoFnRunners.OutputManager outputManager(Deque<Tuple2<Integer, WindowedValue<OutT
return new DoFnRunners.OutputManager() {
@Override
public <T> void output(TupleTag<T> tag, WindowedValue<T> output) {
Integer columnIdx = checkStateNotNull(tagColIdx.get(tag.getId()), "Unknown tag %s", tag);
buffer.add(tuple(columnIdx, (WindowedValue<OutT>) output));
Integer columnIdx = tagColIdx.get(tag.getId());
if (columnIdx != null) {
buffer.add(tuple(columnIdx, (WindowedValue<OutT>) output));
}
}
};
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -110,12 +110,19 @@ public void translate(ParDo.MultiOutput<InputT, OutputT> transform, Context cxt)
throws IOException {

PCollection<InputT> input = (PCollection<InputT>) cxt.getInput();
Map<TupleTag<?>, PCollection<?>> outputs = cxt.getOutputs();

Dataset<WindowedValue<InputT>> inputDs = cxt.getDataset(input);
SideInputReader sideInputReader =
createSideInputReader(transform.getSideInputs().values(), cxt);

TupleTag<OutputT> mainOut = transform.getMainOutputTag();
// Filter out unconsumed PCollections (except mainOut) to potentially avoid the costs of caching
// if not really beneficial.
Map<TupleTag<?>, PCollection<?>> outputs =
Maps.filterEntries(
cxt.getOutputs(),
e -> e != null && (e.getKey().equals(mainOut) || !cxt.isLeave(e.getValue())));

if (outputs.size() > 1) {
// In case of multiple outputs / tags, map each tag to a column by index.
// At the end split the result into multiple datasets selecting one column each.
Expand Down Expand Up @@ -176,7 +183,7 @@ public void translate(ParDo.MultiOutput<InputT, OutputT> transform, Context cxt)
}
}
} else {
PCollection<OutputT> output = cxt.getOutput(transform.getMainOutputTag());
PCollection<OutputT> output = cxt.getOutput(mainOut);
DoFnPartitionIteratorFactory<InputT, ?, WindowedValue<OutputT>> doFnMapper =
DoFnPartitionIteratorFactory.singleOutput(
cxt.getCurrentTransform(), cxt.getOptionsSupplier(), input, sideInputReader);
Expand Down