Skip to content

Commit

Permalink
[FLINK-7375] Replace ActorGateway with JobManagerGateway in JobClient
Browse files Browse the repository at this point in the history
In order to make the JobClient code independent of Akka, this PR replaces the
ActorGateway parameters by JobManagerGateway. AkkaJobManagerGateway is the
respective implementation of the JobManagerGateway for Akka. Moreover, this
PR introduces useful ExceptionUtils method for handling of Future exceptions.
Additionally, the SerializedThrowable has been moved to flink-core.

This closes #4486.
  • Loading branch information
tillrohrmann committed Aug 10, 2017
1 parent d52ccd2 commit dfaec33
Show file tree
Hide file tree
Showing 27 changed files with 344 additions and 147 deletions.
Expand Up @@ -35,11 +35,11 @@
import org.apache.flink.optimizer.plan.StreamingPlan; import org.apache.flink.optimizer.plan.StreamingPlan;
import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator; import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator;
import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
import org.apache.flink.runtime.akka.AkkaJobManagerGateway;
import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.akka.AkkaUtils;
import org.apache.flink.runtime.client.JobClient; import org.apache.flink.runtime.client.JobClient;
import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.client.JobExecutionException;
import org.apache.flink.runtime.client.JobListeningContext; import org.apache.flink.runtime.client.JobListeningContext;
import org.apache.flink.runtime.client.JobRetrievalException;
import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse; import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse;
import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.concurrent.Executors;
import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
Expand Down Expand Up @@ -500,7 +500,12 @@ public JobSubmissionResult runDetached(JobGraph jobGraph, ClassLoader classLoade


try { try {
logAndSysout("Submitting Job with JobID: " + jobGraph.getJobID() + ". Returning after job submission."); logAndSysout("Submitting Job with JobID: " + jobGraph.getJobID() + ". Returning after job submission.");
JobClient.submitJobDetached(jobManagerGateway, flinkConfig, jobGraph, timeout, classLoader); JobClient.submitJobDetached(
new AkkaJobManagerGateway(jobManagerGateway),
flinkConfig,
jobGraph,
Time.milliseconds(timeout.toMillis()),
classLoader);
return new JobSubmissionResult(jobGraph.getJobID()); return new JobSubmissionResult(jobGraph.getJobID());
} catch (JobExecutionException e) { } catch (JobExecutionException e) {
throw new ProgramInvocationException("The program execution failed: " + e.getMessage(), e); throw new ProgramInvocationException("The program execution failed: " + e.getMessage(), e);
Expand All @@ -525,16 +530,8 @@ public JobExecutionResult retrieveJob(JobID jobID) throws JobExecutionException
fe); fe);
} }


ActorGateway jobManagerGateway;
try {
jobManagerGateway = getJobManagerGateway();
} catch (Exception e) {
throw new JobRetrievalException(jobID, "Could not retrieve the JobManager Gateway");
}

final JobListeningContext listeningContext = JobClient.attachToRunningJob( final JobListeningContext listeningContext = JobClient.attachToRunningJob(
jobID, jobID,
jobManagerGateway,
flinkConfig, flinkConfig,
actorSystem, actorSystem,
highAvailabilityServices, highAvailabilityServices,
Expand Down Expand Up @@ -563,16 +560,8 @@ public JobListeningContext connectToJob(JobID jobID) throws JobExecutionExceptio
fe); fe);
} }


ActorGateway jobManagerGateway;
try {
jobManagerGateway = getJobManagerGateway();
} catch (Exception e) {
throw new JobRetrievalException(jobID, "Could not retrieve the JobManager Gateway", e);
}

