|
@@ -26,7 +26,6 @@ import java.util.Random;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.commons.logging.LogFactory;
|
|
-import org.apache.hadoop.conf.Configuration;
|
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.io.*;
|
|
import org.apache.hadoop.io.*;
|
|
@@ -39,7 +38,7 @@ class LocalJobRunner implements JobSubmissionProtocol {
|
|
|
|
|
|
private FileSystem fs;
|
|
private FileSystem fs;
|
|
private HashMap<String, Job> jobs = new HashMap<String, Job>();
|
|
private HashMap<String, Job> jobs = new HashMap<String, Job>();
|
|
- private Configuration conf;
|
|
|
|
|
|
+ private JobConf conf;
|
|
private int map_tasks = 0;
|
|
private int map_tasks = 0;
|
|
private int reduce_tasks = 0;
|
|
private int reduce_tasks = 0;
|
|
|
|
|
|
@@ -51,7 +50,7 @@ class LocalJobRunner implements JobSubmissionProtocol {
|
|
|
|
|
|
private class Job extends Thread
|
|
private class Job extends Thread
|
|
implements TaskUmbilicalProtocol {
|
|
implements TaskUmbilicalProtocol {
|
|
- private String file;
|
|
|
|
|
|
+ private Path file;
|
|
private String id;
|
|
private String id;
|
|
private JobConf job;
|
|
private JobConf job;
|
|
private Random random = new Random();
|
|
private Random random = new Random();
|
|
@@ -74,18 +73,18 @@ class LocalJobRunner implements JobSubmissionProtocol {
|
|
return TaskUmbilicalProtocol.versionID;
|
|
return TaskUmbilicalProtocol.versionID;
|
|
}
|
|
}
|
|
|
|
|
|
- public Job(String file, Configuration conf) throws IOException {
|
|
|
|
- this.file = file;
|
|
|
|
- this.id = "job_" + newId();
|
|
|
|
|
|
+ public Job(String jobid, JobConf conf) throws IOException {
|
|
|
|
+ this.file = new Path(conf.getSystemDir(), jobid + "/job.xml");
|
|
|
|
+ this.id = jobid;
|
|
this.mapoutputFile = new MapOutputFile();
|
|
this.mapoutputFile = new MapOutputFile();
|
|
this.mapoutputFile.setConf(conf);
|
|
this.mapoutputFile.setConf(conf);
|
|
|
|
|
|
this.localFile = new JobConf(conf).getLocalPath("localRunner/"+id+".xml");
|
|
this.localFile = new JobConf(conf).getLocalPath("localRunner/"+id+".xml");
|
|
this.localFs = FileSystem.getLocal(conf);
|
|
this.localFs = FileSystem.getLocal(conf);
|
|
|
|
|
|
- fs.copyToLocalFile(new Path(file), localFile);
|
|
|
|
|
|
+ fs.copyToLocalFile(file, localFile);
|
|
this.job = new JobConf(localFile);
|
|
this.job = new JobConf(localFile);
|
|
- profile = new JobProfile(job.getUser(), id, file,
|
|
|
|
|
|
+ profile = new JobProfile(job.getUser(), id, file.toString(),
|
|
"http://localhost:8080/", job.getJobName());
|
|
"http://localhost:8080/", job.getJobName());
|
|
status = new JobStatus(id, 0.0f, 0.0f, JobStatus.RUNNING);
|
|
status = new JobStatus(id, 0.0f, 0.0f, JobStatus.RUNNING);
|
|
|
|
|
|
@@ -119,7 +118,7 @@ class LocalJobRunner implements JobSubmissionProtocol {
|
|
splits[i].write(buffer);
|
|
splits[i].write(buffer);
|
|
BytesWritable split = new BytesWritable();
|
|
BytesWritable split = new BytesWritable();
|
|
split.set(buffer.getData(), 0, buffer.getLength());
|
|
split.set(buffer.getData(), 0, buffer.getLength());
|
|
- MapTask map = new MapTask(jobId, file, "tip_m_" + mapId,
|
|
|
|
|
|
+ MapTask map = new MapTask(jobId, file.toString(), "tip_m_" + mapId,
|
|
mapId, i,
|
|
mapId, i,
|
|
splits[i].getClass().getName(),
|
|
splits[i].getClass().getName(),
|
|
split);
|
|
split);
|
|
@@ -152,8 +151,9 @@ class LocalJobRunner implements JobSubmissionProtocol {
|
|
}
|
|
}
|
|
|
|
|
|
{
|
|
{
|
|
- ReduceTask reduce = new ReduceTask(jobId, file, "tip_r_0001",
|
|
|
|
- reduceId, 0, mapIds.size());
|
|
|
|
|
|
+ ReduceTask reduce = new ReduceTask(jobId, file.toString(),
|
|
|
|
+ "tip_r_0001",
|
|
|
|
+ reduceId, 0, mapIds.size());
|
|
JobConf localConf = new JobConf(job);
|
|
JobConf localConf = new JobConf(job);
|
|
reduce.localizeConfiguration(localConf);
|
|
reduce.localizeConfiguration(localConf);
|
|
reduce.setConf(localConf);
|
|
reduce.setConf(localConf);
|
|
@@ -187,7 +187,7 @@ class LocalJobRunner implements JobSubmissionProtocol {
|
|
|
|
|
|
} finally {
|
|
} finally {
|
|
try {
|
|
try {
|
|
- fs.delete(new Path(file).getParent()); // delete submit dir
|
|
|
|
|
|
+ fs.delete(file.getParent()); // delete submit dir
|
|
localFs.delete(localFile); // delete local copy
|
|
localFs.delete(localFile); // delete local copy
|
|
} catch (IOException e) {
|
|
} catch (IOException e) {
|
|
LOG.warn("Error cleaning up "+id+": "+e);
|
|
LOG.warn("Error cleaning up "+id+": "+e);
|
|
@@ -258,7 +258,7 @@ class LocalJobRunner implements JobSubmissionProtocol {
|
|
|
|
|
|
}
|
|
}
|
|
|
|
|
|
- public LocalJobRunner(Configuration conf) throws IOException {
|
|
|
|
|
|
+ public LocalJobRunner(JobConf conf) throws IOException {
|
|
this.fs = FileSystem.get(conf);
|
|
this.fs = FileSystem.get(conf);
|
|
this.conf = conf;
|
|
this.conf = conf;
|
|
myMetrics = new JobTrackerMetrics(new JobConf(conf));
|
|
myMetrics = new JobTrackerMetrics(new JobConf(conf));
|
|
@@ -266,8 +266,13 @@ class LocalJobRunner implements JobSubmissionProtocol {
|
|
|
|
|
|
// JobSubmissionProtocol methods
|
|
// JobSubmissionProtocol methods
|
|
|
|
|
|
- public JobStatus submitJob(String jobFile) throws IOException {
|
|
|
|
- return new Job(jobFile, this.conf).status;
|
|
|
|
|
|
+ private int jobid = 0;
|
|
|
|
+ public String getNewJobId() {
|
|
|
|
+ return "job_local_" + Integer.toString(++jobid);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public JobStatus submitJob(String jobid) throws IOException {
|
|
|
|
+ return new Job(jobid, this.conf).status;
|
|
}
|
|
}
|
|
|
|
|
|
public void killJob(String id) {
|
|
public void killJob(String id) {
|