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] Skip unused outputs of ParDo in SparkRunner (#33771) #33772

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
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
@@ -0,0 +1,52 @@
/*
* 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.spark;

import java.util.Map;
import org.apache.beam.runners.spark.translation.EvaluationContext;
import org.apache.beam.runners.spark.translation.SparkPipelineTranslator;
import org.apache.beam.sdk.runners.TransformHierarchy;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.TupleTag;

/**
* Traverses the pipeline to populate information on how many {@link
* org.apache.beam.sdk.transforms.PTransform}s do consume / depends on each {@link PCollection} in
* the pipeline.
*/
class DependentTransformsVisitor extends SparkRunner.Evaluator {

DependentTransformsVisitor(
SparkPipelineTranslator translator, EvaluationContext evaluationContext) {
super(translator, evaluationContext);
}

@Override
public void doVisitTransform(TransformHierarchy.Node node) {

Map<PCollection<?>, Integer> dependentTransforms = ctxt.getDependentTransforms();
for (Map.Entry<TupleTag<?>, PCollection<?>> entry : node.getInputs().entrySet()) {
int dependants = dependentTransforms.getOrDefault(entry.getValue(), 0);
dependentTransforms.put(entry.getValue(), dependants + 1);
}

for (PCollection<?> pOut : node.getOutputs().values()) {
dependentTransforms.computeIfAbsent(pOut, k -> 0);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -214,6 +214,7 @@ public SparkPipelineResult run(final Pipeline pipeline) {

// update the cache candidates
updateCacheCandidates(pipeline, translator, evaluationContext);
updateDependentTransforms(pipeline, translator, evaluationContext);

// update GBK candidates for memory optimized transform
pipeline.traverseTopologically(new GroupByKeyVisitor(translator, evaluationContext));
Expand Down Expand Up @@ -275,8 +276,13 @@ static void detectTranslationMode(Pipeline pipeline, SparkPipelineOptions pipeli
/** Evaluator that update/populate the cache candidates. */
public static void updateCacheCandidates(
Pipeline pipeline, SparkPipelineTranslator translator, EvaluationContext evaluationContext) {
CacheVisitor cacheVisitor = new CacheVisitor(translator, evaluationContext);
pipeline.traverseTopologically(cacheVisitor);
pipeline.traverseTopologically(new CacheVisitor(translator, evaluationContext));
}

/** Evaluator that update/populate information about dependent transforms for pCollections. */
public static void updateDependentTransforms(
Pipeline pipeline, SparkPipelineTranslator translator, EvaluationContext evaluationContext) {
pipeline.traverseTopologically(new DependentTransformsVisitor(translator, evaluationContext));
}

/** The translation mode of the Beam Pipeline. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@ public class EvaluationContext {
private final Map<PValue, Dataset> datasets = new LinkedHashMap<>();
private final Map<PValue, Dataset> pcollections = new LinkedHashMap<>();
private final Set<Dataset> leaves = new LinkedHashSet<>();
private final Map<PCollection<?>, Integer> dependentTransforms = new HashMap<>();
private final Map<PValue, Object> pobjects = new LinkedHashMap<>();
private AppliedPTransform<?, ?, ?> currentTransform;
private final SparkPCollectionView pviews = new SparkPCollectionView();
Expand Down Expand Up @@ -307,6 +308,26 @@ public <K, V> boolean isCandidateForGroupByKeyAndWindow(GroupByKey<K, V> transfo
return groupByKeyCandidatesForMemoryOptimizedTranslation.containsKey(transform);
}

/**
* Get the map of dependent transforms hold by the evaluation context.
*
* @return The current {@link Map} of dependent transforms.
*/
public Map<PCollection<?>, Integer> getDependentTransforms() {
return this.dependentTransforms;
}

/**
* Get if given {@link PCollection} is a leaf or not. {@link PCollection} is a leaf when there is
* no other {@link PTransform} consuming it / depending on it.
*
* @param pCollection to be checked if it is a leaf
* @return true if pCollection is leaf; otherwise false
*/
public boolean isLeaf(PCollection<?> pCollection) {
return this.dependentTransforms.get(pCollection) == 0;
}

<T> Iterable<WindowedValue<T>> getWindowedValues(PCollection<T> pcollection) {
@SuppressWarnings("unchecked")
BoundedDataset<T> boundedDataset = (BoundedDataset<T>) datasets.get(pcollection);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.beam.runners.spark.translation;

import static org.apache.beam.runners.spark.translation.TranslationUtils.canAvoidRddSerialization;
import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;
import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState;

import java.util.Arrays;
Expand Down Expand Up @@ -70,13 +71,15 @@
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionView;
import org.apache.beam.sdk.values.TupleTag;
import org.apache.beam.sdk.values.TupleTagList;
import org.apache.beam.sdk.values.WindowingStrategy;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.AbstractIterator;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.FluentIterable;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps;
import org.apache.spark.HashPartitioner;
import org.apache.spark.Partitioner;
import org.apache.spark.api.java.JavaPairRDD;
Expand Down Expand Up @@ -428,13 +431,14 @@ public void evaluate(
Map<String, PCollectionView<?>> sideInputMapping =
ParDoTranslation.getSideInputMapping(context.getCurrentTransform());

TupleTag<OutputT> mainOutputTag = transform.getMainOutputTag();
MultiDoFnFunction<InputT, OutputT> multiDoFnFunction =
new MultiDoFnFunction<>(
metricsAccum,
stepName,
doFn,
context.getSerializableOptions(),
transform.getMainOutputTag(),
mainOutputTag,
transform.getAdditionalOutputTags().getAll(),
inputCoder,
outputCoders,
Expand All @@ -460,7 +464,13 @@ public void evaluate(
all = inRDD.mapPartitionsToPair(multiDoFnFunction);
}

Map<TupleTag<?>, PCollection<?>> outputs = context.getOutputs(transform);
// Filter out obsolete PCollections to only cache when absolutely necessary
Map<TupleTag<?>, PCollection<?>> outputs =
skipObsoleteOutputs(
context.getOutputs(transform),
mainOutputTag,
transform.getAdditionalOutputTags(),
context);
if (hasMultipleOutputs(outputs)) {
StorageLevel level = StorageLevel.fromString(context.storageLevel());
if (canAvoidRddSerialization(level)) {
Expand Down Expand Up @@ -498,6 +508,37 @@ private boolean hasMultipleOutputs(Map<TupleTag<?>, PCollection<?>> outputs) {
return outputs.size() > 1;
}

/**
* Filter out obsolete, unused output tags except for {@code mainTag}.
*
* <p>This can help to avoid unnecessary caching in case of multiple outputs if only {@code
* mainTag} is consumed.
*/
private Map<TupleTag<?>, PCollection<?>> skipObsoleteOutputs(
Map<TupleTag<?>, PCollection<?>> outputs,
TupleTag<?> mainTag,
TupleTagList otherTags,
EvaluationContext cxt) {
switch (outputs.size()) {
case 1:
return outputs; // always keep main output
case 2:
TupleTag<?> otherTag = otherTags.get(0);
return cxt.isLeaf(checkStateNotNull(outputs.get(otherTag)))
? Collections.singletonMap(mainTag, checkStateNotNull(outputs.get(mainTag)))
: outputs;
default:
Map<TupleTag<?>, PCollection<?>> filtered =
Maps.newHashMapWithExpectedSize(outputs.size());
for (Map.Entry<TupleTag<?>, PCollection<?>> e : outputs.entrySet()) {
if (e.getKey().equals(mainTag) || !cxt.isLeaf(e.getValue())) {
filtered.put(e.getKey(), e.getValue());
}
}
return filtered;
}
}

@Override
public String toNativeString() {
return "mapPartitions(new <fn>())";
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,141 @@
/*
* 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.spark;

import static org.junit.Assert.assertEquals;

import java.util.List;
import java.util.Objects;
import org.apache.beam.runners.spark.translation.EvaluationContext;
import org.apache.beam.runners.spark.translation.TransformTranslator;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.coders.VarLongCoder;
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.ParDo;
import org.apache.beam.sdk.transforms.Sum;
import org.apache.beam.sdk.transforms.View;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionTuple;
import org.apache.beam.sdk.values.PCollectionView;
import org.apache.beam.sdk.values.TupleTag;
import org.apache.beam.sdk.values.TupleTagList;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists;
import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;

/** Tests of {@link DependentTransformsVisitor}. */
public class DependentTransformsVisitorTest {

@ClassRule public static SparkContextRule contextRule = new SparkContextRule();

@Rule public TemporaryFolder tmpFolder = new TemporaryFolder();

@Test
public void testCountDependentTransformsOnApplyAndSideInputs() {
SparkPipelineOptions options = contextRule.createPipelineOptions();
Pipeline pipeline = Pipeline.create(options);
PCollection<String> pCollection = pipeline.apply(Create.of("foo", "bar"));

// First use of pCollection.
PCollection<Long> leaf1 = pCollection.apply(Count.globally());
// Second use of pCollection.
PCollectionView<List<String>> view = pCollection.apply("yyy", View.asList());

PCollection<String> leaf2 =
pipeline
.apply(Create.of("foo", "baz"))
.apply(
ParDo.of(
new DoFn<String, String>() {
@ProcessElement
public void processElement(ProcessContext processContext) {
if (processContext.sideInput(view).contains(processContext.element())) {
processContext.output(processContext.element());
}
}
})
.withSideInputs(view));

EvaluationContext ctxt =
new EvaluationContext(contextRule.getSparkContext(), pipeline, options);
TransformTranslator.Translator translator = new TransformTranslator.Translator();
pipeline.traverseTopologically(new DependentTransformsVisitor(translator, ctxt));

assertEquals(2, ctxt.getDependentTransforms().get(pCollection).intValue());
assertEquals(0, ctxt.getDependentTransforms().get(leaf1).intValue());
assertEquals(0, ctxt.getDependentTransforms().get(leaf2).intValue());
assertEquals(2, ctxt.getDependentTransforms().get(view.getPCollection()).intValue());
}

@Test
public void testCountDependentTransformsOnSideOutputs() {
SparkPipelineOptions options = contextRule.createPipelineOptions();
Pipeline pipeline = Pipeline.create(options);

TupleTag<String> passOutTag = new TupleTag<>("passOut");
TupleTag<Long> lettersCountOutTag = new TupleTag<>("lettersOut");
TupleTag<Long> wordCountOutTag = new TupleTag<>("wordsOut");

PCollectionTuple result =
pipeline
.apply(Create.of("foo", "baz"))
.apply(
ParDo.of(
new DoFn<String, String>() {
@ProcessElement
public void processElement(ProcessContext processContext) {
String element = processContext.element();
processContext.output(element);
processContext.output(
lettersCountOutTag,
(long) Objects.requireNonNull(element).length());
processContext.output(wordCountOutTag, 1L);
}
})
.withOutputTags(
passOutTag,
TupleTagList.of(Lists.newArrayList(lettersCountOutTag, wordCountOutTag))));

// consume main output and words side output. leave letters side output left alone
result.get(wordCountOutTag).setCoder(VarLongCoder.of()).apply(Sum.longsGlobally());
result.get(lettersCountOutTag).setCoder(VarLongCoder.of());
result
.get(passOutTag)
.apply(
ParDo.of(
new DoFn<String, String>() {
@ProcessElement
public void processElement(ProcessContext processContext) {
// do nothing
}
}));

EvaluationContext ctxt =
new EvaluationContext(contextRule.getSparkContext(), pipeline, options);
TransformTranslator.Translator translator = new TransformTranslator.Translator();
pipeline.traverseTopologically(new DependentTransformsVisitor(translator, ctxt));

assertEquals(1, ctxt.getDependentTransforms().get(result.get(passOutTag)).intValue());
assertEquals(1, ctxt.getDependentTransforms().get(result.get(wordCountOutTag)).intValue());
assertEquals(0, ctxt.getDependentTransforms().get(result.get(lettersCountOutTag)).intValue());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ public static List<RDDNode> parse(String debugString) {

for (String line : lines) {
line = line.trim();
if (line.isEmpty()) {
if (line.isEmpty() || isStatsLine(line)) {
continue;
}

Expand All @@ -48,6 +48,10 @@ public static List<RDDNode> parse(String debugString) {
return list;
}

private static boolean isStatsLine(String line) {
return line.contains("MemorySize:") && line.contains("DiskSize:");
}

private static int extractId(String line) {
String idPart = line.substring(line.indexOf('[') + 1, line.indexOf(']'));
return Integer.parseInt(idPart);
Expand Down
Loading
Loading