tillrohrmann commented on a change in pull request #11473: [FLINK-16705] Ensure 
MiniCluster shutdown does not interfere with JobResult retrieval
URL: https://github.com/apache/flink/pull/11473#discussion_r400230703
 
 

 ##########
 File path: 
flink-clients/src/main/java/org/apache/flink/client/program/PerJobMiniCluster.java
 ##########
 @@ -0,0 +1,182 @@
+/*
+ * 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.client.program;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.JobStatus;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.RestOptions;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.core.execution.JobClient;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobmaster.JobResult;
+import org.apache.flink.runtime.minicluster.MiniCluster;
+import org.apache.flink.runtime.minicluster.MiniClusterConfiguration;
+import org.apache.flink.runtime.minicluster.RpcServiceSharing;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.function.Function;
+
+/**
+ * Starts a {@link MiniCluster} for every submitted job.
+ * This class guarantees to tear down the MiniCluster in case of normal or 
exceptional job completion.
+ * */
+public class PerJobMiniCluster {
+
+       private static final Logger LOG = 
LoggerFactory.getLogger(PerJobMiniCluster.class);
+
+       private final Configuration configuration;
+       private final Function<MiniClusterConfiguration, MiniCluster> 
miniClusterFactory;
+
+       public PerJobMiniCluster() {
+               this(new Configuration());
+       }
+
+       public PerJobMiniCluster(Configuration configuration) {
+               this(configuration, MiniCluster::new);
+       }
+
+       public PerJobMiniCluster(Configuration configuration, 
Function<MiniClusterConfiguration, MiniCluster> miniClusterFactory) {
+               this.configuration = configuration;
+               this.miniClusterFactory = miniClusterFactory;
+       }
+
+       /**
+        * Starts a {@link MiniCluster} and submits a job.
+        */
+       public CompletableFuture<JobClient> submitJob(JobGraph jobGraph) throws 
Exception {
+               MiniClusterConfiguration miniClusterConfig = 
getMiniClusterConfig(jobGraph.getMaximumParallelism());
+               MiniCluster miniCluster = 
miniClusterFactory.apply(miniClusterConfig);
+               miniCluster.start();
+
+               return miniCluster
+                       .submitJob(jobGraph)
+                       .handle((result, throwable) -> {
+                               if (throwable != null) {
+                                       try {
+                                               throw new 
RuntimeException("Error submitting job to MiniCluster.", throwable);
+                                       } finally {
+                                               shutDownCluster(miniCluster);
+                                       }
+                               }
+                               return new 
PerJobMiniClusterJobClient(result.getJobID(), miniCluster);
+                       });
+       }
+
+       private MiniClusterConfiguration getMiniClusterConfig(int 
maximumParallelism) {
+               Configuration configuration = this.configuration.clone();
+
+               if (!configuration.contains(RestOptions.BIND_PORT)) {
+                       configuration.setString(RestOptions.BIND_PORT, "0");
+               }
+
+               int numTaskManagers = configuration.getInteger(
+                       ConfigConstants.LOCAL_NUMBER_TASK_MANAGER,
+                       ConfigConstants.DEFAULT_LOCAL_NUMBER_TASK_MANAGER);
+
+               // we have to use the maximum parallelism as a default here, 
otherwise streaming pipelines would not run
+               int numSlotsPerTaskManager = 
configuration.getInteger(TaskManagerOptions.NUM_TASK_SLOTS, maximumParallelism);
+
+               return new MiniClusterConfiguration.Builder()
+                       .setConfiguration(configuration)
+                       .setNumTaskManagers(numTaskManagers)
+                       .setRpcServiceSharing(RpcServiceSharing.SHARED)
+                       .setNumSlotsPerTaskManager(numSlotsPerTaskManager)
+                       .build();
+       }
+
+       private static void shutDownCluster(MiniCluster miniCluster) {
+               try {
+                       miniCluster.closeAsync().get();
+               } catch (InterruptedException e) {
+                       Thread.currentThread().interrupt();
+               } catch (ExecutionException e) {
+                       LOG.error("Error while shutting down cluster", e);
+               }
+       }
+
+       /**
+        * A {@link JobClient} for a {@link PerJobMiniCluster}.
+        */
+       private static class PerJobMiniClusterJobClient implements JobClient {
+
+               private final JobID jobID;
+               private final MiniCluster miniCluster;
+               private final CompletableFuture<JobResult> jobResultFuture;
+
+               public PerJobMiniClusterJobClient(JobID jobID, MiniCluster 
miniCluster) {
 
 Review comment:
   Public constructor in a non-public class.

----------------------------------------------------------------
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:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to