Skip to content

Commit

Permalink
[FLINK-16866] Make jobsubmission non-blocking
Browse files Browse the repository at this point in the history
This closes apache#13217
  • Loading branch information
rmetzger committed Sep 6, 2020
1 parent 7da74dc commit 65ed039
Show file tree
Hide file tree
Showing 48 changed files with 1,569 additions and 360 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -18,22 +18,31 @@

package org.apache.flink.client;

import org.apache.flink.api.common.JobStatus;
import org.apache.flink.client.program.ContextEnvironment;
import org.apache.flink.client.program.PackagedProgram;
import org.apache.flink.client.program.ProgramInvocationException;
import org.apache.flink.client.program.StreamContextEnvironment;
import org.apache.flink.client.program.rest.retry.ExponentialWaitStrategy;
import org.apache.flink.client.program.rest.retry.WaitStrategy;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.CoreOptions;
import org.apache.flink.configuration.DeploymentOptions;
import org.apache.flink.core.execution.PipelineExecutorServiceLoader;
import org.apache.flink.runtime.client.JobInitializationException;
import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders;
import org.apache.flink.runtime.jobmaster.JobResult;
import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.SerializedThrowable;
import org.apache.flink.util.function.SupplierWithException;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.net.URL;
import java.net.URLClassLoader;
import java.util.List;
import java.util.Optional;

import static org.apache.flink.util.FlinkUserCodeClassLoader.NOOP_EXCEPTION_HANDLER;
import static org.apache.flink.util.Preconditions.checkNotNull;
Expand Down Expand Up @@ -111,4 +120,42 @@ public static void executeProgram(
Thread.currentThread().setContextClassLoader(contextClassLoader);
}
}

