http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java ---------------------------------------------------------------------- diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java index 1a45db3..5a7129f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -18,312 +18,48 @@ package org.apache.hadoop.fs.s3native; -import java.io.BufferedOutputStream; -import java.io.EOFException; -import java.io.File; -import java.io.FileNotFoundException; -import java.io.FileOutputStream; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; import java.net.URI; -import java.security.DigestOutputStream; -import java.security.MessageDigest; -import java.security.NoSuchAlgorithmException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.TreeSet; -import java.util.concurrent.TimeUnit; -import com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.BufferedFSInputStream; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FSExceptionMessages; -import org.apache.hadoop.fs.FSInputStream; -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.io.IOUtils; -import org.apache.hadoop.io.retry.RetryPolicies; -import org.apache.hadoop.io.retry.RetryPolicy; -import org.apache.hadoop.io.retry.RetryProxy; import org.apache.hadoop.util.Progressable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import static org.apache.hadoop.fs.s3native.S3NativeFileSystemConfigKeys.S3_NATIVE_BUFFER_DIR_DEFAULT; -import static org.apache.hadoop.fs.s3native.S3NativeFileSystemConfigKeys.S3_NATIVE_BUFFER_DIR_KEY; -import static org.apache.hadoop.fs.s3native.S3NativeFileSystemConfigKeys.S3_NATIVE_MAX_RETRIES_DEFAUL; -import static org.apache.hadoop.fs.s3native.S3NativeFileSystemConfigKeys.S3_NATIVE_MAX_RETRIES_KEY; -import static org.apache.hadoop.fs.s3native.S3NativeFileSystemConfigKeys.S3_NATIVE_SLEEP_TIME_DEFAULT; -import static org.apache.hadoop.fs.s3native.S3NativeFileSystemConfigKeys.S3_NATIVE_SLEEP_TIME_KEY; -import static org.apache.hadoop.fs.s3native.S3NativeFileSystemConfigKeys.addDeprecatedConfigKeys; /** - * A {@link FileSystem} for reading and writing files stored on - * <a href="http://aws.amazon.com/s3">Amazon S3</a>. - * This implementation stores files on S3 in their native form so they can be - * read by other S3 tools. - * <p> - * A note about directories. S3 of course has no "native" support for them. - * The idiom we choose then is: for any directory created by this class, - * we use an empty object "#{dirpath}_$folder$" as a marker. - * Further, to interoperate with other S3 tools, we also accept the following: - * <ul> - * <li>an object "#{dirpath}/' denoting a directory marker</li> - * <li> - * if there exists any objects with the prefix "#{dirpath}/", then the - * directory is said to exist - * </li> - * <li> - * if both a file with the name of a directory and a marker for that - * directory exists, then the *file masks the directory*, and the directory - * is never returned. - * </li> - * </ul> + * This is a stub filesystem purely present to fail meaningfully when + * someone who explicitly declares + * {@code fs.s3n.impl=org.apache.hadoop.fs.s3native.NativeS3FileSystem} + * and then tries to create a filesystem off an s3n:// URL. + * + * The {@link #initialize(URI, Configuration)} method will throw + * an IOException informing the user of their need to migrate. + * @deprecated Replaced by the S3A client. */ @InterfaceAudience.Public @InterfaceStability.Stable -public class NativeS3FileSystem extends FileSystem { +public final class NativeS3FileSystem extends FileSystem { public static final Logger LOG = LoggerFactory.getLogger(NativeS3FileSystem.class); - - private static final String FOLDER_SUFFIX = "_$folder$"; - static final String PATH_DELIMITER = Path.SEPARATOR; - private static final int S3_MAX_LISTING_LENGTH = 1000; - - static { - // Add the deprecated config keys - addDeprecatedConfigKeys(); - } - - static class NativeS3FsInputStream extends FSInputStream { - - private NativeFileSystemStore store; - private Statistics statistics; - private InputStream in; - private final String key; - private long pos = 0; - - public NativeS3FsInputStream(NativeFileSystemStore store, Statistics statistics, InputStream in, String key) { - Preconditions.checkNotNull(in, "Null input stream"); - this.store = store; - this.statistics = statistics; - this.in = in; - this.key = key; - } - - @Override - public synchronized int read() throws IOException { - int result; - try { - result = in.read(); - } catch (IOException e) { - LOG.info("Received IOException while reading '{}', attempting to reopen", - key); - LOG.debug("{}", e, e); - try { - reopen(pos); - result = in.read(); - } catch (EOFException eof) { - LOG.debug("EOF on input stream read: {}", eof, eof); - result = -1; - } - } - if (result != -1) { - pos++; - } - if (statistics != null && result != -1) { - statistics.incrementBytesRead(1); - } - return result; - } - @Override - public synchronized int read(byte[] b, int off, int len) - throws IOException { - if (in == null) { - throw new EOFException("Cannot read closed stream"); - } - int result = -1; - try { - result = in.read(b, off, len); - } catch (EOFException eof) { - throw eof; - } catch (IOException e) { - LOG.info( "Received IOException while reading '{}'," + - " attempting to reopen.", key); - reopen(pos); - result = in.read(b, off, len); - } - if (result > 0) { - pos += result; - } - if (statistics != null && result > 0) { - statistics.incrementBytesRead(result); - } - return result; - } - @Override - public synchronized void close() throws IOException { - closeInnerStream(); - } - - /** - * Close the inner stream if not null. Even if an exception - * is raised during the close, the field is set to null - */ - private void closeInnerStream() { - IOUtils.closeStream(in); - in = null; - } - - /** - * Reopen a new input stream with the specified position - * @param pos the position to reopen a new stream - * @throws IOException - */ - private synchronized void reopen(long pos) throws IOException { - LOG.debug("Reopening key '{}' for reading at position '{}", key, pos); - InputStream newStream = store.retrieve(key, pos); - updateInnerStream(newStream, pos); - } - - /** - * Update inner stream with a new stream and position - * @param newStream new stream -must not be null - * @param newpos new position - * @throws IOException IO exception on a failure to close the existing - * stream. - */ - private synchronized void updateInnerStream(InputStream newStream, long newpos) throws IOException { - Preconditions.checkNotNull(newStream, "Null newstream argument"); - closeInnerStream(); - in = newStream; - this.pos = newpos; - } - - @Override - public synchronized void seek(long newpos) throws IOException { - if (newpos < 0) { - throw new EOFException( - FSExceptionMessages.NEGATIVE_SEEK); - } - if (pos != newpos) { - // the seek is attempting to move the current position - reopen(newpos); - } - } - - @Override - public synchronized long getPos() throws IOException { - return pos; - } - @Override - public boolean seekToNewSource(long targetPos) throws IOException { - return false; - } - } - - private class NativeS3FsOutputStream extends OutputStream { - - private Configuration conf; - private String key; - private File backupFile; - private OutputStream backupStream; - private MessageDigest digest; - private boolean closed; - private LocalDirAllocator lDirAlloc; - - public NativeS3FsOutputStream(Configuration conf, - NativeFileSystemStore store, String key, Progressable progress, - int bufferSize) throws IOException { - this.conf = conf; - this.key = key; - this.backupFile = newBackupFile(); - LOG.info("OutputStream for key '" + key + "' writing to tempfile '" + this.backupFile + "'"); - try { - this.digest = MessageDigest.getInstance("MD5"); - this.backupStream = new BufferedOutputStream(new DigestOutputStream( - new FileOutputStream(backupFile), this.digest)); - } catch (NoSuchAlgorithmException e) { - LOG.warn("Cannot load MD5 digest algorithm," + - "skipping message integrity check.", e); - this.backupStream = new BufferedOutputStream( - new FileOutputStream(backupFile)); - } - } - - private File newBackupFile() throws IOException { - if (conf.get(S3_NATIVE_BUFFER_DIR_KEY, null) != null) { - lDirAlloc = new LocalDirAllocator(S3_NATIVE_BUFFER_DIR_KEY); - } else { - lDirAlloc = new LocalDirAllocator(S3_NATIVE_BUFFER_DIR_DEFAULT); - } - File result = lDirAlloc.createTmpFileForWrite("output-", LocalDirAllocator.SIZE_UNKNOWN, conf); - result.deleteOnExit(); - return result; - } - - @Override - public void flush() throws IOException { - backupStream.flush(); - } - - @Override - public synchronized void close() throws IOException { - if (closed) { - return; - } - - backupStream.close(); - LOG.info("OutputStream for key '{}' closed. Now beginning upload", key); - - try { - byte[] md5Hash = digest == null ? null : digest.digest(); - store.storeFile(key, backupFile, md5Hash); - } finally { - if (!backupFile.delete()) { - LOG.warn("Could not delete temporary s3n file: " + backupFile); - } - super.close(); - closed = true; - } - LOG.info("OutputStream for key '{}' upload complete", key); - } - - @Override - public void write(int b) throws IOException { - backupStream.write(b); - } + /** + * Message in thrown exceptions: {@value}. + */ + private static final String UNSUPPORTED = + "The s3n:// client to Amazon S3 is no longer available:" + + " please migrate to the s3a:// client"; - @Override - public void write(byte[] b, int off, int len) throws IOException { - backupStream.write(b, off, len); - } - } - - private URI uri; - private NativeFileSystemStore store; - private Path workingDir; - public NativeS3FileSystem() { - // set store in initialize() - } - - public NativeS3FileSystem(NativeFileSystemStore store) { - this.store = store; } /** @@ -336,504 +72,77 @@ public class NativeS3FileSystem extends FileSystem { return "s3n"; } - @Override - public void initialize(URI uri, Configuration conf) throws IOException { - super.initialize(uri, conf); - if (store == null) { - store = createDefaultStore(conf); - } - store.initialize(uri, conf); - setConf(conf); - this.uri = S3xLoginHelper.buildFSURI(uri); - this.workingDir = - new Path("/user", System.getProperty("user.name")).makeQualified(this.uri, this.getWorkingDirectory()); - } - - private static NativeFileSystemStore createDefaultStore(Configuration conf) { - NativeFileSystemStore store = new Jets3tNativeFileSystemStore(); - - RetryPolicy basePolicy = RetryPolicies.retryUpToMaximumCountWithFixedSleep( - conf.getInt(S3_NATIVE_MAX_RETRIES_KEY, S3_NATIVE_MAX_RETRIES_DEFAUL), - conf.getLong(S3_NATIVE_SLEEP_TIME_KEY, S3_NATIVE_SLEEP_TIME_DEFAULT), - TimeUnit.SECONDS); - Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap = - new HashMap<Class<? extends Exception>, RetryPolicy>(); - exceptionToPolicyMap.put(IOException.class, basePolicy); - exceptionToPolicyMap.put(S3Exception.class, basePolicy); - - RetryPolicy methodPolicy = RetryPolicies.retryByException( - RetryPolicies.TRY_ONCE_THEN_FAIL, exceptionToPolicyMap); - Map<String, RetryPolicy> methodNameToPolicyMap = - new HashMap<String, RetryPolicy>(); - methodNameToPolicyMap.put("storeFile", methodPolicy); - methodNameToPolicyMap.put("rename", methodPolicy); - - return (NativeFileSystemStore) - RetryProxy.create(NativeFileSystemStore.class, store, - methodNameToPolicyMap); - } - - private static String pathToKey(Path path) { - if (path.toUri().getScheme() != null && path.toUri().getPath().isEmpty()) { - // allow uris without trailing slash after bucket to refer to root, - // like s3n://mybucket - return ""; - } - if (!path.isAbsolute()) { - throw new IllegalArgumentException("Path must be absolute: " + path); - } - String ret = path.toUri().getPath().substring(1); // remove initial slash - if (ret.endsWith("/") && (ret.indexOf("/") != ret.length() - 1)) { - ret = ret.substring(0, ret.length() -1); - } - return ret; - } - - private static Path keyToPath(String key) { - return new Path("/" + key); - } - - private Path makeAbsolute(Path path) { - if (path.isAbsolute()) { - return path; - } - return new Path(workingDir, path); - } - /** - * Check that a Path belongs to this FileSystem. - * Unlike the superclass, this version does not look at authority, - * only hostnames. - * @param path to check - * @throws IllegalArgumentException if there is an FS mismatch + * Always fail to initialize. + * @throws IOException always. */ @Override - protected void checkPath(Path path) { - S3xLoginHelper.checkPath(getConf(), getUri(), path, getDefaultPort()); + public void initialize(URI uri, Configuration conf) throws IOException { + super.initialize(uri, conf); + throw new IOException(UNSUPPORTED); } @Override - protected URI canonicalizeUri(URI rawUri) { - return S3xLoginHelper.canonicalizeUri(rawUri, getDefaultPort()); + public FileStatus getFileStatus(Path f) throws IOException { + throw new UnsupportedOperationException(UNSUPPORTED); } - /** This optional operation is not yet supported. */ @Override - public FSDataOutputStream append(Path f, int bufferSize, - Progressable progress) throws IOException { - throw new UnsupportedOperationException("Append is not supported " - + "by NativeS3FileSystem"); + public URI getUri() { + throw new UnsupportedOperationException(UNSUPPORTED); } - - @Override - public FSDataOutputStream create(Path f, FsPermission permission, - boolean overwrite, int bufferSize, short replication, long blockSize, - Progressable progress) throws IOException { - if (exists(f) && !overwrite) { - throw new FileAlreadyExistsException("File already exists: " + f); - } - - if(LOG.isDebugEnabled()) { - LOG.debug("Creating new file '" + f + "' in S3"); - } - Path absolutePath = makeAbsolute(f); - String key = pathToKey(absolutePath); - return new FSDataOutputStream(new NativeS3FsOutputStream(getConf(), store, - key, progress, bufferSize), statistics); - } - @Override - public boolean delete(Path f, boolean recurse) throws IOException { - FileStatus status; - try { - status = getFileStatus(f); - } catch (FileNotFoundException e) { - if(LOG.isDebugEnabled()) { - LOG.debug("Delete called for '" + f + - "' but file does not exist, so returning false"); - } - return false; - } - Path absolutePath = makeAbsolute(f); - String key = pathToKey(absolutePath); - if (status.isDirectory()) { - if (!recurse && listStatus(f).length > 0) { - throw new IOException("Can not delete " + f + " as is a not empty directory and recurse option is false"); - } - - createParent(f); - - if(LOG.isDebugEnabled()) { - LOG.debug("Deleting directory '" + f + "'"); - } - String priorLastKey = null; - do { - PartialListing listing = store.list(key, S3_MAX_LISTING_LENGTH, priorLastKey, true); - for (FileMetadata file : listing.getFiles()) { - store.delete(file.getKey()); - } - priorLastKey = listing.getPriorLastKey(); - } while (priorLastKey != null); - - try { - store.delete(key + FOLDER_SUFFIX); - } catch (FileNotFoundException e) { - //this is fine, we don't require a marker - } - } else { - if(LOG.isDebugEnabled()) { - LOG.debug("Deleting file '" + f + "'"); - } - createParent(f); - store.delete(key); - } - return true; + public FSDataInputStream open(Path f, int bufferSize) throws IOException { + throw new UnsupportedOperationException(UNSUPPORTED); } @Override - public FileStatus getFileStatus(Path f) throws IOException { - Path absolutePath = makeAbsolute(f); - String key = pathToKey(absolutePath); - - if (key.length() == 0) { // root always exists - return newDirectory(absolutePath); - } - - if(LOG.isDebugEnabled()) { - LOG.debug("getFileStatus retrieving metadata for key '" + key + "'"); - } - FileMetadata meta = store.retrieveMetadata(key); - if (meta != null) { - if(LOG.isDebugEnabled()) { - LOG.debug("getFileStatus returning 'file' for key '" + key + "'"); - } - return newFile(meta, absolutePath); - } - if (store.retrieveMetadata(key + FOLDER_SUFFIX) != null) { - if(LOG.isDebugEnabled()) { - LOG.debug("getFileStatus returning 'directory' for key '" + key + - "' as '" + key + FOLDER_SUFFIX + "' exists"); - } - return newDirectory(absolutePath); - } - - if(LOG.isDebugEnabled()) { - LOG.debug("getFileStatus listing key '" + key + "'"); - } - PartialListing listing = store.list(key, 1); - if (listing.getFiles().length > 0 || - listing.getCommonPrefixes().length > 0) { - if(LOG.isDebugEnabled()) { - LOG.debug("getFileStatus returning 'directory' for key '" + key + - "' as it has contents"); - } - return newDirectory(absolutePath); - } - - if(LOG.isDebugEnabled()) { - LOG.debug("getFileStatus could not find key '" + key + "'"); - } - throw new FileNotFoundException("No such file or directory '" + absolutePath + "'"); + public FSDataOutputStream create(Path f, + FsPermission permission, + boolean overwrite, + int bufferSize, + short replication, + long blockSize, + Progressable progress) throws IOException { + throw new UnsupportedOperationException(UNSUPPORTED); } @Override - public URI getUri() { - return uri; + public FSDataOutputStream append(Path f, + int bufferSize, + Progressable progress) throws IOException { + throw new UnsupportedOperationException(UNSUPPORTED); } - /** - * <p> - * If <code>f</code> is a file, this method will make a single call to S3. - * If <code>f</code> is a directory, this method will make a maximum of - * (<i>n</i> / 1000) + 2 calls to S3, where <i>n</i> is the total number of - * files and directories contained directly in <code>f</code>. - * </p> - */ @Override - public FileStatus[] listStatus(Path f) throws IOException { - - Path absolutePath = makeAbsolute(f); - String key = pathToKey(absolutePath); - - if (key.length() > 0) { - FileMetadata meta = store.retrieveMetadata(key); - if (meta != null) { - return new FileStatus[] { newFile(meta, absolutePath) }; - } - } - - URI pathUri = absolutePath.toUri(); - Set<FileStatus> status = new TreeSet<FileStatus>(); - String priorLastKey = null; - do { - PartialListing listing = store.list(key, S3_MAX_LISTING_LENGTH, priorLastKey, false); - for (FileMetadata fileMetadata : listing.getFiles()) { - Path subpath = keyToPath(fileMetadata.getKey()); - String relativePath = pathUri.relativize(subpath.toUri()).getPath(); - - if (fileMetadata.getKey().equals(key + "/")) { - // this is just the directory we have been asked to list - } - else if (relativePath.endsWith(FOLDER_SUFFIX)) { - status.add(newDirectory(new Path( - absolutePath, - relativePath.substring(0, relativePath.indexOf(FOLDER_SUFFIX))))); - } - else { - status.add(newFile(fileMetadata, subpath)); - } - } - for (String commonPrefix : listing.getCommonPrefixes()) { - Path subpath = keyToPath(commonPrefix); - String relativePath = pathUri.relativize(subpath.toUri()).getPath(); - // sometimes the common prefix includes the base dir (HADOOP-13830). - // avoid that problem by detecting it and keeping it out - // of the list - if (!relativePath.isEmpty()) { - status.add(newDirectory(new Path(absolutePath, relativePath))); - } - } - priorLastKey = listing.getPriorLastKey(); - } while (priorLastKey != null); - - if (status.isEmpty() && - key.length() > 0 && - store.retrieveMetadata(key + FOLDER_SUFFIX) == null) { - throw new FileNotFoundException("File " + f + " does not exist."); - } - - return status.toArray(new FileStatus[status.size()]); - } - - private FileStatus newFile(FileMetadata meta, Path path) { - return new FileStatus(meta.getLength(), false, 1, getDefaultBlockSize(), - meta.getLastModified(), path.makeQualified(this.getUri(), this.getWorkingDirectory())); - } - - private FileStatus newDirectory(Path path) { - return new FileStatus(0, true, 1, 0, 0, path.makeQualified(this.getUri(), this.getWorkingDirectory())); + public boolean rename(Path src, Path dst) throws IOException { + throw new UnsupportedOperationException(UNSUPPORTED); } @Override - public boolean mkdirs(Path f, FsPermission permission) throws IOException { - Path absolutePath = makeAbsolute(f); - List<Path> paths = new ArrayList<Path>(); - do { - paths.add(0, absolutePath); - absolutePath = absolutePath.getParent(); - } while (absolutePath != null); - - boolean result = true; - for (Path path : paths) { - result &= mkdir(path); - } - return result; - } - - private boolean mkdir(Path f) throws IOException { - try { - FileStatus fileStatus = getFileStatus(f); - if (fileStatus.isFile()) { - throw new FileAlreadyExistsException(String.format( - "Can't make directory for path '%s' since it is a file.", f)); - - } - } catch (FileNotFoundException e) { - if(LOG.isDebugEnabled()) { - LOG.debug("Making dir '" + f + "' in S3"); - } - String key = pathToKey(f) + FOLDER_SUFFIX; - store.storeEmptyFile(key); - } - return true; + public boolean delete(Path f, boolean recursive) throws IOException { + throw new UnsupportedOperationException(UNSUPPORTED); } @Override - public FSDataInputStream open(Path f, int bufferSize) throws IOException { - FileStatus fs = getFileStatus(f); // will throw if the file doesn't exist - if (fs.isDirectory()) { - throw new FileNotFoundException("'" + f + "' is a directory"); - } - LOG.info("Opening '" + f + "' for reading"); - Path absolutePath = makeAbsolute(f); - String key = pathToKey(absolutePath); - return new FSDataInputStream(new BufferedFSInputStream( - new NativeS3FsInputStream(store, statistics, store.retrieve(key), key), bufferSize)); - } - - // rename() and delete() use this method to ensure that the parent directory - // of the source does not vanish. - private void createParent(Path path) throws IOException { - Path parent = path.getParent(); - if (parent != null) { - String key = pathToKey(makeAbsolute(parent)); - if (key.length() > 0) { - store.storeEmptyFile(key + FOLDER_SUFFIX); - } - } + public FileStatus[] listStatus(Path f) + throws IOException { + throw new UnsupportedOperationException(UNSUPPORTED); } - - - @Override - public boolean rename(Path src, Path dst) throws IOException { - - String srcKey = pathToKey(makeAbsolute(src)); - final String debugPreamble = "Renaming '" + src + "' to '" + dst + "' - "; - - if (srcKey.length() == 0) { - // Cannot rename root of file system - if (LOG.isDebugEnabled()) { - LOG.debug(debugPreamble + - "returning false as cannot rename the root of a filesystem"); - } - return false; - } - - //get status of source - boolean srcIsFile; - try { - srcIsFile = getFileStatus(src).isFile(); - } catch (FileNotFoundException e) { - //bail out fast if the source does not exist - if (LOG.isDebugEnabled()) { - LOG.debug(debugPreamble + "returning false as src does not exist"); - } - return false; - } - // Figure out the final destination - String dstKey = pathToKey(makeAbsolute(dst)); - - try { - boolean dstIsFile = getFileStatus(dst).isFile(); - if (dstIsFile) { - //destination is a file. - //you can't copy a file or a directory onto an existing file - //except for the special case of dest==src, which is a no-op - if(LOG.isDebugEnabled()) { - LOG.debug(debugPreamble + - "returning without rename as dst is an already existing file"); - } - //exit, returning true iff the rename is onto self - return srcKey.equals(dstKey); - } else { - //destination exists and is a directory - if(LOG.isDebugEnabled()) { - LOG.debug(debugPreamble + "using dst as output directory"); - } - //destination goes under the dst path, with the name of the - //source entry - dstKey = pathToKey(makeAbsolute(new Path(dst, src.getName()))); - } - } catch (FileNotFoundException e) { - //destination does not exist => the source file or directory - //is copied over with the name of the destination - if(LOG.isDebugEnabled()) { - LOG.debug(debugPreamble + "using dst as output destination"); - } - try { - if (getFileStatus(dst.getParent()).isFile()) { - if(LOG.isDebugEnabled()) { - LOG.debug(debugPreamble + - "returning false as dst parent exists and is a file"); - } - return false; - } - } catch (FileNotFoundException ex) { - if(LOG.isDebugEnabled()) { - LOG.debug(debugPreamble + - "returning false as dst parent does not exist"); - } - return false; - } - } - //rename to self behavior follows Posix rules and is different - //for directories and files -the return code is driven by src type - if (srcKey.equals(dstKey)) { - //fully resolved destination key matches source: fail - if (LOG.isDebugEnabled()) { - LOG.debug(debugPreamble + "renamingToSelf; returning true"); - } - return true; - } - if (srcIsFile) { - //source is a file; COPY then DELETE - if(LOG.isDebugEnabled()) { - LOG.debug(debugPreamble + - "src is file, so doing copy then delete in S3"); - } - store.copy(srcKey, dstKey); - store.delete(srcKey); - } else { - //src is a directory - if(LOG.isDebugEnabled()) { - LOG.debug(debugPreamble + "src is directory, so copying contents"); - } - //Verify dest is not a child of the parent - if (dstKey.startsWith(srcKey + "/")) { - if (LOG.isDebugEnabled()) { - LOG.debug( - debugPreamble + "cannot rename a directory to a subdirectory of self"); - } - return false; - } - //create the subdir under the destination - store.storeEmptyFile(dstKey + FOLDER_SUFFIX); - - List<String> keysToDelete = new ArrayList<String>(); - String priorLastKey = null; - do { - PartialListing listing = store.list(srcKey, S3_MAX_LISTING_LENGTH, priorLastKey, true); - for (FileMetadata file : listing.getFiles()) { - keysToDelete.add(file.getKey()); - store.copy(file.getKey(), dstKey + file.getKey().substring(srcKey.length())); - } - priorLastKey = listing.getPriorLastKey(); - } while (priorLastKey != null); - - if(LOG.isDebugEnabled()) { - LOG.debug(debugPreamble + - "all files in src copied, now removing src files"); - } - for (String key: keysToDelete) { - store.delete(key); - } - - try { - store.delete(srcKey + FOLDER_SUFFIX); - } catch (FileNotFoundException e) { - //this is fine, we don't require a marker - } - if(LOG.isDebugEnabled()) { - LOG.debug(debugPreamble + "done"); - } - } - - return true; - } - @Override - public long getDefaultBlockSize() { - return getConf().getLong("fs.s3n.block.size", 64 * 1024 * 1024); + public void setWorkingDirectory(Path new_dir) { + throw new UnsupportedOperationException(UNSUPPORTED); } - /** - * Set the working directory to the given directory. - */ - @Override - public void setWorkingDirectory(Path newDir) { - workingDir = newDir; - } - @Override public Path getWorkingDirectory() { - return workingDir; + throw new UnsupportedOperationException(UNSUPPORTED); } @Override - public String getCanonicalServiceName() { - // Does not support Token - return null; + public boolean mkdirs(Path f, FsPermission permission) throws IOException { + throw new UnsupportedOperationException(UNSUPPORTED); } }
http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/PartialListing.java ---------------------------------------------------------------------- diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/PartialListing.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/PartialListing.java deleted file mode 100644 index 8290092..0000000 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/PartialListing.java +++ /dev/null @@ -1,64 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.s3native; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; - -/** - * <p> - * Holds information on a directory listing for a - * {@link NativeFileSystemStore}. - * This includes the {@link FileMetadata files} and directories - * (their names) contained in a directory. - * </p> - * <p> - * This listing may be returned in chunks, so a <code>priorLastKey</code> - * is provided so that the next chunk may be requested. - * </p> - * @see NativeFileSystemStore#list(String, int, String) - */ -@InterfaceAudience.Private -@InterfaceStability.Unstable -class PartialListing { - - private final String priorLastKey; - private final FileMetadata[] files; - private final String[] commonPrefixes; - - public PartialListing(String priorLastKey, FileMetadata[] files, - String[] commonPrefixes) { - this.priorLastKey = priorLastKey; - this.files = files; - this.commonPrefixes = commonPrefixes; - } - - public FileMetadata[] getFiles() { - return files; - } - - public String[] getCommonPrefixes() { - return commonPrefixes; - } - - public String getPriorLastKey() { - return priorLastKey; - } - -} http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3Credentials.java ---------------------------------------------------------------------- diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3Credentials.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3Credentials.java deleted file mode 100644 index 713b149..0000000 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3Credentials.java +++ /dev/null @@ -1,100 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.s3native; - -import java.io.IOException; -import java.net.URI; - -import com.google.common.base.Preconditions; -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.conf.Configuration; - -import static org.apache.hadoop.fs.s3native.S3NativeFileSystemConfigKeys.S3_NATIVE_AWS_ACCESS_KEY_ID; -import static org.apache.hadoop.fs.s3native.S3NativeFileSystemConfigKeys.S3_NATIVE_AWS_SECRET_ACCESS_KEY; - -/** - * <p> - * Extracts AWS credentials from the filesystem URI or configuration. - * </p> - */ -@InterfaceAudience.Private -@InterfaceStability.Unstable -public class S3Credentials { - - private String accessKey; - private String secretAccessKey; - - /** - * @param uri bucket URI optionally containing username and password. - * @param conf configuration - * @throws IllegalArgumentException if credentials for S3 cannot be - * determined. - * @throws IOException if credential providers are misconfigured and we have - * to talk to them. - */ - public void initialize(URI uri, Configuration conf) throws IOException { - Preconditions.checkArgument(uri.getHost() != null, - "Invalid hostname in URI " + uri); - - String userInfo = uri.getUserInfo(); - if (userInfo != null) { - int index = userInfo.indexOf(':'); - if (index != -1) { - accessKey = userInfo.substring(0, index); - secretAccessKey = userInfo.substring(index + 1); - } else { - accessKey = userInfo; - } - } - - if (accessKey == null) { - accessKey = conf.getTrimmed(S3_NATIVE_AWS_ACCESS_KEY_ID); - } - if (secretAccessKey == null) { - final char[] pass = conf.getPassword(S3_NATIVE_AWS_SECRET_ACCESS_KEY); - if (pass != null) { - secretAccessKey = (new String(pass)).trim(); - } - } - - final String scheme = uri.getScheme(); - Preconditions.checkArgument(!(accessKey == null && secretAccessKey == null), - "AWS Access Key ID and Secret Access Key must be specified as the " + - "username or password (respectively) of a " + scheme + " URL, or " + - "by setting the " + S3_NATIVE_AWS_ACCESS_KEY_ID + " or " + - S3_NATIVE_AWS_SECRET_ACCESS_KEY + " properties (respectively)."); - Preconditions.checkArgument(accessKey != null, - "AWS Access Key ID must be specified as the username of a " + scheme + - " URL, or by setting the " + S3_NATIVE_AWS_ACCESS_KEY_ID + - " property."); - Preconditions.checkArgument(secretAccessKey != null, - "AWS Secret Access Key must be specified as the password of a " + scheme - + " URL, or by setting the " + S3_NATIVE_AWS_SECRET_ACCESS_KEY + - " property."); - } - - public String getAccessKey() { - return accessKey; - } - - public String getSecretAccessKey() { - return secretAccessKey; - } -} http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3Exception.java ---------------------------------------------------------------------- diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3Exception.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3Exception.java deleted file mode 100644 index 9258fd7..0000000 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3Exception.java +++ /dev/null @@ -1,39 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.s3native; - -import java.io.IOException; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; - -/** - * Thrown if there is a problem communicating with Amazon S3. - */ -@InterfaceAudience.Public -@InterfaceStability.Stable -public class S3Exception extends IOException { - - private static final long serialVersionUID = 1L; - - public S3Exception(Throwable t) { - super(t); - } - -} http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3NativeFileSystemConfigKeys.java ---------------------------------------------------------------------- diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3NativeFileSystemConfigKeys.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3NativeFileSystemConfigKeys.java deleted file mode 100644 index 7c8b345..0000000 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3NativeFileSystemConfigKeys.java +++ /dev/null @@ -1,66 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.s3native; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.conf.Configuration.DeprecationDelta; -import org.apache.hadoop.fs.CommonConfigurationKeys; - -/** - * This class contains constants for configuration keys used - * in the s3 file system. - * - */ -@InterfaceAudience.Private -@InterfaceStability.Unstable -public class S3NativeFileSystemConfigKeys extends CommonConfigurationKeys { - public static final String S3_NATIVE_BLOCK_SIZE_KEY = "s3native.blocksize"; - public static final long S3_NATIVE_BLOCK_SIZE_DEFAULT = 64*1024*1024; - public static final String S3_NATIVE_REPLICATION_KEY = "s3native.replication"; - public static final short S3_NATIVE_REPLICATION_DEFAULT = 1; - public static final String S3_NATIVE_STREAM_BUFFER_SIZE_KEY = - "s3native.stream-buffer-size"; - public static final int S3_NATIVE_STREAM_BUFFER_SIZE_DEFAULT = 4096; - public static final String S3_NATIVE_BYTES_PER_CHECKSUM_KEY = - "s3native.bytes-per-checksum"; - public static final int S3_NATIVE_BYTES_PER_CHECKSUM_DEFAULT = 512; - public static final String S3_NATIVE_CLIENT_WRITE_PACKET_SIZE_KEY = - "s3native.client-write-packet-size"; - public static final int S3_NATIVE_CLIENT_WRITE_PACKET_SIZE_DEFAULT = 64*1024; - static final String S3_NATIVE_BUFFER_DIR_KEY = "fs.s3n.buffer.dir"; - static final String S3_NATIVE_BUFFER_DIR_DEFAULT = "${hadoop.tmp.dir}/s3n"; - static final String S3_NATIVE_MAX_RETRIES_KEY = "fs.s3n.maxRetries"; - static final int S3_NATIVE_MAX_RETRIES_DEFAUL = 4; - static final String S3_NATIVE_SLEEP_TIME_KEY = "fs.s3n.sleepTimeSeconds"; - static final int S3_NATIVE_SLEEP_TIME_DEFAULT = 10; - static final String S3_NATIVE_AWS_ACCESS_KEY_ID = "fs.s3n.awsAccessKeyId"; - static final String S3_NATIVE_AWS_SECRET_ACCESS_KEY = - "fs.s3n.awsSecretAccessKey"; - - static void addDeprecatedConfigKeys() { - Configuration.addDeprecations(new DeprecationDelta[]{ - new DeprecationDelta("fs.s3.buffer.dir", S3_NATIVE_BUFFER_DIR_KEY), - new DeprecationDelta("fs.s3.maxRetries", S3_NATIVE_MAX_RETRIES_KEY), - new DeprecationDelta("fs.s3.sleepTimeSeconds", S3_NATIVE_SLEEP_TIME_KEY) - }); - } - -} http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/package.html ---------------------------------------------------------------------- diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/package.html b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/package.html index 4d3bde9..eb2c471 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/package.html +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/package.html @@ -23,8 +23,11 @@ A distributed implementation of {@link org.apache.hadoop.fs.FileSystem} for reading and writing files on <a href="http://aws.amazon.com/s3">Amazon S3</a>. -This implementation stores files on S3 in their native form for interoperability +This implementation stored files on S3 in their native form for interoperability with other S3 tools. + +It has been replaced by the S3A client. + </p> </body> http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/encryption.md ---------------------------------------------------------------------- diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/encryption.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/encryption.md new file mode 100644 index 0000000..719c5e5 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/encryption.md @@ -0,0 +1,427 @@ + +<!--- + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. See accompanying LICENSE file. +--> + +# Working with Encrypted S3 Data + +<!-- MACRO{toc|fromDepth=0|toDepth=2} --> + + +## <a name="introduction"></a> Introduction + +The S3A filesystem client supports Amazon S3's Server Side Encryption +for at-rest data encryption. +You should to read up on the [AWS documentation](https://docs.aws.amazon.com/AmazonS3/latest/dev/serv-side-encryption.html) +for S3 Server Side Encryption for up to date information on the encryption mechansims. + + + +When configuring an encryption method in the `core-site.xml`, this will apply cluster wide. +Any new file written will be encrypted with this encryption configuration. +When the S3A client reads a file, S3 will attempt to decrypt it using the mechanism +and keys with which the file was encrypted. + +* It is **NOT** advised to mix and match encryption types in a bucket +* It is much simpler and safer to encrypt with just one type and key per bucket. +* You can use AWS bucket policies to mandate encryption rules for a bucket. +* You can use S3A per-bucket configuration to ensure that S3A clients use encryption +policies consistent with the mandated rules. +* Changing the encryption options on the client does not change how existing +files were encrypted, except when the files are renamed. +* For all mechanisms other than SSE-C, clients do not need any configuration +options set in order to read encrypted data: it is all automatically handled +in S3 itself. + +## <a name="encryption_types"></a>How data is encrypted + +AWS S3 supports server-side encryption inside the storage system itself. +When an S3 client uploading data requests data to be encrypted, then an encryption key is used +to encrypt the data as it saved to S3. It remains encrypted on S3 until deleted: +clients cannot change the encryption attributes of an object once uploaded. + +The Amazon AWS SDK also offers client-side encryption, in which all the encoding +and decoding of data is performed on the client. This is *not* supported by +the S3A client. + +The server-side "SSE" encryption is performed with symmetric AES256 encryption; +S3 offers different mechanisms for actually defining the key to use. + + +There are thrre key management mechanisms, which in order of simplicity of use, +are: + +* SSE-S3: an AES256 key is generated in S3, and saved alongside the data. +* SSE-KMS: an AES256 key is generated in S3, and encrypted with a secret key provided +by Amazon's Key Management Service, a key referenced by name in the uploading client. +* SSE-C : the client specifies an actual base64 encoded AES-256 key to be used +to encrypt and decrypt the data. + + +## <a name="sse-s3"></a> SSE-S3 Amazon S3-Managed Encryption Keys + +In SSE-S3, all keys and secrets are managed inside S3. This is the simplest encryption mechanism. +There is no extra cost for storing data with this option. + + +### Enabling SSE-S3 + +To write S3-SSE encrypted files, the value of +`fs.s3a.server-side-encryption-algorithm` must be set to that of +the encryption mechanism used in `core-site`; currently only `AES256` is supported. + +```xml +<property> + <name>fs.s3a.server-side-encryption-algorithm</name> + <value>AES256</value> +</property> +``` + +Once set, all new data will be stored encrypted. There is no need to set this property when downloading data â the data will be automatically decrypted when read using +the Amazon S3-managed key. + +To learn more, refer to +[Protecting Data Using Server-Side Encryption with Amazon S3-Managed Encryption Keys (SSE-S3) in AWS documentation](http://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html). + + +### <a name="sse-kms"></a> SSE-KMS: Amazon S3-KMS Managed Encryption Keys + + +Amazon offers a pay-per-use key management service, [AWS KMS](https://aws.amazon.com/documentation/kms/). +This service can be used to encrypt data on S3 by defining "customer master keys", CMKs, +which can be centrally managed and assigned to specific roles and IAM accounts. + +The AWS KMS [can be used encrypt data on S3uploaded data](http://docs.aws.amazon.com/kms/latest/developerguide/services-s3.html). + +> The AWS KMS service is **not** related to the Key Management Service built into Hadoop (*Hadoop KMS*). The *Hadoop KMS* primarily focuses on + managing keys for *HDFS Transparent Encryption*. Similarly, HDFS encryption is unrelated to S3 data encryption. + +When uploading data encrypted with SSE-KMS, the sequence is as follows. + +1. The S3A client must declare a specific CMK in the property `fs.s3a.server-side-encryption.key`, or leave +it blank to use the default configured for that region. + +1. The S3A client uploads all the data as normal, now including encryption information. + +1. The S3 service encrypts the data with a symmetric key unique to the new object. + +1. The S3 service retrieves the chosen CMK key from the KMS service, and, if the user has +the right to use it, uses it to encrypt the object-specific key. + + +When downloading SSE-KMS encrypte data, the sequence is as follows + +1. The S3A client issues an HTTP GET request to read the data. +1. S3 sees that the data was encrypted with SSE-KMS, and looks up the specific key in the KMS service +1. If and only if the requesting user has been granted permission to use the CMS key does +the KMS service provide S3 with the key. +1. As a result, S3 will only decode the data if the user has been granted access to the key. + + +KMS keys can be managed by an organization's administrators in AWS, including +having access permissions assigned and removed from specific users, groups, and IAM roles. +Only those "principals" with granted rights to a key may access it, +hence only they may encrypt data with the key, *and decrypt data encrypted with it*. +This allows KMS to be used to provide a cryptographically secure access control mechanism for data stores on S3. + + +Each KMS server is region specific, and accordingly, so is each CMK configured. +A CMK defined in one region cannot be used with an S3 bucket in a different region. + + +Notes + +* Callers are charged for every use of a key, both for encrypting the data in uploads + and for decrypting it when reading it back. +* Random-access IO on files may result in multiple GET requests of an object during a read +sequence (especially for columnar data), so may require more than one key retrieval to process a single file, +* The KMS service is throttled: too many requests may cause requests to fail. +* As well as incurring charges, heavy I/O *may* reach IO limits for a customer. If those limits are reached, +they can be increased through the AWS console. + + +### Enabling SSE-KMS + +To enable SSE-KMS, the property `fs.s3a.server-side-encryption-algorithm` must be set to `SSE-KMS` in `core-site`: + +```xml +<property> + <name>fs.s3a.server-side-encryption-algorithm</name> + <value>SSE-KMS</value> +</property> +``` + +The ID of the specific key used to encrypt the data should also be set in the property `fs.s3a.server-side-encryption.key`: + +```xml +<property> + <name>fs.s3a.server-side-encryption.key</name> + <value>arn:aws:kms:us-west-2:360379543683:key/071a86ff-8881-4ba0-9230-95af6d01ca01</value> +</property> +``` + +Organizations may define a default key in the Amazon KMS; if a default key is set, +then it will be used whenever SSE-KMS encryption is chosen and the value of `fs.s3a.server-side-encryption.key` is empty. + +### the S3A `fs.s3a.encryption.key` key only affects created files + +With SSE-KMS, the S3A client option `fs.s3a.server-side-encryption.key` sets the +key to be used when new files are created. When reading files, this key, +and indeed the value of `fs.s3a.server-side-encryption-algorithme` is ignored: +S3 will attempt to retrieve the key and decrypt the file based on the create-time settings. + +This means that + +* There's no need to configure any client simply reading data. +* It is possible for a client to read data encrypted with one KMS key, and +write it with another. + + +## <a name="sse-c"></a> SSE-C: Server side encryption with a client-supplied key. + +In SSE-C, the client supplies the secret key needed to read and write data. +Every client trying to read or write data must be configured with the same +secret key. + + +SSE-C integration with Hadoop is still stabilizing; issues related to it are still surfacing. +It is already clear that SSE-C with a common key <b>must</b> be used exclusively within +a bucket if it is to be used at all. This is the only way to ensure that path and +directory listings do not fail with "Bad Request" errors. + +### Enabling SSE-C + +To use SSE-C, the configuration option `fs.s3a.server-side-encryption-algorithm` +must be set to `SSE-C`, and a base-64 encoding of the key placed in +`fs.s3a.server-side-encryption.key`. + +```xml +<property> + <name>fs.s3a.server-side-encryption-algorithm</name> + <value>SSE-C</value> +</property> + +<property> + <name>fs.s3a.server-side-encryption.key</name> + <value>SGVscCwgSSdtIHRyYXBwZWQgaW5zaWRlIGEgYmFzZS02NC1jb2RlYyE=</value> +</property> +``` + +All clients must share this same key. + +### The `fs.s3a.encryption.key` value is used to read and write data + +With SSE-C, the S3A client option `fs.s3a.server-side-encryption.key` sets the +key to be used for both reading *and* writing data. + +When reading any file written with SSE-C, the same key must be set +in the property `fs.s3a.server-side-encryption.key`. + +This is unlike SSE-S3 and SSE-KMS, where the information needed to +decode data is kept in AWS infrastructure. + + +### SSE-C Warning + +You need to fully understand how SSE-C works in the S3 +environment before using this encryption type. Please refer to the Server Side +Encryption documentation available from AWS. SSE-C is only recommended for +advanced users with advanced encryption use cases. Failure to properly manage +encryption keys can cause data loss. Currently, the AWS S3 API(and thus S3A) +only supports one encryption key and cannot support decrypting objects during +moves under a previous key to a new destination. It is **NOT** advised to use +multiple encryption keys in a bucket, and is recommended to use one key per +bucket and to not change this key. This is due to when a request is made to S3, +the actual encryption key must be provided to decrypt the object and access the +metadata. Since only one encryption key can be provided at a time, S3A will not +pass the correct encryption key to decrypt the data. + + +## <a name="best_practises"></a> Encryption best practises + + +### <a name="bucket_policy"></a> Mandate encryption through policies + +Because it is up to the clients to enable encryption on new objects, all clients +must be correctly configured in order to guarantee that data is encrypted. + + +To mandate that all data uploaded to a bucket is encrypted, +you can set a [bucket policy](https://aws.amazon.com/blogs/security/how-to-prevent-uploads-of-unencrypted-objects-to-amazon-s3/) +declaring that clients must provide encryption information with all data uploaded. + + +* Mandating an encryption mechanism on newly uploaded data does not encrypt existing data; existing data will retain whatever encryption (if any) applied at the time of creation* + +Here is a policy to mandate `SSE-S3/AES265` encryption on all data uploaded to a bucket. This covers uploads as well as the copy operations which take place when file/directory rename operations are mimicked. + + +```json +{ + "Version": "2012-10-17", + "Id": "EncryptionPolicy", + "Statement": [ + { + "Sid": "RequireEncryptionHeaderOnPut", + "Effect": "Deny", + "Principal": "*", + "Action": [ + "s3:PutObject" + ], + "Resource": "arn:aws:s3:::BUCKET/*", + "Condition": { + "Null": { + "s3:x-amz-server-side-encryption": true + } + } + }, + { + "Sid": "RequireAESEncryptionOnPut", + "Effect": "Deny", + "Principal": "*", + "Action": [ + "s3:PutObject" + ], + "Resource": "arn:aws:s3:::BUCKET/*", + "Condition": { + "StringNotEquals": { + "s3:x-amz-server-side-encryption": "AES256" + } + } + } + ] +} +``` + +To use SSE-KMS, a different restriction must be defined: + + +```json +{ + "Version": "2012-10-17", + "Id": "EncryptionPolicy", + "Statement": [ + { + "Sid": "RequireEncryptionHeaderOnPut", + "Effect": "Deny", + "Principal": "*", + "Action": [ + "s3:PutObject" + ], + "Resource": "arn:aws:s3:::BUCKET/*", + "Condition": { + "Null": { + "s3:x-amz-server-side-encryption": true + } + } + }, + { + "Sid": "RequireKMSEncryptionOnPut", + "Effect": "Deny", + "Principal": "*", + "Action": [ + "s3:PutObject" + ], + "Resource": "arn:aws:s3:::BUCKET/*", + "Condition": { + "StringNotEquals": { + "s3:x-amz-server-side-encryption": "SSE-KMS" + } + } + } + ] +} +``` + +To use one of these policies: + +1. Replace `BUCKET` with the specific name of the bucket being secured. +1. Locate the bucket in the AWS console [S3 section](https://console.aws.amazon.com/s3/home). +1. Select the "Permissions" tab. +1. Select the "Bucket Policy" tab in the permissions section. +1. Paste the edited policy into the form. +1. Save the policy. + +### <a name="per_bucket_config"></a> Use S3a per-bucket configuration to control encryption settings + +In an organisation which has embraced S3 encryption, different buckets inevitably have +different encryption policies, such as different keys for SSE-KMS encryption. +In particular, as different keys need to be named for different regions, unless +you rely on the administrator-managed "default" key for each S3 region, you +will need unique keys. + +S3A's per-bucket configuration enables this. + + +Here, for example, are settings for a bucket in London, `london-stats`: + + +```xml +<property> + <name>fs.s3a.bucket.london-stats.server-side-encryption-algorithm</name> + <value>AES256</value> +</property> +``` + +This requests SSE-S; if matched with a bucket policy then all data will +be encrypted as it is uploaded. + + +A different bucket can use a different policy +(here SSE-KMS) and, when necessary, declare a key. + +Here is an example bucket in S3 Ireland, which uses SSE-KMS and +a KMS key hosted in the AWS-KMS service in the same region. + + +```xml +<property> + <name>fs.s3a.bucket.ireland-dev.server-side-encryption-algorithm</name> + <value>SSE-KMS</value> +</property> + +<property> + <name>fs.s3a.bucket.ireland-dev.server-side-encryption.key</name> + <value>arn:aws:kms:eu-west-1:98067faff834c:key/071a86ff-8881-4ba0-9230-95af6d01ca01</value> +</property> + +``` + +Again the approprate bucket policy can be used to guarantee that all callers +will use SSE-KMS; they can even mandata the name of the key used to encrypt +the data, so guaranteeing that access to thee data can be read by everyone +granted access to that key, and nobody without access to it. + + +###<a name="changing-encryption"></a> Use rename() to encrypt files with new keys + +The encryption of an object is set when it is uploaded. If you want to encrypt +an unencrypted file, or change the SEE-KMS key of a file, the only way to do +so is by copying the object. + +How can you do that from Hadoop? With `rename()`. + +The S3A client mimics a real filesystem's' rename operation by copying all the +source files to the destination paths, then deleting the old ones. +If you do a rename() + +Note: this does not work for SSE-C, because you cannot set a different key +for reading as for writing, and you must supply that key for reading. There +you need to copy one bucket to a different bucket, one with a different key. +Use `distCp`for this, with per-bucket encryption policies. + + +## <a name="Troubleshooting"></a> Troubleshooting Encryption + +The [troubleshooting](./troubleshooting_s3a.html) document covers +stack traces which may surface when working with encrypted data. --------------------------------------------------------------------- To unsubscribe, e-mail: common-commits-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-commits-h...@hadoop.apache.org