kl0u commented on a change in pull request #10313: [FLINK-14840] Use Executor 
interface in SQL cli
URL: https://github.com/apache/flink/pull/10313#discussion_r351799855
 
 

 ##########
 File path: 
flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ProgramDeployer.java
 ##########
 @@ -18,163 +18,83 @@
 
 package org.apache.flink.table.client.gateway.local;
 
-import org.apache.flink.api.common.JobExecutionResult;
-import org.apache.flink.client.ClientUtils;
-import org.apache.flink.client.deployment.ClusterDescriptor;
-import org.apache.flink.client.program.ClusterClient;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.table.client.gateway.SqlExecutionException;
-import org.apache.flink.table.client.gateway.local.result.Result;
+import org.apache.flink.api.dag.Pipeline;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.DeploymentOptions;
+import org.apache.flink.core.execution.DefaultExecutorServiceLoader;
+import org.apache.flink.core.execution.Executor;
+import org.apache.flink.core.execution.ExecutorFactory;
+import org.apache.flink.core.execution.ExecutorServiceLoader;
+import org.apache.flink.core.execution.JobClient;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.CompletableFuture;
 
 /**
  * The helper class to deploy a table program on the cluster.
  */
-public class ProgramDeployer<C> implements Runnable {
+public class ProgramDeployer<C> {
        private static final Logger LOG = 
LoggerFactory.getLogger(ProgramDeployer.class);
 
        private final ExecutionContext<C> context;
-       private final JobGraph jobGraph;
+       private final Pipeline pipeline;
        private final String jobName;
-       private final Result<C> result;
        private final boolean awaitJobResult;
-       private final BlockingQueue<JobExecutionResult> executionResultBucket;
 
        /**
         * Deploys a table program on the cluster.
         *
         * @param context        context with deployment information
         * @param jobName        job name of the Flink job to be submitted
-        * @param jobGraph       Flink job graph
-        * @param result         result that receives information about the 
target cluster
+        * @param pipeline       Flink {@link Pipeline} to execute
         * @param awaitJobResult block for a job execution result from the 
cluster
         */
        public ProgramDeployer(
                        ExecutionContext<C> context,
                        String jobName,
-                       JobGraph jobGraph,
-                       Result<C> result,
+                       Pipeline pipeline,
                        boolean awaitJobResult) {
                this.context = context;
-               this.jobGraph = jobGraph;
+               this.pipeline = pipeline;
                this.jobName = jobName;
-               this.result = result;
                this.awaitJobResult = awaitJobResult;
-               executionResultBucket = new LinkedBlockingDeque<>(1);
        }
 
-       @Override
-       public void run() {
-               LOG.info("Submitting job {} for query {}`", 
jobGraph.getJobID(), jobName);
+       public CompletableFuture<JobClient> deploy() {
+               LOG.info("Submitting job {} for query {}`", pipeline, jobName);
                if (LOG.isDebugEnabled()) {
                        LOG.debug("Submitting job {} with the following 
environment: \n{}",
-                                       jobGraph.getJobID(), 
context.getMergedEnvironment());
+                                       pipeline, 
context.getMergedEnvironment());
                }
-               deployJob(context, jobGraph, result);
-       }
 
-       public JobExecutionResult fetchExecutionResult() {
-               return executionResultBucket.poll();
-       }
+               // create a copy so that we can change settings without 
affecting the original config
+               Configuration configuration = new 
Configuration(context.getFlinkConfig());
+               if (configuration.get(DeploymentOptions.TARGET) == null) {
+                       throw new RuntimeException("No execution.target 
specified in your configuration file.");
+               }
 
-       /**
-        * Deploys a job. Depending on the deployment creates a new job 
cluster. It saves the cluster id in
-        * the result and blocks until job completion.
-        */
-       private <T> void deployJob(ExecutionContext<T> context, JobGraph 
jobGraph, Result<T> result) {
-               // create or retrieve cluster and deploy job
-               try (final ClusterDescriptor<T> clusterDescriptor = 
context.createClusterDescriptor()) {
-                       try {
-                               // new cluster
-                               if (context.getClusterId() == null) {
-                                       
deployJobOnNewCluster(clusterDescriptor, jobGraph, result, 
context.getClassLoader());
-                               }
-                               // reuse existing cluster
-                               else {
-                                       
deployJobOnExistingCluster(context.getClusterId(), clusterDescriptor, jobGraph, 
result);
-                               }
-                       } catch (Exception e) {
-                               throw new SqlExecutionException("Could not 
retrieve or create a cluster.", e);
-                       }
-               } catch (SqlExecutionException e) {
-                       throw e;
-               } catch (Exception e) {
-                       throw new SqlExecutionException("Could not locate a 
cluster.", e);
+               if (awaitJobResult) {
 
 Review comment:
   What about `configuration.set(DeploymentOptions.ATTACHED, awaitJobResult);` ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

Reply via email to