Skip to content

Commit

Permalink
[FLINK-10566] Fix exponential planning time of large programs
Browse files Browse the repository at this point in the history
The traversal of the DAG is not efficient enough at some places which can lead
to very long plan creation times.

This introduces caching for the traversal to avoid traversing nodes multiple
times. Caching is performed at two places:

- when registering Kryo types
- when determining the maximum parallelism
  • Loading branch information
mxm committed Dec 14, 2018
1 parent bc4194a commit 2870c7a
Show file tree
Hide file tree
Showing 3 changed files with 94 additions and 3 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map.Entry;
import java.util.Set;
Expand Down Expand Up @@ -361,10 +362,14 @@ public int getMaximumParallelism() {

private static final class MaxDopVisitor implements Visitor<Operator<?>> {

private final Set<Operator> visitedOperators = new HashSet<>();
private int maxDop = -1;

@Override
public boolean preVisit(Operator<?> visitable) {
if (!visitedOperators.add(visitable)) {
return false;
}
this.maxDop = Math.max(this.maxDop, visitable.getParallelism());
return true;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,7 @@
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Set;

import static org.apache.flink.util.Preconditions.checkNotNull;

Expand Down Expand Up @@ -963,12 +964,16 @@ public Plan createProgramPlan(String jobName, boolean clearSinks) {
if (!config.isAutoTypeRegistrationDisabled()) {
plan.accept(new Visitor<org.apache.flink.api.common.operators.Operator<?>>() {

private final HashSet<Class<?>> deduplicator = new HashSet<>();
private final Set<Class<?>> registeredTypes = new HashSet<>();
private final Set<org.apache.flink.api.common.operators.Operator<?>> visitedOperators = new HashSet<>();

@Override
public boolean preVisit(org.apache.flink.api.common.operators.Operator<?> visitable) {
if (!visitedOperators.add(visitable)) {
return false;
}
OperatorInformation<?> opInfo = visitable.getOperatorInfo();
Serializers.recursivelyRegisterType(opInfo.getOutputType(), config, deduplicator);
Serializers.recursivelyRegisterType(opInfo.getOutputType(), config, registeredTypes);
return true;
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,81 @@
/*
* 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:https://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.flink.test.planning;

import org.apache.flink.api.common.functions.RichMapFunction;
import org.apache.flink.api.common.typeinfo.Types;
import org.apache.flink.api.java.DataSet;
import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.io.DiscardingOutputFormat;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.client.program.OptimizerPlanEnvironment;
import org.apache.flink.client.program.PreviewPlanEnvironment;

import org.junit.Test;

/**
* Tests that large programs can be compiled to a Plan in reasonable amount of time.
*/
public class LargePlanTest {

@Test(expected = OptimizerPlanEnvironment.ProgramAbortException.class, timeout = 15_000)
public void testPlanningOfLargePlan() throws Exception {
runProgram(new PreviewPlanEnvironment(), 10, 50);
}

private static void runProgram(ExecutionEnvironment env, int depth, int width) throws Exception {
DataSet<String> input = env.fromElements("a", "b", "c");
DataSet<String> stats = null;

for (int i = 0; i < depth; i++) {
stats = analyze(input, stats, width / (i + 1) + 1);
}

stats.output(new DiscardingOutputFormat<>());
env.execute("depth " + depth + " width " + width);
}

private static DataSet<String> analyze(DataSet<String> input, DataSet<String> stats, int branches) {
for (int i = 0; i < branches; i++) {
final int ii = i;

if (stats != null) {
input = input.map(
new RichMapFunction<String, String>() {
@Override
public String map(String value) {
return value;
}
}).withBroadcastSet(stats.map(s -> "(" + s + ").map"), "stats");
}

DataSet<String> branch = input
.map(s -> new Tuple2<>(0, s + ii)).returns(Types.TUPLE(Types.STRING, Types.INT))
.groupBy(0)
.minBy(1)
.map(kv -> kv.f1).returns(Types.STRING);
if (stats == null) {
stats = branch;
} else {
stats = stats.union(branch);
}
}
return stats.map(s -> "(" + s + ").stats");
}
}

0 comments on commit 2870c7a

Please sign in to comment.