Author: atm Date: Thu Jul 17 21:56:22 2014 New Revision: 1611489 URL: http://svn.apache.org/r1611489 Log: HADOOP-10610. Upgrade S3n s3.fs.buffer.dir to support multi directories. Contributed by Ted Malaska.
Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt?rev=1611489&r1=1611488&r2=1611489&view=diff ============================================================================== --- hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt (original) +++ hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt Thu Jul 17 21:56:22 2014 @@ -423,6 +423,9 @@ Release 2.6.0 - UNRELEASED HADOOP-10733. Fix potential null dereference in CredShell. (Ted Yu via omalley) + HADOOP-10610. Upgrade S3n s3.fs.buffer.dir to support multi directories. + (Ted Malaska via atm) + OPTIMIZATIONS BUG FIXES Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java?rev=1611489&r1=1611488&r2=1611489&view=diff ============================================================================== --- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java (original) +++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java Thu Jul 17 21:56:22 2014 @@ -50,6 +50,7 @@ import org.apache.hadoop.fs.FSInputStrea import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.s3.S3Exception; @@ -225,6 +226,7 @@ public class NativeS3FileSystem extends private OutputStream backupStream; private MessageDigest digest; private boolean closed; + private LocalDirAllocator lDirAlloc; public NativeS3FsOutputStream(Configuration conf, NativeFileSystemStore store, String key, Progressable progress, @@ -246,11 +248,10 @@ public class NativeS3FileSystem extends } private File newBackupFile() throws IOException { - File dir = new File(conf.get("fs.s3.buffer.dir")); - if (!dir.mkdirs() && !dir.exists()) { - throw new IOException("Cannot create S3 buffer directory: " + dir); + if (lDirAlloc == null) { + lDirAlloc = new LocalDirAllocator("fs.s3.buffer.dir"); } - File result = File.createTempFile("output-", ".tmp", dir); + File result = lDirAlloc.createTmpFileForWrite("output-", LocalDirAllocator.SIZE_UNKNOWN, conf); result.deleteOnExit(); return result; }