return JobClient.attachToRunningJob( return JobClient.attachToRunningJob(
jobID, jobID,
jobManagerGateway,
flinkConfig, flinkConfig,
actorSystem, actorSystem,
highAvailabilityServices, highAvailabilityServices,
Expand Down
Expand Up @@ -43,8 +43,8 @@
import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
import org.apache.flink.runtime.jobmaster.JobMaster; import org.apache.flink.runtime.jobmaster.JobMaster;
import org.apache.flink.runtime.messages.JobManagerMessages; import org.apache.flink.runtime.messages.JobManagerMessages;
import org.apache.flink.runtime.util.SerializedThrowable;
import org.apache.flink.util.NetUtils; import org.apache.flink.util.NetUtils;
import org.apache.flink.util.SerializedThrowable;
import org.apache.flink.util.TestLogger; import org.apache.flink.util.TestLogger;


import akka.actor.ActorSystem; import akka.actor.ActorSystem;
Expand Down
38 changes: 38 additions & 0 deletions flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java
Expand Up @@ -30,6 +30,7 @@
import java.io.IOException; import java.io.IOException;
import java.io.PrintWriter; import java.io.PrintWriter;
import java.io.StringWriter; import java.io.StringWriter;
import java.util.concurrent.ExecutionException;


import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.util.Preconditions.checkNotNull;


Expand Down Expand Up @@ -298,6 +299,43 @@ public static boolean containsThrowable(Throwable throwable, Class<?> searchType
return false; return false;
} }


/**
* Unpacks an {@link ExecutionException} and returns its cause. Otherwise the given
* Throwable is returned.
*
* @param throwable to unpack if it is an ExecutionException
* @return Cause of ExecutionException or given Throwable
*/
public static Throwable stripExecutionException(Throwable throwable) {
while (throwable instanceof ExecutionException && throwable.getCause() != null) {
throwable = throwable.getCause();
}

return throwable;
}

/**
* Tries to find a {@link SerializedThrowable} as the cause of the given throwable and throws its
* deserialized value. If there is no such throwable, then the original throwable is thrown.
*
* @param throwable to check for a SerializedThrowable
* @param classLoader to be used for the deserialization of the SerializedThrowable
* @throws Throwable either the deserialized throwable or the given throwable
*/
public static void tryDeserializeAndThrow(Throwable throwable, ClassLoader classLoader) throws Throwable {
Throwable current = throwable;

while (!(current instanceof SerializedThrowable) && current.getCause() != null) {
current = current.getCause();
}

if (current instanceof SerializedThrowable) {
throw ((SerializedThrowable) current).deserializeError(classLoader);
} else {
throw throwable;
}
}

// ------------------------------------------------------------------------ // ------------------------------------------------------------------------


/** Private constructor to prevent instantiation. */ /** Private constructor to prevent instantiation. */
Expand Down
Expand Up @@ -16,10 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */


package org.apache.flink.runtime.util; package org.apache.flink.util;

import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.InstantiationUtil;


import java.io.PrintStream; import java.io.PrintStream;
import java.io.PrintWriter; import java.io.PrintWriter;
Expand Down
Expand Up @@ -18,9 +18,11 @@


package org.apache.flink.runtime.webmonitor.handlers; package org.apache.flink.runtime.webmonitor.handlers;


import org.apache.flink.api.common.time.Time;
import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.client.program.ProgramInvocationException; import org.apache.flink.client.program.ProgramInvocationException;
import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.akka.AkkaJobManagerGateway;
import org.apache.flink.runtime.client.JobClient; import org.apache.flink.runtime.client.JobClient;
import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.client.JobExecutionException;
import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.instance.ActorGateway;
Expand All @@ -29,12 +31,8 @@
import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.core.JsonGenerator;


import java.io.File; import java.io.File;
import java.io.IOException;
import java.io.StringWriter; import java.io.StringWriter;
import java.net.InetSocketAddress;
import java.util.Map; import java.util.Map;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;


import scala.concurrent.duration.FiniteDuration; import scala.concurrent.duration.FiniteDuration;


Expand Down Expand Up @@ -65,23 +63,13 @@ public String handleJsonRequest(Map<String, String> pathParams, Map<String, Stri
JarActionHandlerConfig config = JarActionHandlerConfig.fromParams(pathParams, queryParams); JarActionHandlerConfig config = JarActionHandlerConfig.fromParams(pathParams, queryParams);
Tuple2<JobGraph, ClassLoader> graph = getJobGraphAndClassLoader(config); Tuple2<JobGraph, ClassLoader> graph = getJobGraphAndClassLoader(config);


final CompletableFuture<InetSocketAddress> blobServerAddressFuture = JobClient.retrieveBlobServerAddress(jobManager, timeout);
final InetSocketAddress blobServerAddress;

try {
blobServerAddress = blobServerAddressFuture.get(timeout.toMillis(), TimeUnit.MILLISECONDS);
} catch (Exception e) {
throw new ProgramInvocationException("Failed to retrieve BlobServer address.", e);
}

try {
graph.f0.uploadUserJars(blobServerAddress, clientConfig);
} catch (IOException e) {
throw new ProgramInvocationException("Failed to upload jar files to the job manager", e);
}

try { try {
JobClient.submitJobDetached(jobManager, clientConfig, graph.f0, timeout, graph.f1); JobClient.submitJobDetached(
new AkkaJobManagerGateway(jobManager),
clientConfig,
graph.f0,
Time.milliseconds(timeout.toMillis()),
graph.f1);
} catch (JobExecutionException e) { } catch (JobExecutionException e) {
throw new ProgramInvocationException("Failed to submit the job to the job manager", e); throw new ProgramInvocationException("Failed to submit the job to the job manager", e);
} }
Expand Down
Expand Up @@ -22,6 +22,7 @@
import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.memory.MemoryType; import org.apache.flink.core.memory.MemoryType;
import org.apache.flink.runtime.akka.AkkaJobManagerGateway;
import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.akka.AkkaUtils;
import org.apache.flink.runtime.client.JobClient; import org.apache.flink.runtime.client.JobClient;
import org.apache.flink.runtime.executiongraph.ExecutionGraph; import org.apache.flink.runtime.executiongraph.ExecutionGraph;
Expand Down Expand Up @@ -156,10 +157,10 @@ protected void run() {


// Submit the job and wait until it is running // Submit the job and wait until it is running
JobClient.submitJobDetached( JobClient.submitJobDetached(
jm, new AkkaJobManagerGateway(jm),
config, config,
jobGraph, jobGraph,
deadline, Time.milliseconds(deadline.toMillis()),
ClassLoader.getSystemClassLoader()); ClassLoader.getSystemClassLoader());


jm.tell(new WaitForAllVerticesToBeRunning(jobGraph.getJobID()), testActor); jm.tell(new WaitForAllVerticesToBeRunning(jobGraph.getJobID()), testActor);
Expand Down
Expand Up @@ -21,6 +21,7 @@
import org.apache.flink.api.common.time.Time; import org.apache.flink.api.common.time.Time;
import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.akka.AkkaJobManagerGateway;
import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.akka.AkkaUtils;
import org.apache.flink.runtime.client.JobClient; import org.apache.flink.runtime.client.JobClient;
import org.apache.flink.runtime.executiongraph.ExecutionGraph; import org.apache.flink.runtime.executiongraph.ExecutionGraph;
Expand Down Expand Up @@ -131,10 +132,10 @@ protected void run() {
for (int i = 0; i < maxAttempts; i++, sleepTime *= 2) { for (int i = 0; i < maxAttempts; i++, sleepTime *= 2) {
// Submit the job and wait until it is running // Submit the job and wait until it is running
JobClient.submitJobDetached( JobClient.submitJobDetached(
jm, new AkkaJobManagerGateway(jm),
config, config,
jobGraph, jobGraph,
deadline, Time.milliseconds(deadline.toMillis()),
ClassLoader.getSystemClassLoader()); ClassLoader.getSystemClassLoader());


jm.tell(new WaitForAllVerticesToBeRunning(jobGraph.getJobID()), testActor); jm.tell(new WaitForAllVerticesToBeRunning(jobGraph.getJobID()), testActor);
Expand Down
@@ -0,0 +1,122 @@
/*
* 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.runtime.akka;

import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.time.Time;
import org.apache.flink.runtime.concurrent.FlinkFutureException;
import org.apache.flink.runtime.concurrent.FutureUtils;
import org.apache.flink.runtime.instance.ActorGateway;
import org.apache.flink.runtime.jobgraph.JobGraph;
import org.apache.flink.runtime.jobmaster.JobManagerGateway;
import org.apache.flink.runtime.messages.Acknowledge;
import org.apache.flink.runtime.messages.JobManagerMessages;
import org.apache.flink.util.Preconditions;

import java.util.Objects;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;

import scala.Option;
import scala.reflect.ClassTag$;

/**
* Implementation of the {@link JobManagerGateway} for the {@link ActorGateway}.
*/
public class AkkaJobManagerGateway implements JobManagerGateway {

private final ActorGateway jobManagerGateway;
private final String hostname;

public AkkaJobManagerGateway(ActorGateway jobManagerGateway) {
this.jobManagerGateway = Preconditions.checkNotNull(jobManagerGateway);

final Option<String> optHostname = jobManagerGateway.actor().path().address().host();

hostname = optHostname.isDefined() ? optHostname.get() : "localhost";
}

@Override
public String getAddress() {
return jobManagerGateway.path();
}

@Override
public String getHostname() {
return hostname;
}

@Override
public CompletableFuture<Optional<JobManagerMessages.ClassloadingProps>> requestClassloadingProps(JobID jobId, Time timeout) {
return FutureUtils
.toJava(jobManagerGateway
.ask(
new JobManagerMessages.RequestClassloadingProps(jobId),
FutureUtils.toFiniteDuration(timeout)))
.thenApply(
(Object response) -> {
if (response instanceof JobManagerMessages.ClassloadingProps) {
return Optional.of(((JobManagerMessages.ClassloadingProps) response));
} else if (response instanceof JobManagerMessages.JobNotFound) {
return Optional.empty();
} else {
throw new FlinkFutureException("Unknown response: " + response + '.');
}
});
}

@Override
public CompletableFuture<Integer> requestBlobServerPort(Time timeout) {
return FutureUtils.toJava(
jobManagerGateway
.ask(JobManagerMessages.getRequestBlobManagerPort(), FutureUtils.toFiniteDuration(timeout))
.mapTo(ClassTag$.MODULE$.apply(Integer.class)));
}

@Override
public CompletableFuture<Acknowledge> submitJob(JobGraph jobGraph, ListeningBehaviour listeningBehaviour, Time timeout) {
return FutureUtils
.toJava(
jobManagerGateway.ask(
new JobManagerMessages.SubmitJob(
jobGraph,
listeningBehaviour),
FutureUtils.toFiniteDuration(timeout)))
.thenApply(
(Object response) -> {
if (response instanceof JobManagerMessages.JobSubmitSuccess) {
JobManagerMessages.JobSubmitSuccess success = ((JobManagerMessages.JobSubmitSuccess) response);

if (Objects.equals(success.jobId(), jobGraph.getJobID())) {
return Acknowledge.get();
} else {
throw new FlinkFutureException("JobManager responded for wrong Job. This Job: " +
jobGraph.getJobID() + ", response: " + success.jobId());
}
} else if (response instanceof JobManagerMessages.JobResultFailure) {
JobManagerMessages.JobResultFailure failure = ((JobManagerMessages.JobResultFailure) response);

throw new FlinkFutureException("Job submission failed.", failure.cause());
} else {
throw new FlinkFutureException("Unknown response to SubmitJob message: " + response + '.');
}
}
);
}
}

0 comments on commit dfaec33

Please sign in to comment.