You are viewing a plain text version of this content. The canonical link for it is here.
Posted to mapreduce-commits@hadoop.apache.org by to...@apache.org on 2011/10/25 00:32:48 UTC
svn commit: r1188423 - in
/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project: CHANGES.txt
hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java
Author: todd
Date: Mon Oct 24 22:32:47 2011
New Revision: 1188423
URL: http://svn.apache.org/viewvc?rev=1188423&view=rev
Log:
MAPREDUCE-3252. Fix map tasks to not rewrite data an extra time when map output fits in spill buffer. Contributed by Todd Lipcon.
Modified:
hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/CHANGES.txt
hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java
Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/CHANGES.txt?rev=1188423&r1=1188422&r2=1188423&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/CHANGES.txt (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/CHANGES.txt Mon Oct 24 22:32:47 2011
@@ -1691,6 +1691,9 @@ Release 0.23.0 - Unreleased
MAPREDUCE-3028. Added job-end notification support. (Ravi Prakash via
acmurthy)
+ MAPREDUCE-3252. Fix map tasks to not rewrite data an extra time when
+ map output fits in spill buffer. (todd)
+
Release 0.22.0 - Unreleased
INCOMPATIBLE CHANGES
Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java?rev=1188423&r1=1188422&r2=1188423&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java Mon Oct 24 22:32:47 2011
@@ -21,6 +21,7 @@ package org.apache.hadoop.mapred;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.DataOutputStream;
+import java.io.File;
import java.io.IOException;
import java.io.OutputStream;
import java.nio.ByteBuffer;
@@ -36,8 +37,10 @@ import org.apache.hadoop.fs.FSDataInputS
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileSystem.Statistics;
+import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RawLocalFileSystem;
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.RawComparator;
import org.apache.hadoop.io.SequenceFile;
@@ -1727,10 +1730,10 @@ class MapTask extends Task {
finalOutFileSize += rfs.getFileStatus(filename[i]).getLen();
}
if (numSpills == 1) { //the spill is the final output
- rfs.rename(filename[0],
+ sameVolRename(filename[0],
mapOutputFile.getOutputFileForWriteInVolume(filename[0]));
if (indexCacheList.size() == 0) {
- rfs.rename(mapOutputFile.getSpillIndexFile(0),
+ sameVolRename(mapOutputFile.getSpillIndexFile(0),
mapOutputFile.getOutputIndexFileForWriteInVolume(filename[0]));
} else {
indexCacheList.get(0).writeToFile(
@@ -1847,7 +1850,29 @@ class MapTask extends Task {
}
}
}
-
+
+ /**
+ * Rename srcPath to dstPath on the same volume. This is the same
+ * as RawLocalFileSystem's rename method, except that it will not
+ * fall back to a copy, and it will create the target directory
+ * if it doesn't exist.
+ */
+ private void sameVolRename(Path srcPath,
+ Path dstPath) throws IOException {
+ RawLocalFileSystem rfs = (RawLocalFileSystem)this.rfs;
+ File src = rfs.pathToFile(srcPath);
+ File dst = rfs.pathToFile(dstPath);
+ if (!dst.getParentFile().exists()) {
+ if (!dst.getParentFile().mkdirs()) {
+ throw new IOException("Unable to rename " + src + " to "
+ + dst + ": couldn't create parent directory");
+ }
+ }
+
+ if (!src.renameTo(dst)) {
+ throw new IOException("Unable to rename " + src + " to " + dst);
+ }
+ }
} // MapOutputBuffer
/**