/**
* This method blocks until the job status is not INITIALIZING anymore.
* @param jobStatusSupplier supplier returning the job status.
* @param jobResultSupplier supplier returning the job result. This will only be called if the job reaches the FAILED state.
* @throws JobInitializationException If the initialization failed
*/
public static void waitUntilJobInitializationFinished(
SupplierWithException<JobStatus, Exception> jobStatusSupplier,
SupplierWithException<JobResult, Exception> jobResultSupplier,
ClassLoader userCodeClassloader)
throws JobInitializationException {
LOG.debug("Wait until job initialization is finished");
WaitStrategy waitStrategy = new ExponentialWaitStrategy(50, 2000);
try {
JobStatus status = jobStatusSupplier.get();
long attempt = 0;
while (status == JobStatus.INITIALIZING) {
Thread.sleep(waitStrategy.sleepTime(attempt++));
status = jobStatusSupplier.get();
}
if (status == JobStatus.FAILED) {
JobResult result = jobResultSupplier.get();
Optional<SerializedThrowable> throwable = result.getSerializedThrowable();
if (throwable.isPresent()) {
Throwable t = throwable.get().deserializeError(userCodeClassloader);
if (t instanceof JobInitializationException) {
throw t;
}
}
}
} catch (JobInitializationException initializationException) {
throw initializationException;
} catch (Throwable throwable) {
ExceptionUtils.checkInterrupted(throwable);
throw new RuntimeException("Error while waiting for job to be initialized", throwable);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import org.apache.flink.runtime.dispatcher.DispatcherGateway;
import org.apache.flink.runtime.jobgraph.JobGraph;
import org.apache.flink.util.FlinkException;
import org.apache.flink.util.function.FunctionUtils;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -82,7 +83,7 @@ public EmbeddedExecutor(
}

@Override
public CompletableFuture<JobClient> execute(final Pipeline pipeline, final Configuration configuration) throws MalformedURLException {
public CompletableFuture<JobClient> execute(final Pipeline pipeline, final Configuration configuration, ClassLoader userCodeClassloader) throws MalformedURLException {
checkNotNull(pipeline);
checkNotNull(configuration);

Expand All @@ -94,15 +95,15 @@ public CompletableFuture<JobClient> execute(final Pipeline pipeline, final Confi
return getJobClientFuture(optJobId.get());
}

return submitAndGetJobClientFuture(pipeline, configuration);
return submitAndGetJobClientFuture(pipeline, configuration, userCodeClassloader);
}

private CompletableFuture<JobClient> getJobClientFuture(final JobID jobId) {
LOG.info("Job {} was recovered successfully.", jobId);
return CompletableFuture.completedFuture(jobClientCreator.getJobClient(jobId));
}

private CompletableFuture<JobClient> submitAndGetJobClientFuture(final Pipeline pipeline, final Configuration configuration) throws MalformedURLException {
private CompletableFuture<JobClient> submitAndGetJobClientFuture(final Pipeline pipeline, final Configuration configuration, final ClassLoader userCodeClassloader) throws MalformedURLException {
final Time timeout = Time.milliseconds(configuration.get(ClientOptions.CLIENT_TIMEOUT).toMillis());

final JobGraph jobGraph = PipelineExecutorUtils.getJobGraph(pipeline, configuration);
Expand All @@ -122,6 +123,13 @@ private CompletableFuture<JobClient> submitAndGetJobClientFuture(final Pipeline
timeout);

return jobSubmissionFuture
.thenApplyAsync(FunctionUtils.uncheckedFunction(jobId -> {
org.apache.flink.client.ClientUtils.waitUntilJobInitializationFinished(
() -> dispatcherGateway.requestJobStatus(jobId, timeout).get(),
() -> dispatcherGateway.requestJobResult(jobId, timeout).get(),
userCodeClassloader);
return jobId;
}))
.thenApplyAsync(jobID -> jobClientCreator.getJobClient(actualJobId));
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ public AbstractJobClusterExecutor(@Nonnull final ClientFactory clusterClientFact
}

@Override
public CompletableFuture<JobClient> execute(@Nonnull final Pipeline pipeline, @Nonnull final Configuration configuration) throws Exception {
public CompletableFuture<JobClient> execute(@Nonnull final Pipeline pipeline, @Nonnull final Configuration configuration, @Nonnull final ClassLoader userCodeClassloader) throws Exception {
final JobGraph jobGraph = PipelineExecutorUtils.getJobGraph(pipeline, configuration);

try (final ClusterDescriptor<ClusterID> clusterDescriptor = clusterClientFactory.createClusterDescriptor(configuration)) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@

import org.apache.flink.annotation.Internal;
import org.apache.flink.api.dag.Pipeline;
import org.apache.flink.client.ClientUtils;
import org.apache.flink.client.deployment.ClusterClientFactory;
import org.apache.flink.client.deployment.ClusterClientJobClientAdapter;
import org.apache.flink.client.deployment.ClusterDescriptor;
Expand All @@ -29,6 +30,7 @@
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.util.function.FunctionUtils;

import javax.annotation.Nonnull;

Expand All @@ -53,7 +55,7 @@ public AbstractSessionClusterExecutor(@Nonnull final ClientFactory clusterClient
}

@Override
public CompletableFuture<JobClient> execute(@Nonnull final Pipeline pipeline, @Nonnull final Configuration configuration) throws Exception {
public CompletableFuture<JobClient> execute(@Nonnull final Pipeline pipeline, @Nonnull final Configuration configuration, @Nonnull final ClassLoader userCodeClassloader) throws Exception {
final JobGraph jobGraph = PipelineExecutorUtils.getJobGraph(pipeline, configuration);

try (final ClusterDescriptor<ClusterID> clusterDescriptor = clusterClientFactory.createClusterDescriptor(configuration)) {
Expand All @@ -64,6 +66,13 @@ public CompletableFuture<JobClient> execute(@Nonnull final Pipeline pipeline, @N
ClusterClient<ClusterID> clusterClient = clusterClientProvider.getClusterClient();
return clusterClient
.submitJob(jobGraph)
.thenApplyAsync(FunctionUtils.uncheckedFunction(jobId -> {
ClientUtils.waitUntilJobInitializationFinished(
() -> clusterClient.getJobStatus(jobId).get(),
() -> clusterClient.requestJobResult(jobId).get(),
userCodeClassloader);
return jobId;
}))
.thenApplyAsync(jobID -> (JobClient) new ClusterClientJobClientAdapter<>(
clusterClientProvider,
jobID))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,7 @@ private LocalExecutor(Configuration configuration, Function<MiniClusterConfigura
}

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

Expand All @@ -78,7 +78,7 @@ public CompletableFuture<JobClient> execute(Pipeline pipeline, Configuration con

final JobGraph jobGraph = getJobGraph(pipeline, effectiveConfig);

return PerJobMiniClusterFactory.createWithFactory(effectiveConfig, miniClusterFactory).submitJob(jobGraph);
return PerJobMiniClusterFactory.createWithFactory(effectiveConfig, miniClusterFactory).submitJob(jobGraph, userCodeClassloader);
}

private JobGraph getJobGraph(Pipeline pipeline, Configuration configuration) throws MalformedURLException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@
import org.apache.flink.runtime.operators.coordination.CoordinationResponse;
import org.apache.flink.util.MathUtils;
import org.apache.flink.util.SerializedValue;
import org.apache.flink.util.function.FunctionUtils;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -81,13 +82,20 @@ private PerJobMiniClusterFactory(
/**
* Starts a {@link MiniCluster} and submits a job.
*/
public CompletableFuture<JobClient> submitJob(JobGraph jobGraph) throws Exception {
public CompletableFuture<JobClient> submitJob(JobGraph jobGraph, ClassLoader userCodeClassloader) throws Exception {
MiniClusterConfiguration miniClusterConfig = getMiniClusterConfig(jobGraph.getMaximumParallelism());
MiniCluster miniCluster = miniClusterFactory.apply(miniClusterConfig);
miniCluster.start();

return miniCluster
.submitJob(jobGraph)
.thenApplyAsync(FunctionUtils.uncheckedFunction(submissionResult -> {
org.apache.flink.client.ClientUtils.waitUntilJobInitializationFinished(
() -> miniCluster.getJobStatus(submissionResult.getJobID()).get(),
() -> miniCluster.requestJobResult(submissionResult.getJobID()).get(),
userCodeClassloader);
return submissionResult;
}))
.thenApply(result -> new PerJobMiniClusterJobClient(result.getJobID(), miniCluster))
.whenComplete((ignored, throwable) -> {
if (throwable != null) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,127 @@
/*
* 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: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;

import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.JobStatus;
import org.apache.flink.core.testutils.CommonTestUtils;
import org.apache.flink.runtime.client.JobInitializationException;
import org.apache.flink.runtime.jobmaster.JobResult;
import org.apache.flink.util.SerializedThrowable;
import org.apache.flink.util.TestLogger;

import org.junit.Assert;
import org.junit.Test;

import java.util.Arrays;
import java.util.Iterator;

/**
* Test for the ClientUtils.
*/
public class ClientUtilsTest extends TestLogger {

private static final JobID TESTING_JOB_ID = new JobID();

/**
* Ensure that the waitUntilJobInitializationFinished() method throws JobInitializationException.
*/
@Test
public void testWaitUntilJobInitializationFinished_throwsInitializationException() {
Iterator<JobStatus> statusSequenceIterator = Arrays.asList(
JobStatus.INITIALIZING,
JobStatus.INITIALIZING,
JobStatus.FAILED).iterator();

CommonTestUtils.assertThrows("Something is wrong", JobInitializationException.class, () -> {
ClientUtils.waitUntilJobInitializationFinished(
statusSequenceIterator::next,
() -> {
Throwable throwable = new JobInitializationException(
TESTING_JOB_ID,
"Something is wrong",
new RuntimeException("Err"));
return buildJobResult(throwable);
},
ClassLoader.getSystemClassLoader());
return null;
});
}

/**
* Ensure that waitUntilJobInitializationFinished() does not throw non-initialization exceptions.
*/
@Test
public void testWaitUntilJobInitializationFinished_doesNotThrowRuntimeException() throws Exception {
Iterator<JobStatus> statusSequenceIterator = Arrays.asList(
JobStatus.INITIALIZING,
JobStatus.INITIALIZING,
JobStatus.FAILED).iterator();
ClientUtils.waitUntilJobInitializationFinished(
statusSequenceIterator::next,
() -> buildJobResult(new RuntimeException("Err")),
ClassLoader.getSystemClassLoader());
}

/**
* Ensure that other errors are thrown.
*/
@Test
public void testWaitUntilJobInitializationFinished_throwsOtherErrors() {
CommonTestUtils.assertThrows("Error while waiting for job to be initialized", RuntimeException.class, () -> {
ClientUtils.waitUntilJobInitializationFinished(() -> {
throw new RuntimeException("other error");
},
() -> {
Throwable throwable = new JobInitializationException(
TESTING_JOB_ID,
"Something is wrong",
new RuntimeException("Err"));
return buildJobResult(throwable);
},
ClassLoader.getSystemClassLoader());
return null;
});
}

private JobResult buildJobResult(Throwable throwable) {
return new JobResult.Builder()
.jobId(TESTING_JOB_ID)
.serializedThrowable(new SerializedThrowable(throwable))
.netRuntime(1)
.build();
}

/**
* Test normal operation.
*/
@Test
public void testWaitUntilJobInitializationFinished_regular() throws Exception {
Iterator<JobStatus> statusSequenceIterator = Arrays.asList(
JobStatus.INITIALIZING,
JobStatus.INITIALIZING,
JobStatus.RUNNING).iterator();
ClientUtils.waitUntilJobInitializationFinished(
statusSequenceIterator::next, () -> {
Assert.fail("unexpected call");
return null;
},
ClassLoader.getSystemClassLoader());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -420,7 +420,7 @@ public boolean isCompatibleWith(@Nonnull Configuration configuration) {

@Override
public PipelineExecutor getExecutor(@Nonnull Configuration configuration) {
return (pipeline, config) -> {
return (pipeline, config, classLoader) -> {
final int parallelism = config.getInteger(CoreOptions.DEFAULT_PARALLELISM);
final JobGraph jobGraph = FlinkPipelineTranslationUtil.getJobGraph(plan, config, parallelism);

Expand Down
Loading

0 comments on commit 65ed039

Please sign in to comment.