|
@@ -241,9 +241,12 @@ public class JobClient implements MRConstants {
|
|
|
FileSystem localFs = FileSystem.getNamed("local", job);
|
|
|
FileSystem fs = getFs();
|
|
|
|
|
|
+ short replication = (short)job.getInt("mapred.submit.replication", 10);
|
|
|
+
|
|
|
if (originalJarPath != null) { // copy jar to JobTracker's fs
|
|
|
job.setJar(submitJarFile.toString());
|
|
|
fs.copyFromLocalFile(new Path(originalJarPath), submitJarFile);
|
|
|
+ fs.setReplication(submitJarFile, replication);
|
|
|
}
|
|
|
|
|
|
// Set the user's name and working directory
|
|
@@ -257,7 +260,7 @@ public class JobClient implements MRConstants {
|
|
|
job.getOutputFormat().checkOutputSpecs(fs, job);
|
|
|
|
|
|
// Write job file to JobTracker's fs
|
|
|
- FSDataOutputStream out = fs.create(submitJobFile);
|
|
|
+ FSDataOutputStream out = fs.create(submitJobFile, replication);
|
|
|
try {
|
|
|
job.write(out);
|
|
|
} finally {
|