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:37 UTC
[flink] 08/08: [FLINK-14067] Remove unused
PlanExecutor.getOptimizerPlanAsJSON()
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 e2728c0dddafcfe7fac0652084be6c7fd9714d85
Author: Aljoscha Krettek <al...@apache.org>
AuthorDate: Thu Sep 5 18:09:37 2019 +0200
[FLINK-14067] Remove unused PlanExecutor.getOptimizerPlanAsJSON()
---
.../java/org/apache/flink/client/LocalExecutor.java | 19 -------------------
.../java/org/apache/flink/client/RemoteExecutor.java | 12 ------------
.../org/apache/flink/api/common/PlanExecutor.java | 8 --------
.../org/apache/flink/api/java/FlinkILoopTest.java | 5 -----
4 files changed, 44 deletions(-)
diff --git a/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java b/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java
index 08b8448..d9444d5 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java
@@ -18,7 +18,6 @@
package org.apache.flink.client;
-import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.api.common.Plan;
import org.apache.flink.api.common.PlanExecutor;
@@ -29,7 +28,6 @@ import org.apache.flink.configuration.TaskManagerOptions;
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.JobExecutorService;
@@ -139,21 +137,4 @@ public class LocalExecutor extends PlanExecutor {
plan.setDefaultParallelism(slotsPerTaskManager * numTaskManagers);
}
-
- /**
- * Creates a JSON representation of the given dataflow's execution plan.
- *
- * @param plan The dataflow plan.
- * @return The dataflow's execution plan, as a JSON string.
- */
- @Override
- public String getOptimizerPlanAsJSON(Plan plan) {
- final int parallelism = plan.getDefaultParallelism() == ExecutionConfig.PARALLELISM_DEFAULT ? 1 : plan.getDefaultParallelism();
-
- Optimizer pc = new Optimizer(new DataStatistics(), this.baseConfiguration);
- pc.setDefaultParallelism(parallelism);
- OptimizedPlan op = pc.compile(plan);
-
- return new PlanJSONDumpGenerator().getOptimizerPlanAsJSON(op);
- }
}
diff --git a/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java b/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java
index dbe0efe..37f5eff 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java
@@ -27,11 +27,6 @@ import org.apache.flink.client.program.rest.RestClusterClient;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.JobManagerOptions;
import org.apache.flink.configuration.RestOptions;
-import org.apache.flink.optimizer.DataStatistics;
-import org.apache.flink.optimizer.Optimizer;
-import org.apache.flink.optimizer.costs.DefaultCostEstimator;
-import org.apache.flink.optimizer.plan.OptimizedPlan;
-import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator;
import java.net.InetSocketAddress;
import java.net.URL;
@@ -127,11 +122,4 @@ public class RemoteExecutor extends PlanExecutor {
return client.run(program, defaultParallelism).getJobExecutionResult();
}
}
-
- @Override
- public String getOptimizerPlanAsJSON(Plan plan) {
- Optimizer opt = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), new Configuration());
- OptimizedPlan optPlan = opt.compile(plan);
- return new PlanJSONDumpGenerator().getOptimizerPlanAsJSON(optPlan);
- }
}
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/PlanExecutor.java b/flink-core/src/main/java/org/apache/flink/api/common/PlanExecutor.java
index da29459..2095c63 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/PlanExecutor.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/PlanExecutor.java
@@ -63,14 +63,6 @@ public abstract class PlanExecutor {
* @throws Exception Thrown, if job submission caused an exception.
*/
public abstract JobExecutionResult executePlan(Plan plan) throws Exception;
-
- /**
- * Gets the programs execution plan in a JSON format.
- *
- * @param plan The program to get the execution plan for.
- * @return The execution plan, as a JSON string.
- */
- public abstract String getOptimizerPlanAsJSON(Plan plan);
// ------------------------------------------------------------------------
// Executor Factories
diff --git a/flink-scala-shell/src/test/java/org/apache/flink/api/java/FlinkILoopTest.java b/flink-scala-shell/src/test/java/org/apache/flink/api/java/FlinkILoopTest.java
index 3e27fad..eddf66c 100644
--- a/flink-scala-shell/src/test/java/org/apache/flink/api/java/FlinkILoopTest.java
+++ b/flink-scala-shell/src/test/java/org/apache/flink/api/java/FlinkILoopTest.java
@@ -132,11 +132,6 @@ public class FlinkILoopTest extends TestLogger {
return null;
}
- @Override
- public String getOptimizerPlanAsJSON(Plan plan) {
- return null;
- }
-
public String getHost() {
return host;
}