Author: cutting Date: Thu Apr 27 11:29:05 2006 New Revision: 397601 URL: http://svn.apache.org/viewcvs?rev=397601&view=rev Log: Permit configuration to specify higher replication for job submission files. Also reduce complaints when a file's replication is greater than the size of the cluster. Modified: lucene/hadoop/trunk/CHANGES.txt lucene/hadoop/trunk/conf/hadoop-default.xml lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobClient.java Modified: lucene/hadoop/trunk/CHANGES.txt URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/CHANGES.txt?rev=397601&r1=397600&r2=397601&view=diff ============================================================================== --- lucene/hadoop/trunk/CHANGES.txt (original) +++ lucene/hadoop/trunk/CHANGES.txt Thu Apr 27 11:29:05 2006 @@ -112,6 +112,10 @@ the replication count of individual files. Also fix a few replication-related bugs. (Konstantin Shvachko via cutting) +30. Permit specification of a higher replication levels for job + submission files (job.xml and job.jar). This helps with large + clusters, since these files are read by every node. + Release 0.1.1 - 2006-04-08 Modified: lucene/hadoop/trunk/conf/hadoop-default.xml URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/conf/hadoop-default.xml?rev=397601&r1=397600&r2=397601&view=diff ============================================================================== --- lucene/hadoop/trunk/conf/hadoop-default.xml (original) +++ lucene/hadoop/trunk/conf/hadoop-default.xml Thu Apr 27 11:29:05 2006 @@ -258,6 +258,14 @@ take priority over this setting. + + mapred.submit.replication + 10 + The replication level for submitted job files. This + should be around the square root of the number of nodes. + + + Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java?rev=397601&r1=397600&r2=397601&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java Thu Apr 27 11:29:05 2006 @@ -1392,6 +1392,13 @@ * @return array of DatanodeInfo instances uses as targets. */ DatanodeInfo[] chooseTargets(int desiredReplicates, TreeSet forbiddenNodes, UTF8 clientMachine) { + if (desiredReplicates > datanodeMap.size()) { + LOG.warning("Replication requested of "+desiredReplicates + +" is larger than cluster size ("+datanodeMap.size() + +"). Using cluster size."); + desiredReplicates = datanodeMap.size(); + } + TreeSet alreadyChosen = new TreeSet(); Vector targets = new Vector(); Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java?rev=397601&r1=397600&r2=397601&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java Thu Apr 27 11:29:05 2006 @@ -180,6 +180,17 @@ /** * Opens an FSDataOutputStream at the indicated Path. + * Files are overwritten by default. + */ + public FSDataOutputStream create(Path f, short replication) + throws IOException { + return create(f, true, + getConf().getInt("io.file.buffer.size", 4096), + replication); + } + + /** + * Opens an FSDataOutputStream at the indicated Path. * @param f the file name to open * @param overwrite if a file with this name already exists, then if true, * the file will be overwritten, and if false an error will be thrown. Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobClient.java URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobClient.java?rev=397601&r1=397600&r2=397601&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobClient.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobClient.java Thu Apr 27 11:29:05 2006 @@ -241,9 +241,12 @@ 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 @@ 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 {