From 241c1ca4db1e5f4b3f81b3cdf694246062f032d1 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 2 Oct 2014 11:19:52 +0200 Subject: [PATCH] Replaced the JobClient by an actor. --- .../mapred/wrapper/HadoopInputSplit.java | 26 + .../mapreduce/wrapper/HadoopInputSplit.java | 27 + .../org/apache/flink/client/CliFrontend.java | 51 +- .../apache/flink/client/LocalExecutor.java | 68 +- .../apache/flink/client/program/Client.java | 56 +- .../flink/client/web/JobsInfoServlet.java | 12 +- .../client/CliFrontendListCancelTest.java | 15 +- .../flink/client/program/ClientTest.java | 32 +- .../flink/api/common/JobExecutionResult.java | 4 +- .../api/common/accumulators/Accumulator.java | 2 +- .../accumulators/AccumulatorHelper.java | 4 +- .../api/common/accumulators/Histogram.java | 10 +- .../accumulators/SimpleAccumulator.java | 5 +- .../api/common/functions/RuntimeContext.java | 5 +- .../flink/api/common/io/FileOutputFormat.java | 12 +- .../EmptyFieldsCountAccumulator.java | 12 +- .../apache/flink/runtime/blob/BlobKey.java | 23 +- .../client/{JobClient.java => JobClient2.old} | 23 +- .../executiongraph/ExecutionGraph.java | 7 +- .../runtime/io/network/RemoteReceiver.java | 3 +- .../flink/runtime/jobgraph/JobGraph.java | 40 - .../accumulators/AccumulatorManager.java | 21 +- .../NoResourceAvailableException.java | 15 +- .../runtime/minicluster/FlinkMiniCluster.java | 24 +- .../minicluster/LocalFlinkMiniCluster.java | 65 +- .../flink/runtime/taskmanager/Task.java | 3 +- .../apache/flink/runtime/akka/AkkaUtils.scala | 24 +- .../serialization/WritableSerializer.scala | 62 ++ .../flink/runtime/client/JobClient.scala | 139 ++++ .../runtime/jobmanager/EventCollector.scala | 205 ----- .../flink/runtime/jobmanager/JobInfo.scala | 38 + .../flink/runtime/jobmanager/JobManager.scala | 223 +++--- .../runtime/jobmanager/MemoryArchivist.scala | 33 - .../runtime/jobmanager/RunningJob.scala} | 7 +- .../runtime/messages/ArchiveMessages.scala | 2 - .../messages/EventCollectorMessages.scala | 60 -- .../messages/ExecutionGraphMessages.scala | 9 +- ...obResult.scala => JobClientMessages.scala} | 23 +- .../runtime/messages/JobmanagerMessages.scala | 52 +- .../taskmanager/TaskManagerProfiler.scala | 3 +- .../flink/runtime/blob/BlobKeyTest.java | 5 +- .../src/test/resources/logback-test.xml | 20 +- .../CoLocationConstraintITCase.scala | 15 +- .../runtime/jobmanager/JobManagerITCase.scala | 704 +++++++++--------- .../jobmanager/SlotSharingITCase.scala | 52 +- .../jobmanager/TaskManagerFailsITCase.scala | 47 +- ...askManagerFailsWithSlotSharingITCase.scala | 45 +- .../runtime/testingUtils/TestingCluster.scala | 8 +- .../testingUtils/TestingEventCollector.scala | 61 -- .../testingUtils/TestingJobManager.scala | 49 +- .../runtime/testingUtils/TestingUtils.scala | 4 +- .../flink/test/util/AbstractTestBase.java | 3 +- .../flink/test/util/JavaProgramTestBase.java | 10 +- .../flink/test/util/RecordAPITestBase.java | 6 +- .../test/accumulators/AccumulatorITCase.java | 8 +- .../test/cancelling/CancellingTestBase.java | 149 +--- .../test/iterative/nephele/JobGraphUtils.java | 6 - .../wordcount/WordCountAccumulators.java | 8 +- .../flink/test/util/FailingTestBase.java | 11 +- .../src/test/resources/log4j.properties | 2 +- .../src/test/resources/logback-test.xml | 2 +- pom.xml | 12 + 62 files changed, 1256 insertions(+), 1416 deletions(-) rename flink-runtime/src/main/java/org/apache/flink/runtime/client/{JobClient.java => JobClient2.old} (93%) create mode 100644 flink-runtime/src/main/scala/org/apache/flink/runtime/akka/serialization/WritableSerializer.scala create mode 100644 flink-runtime/src/main/scala/org/apache/flink/runtime/client/JobClient.scala delete mode 100644 flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/EventCollector.scala create mode 100644 flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobInfo.scala rename flink-runtime/src/main/{java/org/apache/flink/runtime/ExecutionMode.java => scala/org/apache/flink/runtime/jobmanager/RunningJob.scala} (80%) delete mode 100644 flink-runtime/src/main/scala/org/apache/flink/runtime/messages/EventCollectorMessages.scala rename flink-runtime/src/main/scala/org/apache/flink/runtime/messages/{JobResult.scala => JobClientMessages.scala} (58%) delete mode 100644 flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingEventCollector.scala diff --git a/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopInputSplit.java b/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopInputSplit.java index cf36a9dd3ab..3fb66c20434 100644 --- a/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopInputSplit.java +++ b/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopInputSplit.java @@ -20,6 +20,8 @@ package org.apache.flink.hadoopcompatibility.mapred.wrapper; import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; import org.apache.flink.core.io.InputSplit; import org.apache.flink.core.memory.DataInputView; @@ -87,6 +89,30 @@ public class HadoopInputSplit implements InputSplit { } + private void writeObject(ObjectOutputStream out) throws IOException { + out.writeInt(splitNumber); + out.writeUTF(hadoopInputSplitTypeName); + hadoopInputSplit.write(out); + + } + + @SuppressWarnings("unchecked") + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + this.splitNumber=in.readInt(); + this.hadoopInputSplitTypeName = in.readUTF(); + if(hadoopInputSplit == null) { + try { + Class inputSplit = + Class.forName(hadoopInputSplitTypeName).asSubclass(org.apache.hadoop.io.Writable.class); + this.hadoopInputSplit = (org.apache.hadoop.mapred.InputSplit) WritableFactories.newInstance( inputSplit ); + } + catch (Exception e) { + throw new RuntimeException("Unable to create InputSplit", e); + } + } + this.hadoopInputSplit.readFields(in); + } + @Override public int getSplitNumber() { return this.splitNumber; diff --git a/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapreduce/wrapper/HadoopInputSplit.java b/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapreduce/wrapper/HadoopInputSplit.java index 25cd0d8900c..5afd89bee8e 100644 --- a/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapreduce/wrapper/HadoopInputSplit.java +++ b/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapreduce/wrapper/HadoopInputSplit.java @@ -20,6 +20,8 @@ package org.apache.flink.hadoopcompatibility.mapreduce.wrapper; import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; import org.apache.flink.core.io.InputSplit; import org.apache.flink.core.memory.DataInputView; @@ -78,6 +80,31 @@ public class HadoopInputSplit implements InputSplit { } ((Writable)this.mapreduceInputSplit).readFields(in); } + + private void writeObject(ObjectOutputStream out) throws IOException { + out.writeInt(this.splitNumber); + out.writeUTF(this.mapreduceInputSplit.getClass().getName()); + Writable w = (Writable) this.mapreduceInputSplit; + w.write(out); + + } + + @SuppressWarnings("unchecked") + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + this.splitNumber=in.readInt(); + String className = in.readUTF(); + + if(this.mapreduceInputSplit == null) { + try { + Class inputSplit = + Class.forName(className).asSubclass(org.apache.hadoop.io.Writable.class); + this.mapreduceInputSplit = (org.apache.hadoop.mapreduce.InputSplit) WritableFactories.newInstance(inputSplit); + } catch (Exception e) { + throw new RuntimeException("Unable to create InputSplit", e); + } + } + ((Writable)this.mapreduceInputSplit).readFields(in); + } @Override public int getSplitNumber() { diff --git a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java index 2ee0f8fa749..c9d8051c0dd 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java +++ b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java @@ -35,6 +35,7 @@ import java.util.Map; import java.util.Properties; import akka.actor.ActorRef; +import akka.actor.ActorSystem; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.CommandLineParser; import org.apache.commons.cli.HelpFormatter; @@ -54,12 +55,13 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.runtime.akka.AkkaUtils; -import org.apache.flink.runtime.event.job.RecentJobEvent; +import org.apache.flink.runtime.jobmanager.RunningJob; import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobmanager.JobManager; -import org.apache.flink.runtime.messages.EventCollectorMessages; -import org.apache.flink.runtime.messages.JobManagerMessages; +import org.apache.flink.runtime.messages.JobManagerMessages.CancelJob; +import org.apache.flink.runtime.messages.JobManagerMessages.RequestRunningJobs$; +import org.apache.flink.runtime.messages.JobManagerMessages.RunningJobsResponse; import org.apache.flink.util.StringUtils; /** @@ -508,34 +510,34 @@ public class CliFrontend { return 1; } - List recentJobs = AkkaUtils.ask(jobManager, - EventCollectorMessages.RequestRecentJobEvents$.MODULE$).asJavaList(); + List jobs = AkkaUtils.ask(jobManager, + RequestRunningJobs$.MODULE$).asJavaList(); - ArrayList runningJobs = null; - ArrayList scheduledJobs = null; + ArrayList runningJobs = null; + ArrayList scheduledJobs = null; if (running) { - runningJobs = new ArrayList(); + runningJobs = new ArrayList(); } if (scheduled) { - scheduledJobs = new ArrayList(); + scheduledJobs = new ArrayList(); } - for (RecentJobEvent rje : recentJobs) { + for (RunningJob rj : jobs) { - if (running && rje.getJobStatus().equals(JobStatus.RUNNING)) { - runningJobs.add(rje); + if (running && rj.jobStatus().equals(JobStatus.RUNNING)) { + runningJobs.add(rj); } - if (scheduled && rje.getJobStatus().equals(JobStatus.CREATED)) { - scheduledJobs.add(rje); + if (scheduled && rj.jobStatus().equals(JobStatus.CREATED)) { + scheduledJobs.add(rj); } } SimpleDateFormat df = new SimpleDateFormat("dd.MM.yyyy HH:mm:ss"); - Comparator njec = new Comparator(){ + Comparator njec = new Comparator(){ @Override - public int compare(RecentJobEvent o1, RecentJobEvent o2) { - return (int)(o1.getTimestamp()-o2.getTimestamp()); + public int compare(RunningJob o1, RunningJob o2) { + return (int)(o1.timestamp()-o2.timestamp()); } }; @@ -546,8 +548,9 @@ public class CliFrontend { Collections.sort(runningJobs, njec); System.out.println("------------------------ Running Jobs ------------------------"); - for(RecentJobEvent je : runningJobs) { - System.out.println(df.format(new Date(je.getTimestamp()))+" : "+je.getJobID().toString()+" : "+je.getJobName()); + for(RunningJob rj : runningJobs) { + System.out.println(df.format(new Date(rj.timestamp()))+" : "+rj + .jobID().toString()+" : "+rj.jobName()); } System.out.println("--------------------------------------------------------------"); } @@ -559,8 +562,9 @@ public class CliFrontend { Collections.sort(scheduledJobs, njec); System.out.println("----------------------- Scheduled Jobs -----------------------"); - for(RecentJobEvent je : scheduledJobs) { - System.out.println(df.format(new Date(je.getTimestamp()))+" : "+je.getJobID().toString()+" : "+je.getJobName()); + for(RunningJob rj : scheduledJobs) { + System.out.println(df.format(new Date(rj.timestamp()))+" : "+rj.jobID() + .toString()+" : "+rj.jobName()); } System.out.println("--------------------------------------------------------------"); } @@ -627,7 +631,7 @@ public class CliFrontend { return 1; } - AkkaUtils.ask(jobManager, new JobManagerMessages.CancelJob(jobId)); + AkkaUtils.ask(jobManager, new CancelJob(jobId)); return 0; } catch (Throwable t) { @@ -750,7 +754,8 @@ public class CliFrontend { return null; } - return JobManager.getJobManager(jobManagerAddress); + return JobManager.getJobManager(jobManagerAddress, + ActorSystem.create("CliFrontendActorSystem", AkkaUtils.getDefaultActorSystemConfig())); } diff --git a/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java b/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java index b07c96740a0..c63675702ca 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java +++ b/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java @@ -21,11 +21,13 @@ package org.apache.flink.client; import java.util.List; +import akka.actor.ActorRef; import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.PlanExecutor; import org.apache.flink.api.common.Program; +import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.client.JobClient; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -54,22 +56,13 @@ public class LocalExecutor extends PlanExecutor { // ---------------------------------- config options ------------------------------------------ - private int jobManagerRpcPort = -1; - - private int taskManagerRpcPort = -1; - - private int taskManagerDataPort = -1; private int taskManagerNumSlots = DEFAULT_TASK_MANAGER_NUM_SLOTS; private String configDir; - private String hdfsConfigFile; - private boolean defaultOverwriteFiles = DEFAULT_OVERWRITE; - private boolean defaultAlwaysCreateDirectory = false; - // -------------------------------------------------------------------------------------------- public LocalExecutor() { @@ -78,45 +71,7 @@ public class LocalExecutor extends PlanExecutor { } } - public int getJobManagerRpcPort() { - return jobManagerRpcPort; - } - - public void setJobManagerRpcPort(int jobManagerRpcPort) { - this.jobManagerRpcPort = jobManagerRpcPort; - } - - public int getTaskManagerRpcPort() { - return taskManagerRpcPort; - } - - public void setTaskManagerRpcPort(int taskManagerRpcPort) { - this.taskManagerRpcPort = taskManagerRpcPort; - } - - public int getTaskManagerDataPort() { - return taskManagerDataPort; - } - - public void setTaskManagerDataPort(int taskManagerDataPort) { - this.taskManagerDataPort = taskManagerDataPort; - } - - public String getConfigDir() { - return configDir; - } - - public void setConfigDir(String configDir) { - this.configDir = configDir; - } - public String getHdfsConfig() { - return hdfsConfigFile; - } - - public void setHdfsConfig(String hdfsConfig) { - this.hdfsConfigFile = hdfsConfig; - } public boolean isDefaultOverwriteFiles() { return defaultOverwriteFiles; @@ -126,14 +81,6 @@ public class LocalExecutor extends PlanExecutor { this.defaultOverwriteFiles = defaultOverwriteFiles; } - public boolean isDefaultAlwaysCreateDirectory() { - return defaultAlwaysCreateDirectory; - } - - public void setDefaultAlwaysCreateDirectory(boolean defaultAlwaysCreateDirectory) { - this.defaultAlwaysCreateDirectory = defaultAlwaysCreateDirectory; - } - public void setTaskManagerNumSlots(int taskManagerNumSlots) { this.taskManagerNumSlots = taskManagerNumSlots; } public int getTaskManagerNumSlots() { return this.taskManagerNumSlots; } @@ -147,7 +94,8 @@ public class LocalExecutor extends PlanExecutor { // create the embedded runtime this.flink = new LocalFlinkMiniCluster(configDir); Configuration configuration = new Configuration(); - + configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, getTaskManagerNumSlots()); + configuration.setBoolean(ConfigConstants.FILESYSTEM_DEFAULT_OVERWRITE_KEY, isDefaultOverwriteFiles()); // start it up this.flink.start(configuration); } else { @@ -216,10 +164,10 @@ public class LocalExecutor extends PlanExecutor { NepheleJobGraphGenerator jgg = new NepheleJobGraphGenerator(); JobGraph jobGraph = jgg.compileJobGraph(op); - - JobClient jobClient = this.flink.getJobClient(jobGraph); - JobExecutionResult result = jobClient.submitJobAndWait(); - return result; + + ActorRef jobClient = flink.getJobClient(); + + return JobClient.submitJobAndWait(jobGraph, true, jobClient); } finally { if (shutDownAtEnd) { diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java b/flink-clients/src/main/java/org/apache/flink/client/program/Client.java index 415d8e89fae..a01223dcb03 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/Client.java @@ -25,6 +25,10 @@ import java.io.PrintStream; import java.net.InetSocketAddress; import java.util.List; +import akka.actor.ActorRef; +import akka.actor.ActorSystem; +import org.apache.flink.runtime.messages.JobManagerMessages.SubmissionFailure; +import org.apache.flink.runtime.messages.JobManagerMessages.SubmissionResponse; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.flink.api.common.JobExecutionResult; @@ -44,8 +48,6 @@ import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.client.JobClient; import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.messages.JobResult; -import org.apache.flink.runtime.messages.JobResult.JobSubmissionResult; import com.google.common.base.Preconditions; @@ -64,8 +66,6 @@ public class Client { private final PactCompiler compiler; // the compiler to compile the jobs - private final ClassLoader userCodeClassLoader; - private boolean printStatusDuringExecution; // ------------------------------------------------------------------------ @@ -81,10 +81,10 @@ public class Client { public Client(InetSocketAddress jobManagerAddress, Configuration config, ClassLoader userCodeClassLoader) { Preconditions.checkNotNull(config, "Configuration is null"); this.configuration = config; - configuration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, jobManagerAddress.getAddress().getHostAddress()); + configuration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, + jobManagerAddress.getAddress().getCanonicalHostName()); configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerAddress.getPort()); - this.userCodeClassLoader = userCodeClassLoader; this.compiler = new PactCompiler(new DataStatistics(), new DefaultCostEstimator()); } @@ -109,7 +109,6 @@ public class Client { throw new CompilerException("Cannot find port to job manager's RPC service in the global configuration."); } - this.userCodeClassLoader = userCodeClassLoader; this.compiler = new PactCompiler(new DataStatistics(), new DefaultCostEstimator()); } @@ -295,39 +294,50 @@ public class Client { } public JobExecutionResult run(JobGraph jobGraph, boolean wait) throws ProgramInvocationException { - JobClient client; - try { - client = new JobClient(jobGraph, configuration, this.userCodeClassLoader); + Tuple2 pair = JobClient.startActorSystemAndActor(configuration); + + ActorRef client = pair._2(); + + String hostname = configuration.getString(ConfigConstants + .JOB_MANAGER_IPC_ADDRESS_KEY, null); + + if(hostname == null){ + throw new ProgramInvocationException("Could not find hostname of job manager."); } - catch (IOException e) { - throw new ProgramInvocationException("Could not open job manager: " + e.getMessage()); + + try { + JobClient.uploadJarFiles(jobGraph, hostname, client); + }catch(IOException e){ + throw new ProgramInvocationException("Could not upload blobs.", e); } - - client.setConsoleStreamForReporting(this.printStatusDuringExecution ? System.out : null); try { + if (wait) { - return client.submitJobAndWait(); + return JobClient.submitJobAndWait(jobGraph, printStatusDuringExecution, client); } else { - JobSubmissionResult result = client.submitJob(); - - if (result.returnCode() != JobResult.SUCCESS()) { - throw new ProgramInvocationException("The job was not successfully submitted to the nephele job manager" - + (result.description() == null ? "." : ": " + result.description())); + SubmissionResponse response =JobClient.submitJobDetached(jobGraph, + printStatusDuringExecution, client); + + if(response instanceof SubmissionFailure){ + SubmissionFailure failure = (SubmissionFailure) response; + throw new ProgramInvocationException("The job was not successfully submitted " + + "to the flink job manager", failure.cause()); } } } - catch (IOException e) { - throw new ProgramInvocationException("Could not submit job to job manager: " + e.getMessage()); - } catch (JobExecutionException jex) { if(jex.isJobCanceledByUser()) { throw new ProgramInvocationException("The program has been canceled"); } else { throw new ProgramInvocationException("The program execution failed: " + jex.getMessage()); } + }finally{ + pair._1().shutdown(); + pair._1().awaitTermination(); } + return new JobExecutionResult(-1, null); } diff --git a/flink-clients/src/main/java/org/apache/flink/client/web/JobsInfoServlet.java b/flink-clients/src/main/java/org/apache/flink/client/web/JobsInfoServlet.java index 6186a5d7a5c..4c0d5c394c6 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/web/JobsInfoServlet.java +++ b/flink-clients/src/main/java/org/apache/flink/client/web/JobsInfoServlet.java @@ -31,12 +31,12 @@ import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; import akka.actor.ActorRef; +import akka.actor.ActorSystem; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.event.job.RecentJobEvent; import org.apache.flink.runtime.jobmanager.JobManager; -import org.apache.flink.runtime.messages.EventCollectorMessages; public class JobsInfoServlet extends HttpServlet { @@ -48,9 +48,13 @@ public class JobsInfoServlet extends HttpServlet { // ------------------------------------------------------------------------ private final Configuration config; + + private final ActorSystem system; public JobsInfoServlet(Configuration nepheleConfig) { this.config = nepheleConfig; + system = ActorSystem.create("JobsInfoServletActorSystem", + AkkaUtils.getDefaultActorSystemConfig()); } @Override @@ -62,10 +66,10 @@ public class JobsInfoServlet extends HttpServlet { int jmPort = config.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT); - ActorRef jm = JobManager.getJobManager(new InetSocketAddress(jmHost, jmPort)); + ActorRef jm = JobManager.getJobManager(new InetSocketAddress(jmHost, jmPort), system); - List recentJobs = AkkaUtils.ask(jm, - EventCollectorMessages.RequestRecentJobEvents$.MODULE$).asJavaList(); + // TODO: fix + List recentJobs = null; ArrayList jobs = new ArrayList(recentJobs); diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java index f88ae60f613..c57a5fbe1c7 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java @@ -25,19 +25,17 @@ import static org.junit.Assert.fail; import akka.actor.ActorRef; import akka.actor.ActorSystem; import akka.actor.Props; +import akka.actor.Status; import akka.actor.UntypedActor; import akka.testkit.JavaTestKit; import org.apache.commons.cli.CommandLine; -import org.apache.flink.runtime.event.job.RecentJobEvent; +import org.apache.flink.runtime.jobmanager.RunningJob; import org.apache.flink.runtime.jobgraph.JobID; -import org.apache.flink.runtime.messages.EventCollectorMessages; import org.apache.flink.runtime.messages.JobManagerMessages; -import org.apache.flink.runtime.messages.JobResult; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; - -import java.util.ArrayList; +import scala.collection.Seq; //TODO: Update test case public class CliFrontendListCancelTest { @@ -162,12 +160,13 @@ public class CliFrontendListCancelTest { public void onReceive(Object message) throws Exception { if(message instanceof JobManagerMessages.RequestAvailableSlots$){ getSender().tell(1, getSelf()); - }else if(message instanceof EventCollectorMessages.RequestRecentJobEvents$) { - getSender().tell(new EventCollectorMessages.RecentJobs(new ArrayList()), getSelf()); }else if(message instanceof JobManagerMessages.CancelJob){ JobManagerMessages.CancelJob cancelJob = (JobManagerMessages.CancelJob) message; assertEquals(jobID, cancelJob.jobID()); - getSender().tell(new JobResult.JobCancelResult(JobResult.SUCCESS(), null), getSelf()); + getSender().tell(new Status.Success(new Object()), getSelf()); + }else if(message instanceof JobManagerMessages.RequestRunningJobs$){ + getSender().tell(new JobManagerMessages.RunningJobsResponse(), + getSelf()); } } } diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java index 70138d8472b..00a797708fb 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java @@ -18,6 +18,8 @@ package org.apache.flink.client.program; +import akka.actor.ActorRef; +import akka.actor.ActorSystem; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.Plan; @@ -28,10 +30,9 @@ import org.apache.flink.compiler.plan.OptimizedPlan; import org.apache.flink.compiler.plantranslate.NepheleJobGraphGenerator; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.client.JobClient; +import org.apache.flink.runtime.client.JobClient$; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.messages.JobResult; -import org.apache.flink.runtime.messages.JobResult.JobSubmissionResult; +import org.apache.flink.runtime.messages.JobManagerMessages; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -40,6 +41,8 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; +import org.powermock.reflect.Whitebox; +import scala.Tuple2; import java.io.IOException; @@ -57,7 +60,7 @@ import static org.powermock.api.mockito.PowerMockito.whenNew; * Simple and maybe stupid test to check the {@link Client} class. */ @RunWith(PowerMockRunner.class) -@PrepareForTest(Client.class) +@PrepareForTest({Client.class, JobClient$.class}) public class ClientTest { @Mock Configuration configMock; @@ -68,8 +71,11 @@ public class ClientTest { @Mock OptimizedPlan optimizedPlanMock; @Mock NepheleJobGraphGenerator generatorMock; @Mock JobGraph jobGraphMock; - @Mock JobClient jobClientMock; - @Mock JobSubmissionResult jobSubmissionResultMock; + @Mock ActorSystem mockSystem; + @Mock JobClient$ mockJobClient; + @Mock JobManagerMessages.SubmissionSuccess mockSubmissionSuccess; + @Mock JobManagerMessages.SubmissionFailure mockSubmissionFailure; + @Mock ActorRef mockJobClientActor; @Before public void setUp() throws Exception { @@ -90,14 +96,16 @@ public class ClientTest { whenNew(NepheleJobGraphGenerator.class).withNoArguments().thenReturn(generatorMock); when(generatorMock.compileJobGraph(optimizedPlanMock)).thenReturn(jobGraphMock); - whenNew(JobClient.class).withArguments(any(JobGraph.class), any(Configuration.class), any(ClassLoader.class)).thenReturn(this.jobClientMock); + Whitebox.setInternalState(JobClient$.class, mockJobClient); - when(this.jobClientMock.submitJob()).thenReturn(jobSubmissionResultMock); + when(mockJobClient.startActorSystemAndActor(configMock)).thenReturn(new Tuple2(mockSystem, mockJobClientActor)); } @Test public void shouldSubmitToJobClient() throws ProgramInvocationException, IOException { - when(jobSubmissionResultMock.returnCode()).thenReturn(JobResult.SUCCESS()); + when(mockJobClient.submitJobDetached(any(JobGraph.class), any(boolean.class), + any(ActorRef.class))).thenReturn(mockSubmissionSuccess); Client out = new Client(configMock, getClass().getClassLoader()); out.run(program.getPlanWithJars(), -1, false); @@ -105,18 +113,16 @@ public class ClientTest { verify(this.compilerMock, times(1)).compile(planMock); verify(this.generatorMock, times(1)).compileJobGraph(optimizedPlanMock); - verify(this.jobClientMock, times(1)).submitJob(); } @Test(expected = ProgramInvocationException.class) public void shouldThrowException() throws Exception { - when(jobSubmissionResultMock.returnCode()).thenReturn(JobResult.ERROR()); + when(mockJobClient.submitJobDetached(any(JobGraph.class), any(boolean.class), + any(ActorRef.class))).thenReturn(mockSubmissionFailure); Client out = new Client(configMock, getClass().getClassLoader()); out.run(program.getPlanWithJars(), -1, false); program.deleteExtractedLibraries(); - - verify(this.jobClientMock).submitJob(); } @Test(expected = InvalidProgramException.class) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java b/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java index e0582c20160..dd0f057a0f5 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java @@ -22,10 +22,10 @@ package org.apache.flink.api.common; import java.util.Map; public class JobExecutionResult { - + private long netRuntime; private Map accumulatorResults; - + public JobExecutionResult(long netRuntime, Map accumulators) { this.netRuntime = netRuntime; this.accumulatorResults = accumulators; diff --git a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/Accumulator.java b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/Accumulator.java index be78b89c161..f1aac4da7cd 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/Accumulator.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/Accumulator.java @@ -41,7 +41,7 @@ import java.io.Serializable; * Type of the accumulator result as it will be reported to the * client */ -public interface Accumulator extends Serializable, Cloneable{ +public interface Accumulator extends Serializable, Cloneable{ /** * @param value diff --git a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/AccumulatorHelper.java b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/AccumulatorHelper.java index 61947f4c0e2..da871efe45c 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/AccumulatorHelper.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/AccumulatorHelper.java @@ -19,6 +19,7 @@ package org.apache.flink.api.common.accumulators; +import java.io.Serializable; import java.util.HashMap; import java.util.Map; @@ -53,7 +54,8 @@ public class AccumulatorHelper { /** * Workaround method for type safety */ - private static final void mergeSingle(Accumulator target, Accumulator toMerge) { + private static final void mergeSingle(Accumulator target, + Accumulator toMerge) { @SuppressWarnings("unchecked") Accumulator typedTarget = (Accumulator) target; diff --git a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/Histogram.java b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/Histogram.java index e6c85a12607..e204b818a78 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/Histogram.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/Histogram.java @@ -33,11 +33,11 @@ import java.util.TreeMap; * Could be extended to continuous values later, but then we need to dynamically * decide about the bin size in an online algorithm (or ask the user) */ -public class Histogram implements Accumulator> { +public class Histogram implements Accumulator> { private static final long serialVersionUID = 1L; - private Map treeMap = new TreeMap(); + private TreeMap treeMap = new TreeMap(); @Override public void add(Integer value) { @@ -47,12 +47,12 @@ public class Histogram implements Accumulator> { } @Override - public Map getLocalValue() { + public TreeMap getLocalValue() { return this.treeMap; } @Override - public void merge(Accumulator> other) { + public void merge(Accumulator> other) { // Merge the values into this map for (Map.Entry entryFromOther : ((Histogram) other).getLocalValue() .entrySet()) { @@ -93,7 +93,7 @@ public class Histogram implements Accumulator> { } @Override - public Accumulator> clone() { + public Accumulator> clone() { Histogram result = new Histogram(); result.treeMap = new TreeMap(treeMap); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/SimpleAccumulator.java b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/SimpleAccumulator.java index abc8534293e..596bf63c1a3 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/SimpleAccumulator.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/SimpleAccumulator.java @@ -19,9 +19,10 @@ package org.apache.flink.api.common.accumulators; +import java.io.Serializable; + /** * Similar to Accumulator, but the type of items to add and the result value * must be the same. */ -public interface SimpleAccumulator extends Accumulator { -} +public interface SimpleAccumulator extends Accumulator {} diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java index 7459a5a1b69..ab938c02865 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java @@ -18,6 +18,7 @@ package org.apache.flink.api.common.functions; +import java.io.Serializable; import java.util.HashMap; import java.util.List; @@ -77,7 +78,7 @@ public interface RuntimeContext { * This is only needed to support generic accumulators (e.g. for * Set). Didn't find a way to get this work with getAccumulator. */ - void addAccumulator(String name, Accumulator accumulator); + void addAccumulator(String name, Accumulator accumulator); /** * Get an existing accumulator object. The accumulator must have been added @@ -86,7 +87,7 @@ public interface RuntimeContext { * Throws an exception if the accumulator does not exist or if the * accumulator exists, but with different type. */ - Accumulator getAccumulator(String name); + Accumulator getAccumulator(String name); /** * For system internal usage only. Use getAccumulator(...) to obtain a diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/FileOutputFormat.java b/flink-core/src/main/java/org/apache/flink/api/common/io/FileOutputFormat.java index bc7ab73ad1a..063621d6f3c 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/io/FileOutputFormat.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/io/FileOutputFormat.java @@ -62,20 +62,22 @@ public abstract class FileOutputFormat implements OutputFormat, Initiali private static OutputDirectoryMode DEFAULT_OUTPUT_DIRECTORY_MODE; - private static final void initDefaultsFromConfiguration() { - final boolean overwrite = GlobalConfiguration.getBoolean(ConfigConstants.FILESYSTEM_DEFAULT_OVERWRITE_KEY, - ConfigConstants.DEFAULT_FILESYSTEM_OVERWRITE); + private static final void initDefaultsFromConfiguration(Configuration configuration) { + final boolean overwrite = configuration.getBoolean(ConfigConstants + .FILESYSTEM_DEFAULT_OVERWRITE_KEY, + ConfigConstants.DEFAULT_FILESYSTEM_OVERWRITE); DEFAULT_WRITE_MODE = overwrite ? WriteMode.OVERWRITE : WriteMode.NO_OVERWRITE; - final boolean alwaysCreateDirectory = GlobalConfiguration.getBoolean(ConfigConstants.FILESYSTEM_OUTPUT_ALWAYS_CREATE_DIRECTORY_KEY, + final boolean alwaysCreateDirectory = configuration.getBoolean(ConfigConstants + .FILESYSTEM_OUTPUT_ALWAYS_CREATE_DIRECTORY_KEY, ConfigConstants.DEFAULT_FILESYSTEM_ALWAYS_CREATE_DIRECTORY); DEFAULT_OUTPUT_DIRECTORY_MODE = alwaysCreateDirectory ? OutputDirectoryMode.ALWAYS : OutputDirectoryMode.PARONLY; } static { - initDefaultsFromConfiguration(); + initDefaultsFromConfiguration(GlobalConfiguration.getConfiguration()); } // -------------------------------------------------------------------------------------------- diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/EmptyFieldsCountAccumulator.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/EmptyFieldsCountAccumulator.java index 0cd261de05b..41929107656 100644 --- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/EmptyFieldsCountAccumulator.java +++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/EmptyFieldsCountAccumulator.java @@ -189,16 +189,16 @@ public class EmptyFieldsCountAccumulator { * increase the n-th vector component by 1, whereat n is the methods parameter. The size of the vector * is automatically managed. */ - public static class VectorAccumulator implements Accumulator> { + public static class VectorAccumulator implements Accumulator> { /** Stores the accumulated vector components. */ - private final List resultVector; + private final ArrayList resultVector; public VectorAccumulator(){ this(new ArrayList()); } - public VectorAccumulator(List resultVector){ + public VectorAccumulator(ArrayList resultVector){ this.resultVector = resultVector; } @@ -225,7 +225,7 @@ public class EmptyFieldsCountAccumulator { } @Override - public List getLocalValue() { + public ArrayList getLocalValue() { return this.resultVector; } @@ -236,7 +236,7 @@ public class EmptyFieldsCountAccumulator { } @Override - public void merge(final Accumulator> other) { + public void merge(final Accumulator> other) { // merge two vector accumulators by adding their up their vector components final List otherVector = other.getLocalValue(); for (int index = 0; index < otherVector.size(); index++) { @@ -265,7 +265,7 @@ public class EmptyFieldsCountAccumulator { } @Override - public Accumulator> clone() { + public Accumulator> clone() { VectorAccumulator result = new VectorAccumulator(new ArrayList(resultVector)); return result; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobKey.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobKey.java index 31250625e08..c12fd23b2b0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobKey.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobKey.java @@ -23,17 +23,14 @@ import java.io.EOFException; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.io.Serializable; import java.security.MessageDigest; import java.util.Arrays; -import org.apache.flink.core.io.IOReadableWritable; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; - /** * A BLOB key uniquely identifies a BLOB. */ -public final class BlobKey implements IOReadableWritable, Comparable { +public final class BlobKey implements Serializable, Comparable { /** * Array of hex characters to facilitate fast toString() method. @@ -72,22 +69,6 @@ public final class BlobKey implements IOReadableWritable, Comparable { this.key = key; } - /** - * {@inheritDoc} - */ - @Override - public void write(final DataOutputView out) throws IOException { - out.write(this.key); - } - - /** - * {@inheritDoc} - */ - @Override - public void read(final DataInputView in) throws IOException { - in.readFully(this.key); - } - /** * {@inheritDoc} */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient2.old similarity index 93% rename from flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient2.old index afa0667765d..aaab113108d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient2.old @@ -24,7 +24,20 @@ import java.net.InetSocketAddress; import java.util.Iterator; import java.util.Map; +import akka.actor.ActorPath; import akka.actor.ActorRef; +import akka.actor.ActorRefProvider; +import akka.actor.ActorSystem; +import akka.actor.ActorSystemImpl; +import akka.actor.Extension; +import akka.actor.ExtensionId; +import akka.actor.InternalActorRef; +import akka.actor.Props; +import akka.actor.Scheduler; +import akka.dispatch.Dispatchers; +import akka.dispatch.Mailboxes; +import akka.event.EventStream; +import akka.event.LoggingAdapter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.flink.api.common.JobExecutionResult; @@ -45,6 +58,9 @@ import org.apache.flink.runtime.messages.JobResult.JobCancelResult; import org.apache.flink.runtime.messages.JobResult.JobSubmissionResult; import org.apache.flink.runtime.messages.JobResult.JobProgressResult; import org.apache.flink.util.StringUtils; +import scala.Function0; +import scala.concurrent.ExecutionContextExecutor; +import scala.concurrent.duration.Duration; /** * The job client is able to submit, control, and abort jobs. @@ -107,7 +123,9 @@ public class JobClient { ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT); final InetSocketAddress inetaddr = new InetSocketAddress(address, port); - this.jobManager = JobManager.getJobManager(inetaddr); + ActorSystem system = ActorSystem.create("JobClientActorSystem", + AkkaUtils.getDefaultActorSystemConfig()); + this.jobManager = JobManager.getJobManager(inetaddr, system); this.jobGraph = jobGraph; this.configuration = configuration; this.userCodeClassLoader = userCodeClassLoader; @@ -222,7 +240,9 @@ public class JobClient { } JobResult.JobProgressResult jobProgressResult = null; + LOG.info("Request job progress."); jobProgressResult = getJobProgress(); + LOG.info("Requested job progress."); if (jobProgressResult == null) { logErrorAndRethrow("Returned job progress is unexpectedly null!"); @@ -232,6 +252,7 @@ public class JobClient { logErrorAndRethrow("Could not retrieve job progress: " + jobProgressResult.description()); } + final Iterator it = jobProgressResult.asJavaList().iterator(); while (it.hasNext()) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index 275dbbdf4bb..4116f141c1c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -676,8 +676,8 @@ public class ExecutionGraph { if(jobStatusListenerActors.size() > 0){ String message = error == null ? null : ExceptionUtils.stringifyException(error); for(ActorRef listener: jobStatusListenerActors){ - listener.tell(new JobStatusChanged(this, newState, message), - ActorRef.noSender()); + listener.tell(new JobStatusChanged(jobID, newState, System.currentTimeMillis(), + message), ActorRef.noSender()); } } @@ -708,7 +708,8 @@ public class ExecutionGraph { String message = error == null ? null : ExceptionUtils.stringifyException(error); for(ActorRef listener : executionListenerActors){ listener.tell(new ExecutionGraphMessages.ExecutionStateChanged(jobID, vertexId, subtask, executionID, - newExecutionState, message), ActorRef.noSender()); + newExecutionState, System.currentTimeMillis(), message), + ActorRef.noSender()); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/RemoteReceiver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/RemoteReceiver.java index 32b835d7f29..78dcbaea9ae 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/RemoteReceiver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/RemoteReceiver.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.io.network; import java.io.IOException; +import java.io.Serializable; import java.net.InetAddress; import java.net.InetSocketAddress; @@ -29,7 +30,7 @@ import org.apache.flink.core.memory.DataOutputView; /** * Objects of this class uniquely identify a connection to a remote {@link org.apache.flink.runtime.taskmanager.TaskManager}. */ -public final class RemoteReceiver implements IOReadableWritable { +public final class RemoteReceiver implements IOReadableWritable, Serializable { /** * The address of the connection to the remote TaskManager. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java index 26a9eaf829d..2db4ff40839 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java @@ -35,8 +35,6 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.FSDataInputStream; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.Path; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.runtime.blob.BlobClient; import org.apache.flink.runtime.blob.BlobKey; @@ -335,44 +333,6 @@ public class JobGraph implements Serializable { this.numExecutionRetries = in.readInt(); out.writeInt(numExecutionRetries); - /** - * Writes the BLOB keys of the jar files required to run this job to the given {@link org.apache.flink.core.memory.DataOutputView}. - * - * @param out - * the data output to write the BLOB keys to - * @throws IOException - * thrown if an error occurs while writing to the data output - */ - private void writeJarBlobKeys(final DataOutputView out) throws IOException { - - out.writeInt(this.userJarBlobKeys.size()); - - for (BlobKey userJarBlobKey : this.userJarBlobKeys) { - userJarBlobKey.write(out); - } - } - - /** - * Reads the BLOB keys for the JAR files required to run this job and registers them. - * - * @param in - * the data stream to read the BLOB keys from - * @throws IOException - * thrown if an error occurs while reading the stream - */ - private void readJarBlobKeys(final DataInputView in) throws IOException { - - // Do jar files follow; - final int numberOfBlobKeys = in.readInt(); - - for (int i = 0; i < numberOfBlobKeys; ++i) { - final BlobKey key = new BlobKey(); - key.read(in); - this.userJarBlobKeys.add(key); - } - } - - // -------------------------------------------------------------------------------------------- // Handling of attached JAR files // -------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/accumulators/AccumulatorManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/accumulators/AccumulatorManager.java index 70a7bda2bf4..37b044a6033 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/accumulators/AccumulatorManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/accumulators/AccumulatorManager.java @@ -70,16 +70,19 @@ public class AccumulatorManager { } } - /** - * Returns all collected accumulators for the job. For efficiency the - * internal accumulator is returned, so please use it read-only. - */ - public Map> getJobAccumulators(JobID jobID) { - JobAccumulators jobAccumulators = this.jobAccumulators.get(jobID); - if (jobAccumulators == null) { - return new HashMap>(); + public Map getJobAccumulatorResults(JobID jobID) { + Map result = new HashMap(); + + JobAccumulators jobAccumulator = jobAccumulators.get(jobID); + + if(jobAccumulator != null) { + for (Map.Entry> entry : jobAccumulators.get(jobID) + .getAccumulators().entrySet()) { + result.put(entry.getKey(), entry.getValue().getLocalValue()); + } } - return jobAccumulators.getAccumulators(); + + return result; } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/NoResourceAvailableException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/NoResourceAvailableException.java index 5df7d48f06b..730952b1e9c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/NoResourceAvailableException.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/NoResourceAvailableException.java @@ -36,7 +36,7 @@ public class NoResourceAvailableException extends JobException { + ". You can decrease the operator parallelism or increase the number of slots per TaskManager in the configuration."); } - NoResourceAvailableException(int numInstances, int numSlotsTotal) { + public NoResourceAvailableException(int numInstances, int numSlotsTotal) { super(String.format("%s Resources available to scheduler: Number of instances=%d, total number of slots=%d", BASE_MESSAGE, numInstances, numSlotsTotal)); } @@ -49,4 +49,17 @@ public class NoResourceAvailableException extends JobException { public NoResourceAvailableException(String message) { super(message); } + + @Override + public boolean equals(Object obj){ + if(obj == null){ + return false; + } + + if(!(obj instanceof NoResourceAvailableException)){ + return false; + }else{ + return getMessage().equals(((NoResourceAvailableException)obj).getMessage()); + } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/FlinkMiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/FlinkMiniCluster.java index 6fb7f7716f8..f69859a51c1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/FlinkMiniCluster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/FlinkMiniCluster.java @@ -44,7 +44,7 @@ abstract public class FlinkMiniCluster { protected List taskManagerActorSystems = new ArrayList(); protected List taskManagerActors = new ArrayList(); - public abstract Configuration getConfiguration(final Configuration userConfiguration); + protected abstract Configuration generateConfiguration(final Configuration userConfiguration); public abstract ActorRef startJobManager(final ActorSystem system, final Configuration configuration); public abstract ActorRef startTaskManager(final ActorSystem system, final Configuration configuration, @@ -81,9 +81,9 @@ abstract public class FlinkMiniCluster { // ------------------------------------------------------------------------ - public void start(final Configuration configuration) throws Exception { + public void start(final Configuration configuration) { - Configuration clusterConfiguration = getConfiguration(configuration); + Configuration clusterConfiguration = generateConfiguration(configuration); jobManagerActorSystem = startJobManagerActorSystem(clusterConfiguration); jobManagerActor = startJobManager(jobManagerActorSystem, clusterConfiguration); @@ -101,22 +101,32 @@ abstract public class FlinkMiniCluster { waitForTaskManagersToBeRegistered(); } - public void stop() throws Exception { + public void stop() { LOG.info("Stopping FlinkMiniCluster."); + shutdown(); + + awaitTermination(); + + taskManagerActorSystems.clear(); + taskManagerActors.clear(); + } + + protected void shutdown() { for(ActorSystem system: taskManagerActorSystems){ system.shutdown(); } jobManagerActorSystem.shutdown(); + } + protected void awaitTermination() { for(ActorSystem system: taskManagerActorSystems){ system.awaitTermination(); } jobManagerActorSystem.awaitTermination(); - - taskManagerActorSystems.clear(); - taskManagerActors.clear(); } + + // ------------------------------------------------------------------------ // Network utility methods // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.java index 403a43e2e41..2a7997b30ed 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.java @@ -20,13 +20,12 @@ package org.apache.flink.runtime.minicluster; import akka.actor.ActorRef; import akka.actor.ActorSystem; -import org.apache.flink.api.common.io.FileInputFormat; import org.apache.flink.api.common.io.FileOutputFormat; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.client.JobClient; -import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.JobManager; import org.apache.flink.runtime.taskmanager.TaskManager; import org.apache.flink.runtime.util.EnvironmentInformation; @@ -40,6 +39,7 @@ public class LocalFlinkMiniCluster extends FlinkMiniCluster { private Configuration configuration; private final String configDir; + private ActorSystem actorSystem; public LocalFlinkMiniCluster(String configDir){ this.configDir = configDir; @@ -50,23 +50,53 @@ public class LocalFlinkMiniCluster extends FlinkMiniCluster { // Life cycle and Job Submission // ------------------------------------------------------------------------ - public JobClient getJobClient(JobGraph jobGraph) throws Exception { - if(configuration == null){ - throw new RuntimeException("The cluster has not been started yet."); + public ActorRef getJobClient() { + Configuration config = new Configuration(); + + config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, HOSTNAME); + config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, getJobManagerRPCPort()); + + return JobClient.startActorWithConfiguration(config, actorSystem); + } + + public ActorSystem getJobClientActorSystem(){ + return actorSystem; + } + + @Override + public void start(Configuration configuration){ + super.start(configuration); + + actorSystem = AkkaUtils.createActorSystem(); + } + + @Override + protected void shutdown() { + super.shutdown(); + + if(actorSystem != null){ + actorSystem.shutdown(); } + } - Configuration jobConfiguration = jobGraph.getJobConfiguration(); - int jobManagerRPCPort = getJobManagerRPCPort(); - jobConfiguration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, HOSTNAME); - jobConfiguration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerRPCPort); - return new JobClient(jobGraph, jobConfiguration, getClass().getClassLoader()); + @Override + protected void awaitTermination() { + if(actorSystem != null){ + actorSystem.awaitTermination(); + } + + super.awaitTermination(); } public int getJobManagerRPCPort() { + if(configuration == null){ + throw new RuntimeException("Configuration has not been set."); + } + return configuration.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, -1); } - public Configuration getConfiguration(final Configuration userConfiguration) { + protected Configuration generateConfiguration(final Configuration userConfiguration) { if(configuration == null){ String forkNumberString = System.getProperty("forkNumber"); int forkNumber = -1; @@ -97,7 +127,7 @@ public class LocalFlinkMiniCluster extends FlinkMiniCluster { configuration.setInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, taskManagerDATA); } - initializeIOFormatClasses(); + initializeIOFormatClasses(configuration); } return configuration; @@ -130,15 +160,12 @@ public class LocalFlinkMiniCluster extends FlinkMiniCluster { return TaskManager.startActorWithConfiguration(HOSTNAME, config, false, system); } - private static void initializeIOFormatClasses() { + private static void initializeIOFormatClasses(Configuration configuration) { try { - Method im = FileInputFormat.class.getDeclaredMethod("initDefaultsFromConfiguration"); - im.setAccessible(true); - im.invoke(null); - - Method om = FileOutputFormat.class.getDeclaredMethod("initDefaultsFromConfiguration"); + Method om = FileOutputFormat.class.getDeclaredMethod("initDefaultsFromConfiguration", + Configuration.class); om.setAccessible(true); - om.invoke(null); + om.invoke(null, configuration); } catch (Exception e) { LOG.error("Cannot (re) initialize the globally loaded defaults. Some classes might mot follow the specified default behavior."); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java index 68a0cf97a75..dcb497ad9ad 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java @@ -411,7 +411,8 @@ public final class Task { for(ActorRef listener: executionListenerActors){ listener.tell(new ExecutionGraphMessages.ExecutionStateChanged(jobId, vertexId, subtaskIndex, - executionId, newState, message), ActorRef.noSender()); + executionId, newState, System.currentTimeMillis(), message), + ActorRef.noSender()); } } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala index 75a53f131b3..c64983c8c9d 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala @@ -27,13 +27,13 @@ import akka.util.Timeout import com.typesafe.config.ConfigFactory import org.apache.flink.configuration.{ConfigConstants, Configuration} import org.apache.flink.core.io.IOReadableWritable -import org.apache.flink.runtime.akka.serialization.IOReadableWritableSerializer +import org.apache.flink.runtime.akka.serialization.{WritableSerializer, +IOReadableWritableSerializer} +import org.apache.hadoop.io.Writable import scala.concurrent.{ExecutionContext, Future, Await} import scala.concurrent.duration._ object AkkaUtils { - lazy val defaultActorSystem = ActorSystem.create("DefaultActorSystem", - ConfigFactory.parseString(getDefaultActorSystemConfigString)) implicit val FUTURE_TIMEOUT: Timeout = 1 minute implicit val AWAIT_DURATION: FiniteDuration = 1 minute implicit val FUTURE_DURATION: FiniteDuration = 1 minute @@ -47,6 +47,10 @@ object AkkaUtils { actorSystem } + def createActorSystem(): ActorSystem = { + ActorSystem.create("default", getDefaultActorSystemConfig) + } + def getConfigString(host: String, port: Int, configuration: Configuration): String = { val transportHeartbeatInterval = configuration.getString(ConfigConstants. AKKA_TRANSPORT_HEARTBEAT_INTERVAL, @@ -96,23 +100,29 @@ object AkkaUtils { def getDefaultActorSystemConfigString: String = { s"""akka.daemonic = on |akka.loggers = ["akka.event.slf4j.Slf4jLogger"] - |akka.loglevel = "INFO" + |akka.loglevel = "WARNING" |akka.logging-filter = "akka.event.slf4j.Slf4jLoggingFilter" - |akka.stdout-loglevel = "INFO" + |akka.stdout-loglevel = "WARNING" |akka.jvm-exit-on-fata-error = off |akka.actor.provider = "akka.remote.RemoteActorRefProvider" |akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport" |akka.remote.netty.tcp.tcp-nodelay = on |akka.log-config-on-start = off + |akka.remote.netty.tcp.port = 0 + |akka.remote.netty.tcp.maximum-frame-size = 1MB """.stripMargin } + def getDefaultActorSystemConfig = { + ConfigFactory.parseString(getDefaultActorSystemConfigString) + } + def getChild(parent: ActorRef, child: String)(implicit system: ActorSystem): ActorRef = { Await.result(system.actorSelection(parent.path / child).resolveOne(), AWAIT_DURATION) } - def getReference(path: String): ActorRef = { - Await.result(defaultActorSystem.actorSelection(path).resolveOne(), AWAIT_DURATION) + def getReference(path: String)(implicit system: ActorSystem): ActorRef = { + Await.result(system.actorSelection(path).resolveOne(), AWAIT_DURATION) } @throws(classOf[IOException]) diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/serialization/WritableSerializer.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/serialization/WritableSerializer.scala new file mode 100644 index 00000000000..0ac7ed6cd75 --- /dev/null +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/serialization/WritableSerializer.scala @@ -0,0 +1,62 @@ +/* + * 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.serialization + +import akka.serialization.JSerializer +import org.apache.flink.runtime.io.network.serialization.{DataInputDeserializer, +DataOutputSerializer} +import org.apache.flink.util.InstantiationUtil +import org.apache.hadoop.io.Writable + +class WritableSerializer extends JSerializer { + val INITIAL_BUFFER_SIZE = 8096 + + override protected def fromBinaryJava(bytes: Array[Byte], manifest: Class[_]): AnyRef = { + val in = new DataInputDeserializer(bytes, 0, bytes.length) + + val instance = InstantiationUtil.instantiate(manifest) + + if(!instance.isInstanceOf[Writable]){ + throw new RuntimeException(s"Class $manifest is not of type IOReadableWritable.") + } + + val writable = instance.asInstanceOf[Writable] + + writable.readFields(in) + + writable + } + + override def includeManifest: Boolean = true + + override def toBinary(o: AnyRef): Array[Byte] = { + if(!o.isInstanceOf[Writable]){ + throw new RuntimeException("Object is not of type Writable.") + } + + val writable = o.asInstanceOf[Writable] + val out = new DataOutputSerializer(INITIAL_BUFFER_SIZE) + + writable.write(out) + + out.wrapAsByteBuffer().array() + } + + override def identifier: Int = 1337 +} diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/client/JobClient.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/client/JobClient.scala new file mode 100644 index 00000000000..5a762736b8d --- /dev/null +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/client/JobClient.scala @@ -0,0 +1,139 @@ +/* + * 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.client + +import java.io.IOException +import java.net.InetSocketAddress + +import akka.actor.Status.Failure +import akka.actor._ +import akka.pattern.ask +import org.apache.flink.api.common.JobExecutionResult +import org.apache.flink.configuration.{ConfigConstants, Configuration} +import org.apache.flink.runtime.ActorLogMessages +import org.apache.flink.runtime.akka.AkkaUtils +import org.apache.flink.runtime.jobgraph.JobGraph +import org.apache.flink.runtime.jobmanager.JobManager +import org.apache.flink.runtime.messages.ExecutionGraphMessages.ExecutionStateChanged +import org.apache.flink.runtime.messages.JobClientMessages.{SubmitJobDetached, SubmitJobAndWait} +import org.apache.flink.runtime.messages.JobManagerMessages._ + +import scala.concurrent.Await +import scala.concurrent.duration.Duration + + +class JobClient(jobManagerURL: String) extends Actor with ActorLogMessages with ActorLogging{ + import context._ + + val jobManager = AkkaUtils.getReference(jobManagerURL) + + override def receiveWithLogMessages: Receive = { + case SubmitJobDetached(jobGraph, listen) => + jobManager.tell(SubmitJob(jobGraph, listenToEvents = listen, detach = true), sender()) + case cancelJob: CancelJob => + jobManager forward cancelJob + case SubmitJobAndWait(jobGraph, listen) => + val listener = context.actorOf(Props(classOf[JobClientListener], sender())) + jobManager.tell(SubmitJob(jobGraph, listenToEvents = listen, detach = false), listener) + case RequestBlobManagerPort => + jobManager forward RequestBlobManagerPort + } +} + +class JobClientListener(client: ActorRef) extends Actor with ActorLogMessages with ActorLogging { + override def receiveWithLogMessages: Receive = { + case SubmissionFailure(_, t) => + client ! Failure(t) + self ! PoisonPill + case JobResultSuccess(_, duration, accumulatorResults) => + client ! new JobExecutionResult(duration, accumulatorResults) + self ! PoisonPill + case JobResultCanceled(_, msg) => + client ! Failure(new JobExecutionException(msg, true)) + self ! PoisonPill + case JobResultFailed(_, msg) => + client ! Failure(new JobExecutionException(msg, false)) + self ! PoisonPill + case e:ExecutionStateChanged => + println(e.toString) + } +} + +object JobClient{ + val JOB_CLIENT_NAME = "jobclient" + + def startActorSystemAndActor(config: Configuration): (ActorSystem, ActorRef) = { + implicit val actorSystem = AkkaUtils.createActorSystem(host = "localhost", + port =0, configuration = config) + (actorSystem, startActorWithConfiguration(config)) + } + + def startActor(jobManagerURL: String)(implicit actorSystem: ActorSystem): ActorRef = { + actorSystem.actorOf(Props(classOf[JobClient], jobManagerURL), JOB_CLIENT_NAME) + } + + def parseConfiguration(configuration: Configuration): String = { + configuration.getString(ConfigConstants.JOB_MANAGER_AKKA_URL, null) match { + case url: String => url + case _ => + val jobManagerAddress = configuration.getString(ConfigConstants + .JOB_MANAGER_IPC_ADDRESS_KEY, null); + val jobManagerRPCPort = configuration.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, + ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT); + + if (jobManagerAddress == null) { + throw new RuntimeException("JobManager address has not been specified in the " + + "configuration.") + } + + JobManager.getAkkaURL(jobManagerAddress + ":" + jobManagerRPCPort) + } + } + + def startActorWithConfiguration(config: Configuration)(implicit actorSystem: ActorSystem): + ActorRef= { + startActor(parseConfiguration(config)) + } + + @throws(classOf[JobExecutionException]) + def submitJobAndWait(jobGraph: JobGraph, listen: Boolean, + jobClient: ActorRef): JobExecutionResult = { + import AkkaUtils.FUTURE_TIMEOUT + val response = jobClient ? SubmitJobAndWait(jobGraph, listenToEvents = listen) + + Await.result(response.mapTo[JobExecutionResult],Duration.Inf) + } + + + def submitJobDetached(jobGraph: JobGraph, listen: Boolean, jobClient: ActorRef): SubmissionResponse = { + import AkkaUtils.FUTURE_TIMEOUT + val response = jobClient ? SubmitJobDetached(jobGraph, listenToEvents = listen) + + Await.result(response.mapTo[SubmissionResponse],AkkaUtils.FUTURE_DURATION) + } + + @throws(classOf[IOException]) + def uploadJarFiles(jobGraph: JobGraph, hostname: String, jobClient: ActorRef): Unit = { + val port = AkkaUtils.ask[Int](jobClient, RequestBlobManagerPort) + + val serverAddress = new InetSocketAddress(hostname, port) + + jobGraph.uploadRequiredJarFiles(serverAddress) + } +} diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/EventCollector.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/EventCollector.scala deleted file mode 100644 index ee491a47198..00000000000 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/EventCollector.scala +++ /dev/null @@ -1,205 +0,0 @@ -/* - * 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.jobmanager - - -import akka.actor.{Terminated, ActorRef, Actor, ActorLogging} -import akka.pattern.{ask, pipe} -import org.apache.flink.runtime.ActorLogMessages -import org.apache.flink.runtime.akka.AkkaUtils -import org.apache.flink.runtime.event.job._ -import org.apache.flink.runtime.executiongraph._ -import org.apache.flink.runtime.jobgraph.{JobStatus, JobID} -import org.apache.flink.runtime.messages.ArchiveMessages.{ArchiveExecutionGraph, ArchiveJobEvent, -ArchiveEvent} -import org.apache.flink.runtime.messages.EventCollectorMessages._ -import org.apache.flink.runtime.messages.ExecutionGraphMessages.{CurrentJobStatus, JobNotFound, -JobStatusChanged, ExecutionStateChanged} -import org.apache.flink.runtime.messages.JobManagerMessages.RequestJobStatus -import org.apache.flink.runtime.messages.JobResult -import org.apache.flink.runtime.messages.JobResult.JobProgressResult -import scala.collection.convert.{WrapAsScala} -import scala.concurrent.Future -import scala.concurrent.duration._ - -class EventCollector(val timerTaskInterval: Int) extends Actor with ActorLogMessages with -ActorLogging with WrapAsScala { - - import context.dispatcher - import AkkaUtils.FUTURE_TIMEOUT - - val collectedEvents = collection.mutable.HashMap[JobID, List[AbstractEvent]]() - - val recentJobs = collection.mutable.HashMap[JobID, RecentJobEvent]() - - val recentExecutionGraphs = collection.mutable.HashMap[JobID, ExecutionGraph]() - - val archiveListeners = collection.mutable.HashSet[ActorRef]() - - val jobInformation = collection.mutable.HashMap[JobID, (String, Boolean, Long)]() - - override def preStart(): Unit = { - startArchiveExpiredEvent() - } - - override def postStop(): Unit = { - collectedEvents.clear() - recentJobs.clear() - recentExecutionGraphs.clear() - archiveListeners.clear() - jobInformation.clear() - } - - def startArchiveExpiredEvent(): Unit = { - val schedulerDuration = FiniteDuration(2 * timerTaskInterval, SECONDS) - context.system.scheduler.schedule(schedulerDuration, schedulerDuration, self, - ArchiveExpiredEvents) - } - - override def receiveWithLogMessages: Receive = { - case ArchiveExpiredEvents => { - val currentTime = System.currentTimeMillis() - - collectedEvents.retain { (jobID, events) => - val (outdatedElements, currentElements) = events.partition { event => event.getTimestamp + - timerTaskInterval < currentTime - } - - outdatedElements foreach (archiveEvent(jobID, _)) - currentElements.nonEmpty - } - - recentJobs.retain { (jobID, recentJobEvent) => - import JobStatus._ - val status = recentJobEvent.getJobStatus - - // only remove jobs which have stopped running - if ((status == FINISHED || status == CANCELED || status != FAILED) && - recentJobEvent.getTimestamp + timerTaskInterval < currentTime) { - archiveJobEvent(jobID, recentJobEvent) - archiveExecutionGraph(jobID, recentExecutionGraphs.remove(jobID).get) - jobInformation.remove(jobID) - false - } else { - true - } - } - } - case RequestJobProgress(jobID) => { - sender() ! JobProgressResult(JobResult.SUCCESS, null, collectedEvents.getOrElse(jobID, - List())) - } - case RequestRecentJobEvents => { - sender() ! RecentJobs(recentJobs.values.toList) - } - case RegisterJob(executionGraph, profilingAvailable, submissionTimestamp) => { - val jobID = executionGraph.getJobID - - executionGraph.registerExecutionListener(self) - executionGraph.registerJobStatusListener(self) - jobInformation += jobID ->(executionGraph.getJobName, profilingAvailable, submissionTimestamp) - - recentExecutionGraphs += jobID -> executionGraph - } - case ExecutionStateChanged(jobID, vertexID, subtask, executionID, newExecutionState, - optionalMessage) => { - val timestamp = System.currentTimeMillis() - - recentExecutionGraphs.get(jobID) match { - case Some(graph) => - val vertex = graph.getJobVertex(vertexID) - val taskName = if (vertex != null) vertex.getJobVertex.getName else "(null)" - val totalNumberOfSubtasks = if (vertex != null) vertex.getParallelism else -1 - - val vertexEvent = new VertexEvent(timestamp, vertexID, taskName, totalNumberOfSubtasks, - subtask, executionID, newExecutionState, optionalMessage) - - val events = collectedEvents.getOrElse(jobID, List()) - val executionStateChangeEvent = new ExecutionStateChangeEvent(timestamp, vertexID, - subtask, executionID, newExecutionState) - - collectedEvents += jobID -> (executionStateChangeEvent :: vertexEvent :: events) - case None => - log.warning(s"Could not find execution graph with jobID ${jobID}.") - } - } - case JobStatusChanged(executionGraph, newJobStatus, optionalMessage) => { - val jobID = executionGraph.getJobID() - - if (newJobStatus == JobStatus.RUNNING) { - this.recentExecutionGraphs += jobID -> executionGraph - } - - val currentTime = System.currentTimeMillis() - val (jobName, isProfilingEnabled, submissionTimestamp) = jobInformation(jobID) - recentJobs.put(jobID, new RecentJobEvent(jobID, jobName, newJobStatus, isProfilingEnabled, - submissionTimestamp, currentTime)) - - val events = collectedEvents.getOrElse(jobID, List()) - collectedEvents += jobID -> ((new JobEvent(currentTime, newJobStatus, - optionalMessage)) :: events) - } - case ProcessProfilingEvent(profilingEvent) => { - val events = collectedEvents.getOrElse(profilingEvent.getJobID, List()) - collectedEvents += profilingEvent.getJobID -> (profilingEvent :: events) - } - case RegisterArchiveListener(actorListener) => { - context.watch(actorListener) - archiveListeners += actorListener - } - case Terminated(terminatedListener) => { - archiveListeners -= terminatedListener - } - case RequestJobStatus(jobID) => { - recentJobs.get(jobID) match { - case Some(recentJobEvent) => sender() ! CurrentJobStatus(jobID, recentJobEvent.getJobStatus) - case None => - val responses = archiveListeners map { archivist => archivist ? - RequestJobStatus(jobID) filter { - case _: CurrentJobStatus => true - case _ => false - } - } - val noResponse = akka.pattern.after(AkkaUtils.FUTURE_DURATION, context.system.scheduler) { - Future.successful(JobNotFound(jobID)) - } - - Future.firstCompletedOf(responses + noResponse) pipeTo sender() - } - } - } - - private def archiveEvent(jobID: JobID, event: AbstractEvent): Unit = { - for (listener <- archiveListeners) { - listener ! ArchiveEvent(jobID, event) - } - } - - private def archiveJobEvent(jobID: JobID, event: RecentJobEvent): Unit = { - for (listener <- archiveListeners) { - listener ! ArchiveJobEvent(jobID, event) - } - } - - private def archiveExecutionGraph(jobID: JobID, graph: ExecutionGraph): Unit = { - for (listener <- archiveListeners) { - listener ! ArchiveExecutionGraph(jobID, graph) - } - } -} diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobInfo.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobInfo.scala new file mode 100644 index 00000000000..8f5d4d4f9ec --- /dev/null +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobInfo.scala @@ -0,0 +1,38 @@ +/* + * 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.jobmanager + +import akka.actor.ActorRef + +class JobInfo(val client: ActorRef,val start: Long){ + var end: Long = -1 + var detach: Boolean = false + + def duration: Long = { + if(end != -1){ + (end - start)/1000 + }else{ + -1 + } + } +} + +object JobInfo{ + def apply(client: ActorRef, start: Long) = new JobInfo(client, start) +} diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index ca79cf742a1..7d228f7b478 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -26,35 +26,31 @@ import akka.pattern.Patterns import com.google.common.base.Preconditions import org.apache.flink.configuration.{ConfigConstants, GlobalConfiguration, Configuration} import org.apache.flink.core.io.InputSplitAssigner -import org.apache.flink.runtime.accumulators.AccumulatorEvent import org.apache.flink.runtime.blob.BlobServer +import org.apache.flink.runtime.executiongraph.{ExecutionJobVertex, ExecutionGraph} import org.apache.flink.runtime.io.network.ConnectionInfoLookupResponse -import org.apache.flink.runtime.messages.ExecutionGraphMessages.{JobNotFound, CurrentJobStatus, +import org.apache.flink.runtime.messages.ArchiveMessages.ArchiveExecutionGraph +import org.apache.flink.runtime.messages.ExecutionGraphMessages.{CurrentJobStatus, JobStatusChanged} -import org.apache.flink.runtime.messages.JobResult -import org.apache.flink.runtime.messages.JobResult.{JobCancelResult, JobSubmissionResult} import org.apache.flink.runtime.{JobException, ActorLogMessages} import org.apache.flink.runtime.akka.AkkaUtils import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager -import org.apache.flink.runtime.executiongraph.{ExecutionJobVertex, ExecutionGraph} import org.apache.flink.runtime.instance.{InstanceManager} import org.apache.flink.runtime.jobgraph.{JobStatus, JobID} import org.apache.flink.runtime.jobmanager.accumulators.AccumulatorManager import org.apache.flink.runtime.jobmanager.scheduler.{Scheduler => FlinkScheduler} -import org.apache.flink.runtime.messages.EventCollectorMessages._ import org.apache.flink.runtime.messages.JobManagerMessages._ import org.apache.flink.runtime.messages.RegistrationMessages._ import org.apache.flink.runtime.messages.TaskManagerMessages.{NextInputSplit, Heartbeat} import org.apache.flink.runtime.profiling.ProfilingUtils -import org.apache.flink.util.StringUtils import org.slf4j.LoggerFactory import scala.collection.convert.WrapAsScala import scala.concurrent.Future import scala.concurrent.duration._ -class JobManager(val archiveCount: Int, val profiling: Boolean, val recommendedPollingInterval: -Int, cleanupInterval: Long) extends Actor with ActorLogMessages with ActorLogging with WrapAsScala { +class JobManager(val archiveCount: Int, val profiling: Boolean, cleanupInterval: Long) extends +Actor with ActorLogMessages with ActorLogging with WrapAsScala { import context._ @@ -62,8 +58,6 @@ Int, cleanupInterval: Long) extends Actor with ActorLogMessages with ActorLoggin def archiveProps: Props = Props(classOf[MemoryArchivist], archiveCount) - def eventCollectorProps: Props = Props(classOf[EventCollector], recommendedPollingInterval) - val profiler = profiling match { case true => Some(context.actorOf(profilerProps, JobManager.PROFILER_NAME)) case false => None @@ -71,7 +65,6 @@ Int, cleanupInterval: Long) extends Actor with ActorLogMessages with ActorLoggin // will be removed val archive = context.actorOf(archiveProps, JobManager.ARCHIVE_NAME) - val eventCollector = context.actorOf(eventCollectorProps, JobManager.EVENT_COLLECTOR_NAME) val accumulatorManager = new AccumulatorManager(Math.min(1, archiveCount)) val instanceManager = new InstanceManager() @@ -79,10 +72,8 @@ Int, cleanupInterval: Long) extends Actor with ActorLogMessages with ActorLoggin val libraryCacheManager = new BlobLibraryCacheManager(new BlobServer(), cleanupInterval) val webserver = null - val currentJobs = scala.collection.concurrent.TrieMap[JobID, ExecutionGraph]() - val jobTerminationListener = scala.collection.mutable.HashMap[JobID, Set[ActorRef]]() - - eventCollector ! RegisterArchiveListener(archive) + val currentJobs = scala.collection.mutable.HashMap[JobID, (ExecutionGraph, JobInfo)]() + val finalJobStatusListener = scala.collection.mutable.HashMap[JobID, Set[ActorRef]]() instanceManager.addInstanceListener(scheduler) @@ -112,12 +103,11 @@ Int, cleanupInterval: Long) extends Actor with ActorLogMessages with ActorLoggin sender() ! instanceManager.getTotalNumberOfSlots } - case SubmitJob(jobGraph) => { - var executionGraph: ExecutionGraph = null - + case SubmitJob(jobGraph, listenToEvents, detach) => { try { if (jobGraph == null) { - JobSubmissionResult(JobResult.ERROR, "Submitted job is null.") + sender() ! akka.actor.Status.Failure(new IllegalArgumentException("JobGraph must not be" + + " null.")) } else { log.info(s"Received job ${jobGraph.getJobID} (${jobGraph.getName}}).") @@ -125,9 +115,10 @@ Int, cleanupInterval: Long) extends Actor with ActorLogMessages with ActorLoggin // Create the user code class loader libraryCacheManager.register(jobGraph.getJobID, jobGraph.getUserJarBlobKeys) - executionGraph = currentJobs.getOrElseUpdate(jobGraph.getJobID(), - new ExecutionGraph(jobGraph.getJobID, - jobGraph.getName, jobGraph.getJobConfiguration, jobGraph.getUserJarBlobKeys)) + val (executionGraph, jobInfo) = currentJobs.getOrElseUpdate(jobGraph.getJobID(), + (new ExecutionGraph(jobGraph.getJobID, jobGraph.getName, + jobGraph.getJobConfiguration, jobGraph.getUserJarBlobKeys), JobInfo(sender(), + System.currentTimeMillis()))) val userCodeLoader = libraryCacheManager.getClassLoader(jobGraph.getJobID) @@ -163,33 +154,43 @@ Int, cleanupInterval: Long) extends Actor with ActorLogMessages with ActorLoggin // closed iterations) executionGraph.setQueuedSchedulingAllowed(jobGraph.getAllowQueuedScheduling) - eventCollector ! RegisterJob(executionGraph, false, System.currentTimeMillis()) - executionGraph.registerJobStatusListener(self) + if(listenToEvents){ + executionGraph.registerExecutionListener(sender()) + } + + jobInfo.detach = detach + log.info(s"Scheduling job ${jobGraph.getName}.") executionGraph.scheduleForExecution(scheduler) - sender() ! JobSubmissionResult(JobResult.SUCCESS, null) + sender() ! SubmissionSuccess(jobGraph.getJobID) } } catch { case t: Throwable => log.error(t, "Job submission failed.") - if(executionGraph != null){ - executionGraph.fail(t) - val status = Patterns.ask(self, RequestJobStatusWhenTerminated, 10 second) - status.onFailure{ - case _: Throwable => self ! JobStatusChanged(executionGraph, JobStatus.FAILED, - s"Cleanup job ${jobGraph.getJobID}.") - } - }else { - libraryCacheManager.unregister(jobGraph.getJobID) - currentJobs.remove(jobGraph.getJobID) + currentJobs.get(jobGraph.getJobID) match { + case Some((executionGraph, jobInfo)) => + executionGraph.fail(t) + + // don't send the client the final job status + jobInfo.detach = true + + val status = Patterns.ask(self, RequestFinalJobStatus(jobGraph.getJobID), 10 second) + status.onFailure{ + case _: Throwable => self ! JobStatusChanged(executionGraph.getJobID, + JobStatus.FAILED, System.currentTimeMillis(), s"Cleanup job ${jobGraph.getJobID}.") + } + case None => + libraryCacheManager.unregister(jobGraph.getJobID) + currentJobs.remove(jobGraph.getJobID) + } - sender() ! JobSubmissionResult(JobResult.ERROR, StringUtils.stringifyException(t)) + sender() ! SubmissionFailure(jobGraph.getJobID, t) } } @@ -197,14 +198,15 @@ Int, cleanupInterval: Long) extends Actor with ActorLogMessages with ActorLoggin log.info(s"Trying to cancel job with ID ${jobID}.") currentJobs.get(jobID) match { - case Some(executionGraph) => + case Some((executionGraph, _)) => Future { executionGraph.cancel() } - sender() ! JobCancelResult(JobResult.SUCCESS, null) + sender() ! CancellationSuccess(jobID) case None => log.info(s"No job found with ID ${jobID}.") - sender() ! JobCancelResult(JobResult.ERROR, s"Cannot find job with ID ${jobID}") + sender() ! CancellationFailure(jobID, new IllegalArgumentException(s"No job found with " + + s"ID ${jobID}.")) } } @@ -212,7 +214,7 @@ Int, cleanupInterval: Long) extends Actor with ActorLogMessages with ActorLoggin Preconditions.checkNotNull(taskExecutionState) currentJobs.get(taskExecutionState.getJobID) match { - case Some(executionGraph) => sender() ! executionGraph.updateState(taskExecutionState) + case Some((executionGraph, _)) => sender() ! executionGraph.updateState(taskExecutionState) case None => log.error(s"Cannot find execution graph for ID ${taskExecutionState .getJobID} to change state to" + s" ${taskExecutionState.getExecutionState}.") @@ -222,7 +224,7 @@ Int, cleanupInterval: Long) extends Actor with ActorLogMessages with ActorLoggin case RequestNextInputSplit(jobID, vertexID) => { val nextInputSplit = currentJobs.get(jobID) match { - case Some(executionGraph) => executionGraph.getJobVertex(vertexID) match { + case Some((executionGraph,_)) => executionGraph.getJobVertex(vertexID) match { case vertex: ExecutionJobVertex => vertex.getSplitAssigner match { case splitAssigner: InputSplitAssigner => splitAssigner.getNextInputSplit(null) case _ => @@ -241,34 +243,54 @@ Int, cleanupInterval: Long) extends Actor with ActorLogMessages with ActorLoggin sender() ! NextInputSplit(nextInputSplit) } - case JobStatusChanged(executionGraph, newJobStatus, optionalMessage) => { - val jobID = executionGraph.getJobID - - log.info(s"Status of job ${jobID} (${executionGraph.getJobName}) changed to " + - s"${newJobStatus}${optionalMessage}.") + case JobStatusChanged(jobID, newJobStatus, timeStamp, optionalMessage) => { + currentJobs.get(jobID) match { + case Some((executionGraph, jobInfo)) => executionGraph.getJobName + log.info(s"Status of job ${jobID} (${executionGraph.getJobName}) changed to " + + s"${newJobStatus}${optionalMessage}.") + + if(newJobStatus.isTerminalState) { + jobInfo.end = timeStamp + + if(!jobInfo.detach) { + newJobStatus match { + case JobStatus.FINISHED => + val accumulatorResults = accumulatorManager.getJobAccumulatorResults(jobID) + jobInfo.client ! JobResultSuccess(jobID, jobInfo.duration, accumulatorResults) + case JobStatus.CANCELED => + jobInfo.client ! JobResultCanceled(jobID, optionalMessage) + case JobStatus.FAILED => + jobInfo.client ! JobResultFailed(jobID, optionalMessage) + case x => throw new IllegalArgumentException(s"$x is not a terminal state.") + } + } - if (Set(JobStatus.FINISHED, JobStatus.CANCELED, JobStatus.FAILED) contains newJobStatus) { - // send final job status to job termination listeners - jobTerminationListener.get(jobID) foreach { - listeners => - listeners foreach { - _ ! CurrentJobStatus(jobID, newJobStatus) + finalJobStatusListener.get(jobID) foreach { + _ foreach { + _ ! CurrentJobStatus(jobID, newJobStatus) + } } - } - currentJobs.remove(jobID) - try { - libraryCacheManager.unregister(jobID) - } catch { - case t: Throwable => - log.error(t, s"Could not properly unregister job ${jobID} form the library cache.") - } + removeJob(jobID) + } + case None => + removeJob(jobID) + } + } + + case RequestFinalJobStatus(jobID) => { + currentJobs.get(jobID) match { + case Some(_) => + val listeners = finalJobStatusListener.getOrElse(jobID, Set()) + finalJobStatusListener += jobID -> (listeners + sender()) + case None => + archive ! RequestJobStatus(jobID) } } case LookupConnectionInformation(connectionInformation, jobID, sourceChannelID) => { currentJobs.get(jobID) match { - case Some(executionGraph) => + case Some((executionGraph, _)) => sender() ! ConnectionInformation(executionGraph.lookupConnectionInfoAndDeployReceivers (connectionInformation, sourceChannelID)) case None => @@ -283,53 +305,26 @@ Int, cleanupInterval: Long) extends Actor with ActorLogMessages with ActorLoggin (libraryCacheManager.getClassLoader(accumulatorEvent.getJobID))) } - case RequestAccumulatorResult(jobID) => { - sender() ! new AccumulatorEvent(jobID, accumulatorManager.getJobAccumulators(jobID)) - } - - case RegisterJobStatusListener(jobID) => { - currentJobs.get(jobID) match { - case Some(executionGraph) => - executionGraph.registerJobStatusListener(sender()) - sender() ! CurrentJobStatus(jobID, executionGraph.getState) - case None => - log.warning(s"There is no running job with job ID ${jobID}.") - sender() ! JobNotFound(jobID) - } - } - - case RequestJobStatusWhenTerminated(jobID) => { - if (currentJobs.contains(jobID)) { - val listeners = jobTerminationListener.getOrElse(jobID, Set()) - jobTerminationListener += jobID -> (listeners + sender()) - } else { - eventCollector.tell(RequestJobStatus(jobID), sender()) - } - } - case RequestJobStatus(jobID) => { currentJobs.get(jobID) match { - case Some(executionGraph) => sender() ! CurrentJobStatus(jobID, executionGraph.getState) - case None => eventCollector.tell(RequestJobStatus(jobID), sender()) + case Some((executionGraph,_)) => sender() ! CurrentJobStatus(jobID, executionGraph.getState) + case None => archive.tell(RequestJobStatus(jobID), sender()) } } - case RequestRecentJobEvents => { - eventCollector.tell(RequestRecentJobEvents, sender()) - } + case RequestRunningJobs => { + val runningJobs = currentJobs map { + case (jobID, (eg, jobInfo)) => + RunningJob(jobID, eg.getState, eg.getStatusTimestamp(eg.getState), eg.getJobName) + } - case msg: RequestJobProgress => { - eventCollector forward msg + sender() ! RunningJobsResponse(runningJobs.toSeq) } case RequestBlobManagerPort => { sender() ! libraryCacheManager.getBlobServerPort } - case RequestPollingInterval => { - sender() ! recommendedPollingInterval - } - case Heartbeat(instanceID) => { instanceManager.reportHeartBeat(instanceID) } @@ -340,6 +335,20 @@ Int, cleanupInterval: Long) extends Actor with ActorLogMessages with ActorLoggin context.unwatch(taskManager) } } + + private def removeJob(jobID: JobID): Unit = { + currentJobs.remove(jobID) match { + case Some((eg, _)) => archive ! ArchiveExecutionGraph(jobID, eg) + case None => + } + + try { + libraryCacheManager.unregister(jobID) + } catch { + case t: Throwable => + log.error(t, s"Could not properly unregister job ${jobID} form the library cache.") + } + } } object JobManager { @@ -360,9 +369,8 @@ object JobManager { def initialize(args: Array[String]): (String, Int, Configuration) = { val parser = new scopt.OptionParser[JobManagerCLIConfiguration]("jobmanager") { head("flink jobmanager") - opt[String]("configDir") action { (x, c) => - c.copy(configDir = x) - } text ("Specify configuration directory.") + opt[String]("configDir") action { (x, c) => c.copy(configDir = x) } text ("Specify " + + "configuration directory.") } parser.parse(args, JobManagerCLIConfiguration()) map { @@ -390,19 +398,16 @@ object JobManager { (actorSystem, (startActor _).tupled(parseConfiguration(configuration))) } - def parseConfiguration(configuration: Configuration): (Int, Boolean, Int, Long) = { + def parseConfiguration(configuration: Configuration): (Int, Boolean, Long) = { val archiveCount = configuration.getInteger(ConfigConstants.JOB_MANAGER_WEB_ARCHIVE_COUNT, ConfigConstants.DEFAULT_JOB_MANAGER_WEB_ARCHIVE_COUNT) val profilingEnabled = configuration.getBoolean(ProfilingUtils.PROFILE_JOB_KEY, true) - val recommendedPollingInterval = configuration.getInteger(ConfigConstants - .JOBCLIENT_POLLING_INTERVAL_KEY, - ConfigConstants.DEFAULT_JOBCLIENT_POLLING_INTERVAL) val cleanupInterval = configuration.getLong(ConfigConstants .LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL, ConfigConstants.DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) * 1000 - (archiveCount, profilingEnabled, recommendedPollingInterval, cleanupInterval) + (archiveCount, profilingEnabled, cleanupInterval) } def startActorWithConfiguration(configuration: Configuration)(implicit actorSystem: @@ -410,12 +415,10 @@ object JobManager { (startActor _).tupled(parseConfiguration(configuration)) } - def startActor(archiveCount: Int, profilingEnabled: Boolean, recommendedPollingInterval: Int, - cleanupInterval: Long) + def startActor(archiveCount: Int, profilingEnabled: Boolean, cleanupInterval: Long) (implicit actorSystem: ActorSystem): ActorRef = { - actorSystem.actorOf(Props(classOf[JobManager], archiveCount, profilingEnabled, - recommendedPollingInterval, cleanupInterval), + actorSystem.actorOf(Props(classOf[JobManager], archiveCount, profilingEnabled, cleanupInterval), JOB_MANAGER_NAME) } @@ -435,7 +438,7 @@ object JobManager { AkkaUtils.getChild(jobManager, ARCHIVE_NAME) } - def getJobManager(address: InetSocketAddress): ActorRef = { + def getJobManager(address: InetSocketAddress)(implicit system: ActorSystem): ActorRef = { AkkaUtils.getReference(getAkkaURL(address.getHostName + ":" + address.getPort)) } } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala index 24729f23730..e44ac4609e2 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala @@ -32,17 +32,6 @@ import scala.collection.mutable.ListBuffer class MemoryArchivist(private val max_entries: Int) extends Actor with ActorLogMessages with ActorLogging with DecorateAsJava { - /** - * The map which stores all collected events until they are either - * fetched by the client or discarded. - */ - val collectedEvents = collection.mutable.HashMap[JobID, ListBuffer[AbstractEvent]]() - - /** - * Map of recently started jobs with the time stamp of the last received job event. - */ - val oldJobs = collection.mutable.HashMap[JobID, RecentJobEvent]() - /** * Map of execution graphs belonging to recently started jobs with the time stamp of the last * received job event. @@ -53,30 +42,10 @@ ActorLogging with DecorateAsJava { val lru = collection.mutable.Queue[JobID]() override def receiveWithLogMessages: Receive = { - case ArchiveEvent(jobID, event) => { - val list = collectedEvents.getOrElseUpdate(jobID, ListBuffer()) - list += event - cleanup(jobID) - } - - case ArchiveJobEvent(jobID, event) => { - oldJobs.update(jobID, event) - cleanup(jobID) - } - case ArchiveExecutionGraph(jobID, graph) => { graphs.update(jobID, graph) cleanup(jobID) } - - case RequestJobStatus(jobID) => { - val response = oldJobs get jobID match { - case Some(recentJobEvent) => CurrentJobStatus(jobID, recentJobEvent.getJobStatus) - case None => JobNotFound(jobID) - } - - sender() ! response - } } def cleanup(jobID: JobID): Unit = { @@ -86,8 +55,6 @@ ActorLogging with DecorateAsJava { while (lru.size > max_entries) { val removedJobID = lru.dequeue() - collectedEvents.remove(removedJobID) - oldJobs.remove(removedJobID) graphs.remove(removedJobID) } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/ExecutionMode.java b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/RunningJob.scala similarity index 80% rename from flink-runtime/src/main/java/org/apache/flink/runtime/ExecutionMode.java rename to flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/RunningJob.scala index e17c8fd8c86..610c7949219 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/ExecutionMode.java +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/RunningJob.scala @@ -16,9 +16,8 @@ * limitations under the License. */ +package org.apache.flink.runtime.jobmanager -package org.apache.flink.runtime; +import org.apache.flink.runtime.jobgraph.{JobID, JobStatus} -public enum ExecutionMode { - LOCAL, CLUSTER -} +case class RunningJob(jobID: JobID, jobStatus: JobStatus, timestamp: Long, jobName: String) diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/ArchiveMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/ArchiveMessages.scala index 5065009bccd..a23dd96177f 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/ArchiveMessages.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/ArchiveMessages.scala @@ -23,7 +23,5 @@ import org.apache.flink.runtime.executiongraph.ExecutionGraph import org.apache.flink.runtime.jobgraph.JobID object ArchiveMessages { - case class ArchiveEvent(jobID: JobID, event: AbstractEvent) - case class ArchiveJobEvent(jobID: JobID, event: RecentJobEvent) case class ArchiveExecutionGraph(jobID: JobID, graph: ExecutionGraph) } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/EventCollectorMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/EventCollectorMessages.scala deleted file mode 100644 index 2b7d54e53e9..00000000000 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/EventCollectorMessages.scala +++ /dev/null @@ -1,60 +0,0 @@ -/* - * 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.messages - -import akka.actor.ActorRef -import org.apache.flink.runtime.event.job.{AbstractEvent, RecentJobEvent} -import org.apache.flink.runtime.executiongraph.ExecutionGraph -import org.apache.flink.runtime.jobgraph.JobID -import org.apache.flink.runtime.profiling.types.ProfilingEvent - -import scala.collection.convert.{WrapAsScala, DecorateAsJava} - -object EventCollectorMessages extends DecorateAsJava with WrapAsScala { - - case class ProcessProfilingEvent(profilingEvent: ProfilingEvent) - - case class RegisterArchiveListener(listener: ActorRef) - - case class RequestJobProgress(jobID: JobID) - - case class RegisterJob(executionGraph: ExecutionGraph, profilingAvailable: Boolean, - submissionTimestamp: Long) - - case class RecentJobs(jobs: List[RecentJobEvent]) { - def this(_jobs: java.util.List[RecentJobEvent]) = { - this(_jobs.toList) - } - - def asJavaList: java.util.List[RecentJobEvent] = { - jobs.asJava - } - } - - case class JobEvents(jobs: List[AbstractEvent]) { - def asJavaList: java.util.List[AbstractEvent] = { - jobs.asJava - } - } - - case object ArchiveExpiredEvents - - case object RequestRecentJobEvents - -} diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/ExecutionGraphMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/ExecutionGraphMessages.scala index 2badae29463..b48c6214b01 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/ExecutionGraphMessages.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/ExecutionGraphMessages.scala @@ -26,7 +26,8 @@ object ExecutionGraphMessages { case class ExecutionStateChanged(jobID: JobID, vertexID: JobVertexID, subtask: Int, executionID: ExecutionAttemptID, - newExecutionState: ExecutionState, optionalMessage: String) + newExecutionState: ExecutionState, + timestamp: Long, optionalMessage: String) sealed trait JobStatusResponse { @@ -35,9 +36,7 @@ object ExecutionGraphMessages { case class CurrentJobStatus(jobID: JobID, status: JobStatus) extends JobStatusResponse case class JobNotFound(jobID: JobID) extends JobStatusResponse - case class JobStatusChanged(executionGraph: ExecutionGraph, newJobStatus: JobStatus, - optionalMessage: String){ - def jobID = executionGraph.getJobID - } + case class JobStatusChanged(jobID: JobID, newJobStatus: JobStatus, timestamp: Long, + optionalMessage: String) } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobResult.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobClientMessages.scala similarity index 58% rename from flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobResult.scala rename to flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobClientMessages.scala index b8fee74a4b9..d88f7aae41f 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobResult.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobClientMessages.scala @@ -18,24 +18,9 @@ package org.apache.flink.runtime.messages -import org.apache.flink.runtime.event.job.AbstractEvent - -import scala.collection.convert.DecorateAsJava - - -object JobResult extends Enumeration with DecorateAsJava { - type JobResult = Value - val SUCCESS, ERROR = Value - - case class JobProgressResult(returnCode: JobResult, description: String, - events: List[AbstractEvent]) { - def asJavaList: java.util.List[AbstractEvent] = { - events.asJava - } - } - - case class JobCancelResult(returnCode: JobResult, description: String) - - case class JobSubmissionResult(returnCode: JobResult, description: String) +import org.apache.flink.runtime.jobgraph.JobGraph +object JobClientMessages { + case class SubmitJobAndWait(jobGraph: JobGraph, listenToEvents: Boolean = false) + case class SubmitJobDetached(jobGraph: JobGraph, listenToEvents: Boolean = false) } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobmanagerMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobmanagerMessages.scala index e79dd998a48..4d1418af499 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobmanagerMessages.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobmanagerMessages.scala @@ -23,12 +23,15 @@ import org.apache.flink.runtime.instance.InstanceConnectionInfo import org.apache.flink.runtime.io.network.ConnectionInfoLookupResponse import org.apache.flink.runtime.io.network.channels.ChannelID import org.apache.flink.runtime.jobgraph.{JobVertexID, JobID, JobGraph} -import org.apache.flink.runtime.profiling.impl.types.ProfilingDataContainer +import org.apache.flink.runtime.jobmanager.RunningJob import org.apache.flink.runtime.taskmanager.TaskExecutionState +import scala.collection.convert.{WrapAsScala, WrapAsJava} + object JobManagerMessages { - case class SubmitJob(jobGraph: JobGraph) + case class SubmitJob(jobGraph: JobGraph, listenToEvents: Boolean = false, + detach: Boolean = false) case class CancelJob(jobID: JobID) @@ -43,12 +46,6 @@ object JobManagerMessages { case class ReportAccumulatorResult(accumulatorEvent: AccumulatorEvent) - case class RequestAccumulatorResult(jobID: JobID) - - case class RegisterJobStatusListener(jobID: JobID) - - case class RequestJobStatusWhenTerminated(jobID: JobID) - case class RequestJobStatus(jobID: JobID) case object RequestInstances @@ -57,8 +54,43 @@ object JobManagerMessages { case object RequestAvailableSlots - case object RequestPollingInterval - case object RequestBlobManagerPort + case class RequestFinalJobStatus(jobID: JobID) + + sealed trait JobResult{ + def jobID: JobID + } + + case class JobResultSuccess(jobID: JobID, runtime: Long, accumulatorResults: java.util.Map[String, + AnyRef]) extends JobResult {} + + case class JobResultCanceled(jobID: JobID, msg: String) + + case class JobResultFailed(jobID: JobID, msg:String) + + sealed trait SubmissionResponse{ + def jobID: JobID + } + + case class SubmissionSuccess(jobID: JobID) extends SubmissionResponse + case class SubmissionFailure(jobID: JobID, cause: Throwable) extends SubmissionResponse + + sealed trait CancellationResponse{ + def jobID: JobID + } + + case class CancellationSuccess(jobID: JobID) extends CancellationResponse + case class CancellationFailure(jobID: JobID, cause: Throwable) extends CancellationResponse + + case object RequestRunningJobs + + case class RunningJobsResponse(runningJobs: Seq[RunningJob]) { + def this() = this(Seq()) + def asJavaList: java.util.List[RunningJob] = { + import scala.collection.JavaConversions.seqAsJavaList + seqAsJavaList(runningJobs) + } + } + } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManagerProfiler.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManagerProfiler.scala index 6db21666266..e2b72d4d970 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManagerProfiler.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManagerProfiler.scala @@ -117,7 +117,8 @@ ActorLogMessages with ActorLogging { } } - case ExecutionStateChanged(_, vertexID, subtaskIndex, executionID, newExecutionState, _) => { + case ExecutionStateChanged(_, vertexID, subtaskIndex, executionID, newExecutionState, + _, _) => { import ExecutionState._ environments.get(executionID) match { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobKeyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobKeyTest.java index b4070ed794b..020aaf7530d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobKeyTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobKeyTest.java @@ -27,6 +27,7 @@ import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; +import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.testutils.ServerTestUtils; import org.apache.flink.util.StringUtils; import org.junit.Test; @@ -55,14 +56,14 @@ public final class BlobKeyTest { } /** - * Tests the serialization/deserialization of BLOB keys using the regular {@link org.apache.flink.core.io.IOReadableWritable} API. + * Tests the serialization/deserialization of BLOB keys */ @Test public void testSerialization() { final BlobKey k1 = new BlobKey(KEY_ARRAY_1); final BlobKey k2; try { - k2 = ServerTestUtils.createCopy(k1); + k2 = CommonTestUtils.createCopySerializable(k1); } catch (IOException ioe) { fail(StringUtils.stringifyException(ioe)); return; diff --git a/flink-runtime/src/test/resources/logback-test.xml b/flink-runtime/src/test/resources/logback-test.xml index 9c5d4eb830f..17f7020596c 100644 --- a/flink-runtime/src/test/resources/logback-test.xml +++ b/flink-runtime/src/test/resources/logback-test.xml @@ -23,20 +23,20 @@ - + - - - - - - - - - + + + + + + + + + \ No newline at end of file diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/CoLocationConstraintITCase.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/CoLocationConstraintITCase.scala index c482f3752dc..650be4e796e 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/CoLocationConstraintITCase.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/CoLocationConstraintITCase.scala @@ -19,16 +19,13 @@ package org.apache.flink.runtime.jobmanager import akka.actor.ActorSystem +import akka.actor.Status.Success import akka.testkit.{ImplicitSender, TestKit} -import org.apache.flink.runtime.jobgraph.{JobStatus, JobGraph, DistributionPattern, +import org.apache.flink.runtime.jobgraph.{JobGraph, DistributionPattern, AbstractJobVertex} import org.apache.flink.runtime.jobmanager.Tasks.{Receiver, Sender} import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup -import org.apache.flink.runtime.messages.ExecutionGraphMessages.CurrentJobStatus -import org.apache.flink.runtime.messages.JobManagerMessages.{RequestJobStatusWhenTerminated, -SubmitJob} -import org.apache.flink.runtime.messages.JobResult -import org.apache.flink.runtime.messages.JobResult.JobSubmissionResult +import org.apache.flink.runtime.messages.JobManagerMessages.SubmitJob import org.apache.flink.runtime.testingUtils.TestingUtils import org.scalatest.{BeforeAndAfterAll, Matchers, WordSpecLike} import scala.collection.convert.WrapAsJava @@ -71,10 +68,8 @@ ImplicitSender with WordSpecLike with Matchers with BeforeAndAfterAll with WrapA try { within(1 second) { jm ! SubmitJob(jobGraph) - expectMsg(JobSubmissionResult(JobResult.SUCCESS, null)) - - jm ! RequestJobStatusWhenTerminated(jobGraph.getJobID) - expectMsg(CurrentJobStatus(jobGraph.getJobID, JobStatus.FINISHED)) + expectMsg(Success(_)) + expectMsg() } } finally { cluster.stop() diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala index 8a83245aef4..37f8f03172b 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala @@ -21,18 +21,14 @@ package org.apache.flink.runtime.jobmanager import akka.actor.ActorSystem import akka.testkit.{ImplicitSender, TestKit} import org.apache.flink.runtime.akka.AkkaUtils -import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager -import org.apache.flink.runtime.jobgraph.{DistributionPattern, JobStatus, JobGraph, +import org.apache.flink.runtime.jobgraph.{DistributionPattern, JobGraph, AbstractJobVertex} import Tasks._ +import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException import org.apache.flink.runtime.testingUtils.{TestingUtils, TestingJobManagerMessages} import TestingJobManagerMessages.{ExecutionGraphNotFound, ExecutionGraphFound, ResponseExecutionGraph, RequestExecutionGraph} -import org.apache.flink.runtime.messages.ExecutionGraphMessages.CurrentJobStatus -import org.apache.flink.runtime.messages.JobManagerMessages.{RequestJobStatusWhenTerminated, -SubmitJob, RequestAvailableSlots} -import org.apache.flink.runtime.messages.JobResult -import org.apache.flink.runtime.messages.JobResult.JobSubmissionResult +import org.apache.flink.runtime.messages.JobManagerMessages._ import org.scalatest.{Matchers, WordSpecLike, BeforeAndAfterAll} import scala.concurrent.duration._ @@ -59,13 +55,12 @@ WordSpecLike with Matchers with BeforeAndAfterAll { val availableSlots = AkkaUtils.ask[Int](jm, RequestAvailableSlots) availableSlots should equal(1) - val result = AkkaUtils.ask[JobSubmissionResult](jm, SubmitJob(jobGraph)) + within(1 second) { + jm ! SubmitJob(jobGraph) - result.returnCode should equal(JobResult.ERROR) + expectMsg(SubmissionFailure(jobGraph.getJobID, new NoResourceAvailableException(1,1))) - within(1 second) { - jm ! RequestJobStatusWhenTerminated(jobGraph.getJobID) - expectMsg(CurrentJobStatus(jobGraph.getJobID, JobStatus.FAILED)) + expectNoMsg() } val executionGraph = AkkaUtils.ask[ResponseExecutionGraph](jm, @@ -81,473 +76,460 @@ WordSpecLike with Matchers with BeforeAndAfterAll { } } - "support immediate scheduling of a single vertex" in { - val num_tasks = 133 - val vertex = new AbstractJobVertex("Test Vertex") - vertex.setParallelism(num_tasks) - vertex.setInvokableClass(classOf[NoOpInvokable]) + "support immediate scheduling of a single vertex" in { + val num_tasks = 133 + val vertex = new AbstractJobVertex("Test Vertex") + vertex.setParallelism(num_tasks) + vertex.setInvokableClass(classOf[NoOpInvokable]) - val jobGraph = new JobGraph("Test Job", vertex) + val jobGraph = new JobGraph("Test Job", vertex) - val cluster = TestingUtils.startTestingCluster(num_tasks) - val jm = cluster.getJobManager + val cluster = TestingUtils.startTestingCluster(num_tasks) + val jm = cluster.getJobManager - try { - val availableSlots = AkkaUtils.ask[Int](jm, RequestAvailableSlots) - availableSlots should equal(num_tasks) + try { + val availableSlots = AkkaUtils.ask[Int](jm, RequestAvailableSlots) + availableSlots should equal(num_tasks) - val result = AkkaUtils.ask[JobSubmissionResult](jm, SubmitJob(jobGraph)) + within(1 second) { + jm ! SubmitJob(jobGraph) - result.returnCode should equal(JobResult.SUCCESS) + expectMsg(SubmissionSuccess(jobGraph.getJobID)) + val result = expectMsgType[JobResultSuccess] - within(1 second) { - jm ! RequestJobStatusWhenTerminated(jobGraph.getJobID) - expectMsg(CurrentJobStatus(jobGraph.getJobID, JobStatus.FINISHED)) - } + result.jobID should equal(jobGraph.getJobID) - val executionGraph = AkkaUtils.ask[ResponseExecutionGraph](jm, - RequestExecutionGraph(jobGraph.getJobID)) match { - case ExecutionGraphFound(_, eg) => eg - case ExecutionGraphNotFound(jobID) => fail(s"The execution graph for job ID ${jobID} " + - s"was not retrievable.") + expectNoMsg() + } + + val executionGraph = AkkaUtils.ask[ResponseExecutionGraph](jm, + RequestExecutionGraph(jobGraph.getJobID)) match { + case ExecutionGraphFound(_, eg) => eg + case ExecutionGraphNotFound(jobID) => fail(s"The execution graph for job ID ${jobID} " + + s"was not retrievable.") + } + + executionGraph.getRegisteredExecutions.size should equal(0) + } finally { + cluster.stop() + } } - executionGraph.getRegisteredExecutions.size should equal(0) - } finally { - cluster.stop() - } - } + "support queued scheduling of a single vertex" in { + val num_tasks = 111 - "support queued scheduling of a single vertex" in { - val num_tasks = 111 + val vertex = new AbstractJobVertex("Test Vertex") + vertex.setParallelism(num_tasks) + vertex.setInvokableClass(classOf[NoOpInvokable]) - val vertex = new AbstractJobVertex("Test Vertex") - vertex.setParallelism(num_tasks) - vertex.setInvokableClass(classOf[NoOpInvokable]) + val jobGraph = new JobGraph("Test job", vertex) + jobGraph.setAllowQueuedScheduling(true) - val jobGraph = new JobGraph("Test job", vertex) - jobGraph.setAllowQueuedScheduling(true) + val cluster = TestingUtils.startTestingCluster(10) + val jm = cluster.getJobManager - val cluster = TestingUtils.startTestingCluster(10) - val jm = cluster.getJobManager + try { + within(1 second) { + jm ! SubmitJob(jobGraph) - try { - within(1 second) { - jm ! SubmitJob(jobGraph) + expectMsg(SubmissionSuccess(jobGraph.getJobID)) - expectMsg(JobSubmissionResult(JobResult.SUCCESS, null)) + val result = expectMsgType[JobResultSuccess] - jm ! RequestJobStatusWhenTerminated(jobGraph.getJobID) + result.jobID should equal(jobGraph.getJobID) - expectMsg(CurrentJobStatus(jobGraph.getJobID, JobStatus.FINISHED)) - } + expectNoMsg() + } - val executionGraph = AkkaUtils.ask[ResponseExecutionGraph](jm, - RequestExecutionGraph(jobGraph.getJobID)) match { - case ExecutionGraphFound(_, eg) => eg - case ExecutionGraphNotFound(jobID) => fail(s"The execution graph for job ID ${jobID} " + - s"was not retrievable.") - } + val executionGraph = AkkaUtils.ask[ResponseExecutionGraph](jm, + RequestExecutionGraph(jobGraph.getJobID)) match { + case ExecutionGraphFound(_, eg) => eg + case ExecutionGraphNotFound(jobID) => fail(s"The execution graph for job ID ${jobID} " + + s"was not retrievable.") + } - executionGraph.getRegisteredExecutions.size should equal(0) - } finally { - cluster.stop() - } - } + executionGraph.getRegisteredExecutions.size should equal(0) + } finally { + cluster.stop() + } + } - "support forward jobs" in { - val num_tasks = 31 - val sender = new AbstractJobVertex("Sender") - val receiver = new AbstractJobVertex("Receiver") + "support forward jobs" in { + val num_tasks = 31 + val sender = new AbstractJobVertex("Sender") + val receiver = new AbstractJobVertex("Receiver") - sender.setInvokableClass(classOf[Sender]) - receiver.setInvokableClass(classOf[Receiver]) + sender.setInvokableClass(classOf[Sender]) + receiver.setInvokableClass(classOf[Receiver]) - sender.setParallelism(num_tasks) - receiver.setParallelism(num_tasks) + sender.setParallelism(num_tasks) + receiver.setParallelism(num_tasks) - receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) + receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) - val jobGraph = new JobGraph("Pointwise Job", sender, receiver) + val jobGraph = new JobGraph("Pointwise Job", sender, receiver) - val cluster = TestingUtils.startTestingCluster(2 * num_tasks) - val jm = cluster.getJobManager + val cluster = TestingUtils.startTestingCluster(2 * num_tasks) + val jm = cluster.getJobManager - try { - within(1 second) { - jm ! SubmitJob(jobGraph) + try { + within(1 second) { + jm ! SubmitJob(jobGraph) - expectMsg(JobSubmissionResult(JobResult.SUCCESS, null)) + expectMsg(SubmissionSuccess(jobGraph.getJobID)) - jm ! RequestJobStatusWhenTerminated(jobGraph.getJobID) + val result = expectMsgType[JobResultSuccess] - expectMsg(CurrentJobStatus(jobGraph.getJobID, JobStatus.FINISHED)) - } + result.jobID should equal(jobGraph.getJobID) - val executionGraph = AkkaUtils.ask[ResponseExecutionGraph](jm, - RequestExecutionGraph(jobGraph.getJobID)) match { - case ExecutionGraphFound(_, eg) => eg - case ExecutionGraphNotFound(jobID) => fail(s"The execution graph for job ID ${jobID} " + - s"was not retrievable.") - } + expectNoMsg() + } - executionGraph.getRegisteredExecutions.size should equal(0) - } finally { - cluster.stop() - } - } + val executionGraph = AkkaUtils.ask[ResponseExecutionGraph](jm, + RequestExecutionGraph(jobGraph.getJobID)) match { + case ExecutionGraphFound(_, eg) => eg + case ExecutionGraphNotFound(jobID) => fail(s"The execution graph for job ID ${jobID} " + + s"was not retrievable.") + } - "support bipartite job" in { - val num_tasks = 31 - val sender = new AbstractJobVertex("Sender") - val receiver = new AbstractJobVertex("Receiver") + executionGraph.getRegisteredExecutions.size should equal(0) + } finally { + cluster.stop() + } + } - sender.setInvokableClass(classOf[Sender]) - receiver.setInvokableClass(classOf[AgnosticReceiver]) + "support bipartite job" in { + val num_tasks = 31 + val sender = new AbstractJobVertex("Sender") + val receiver = new AbstractJobVertex("Receiver") - receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) + sender.setInvokableClass(classOf[Sender]) + receiver.setInvokableClass(classOf[AgnosticReceiver]) - val jobGraph = new JobGraph("Bipartite Job", sender, receiver) + receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) - val cluster = TestingUtils.startTestingCluster(2 * num_tasks) - val jm = cluster.getJobManager + val jobGraph = new JobGraph("Bipartite Job", sender, receiver) - try { - within(1 second) { - jm ! SubmitJob(jobGraph) + val cluster = TestingUtils.startTestingCluster(2 * num_tasks) + val jm = cluster.getJobManager - expectMsg(JobSubmissionResult(JobResult.SUCCESS, null)) + try { + within(1 second) { + jm ! SubmitJob(jobGraph) - jm ! RequestJobStatusWhenTerminated(jobGraph.getJobID) + expectMsg(SubmissionSuccess(jobGraph.getJobID)) + expectMsgType[JobResultSuccess] + expectNoMsg() + } - expectMsg(CurrentJobStatus(jobGraph.getJobID, JobStatus.FINISHED)) - } + val executionGraph = AkkaUtils.ask[ResponseExecutionGraph](jm, + RequestExecutionGraph(jobGraph.getJobID)) match { + case ExecutionGraphFound(_, eg) => eg + case ExecutionGraphNotFound(jobID) => fail(s"The execution graph for job ID ${jobID} " + + s"was not retrievable.") + } - val executionGraph = AkkaUtils.ask[ResponseExecutionGraph](jm, - RequestExecutionGraph(jobGraph.getJobID)) match { - case ExecutionGraphFound(_, eg) => eg - case ExecutionGraphNotFound(jobID) => fail(s"The execution graph for job ID ${jobID} " + - s"was not retrievable.") + executionGraph.getRegisteredExecutions.size should equal(0) + } finally { + cluster.stop() + } } - executionGraph.getRegisteredExecutions.size should equal(0) - } finally { - cluster.stop() - } - } - - "support two input job failing edge mismatch" in { - val num_tasks = 11 - val sender1 = new AbstractJobVertex("Sender1") - val sender2 = new AbstractJobVertex("Sender2") - val receiver = new AbstractJobVertex("Receiver") + "support two input job failing edge mismatch" in { + val num_tasks = 11 + val sender1 = new AbstractJobVertex("Sender1") + val sender2 = new AbstractJobVertex("Sender2") + val receiver = new AbstractJobVertex("Receiver") - sender1.setInvokableClass(classOf[Sender]) - sender2.setInvokableClass(classOf[Sender]) - receiver.setInvokableClass(classOf[AgnosticReceiver]) + sender1.setInvokableClass(classOf[Sender]) + sender2.setInvokableClass(classOf[Sender]) + receiver.setInvokableClass(classOf[AgnosticReceiver]) - sender1.setParallelism(num_tasks) - sender2.setParallelism(2 * num_tasks) - receiver.setParallelism(3 * num_tasks) + sender1.setParallelism(num_tasks) + sender2.setParallelism(2 * num_tasks) + receiver.setParallelism(3 * num_tasks) - receiver.connectNewDataSetAsInput(sender1, DistributionPattern.POINTWISE) - receiver.connectNewDataSetAsInput(sender2, DistributionPattern.BIPARTITE) + receiver.connectNewDataSetAsInput(sender1, DistributionPattern.POINTWISE) + receiver.connectNewDataSetAsInput(sender2, DistributionPattern.BIPARTITE) - val jobGraph = new JobGraph("Bipartite Job", sender1, receiver, sender2) + val jobGraph = new JobGraph("Bipartite Job", sender1, receiver, sender2) - val cluster = TestingUtils.startTestingCluster(6 * num_tasks) - val jm = cluster.getJobManager + val cluster = TestingUtils.startTestingCluster(6 * num_tasks) + val jm = cluster.getJobManager - try { - within(1 second) { - jm ! SubmitJob(jobGraph) + try { + within(1 second) { + jm ! SubmitJob(jobGraph) - expectMsg(JobSubmissionResult(JobResult.SUCCESS, null)) + expectMsg(SubmissionSuccess(jobGraph.getJobID)) + expectMsgType[JobResultFailed] - jm ! RequestJobStatusWhenTerminated(jobGraph.getJobID) + expectNoMsg() + } - expectMsg(CurrentJobStatus(jobGraph.getJobID, JobStatus.FAILED)) - } + val executionGraph = AkkaUtils.ask[ResponseExecutionGraph](jm, + RequestExecutionGraph(jobGraph.getJobID)) match { + case ExecutionGraphFound(_, eg) => eg + case ExecutionGraphNotFound(jobID) => fail(s"The execution graph for job ID ${jobID} " + + s"was not retrievable.") + } - val executionGraph = AkkaUtils.ask[ResponseExecutionGraph](jm, - RequestExecutionGraph(jobGraph.getJobID)) match { - case ExecutionGraphFound(_, eg) => eg - case ExecutionGraphNotFound(jobID) => fail(s"The execution graph for job ID ${jobID} " + - s"was not retrievable.") + executionGraph.getRegisteredExecutions.size should equal(0) + } finally { + cluster.stop() + } } - executionGraph.getRegisteredExecutions.size should equal(0) - } finally { - cluster.stop() - } - } + "support two input job" in { + val num_tasks = 11 + val sender1 = new AbstractJobVertex("Sender1") + val sender2 = new AbstractJobVertex("Sender2") + val receiver = new AbstractJobVertex("Receiver") - "support two input job" in { - val num_tasks = 11 - val sender1 = new AbstractJobVertex("Sender1") - val sender2 = new AbstractJobVertex("Sender2") - val receiver = new AbstractJobVertex("Receiver") + sender1.setInvokableClass(classOf[Sender]) + sender2.setInvokableClass(classOf[Sender]) + receiver.setInvokableClass(classOf[AgnosticBinaryReceiver]) - sender1.setInvokableClass(classOf[Sender]) - sender2.setInvokableClass(classOf[Sender]) - receiver.setInvokableClass(classOf[AgnosticBinaryReceiver]) + sender1.setParallelism(num_tasks) + sender2.setParallelism(2 * num_tasks) + receiver.setParallelism(3 * num_tasks) - sender1.setParallelism(num_tasks) - sender2.setParallelism(2 * num_tasks) - receiver.setParallelism(3 * num_tasks) + receiver.connectNewDataSetAsInput(sender1, DistributionPattern.POINTWISE) + receiver.connectNewDataSetAsInput(sender2, DistributionPattern.BIPARTITE) - receiver.connectNewDataSetAsInput(sender1, DistributionPattern.POINTWISE) - receiver.connectNewDataSetAsInput(sender2, DistributionPattern.BIPARTITE) + val jobGraph = new JobGraph("Bipartite Job", sender1, receiver, sender2) - val jobGraph = new JobGraph("Bipartite Job", sender1, receiver, sender2) + val cluster = TestingUtils.startTestingCluster(6 * num_tasks) + val jm = cluster.getJobManager - val cluster = TestingUtils.startTestingCluster(6 * num_tasks) - val jm = cluster.getJobManager + try { + within(5 second) { + jm ! SubmitJob(jobGraph) + expectMsg(SubmissionSuccess(jobGraph.getJobID)) - try { - within(1 second) { - jm ! SubmitJob(jobGraph) + expectMsgType[JobResultSuccess] + } - expectMsg(JobSubmissionResult(JobResult.SUCCESS, null)) - } - - within(5 second){ - jm ! RequestJobStatusWhenTerminated(jobGraph.getJobID) + val executionGraph = AkkaUtils.ask[ResponseExecutionGraph](jm, + RequestExecutionGraph(jobGraph.getJobID)) match { + case ExecutionGraphFound(_, eg) => eg + case ExecutionGraphNotFound(jobID) => fail(s"The execution graph for job ID ${jobID} " + + s"was not retrievable.") + } - expectMsg(CurrentJobStatus(jobGraph.getJobID, JobStatus.FINISHED)) - } - - val executionGraph = AkkaUtils.ask[ResponseExecutionGraph](jm, - RequestExecutionGraph(jobGraph.getJobID)) match { - case ExecutionGraphFound(_, eg) => eg - case ExecutionGraphNotFound(jobID) => fail(s"The execution graph for job ID ${jobID} " + - s"was not retrievable.") + executionGraph.getRegisteredExecutions.size should equal(0) + } finally { + cluster.stop() + } } - executionGraph.getRegisteredExecutions.size should equal(0) - } finally { - cluster.stop() - } - } + "handle job with a failing sender vertex" in { + val num_tasks = 100 + val sender = new AbstractJobVertex("Sender") + val receiver = new AbstractJobVertex("Receiver") - "handle job with a failing sender vertex" in { - val num_tasks = 100 - val sender = new AbstractJobVertex("Sender") - val receiver = new AbstractJobVertex("Receiver") + sender.setInvokableClass(classOf[ExceptionSender]) + receiver.setInvokableClass(classOf[Receiver]) - sender.setInvokableClass(classOf[ExceptionSender]) - receiver.setInvokableClass(classOf[Receiver]) + sender.setParallelism(num_tasks) + receiver.setParallelism(num_tasks) - sender.setParallelism(num_tasks) - receiver.setParallelism(num_tasks) + receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) - receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) + val jobGraph = new JobGraph("Pointwise Job", sender, receiver) - val jobGraph = new JobGraph("Pointwise Job", sender, receiver) + val cluster = TestingUtils.startTestingCluster(num_tasks) + val jm = cluster.getJobManager - val cluster = TestingUtils.startTestingCluster(num_tasks) - val jm = cluster.getJobManager + try { + within(1 second) { + jm ! RequestAvailableSlots + expectMsg(num_tasks) + } - try { - within(1 second) { - jm ! RequestAvailableSlots - expectMsg(num_tasks) - } + within(1 second) { + jm ! SubmitJob(jobGraph) + expectMsg(SubmissionSuccess(jobGraph.getJobID)) + expectMsgType[JobResultFailed] + expectNoMsg() + } - within(1 second) { - jm ! SubmitJob(jobGraph) - expectMsg(JobSubmissionResult(JobResult.SUCCESS, null)) + val executionGraph = AkkaUtils.ask[ResponseExecutionGraph](jm, + RequestExecutionGraph(jobGraph.getJobID)) match { + case ExecutionGraphFound(_, eg) => eg + case ExecutionGraphNotFound(jobID) => fail(s"The execution graph for job ID ${jobID} " + + s"was not retrievable.") + } - jm ! RequestJobStatusWhenTerminated(jobGraph.getJobID) - expectMsg(CurrentJobStatus(jobGraph.getJobID, JobStatus.FAILED)) + executionGraph.getRegisteredExecutions.size should equal(0) + } finally { + cluster.stop() + } } - val executionGraph = AkkaUtils.ask[ResponseExecutionGraph](jm, - RequestExecutionGraph(jobGraph.getJobID)) match { - case ExecutionGraphFound(_, eg) => eg - case ExecutionGraphNotFound(jobID) => fail(s"The execution graph for job ID ${jobID} " + - s"was not retrievable.") - } - - executionGraph.getRegisteredExecutions.size should equal(0) - } finally { - cluster.stop() - } - } - - "handle job with an occasionally failing sender vertex" in { - val num_tasks = 100 - val sender = new AbstractJobVertex("Sender") - val receiver = new AbstractJobVertex("Receiver") + "handle job with an occasionally failing sender vertex" in { + val num_tasks = 100 + val sender = new AbstractJobVertex("Sender") + val receiver = new AbstractJobVertex("Receiver") - sender.setInvokableClass(classOf[SometimesExceptionSender]) - receiver.setInvokableClass(classOf[Receiver]) + sender.setInvokableClass(classOf[SometimesExceptionSender]) + receiver.setInvokableClass(classOf[Receiver]) - sender.setParallelism(num_tasks) - receiver.setParallelism(num_tasks) + sender.setParallelism(num_tasks) + receiver.setParallelism(num_tasks) - receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) + receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) - val jobGraph = new JobGraph("Pointwise Job", sender, receiver) + val jobGraph = new JobGraph("Pointwise Job", sender, receiver) - val cluster = TestingUtils.startTestingCluster(num_tasks) - val jm = cluster.getJobManager + val cluster = TestingUtils.startTestingCluster(num_tasks) + val jm = cluster.getJobManager - try { - within(1 second) { - jm ! RequestAvailableSlots - expectMsg(num_tasks) - } + try { + within(1 second) { + jm ! RequestAvailableSlots + expectMsg(num_tasks) + } - within(1 second) { - jm ! SubmitJob(jobGraph) - expectMsg(JobSubmissionResult(JobResult.SUCCESS, null)) + within(1 second) { + jm ! SubmitJob(jobGraph) + expectMsg(SubmissionSuccess(jobGraph.getJobID)) + expectMsgType[JobResultFailed] + expectNoMsg() + } - jm ! RequestJobStatusWhenTerminated(jobGraph.getJobID) - expectMsg(CurrentJobStatus(jobGraph.getJobID, JobStatus.FAILED)) - } + val executionGraph = AkkaUtils.ask[ResponseExecutionGraph](jm, + RequestExecutionGraph(jobGraph.getJobID)) match { + case ExecutionGraphFound(_, eg) => eg + case ExecutionGraphNotFound(jobID) => fail(s"The execution graph for job ID ${jobID} " + + s"was not retrievable.") + } - val executionGraph = AkkaUtils.ask[ResponseExecutionGraph](jm, - RequestExecutionGraph(jobGraph.getJobID)) match { - case ExecutionGraphFound(_, eg) => eg - case ExecutionGraphNotFound(jobID) => fail(s"The execution graph for job ID ${jobID} " + - s"was not retrievable.") + executionGraph.getRegisteredExecutions.size should equal(0) + } finally { + cluster.stop() + } } - executionGraph.getRegisteredExecutions.size should equal(0) - } finally { - cluster.stop() - } - } - - "handle job with a failing receiver vertex" in { - val num_tasks = 200 - val sender = new AbstractJobVertex("Sender") - val receiver = new AbstractJobVertex("Receiver") + "handle job with a failing receiver vertex" in { + val num_tasks = 200 + val sender = new AbstractJobVertex("Sender") + val receiver = new AbstractJobVertex("Receiver") - sender.setInvokableClass(classOf[Sender]) - receiver.setInvokableClass(classOf[ExceptionReceiver]) + sender.setInvokableClass(classOf[Sender]) + receiver.setInvokableClass(classOf[ExceptionReceiver]) - sender.setParallelism(num_tasks) - receiver.setParallelism(num_tasks) + sender.setParallelism(num_tasks) + receiver.setParallelism(num_tasks) - receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) + receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) - val jobGraph = new JobGraph("Pointwise job", sender, receiver) + val jobGraph = new JobGraph("Pointwise job", sender, receiver) - val cluster = TestingUtils.startTestingCluster(2 * num_tasks) - val jm = cluster.getJobManager + val cluster = TestingUtils.startTestingCluster(2 * num_tasks) + val jm = cluster.getJobManager - try { - within(1 second) { - jm ! SubmitJob(jobGraph) - expectMsg(JobSubmissionResult(JobResult.SUCCESS, null)) + try { + within(1 second) { + jm ! SubmitJob(jobGraph) + expectMsg(SubmissionSuccess(jobGraph.getJobID)) + expectMsgType[JobResultFailed] + expectNoMsg() + } - jm ! RequestJobStatusWhenTerminated(jobGraph.getJobID) - expectMsg(CurrentJobStatus(jobGraph.getJobID, JobStatus.FAILED)) - } + val executionGraph = AkkaUtils.ask[ResponseExecutionGraph](jm, + RequestExecutionGraph(jobGraph.getJobID)) match { + case ExecutionGraphFound(_, eg) => eg + case ExecutionGraphNotFound(jobID) => fail(s"The execution graph for job ID ${jobID} " + + s"was not retrievable.") + } - val executionGraph = AkkaUtils.ask[ResponseExecutionGraph](jm, - RequestExecutionGraph(jobGraph.getJobID)) match { - case ExecutionGraphFound(_, eg) => eg - case ExecutionGraphNotFound(jobID) => fail(s"The execution graph for job ID ${jobID} " + - s"was not retrievable.") + executionGraph.getRegisteredExecutions.size should equal(0) + } finally { + cluster.stop() + } } - executionGraph.getRegisteredExecutions.size should equal(0) - } finally { - cluster.stop() - } - } + "handle job with all vertices failing during instantiation" in { + val num_tasks = 200 + val sender = new AbstractJobVertex("Sender") + val receiver = new AbstractJobVertex("Receiver") - "handle job with all vertices failing during instantiation" in { - val num_tasks = 200 - val sender = new AbstractJobVertex("Sender") - val receiver = new AbstractJobVertex("Receiver") + sender.setInvokableClass(classOf[InstantiationErrorSender]) + receiver.setInvokableClass(classOf[Receiver]) - sender.setInvokableClass(classOf[InstantiationErrorSender]) - receiver.setInvokableClass(classOf[Receiver]) + sender.setParallelism(num_tasks) + receiver.setParallelism(num_tasks) - sender.setParallelism(num_tasks) - receiver.setParallelism(num_tasks) + receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) - receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) + val jobGraph = new JobGraph("Pointwise job", sender, receiver) - val jobGraph = new JobGraph("Pointwise job", sender, receiver) + val cluster = TestingUtils.startTestingCluster(num_tasks) + val jm = cluster.getJobManager - val cluster = TestingUtils.startTestingCluster(num_tasks) - val jm = cluster.getJobManager + try { + within(2 second) { + jm ! RequestAvailableSlots + expectMsg(num_tasks) - try { - within(1 second) { - jm ! RequestAvailableSlots - expectMsg(num_tasks) - } + jm ! SubmitJob(jobGraph) + expectMsg(SubmissionSuccess(jobGraph.getJobID)) + expectMsgType[JobResultFailed] + } - within(1 second) { - jm ! SubmitJob(jobGraph) - expectMsg(JobSubmissionResult(JobResult.SUCCESS, null)) + val executionGraph = AkkaUtils.ask[ResponseExecutionGraph](jm, + RequestExecutionGraph(jobGraph.getJobID)) match { + case ExecutionGraphFound(_, eg) => eg + case ExecutionGraphNotFound(jobID) => fail(s"The execution graph for job ID ${jobID} " + + s"was not retrievable.") + } - jm ! RequestJobStatusWhenTerminated(jobGraph.getJobID) - expectMsg(CurrentJobStatus(jobGraph.getJobID, JobStatus.FAILED)) - } - - val executionGraph = AkkaUtils.ask[ResponseExecutionGraph](jm, - RequestExecutionGraph(jobGraph.getJobID)) match { - case ExecutionGraphFound(_, eg) => eg - case ExecutionGraphNotFound(jobID) => fail(s"The execution graph for job ID ${jobID} " + - s"was not retrievable.") + executionGraph.getRegisteredExecutions.size should equal(0) + } finally { + cluster.stop() + } } - executionGraph.getRegisteredExecutions.size should equal(0) - } finally { - cluster.stop() - } - } + "handle job with some vertices failing during instantiation" in { + val num_tasks = 200 + val sender = new AbstractJobVertex("Sender") + val receiver = new AbstractJobVertex("Receiver") - "handle job with some vertices failing during instantiation" in { - val num_tasks = 200 - val sender = new AbstractJobVertex("Sender") - val receiver = new AbstractJobVertex("Receiver") + sender.setInvokableClass(classOf[SometimesInstantiationErrorSender]) + receiver.setInvokableClass(classOf[Receiver]) - sender.setInvokableClass(classOf[SometimesInstantiationErrorSender]) - receiver.setInvokableClass(classOf[Receiver]) + sender.setParallelism(num_tasks) + receiver.setParallelism(num_tasks) - sender.setParallelism(num_tasks) - receiver.setParallelism(num_tasks) + receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) - receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) + val jobGraph = new JobGraph("Pointwise job", sender, receiver) - val jobGraph = new JobGraph("Pointwise job", sender, receiver) + val cluster = TestingUtils.startTestingCluster(num_tasks) + val jm = cluster.getJobManager - val cluster = TestingUtils.startTestingCluster(num_tasks) - val jm = cluster.getJobManager + try { + within(1 second) { + jm ! RequestAvailableSlots + expectMsg(num_tasks) - try { - within(1 second) { - jm ! RequestAvailableSlots - expectMsg(num_tasks) - } + jm ! SubmitJob(jobGraph) + expectMsg(SubmissionSuccess(jobGraph.getJobID)) + expectMsgType[JobResultFailed] + } - within(1 second) { - jm ! SubmitJob(jobGraph) - expectMsg(JobSubmissionResult(JobResult.SUCCESS, null)) + val executionGraph = AkkaUtils.ask[ResponseExecutionGraph](jm, + RequestExecutionGraph(jobGraph.getJobID)) match { + case ExecutionGraphFound(_, eg) => eg + case ExecutionGraphNotFound(jobID) => fail(s"The execution graph for job ID ${jobID} " + + s"was not retrievable.") + } - jm ! RequestJobStatusWhenTerminated(jobGraph.getJobID) - expectMsg(CurrentJobStatus(jobGraph.getJobID, JobStatus.FAILED)) + executionGraph.getRegisteredExecutions.size should equal(0) + } finally { + cluster.stop() + } } - - val executionGraph = AkkaUtils.ask[ResponseExecutionGraph](jm, - RequestExecutionGraph(jobGraph.getJobID)) match { - case ExecutionGraphFound(_, eg) => eg - case ExecutionGraphNotFound(jobID) => fail(s"The execution graph for job ID ${jobID} " + - s"was not retrievable.") - } - - executionGraph.getRegisteredExecutions.size should equal(0) - } finally { - cluster.stop() - } - } } } diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/SlotSharingITCase.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/SlotSharingITCase.scala index 836176ced5b..92f3971a5b1 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/SlotSharingITCase.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/SlotSharingITCase.scala @@ -1,35 +1,31 @@ /* - * 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. - */ +* 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.jobmanager import akka.actor.ActorSystem import akka.testkit.{ImplicitSender, TestKit} -import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager -import org.apache.flink.runtime.jobgraph.{JobStatus, JobGraph, DistributionPattern, +import org.apache.flink.runtime.jobgraph.{JobGraph, DistributionPattern, AbstractJobVertex} import org.apache.flink.runtime.jobmanager.Tasks.{AgnosticBinaryReceiver, Receiver} import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup -import org.apache.flink.runtime.messages.ExecutionGraphMessages.CurrentJobStatus -import org.apache.flink.runtime.messages.JobManagerMessages.{RequestJobStatusWhenTerminated, +import org.apache.flink.runtime.messages.JobManagerMessages.{JobResultSuccess, SubmissionSuccess, SubmitJob} -import org.apache.flink.runtime.messages.JobResult -import org.apache.flink.runtime.messages.JobResult.JobSubmissionResult import org.apache.flink.runtime.taskmanager.TaskManagerTest.Sender import org.apache.flink.runtime.testingUtils.TestingUtils import org.scalatest.{Matchers, WordSpecLike, BeforeAndAfterAll} @@ -70,10 +66,10 @@ WordSpecLike with Matchers with BeforeAndAfterAll { try { within(1 second) { jm ! SubmitJob(jobGraph) - expectMsg(new JobSubmissionResult(JobResult.SUCCESS, null)) + expectMsg(SubmissionSuccess(jobGraph.getJobID)) + expectMsgType[JobResultSuccess] - jm ! RequestJobStatusWhenTerminated(jobGraph.getJobID) - expectMsg(CurrentJobStatus(jobGraph.getJobID, JobStatus.FINISHED)) + expectNoMsg() } } finally { cluster.stop() @@ -114,10 +110,10 @@ WordSpecLike with Matchers with BeforeAndAfterAll { try { within(1 second) { jm ! SubmitJob(jobGraph) - expectMsg(JobSubmissionResult(JobResult.SUCCESS, null)) + expectMsg(SubmissionSuccess(jobGraph.getJobID)) + expectMsgType[JobResultSuccess] - jm ! RequestJobStatusWhenTerminated(jobGraph.getJobID) - expectMsg(CurrentJobStatus(jobGraph.getJobID, JobStatus.FINISHED)) + expectNoMsg() } } finally { cluster.stop() diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/TaskManagerFailsITCase.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/TaskManagerFailsITCase.scala index 164efe4f80a..fb2e3c7449d 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/TaskManagerFailsITCase.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/TaskManagerFailsITCase.scala @@ -1,33 +1,30 @@ /* - * 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. - */ +* 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.jobmanager import akka.actor.{PoisonPill, ActorSystem} import akka.testkit.{ImplicitSender, TestKit} -import org.apache.flink.runtime.jobgraph.{JobStatus, JobGraph, DistributionPattern, +import org.apache.flink.runtime.jobgraph.{JobGraph, DistributionPattern, AbstractJobVertex} import org.apache.flink.runtime.jobmanager.Tasks.{BlockingReceiver, Sender} -import org.apache.flink.runtime.messages.ExecutionGraphMessages.CurrentJobStatus -import org.apache.flink.runtime.messages.JobManagerMessages.{RequestJobStatusWhenTerminated, +import org.apache.flink.runtime.messages.JobManagerMessages.{JobResultFailed, SubmissionSuccess, SubmitJob} -import org.apache.flink.runtime.messages.JobResult -import org.apache.flink.runtime.messages.JobResult.JobSubmissionResult import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.{AllVerticesRunning, WaitForAllVerticesToBeRunning} import org.apache.flink.runtime.testingUtils.TestingUtils @@ -65,18 +62,16 @@ with WordSpecLike with Matchers with BeforeAndAfterAll { try { within(1 second) { jm ! SubmitJob(jobGraph) - - expectMsg(new JobSubmissionResult(JobResult.SUCCESS, null)) + expectMsg(SubmissionSuccess(jobGraph.getJobID)) jm ! WaitForAllVerticesToBeRunning(jobID) expectMsg(AllVerticesRunning(jobID)) - // kill one task manager taskManagers.get(0) ! PoisonPill + expectMsgType[JobResultFailed] - jm ! RequestJobStatusWhenTerminated(jobGraph.getJobID) - expectMsg(CurrentJobStatus(jobID, JobStatus.FAILED)) + expectNoMsg() } }finally{ cluster.stop() diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/TaskManagerFailsWithSlotSharingITCase.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/TaskManagerFailsWithSlotSharingITCase.scala index 02273166534..38c2a84e10c 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/TaskManagerFailsWithSlotSharingITCase.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/TaskManagerFailsWithSlotSharingITCase.scala @@ -1,34 +1,31 @@ /* - * 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. - */ +* 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.jobmanager import akka.actor.{PoisonPill, ActorSystem} import akka.testkit.{ImplicitSender, TestKit} -import org.apache.flink.runtime.jobgraph.{JobStatus, JobGraph, DistributionPattern, +import org.apache.flink.runtime.jobgraph.{JobGraph, DistributionPattern, AbstractJobVertex} import org.apache.flink.runtime.jobmanager.Tasks.{Sender, BlockingReceiver} import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup -import org.apache.flink.runtime.messages.ExecutionGraphMessages.CurrentJobStatus -import org.apache.flink.runtime.messages.JobManagerMessages.{RequestJobStatusWhenTerminated, +import org.apache.flink.runtime.messages.JobManagerMessages.{JobResultFailed, SubmissionSuccess, SubmitJob} -import org.apache.flink.runtime.messages.JobResult -import org.apache.flink.runtime.messages.JobResult.JobSubmissionResult import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.{AllVerticesRunning, WaitForAllVerticesToBeRunning} import org.apache.flink.runtime.testingUtils.TestingUtils @@ -73,7 +70,7 @@ ImplicitSender with WordSpecLike with Matchers with BeforeAndAfterAll { try{ within(1 second) { jm ! SubmitJob(jobGraph) - expectMsg(new JobSubmissionResult(JobResult.SUCCESS, null)) + expectMsg(SubmissionSuccess(jobGraph.getJobID)) jm ! WaitForAllVerticesToBeRunning(jobID) expectMsg(AllVerticesRunning(jobID)) @@ -81,9 +78,9 @@ ImplicitSender with WordSpecLike with Matchers with BeforeAndAfterAll { //kill task manager taskManagers.get(0) ! PoisonPill - jm ! RequestJobStatusWhenTerminated(jobGraph.getJobID) + expectMsgType[JobResultFailed] - expectMsg(CurrentJobStatus(jobID, JobStatus.FAILED)) + expectNoMsg() } }finally{ cluster.stop() diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala index 60b87b506fa..b0f4a97a579 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala @@ -27,7 +27,7 @@ import org.apache.flink.runtime.net.NetUtils import org.apache.flink.runtime.taskmanager.TaskManager class TestingCluster extends FlinkMiniCluster { - override def getConfiguration(userConfig: Configuration): Configuration = { + override def generateConfiguration(userConfig: Configuration): Configuration = { val cfg = new Configuration() cfg.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost") cfg.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, NetUtils.getAvailablePort) @@ -38,11 +38,9 @@ class TestingCluster extends FlinkMiniCluster { } override def startJobManager(system: ActorSystem, config: Configuration) = { - val (archiveCount, profiling, recommendedPollingInterval, cleanupInterval) = - JobManager.parseConfiguration(config) + val (archiveCount, profiling, cleanupInterval) = JobManager.parseConfiguration(config) - system.actorOf(Props(new JobManager(archiveCount, profiling, recommendedPollingInterval, - cleanupInterval) with + system.actorOf(Props(new JobManager(archiveCount, profiling, cleanupInterval) with TestingJobManager), JobManager.JOB_MANAGER_NAME) } diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingEventCollector.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingEventCollector.scala deleted file mode 100644 index 31021c57a97..00000000000 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingEventCollector.scala +++ /dev/null @@ -1,61 +0,0 @@ -/* - * 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.testingUtils - -import akka.pattern.{ask, pipe} -import org.apache.flink.runtime.ActorLogMessages -import org.apache.flink.runtime.akka.AkkaUtils -import org.apache.flink.runtime.jobmanager.EventCollector -import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.{ExecutionGraphNotFound, ExecutionGraphFound, RequestExecutionGraph} - -import scala.concurrent.Future -import scala.concurrent.duration._ - -trait TestingEventCollector extends ActorLogMessages { - self: EventCollector => - - import context.dispatcher - import org.apache.flink.runtime.akka.AkkaUtils.FUTURE_TIMEOUT - - abstract override def receiveWithLogMessages: Receive = { - receiveTestingMessages orElse super.receiveWithLogMessages - } - - def receiveTestingMessages: Receive = { - case RequestExecutionGraph(jobID) => - recentExecutionGraphs.get(jobID) match { - case Some(executionGraph) => sender() ! ExecutionGraphFound(jobID, executionGraph) - case None => - val responses = archiveListeners map { - listener => - listener ? RequestExecutionGraph(jobID) filter { - case x: ExecutionGraphFound => true - case x: ExecutionGraphNotFound => false - } - } - - val notFound = akka.pattern.after(200 millis, this.context.system.scheduler){ - Future.successful{ExecutionGraphNotFound(jobID)} - } - - Future firstCompletedOf(responses + notFound) pipeTo sender() - } - - } -} diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala index 735ede16927..31f243f833e 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala @@ -1,20 +1,20 @@ /* - * 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. - */ +* 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.testingUtils @@ -22,9 +22,8 @@ import akka.actor.{ActorRef, Props} import org.apache.flink.runtime.ActorLogMessages import org.apache.flink.runtime.execution.ExecutionState import org.apache.flink.runtime.jobgraph.JobID -import org.apache.flink.runtime.jobmanager.{EventCollector, JobManager, MemoryArchivist} +import org.apache.flink.runtime.jobmanager.{JobManager, MemoryArchivist} import org.apache.flink.runtime.messages.ExecutionGraphMessages.ExecutionStateChanged -import org.apache.flink.runtime.messages.JobManagerMessages.UpdateTaskExecutionState import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.{AllVerticesRunning, WaitForAllVerticesToBeRunning, ExecutionGraphFound, RequestExecutionGraph} @@ -38,9 +37,6 @@ trait TestingJobManager extends ActorLogMessages with WrapAsScala { override def archiveProps = Props(new MemoryArchivist(archiveCount) with TestingMemoryArchivist) - override def eventCollectorProps = Props(new EventCollector(recommendedPollingInterval) with - TestingEventCollector) - abstract override def receiveWithLogMessages: Receive = { receiveTestingMessages orElse super.receiveWithLogMessages } @@ -48,21 +44,22 @@ trait TestingJobManager extends ActorLogMessages with WrapAsScala { def receiveTestingMessages: Receive = { case RequestExecutionGraph(jobID) => currentJobs.get(jobID) match { - case Some(executionGraph) => sender() ! ExecutionGraphFound(jobID, executionGraph) - case None => eventCollector.tell(RequestExecutionGraph(jobID), sender()) + case Some((executionGraph, jobInfo)) => sender() ! ExecutionGraphFound(jobID, + executionGraph) + case None => archive.tell(RequestExecutionGraph(jobID), sender()) } case WaitForAllVerticesToBeRunning(jobID) => if(checkIfAllVerticesRunning(jobID)){ sender() ! AllVerticesRunning(jobID) }else{ currentJobs.get(jobID) match { - case Some(eg) => eg.registerExecutionListener(self) + case Some((eg, _)) => eg.registerExecutionListener(self) case None => } val waiting = waitForAllVerticesToBeRunning.getOrElse(jobID, Set[ActorRef]()) waitForAllVerticesToBeRunning += jobID -> (waiting + sender()) } - case ExecutionStateChanged(jobID, _, _, _, _, _) => + case ExecutionStateChanged(jobID, _, _, _, _, _, _) => val cleanup = waitForAllVerticesToBeRunning.get(jobID) match { case Some(listeners) if checkIfAllVerticesRunning(jobID) => for(listener <- listeners){ @@ -80,7 +77,7 @@ trait TestingJobManager extends ActorLogMessages with WrapAsScala { def checkIfAllVerticesRunning(jobID: JobID): Boolean = { currentJobs.get(jobID) match { - case Some(eg) => + case Some((eg, _)) => eg.getAllExecutionVertices.forall( _.getExecutionState == ExecutionState.RUNNING) case None => false } diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala index 6f4a3f86a05..492ae451713 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala @@ -40,9 +40,9 @@ object TestingUtils { s"""akka.daemonic = on |akka.loggers = ["akka.event.slf4j.Slf4jLogger"] - |akka.loglevel = "INFO" + |akka.loglevel = "WARNING" |akka.logging-filter = "akka.event.slf4j.Slf4jLoggingFilter" - |akka.stdout-loglevel = "INFO" + |akka.stdout-loglevel = "WARNING" |akka.jvm-exit-on-fata-error = off |akka.log-config-on-start = off |akka.actor.serializers { diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java b/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java index 023881efbe8..bb30099189c 100644 --- a/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java +++ b/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java @@ -60,7 +60,7 @@ public abstract class AbstractTestBase { protected final Configuration config; protected LocalFlinkMiniCluster executor; - + private final List tempFiles; protected int taskManagerNumSlots = DEFAULT_TASK_MANAGER_NUM_SLOTS; @@ -83,6 +83,7 @@ public abstract class AbstractTestBase { // -------------------------------------------------------------------------------------------- public void startCluster() throws Exception { + Thread.sleep(250); this.executor = new LocalFlinkMiniCluster(null); Configuration config = new Configuration(); config.setBoolean(ConfigConstants.FILESYSTEM_DEFAULT_OVERWRITE_KEY, true); diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java b/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java index 5a552cfb229..ca3d29451e7 100644 --- a/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java +++ b/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java @@ -20,6 +20,7 @@ package org.apache.flink.test.util; import java.util.Comparator; +import akka.actor.ActorRef; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.Plan; import org.apache.flink.compiler.DataStatistics; @@ -194,7 +195,7 @@ public abstract class JavaProgramTestBase extends AbstractTestBase { private static final class TestEnvironment extends ExecutionEnvironment { private final LocalFlinkMiniCluster executor; - + private JobExecutionResult latestResult; @@ -211,10 +212,9 @@ public abstract class JavaProgramTestBase extends AbstractTestBase { NepheleJobGraphGenerator jgg = new NepheleJobGraphGenerator(); JobGraph jobGraph = jgg.compileJobGraph(op); - JobClient client = this.executor.getJobClient(jobGraph); - client.setConsoleStreamForReporting(AbstractTestBase.getNullPrintStream()); - JobExecutionResult result = client.submitJobAndWait(); - + ActorRef client = this.executor.getJobClient(); + JobExecutionResult result = JobClient.submitJobAndWait(jobGraph, false, client); + this.latestResult = result; return result; } diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/util/RecordAPITestBase.java b/flink-test-utils/src/main/java/org/apache/flink/test/util/RecordAPITestBase.java index 93de112b2d4..8c4c3a6c530 100644 --- a/flink-test-utils/src/main/java/org/apache/flink/test/util/RecordAPITestBase.java +++ b/flink-test-utils/src/main/java/org/apache/flink/test/util/RecordAPITestBase.java @@ -19,6 +19,7 @@ package org.apache.flink.test.util; +import akka.actor.ActorRef; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.Plan; import org.apache.flink.compiler.DataStatistics; @@ -119,9 +120,8 @@ public abstract class RecordAPITestBase extends AbstractTestBase { Assert.assertNotNull("Obtained null JobGraph", jobGraph); try { - JobClient client = this.executor.getJobClient(jobGraph); - client.setConsoleStreamForReporting(getNullPrintStream()); - this.jobExecutionResult = client.submitJobAndWait(); + ActorRef client = this.executor.getJobClient(); + this.jobExecutionResult = JobClient.submitJobAndWait(jobGraph, false, client); } catch(Exception e) { System.err.println(e.getMessage()); diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java index 9f9e71a6c00..38a9708a876 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java @@ -224,7 +224,7 @@ public class AccumulatorITCase extends JavaProgramTestBase { /** * Custom accumulator */ - public static class SetAccumulator implements Accumulator> { + public static class SetAccumulator implements Accumulator> { private static final long serialVersionUID = 1L; @@ -236,7 +236,7 @@ public class AccumulatorITCase extends JavaProgramTestBase { } @Override - public Set getLocalValue() { + public SerializableHashSet getLocalValue() { return this.set; } @@ -246,7 +246,7 @@ public class AccumulatorITCase extends JavaProgramTestBase { } @Override - public void merge(Accumulator> other) { + public void merge(Accumulator> other) { // build union this.set.addAll(((SetAccumulator) other).getLocalValue()); } @@ -262,7 +262,7 @@ public class AccumulatorITCase extends JavaProgramTestBase { } @Override - public Accumulator> clone() { + public Accumulator> clone() { SetAccumulator result = new SetAccumulator(); result.set.addAll(set); diff --git a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java index ca43e5a79fe..c967813ee36 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java @@ -19,14 +19,19 @@ package org.apache.flink.test.cancelling; -import java.util.Iterator; +import java.util.concurrent.TimeUnit; +import akka.actor.ActorRef; +import akka.actor.ActorSystem; +import akka.dispatch.ExecutionContexts; +import akka.pattern.Patterns; +import com.amazonaws.http.ExecutionContext; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.messages.JobResult; -import org.apache.flink.runtime.messages.JobResult.JobProgressResult; -import org.apache.flink.runtime.messages.JobResult.JobSubmissionResult; -import org.apache.flink.runtime.messages.JobResult.JobCancelResult; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.runtime.messages.JobClientMessages; +import org.apache.flink.runtime.messages.JobManagerMessages; import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.junit.Assert; @@ -37,15 +42,14 @@ import org.apache.flink.compiler.DataStatistics; import org.apache.flink.compiler.PactCompiler; import org.apache.flink.compiler.plan.OptimizedPlan; import org.apache.flink.compiler.plantranslate.NepheleJobGraphGenerator; -import org.apache.flink.runtime.client.JobClient; -import org.apache.flink.runtime.event.job.AbstractEvent; -import org.apache.flink.runtime.event.job.JobEvent; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.util.StringUtils; import org.apache.hadoop.fs.FileSystem; import org.junit.After; import org.junit.Before; +import scala.concurrent.Await; +import scala.concurrent.Future; +import scala.concurrent.duration.FiniteDuration; /** * @@ -108,124 +112,37 @@ public abstract class CancellingTestBase { try { // submit job final JobGraph jobGraph = getJobGraph(plan); + final ActorRef client = this.executor.getJobClient(); + final ActorSystem actorSystem = executor.getJobClientActorSystem(); + boolean jobSuccessfullyCancelled = false; - final long startingTime = System.currentTimeMillis(); - long cancelTime = -1L; - final JobClient client = this.executor.getJobClient(jobGraph); - final JobSubmissionResult submissionResult = client.submitJob(); - if (submissionResult.returnCode() != JobResult.SUCCESS()) { - throw new IllegalStateException(submissionResult.description()); - } - - - final int interval = client.getRecommendedPollingInterval(); - final long sleep = interval * 1000L; - - Thread.sleep(sleep / 2); - - long lastProcessedEventSequenceNumber = -1L; - - while (true) { - - if (Thread.interrupted()) { - throw new IllegalStateException("Job client has been interrupted"); - } - - final long now = System.currentTimeMillis(); - - if (cancelTime < 0L) { - - // Cancel job - if (startingTime + msecsTillCanceling < now) { - - LOG.info("Issuing cancel request"); - - final JobCancelResult jcr = client.cancelJob(); - - if (jcr == null) { - throw new IllegalStateException("Return value of cancelJob is null!"); - } + Future result = Patterns.ask(client, new JobClientMessages.SubmitJobAndWait + (jobGraph, false), AkkaUtils.FUTURE_TIMEOUT()); - if (jcr.returnCode() != JobResult.SUCCESS()) { - throw new IllegalStateException(jcr.description()); - } - - // Save when the cancel request has been issued - cancelTime = now; - } - } else { - - // Job has already been canceled - if (cancelTime + maxTimeTillCanceled < now) { - throw new IllegalStateException("Cancelling of job took " + (now - cancelTime) - + " milliseconds, only " + maxTimeTillCanceled + " milliseconds are allowed"); - } - } - - final JobProgressResult jobProgressResult = client.getJobProgress(); - - if (jobProgressResult == null) { - throw new IllegalStateException("Returned job progress is unexpectedly null!"); - } - - if (jobProgressResult.returnCode() == JobResult.ERROR()) { - throw new IllegalStateException("Could not retrieve job progress: " - + jobProgressResult.description()); - } - - boolean exitLoop = false; - - final Iterator it = jobProgressResult.asJavaList().iterator(); - while (it.hasNext()) { - - final AbstractEvent event = it.next(); - - // Did we already process that event? - if (lastProcessedEventSequenceNumber >= event.getSequenceNumber()) { - continue; - } - - lastProcessedEventSequenceNumber = event.getSequenceNumber(); - - // Check if we can exit the loop - if (event instanceof JobEvent) { - final JobEvent jobEvent = (JobEvent) event; - final JobStatus jobStatus = jobEvent.getCurrentJobStatus(); - - switch (jobStatus) { - case FINISHED: - throw new IllegalStateException("Job finished successfully"); - case FAILED: - throw new IllegalStateException("Job failed"); - case CANCELED: - exitLoop = true; - break; - case RUNNING: - case CANCELLING: - case FAILING: - case CREATED: - break; + actorSystem.scheduler().scheduleOnce(new FiniteDuration(msecsTillCanceling, + TimeUnit.MILLISECONDS), client, new JobManagerMessages.CancelJob(jobGraph.getJobID()), + actorSystem.dispatcher(), ActorRef.noSender()); case RESTARTING: throw new IllegalStateException("Job restarted"); - } - } - - if (exitLoop) { - break; - } - } - if (exitLoop) { - break; + try { + Await.result(result, AkkaUtils.AWAIT_DURATION()); + } catch (JobExecutionException exception) { + if (!exception.isJobCanceledByUser()) { + throw new IllegalStateException("Job Failed."); } - Thread.sleep(sleep); + jobSuccessfullyCancelled = true; } - } catch (Exception e) { - LOG.error("Exception while running runAndCancelJob.", e); + if (!jobSuccessfullyCancelled) { + throw new IllegalStateException("Job was not successfully cancelled."); + } + }catch(Exception e){ + LOG.error("Exception found in runAndCancelJob.", e); Assert.fail(StringUtils.stringifyException(e)); } + } private JobGraph getJobGraph(final Plan plan) throws Exception { diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java index 26fb4a37abb..eb27a0ae237 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java @@ -45,12 +45,6 @@ public class JobGraphUtils { private JobGraphUtils() {} - - public static void submit(JobGraph graph, Configuration nepheleConfig) throws IOException, JobExecutionException { - JobClient client = new JobClient(graph, nepheleConfig, JobGraphUtils.class.getClassLoader()); - client.submitJobAndWait(); - } - public static > InputFormatVertex createInput(T stub, String path, String name, JobGraph graph, int degreeOfParallelism) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCountAccumulators.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCountAccumulators.java index 6a743639d97..9a032eb9241 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCountAccumulators.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCountAccumulators.java @@ -194,7 +194,7 @@ public class WordCountAccumulators implements Program, ProgramDescription { /** * Custom accumulator */ - public static class SetAccumulator implements Accumulator> { + public static class SetAccumulator implements Accumulator> { private static final long serialVersionUID = 1L; @@ -206,7 +206,7 @@ public class WordCountAccumulators implements Program, ProgramDescription { } @Override - public Set getLocalValue() { + public SerializableHashSet getLocalValue() { return this.set; } @@ -216,7 +216,7 @@ public class WordCountAccumulators implements Program, ProgramDescription { } @Override - public void merge(Accumulator> other) { + public void merge(Accumulator> other) { // build union this.set.addAll(((SetAccumulator) other).getLocalValue()); } @@ -232,7 +232,7 @@ public class WordCountAccumulators implements Program, ProgramDescription { } @Override - public Accumulator> clone() { + public Accumulator> clone() { SetAccumulator result = new SetAccumulator(); result.set.addAll(set); diff --git a/flink-tests/src/test/java/org/apache/flink/test/util/FailingTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/util/FailingTestBase.java index 777ad3b8283..0d27f8deead 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/util/FailingTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/util/FailingTestBase.java @@ -18,6 +18,7 @@ package org.apache.flink.test.util; +import akka.actor.ActorRef; import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.junit.Assert; @@ -142,11 +143,11 @@ public abstract class FailingTestBase extends RecordAPITestBase { */ @Override public void run() { + ActorRef client = this.executor.getJobClient(); + try { // submit failing job - JobClient client = this.executor.getJobClient(this.failingJob); - client.setConsoleStreamForReporting(AbstractTestBase.getNullPrintStream()); - client.submitJobAndWait(); + JobClient.submitJobAndWait(this.failingJob, false, client); this.error = new Exception("The job did not fail."); } catch(JobExecutionException jee) { @@ -158,9 +159,7 @@ public abstract class FailingTestBase extends RecordAPITestBase { try { // submit working job - JobClient client = this.executor.getJobClient(this.job); - client.setConsoleStreamForReporting(AbstractTestBase.getNullPrintStream()); - client.submitJobAndWait(); + JobClient.submitJobAndWait(this.job, false, client); } catch (Exception e) { this.error = e; } diff --git a/flink-tests/src/test/resources/log4j.properties b/flink-tests/src/test/resources/log4j.properties index 2c2d4fffa28..eb65ff48720 100644 --- a/flink-tests/src/test/resources/log4j.properties +++ b/flink-tests/src/test/resources/log4j.properties @@ -17,7 +17,7 @@ ################################################################################ # Set root logger level to DEBUG and its only appender to A1. -log4j.rootLogger=DEBUG, A1 +log4j.rootLogger=WARN, A1 # A1 is set to be a ConsoleAppender. log4j.appender.A1=org.apache.log4j.ConsoleAppender diff --git a/flink-tests/src/test/resources/logback-test.xml b/flink-tests/src/test/resources/logback-test.xml index c4167334ec0..9c2e75fd88b 100644 --- a/flink-tests/src/test/resources/logback-test.xml +++ b/flink-tests/src/test/resources/logback-test.xml @@ -23,7 +23,7 @@ - + diff --git a/pom.xml b/pom.xml index 59afe90ad22..199670943b5 100644 --- a/pom.xml +++ b/pom.xml @@ -448,6 +448,18 @@ under the License. + + + io.netty + netty + + + + + io.netty + netty + + -- GitLab