From mapreduce-commits-return-2591-apmail-hadoop-mapreduce-commits-archive=hadoop.apache.org@hadoop.apache.org Mon Oct 24 22:33:26 2011 Return-Path: X-Original-To: apmail-hadoop-mapreduce-commits-archive@minotaur.apache.org Delivered-To: apmail-hadoop-mapreduce-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id E04F47676 for ; Mon, 24 Oct 2011 22:33:26 +0000 (UTC) Received: (qmail 93505 invoked by uid 500); 24 Oct 2011 22:33:26 -0000 Delivered-To: apmail-hadoop-mapreduce-commits-archive@hadoop.apache.org Received: (qmail 93480 invoked by uid 500); 24 Oct 2011 22:33:26 -0000 Mailing-List: contact mapreduce-commits-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: mapreduce-dev@hadoop.apache.org Delivered-To: mailing list mapreduce-commits@hadoop.apache.org Received: (qmail 93472 invoked by uid 99); 24 Oct 2011 22:33:26 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 24 Oct 2011 22:33:26 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 24 Oct 2011 22:33:23 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id B0B0523889E3; Mon, 24 Oct 2011 22:33:02 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1188424 - in /hadoop/common/trunk/hadoop-mapreduce-project: CHANGES.txt hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java Date: Mon, 24 Oct 2011 22:33:02 -0000 To: mapreduce-commits@hadoop.apache.org From: todd@apache.org X-Mailer: svnmailer-1.0.8-patched Message-Id: <20111024223302.B0B0523889E3@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: todd Date: Mon Oct 24 22:33:02 2011 New Revision: 1188424 URL: http://svn.apache.org/viewvc?rev=1188424&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/trunk/hadoop-mapreduce-project/CHANGES.txt hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java Modified: hadoop/common/trunk/hadoop-mapreduce-project/CHANGES.txt URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/CHANGES.txt?rev=1188424&r1=1188423&r2=1188424&view=diff ============================================================================== --- hadoop/common/trunk/hadoop-mapreduce-project/CHANGES.txt (original) +++ hadoop/common/trunk/hadoop-mapreduce-project/CHANGES.txt Mon Oct 24 22:33:02 2011 @@ -1750,6 +1750,9 @@ Release 0.23.0 - Unreleased MAPREDUCE-3249. Ensure shuffle-port is correctly used duringMR AM recovery. (vinodkv 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/trunk/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/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java?rev=1188424&r1=1188423&r2=1188424&view=diff ============================================================================== --- hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java (original) +++ hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java Mon Oct 24 22:33:02 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 /**