HADOOP-10610. Upgrade S3n s3.fs.buffer.dir to support multi directories. Contributed by Ted Malaska.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1611489 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Aaron Myers 2014-07-17 21:56:22 +00:00
parent ce1659f070
commit dae1b6cc2a
2 changed files with 8 additions and 4 deletions

View File

@ -423,6 +423,9 @@ Release 2.6.0 - UNRELEASED
HADOOP-10733. Fix potential null dereference in CredShell. (Ted Yu via HADOOP-10733. Fix potential null dereference in CredShell. (Ted Yu via
omalley) omalley)
HADOOP-10610. Upgrade S3n s3.fs.buffer.dir to support multi directories.
(Ted Malaska via atm)
OPTIMIZATIONS OPTIMIZATIONS
BUG FIXES BUG FIXES

View File

@ -50,6 +50,7 @@ import org.apache.hadoop.fs.FSInputStream;
import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocalDirAllocator;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.s3.S3Exception; import org.apache.hadoop.fs.s3.S3Exception;
@ -225,6 +226,7 @@ public class NativeS3FileSystem extends FileSystem {
private OutputStream backupStream; private OutputStream backupStream;
private MessageDigest digest; private MessageDigest digest;
private boolean closed; private boolean closed;
private LocalDirAllocator lDirAlloc;
public NativeS3FsOutputStream(Configuration conf, public NativeS3FsOutputStream(Configuration conf,
NativeFileSystemStore store, String key, Progressable progress, NativeFileSystemStore store, String key, Progressable progress,
@ -246,11 +248,10 @@ public class NativeS3FileSystem extends FileSystem {
} }
private File newBackupFile() throws IOException { private File newBackupFile() throws IOException {
File dir = new File(conf.get("fs.s3.buffer.dir")); if (lDirAlloc == null) {
if (!dir.mkdirs() && !dir.exists()) { lDirAlloc = new LocalDirAllocator("fs.s3.buffer.dir");
throw new IOException("Cannot create S3 buffer directory: " + dir);
} }
File result = File.createTempFile("output-", ".tmp", dir); File result = lDirAlloc.createTmpFileForWrite("output-", LocalDirAllocator.SIZE_UNKNOWN, conf);
result.deleteOnExit(); result.deleteOnExit();
return result; return result;
} }