Skip to content

Commit

Permalink
[FLINK-16705] Ensure MiniCluster shutdown does not interfere with Job…
Browse files Browse the repository at this point in the history
…Result retrieval

There is a race condition in `LocalExecutor` between (a) shutting down the
cluster when the job has finished and (b) the client which retrieves the result
of the job execution.

This was observed in Beam, running a large test suite with the Flink Runner.

We should make sure the job result retrieval and the cluster shutdown do not
interfere. This adds a PerJobMiniClusterClient which guarantees that.

Improve message for running flag state checks in MiniCluster

Additionally check for the JobID in PerJobMiniClusterClient

Introduce PerJobMiniCluster and a corresponding JobClient

Add TestLogger to test

Convert shutdown methods to be async

This closes apache#11473.
  • Loading branch information
mxm authored and tillrohrmann committed Apr 3, 2020
1 parent f87734d commit db81417
Show file tree
Hide file tree
Showing 11 changed files with 504 additions and 73 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -19,26 +19,22 @@
package org.apache.flink.client.deployment.executors;

import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.Plan;
import org.apache.flink.api.dag.Pipeline;
import org.apache.flink.client.FlinkPipelineTranslationUtil;
import org.apache.flink.client.deployment.ClusterClientJobClientAdapter;
import org.apache.flink.client.program.MiniClusterClient;
import org.apache.flink.client.program.PerJobMiniClusterFactory;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.DeploymentOptions;
import org.apache.flink.configuration.RestOptions;
import org.apache.flink.configuration.TaskManagerOptions;
import org.apache.flink.core.execution.JobClient;
import org.apache.flink.core.execution.PipelineExecutor;
import org.apache.flink.runtime.jobgraph.JobGraph;
import org.apache.flink.runtime.minicluster.MiniCluster;
import org.apache.flink.runtime.minicluster.MiniClusterConfiguration;
import org.apache.flink.runtime.minicluster.RpcServiceSharing;

import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
import java.util.function.Function;

