fhueske commented on a change in pull request #7276: [FLINK-10566] Fix exponential planning time of large programs URL: https://github.com/apache/flink/pull/7276#discussion_r240855467
########## File path: flink-tests/src/test/java/org/apache/flink/test/planning/LargePlanTest.java ########## @@ -0,0 +1,92 @@ +/* + * 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.flink.test.planning; + +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +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.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.client.program.OptimizerPlanEnvironment; +import org.apache.flink.client.program.PreviewPlanEnvironment; +import org.apache.flink.configuration.Configuration; + +import org.junit.Test; + +import java.util.Collection; + +/** + * 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 void open(Configuration parameters) throws Exception { + Collection<String> broadcastSet = getRuntimeContext().getBroadcastVariable("stats"); + } + + @Override + public String map(String value) throws Exception { + return value; + } + }).withBroadcastSet(stats.map(s -> "(" + s + ").map"), "stats"); + } + + TupleTypeInfo<Tuple2<Integer, String>> typeInfo = + new TupleTypeInfo<>(TypeInformation.of(Integer.class), TypeInformation.of(String.class)); + DataSet<String> branch = input + .map(s -> new Tuple2<>(0, s + ii)).returns(typeInfo) + .groupBy(0) + .minBy(1) + .map(kv -> kv.f1).returns(TypeInformation.of(String.class)); Review comment: `returns(Types.STRING)` ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services