You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by al...@apache.org on 2019/09/18 07:53:35 UTC
[flink] 06/08: [FLINK-14067] Refactor
ExecutionEnvironment.getExecutionPlan() to be in root class
This is an automated email from the ASF dual-hosted git repository.
aljoscha pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git
commit 394a6e38189b4960e8dd697babdbb33194e9ecc4
Author: Aljoscha Krettek <al...@apache.org>
AuthorDate: Thu Sep 5 17:49:02 2019 +0200
[FLINK-14067] Refactor ExecutionEnvironment.getExecutionPlan() to be in root class
Before, each subclass had a slightly different way of getting the
execution plan (as JSON). Now, we factor that part out into a utility
and use that in the ExecutionEnvironment root class. This does mean,
that we don't take into account special information that a cluster
client or some other environment might have for plan generation.
Also, we can't remove the throws Exception from getExecutionPlan()
because it is a @Public method.
---
.../flink/client/program/ContextEnvironment.java | 11 ------
.../client/program/OptimizerPlanEnvironment.java | 9 -----
.../flink/api/java/CollectionEnvironment.java | 5 ---
.../flink/api/java/ExecutionEnvironment.java | 17 +++++----
.../apache/flink/api/java/ExecutionPlanUtil.java | 40 ++++++++++++++++++++++
.../apache/flink/api/java/LocalEnvironment.java | 7 ----
.../apache/flink/api/java/RemoteEnvironment.java | 7 ----
.../apache/flink/test/util/TestEnvironment.java | 9 -----
.../jsonplan/JsonJobGraphGenerationTest.java | 5 ---
.../flink/test/util/PlanExposingEnvironment.java | 5 ---
10 files changed, 48 insertions(+), 67 deletions(-)
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java b/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java
index bfd17d1..3fdae5e 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java
@@ -24,8 +24,6 @@ import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.api.common.JobSubmissionResult;
import org.apache.flink.api.common.Plan;
import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.optimizer.plan.OptimizedPlan;
-import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator;
import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings;
import java.net.URL;
@@ -74,15 +72,6 @@ public class ContextEnvironment extends ExecutionEnvironment {
return lastJobExecutionResult;
}
- @Override
- public String getExecutionPlan() throws Exception {
- Plan plan = createProgramPlan("unnamed job");
-
- OptimizedPlan op = ClusterClient.getOptimizedPlan(client.compiler, plan, getParallelism());
- PlanJSONDumpGenerator gen = new PlanJSONDumpGenerator();
- return gen.getOptimizerPlanAsJSON(op);
- }
-
private void verifyExecuteIsCalledOnceWhenInDetachedMode() {
if (alreadyCalled && detached) {
throw new InvalidProgramException(DetachedJobExecutionResult.DETACHED_MESSAGE + DetachedJobExecutionResult.EXECUTE_TWICE_MESSAGE);
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/OptimizerPlanEnvironment.java b/flink-clients/src/main/java/org/apache/flink/client/program/OptimizerPlanEnvironment.java
index 5b391af..d2801ac 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/OptimizerPlanEnvironment.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/OptimizerPlanEnvironment.java
@@ -54,15 +54,6 @@ public class OptimizerPlanEnvironment extends ExecutionEnvironment {
throw new ProgramAbortException();
}
- @Override
- public String getExecutionPlan() throws Exception {
- Plan plan = createProgramPlan(null, false);
- this.optimizerPlan = compiler.compile(plan);
-
- // do not go on with anything now!
- throw new ProgramAbortException();
- }
-
public FlinkPlan getOptimizedPlan(PackagedProgram prog) throws ProgramInvocationException {
// temporarily write syserr and sysout to a byte array.
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/CollectionEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/CollectionEnvironment.java
index f05322a..9be96df 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/CollectionEnvironment.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/CollectionEnvironment.java
@@ -43,9 +43,4 @@ public class CollectionEnvironment extends ExecutionEnvironment {
public int getParallelism() {
return 1; // always serial
}
-
- @Override
- public String getExecutionPlan() throws Exception {
- throw new UnsupportedOperationException("Execution plans are not used for collection-based execution.");
- }
}
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
index 337086e..ba64bff 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
@@ -766,13 +766,14 @@ public abstract class ExecutionEnvironment {
/**
* Creates the plan with which the system will execute the program, and returns it as
* a String using a JSON representation of the execution data flow graph.
- * Note that this needs to be called, before the plan is executed.
*
* @return The execution plan of the program, as a JSON String.
- * @throws Exception Thrown, if the compiler could not be instantiated, or the master could not
- * be contacted to retrieve information relevant to the execution planning.
+ * @throws Exception Thrown, if the compiler could not be instantiated.
*/
- public abstract String getExecutionPlan() throws Exception;
+ public String getExecutionPlan() throws Exception {
+ Plan p = createProgramPlan(getDefaultName(), false);
+ return ExecutionPlanUtil.getExecutionPlanAsJSON(p);
+ }
/**
* Registers a file at the distributed cache under the given name. The file will be accessible
@@ -836,7 +837,7 @@ public abstract class ExecutionEnvironment {
*/
@Internal
public Plan createProgramPlan() {
- return createProgramPlan(null);
+ return createProgramPlan(getDefaultName());
}
/**
@@ -868,6 +869,8 @@ public abstract class ExecutionEnvironment {
*/
@Internal
public Plan createProgramPlan(String jobName, boolean clearSinks) {
+ checkNotNull(jobName);
+
if (this.sinks.isEmpty()) {
if (wasExecuted) {
throw new RuntimeException("No new data sinks have been defined since the " +
@@ -880,10 +883,6 @@ public abstract class ExecutionEnvironment {
}
}
- if (jobName == null) {
- jobName = getDefaultName();
- }
-
OperatorTranslation translator = new OperatorTranslation();
Plan plan = translator.translateToPlan(this.sinks, jobName);
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionPlanUtil.java b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionPlanUtil.java
new file mode 100644
index 0000000..b64f39b
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionPlanUtil.java
@@ -0,0 +1,40 @@
+/*
+ * 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.api.java;
+
+import org.apache.flink.api.common.Plan;
+import org.apache.flink.api.common.PlanExecutor;
+import org.apache.flink.configuration.Configuration;
+
+/**
+ * A utility for extracting an execution plan (as JSON) from a {@link Plan}.
+ */
+class ExecutionPlanUtil {
+
+ /**
+ * Extracts the execution plan (as JSON) from the given {@link Plan}.
+ */
+ static String getExecutionPlanAsJSON(Plan plan) {
+ // make sure that we do not start an executor in any case here.
+ // if one runs, fine, of not, we only create the class but disregard immediately afterwards
+ PlanExecutor tempExecutor = PlanExecutor.createLocalExecutor(new Configuration());
+ return tempExecutor.getOptimizerPlanAsJSON(plan);
+ }
+}
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/LocalEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/LocalEnvironment.java
index f00b954..9ce0946 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/LocalEnvironment.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/LocalEnvironment.java
@@ -78,13 +78,6 @@ public class LocalEnvironment extends ExecutionEnvironment {
}
@Override
- public String getExecutionPlan() throws Exception {
- final Plan p = createProgramPlan("plan", false);
- final PlanExecutor tempExecutor = PlanExecutor.createLocalExecutor(configuration);
- return tempExecutor.getOptimizerPlanAsJSON(p);
- }
-
- @Override
public String toString() {
return "Local Environment (parallelism = " + (getParallelism() == ExecutionConfig.PARALLELISM_DEFAULT ? "default" : getParallelism()) + ").";
}
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/RemoteEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/RemoteEnvironment.java
index ec1c7df..786add1 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/RemoteEnvironment.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/RemoteEnvironment.java
@@ -161,13 +161,6 @@ public class RemoteEnvironment extends ExecutionEnvironment {
}
@Override
- public String getExecutionPlan() throws Exception {
- final Plan p = createProgramPlan("plan", false);
- final PlanExecutor tempExecutor = PlanExecutor.createLocalExecutor(new Configuration());
- return tempExecutor.getOptimizerPlanAsJSON(p);
- }
-
- @Override
public String toString() {
return "Remote Environment (" + this.host + ":" + this.port + " - parallelism = " +
(getParallelism() == -1 ? "default" : getParallelism()) + ").";
diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java
index 01c29ca..c1c632d 100644
--- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java
+++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java
@@ -27,7 +27,6 @@ import org.apache.flink.core.fs.Path;
import org.apache.flink.optimizer.DataStatistics;
import org.apache.flink.optimizer.Optimizer;
import org.apache.flink.optimizer.plan.OptimizedPlan;
-import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator;
import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
import org.apache.flink.runtime.jobgraph.JobGraph;
import org.apache.flink.runtime.minicluster.JobExecutor;
@@ -113,14 +112,6 @@ public class TestEnvironment extends ExecutionEnvironment {
return this.lastJobExecutionResult;
}
- @Override
- public String getExecutionPlan() throws Exception {
- OptimizedPlan op = compileProgram("unused");
-
- PlanJSONDumpGenerator jsonGen = new PlanJSONDumpGenerator();
- return jsonGen.getOptimizerPlanAsJSON(op);
- }
-
private OptimizedPlan compileProgram(String jobName) {
Plan p = createProgramPlan(jobName);
diff --git a/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/JsonJobGraphGenerationTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/JsonJobGraphGenerationTest.java
index 64c3923..7414dcd 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/JsonJobGraphGenerationTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/JsonJobGraphGenerationTest.java
@@ -354,11 +354,6 @@ public class JsonJobGraphGenerationTest {
throw new AbortError();
}
- @Override
- public String getExecutionPlan() throws Exception {
- throw new UnsupportedOperationException();
- }
-
public static void setAsNext(final JsonValidator validator, final int defaultParallelism) {
initializeContextEnvironment(new ExecutionEnvironmentFactory() {
@Override
diff --git a/flink-tests/src/test/java/org/apache/flink/test/util/PlanExposingEnvironment.java b/flink-tests/src/test/java/org/apache/flink/test/util/PlanExposingEnvironment.java
index 5da0e6c..9488732 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/util/PlanExposingEnvironment.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/util/PlanExposingEnvironment.java
@@ -39,11 +39,6 @@ public final class PlanExposingEnvironment extends ExecutionEnvironment {
throw new OptimizerPlanEnvironment.ProgramAbortException();
}
- @Override
- public String getExecutionPlan() throws Exception {
- throw new OptimizerPlanEnvironment.ProgramAbortException();
- }
-
public void setAsContext() {
ExecutionEnvironmentFactory factory = new ExecutionEnvironmentFactory() {
@Override