[ 
https://issues.apache.org/jira/browse/FLINK-10566?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16718336#comment-16718336
 ] 

ASF GitHub Bot commented on FLINK-10566:
----------------------------------------

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_r240855139
 
 

 ##########
 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)
 
 Review comment:
   can be simplified to `.returns(Types.TUPLE(Types.STRING, Types.INT))`

----------------------------------------------------------------
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


> Flink Planning is exponential in the number of stages
> -----------------------------------------------------
>
>                 Key: FLINK-10566
>                 URL: https://issues.apache.org/jira/browse/FLINK-10566
>             Project: Flink
>          Issue Type: Bug
>          Components: Optimizer
>    Affects Versions: 1.5.4, 1.6.1, 1.7.0
>            Reporter: Robert Bradshaw
>            Assignee: Maximilian Michels
>            Priority: Major
>              Labels: pull-request-available
>         Attachments: chart.png
>
>
> This makes it nearly impossible to run graphs with 100 or more stages. (The 
> execution itself is still sub-second, but the job submission takes 
> increasingly long.)
> I can reproduce this with the following pipeline, which resembles my 
> real-world workloads (with depth up to 10 and width up, and past, 50). On 
> Flink it seems getting width beyond width 10 is problematic (times out after 
> hours). Note the log scale on the chart for time. 
>  
> {code:java}
>   public static void runPipeline(int depth, int width) throws Exception {
>     final ExecutionEnvironment env = 
> ExecutionEnvironment.getExecutionEnvironment();
>     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.writeAsText("out.txt");
>     env.execute("depth " + depth + " width " + width);
>   }
>   public static DataSet<String> analyze(DataSet<String> input, 
> DataSet<String> stats, int branches) {
>     System.out.println("analyze " + 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");
>       }
>       DataSet<String> branch = input
>                                .map(s -> new Tuple2<Integer, String>(0, s + 
> ii))
>                                .groupBy(0)
>                                .minBy(1)
>                                .map(kv -> kv.f1);
>       if (stats == null) {
>         stats = branch;
>       } else {
>         stats = stats.union(branch);
>       }
>     }
>     return stats.map(s -> "(" + s + ").stats");
>   }
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to