提交 6b0d4076 编写于 作者: R Robert Metzger

[FLINK-1771] Add support for submitting single jobs to a detached YARN session

With this change, users can submit a Flink job to a YARN cluster without having a local client monitoring the Application Master or job. You can basically fire and forget a Flink job to YARN.
For supporting this, the ApplicationMaster can now monitor the status of a job and shutdown itself once it is in a terminal state.

The change also verifies that various ways of setting the parallelism on YARN are passed through the system correctly (per job, session).

There was a bug in YARN container creation which made the configuration values for the heap offset useless. This change fixes this error.

All mentioned features and bugs are covered by the flink-yarn-tests.

This closes #542
上级 121a5a0c
...@@ -42,6 +42,7 @@ import akka.pattern.Patterns; ...@@ -42,6 +42,7 @@ import akka.pattern.Patterns;
import akka.util.Timeout; import akka.util.Timeout;
import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.CommandLine;
import org.apache.flink.api.common.JobSubmissionResult;
import org.apache.flink.client.cli.CancelOptions; import org.apache.flink.client.cli.CancelOptions;
import org.apache.flink.client.cli.CliArgsException; import org.apache.flink.client.cli.CliArgsException;
import org.apache.flink.client.cli.CliFrontendParser; import org.apache.flink.client.cli.CliFrontendParser;
...@@ -66,7 +67,7 @@ import org.apache.flink.runtime.messages.JobManagerMessages; ...@@ -66,7 +67,7 @@ import org.apache.flink.runtime.messages.JobManagerMessages;
import org.apache.flink.runtime.security.SecurityUtils; import org.apache.flink.runtime.security.SecurityUtils;
import org.apache.flink.runtime.util.EnvironmentInformation; import org.apache.flink.runtime.util.EnvironmentInformation;
import org.apache.flink.runtime.yarn.AbstractFlinkYarnClient; import org.apache.flink.runtime.yarn.AbstractFlinkYarnClient;
import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.api.common.JobID;
import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.JobStatus;
import org.apache.flink.runtime.jobmanager.JobManager; import org.apache.flink.runtime.jobmanager.JobManager;
import org.apache.flink.runtime.messages.JobManagerMessages.CancelJob; import org.apache.flink.runtime.messages.JobManagerMessages.CancelJob;
...@@ -265,12 +266,32 @@ public class CliFrontend { ...@@ -265,12 +266,32 @@ public class CliFrontend {
} }
try { try {
Client client = getClient(options, program.getUserCodeClassLoader(), program.getMainClassName()); int userParallelism = options.getParallelism();
LOG.debug("User parallelism is set to {}", userParallelism);
int parallelism = options.getParallelism();
int exitCode = executeProgram(program, client, parallelism); Client client = getClient(options, program.getUserCodeClassLoader(), program.getMainClassName(), userParallelism);
LOG.debug("Client slots is set to {}", client.getMaxSlots());
if (yarnCluster != null) { if(client.getMaxSlots() != -1 && userParallelism == -1) {
logAndSysout("Using the parallelism provided by the remote cluster ("+client.getMaxSlots()+"). " +
"To use another parallelism, set it at the ./bin/flink client.");
userParallelism = client.getMaxSlots();
}
int exitCode = 0;
// check if detached per job yarn cluster is used to start flink
if(yarnCluster != null && yarnCluster.isDetached()) {
logAndSysout("The Flink YARN client has been started in detached mode. In order to stop " +
"Flink on YARN, use the following command or a YARN web interface to stop it:\n" +
"yarn application -kill "+yarnCluster.getApplicationId()+"\n" +
"Please also note that the temporary files of the YARN session in the home directoy will not be removed.");
executeProgram(program, client, userParallelism, false);
} else {
// regular (blocking) execution.
exitCode = executeProgram(program, client, userParallelism, true);
}
// show YARN cluster status if its not a detached YARN cluster.
if (yarnCluster != null && !yarnCluster.isDetached()) {
List<String> msgs = yarnCluster.getNewMessages(); List<String> msgs = yarnCluster.getNewMessages();
if (msgs != null && msgs.size() > 1) { if (msgs != null && msgs.size() > 1) {
...@@ -291,7 +312,7 @@ public class CliFrontend { ...@@ -291,7 +312,7 @@ public class CliFrontend {
return handleError(t); return handleError(t);
} }
finally { finally {
if (yarnCluster != null) { if (yarnCluster != null && !yarnCluster.isDetached()) {
logAndSysout("Shutting down YARN cluster"); logAndSysout("Shutting down YARN cluster");
yarnCluster.shutdown(); yarnCluster.shutdown();
} }
...@@ -346,7 +367,7 @@ public class CliFrontend { ...@@ -346,7 +367,7 @@ public class CliFrontend {
int parallelism = options.getParallelism(); int parallelism = options.getParallelism();
LOG.info("Creating program plan dump"); LOG.info("Creating program plan dump");
Client client = getClient(options, program.getUserCodeClassLoader(), program.getMainClassName()); Client client = getClient(options, program.getUserCodeClassLoader(), program.getMainClassName(), parallelism);
String jsonPlan = client.getOptimizedPlanAsJson(program, parallelism); String jsonPlan = client.getOptimizedPlanAsJson(program, parallelism);
if (jsonPlan != null) { if (jsonPlan != null) {
...@@ -555,12 +576,12 @@ public class CliFrontend { ...@@ -555,12 +576,12 @@ public class CliFrontend {
// Interaction with programs and JobManager // Interaction with programs and JobManager
// -------------------------------------------------------------------------------------------- // --------------------------------------------------------------------------------------------
protected int executeProgram(PackagedProgram program, Client client, int parallelism) { protected int executeProgram(PackagedProgram program, Client client, int parallelism, boolean wait) {
LOG.info("Starting execution or program"); LOG.info("Starting execution of program");
JobExecutionResult execResult; JobSubmissionResult execResult;
try { try {
client.setPrintStatusDuringExecution(true); client.setPrintStatusDuringExecution(true);
execResult = client.run(program, parallelism, true); execResult = client.run(program, parallelism, wait);
} }
catch (ProgramInvocationException e) { catch (ProgramInvocationException e) {
return handleError(e); return handleError(e);
...@@ -569,15 +590,33 @@ public class CliFrontend { ...@@ -569,15 +590,33 @@ public class CliFrontend {
program.deleteExtractedLibraries(); program.deleteExtractedLibraries();
} }
LOG.info("Program execution finished"); if(wait) {
LOG.info("Program execution finished");
}
// we come here after the job has finished // we come here after the job has finished (or the job has been submitted)
if (execResult != null) { if (execResult != null) {
System.out.println("Job Runtime: " + execResult.getNetRuntime()); // if the job has been submitted to a detached YARN cluster, there won't be any
Map<String, Object> accumulatorsResult = execResult.getAllAccumulatorResults(); // exec results, but the object will be set (for the job id)
if (accumulatorsResult.size() > 0) { if (yarnCluster != null && yarnCluster.isDetached()) {
System.out.println("Accumulator Results: "); if(execResult.getJobID() == null) {
System.out.println(AccumulatorHelper.getResultsFormated(accumulatorsResult)); throw new RuntimeException("Error while starting job. No Job ID set.");
}
yarnCluster.stopAfterJob(execResult.getJobID());
yarnCluster.disconnect();
System.out.println("The Job has been submitted with JobID "+execResult.getJobID());
return 0;
}
if (execResult instanceof JobExecutionResult) {
JobExecutionResult result = (JobExecutionResult) execResult;
System.out.println("Job Runtime: " + result.getNetRuntime());
Map<String, Object> accumulatorsResult = result.getAllAccumulatorResults();
if (accumulatorsResult.size() > 0) {
System.out.println("Accumulator Results: ");
System.out.println(AccumulatorHelper.getResultsFormated(accumulatorsResult));
}
} else {
LOG.info("The Job did not return an execution result");
} }
} }
return 0; return 0;
...@@ -681,26 +720,47 @@ public class CliFrontend { ...@@ -681,26 +720,47 @@ public class CliFrontend {
LOG.info("JobManager is at " + jmActor.path()); LOG.info("JobManager is at " + jmActor.path());
return jmActor; return jmActor;
} }
protected Client getClient(CommandLineOptions options, ClassLoader classLoader, String programName) throws Exception {
/**
*
* @param options
* @param classLoader
* @param programName
* @param userParallelism The parallelism requested by the user in the CLI frontend.
* @return
* @throws Exception
*/
protected Client getClient(CommandLineOptions options, ClassLoader classLoader, String programName, int userParallelism) throws Exception {
InetSocketAddress jobManagerAddress; InetSocketAddress jobManagerAddress;
int maxSlots = -1;
if (YARN_DEPLOY_JOBMANAGER.equals(options.getJobManagerAddress())) { if (YARN_DEPLOY_JOBMANAGER.equals(options.getJobManagerAddress())) {
logAndSysout("YARN cluster mode detected. Switching Log4j output to console"); logAndSysout("YARN cluster mode detected. Switching Log4j output to console");
// user wants to run Flink in YARN cluster. // user wants to run Flink in YARN cluster.
CommandLine commandLine = options.getCommandLine(); CommandLine commandLine = options.getCommandLine();
AbstractFlinkYarnClient flinkYarnClient = AbstractFlinkYarnClient flinkYarnClient = CliFrontendParser.getFlinkYarnSessionCli().createFlinkYarnClient(commandLine);
CliFrontendParser.getFlinkYarnSessionCli().createFlinkYarnClient(commandLine);
if (flinkYarnClient == null) { if (flinkYarnClient == null) {
throw new RuntimeException("Unable to create Flink YARN Client. Check previous log messages"); throw new RuntimeException("Unable to create Flink YARN Client. Check previous log messages");
} }
// the number of slots available from YARN:
int yarnTmSlots = flinkYarnClient.getTaskManagerSlots();
if(yarnTmSlots == -1) {
yarnTmSlots = 1;
}
maxSlots = yarnTmSlots * flinkYarnClient.getTaskManagerCount();
if(userParallelism != -1) {
int slotsPerTM = userParallelism / flinkYarnClient.getTaskManagerCount();
logAndSysout("The YARN cluster has "+maxSlots+" slots available, but the user requested a parallelism of "+userParallelism+" on YARN. " +
"Each of the "+flinkYarnClient.getTaskManagerCount()+" TaskManagers will get "+slotsPerTM+" slots.");
flinkYarnClient.setTaskManagerSlots(slotsPerTM);
}
try { try {
yarnCluster = flinkYarnClient.deploy("Flink Application: " + programName); yarnCluster = flinkYarnClient.deploy("Flink Application: " + programName);
yarnCluster.connectToCluster();
} }
catch(Exception e) { catch(Exception e) {
throw new RuntimeException("Error deploying the YARN cluster", e); throw new RuntimeException("Error deploying the YARN cluster", e);
...@@ -722,7 +782,7 @@ public class CliFrontend { ...@@ -722,7 +782,7 @@ public class CliFrontend {
break; break;
} }
} else { } else {
logAndSysout("No status updates from YARN cluster received so far. Waiting ..."); logAndSysout("No status updates from the YARN cluster received so far. Waiting ...");
} }
try { try {
...@@ -738,7 +798,7 @@ public class CliFrontend { ...@@ -738,7 +798,7 @@ public class CliFrontend {
else { else {
jobManagerAddress = getJobManagerAddress(options); jobManagerAddress = getJobManagerAddress(options);
} }
return new Client(jobManagerAddress, config, classLoader); return new Client(jobManagerAddress, config, classLoader, maxSlots);
} }
// -------------------------------------------------------------------------------------------- // --------------------------------------------------------------------------------------------
......
...@@ -122,12 +122,6 @@ public class FlinkYarnSessionCli { ...@@ -122,12 +122,6 @@ public class FlinkYarnSessionCli {
} else { } else {
LOG.info("No path for the flink jar passed. Using the location of "+flinkYarnClient.getClass()+" to locate the jar"); LOG.info("No path for the flink jar passed. Using the location of "+flinkYarnClient.getClass()+" to locate the jar");
localJarPath = new Path("file://"+flinkYarnClient.getClass().getProtectionDomain().getCodeSource().getLocation().getPath()); localJarPath = new Path("file://"+flinkYarnClient.getClass().getProtectionDomain().getCodeSource().getLocation().getPath());
if(!localJarPath.toString().contains("uberjar")) {
// we need to have a proper uberjar because otherwise we don't have the required classes available on the cluster.
// most likely the user did try to start yarn in a regular hadoop2 flink build (not a yarn package) (using ./bin/flink -m yarn-cluster)
LOG.error("The detected jar file '"+localJarPath+"' is not a uberjar.");
return null;
}
} }
flinkYarnClient.setLocalJarPath(localJarPath); flinkYarnClient.setLocalJarPath(localJarPath);
...@@ -392,6 +386,10 @@ public class FlinkYarnSessionCli { ...@@ -392,6 +386,10 @@ public class FlinkYarnSessionCli {
try { try {
yarnCluster = flinkYarnClient.deploy(null); yarnCluster = flinkYarnClient.deploy(null);
// only connect to cluster if its not a detached session.
if(!flinkYarnClient.isDetached()) {
yarnCluster.connectToCluster();
}
} catch (Exception e) { } catch (Exception e) {
System.err.println("Error while deploying YARN cluster: "+e.getMessage()); System.err.println("Error while deploying YARN cluster: "+e.getMessage());
e.printStackTrace(System.err); e.printStackTrace(System.err);
...@@ -423,7 +421,7 @@ public class FlinkYarnSessionCli { ...@@ -423,7 +421,7 @@ public class FlinkYarnSessionCli {
if (detachedMode) { if (detachedMode) {
// print info and quit: // print info and quit:
LOG.info("The Flink YARN client has been started in detached mode. In order to stop" + LOG.info("The Flink YARN client has been started in detached mode. In order to stop " +
"Flink on YARN, use the following command or a YARN web interface to stop it:\n" + "Flink on YARN, use the following command or a YARN web interface to stop it:\n" +
"yarn application -kill "+yarnCluster.getApplicationId()+"\n" + "yarn application -kill "+yarnCluster.getApplicationId()+"\n" +
"Please also note that the temporary files of the YARN session in {} will not be removed.", "Please also note that the temporary files of the YARN session in {} will not be removed.",
......
...@@ -89,15 +89,20 @@ public class LocalExecutor extends PlanExecutor { ...@@ -89,15 +89,20 @@ public class LocalExecutor extends PlanExecutor {
} }
// -------------------------------------------------------------------------------------------- // --------------------------------------------------------------------------------------------
public static Configuration getConfiguration(LocalExecutor le) {
Configuration configuration = new Configuration();
configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, le.getTaskManagerNumSlots());
configuration.setBoolean(ConfigConstants.FILESYSTEM_DEFAULT_OVERWRITE_KEY, le.isDefaultOverwriteFiles());
return configuration;
}
public void start() throws Exception { public void start() throws Exception {
synchronized (this.lock) { synchronized (this.lock) {
if (this.flink == null) { if (this.flink == null) {
// create the embedded runtime // create the embedded runtime
Configuration configuration = new Configuration(); Configuration configuration = getConfiguration(this);
configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, getTaskManagerNumSlots());
configuration.setBoolean(ConfigConstants.FILESYSTEM_DEFAULT_OVERWRITE_KEY, isDefaultOverwriteFiles());
// start it up // start it up
this.flink = new LocalFlinkMiniCluster(configuration, true); this.flink = new LocalFlinkMiniCluster(configuration, true);
} else { } else {
...@@ -158,7 +163,7 @@ public class LocalExecutor extends PlanExecutor { ...@@ -158,7 +163,7 @@ public class LocalExecutor extends PlanExecutor {
} }
try { try {
Optimizer pc = new Optimizer(new DataStatistics()); Optimizer pc = new Optimizer(new DataStatistics(), this.flink.getConfiguration());
OptimizedPlan op = pc.compile(plan); OptimizedPlan op = pc.compile(plan);
JobGraphGenerator jgg = new JobGraphGenerator(); JobGraphGenerator jgg = new JobGraphGenerator();
...@@ -186,7 +191,7 @@ public class LocalExecutor extends PlanExecutor { ...@@ -186,7 +191,7 @@ public class LocalExecutor extends PlanExecutor {
* @throws Exception * @throws Exception
*/ */
public String getOptimizerPlanAsJSON(Plan plan) throws Exception { public String getOptimizerPlanAsJSON(Plan plan) throws Exception {
Optimizer pc = new Optimizer(new DataStatistics()); Optimizer pc = new Optimizer(new DataStatistics(), getConfiguration(this));
OptimizedPlan op = pc.compile(plan); OptimizedPlan op = pc.compile(plan);
PlanJSONDumpGenerator gen = new PlanJSONDumpGenerator(); PlanJSONDumpGenerator gen = new PlanJSONDumpGenerator();
...@@ -242,7 +247,7 @@ public class LocalExecutor extends PlanExecutor { ...@@ -242,7 +247,7 @@ public class LocalExecutor extends PlanExecutor {
LocalExecutor exec = new LocalExecutor(); LocalExecutor exec = new LocalExecutor();
try { try {
exec.start(); exec.start();
Optimizer pc = new Optimizer(new DataStatistics()); Optimizer pc = new Optimizer(new DataStatistics(), exec.flink.getConfiguration());
OptimizedPlan op = pc.compile(plan); OptimizedPlan op = pc.compile(plan);
PlanJSONDumpGenerator gen = new PlanJSONDumpGenerator(); PlanJSONDumpGenerator gen = new PlanJSONDumpGenerator();
......
...@@ -27,6 +27,7 @@ import java.util.Collections; ...@@ -27,6 +27,7 @@ import java.util.Collections;
import java.util.List; import java.util.List;
import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.api.common.JobSubmissionResult;
import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Plan;
import org.apache.flink.api.common.PlanExecutor; import org.apache.flink.api.common.PlanExecutor;
import org.apache.flink.client.program.Client; import org.apache.flink.client.program.Client;
...@@ -35,11 +36,13 @@ import org.apache.flink.client.program.PackagedProgram; ...@@ -35,11 +36,13 @@ import org.apache.flink.client.program.PackagedProgram;
import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.OptimizedPlan;
import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator; import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator;
import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.Configuration;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class RemoteExecutor extends PlanExecutor { public class RemoteExecutor extends PlanExecutor {
private static final Logger LOG = LoggerFactory.getLogger(RemoteExecutor.class);
private final List<String> jarFiles; private final List<String> jarFiles;
private final InetSocketAddress address; private final InetSocketAddress address;
public RemoteExecutor(String hostname, int port) { public RemoteExecutor(String hostname, int port) {
...@@ -86,22 +89,34 @@ public class RemoteExecutor extends PlanExecutor { ...@@ -86,22 +89,34 @@ public class RemoteExecutor extends PlanExecutor {
} }
public JobExecutionResult executePlanWithJars(JobWithJars p) throws Exception { public JobExecutionResult executePlanWithJars(JobWithJars p) throws Exception {
Client c = new Client(this.address, new Configuration(), p.getUserCodeClassLoader()); Client c = new Client(this.address, new Configuration(), p.getUserCodeClassLoader(), -1);
return c.run(p, -1, true); JobSubmissionResult result = c.run(p, -1, true);
if(result instanceof JobExecutionResult) {
return (JobExecutionResult) result;
} else {
LOG.warn("The Client didn't return a JobExecutionResult");
return new JobExecutionResult(result.getJobID(), -1, null);
}
} }
public JobExecutionResult executeJar(String jarPath, String assemblerClass, String... args) throws Exception { public JobExecutionResult executeJar(String jarPath, String assemblerClass, String... args) throws Exception {
File jarFile = new File(jarPath); File jarFile = new File(jarPath);
PackagedProgram program = new PackagedProgram(jarFile, assemblerClass, args); PackagedProgram program = new PackagedProgram(jarFile, assemblerClass, args);
Client c = new Client(this.address, new Configuration(), program.getUserCodeClassLoader()); Client c = new Client(this.address, new Configuration(), program.getUserCodeClassLoader(), -1);
return c.run(program.getPlanWithJars(), -1, true); JobSubmissionResult result = c.run(program.getPlanWithJars(), -1, true);
if(result instanceof JobExecutionResult) {
return (JobExecutionResult) result;
} else {
LOG.warn("The Client didn't return a JobExecutionResult");
return new JobExecutionResult(result.getJobID(), -1, null);
}
} }
@Override @Override
public String getOptimizerPlanAsJSON(Plan plan) throws Exception { public String getOptimizerPlanAsJSON(Plan plan) throws Exception {
JobWithJars p = new JobWithJars(plan, this.jarFiles); JobWithJars p = new JobWithJars(plan, this.jarFiles);
Client c = new Client(this.address, new Configuration(), p.getUserCodeClassLoader()); Client c = new Client(this.address, new Configuration(), p.getUserCodeClassLoader(), -1);
OptimizedPlan op = (OptimizedPlan) c.getOptimizedPlan(p, -1); OptimizedPlan op = (OptimizedPlan) c.getOptimizedPlan(p, -1);
PlanJSONDumpGenerator jsonGen = new PlanJSONDumpGenerator(); PlanJSONDumpGenerator jsonGen = new PlanJSONDumpGenerator();
......
...@@ -25,6 +25,8 @@ import java.io.PrintStream; ...@@ -25,6 +25,8 @@ import java.io.PrintStream;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.util.List; import java.util.List;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.JobSubmissionResult;
import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.akka.AkkaUtils;
import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Plan;
...@@ -69,6 +71,19 @@ public class Client { ...@@ -69,6 +71,19 @@ public class Client {
private final Optimizer compiler; // the compiler to compile the jobs private final Optimizer compiler; // the compiler to compile the jobs
private boolean printStatusDuringExecution = false; private boolean printStatusDuringExecution = false;
/**
* If != -1, this field specifies the total number of available slots on the cluster
* conntected to the client.
*/
private int maxSlots = -1;
/**
* ID of the last job submitted with this client.
*/
private JobID lastJobId = null;
private ClassLoader userCodeClassLoader; // TODO: use userCodeClassloader to deserialize accumulator results.
// ------------------------------------------------------------------------ // ------------------------------------------------------------------------
// Construction // Construction
...@@ -80,7 +95,7 @@ public class Client { ...@@ -80,7 +95,7 @@ public class Client {
* *
* @param jobManagerAddress Address and port of the job-manager. * @param jobManagerAddress Address and port of the job-manager.
*/ */
public Client(InetSocketAddress jobManagerAddress, Configuration config, ClassLoader userCodeClassLoader) { public Client(InetSocketAddress jobManagerAddress, Configuration config, ClassLoader userCodeClassLoader, int maxSlots) {
Preconditions.checkNotNull(config, "Configuration is null"); Preconditions.checkNotNull(config, "Configuration is null");
this.configuration = config; this.configuration = config;
...@@ -88,7 +103,9 @@ public class Client { ...@@ -88,7 +103,9 @@ public class Client {
configuration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, jobManagerAddress.getAddress().getHostAddress()); configuration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, jobManagerAddress.getAddress().getHostAddress());
configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerAddress.getPort()); configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerAddress.getPort());
this.compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator()); this.compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), configuration);
this.userCodeClassLoader = userCodeClassLoader;
this.maxSlots = maxSlots;
} }
/** /**
...@@ -112,7 +129,8 @@ public class Client { ...@@ -112,7 +129,8 @@ public class Client {
throw new CompilerException("Cannot find port to job manager's RPC service in the global configuration."); throw new CompilerException("Cannot find port to job manager's RPC service in the global configuration.");
} }
this.compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator()); this.compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), configuration);
this.userCodeClassLoader = userCodeClassLoader;
} }
public void setPrintStatusDuringExecution(boolean print) { public void setPrintStatusDuringExecution(boolean print) {
...@@ -126,6 +144,14 @@ public class Client { ...@@ -126,6 +144,14 @@ public class Client {
public int getJobManagerPort() { public int getJobManagerPort() {
return this.configuration.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, -1); return this.configuration.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, -1);
} }
/**
* @return -1 if unknown. The maximum number of available processing slots at the Flink cluster
* connected to this client.
*/
public int getMaxSlots() {
return this.maxSlots;
}
// ------------------------------------------------------------------------ // ------------------------------------------------------------------------
// Compilation and Submission // Compilation and Submission
...@@ -191,8 +217,10 @@ public class Client { ...@@ -191,8 +217,10 @@ public class Client {
public FlinkPlan getOptimizedPlan(Plan p, int parallelism) throws CompilerException { public FlinkPlan getOptimizedPlan(Plan p, int parallelism) throws CompilerException {
if (parallelism > 0 && p.getDefaultParallelism() <= 0) { if (parallelism > 0 && p.getDefaultParallelism() <= 0) {
LOG.debug("Changing plan default parallelism from {} to {}",p.getDefaultParallelism(), parallelism);
p.setDefaultParallelism(parallelism); p.setDefaultParallelism(parallelism);
} }
LOG.debug("Set parallelism {}, plan default parallelism {}", parallelism, p.getDefaultParallelism());
return this.compiler.compile(p); return this.compiler.compile(p);
} }
...@@ -230,49 +258,31 @@ public class Client { ...@@ -230,49 +258,31 @@ public class Client {
return job; return job;
} }
public JobExecutionResult run(final PackagedProgram prog, int parallelism, boolean wait) throws ProgramInvocationException { public JobSubmissionResult run(final PackagedProgram prog, int parallelism, boolean wait) throws ProgramInvocationException {
Thread.currentThread().setContextClassLoader(prog.getUserCodeClassLoader()); Thread.currentThread().setContextClassLoader(prog.getUserCodeClassLoader());
if (prog.isUsingProgramEntryPoint()) { if (prog.isUsingProgramEntryPoint()) {
return run(prog.getPlanWithJars(), parallelism, wait); return run(prog.getPlanWithJars(), parallelism, wait);
} }
else if (prog.isUsingInteractiveMode()) { else if (prog.isUsingInteractiveMode()) {
LOG.info("Starting program in interactive mode");
ContextEnvironment.setAsContext(this, prog.getAllLibraries(), prog.getUserCodeClassLoader(), parallelism); ContextEnvironment.setAsContext(this, prog.getAllLibraries(), prog.getUserCodeClassLoader(), parallelism, wait);
ContextEnvironment.enableLocalExecution(false); ContextEnvironment.enableLocalExecution(false);
if (wait) { // invoke here
// invoke here try {
try { prog.invokeInteractiveModeForExecution();
prog.invokeInteractiveModeForExecution();
}
finally {
ContextEnvironment.enableLocalExecution(true);
}
} }
else { finally {
// invoke in the background ContextEnvironment.enableLocalExecution(true);
Thread backGroundRunner = new Thread("Program Runner") {
public void run() {
try {
prog.invokeInteractiveModeForExecution();
}
catch (Throwable t) {
LOG.error("The program execution failed.", t);
}
finally {
ContextEnvironment.enableLocalExecution(true);
}
}
};
backGroundRunner.start();
} }
return null;
return new JobSubmissionResult(lastJobId);
} }
else { else {
throw new RuntimeException(); throw new RuntimeException();
} }
} }
public JobExecutionResult run(PackagedProgram prog, OptimizedPlan optimizedPlan, boolean wait) throws ProgramInvocationException { public JobSubmissionResult run(PackagedProgram prog, OptimizedPlan optimizedPlan, boolean wait) throws ProgramInvocationException {
return run(optimizedPlan, prog.getAllLibraries(), wait); return run(optimizedPlan, prog.getAllLibraries(), wait);
} }
...@@ -291,17 +301,18 @@ public class Client { ...@@ -291,17 +301,18 @@ public class Client {
* i.e. the job-manager is unreachable, or due to the fact that the * i.e. the job-manager is unreachable, or due to the fact that the
* parallel execution failed. * parallel execution failed.
*/ */
public JobExecutionResult run(JobWithJars prog, int parallelism, boolean wait) throws CompilerException, ProgramInvocationException { public JobSubmissionResult run(JobWithJars prog, int parallelism, boolean wait) throws CompilerException, ProgramInvocationException {
return run((OptimizedPlan) getOptimizedPlan(prog, parallelism), prog.getJarFiles(), wait); return run((OptimizedPlan) getOptimizedPlan(prog, parallelism), prog.getJarFiles(), wait);
} }
public JobExecutionResult run(OptimizedPlan compiledPlan, List<File> libraries, boolean wait) throws ProgramInvocationException { public JobSubmissionResult run(OptimizedPlan compiledPlan, List<File> libraries, boolean wait) throws ProgramInvocationException {
JobGraph job = getJobGraph(compiledPlan, libraries); JobGraph job = getJobGraph(compiledPlan, libraries);
this.lastJobId = job.getJobID();
return run(job, wait); return run(job, wait);
} }
public JobExecutionResult run(JobGraph jobGraph, boolean wait) throws ProgramInvocationException { public JobSubmissionResult run(JobGraph jobGraph, boolean wait) throws ProgramInvocationException {
final String hostname = configuration.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null); final String hostname = configuration.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null);
if (hostname == null) { if (hostname == null) {
...@@ -335,6 +346,8 @@ public class Client { ...@@ -335,6 +346,8 @@ public class Client {
} }
else { else {
JobClient.submitJobDetached(jobGraph, client, timeout); JobClient.submitJobDetached(jobGraph, client, timeout);
// return a "Fake" execution result with the JobId
return new JobSubmissionResult(jobGraph.getJobID());
} }
} }
catch (JobExecutionException e) { catch (JobExecutionException e) {
...@@ -347,8 +360,6 @@ public class Client { ...@@ -347,8 +360,6 @@ public class Client {
actorSystem.shutdown(); actorSystem.shutdown();
actorSystem.awaitTermination(); actorSystem.awaitTermination();
} }
return new JobExecutionResult(-1, null);
} }
// -------------------------------------------------------------------------------------------- // --------------------------------------------------------------------------------------------
......
...@@ -22,37 +22,51 @@ import java.io.File; ...@@ -22,37 +22,51 @@ import java.io.File;
import java.util.List; import java.util.List;
import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.api.common.JobSubmissionResult;
import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Plan;
import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.ExecutionEnvironmentFactory; import org.apache.flink.api.java.ExecutionEnvironmentFactory;
import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.OptimizedPlan;
import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator; import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/** /**
* * Execution Environment for remote execution with the Client.
*/ */
public class ContextEnvironment extends ExecutionEnvironment { public class ContextEnvironment extends ExecutionEnvironment {
private static final Logger LOG = LoggerFactory.getLogger(ContextEnvironment.class);
private final Client client; private final Client client;
private final List<File> jarFilesToAttach; private final List<File> jarFilesToAttach;
private final ClassLoader userCodeClassLoader; private final ClassLoader userCodeClassLoader;
private final boolean wait;
public ContextEnvironment(Client remoteConnection, List<File> jarFiles, ClassLoader userCodeClassLoader) { public ContextEnvironment(Client remoteConnection, List<File> jarFiles, ClassLoader userCodeClassLoader, boolean wait) {
this.client = remoteConnection; this.client = remoteConnection;
this.jarFilesToAttach = jarFiles; this.jarFilesToAttach = jarFiles;
this.userCodeClassLoader = userCodeClassLoader; this.userCodeClassLoader = userCodeClassLoader;
this.wait = wait;
} }
@Override @Override
public JobExecutionResult execute(String jobName) throws Exception { public JobExecutionResult execute(String jobName) throws Exception {
Plan p = createProgramPlan(jobName); Plan p = createProgramPlan(jobName);
JobWithJars toRun = new JobWithJars(p, this.jarFilesToAttach, this.userCodeClassLoader); JobWithJars toRun = new JobWithJars(p, this.jarFilesToAttach, this.userCodeClassLoader);
return this.client.run(toRun, getParallelism(), true); JobSubmissionResult result = this.client.run(toRun, getParallelism(), wait);
if(result instanceof JobExecutionResult) {
return (JobExecutionResult) result;
} else {
LOG.warn("The Client didn't return a JobExecutionResult");
return new JobExecutionResult(result.getJobID(), -1, null);
}
} }
@Override @Override
...@@ -60,7 +74,7 @@ public class ContextEnvironment extends ExecutionEnvironment { ...@@ -60,7 +74,7 @@ public class ContextEnvironment extends ExecutionEnvironment {
Plan p = createProgramPlan("unnamed job"); Plan p = createProgramPlan("unnamed job");
OptimizedPlan op = (OptimizedPlan) this.client.getOptimizedPlan(p, getParallelism()); OptimizedPlan op = (OptimizedPlan) this.client.getOptimizedPlan(p, getParallelism());
PlanJSONDumpGenerator gen = new PlanJSONDumpGenerator(); PlanJSONDumpGenerator gen = new PlanJSONDumpGenerator();
return gen.getOptimizerPlanAsJSON(op); return gen.getOptimizerPlanAsJSON(op);
} }
...@@ -83,15 +97,15 @@ public class ContextEnvironment extends ExecutionEnvironment { ...@@ -83,15 +97,15 @@ public class ContextEnvironment extends ExecutionEnvironment {
// -------------------------------------------------------------------------------------------- // --------------------------------------------------------------------------------------------
static void setAsContext(Client client, List<File> jarFilesToAttach, static void setAsContext(Client client, List<File> jarFilesToAttach,
ClassLoader userCodeClassLoader, int defaultParallelism) ClassLoader userCodeClassLoader, int defaultParallelism, boolean wait)
{ {
initializeContextEnvironment(new ContextEnvironmentFactory(client, jarFilesToAttach, userCodeClassLoader, defaultParallelism)); initializeContextEnvironment(new ContextEnvironmentFactory(client, jarFilesToAttach, userCodeClassLoader, defaultParallelism, wait));
} }
protected static void enableLocalExecution(boolean enabled) { protected static void enableLocalExecution(boolean enabled) {
ExecutionEnvironment.enableLocalExecution(enabled); ExecutionEnvironment.enableLocalExecution(enabled);
} }
// -------------------------------------------------------------------------------------------- // --------------------------------------------------------------------------------------------
public static class ContextEnvironmentFactory implements ExecutionEnvironmentFactory { public static class ContextEnvironmentFactory implements ExecutionEnvironmentFactory {
...@@ -103,20 +117,23 @@ public class ContextEnvironment extends ExecutionEnvironment { ...@@ -103,20 +117,23 @@ public class ContextEnvironment extends ExecutionEnvironment {
private final ClassLoader userCodeClassLoader; private final ClassLoader userCodeClassLoader;
private final int defaultParallelism; private final int defaultParallelism;
private final boolean wait;
public ContextEnvironmentFactory(Client client, List<File> jarFilesToAttach, public ContextEnvironmentFactory(Client client, List<File> jarFilesToAttach,
ClassLoader userCodeClassLoader, int defaultParallelism) ClassLoader userCodeClassLoader, int defaultParallelism, boolean wait)
{ {
this.client = client; this.client = client;
this.jarFilesToAttach = jarFilesToAttach; this.jarFilesToAttach = jarFilesToAttach;
this.userCodeClassLoader = userCodeClassLoader; this.userCodeClassLoader = userCodeClassLoader;
this.defaultParallelism = defaultParallelism; this.defaultParallelism = defaultParallelism;
this.wait = wait;
} }
@Override @Override
public ExecutionEnvironment createExecutionEnvironment() { public ExecutionEnvironment createExecutionEnvironment() {
ContextEnvironment env = new ContextEnvironment(client, jarFilesToAttach, userCodeClassLoader); ContextEnvironment env = new ContextEnvironment(client, jarFilesToAttach, userCodeClassLoader, wait);
if (defaultParallelism > 0) { if (defaultParallelism > 0) {
env.setParallelism(defaultParallelism); env.setParallelism(defaultParallelism);
} }
......
...@@ -105,7 +105,7 @@ public class CliFrontendInfoTest { ...@@ -105,7 +105,7 @@ public class CliFrontendInfoTest {
} }
@Override @Override
protected Client getClient(CommandLineOptions options, ClassLoader loader, String programName) protected Client getClient(CommandLineOptions options, ClassLoader loader, String programName, int par)
throws Exception throws Exception
{ {
return new TestClient(expectedDop); return new TestClient(expectedDop);
...@@ -118,7 +118,7 @@ public class CliFrontendInfoTest { ...@@ -118,7 +118,7 @@ public class CliFrontendInfoTest {
private TestClient(int expectedDop) throws Exception { private TestClient(int expectedDop) throws Exception {
super(new InetSocketAddress(InetAddress.getLocalHost(), 6176), super(new InetSocketAddress(InetAddress.getLocalHost(), 6176),
new Configuration(), CliFrontendInfoTest.class.getClassLoader()); new Configuration(), CliFrontendInfoTest.class.getClassLoader(), -1);
this.expectedDop = expectedDop; this.expectedDop = expectedDop;
} }
......
...@@ -22,7 +22,7 @@ import akka.actor.*; ...@@ -22,7 +22,7 @@ import akka.actor.*;
import akka.testkit.JavaTestKit; import akka.testkit.JavaTestKit;
import org.apache.flink.client.cli.CommandLineOptions; import org.apache.flink.client.cli.CommandLineOptions;
import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.api.common.JobID;
import org.apache.flink.runtime.messages.JobManagerMessages; import org.apache.flink.runtime.messages.JobManagerMessages;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.BeforeClass; import org.junit.BeforeClass;
......
...@@ -93,7 +93,7 @@ public class CliFrontendRunTest { ...@@ -93,7 +93,7 @@ public class CliFrontendRunTest {
} }
@Override @Override
protected int executeProgram(PackagedProgram program, Client client, int parallelism) { protected int executeProgram(PackagedProgram program, Client client, int parallelism, boolean wait) {
assertEquals(this.expectedParallelim, parallelism); assertEquals(this.expectedParallelim, parallelism);
return 0; return 0;
} }
......
...@@ -91,7 +91,7 @@ public class ClientConnectionTest { ...@@ -91,7 +91,7 @@ public class ClientConnectionTest {
vertex.setInvokableClass(TestInvokable.class); vertex.setInvokableClass(TestInvokable.class);
final JobGraph jg = new JobGraph("Test Job", vertex); final JobGraph jg = new JobGraph("Test Job", vertex);
final Client client = new Client(unreachableEndpoint, config, getClass().getClassLoader()); final Client client = new Client(unreachableEndpoint, config, getClass().getClassLoader(), -1);
final AtomicReference<Throwable> error = new AtomicReference<Throwable>(); final AtomicReference<Throwable> error = new AtomicReference<Throwable>();
......
...@@ -24,6 +24,7 @@ import akka.actor.Status; ...@@ -24,6 +24,7 @@ import akka.actor.Status;
import akka.actor.UntypedActor; import akka.actor.UntypedActor;
import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.InvalidProgramException;
import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.api.common.JobSubmissionResult;
import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Plan;
import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.optimizer.DataStatistics; import org.apache.flink.optimizer.DataStatistics;
...@@ -35,7 +36,7 @@ import org.apache.flink.configuration.ConfigConstants; ...@@ -35,7 +36,7 @@ import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.akka.AkkaUtils;
import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobGraph;
import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.api.common.JobID;
import org.apache.flink.runtime.jobmanager.JobManager; import org.apache.flink.runtime.jobmanager.JobManager;
import org.apache.flink.runtime.net.NetUtils; import org.apache.flink.runtime.net.NetUtils;
import org.junit.After; import org.junit.After;
...@@ -101,7 +102,7 @@ public class ClientTest { ...@@ -101,7 +102,7 @@ public class ClientTest {
when(program.getPlanWithJars()).thenReturn(planWithJarsMock); when(program.getPlanWithJars()).thenReturn(planWithJarsMock);
when(planWithJarsMock.getPlan()).thenReturn(planMock); when(planWithJarsMock.getPlan()).thenReturn(planMock);
whenNew(Optimizer.class).withArguments(any(DataStatistics.class), any(CostEstimator.class)).thenReturn(this.compilerMock); whenNew(Optimizer.class).withArguments(any(DataStatistics.class), any(CostEstimator.class), any(Configuration.class)).thenReturn(this.compilerMock);
when(compilerMock.compile(planMock)).thenReturn(optimizedPlanMock); when(compilerMock.compile(planMock)).thenReturn(optimizedPlanMock);
whenNew(JobGraphGenerator.class).withNoArguments().thenReturn(generatorMock); whenNew(JobGraphGenerator.class).withNoArguments().thenReturn(generatorMock);
...@@ -139,11 +140,9 @@ public class ClientTest { ...@@ -139,11 +140,9 @@ public class ClientTest {
jobManagerSystem.actorOf(Props.create(SuccessReturningActor.class), JobManager.JOB_MANAGER_NAME()); jobManagerSystem.actorOf(Props.create(SuccessReturningActor.class), JobManager.JOB_MANAGER_NAME());
Client out = new Client(config, getClass().getClassLoader()); Client out = new Client(config, getClass().getClassLoader());
JobExecutionResult result = out.run(program.getPlanWithJars(), -1, false); JobSubmissionResult result = out.run(program.getPlanWithJars(), -1, false);
assertNotNull(result); assertNotNull(result);
assertEquals(-1, result.getNetRuntime());
assertNull(result.getAllAccumulatorResults());
program.deleteExtractedLibraries(); program.deleteExtractedLibraries();
......
...@@ -71,7 +71,8 @@ public class ExecutionPlanAfterExecutionTest implements java.io.Serializable { ...@@ -71,7 +71,8 @@ public class ExecutionPlanAfterExecutionTest implements java.io.Serializable {
env.getExecutionPlan(); env.getExecutionPlan();
env.createProgramPlan(); env.createProgramPlan();
} catch (Exception e) { } catch (Exception e) {
fail("Cannot run both #getExecutionPlan and #execute."); e.printStackTrace();
fail("Cannot run both #getExecutionPlan and #execute. Message: "+e.getMessage());
} }
} }
} }
...@@ -44,7 +44,7 @@ public class ExecutionPlanCreationTest { ...@@ -44,7 +44,7 @@ public class ExecutionPlanCreationTest {
InetAddress mockAddress = InetAddress.getLocalHost(); InetAddress mockAddress = InetAddress.getLocalHost();
InetSocketAddress mockJmAddress = new InetSocketAddress(mockAddress, 12345); InetSocketAddress mockJmAddress = new InetSocketAddress(mockAddress, 12345);
Client client = new Client(mockJmAddress, new Configuration(), getClass().getClassLoader()); Client client = new Client(mockJmAddress, new Configuration(), getClass().getClassLoader(), -1);
OptimizedPlan op = (OptimizedPlan) client.getOptimizedPlan(prg, -1); OptimizedPlan op = (OptimizedPlan) client.getOptimizedPlan(prg, -1);
assertNotNull(op); assertNotNull(op);
......
...@@ -24,7 +24,7 @@ import java.util.Map; ...@@ -24,7 +24,7 @@ import java.util.Map;
* The result of a job execution. Gives access to the execution time of the job, * The result of a job execution. Gives access to the execution time of the job,
* and to all accumulators created by this job. * and to all accumulators created by this job.
*/ */
public class JobExecutionResult { public class JobExecutionResult extends JobSubmissionResult {
private long netRuntime; private long netRuntime;
private Map<String, Object> accumulatorResults; private Map<String, Object> accumulatorResults;
...@@ -32,10 +32,12 @@ public class JobExecutionResult { ...@@ -32,10 +32,12 @@ public class JobExecutionResult {
/** /**
* Creates a new JobExecutionResult. * Creates a new JobExecutionResult.
* *
* @param jobID
* @param netRuntime The net runtime of the job (excluding pre-flight phase like the optimizer) * @param netRuntime The net runtime of the job (excluding pre-flight phase like the optimizer)
* @param accumulators A map of all accumulators produced by the job. * @param accumulators A map of all accumulators produced by the job.
*/ */
public JobExecutionResult(long netRuntime, Map<String, Object> accumulators) { public JobExecutionResult(JobID jobID, long netRuntime, Map<String, Object> accumulators) {
super(jobID);
this.netRuntime = netRuntime; this.netRuntime = netRuntime;
this.accumulatorResults = accumulators; this.accumulatorResults = accumulators;
} }
...@@ -92,5 +94,6 @@ public class JobExecutionResult { ...@@ -92,5 +94,6 @@ public class JobExecutionResult {
return (Integer) result; return (Integer) result;
} }
// TODO Create convenience methods for the other shipped accumulator types // TODO Create convenience methods for the other shipped accumulator types
} }
...@@ -15,16 +15,15 @@ ...@@ -15,16 +15,15 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.flink.api.common;
package org.apache.flink.runtime.jobgraph;
import javax.xml.bind.DatatypeConverter; import javax.xml.bind.DatatypeConverter;
import org.apache.flink.util.AbstractID; import org.apache.flink.util.AbstractID;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
/**
* Unique Job Identifier
*/
public final class JobID extends AbstractID { public final class JobID extends AbstractID {
private static final long serialVersionUID = 1L; private static final long serialVersionUID = 1L;
......
/*
* 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.api.common;
/**
* The result of a job submission.
* Contains the JobID
*/
public class JobSubmissionResult {
private JobID jobID;
public JobSubmissionResult(JobID jobID) {
this.jobID = jobID;
}
/**
* Returns the JobID assigned to the job by the Flink runtime.
*
* @return jobID, or null if the job has been executed on a runtime without JobIDs or if the execution failed.
*/
public JobID getJobID() {
return jobID;
}
}
...@@ -100,7 +100,7 @@ public class CollectionExecutor { ...@@ -100,7 +100,7 @@ public class CollectionExecutor {
long endTime = System.currentTimeMillis(); long endTime = System.currentTimeMillis();
Map<String, Object> accumulatorResults = AccumulatorHelper.toResultMap(accumulators); Map<String, Object> accumulatorResults = AccumulatorHelper.toResultMap(accumulators);
return new JobExecutionResult(endTime - startTime, accumulatorResults); return new JobExecutionResult(null, endTime - startTime, accumulatorResults);
} }
private List<?> execute(Operator<?> operator) throws Exception { private List<?> execute(Operator<?> operator) throws Exception {
......
...@@ -29,6 +29,7 @@ log4j.appender.file.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m ...@@ -29,6 +29,7 @@ log4j.appender.file.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m
# Log output from org.apache.flink.yarn to the console. This is used by the # Log output from org.apache.flink.yarn to the console. This is used by the
# CliFrontend class when using a per-job YARN cluster. # CliFrontend class when using a per-job YARN cluster.
log4j.logger.org.apache.flink.yarn=INFO, console log4j.logger.org.apache.flink.yarn=INFO, console
log4j.logger.org.apache.flink.client.FlinkYarnSessionCli=INFO, console
log4j.logger.org.apache.hadoop=INFO, console log4j.logger.org.apache.hadoop=INFO, console
log4j.appender.console=org.apache.log4j.ConsoleAppender log4j.appender.console=org.apache.log4j.ConsoleAppender
......
...@@ -54,7 +54,6 @@ public class WordCount { ...@@ -54,7 +54,6 @@ public class WordCount {
public static void main(String[] args) throws Exception { public static void main(String[] args) throws Exception {
if(!parseParameters(args)) { if(!parseParameters(args)) {
return; return;
} }
......
...@@ -52,7 +52,7 @@ under the License. ...@@ -52,7 +52,7 @@ under the License.
<artifactId>flink-java</artifactId> <artifactId>flink-java</artifactId>
<version>${project.version}</version> <version>${project.version}</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>com.google.guava</groupId> <groupId>com.google.guava</groupId>
<artifactId>guava</artifactId> <artifactId>guava</artifactId>
...@@ -60,4 +60,21 @@ under the License. ...@@ -60,4 +60,21 @@ under the License.
</dependency> </dependency>
</dependencies> </dependencies>
<!-- Because flink-tests needs the CompilerTestBsae -->
<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-jar-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>test-jar</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
</project> </project>
...@@ -22,6 +22,7 @@ import java.util.ArrayList; ...@@ -22,6 +22,7 @@ import java.util.ArrayList;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.optimizer.traversals.BinaryUnionReplacer; import org.apache.flink.optimizer.traversals.BinaryUnionReplacer;
import org.apache.flink.optimizer.traversals.BranchesVisitor; import org.apache.flink.optimizer.traversals.BranchesVisitor;
import org.apache.flink.optimizer.traversals.GraphCreatingVisitor; import org.apache.flink.optimizer.traversals.GraphCreatingVisitor;
...@@ -45,7 +46,6 @@ import org.apache.flink.optimizer.plan.SinkJoinerPlanNode; ...@@ -45,7 +46,6 @@ import org.apache.flink.optimizer.plan.SinkJoinerPlanNode;
import org.apache.flink.optimizer.plan.SinkPlanNode; import org.apache.flink.optimizer.plan.SinkPlanNode;
import org.apache.flink.optimizer.postpass.OptimizerPostPass; import org.apache.flink.optimizer.postpass.OptimizerPostPass;
import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.GlobalConfiguration;
import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.InstantiationUtil;
/** /**
...@@ -303,8 +303,8 @@ public class Optimizer { ...@@ -303,8 +303,8 @@ public class Optimizer {
* unknown sizes and hence use only the heuristic cost functions, which result in the selection * unknown sizes and hence use only the heuristic cost functions, which result in the selection
* of the most robust execution strategies. * of the most robust execution strategies.
*/ */
public Optimizer() { public Optimizer(Configuration config) {
this(null, new DefaultCostEstimator()); this(null, new DefaultCostEstimator(), config);
} }
/** /**
...@@ -314,8 +314,8 @@ public class Optimizer { ...@@ -314,8 +314,8 @@ public class Optimizer {
* @param stats * @param stats
* The statistics to be used to determine the input properties. * The statistics to be used to determine the input properties.
*/ */
public Optimizer(DataStatistics stats) { public Optimizer(DataStatistics stats, Configuration config) {
this(stats, new DefaultCostEstimator()); this(stats, new DefaultCostEstimator(), config);
} }
/** /**
...@@ -328,8 +328,8 @@ public class Optimizer { ...@@ -328,8 +328,8 @@ public class Optimizer {
* *
* @param estimator The cost estimator to use to cost the individual operations. * @param estimator The cost estimator to use to cost the individual operations.
*/ */
public Optimizer(CostEstimator estimator) { public Optimizer(CostEstimator estimator, Configuration config) {
this(null, estimator); this(null, estimator, config);
} }
/** /**
...@@ -343,17 +343,17 @@ public class Optimizer { ...@@ -343,17 +343,17 @@ public class Optimizer {
* @param estimator * @param estimator
* The <tt>CostEstimator</tt> to use to cost the individual operations. * The <tt>CostEstimator</tt> to use to cost the individual operations.
*/ */
public Optimizer(DataStatistics stats, CostEstimator estimator) { public Optimizer(DataStatistics stats, CostEstimator estimator, Configuration config) {
this.statistics = stats; this.statistics = stats;
this.costEstimator = estimator; this.costEstimator = estimator;
// determine the default parallelism // determine the default parallelism
// check for old key string first, then for new one // check for old key string first, then for new one
this.defaultParallelism = GlobalConfiguration.getInteger( this.defaultParallelism = config.getInteger(
ConfigConstants.DEFAULT_PARALLELISM_KEY_OLD, ConfigConstants.DEFAULT_PARALLELISM_KEY_OLD,
ConfigConstants.DEFAULT_PARALLELISM); ConfigConstants.DEFAULT_PARALLELISM);
// now check for new one which overwrites old values // now check for new one which overwrites old values
this.defaultParallelism = GlobalConfiguration.getInteger( this.defaultParallelism = config.getInteger(
ConfigConstants.DEFAULT_PARALLELISM_KEY, ConfigConstants.DEFAULT_PARALLELISM_KEY,
this.defaultParallelism); this.defaultParallelism);
......
...@@ -34,6 +34,7 @@ import org.apache.flink.optimizer.util.DummyCrossStub; ...@@ -34,6 +34,7 @@ import org.apache.flink.optimizer.util.DummyCrossStub;
import org.apache.flink.optimizer.util.DummyInputFormat; import org.apache.flink.optimizer.util.DummyInputFormat;
import org.apache.flink.optimizer.util.DummyOutputFormat; import org.apache.flink.optimizer.util.DummyOutputFormat;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType; import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.junit.Test; import org.junit.Test;
/** /**
......
...@@ -30,6 +30,7 @@ import org.apache.flink.api.common.operators.Operator; ...@@ -30,6 +30,7 @@ import org.apache.flink.api.common.operators.Operator;
import org.apache.flink.api.java.io.DiscardingOutputFormat; import org.apache.flink.api.java.io.DiscardingOutputFormat;
import org.apache.flink.optimizer.testfunctions.DummyCoGroupFunction; import org.apache.flink.optimizer.testfunctions.DummyCoGroupFunction;
import org.apache.flink.optimizer.testfunctions.SelectOneReducer; import org.apache.flink.optimizer.testfunctions.SelectOneReducer;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.apache.flink.util.Collector; import org.apache.flink.util.Collector;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
......
...@@ -22,6 +22,7 @@ package org.apache.flink.optimizer; ...@@ -22,6 +22,7 @@ package org.apache.flink.optimizer;
import static org.junit.Assert.*; import static org.junit.Assert.*;
import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Plan;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.junit.Test; import org.junit.Test;
import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.DataSet;
import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.ExecutionEnvironment;
......
...@@ -22,6 +22,7 @@ package org.apache.flink.optimizer; ...@@ -22,6 +22,7 @@ package org.apache.flink.optimizer;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail; import static org.junit.Assert.fail;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.junit.Test; import org.junit.Test;
import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Plan;
import org.apache.flink.api.common.operators.GenericDataSourceBase; import org.apache.flink.api.common.operators.GenericDataSourceBase;
......
...@@ -20,6 +20,7 @@ package org.apache.flink.optimizer; ...@@ -20,6 +20,7 @@ package org.apache.flink.optimizer;
import org.apache.flink.api.common.functions.RichCoGroupFunction; import org.apache.flink.api.common.functions.RichCoGroupFunction;
import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.functions.RichMapFunction;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Plan;
......
...@@ -24,6 +24,7 @@ import org.apache.flink.api.common.Plan; ...@@ -24,6 +24,7 @@ import org.apache.flink.api.common.Plan;
import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.OptimizedPlan;
import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.junit.Test; import org.junit.Test;
@SuppressWarnings("serial") @SuppressWarnings("serial")
......
...@@ -29,6 +29,7 @@ import org.apache.flink.optimizer.plan.SingleInputPlanNode; ...@@ -29,6 +29,7 @@ import org.apache.flink.optimizer.plan.SingleInputPlanNode;
import org.apache.flink.optimizer.plan.SinkPlanNode; import org.apache.flink.optimizer.plan.SinkPlanNode;
import org.apache.flink.optimizer.plan.SourcePlanNode; import org.apache.flink.optimizer.plan.SourcePlanNode;
import org.apache.flink.runtime.operators.DriverStrategy; import org.apache.flink.runtime.operators.DriverStrategy;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.junit.Test; import org.junit.Test;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
......
...@@ -39,6 +39,7 @@ import org.apache.flink.optimizer.util.DummyOutputFormat; ...@@ -39,6 +39,7 @@ import org.apache.flink.optimizer.util.DummyOutputFormat;
import org.apache.flink.optimizer.util.IdentityReduce; import org.apache.flink.optimizer.util.IdentityReduce;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType; import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
import org.apache.flink.runtime.operators.util.LocalStrategy; import org.apache.flink.runtime.operators.util.LocalStrategy;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.apache.flink.types.DoubleValue; import org.apache.flink.types.DoubleValue;
import org.apache.flink.types.IntValue; import org.apache.flink.types.IntValue;
import org.apache.flink.types.LongValue; import org.apache.flink.types.LongValue;
......
...@@ -31,6 +31,7 @@ import org.apache.flink.optimizer.util.DummyInputFormat; ...@@ -31,6 +31,7 @@ import org.apache.flink.optimizer.util.DummyInputFormat;
import org.apache.flink.optimizer.util.DummyOutputFormat; import org.apache.flink.optimizer.util.DummyOutputFormat;
import org.apache.flink.optimizer.util.IdentityMap; import org.apache.flink.optimizer.util.IdentityMap;
import org.apache.flink.optimizer.util.IdentityReduce; import org.apache.flink.optimizer.util.IdentityReduce;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.apache.flink.types.IntValue; import org.apache.flink.types.IntValue;
import org.junit.Test; import org.junit.Test;
......
...@@ -21,6 +21,7 @@ package org.apache.flink.optimizer; ...@@ -21,6 +21,7 @@ package org.apache.flink.optimizer;
import static org.junit.Assert.*; import static org.junit.Assert.*;
import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.functions.RichFlatMapFunction;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.junit.Test; import org.junit.Test;
import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Plan;
import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.DataSet;
......
...@@ -31,6 +31,7 @@ import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; ...@@ -31,6 +31,7 @@ import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
import org.apache.flink.optimizer.testfunctions.DummyFlatJoinFunction; import org.apache.flink.optimizer.testfunctions.DummyFlatJoinFunction;
import org.apache.flink.optimizer.testfunctions.IdentityKeyExtractor; import org.apache.flink.optimizer.testfunctions.IdentityKeyExtractor;
import org.apache.flink.optimizer.testfunctions.IdentityMapper; import org.apache.flink.optimizer.testfunctions.IdentityMapper;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.junit.Test; import org.junit.Test;
@SuppressWarnings({"serial", "unchecked"}) @SuppressWarnings({"serial", "unchecked"})
......
...@@ -29,6 +29,7 @@ import org.apache.flink.optimizer.util.DummyMatchStub; ...@@ -29,6 +29,7 @@ import org.apache.flink.optimizer.util.DummyMatchStub;
import org.apache.flink.optimizer.util.DummyOutputFormat; import org.apache.flink.optimizer.util.DummyOutputFormat;
import org.apache.flink.optimizer.util.IdentityMap; import org.apache.flink.optimizer.util.IdentityMap;
import org.apache.flink.optimizer.util.IdentityReduce; import org.apache.flink.optimizer.util.IdentityReduce;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.junit.Assert; import org.junit.Assert;
import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Plan;
import org.apache.flink.api.java.record.operators.FileDataSink; import org.apache.flink.api.java.record.operators.FileDataSink;
......
...@@ -29,6 +29,7 @@ import org.apache.flink.optimizer.plan.OptimizedPlan; ...@@ -29,6 +29,7 @@ import org.apache.flink.optimizer.plan.OptimizedPlan;
import org.apache.flink.optimizer.plan.SingleInputPlanNode; import org.apache.flink.optimizer.plan.SingleInputPlanNode;
import org.apache.flink.optimizer.plan.SinkPlanNode; import org.apache.flink.optimizer.plan.SinkPlanNode;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType; import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.junit.Test; import org.junit.Test;
@SuppressWarnings("serial") @SuppressWarnings("serial")
......
...@@ -32,6 +32,7 @@ import org.apache.flink.optimizer.dataproperties.PartitioningProperty; ...@@ -32,6 +32,7 @@ import org.apache.flink.optimizer.dataproperties.PartitioningProperty;
import org.apache.flink.optimizer.plan.DualInputPlanNode; import org.apache.flink.optimizer.plan.DualInputPlanNode;
import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.OptimizedPlan;
import org.apache.flink.optimizer.plan.SinkPlanNode; import org.apache.flink.optimizer.plan.SinkPlanNode;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.apache.flink.util.Collector; import org.apache.flink.util.Collector;
import org.junit.Test; import org.junit.Test;
......
...@@ -22,6 +22,7 @@ import static org.junit.Assert.*; ...@@ -22,6 +22,7 @@ import static org.junit.Assert.*;
import org.apache.flink.optimizer.testfunctions.IdentityMapper; import org.apache.flink.optimizer.testfunctions.IdentityMapper;
import org.apache.flink.optimizer.testfunctions.SelectOneReducer; import org.apache.flink.optimizer.testfunctions.SelectOneReducer;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.junit.Test; import org.junit.Test;
import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Plan;
import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.DataSet;
......
...@@ -35,6 +35,7 @@ import org.apache.flink.optimizer.plan.NAryUnionPlanNode; ...@@ -35,6 +35,7 @@ import org.apache.flink.optimizer.plan.NAryUnionPlanNode;
import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.OptimizedPlan;
import org.apache.flink.optimizer.plan.SinkPlanNode; import org.apache.flink.optimizer.plan.SinkPlanNode;
import org.apache.flink.optimizer.plan.SourcePlanNode; import org.apache.flink.optimizer.plan.SourcePlanNode;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
......
...@@ -26,6 +26,7 @@ import org.apache.flink.api.java.record.operators.FileDataSource; ...@@ -26,6 +26,7 @@ import org.apache.flink.api.java.record.operators.FileDataSource;
import org.apache.flink.api.java.record.operators.ReduceOperator; import org.apache.flink.api.java.record.operators.ReduceOperator;
import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.OptimizedPlan;
import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.apache.flink.optimizer.util.DummyInputFormat; import org.apache.flink.optimizer.util.DummyInputFormat;
import org.apache.flink.optimizer.util.DummyOutputFormat; import org.apache.flink.optimizer.util.DummyOutputFormat;
import org.apache.flink.optimizer.util.IdentityReduce; import org.apache.flink.optimizer.util.IdentityReduce;
......
...@@ -39,6 +39,7 @@ import org.apache.flink.optimizer.plan.SinkPlanNode; ...@@ -39,6 +39,7 @@ import org.apache.flink.optimizer.plan.SinkPlanNode;
import org.apache.flink.core.fs.FileInputSplit; import org.apache.flink.core.fs.FileInputSplit;
import org.apache.flink.core.fs.Path; import org.apache.flink.core.fs.Path;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType; import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.apache.flink.util.Collector; import org.apache.flink.util.Collector;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
......
...@@ -38,6 +38,7 @@ import org.apache.flink.optimizer.plan.OptimizedPlan; ...@@ -38,6 +38,7 @@ import org.apache.flink.optimizer.plan.OptimizedPlan;
import org.apache.flink.optimizer.plan.PlanNode; import org.apache.flink.optimizer.plan.PlanNode;
import org.apache.flink.optimizer.plan.SingleInputPlanNode; import org.apache.flink.optimizer.plan.SingleInputPlanNode;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType; import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.apache.flink.util.Visitor; import org.apache.flink.util.Visitor;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
......
...@@ -20,6 +20,7 @@ package org.apache.flink.optimizer; ...@@ -20,6 +20,7 @@ package org.apache.flink.optimizer;
import static org.junit.Assert.*; import static org.junit.Assert.*;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.junit.Test; import org.junit.Test;
import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Plan;
import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.api.common.functions.Partitioner;
......
...@@ -31,6 +31,7 @@ import org.apache.flink.optimizer.plan.NAryUnionPlanNode; ...@@ -31,6 +31,7 @@ import org.apache.flink.optimizer.plan.NAryUnionPlanNode;
import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.OptimizedPlan;
import org.apache.flink.optimizer.plan.SingleInputPlanNode; import org.apache.flink.optimizer.plan.SingleInputPlanNode;
import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.junit.Test; import org.junit.Test;
@SuppressWarnings("serial") @SuppressWarnings("serial")
......
...@@ -31,6 +31,7 @@ import org.apache.flink.api.java.record.operators.FileDataSource; ...@@ -31,6 +31,7 @@ import org.apache.flink.api.java.record.operators.FileDataSource;
import org.apache.flink.api.java.record.operators.ReduceOperator; import org.apache.flink.api.java.record.operators.ReduceOperator;
import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType; import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.apache.flink.types.IntValue; import org.apache.flink.types.IntValue;
import org.apache.flink.util.Collector; import org.apache.flink.util.Collector;
import org.apache.flink.util.Visitor; import org.apache.flink.util.Visitor;
......
...@@ -23,6 +23,7 @@ import org.apache.flink.api.java.ExecutionEnvironment; ...@@ -23,6 +23,7 @@ import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Plan;
import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.OptimizedPlan;
import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.junit.Test; import org.junit.Test;
import static org.junit.Assert.fail; import static org.junit.Assert.fail;
......
...@@ -29,6 +29,7 @@ import org.apache.flink.api.java.tuple.Tuple2; ...@@ -29,6 +29,7 @@ import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.OptimizedPlan;
import org.apache.flink.optimizer.plan.WorksetIterationPlanNode; import org.apache.flink.optimizer.plan.WorksetIterationPlanNode;
import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.junit.Test; import org.junit.Test;
@SuppressWarnings("serial") @SuppressWarnings("serial")
......
...@@ -42,6 +42,7 @@ import org.apache.flink.optimizer.util.DummyOutputFormat; ...@@ -42,6 +42,7 @@ import org.apache.flink.optimizer.util.DummyOutputFormat;
import org.apache.flink.optimizer.util.IdentityMap; import org.apache.flink.optimizer.util.IdentityMap;
import org.apache.flink.optimizer.util.IdentityReduce; import org.apache.flink.optimizer.util.IdentityReduce;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType; import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.apache.flink.types.LongValue; import org.apache.flink.types.LongValue;
import org.junit.Test; import org.junit.Test;
......
...@@ -26,13 +26,13 @@ import org.apache.flink.api.java.DataSet; ...@@ -26,13 +26,13 @@ import org.apache.flink.api.java.DataSet;
import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.optimizer.CompilerTestBase;
import org.apache.flink.optimizer.plan.DualInputPlanNode; import org.apache.flink.optimizer.plan.DualInputPlanNode;
import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.OptimizedPlan;
import org.apache.flink.optimizer.plan.SingleInputPlanNode; import org.apache.flink.optimizer.plan.SingleInputPlanNode;
import org.apache.flink.optimizer.plan.SinkPlanNode; import org.apache.flink.optimizer.plan.SinkPlanNode;
import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
import org.apache.flink.optimizer.testfunctions.DummyCoGroupFunction; import org.apache.flink.optimizer.testfunctions.DummyCoGroupFunction;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType; import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
import org.junit.Test; import org.junit.Test;
......
...@@ -29,13 +29,13 @@ import org.apache.flink.api.java.ExecutionEnvironment; ...@@ -29,13 +29,13 @@ import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.optimizer.CompilerTestBase;
import org.apache.flink.optimizer.plan.DualInputPlanNode; import org.apache.flink.optimizer.plan.DualInputPlanNode;
import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.OptimizedPlan;
import org.apache.flink.optimizer.plan.SinkPlanNode; import org.apache.flink.optimizer.plan.SinkPlanNode;
import org.apache.flink.optimizer.testfunctions.DummyCoGroupFunction; import org.apache.flink.optimizer.testfunctions.DummyCoGroupFunction;
import org.apache.flink.optimizer.testfunctions.IdentityGroupReducer; import org.apache.flink.optimizer.testfunctions.IdentityGroupReducer;
import org.apache.flink.optimizer.testfunctions.IdentityMapper; import org.apache.flink.optimizer.testfunctions.IdentityMapper;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType; import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
import org.junit.Test; import org.junit.Test;
......
...@@ -28,12 +28,12 @@ import org.apache.flink.api.java.DataSet; ...@@ -28,12 +28,12 @@ import org.apache.flink.api.java.DataSet;
import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.optimizer.CompilerTestBase;
import org.apache.flink.optimizer.plan.DualInputPlanNode; import org.apache.flink.optimizer.plan.DualInputPlanNode;
import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.OptimizedPlan;
import org.apache.flink.optimizer.plan.SingleInputPlanNode; import org.apache.flink.optimizer.plan.SingleInputPlanNode;
import org.apache.flink.optimizer.plan.SinkPlanNode; import org.apache.flink.optimizer.plan.SinkPlanNode;
import org.apache.flink.optimizer.testfunctions.IdentityGroupReducer; import org.apache.flink.optimizer.testfunctions.IdentityGroupReducer;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType; import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
import org.junit.Test; import org.junit.Test;
......
...@@ -27,11 +27,11 @@ import org.apache.flink.api.java.DataSet; ...@@ -27,11 +27,11 @@ import org.apache.flink.api.java.DataSet;
import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.optimizer.CompilerTestBase;
import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.OptimizedPlan;
import org.apache.flink.optimizer.plan.SingleInputPlanNode; import org.apache.flink.optimizer.plan.SingleInputPlanNode;
import org.apache.flink.optimizer.plan.SinkPlanNode; import org.apache.flink.optimizer.plan.SinkPlanNode;
import org.apache.flink.optimizer.testfunctions.IdentityPartitionerMapper; import org.apache.flink.optimizer.testfunctions.IdentityPartitionerMapper;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType; import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
import org.junit.Test; import org.junit.Test;
......
...@@ -30,12 +30,12 @@ import org.apache.flink.api.java.functions.KeySelector; ...@@ -30,12 +30,12 @@ import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.optimizer.CompilerTestBase;
import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.OptimizedPlan;
import org.apache.flink.optimizer.plan.SingleInputPlanNode; import org.apache.flink.optimizer.plan.SingleInputPlanNode;
import org.apache.flink.optimizer.plan.SinkPlanNode; import org.apache.flink.optimizer.plan.SinkPlanNode;
import org.apache.flink.optimizer.testfunctions.DummyReducer; import org.apache.flink.optimizer.testfunctions.DummyReducer;
import org.apache.flink.optimizer.testfunctions.IdentityGroupReducer; import org.apache.flink.optimizer.testfunctions.IdentityGroupReducer;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType; import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
import org.junit.Test; import org.junit.Test;
......
...@@ -26,12 +26,12 @@ import org.apache.flink.api.common.functions.Partitioner; ...@@ -26,12 +26,12 @@ import org.apache.flink.api.common.functions.Partitioner;
import org.apache.flink.api.common.operators.Order; import org.apache.flink.api.common.operators.Order;
import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.DataSet;
import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.optimizer.CompilerTestBase;
import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.OptimizedPlan;
import org.apache.flink.optimizer.plan.SingleInputPlanNode; import org.apache.flink.optimizer.plan.SingleInputPlanNode;
import org.apache.flink.optimizer.plan.SinkPlanNode; import org.apache.flink.optimizer.plan.SinkPlanNode;
import org.apache.flink.optimizer.testfunctions.DummyReducer; import org.apache.flink.optimizer.testfunctions.DummyReducer;
import org.apache.flink.optimizer.testfunctions.IdentityGroupReducer; import org.apache.flink.optimizer.testfunctions.IdentityGroupReducer;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType; import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
import org.junit.Test; import org.junit.Test;
......
...@@ -29,12 +29,12 @@ import org.apache.flink.api.java.ExecutionEnvironment; ...@@ -29,12 +29,12 @@ import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.api.java.tuple.Tuple4; import org.apache.flink.api.java.tuple.Tuple4;
import org.apache.flink.optimizer.CompilerTestBase;
import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.OptimizedPlan;
import org.apache.flink.optimizer.plan.SingleInputPlanNode; import org.apache.flink.optimizer.plan.SingleInputPlanNode;
import org.apache.flink.optimizer.plan.SinkPlanNode; import org.apache.flink.optimizer.plan.SinkPlanNode;
import org.apache.flink.optimizer.testfunctions.DummyReducer; import org.apache.flink.optimizer.testfunctions.DummyReducer;
import org.apache.flink.optimizer.testfunctions.IdentityGroupReducer; import org.apache.flink.optimizer.testfunctions.IdentityGroupReducer;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType; import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
import org.junit.Test; import org.junit.Test;
......
...@@ -30,13 +30,13 @@ import org.apache.flink.api.java.ExecutionEnvironment; ...@@ -30,13 +30,13 @@ import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.optimizer.CompilerTestBase;
import org.apache.flink.optimizer.plan.DualInputPlanNode; import org.apache.flink.optimizer.plan.DualInputPlanNode;
import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.OptimizedPlan;
import org.apache.flink.optimizer.plan.SinkPlanNode; import org.apache.flink.optimizer.plan.SinkPlanNode;
import org.apache.flink.optimizer.testfunctions.DummyFlatJoinFunction; import org.apache.flink.optimizer.testfunctions.DummyFlatJoinFunction;
import org.apache.flink.optimizer.testfunctions.IdentityGroupReducer; import org.apache.flink.optimizer.testfunctions.IdentityGroupReducer;
import org.apache.flink.optimizer.testfunctions.IdentityMapper; import org.apache.flink.optimizer.testfunctions.IdentityMapper;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType; import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
import org.junit.Test; import org.junit.Test;
......
...@@ -25,7 +25,6 @@ import org.apache.flink.api.java.DataSet; ...@@ -25,7 +25,6 @@ import org.apache.flink.api.java.DataSet;
import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.io.DiscardingOutputFormat; import org.apache.flink.api.java.io.DiscardingOutputFormat;
import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.optimizer.CompilerTestBase;
import org.apache.flink.optimizer.plan.DualInputPlanNode; import org.apache.flink.optimizer.plan.DualInputPlanNode;
import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.OptimizedPlan;
import org.apache.flink.optimizer.plan.SingleInputPlanNode; import org.apache.flink.optimizer.plan.SingleInputPlanNode;
...@@ -35,6 +34,7 @@ import org.apache.flink.optimizer.testfunctions.DummyFlatJoinFunction; ...@@ -35,6 +34,7 @@ import org.apache.flink.optimizer.testfunctions.DummyFlatJoinFunction;
import org.apache.flink.optimizer.testfunctions.IdentityFlatMapper; import org.apache.flink.optimizer.testfunctions.IdentityFlatMapper;
import org.apache.flink.optimizer.testfunctions.SelectOneReducer; import org.apache.flink.optimizer.testfunctions.SelectOneReducer;
import org.apache.flink.optimizer.testfunctions.Top1GroupReducer; import org.apache.flink.optimizer.testfunctions.Top1GroupReducer;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.apache.flink.runtime.io.network.DataExchangeMode; import org.apache.flink.runtime.io.network.DataExchangeMode;
import org.junit.Test; import org.junit.Test;
......
...@@ -24,12 +24,12 @@ import org.apache.flink.api.common.functions.MapFunction; ...@@ -24,12 +24,12 @@ import org.apache.flink.api.common.functions.MapFunction;
import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.DataSet;
import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.io.DiscardingOutputFormat; import org.apache.flink.api.java.io.DiscardingOutputFormat;
import org.apache.flink.optimizer.CompilerTestBase;
import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.OptimizedPlan;
import org.apache.flink.optimizer.plan.SingleInputPlanNode; import org.apache.flink.optimizer.plan.SingleInputPlanNode;
import org.apache.flink.optimizer.plan.SinkPlanNode; import org.apache.flink.optimizer.plan.SinkPlanNode;
import org.apache.flink.optimizer.testfunctions.IdentityKeyExtractor; import org.apache.flink.optimizer.testfunctions.IdentityKeyExtractor;
import org.apache.flink.optimizer.testfunctions.Top1GroupReducer; import org.apache.flink.optimizer.testfunctions.Top1GroupReducer;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.apache.flink.runtime.io.network.DataExchangeMode; import org.apache.flink.runtime.io.network.DataExchangeMode;
import org.junit.Test; import org.junit.Test;
......
...@@ -25,11 +25,11 @@ import org.apache.flink.api.java.DataSet; ...@@ -25,11 +25,11 @@ import org.apache.flink.api.java.DataSet;
import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.io.DiscardingOutputFormat; import org.apache.flink.api.java.io.DiscardingOutputFormat;
import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.optimizer.CompilerTestBase;
import org.apache.flink.optimizer.plan.DualInputPlanNode; import org.apache.flink.optimizer.plan.DualInputPlanNode;
import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.OptimizedPlan;
import org.apache.flink.optimizer.plan.SingleInputPlanNode; import org.apache.flink.optimizer.plan.SingleInputPlanNode;
import org.apache.flink.optimizer.plan.SinkPlanNode; import org.apache.flink.optimizer.plan.SinkPlanNode;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.apache.flink.runtime.io.network.DataExchangeMode; import org.apache.flink.runtime.io.network.DataExchangeMode;
import org.junit.Test; import org.junit.Test;
......
...@@ -20,6 +20,7 @@ package org.apache.flink.optimizer.java; ...@@ -20,6 +20,7 @@ package org.apache.flink.optimizer.java;
import static org.junit.Assert.fail; import static org.junit.Assert.fail;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.junit.Test; import org.junit.Test;
import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Plan;
...@@ -28,7 +29,6 @@ import org.apache.flink.api.java.ExecutionEnvironment; ...@@ -28,7 +29,6 @@ import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.operators.DeltaIteration; import org.apache.flink.api.java.operators.DeltaIteration;
import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.optimizer.CompilerException; import org.apache.flink.optimizer.CompilerException;
import org.apache.flink.optimizer.CompilerTestBase;
@SuppressWarnings({"serial", "unchecked"}) @SuppressWarnings({"serial", "unchecked"})
......
...@@ -20,12 +20,12 @@ package org.apache.flink.optimizer.java; ...@@ -20,12 +20,12 @@ package org.apache.flink.optimizer.java;
import static org.junit.Assert.*; import static org.junit.Assert.*;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.junit.Test; import org.junit.Test;
import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Plan;
import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.DataSet;
import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.optimizer.CompilerTestBase;
import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.OptimizedPlan;
import org.apache.flink.optimizer.plan.SingleInputPlanNode; import org.apache.flink.optimizer.plan.SingleInputPlanNode;
import org.apache.flink.optimizer.plan.SinkPlanNode; import org.apache.flink.optimizer.plan.SinkPlanNode;
......
...@@ -24,12 +24,12 @@ import org.apache.flink.api.common.functions.RichGroupReduceFunction; ...@@ -24,12 +24,12 @@ import org.apache.flink.api.common.functions.RichGroupReduceFunction;
import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.api.java.operators.GroupReduceOperator; import org.apache.flink.api.java.operators.GroupReduceOperator;
import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.apache.flink.runtime.operators.DriverStrategy; import org.apache.flink.runtime.operators.DriverStrategy;
import org.apache.flink.util.Collector; import org.apache.flink.util.Collector;
import org.junit.Test; import org.junit.Test;
import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.DataSet;
import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.optimizer.CompilerTestBase;
import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.OptimizedPlan;
import org.apache.flink.optimizer.plan.SingleInputPlanNode; import org.apache.flink.optimizer.plan.SingleInputPlanNode;
import org.apache.flink.optimizer.plan.SinkPlanNode; import org.apache.flink.optimizer.plan.SinkPlanNode;
......
...@@ -27,7 +27,6 @@ import org.apache.flink.api.java.ExecutionEnvironment; ...@@ -27,7 +27,6 @@ import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.operators.DeltaIteration; import org.apache.flink.api.java.operators.DeltaIteration;
import org.apache.flink.api.java.operators.IterativeDataSet; import org.apache.flink.api.java.operators.IterativeDataSet;
import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.optimizer.CompilerTestBase;
import org.apache.flink.optimizer.plan.BulkIterationPlanNode; import org.apache.flink.optimizer.plan.BulkIterationPlanNode;
import org.apache.flink.optimizer.plan.NAryUnionPlanNode; import org.apache.flink.optimizer.plan.NAryUnionPlanNode;
import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.OptimizedPlan;
...@@ -36,6 +35,7 @@ import org.apache.flink.optimizer.plan.SinkPlanNode; ...@@ -36,6 +35,7 @@ import org.apache.flink.optimizer.plan.SinkPlanNode;
import org.apache.flink.optimizer.plan.WorksetIterationPlanNode; import org.apache.flink.optimizer.plan.WorksetIterationPlanNode;
import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
import org.apache.flink.optimizer.testfunctions.IdentityMapper; import org.apache.flink.optimizer.testfunctions.IdentityMapper;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.junit.Test; import org.junit.Test;
@SuppressWarnings("serial") @SuppressWarnings("serial")
......
...@@ -27,10 +27,10 @@ import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint; ...@@ -27,10 +27,10 @@ import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint;
import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.DataSet;
import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.optimizer.CompilerTestBase;
import org.apache.flink.optimizer.plan.DualInputPlanNode; import org.apache.flink.optimizer.plan.DualInputPlanNode;
import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.OptimizedPlan;
import org.apache.flink.optimizer.plan.SinkPlanNode; import org.apache.flink.optimizer.plan.SinkPlanNode;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.apache.flink.runtime.operators.DriverStrategy; import org.apache.flink.runtime.operators.DriverStrategy;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType; import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
import org.apache.flink.util.Visitor; import org.apache.flink.util.Visitor;
......
...@@ -27,8 +27,8 @@ import org.apache.flink.api.java.ExecutionEnvironment; ...@@ -27,8 +27,8 @@ import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.operators.DeltaIteration; import org.apache.flink.api.java.operators.DeltaIteration;
import org.apache.flink.api.java.operators.IterativeDataSet; import org.apache.flink.api.java.operators.IterativeDataSet;
import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.optimizer.CompilerTestBase;
import org.apache.flink.optimizer.testfunctions.IdentityMapper; import org.apache.flink.optimizer.testfunctions.IdentityMapper;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.junit.Test; import org.junit.Test;
@SuppressWarnings("serial") @SuppressWarnings("serial")
......
...@@ -27,11 +27,11 @@ import org.apache.flink.api.common.functions.Partitioner; ...@@ -27,11 +27,11 @@ import org.apache.flink.api.common.functions.Partitioner;
import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.DataSet;
import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.optimizer.CompilerTestBase;
import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.OptimizedPlan;
import org.apache.flink.optimizer.plan.SingleInputPlanNode; import org.apache.flink.optimizer.plan.SingleInputPlanNode;
import org.apache.flink.optimizer.plan.SinkPlanNode; import org.apache.flink.optimizer.plan.SinkPlanNode;
import org.apache.flink.optimizer.testfunctions.IdentityGroupReducer; import org.apache.flink.optimizer.testfunctions.IdentityGroupReducer;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType; import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
import org.junit.Test; import org.junit.Test;
......
...@@ -23,10 +23,10 @@ import org.apache.flink.api.common.operators.util.FieldList; ...@@ -23,10 +23,10 @@ import org.apache.flink.api.common.operators.util.FieldList;
import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.api.common.functions.RichReduceFunction; import org.apache.flink.api.common.functions.RichReduceFunction;
import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.junit.Test; import org.junit.Test;
import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.DataSet;
import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.optimizer.CompilerTestBase;
import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.OptimizedPlan;
import org.apache.flink.optimizer.plan.SingleInputPlanNode; import org.apache.flink.optimizer.plan.SingleInputPlanNode;
import org.apache.flink.optimizer.plan.SinkPlanNode; import org.apache.flink.optimizer.plan.SinkPlanNode;
......
...@@ -33,11 +33,11 @@ import org.apache.flink.api.common.functions.RichGroupReduceFunction; ...@@ -33,11 +33,11 @@ import org.apache.flink.api.common.functions.RichGroupReduceFunction;
import org.apache.flink.api.common.functions.RichJoinFunction; import org.apache.flink.api.common.functions.RichJoinFunction;
import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.functions.RichMapFunction;
import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.optimizer.CompilerTestBase;
import org.apache.flink.optimizer.plan.DualInputPlanNode; import org.apache.flink.optimizer.plan.DualInputPlanNode;
import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.OptimizedPlan;
import org.apache.flink.optimizer.plan.SingleInputPlanNode; import org.apache.flink.optimizer.plan.SingleInputPlanNode;
import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType; import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
import org.apache.flink.util.Collector; import org.apache.flink.util.Collector;
import org.junit.Test; import org.junit.Test;
......
...@@ -25,10 +25,10 @@ import org.apache.flink.api.java.DataSet; ...@@ -25,10 +25,10 @@ import org.apache.flink.api.java.DataSet;
import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.optimizer.CompilerException; import org.apache.flink.optimizer.CompilerException;
import org.apache.flink.optimizer.CompilerTestBase;
import org.apache.flink.optimizer.Optimizer; import org.apache.flink.optimizer.Optimizer;
import org.apache.flink.optimizer.testfunctions.DummyCoGroupFunction; import org.apache.flink.optimizer.testfunctions.DummyCoGroupFunction;
import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.Configuration;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.junit.Test; import org.junit.Test;
@SuppressWarnings({"serial", "unchecked"}) @SuppressWarnings({"serial", "unchecked"})
......
...@@ -25,9 +25,9 @@ import org.apache.flink.api.java.DataSet; ...@@ -25,9 +25,9 @@ import org.apache.flink.api.java.DataSet;
import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.optimizer.CompilerException; import org.apache.flink.optimizer.CompilerException;
import org.apache.flink.optimizer.CompilerTestBase;
import org.apache.flink.optimizer.Optimizer; import org.apache.flink.optimizer.Optimizer;
import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.Configuration;
import org.apache.flink.optimizer.util.CompilerTestBase;
import org.junit.Test; import org.junit.Test;
@SuppressWarnings({"serial", "unchecked"}) @SuppressWarnings({"serial", "unchecked"})
......
...@@ -16,7 +16,7 @@ ...@@ -16,7 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */
package org.apache.flink.optimizer; package org.apache.flink.optimizer.util;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
...@@ -30,6 +30,9 @@ import org.apache.flink.api.common.io.FileInputFormat.FileBaseStatistics; ...@@ -30,6 +30,9 @@ import org.apache.flink.api.common.io.FileInputFormat.FileBaseStatistics;
import org.apache.flink.api.common.operators.GenericDataSourceBase; import org.apache.flink.api.common.operators.GenericDataSourceBase;
import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.Operator;
import org.apache.flink.api.common.operators.base.BulkIterationBase; import org.apache.flink.api.common.operators.base.BulkIterationBase;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.optimizer.DataStatistics;
import org.apache.flink.optimizer.Optimizer;
import org.apache.flink.optimizer.costs.DefaultCostEstimator; import org.apache.flink.optimizer.costs.DefaultCostEstimator;
import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.OptimizedPlan;
import org.apache.flink.optimizer.plan.PlanNode; import org.apache.flink.optimizer.plan.PlanNode;
...@@ -44,82 +47,90 @@ import org.junit.Before; ...@@ -44,82 +47,90 @@ import org.junit.Before;
* the the node in the program plan. * the the node in the program plan.
*/ */
public abstract class CompilerTestBase implements java.io.Serializable { public abstract class CompilerTestBase implements java.io.Serializable {
private static final long serialVersionUID = 1L; private static final long serialVersionUID = 1L;
protected static final String IN_FILE = OperatingSystem.isWindows() ? "file:/c:/" : "file:///dev/random"; protected static final String IN_FILE = OperatingSystem.isWindows() ? "file:/c:/" : "file:///dev/random";
protected static final String OUT_FILE = OperatingSystem.isWindows() ? "file:/c:/" : "file:///dev/null"; protected static final String OUT_FILE = OperatingSystem.isWindows() ? "file:/c:/" : "file:///dev/null";
protected static final int DEFAULT_PARALLELISM = 8; protected static final int DEFAULT_PARALLELISM = 8;
protected static final String DEFAULT_PARALLELISM_STRING = String.valueOf(DEFAULT_PARALLELISM);
private static final String CACHE_KEY = "cachekey"; private static final String CACHE_KEY = "cachekey";
// ------------------------------------------------------------------------ // ------------------------------------------------------------------------
protected transient DataStatistics dataStats; protected transient DataStatistics dataStats;
protected transient Optimizer withStatsCompiler; protected transient Optimizer withStatsCompiler;
protected transient Optimizer noStatsCompiler; protected transient Optimizer noStatsCompiler;
private transient int statCounter; private transient int statCounter;
// ------------------------------------------------------------------------ // ------------------------------------------------------------------------
@Before @Before
public void setup() { public void setup() {
Configuration flinkConf = new Configuration();
this.dataStats = new DataStatistics(); this.dataStats = new DataStatistics();
this.withStatsCompiler = new Optimizer(this.dataStats, new DefaultCostEstimator()); this.withStatsCompiler = new Optimizer(this.dataStats, new DefaultCostEstimator(), flinkConf);
this.withStatsCompiler.setDefaultParallelism(DEFAULT_PARALLELISM); this.withStatsCompiler.setDefaultParallelism(DEFAULT_PARALLELISM);
this.noStatsCompiler = new Optimizer(null, new DefaultCostEstimator()); this.noStatsCompiler = new Optimizer(null, new DefaultCostEstimator(), flinkConf);
this.noStatsCompiler.setDefaultParallelism(DEFAULT_PARALLELISM); this.noStatsCompiler.setDefaultParallelism(DEFAULT_PARALLELISM);
} }
// ------------------------------------------------------------------------ // ------------------------------------------------------------------------
public OptimizedPlan compileWithStats(Plan p) { public OptimizedPlan compileWithStats(Plan p) {
return this.withStatsCompiler.compile(p); return this.withStatsCompiler.compile(p);
} }
public OptimizedPlan compileNoStats(Plan p) { public OptimizedPlan compileNoStats(Plan p) {
return this.noStatsCompiler.compile(p); return this.noStatsCompiler.compile(p);
} }
public static OperatorResolver getContractResolver(Plan plan) {
return new OperatorResolver(plan);
}
public void setSourceStatistics(GenericDataSourceBase<?, ?> source, long size, float recordWidth) { public void setSourceStatistics(GenericDataSourceBase<?, ?> source, long size, float recordWidth) {
setSourceStatistics(source, new FileBaseStatistics(Long.MAX_VALUE, size, recordWidth)); setSourceStatistics(source, new FileBaseStatistics(Long.MAX_VALUE, size, recordWidth));
} }
public void setSourceStatistics(GenericDataSourceBase<?, ?> source, FileBaseStatistics stats) { public void setSourceStatistics(GenericDataSourceBase<?, ?> source, FileBaseStatistics stats) {
final String key = CACHE_KEY + this.statCounter++; final String key = CACHE_KEY + this.statCounter++;
this.dataStats.cacheBaseStatistics(stats, key); this.dataStats.cacheBaseStatistics(stats, key);
source.setStatisticsKey(key); source.setStatisticsKey(key);
} }
// ------------------------------------------------------------------------
public static OptimizerPlanNodeResolver getOptimizerPlanNodeResolver(OptimizedPlan plan) { public static OptimizerPlanNodeResolver getOptimizerPlanNodeResolver(OptimizedPlan plan) {
return new OptimizerPlanNodeResolver(plan); return new OptimizerPlanNodeResolver(plan);
} }
// ------------------------------------------------------------------------
public static final class OptimizerPlanNodeResolver { public static final class OptimizerPlanNodeResolver {
private final Map<String, ArrayList<PlanNode>> map; private final Map<String, ArrayList<PlanNode>> map;
OptimizerPlanNodeResolver(OptimizedPlan p) { public OptimizerPlanNodeResolver(OptimizedPlan p) {
HashMap<String, ArrayList<PlanNode>> map = new HashMap<String, ArrayList<PlanNode>>(); HashMap<String, ArrayList<PlanNode>> map = new HashMap<String, ArrayList<PlanNode>>();
for (PlanNode n : p.getAllNodes()) { for (PlanNode n : p.getAllNodes()) {
Operator<?> c = n.getOriginalOptimizerNode().getOperator(); Operator<?> c = n.getOriginalOptimizerNode().getOperator();
String name = c.getName(); String name = c.getName();
ArrayList<PlanNode> list = map.get(name); ArrayList<PlanNode> list = map.get(name);
if (list == null) { if (list == null) {
list = new ArrayList<PlanNode>(2); list = new ArrayList<PlanNode>(2);
map.put(name, list); map.put(name, list);
} }
// check whether this node is a child of a node with the same contract (aka combiner) // check whether this node is a child of a node with the same contract (aka combiner)
boolean shouldAdd = true; boolean shouldAdd = true;
for (Iterator<PlanNode> iter = list.iterator(); iter.hasNext();) { for (Iterator<PlanNode> iter = list.iterator(); iter.hasNext();) {
...@@ -129,7 +140,7 @@ public abstract class CompilerTestBase implements java.io.Serializable { ...@@ -129,7 +140,7 @@ public abstract class CompilerTestBase implements java.io.Serializable {
if (in instanceof SingleInputPlanNode && n instanceof SingleInputPlanNode) { if (in instanceof SingleInputPlanNode && n instanceof SingleInputPlanNode) {
SingleInputPlanNode thisNode = (SingleInputPlanNode) n; SingleInputPlanNode thisNode = (SingleInputPlanNode) n;
SingleInputPlanNode otherNode = (SingleInputPlanNode) in; SingleInputPlanNode otherNode = (SingleInputPlanNode) in;
if (thisNode.getPredecessor() == otherNode) { if (thisNode.getPredecessor() == otherNode) {
// other node is child, remove it // other node is child, remove it
iter.remove(); iter.remove();
...@@ -141,16 +152,16 @@ public abstract class CompilerTestBase implements java.io.Serializable { ...@@ -141,16 +152,16 @@ public abstract class CompilerTestBase implements java.io.Serializable {
} }
} }
} }
if (shouldAdd) { if (shouldAdd) {
list.add(n); list.add(n);
} }
} }
this.map = map; this.map = map;
} }
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
public <T extends PlanNode> T getNode(String name) { public <T extends PlanNode> T getNode(String name) {
List<PlanNode> nodes = this.map.get(name); List<PlanNode> nodes = this.map.get(name);
...@@ -162,7 +173,7 @@ public abstract class CompilerTestBase implements java.io.Serializable { ...@@ -162,7 +173,7 @@ public abstract class CompilerTestBase implements java.io.Serializable {
return (T) nodes.get(0); return (T) nodes.get(0);
} }
} }
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
public <T extends PlanNode> T getNode(String name, Class<? extends Function> stubClass) { public <T extends PlanNode> T getNode(String name, Class<? extends Function> stubClass) {
List<PlanNode> nodes = this.map.get(name); List<PlanNode> nodes = this.map.get(name);
...@@ -186,7 +197,7 @@ public abstract class CompilerTestBase implements java.io.Serializable { ...@@ -186,7 +197,7 @@ public abstract class CompilerTestBase implements java.io.Serializable {
} }
} }
} }
public List<PlanNode> getNodes(String name) { public List<PlanNode> getNodes(String name) {
List<PlanNode> nodes = this.map.get(name); List<PlanNode> nodes = this.map.get(name);
if (nodes == null || nodes.isEmpty()) { if (nodes == null || nodes.isEmpty()) {
...@@ -202,28 +213,28 @@ public abstract class CompilerTestBase implements java.io.Serializable { ...@@ -202,28 +213,28 @@ public abstract class CompilerTestBase implements java.io.Serializable {
* *
*/ */
public static class SourceCollectorVisitor implements Visitor<Operator<?>> { public static class SourceCollectorVisitor implements Visitor<Operator<?>> {
protected final List<GenericDataSourceBase<?, ?>> sources = new ArrayList<GenericDataSourceBase<?, ?>>(4); protected final List<GenericDataSourceBase<?, ?>> sources = new ArrayList<GenericDataSourceBase<?, ?>>(4);
@Override @Override
public boolean preVisit(Operator<?> visitable) { public boolean preVisit(Operator<?> visitable) {
if(visitable instanceof GenericDataSourceBase) { if(visitable instanceof GenericDataSourceBase) {
sources.add((GenericDataSourceBase<?, ?>) visitable); sources.add((GenericDataSourceBase<?, ?>) visitable);
} }
else if(visitable instanceof BulkIterationBase) { else if(visitable instanceof BulkIterationBase) {
((BulkIterationBase<?>) visitable).getNextPartialSolution().accept(this); ((BulkIterationBase<?>) visitable).getNextPartialSolution().accept(this);
} }
return true; return true;
} }
@Override @Override
public void postVisit(Operator<?> visitable) {} public void postVisit(Operator<?> visitable) {}
public List<GenericDataSourceBase<?, ?>> getSources() { public List<GenericDataSourceBase<?, ?>> getSources() {
return this.sources; return this.sources;
} }
} }
} }
...@@ -17,7 +17,7 @@ ...@@ -17,7 +17,7 @@
*/ */
package org.apache.flink.test.compiler.util; package org.apache.flink.optimizer.util;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
......
...@@ -59,7 +59,7 @@ under the License. ...@@ -59,7 +59,7 @@ under the License.
<artifactId>maven-archetype-plugin</artifactId> <artifactId>maven-archetype-plugin</artifactId>
<version>2.2</version><!--$NO-MVN-MAN-VER$--> <version>2.2</version><!--$NO-MVN-MAN-VER$-->
<configuration> <configuration>
<skip>${skipTests}</skip> <skip>true</skip>
</configuration> </configuration>
</plugin> </plugin>
<!-- deactivate the shade plugin for the quickstart archetypes --> <!-- deactivate the shade plugin for the quickstart archetypes -->
......
...@@ -29,7 +29,7 @@ import java.util.HashMap; ...@@ -29,7 +29,7 @@ import java.util.HashMap;
import java.util.Map; import java.util.Map;
import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.accumulators.Accumulator;
import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.api.common.JobID;
import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.InstantiationUtil;
/** /**
......
...@@ -32,7 +32,7 @@ import org.apache.flink.util.InstantiationUtil; ...@@ -32,7 +32,7 @@ import org.apache.flink.util.InstantiationUtil;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.api.common.JobID;
import static org.apache.flink.runtime.blob.BlobServerProtocol.CONTENT_ADDRESSABLE; import static org.apache.flink.runtime.blob.BlobServerProtocol.CONTENT_ADDRESSABLE;
import static org.apache.flink.runtime.blob.BlobServerProtocol.JOB_ID_SCOPE; import static org.apache.flink.runtime.blob.BlobServerProtocol.JOB_ID_SCOPE;
......
...@@ -33,7 +33,7 @@ import java.util.concurrent.atomic.AtomicInteger; ...@@ -33,7 +33,7 @@ import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.api.common.JobID;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
......
...@@ -29,7 +29,7 @@ import java.net.Socket; ...@@ -29,7 +29,7 @@ import java.net.Socket;
import java.net.SocketException; import java.net.SocketException;
import java.security.MessageDigest; import java.security.MessageDigest;
import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.api.common.JobID;
import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.InstantiationUtil;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
......
...@@ -20,7 +20,7 @@ package org.apache.flink.runtime.blob; ...@@ -20,7 +20,7 @@ package org.apache.flink.runtime.blob;
import com.google.common.io.BaseEncoding; import com.google.common.io.BaseEncoding;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.api.common.JobID;
import org.slf4j.Logger; import org.slf4j.Logger;
import java.io.EOFException; import java.io.EOFException;
......
...@@ -18,7 +18,7 @@ ...@@ -18,7 +18,7 @@
package org.apache.flink.runtime.client; package org.apache.flink.runtime.client;
import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.api.common.JobID;
/** /**
* An exception which is thrown by the JobClient if a job is aborted as a result of a user * An exception which is thrown by the JobClient if a job is aborted as a result of a user
......
...@@ -18,7 +18,7 @@ ...@@ -18,7 +18,7 @@
package org.apache.flink.runtime.client; package org.apache.flink.runtime.client;
import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.api.common.JobID;
/** /**
* This exception is the base exception for all exceptions that denote any failure during * This exception is the base exception for all exceptions that denote any failure during
......
...@@ -18,7 +18,7 @@ ...@@ -18,7 +18,7 @@
package org.apache.flink.runtime.client; package org.apache.flink.runtime.client;
import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.api.common.JobID;
import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.JobStatus;
/** /**
......
...@@ -18,7 +18,7 @@ ...@@ -18,7 +18,7 @@
package org.apache.flink.runtime.client; package org.apache.flink.runtime.client;
import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.api.common.JobID;
/** /**
* This exception denotes an error while submitting a job to the JobManager * This exception denotes an error while submitting a job to the JobManager
......
...@@ -18,7 +18,7 @@ ...@@ -18,7 +18,7 @@
package org.apache.flink.runtime.client; package org.apache.flink.runtime.client;
import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.api.common.JobID;
/** /**
* An exception which is thrown by the JobClient if the job manager is no longer reachable. * An exception which is thrown by the JobClient if the job manager is no longer reachable.
......
...@@ -21,7 +21,7 @@ package org.apache.flink.runtime.deployment; ...@@ -21,7 +21,7 @@ package org.apache.flink.runtime.deployment;
import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.blob.BlobKey;
import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.api.common.JobID;
import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.state.StateHandle; import org.apache.flink.runtime.state.StateHandle;
......
...@@ -26,7 +26,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager; ...@@ -26,7 +26,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
import org.apache.flink.runtime.io.network.partition.consumer.InputGate; import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobGraph;
import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.api.common.JobID;
import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
import org.apache.flink.runtime.memorymanager.MemoryManager; import org.apache.flink.runtime.memorymanager.MemoryManager;
......
...@@ -33,7 +33,7 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionID; ...@@ -33,7 +33,7 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
import org.apache.flink.runtime.io.network.partition.consumer.InputGate; import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate; import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.api.common.JobID;
import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
......
...@@ -35,7 +35,7 @@ import java.util.TimerTask; ...@@ -35,7 +35,7 @@ import java.util.TimerTask;
import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.blob.BlobKey;
import org.apache.flink.runtime.blob.BlobService; import org.apache.flink.runtime.blob.BlobService;
import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.api.common.JobID;
import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.ExceptionUtils;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
......
...@@ -20,7 +20,7 @@ package org.apache.flink.runtime.execution.librarycache; ...@@ -20,7 +20,7 @@ package org.apache.flink.runtime.execution.librarycache;
import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.blob.BlobKey;
import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.api.common.JobID;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
......
...@@ -20,7 +20,7 @@ package org.apache.flink.runtime.execution.librarycache; ...@@ -20,7 +20,7 @@ package org.apache.flink.runtime.execution.librarycache;
import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.blob.BlobKey;
import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.api.common.JobID;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
......
...@@ -28,7 +28,7 @@ import org.apache.flink.runtime.execution.ExecutionState; ...@@ -28,7 +28,7 @@ import org.apache.flink.runtime.execution.ExecutionState;
import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
import org.apache.flink.runtime.jobgraph.AbstractJobVertex; import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.api.common.JobID;
import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.JobStatus;
import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.jobgraph.ScheduleMode; import org.apache.flink.runtime.jobgraph.ScheduleMode;
......
...@@ -29,7 +29,7 @@ import org.apache.flink.runtime.jobgraph.AbstractJobVertex; ...@@ -29,7 +29,7 @@ import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
import org.apache.flink.runtime.jobgraph.IntermediateDataSet; import org.apache.flink.runtime.jobgraph.IntermediateDataSet;
import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
import org.apache.flink.runtime.jobgraph.JobEdge; import org.apache.flink.runtime.jobgraph.JobEdge;
import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.api.common.JobID;
import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup; import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup;
import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException; import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
......
...@@ -34,7 +34,7 @@ import org.apache.flink.runtime.jobgraph.DistributionPattern; ...@@ -34,7 +34,7 @@ import org.apache.flink.runtime.jobgraph.DistributionPattern;
import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
import org.apache.flink.runtime.jobgraph.JobEdge; import org.apache.flink.runtime.jobgraph.JobEdge;
import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.api.common.JobID;
import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.jobmanager.scheduler.CoLocationConstraint; import org.apache.flink.runtime.jobmanager.scheduler.CoLocationConstraint;
import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup; import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup;
......
...@@ -38,7 +38,7 @@ import org.apache.flink.core.fs.FileStatus; ...@@ -38,7 +38,7 @@ import org.apache.flink.core.fs.FileStatus;
import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.core.fs.Path; import org.apache.flink.core.fs.Path;
import org.apache.flink.core.fs.local.LocalFileSystem; import org.apache.flink.core.fs.local.LocalFileSystem;
import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.api.common.JobID;
import org.apache.flink.runtime.util.ExecutorThreadFactory; import org.apache.flink.runtime.util.ExecutorThreadFactory;
import org.apache.flink.runtime.util.IOUtils; import org.apache.flink.runtime.util.IOUtils;
import org.slf4j.Logger; import org.slf4j.Logger;
......
...@@ -27,7 +27,7 @@ import java.util.Set; ...@@ -27,7 +27,7 @@ import java.util.Set;
import akka.actor.ActorRef; import akka.actor.ActorRef;
import org.apache.flink.util.AbstractID; import org.apache.flink.util.AbstractID;
import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.api.common.JobID;
import org.apache.flink.runtime.jobmanager.scheduler.SlotAvailabilityListener; import org.apache.flink.runtime.jobmanager.scheduler.SlotAvailabilityListener;
import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroupAssignment; import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroupAssignment;
......
...@@ -19,7 +19,7 @@ ...@@ -19,7 +19,7 @@
package org.apache.flink.runtime.instance; package org.apache.flink.runtime.instance;
import org.apache.flink.util.AbstractID; import org.apache.flink.util.AbstractID;
import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.api.common.JobID;
import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroupAssignment; import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroupAssignment;
import java.util.HashSet; import java.util.HashSet;
......
...@@ -20,7 +20,7 @@ package org.apache.flink.runtime.instance; ...@@ -20,7 +20,7 @@ package org.apache.flink.runtime.instance;
import org.apache.flink.util.AbstractID; import org.apache.flink.util.AbstractID;
import org.apache.flink.runtime.executiongraph.Execution; import org.apache.flink.runtime.executiongraph.Execution;
import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.api.common.JobID;
import org.apache.flink.runtime.jobmanager.scheduler.Locality; import org.apache.flink.runtime.jobmanager.scheduler.Locality;
import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
......
...@@ -19,7 +19,7 @@ ...@@ -19,7 +19,7 @@
package org.apache.flink.runtime.instance; package org.apache.flink.runtime.instance;
import org.apache.flink.util.AbstractID; import org.apache.flink.util.AbstractID;
import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.api.common.JobID;
import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
......
...@@ -22,6 +22,7 @@ import akka.actor.ActorRef; ...@@ -22,6 +22,7 @@ import akka.actor.ActorRef;
import akka.dispatch.OnFailure; import akka.dispatch.OnFailure;
import akka.pattern.Patterns; import akka.pattern.Patterns;
import akka.util.Timeout; import akka.util.Timeout;
import org.apache.flink.api.common.JobID;
import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.akka.AkkaUtils;
import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode; import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode;
...@@ -35,7 +36,6 @@ import org.apache.flink.runtime.io.network.partition.ResultPartition; ...@@ -35,7 +36,6 @@ import org.apache.flink.runtime.io.network.partition.ResultPartition;
import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; import org.apache.flink.runtime.io.network.partition.ResultPartitionManager;
import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate; import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
import org.apache.flink.runtime.jobgraph.JobID;
import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration;
import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.runtime.taskmanager.Task;
import org.apache.flink.runtime.taskmanager.TaskManager; import org.apache.flink.runtime.taskmanager.TaskManager;
......
...@@ -28,7 +28,7 @@ import org.apache.flink.runtime.io.network.buffer.BufferProvider; ...@@ -28,7 +28,7 @@ import org.apache.flink.runtime.io.network.buffer.BufferProvider;
import org.apache.flink.runtime.io.network.partition.consumer.LocalInputChannel; import org.apache.flink.runtime.io.network.partition.consumer.LocalInputChannel;
import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel; import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel;
import org.apache.flink.runtime.jobgraph.DistributionPattern; import org.apache.flink.runtime.jobgraph.DistributionPattern;
import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.api.common.JobID;
import org.apache.flink.runtime.taskmanager.TaskManager; import org.apache.flink.runtime.taskmanager.TaskManager;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
......
...@@ -18,7 +18,8 @@ ...@@ -18,7 +18,8 @@
package org.apache.flink.runtime.io.network.partition; package org.apache.flink.runtime.io.network.partition;
import org.apache.flink.runtime.jobgraph.JobID;
import org.apache.flink.api.common.JobID;
public interface ResultPartitionConsumableNotifier { public interface ResultPartitionConsumableNotifier {
......
...@@ -31,6 +31,7 @@ import java.util.Map; ...@@ -31,6 +31,7 @@ import java.util.Map;
import java.util.Set; import java.util.Set;
import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.InvalidProgramException;
import org.apache.flink.api.common.JobID;
import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.fs.FSDataInputStream; import org.apache.flink.core.fs.FSDataInputStream;
import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.FileSystem;
......
...@@ -24,7 +24,7 @@ import java.util.LinkedList; ...@@ -24,7 +24,7 @@ import java.util.LinkedList;
import java.util.Map; import java.util.Map;
import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.accumulators.Accumulator;
import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.api.common.JobID;
/** /**
* This class manages the accumulators for different jobs. Either the jobs are * This class manages the accumulators for different jobs. Either the jobs are
......
此差异已折叠。
此差异已折叠。
此差异已折叠。
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册