import static org.apache.flink.util.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkState;
Expand All @@ -51,26 +47,38 @@ public class LocalExecutor implements PipelineExecutor {

public static final String NAME = "local";

private final Configuration configuration;
private final Function<MiniClusterConfiguration, MiniCluster> miniClusterFactory;

public static LocalExecutor create(Configuration configuration) {
return new LocalExecutor(configuration, MiniCluster::new);
}

public static LocalExecutor createWithFactory(
Configuration configuration, Function<MiniClusterConfiguration, MiniCluster> miniClusterFactory) {
return new LocalExecutor(configuration, miniClusterFactory);
}

private LocalExecutor(Configuration configuration, Function<MiniClusterConfiguration, MiniCluster> miniClusterFactory) {
this.configuration = configuration;
this.miniClusterFactory = miniClusterFactory;
}

@Override
public CompletableFuture<JobClient> execute(Pipeline pipeline, Configuration configuration) throws Exception {
public CompletableFuture<? extends JobClient> execute(Pipeline pipeline, Configuration configuration) throws Exception {
checkNotNull(pipeline);
checkNotNull(configuration);

Configuration effectiveConfig = new Configuration();
effectiveConfig.addAll(this.configuration);
effectiveConfig.addAll(configuration);

// we only support attached execution with the local executor.
checkState(configuration.getBoolean(DeploymentOptions.ATTACHED));

final JobGraph jobGraph = getJobGraph(pipeline, configuration);
final MiniCluster miniCluster = startMiniCluster(jobGraph, configuration);
final MiniClusterClient clusterClient = new MiniClusterClient(configuration, miniCluster);

CompletableFuture<JobID> jobIdFuture = clusterClient.submitJob(jobGraph);
final JobGraph jobGraph = getJobGraph(pipeline, effectiveConfig);

jobIdFuture
.thenCompose(clusterClient::requestJobResult)
.thenAccept((jobResult) -> clusterClient.shutDownCluster());

return jobIdFuture.thenApply(jobID ->
new ClusterClientJobClientAdapter<>(() -> clusterClient, jobID));
return PerJobMiniClusterFactory.createWithFactory(effectiveConfig, miniClusterFactory).submitJob(jobGraph);
}

private JobGraph getJobGraph(Pipeline pipeline, Configuration configuration) {
Expand All @@ -89,45 +97,4 @@ private JobGraph getJobGraph(Pipeline pipeline, Configuration configuration) {

return FlinkPipelineTranslationUtil.getJobGraph(pipeline, configuration, 1);
}

private MiniCluster startMiniCluster(final JobGraph jobGraph, final Configuration configuration) throws Exception {
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,
jobGraph.getMaximumParallelism());

final MiniClusterConfiguration miniClusterConfiguration =
new MiniClusterConfiguration.Builder()
.setConfiguration(configuration)
.setNumTaskManagers(numTaskManagers)
.setRpcServiceSharing(RpcServiceSharing.SHARED)
.setNumSlotsPerTaskManager(numSlotsPerTaskManager)
.build();

final MiniCluster miniCluster = new MiniCluster(miniClusterConfiguration);
miniCluster.start();

configuration.setInteger(RestOptions.PORT, miniCluster.getRestAddress().get().getPort());

return miniCluster;
}

private void shutdownMiniCluster(final MiniCluster miniCluster) {
try {
if (miniCluster != null) {
miniCluster.close();
}
} catch (Exception e) {
throw new CompletionException(e);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,6 @@ public boolean isCompatibleWith(final Configuration configuration) {

@Override
public PipelineExecutor getExecutor(final Configuration configuration) {
return new LocalExecutor();
return LocalExecutor.create(configuration);
}
}
Original file line number Diff line number Diff line change
@@ -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
*
* https://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.CompletionException;
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 final class PerJobMiniClusterFactory {

private static final Logger LOG = LoggerFactory.getLogger(PerJobMiniClusterFactory.class);

private final Configuration configuration;
private final Function<? super MiniClusterConfiguration, ? extends MiniCluster> miniClusterFactory;

public static PerJobMiniClusterFactory create() {
return new PerJobMiniClusterFactory(new Configuration(), MiniCluster::new);
}

public static PerJobMiniClusterFactory createWithFactory(
Configuration configuration,
Function<? super MiniClusterConfiguration, ? extends MiniCluster> miniClusterFactory) {
return new PerJobMiniClusterFactory(configuration, miniClusterFactory);
}

private PerJobMiniClusterFactory(
Configuration configuration,
Function<? super MiniClusterConfiguration, ? extends MiniCluster> miniClusterFactory) {
this.configuration = configuration;
this.miniClusterFactory = miniClusterFactory;
}

/**
* Starts a {@link MiniCluster} and submits a job.
*/
public CompletableFuture<? extends JobClient> submitJob(JobGraph jobGraph) throws Exception {
MiniClusterConfiguration miniClusterConfig = getMiniClusterConfig(jobGraph.getMaximumParallelism());
MiniCluster miniCluster = miniClusterFactory.apply(miniClusterConfig);
miniCluster.start();

return miniCluster
.submitJob(jobGraph)
.thenApply(result -> new PerJobMiniClusterJobClient(result.getJobID(), miniCluster))
.whenComplete((ignored, throwable) -> {
if (throwable != null) {
// We failed to create the JobClient and must shutdown to ensure cleanup.
shutDownCluster(miniCluster);
}
});
}

private MiniClusterConfiguration getMiniClusterConfig(int maximumParallelism) {
Configuration configuration = new Configuration(this.configuration);

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) {
miniCluster.closeAsync()
.whenComplete((ignored, throwable) -> {
if (throwable != null) {
LOG.warn("Shutdown of MiniCluster failed.", throwable);
}
});
}

/**
* A {@link JobClient} for a {@link PerJobMiniClusterFactory}.
*/
private static final class PerJobMiniClusterJobClient implements JobClient {

private final JobID jobID;
private final MiniCluster miniCluster;
private final CompletableFuture<JobResult> jobResultFuture;

private PerJobMiniClusterJobClient(JobID jobID, MiniCluster miniCluster) {
this.jobID = jobID;
this.miniCluster = miniCluster;
this.jobResultFuture = miniCluster
.requestJobResult(jobID)
// Make sure to shutdown the cluster when the job completes.
.whenComplete((result, throwable) -> shutDownCluster(miniCluster));
}

@Override
public JobID getJobID() {
return jobID;
}

@Override
public CompletableFuture<JobStatus> getJobStatus() {
return miniCluster.getJobStatus(jobID);
}

@Override
public CompletableFuture<Void> cancel() {
return miniCluster.cancelJob(jobID).thenAccept(result -> {});
}

@Override
public CompletableFuture<String> stopWithSavepoint(boolean advanceToEndOfEventTime, @Nullable String savepointDirectory) {
return miniCluster.stopWithSavepoint(jobID, savepointDirectory, advanceToEndOfEventTime);
}

@Override
public CompletableFuture<String> triggerSavepoint(@Nullable String savepointDirectory) {
return miniCluster.triggerSavepoint(jobID, savepointDirectory, false);
}

@Override
public CompletableFuture<Map<String, Object>> getAccumulators(ClassLoader classLoader) {
return getJobExecutionResult(classLoader).thenApply(JobExecutionResult::getAllAccumulatorResults);
}

@Override
public CompletableFuture<JobExecutionResult> getJobExecutionResult(ClassLoader classLoader) {
return jobResultFuture.thenApply(result -> {
try {
return result.toJobExecutionResult(classLoader);
} catch (Exception e) {
throw new CompletionException("Failed to convert JobResult to JobExecutionResult.", e);
}
});
}
}
}
Loading

0 comments on commit db81417

Please sign in to comment.