HADOOP-14738 Remove S3N and obsolete bits of S3A; rework docs. Contributed by Steve Loughran.

This commit is contained in:
Aaron Fabbri 2017-09-14 09:58:17 -07:00
parent 7ee02d1065
commit 49467165a5
No known key found for this signature in database
GPG Key ID: B2EEFA9E78118A29
58 changed files with 1871 additions and 4741 deletions

View File

@ -354,29 +354,6 @@
<artifactId>kfs</artifactId>
<optional>true</optional>
</dependency>
<dependency>
<groupId>net.java.dev.jets3t</groupId>
<artifactId>jets3t</artifactId>
<optional>true</optional>
<exclusions>
<exclusion>
<groupId>commons-codec</groupId>
<artifactId>commons-codec</artifactId>
</exclusion>
<exclusion>
<groupId>commons-logging</groupId>
<artifactId>commons-logging</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.httpcomponents</groupId>
<artifactId>httpclient</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.httpcomponents</groupId>
<artifactId>httpcore</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>com.jcraft</groupId>
<artifactId>jsch</artifactId>

View File

@ -182,8 +182,6 @@ log4j.appender.DNMETRICSRFA.MaxFileSize=64MB
#log4j.logger.org.apache.hadoop.mapred.TaskTracker=DEBUG
#log4j.logger.org.apache.hadoop.hdfs.server.namenode.FSNamesystem.audit=DEBUG
# Jets3t library
log4j.logger.org.jets3t.service.impl.rest.httpclient.RestS3Service=ERROR
# AWS SDK & S3A FileSystem
log4j.logger.com.amazonaws=ERROR

View File

@ -871,30 +871,6 @@
<description>File space usage statistics refresh interval in msec.</description>
</property>
<property>
<name>fs.s3n.buffer.dir</name>
<value>${hadoop.tmp.dir}/s3n</value>
<description>Determines where on the local filesystem the s3n:// filesystem
should store files before sending them to S3
(or after retrieving them from S3).
</description>
</property>
<property>
<name>fs.s3n.maxRetries</name>
<value>4</value>
<description>The maximum number of retries for reading or writing files to S3,
before we signal failure to the application.
</description>
</property>
<property>
<name>fs.s3n.sleepTimeSeconds</name>
<value>10</value>
<description>The number of seconds to sleep between each S3 retry.
</description>
</property>
<property>
<name>fs.swift.impl</name>
<value>org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem</value>
@ -911,56 +887,6 @@
</description>
</property>
<property>
<name>fs.s3n.awsAccessKeyId</name>
<description>AWS access key ID used by S3 native file system.</description>
</property>
<property>
<name>fs.s3n.awsSecretAccessKey</name>
<description>AWS secret key used by S3 native file system.</description>
</property>
<property>
<name>fs.s3n.block.size</name>
<value>67108864</value>
<description>Block size to use when reading files using the native S3
filesystem (s3n: URIs).</description>
</property>
<property>
<name>fs.s3n.multipart.uploads.enabled</name>
<value>false</value>
<description>Setting this property to true enables multiple uploads to
native S3 filesystem. When uploading a file, it is split into blocks
if the size is larger than fs.s3n.multipart.uploads.block.size.
</description>
</property>
<property>
<name>fs.s3n.multipart.uploads.block.size</name>
<value>67108864</value>
<description>The block size for multipart uploads to native S3 filesystem.
Default size is 64MB.
</description>
</property>
<property>
<name>fs.s3n.multipart.copy.block.size</name>
<value>5368709120</value>
<description>The block size for multipart copy in native S3 filesystem.
Default size is 5GB.
</description>
</property>
<property>
<name>fs.s3n.server-side-encryption-algorithm</name>
<value></value>
<description>Specify a server-side encryption algorithm for S3.
Unset by default, and the only other currently allowable value is AES256.
</description>
</property>
<property>
<name>fs.s3a.access.key</name>
<description>AWS access key ID used by S3A file system. Omit for IAM role-based or provider-based authentication.</description>
@ -1234,22 +1160,12 @@
uploads to.</description>
</property>
<property>
<name>fs.s3a.fast.upload</name>
<value>false</value>
<description>
Use the incremental block-based fast upload mechanism with
the buffering mechanism set in fs.s3a.fast.upload.buffer.
</description>
</property>
<property>
<name>fs.s3a.fast.upload.buffer</name>
<value>disk</value>
<description>
The buffering mechanism to use when using S3A fast upload
(fs.s3a.fast.upload=true). Values: disk, array, bytebuffer.
This configuration option has no effect if fs.s3a.fast.upload is false.
The buffering mechanism to for data being written.
Values: disk, array, bytebuffer.
"disk" will use the directories listed in fs.s3a.buffer.dir as
the location(s) to save data prior to being uploaded.
@ -1803,42 +1719,6 @@
<description>Replication factor</description>
</property>
<!-- s3native File System -->
<property>
<name>s3native.stream-buffer-size</name>
<value>4096</value>
<description>The size of buffer to stream files.
The size of this buffer should probably be a multiple of hardware
page size (4096 on Intel x86), and it determines how much data is
buffered during read and write operations.</description>
</property>
<property>
<name>s3native.bytes-per-checksum</name>
<value>512</value>
<description>The number of bytes per checksum. Must not be larger than
s3native.stream-buffer-size</description>
</property>
<property>
<name>s3native.client-write-packet-size</name>
<value>65536</value>
<description>Packet size for clients to write</description>
</property>
<property>
<name>s3native.blocksize</name>
<value>67108864</value>
<description>Block size</description>
</property>
<property>
<name>s3native.replication</name>
<value>3</value>
<description>Replication factor</description>
</property>
<!-- FTP file system -->
<property>
<name>ftp.stream-buffer-size</name>

View File

@ -605,7 +605,7 @@ The result is `FSDataOutputStream`, which through its operations may generate ne
clients creating files with `overwrite==true` to fail if the file is created
by another client between the two tests.
* S3N, S3A, Swift and potentially other Object Stores do not currently change the FS state
* S3A, Swift and potentially other Object Stores do not currently change the FS state
until the output stream `close()` operation is completed.
This MAY be a bug, as it allows >1 client to create a file with `overwrite==false`,
and potentially confuse file/directory logic
@ -961,7 +961,7 @@ The outcome is no change to FileSystem state, with a return value of false.
FS' = FS; result = False
*Local Filesystem, S3N*
*Local Filesystem*
The outcome is as a normal rename, with the additional (implicit) feature
that the parent directories of the destination also exist.

View File

@ -29,11 +29,10 @@ return codes of Unix filesystem actions as a reference. Even so, there
are places where HDFS diverges from the expected behaviour of a POSIX
filesystem.
The behaviour of other Hadoop filesystems are not as rigorously tested.
The bundled S3N and S3A FileSystem clients make Amazon's S3 Object Store ("blobstore")
The bundled S3A FileSystem clients make Amazon's S3 Object Store ("blobstore")
accessible through the FileSystem API. The Swift FileSystem driver provides similar
functionality for the OpenStack Swift blobstore. The Azure object storage
FileSystem talks to Microsoft's Azure equivalent. All of these
functionality for the OpenStack Swift blobstore. The Azure WASB and ADL object
storage FileSystems talks to Microsoft's Azure storage. All of these
bind to object stores, which do have different behaviors, especially regarding
consistency guarantees, and atomicity of operations.

View File

@ -195,21 +195,21 @@ equivalent. Furthermore, the build MUST be configured to never bundle this file
In addition, `src/test/resources/auth-keys.xml` will need to be created. It can be a copy of `contract-test-options.xml`.
The `AbstractFSContract` class automatically loads this resource file if present; specific keys for specific test cases can be added.
As an example, here are what S3N test keys look like:
As an example, here are what S3A test keys look like:
<configuration>
<property>
<name>fs.contract.test.fs.s3n</name>
<value>s3n://tests3contract</value>
<name>fs.contract.test.fs.s3a</name>
<value>s3a://tests3contract</value>
</property>
<property>
<name>fs.s3n.awsAccessKeyId</name>
<name>fs.s3a.access.key</name>
<value>DONOTPCOMMITTHISKEYTOSCM</value>
</property>
<property>
<name>fs.s3n.awsSecretAccessKey</name>
<name>fs.s3a.secret.key</name>
<value>DONOTEVERSHARETHISSECRETKEY!</value>
</property>
</configuration>

View File

@ -94,14 +94,10 @@ public class TestCommonConfigurationFields extends TestConfigurationFieldsBase {
xmlPropsToSkipCompare.add("hadoop.tmp.dir");
xmlPropsToSkipCompare.add("nfs3.mountd.port");
xmlPropsToSkipCompare.add("nfs3.server.port");
xmlPropsToSkipCompare.add("test.fs.s3n.name");
xmlPropsToSkipCompare.add("fs.viewfs.rename.strategy");
// S3N/S3A properties are in a different subtree.
// - org.apache.hadoop.fs.s3native.S3NativeFileSystemConfigKeys
// S3A properties are in a different subtree.
xmlPrefixToSkipCompare.add("fs.s3a.");
xmlPrefixToSkipCompare.add("fs.s3n.");
xmlPrefixToSkipCompare.add("s3native.");
// WASB properties are in a different subtree.
// - org.apache.hadoop.fs.azure.NativeAzureFileSystem

View File

@ -54,7 +54,6 @@ public class TestConfigRedactor {
"fs.s3a.bucket.BUCKET.secret.key",
"fs.s3a.server-side-encryption.key",
"fs.s3a.bucket.engineering.server-side-encryption.key",
"fs.s3n.awsSecretKey",
"fs.azure.account.key.abcdefg.blob.core.windows.net",
"fs.adl.oauth2.refresh.token",
"fs.adl.oauth2.credential",

View File

@ -246,7 +246,6 @@ public abstract class FileSystemContractBaseTest {
@Test
public void testMkdirsWithUmask() throws Exception {
if (!isS3(fs)) {
Configuration conf = fs.getConf();
String oldUmask = conf.get(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY);
try {
@ -260,26 +259,6 @@ public abstract class FileSystemContractBaseTest {
conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, oldUmask);
}
}
}
/**
* Skip permission tests for S3FileSystem until HDFS-1333 is fixed.
* Classes that do not implement {@link FileSystem#getScheme()} method
* (e.g {@link RawLocalFileSystem}) will throw an
* {@link UnsupportedOperationException}.
* @param fileSystem FileSystem object to determine if it is S3 or not
* @return true if S3 false in any other case
*/
private boolean isS3(FileSystem fileSystem) {
try {
if (fileSystem.getScheme().equals("s3n")) {
return true;
}
} catch (UnsupportedOperationException e) {
LOG.warn("Unable to determine the schema of filesystem.");
}
return false;
}
@Test
public void testGetFileStatusThrowsExceptionForNonExistentFile()

View File

@ -45,12 +45,6 @@
This is required by FTPFileSystem</description>
</property>
<property>
<name>test.fs.s3n.name</name>
<value>s3n:///</value>
<description>The name of the s3n file system for testing.</description>
</property>
<!-- Turn security off for tests by default -->
<property>
<name>hadoop.security.authentication</name>

View File

@ -1,16 +0,0 @@
# 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.
# Speed up the s3native jets3t test
s3service.max-thread-count=10
threaded-service.max-thread-count=10

View File

@ -811,11 +811,6 @@
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>net.java.dev.jets3t</groupId>
<artifactId>jets3t</artifactId>
<version>0.9.0</version>
</dependency>
<dependency>
<groupId>com.amazonaws</groupId>
<artifactId>aws-java-sdk-bundle</artifactId>

View File

@ -204,6 +204,16 @@ in both the task configuration and as a Java option.
Existing configs that already specify both are not affected by this change.
See the full release notes of MAPREDUCE-5785 for more details.
S3Guard: Consistency and Metadata Caching for the S3A filesystem client
---------------------
[HADOOP-13345](https://issues.apache.org/jira/browse/HADOOP-13345) adds an
optional feature to the S3A client of Amazon S3 storage: the ability to use
a DynamoDB table as a fast and consistent store of file and directory
metadata.
See [S3Guard](./hadoop-aws/tools/hadoop-aws/s3guard.html) for more details.
Getting Started
===============

View File

@ -22,10 +22,6 @@
<Method name="checkPath" />
<Bug pattern="ES_COMPARING_STRINGS_WITH_EQ" />
</Match>
<!-- S3n warnings about malicious code aren't that relevant given its limited future. -->
<Match>
<Class name="org.apache.hadoop.fs.s3.INode" />
</Match>
<!-- Redundant null check makes code clearer, future-proof here. -->
<Match>
<Class name="org.apache.hadoop.fs.s3a.S3AFileSystem" />

View File

@ -193,7 +193,6 @@
<excludes>
<exclude>**/ITestJets3tNativeS3FileSystemContract.java</exclude>
<exclude>**/ITestS3AContractRootDir.java</exclude>
<exclude>**/ITestS3NContractRootDir.java</exclude>
<exclude>**/ITestS3AFileContextStatistics.java</exclude>
<exclude>**/ITestS3AEncryptionSSEC*.java</exclude>
<exclude>**/ITestS3AHuge*.java</exclude>
@ -226,7 +225,6 @@
<includes>
<include>**/ITestJets3tNativeS3FileSystemContract.java</include>
<include>**/ITestS3AContractRootDir.java</include>
<include>**/ITestS3NContractRootDir.java</include>
<include>**/ITestS3AFileContextStatistics.java</include>
<include>**/ITestS3AHuge*.java</include>
<include>**/ITestS3AEncryptionSSEC*.java</include>
@ -428,11 +426,6 @@
<scope>test</scope>
<type>test-jar</type>
</dependency>
<dependency>
<groupId>net.java.dev.jets3t</groupId>
<artifactId>jets3t</artifactId>
<scope>compile</scope>
</dependency>
<dependency>
<groupId>com.amazonaws</groupId>
<artifactId>aws-java-sdk-bundle</artifactId>

View File

@ -136,7 +136,10 @@ public final class Constants {
public static final String BUFFER_DIR = "fs.s3a.buffer.dir";
// switch to the fast block-by-block upload mechanism
// this is the only supported upload mechanism
@Deprecated
public static final String FAST_UPLOAD = "fs.s3a.fast.upload";
@Deprecated
public static final boolean DEFAULT_FAST_UPLOAD = false;
//initial size of memory buffer for a fast upload

View File

@ -163,7 +163,6 @@ public class S3AFileSystem extends FileSystem {
// The maximum number of entries that can be deleted in any call to s3
private static final int MAX_ENTRIES_TO_DELETE = 1000;
private boolean blockUploadEnabled;
private String blockOutputBuffer;
private S3ADataBlocks.BlockFactory blockFactory;
private int blockOutputActiveBlocks;
@ -281,9 +280,11 @@ public class S3AFileSystem extends FileSystem {
inputPolicy = S3AInputPolicy.getPolicy(
conf.getTrimmed(INPUT_FADVISE, INPUT_FADV_NORMAL));
blockUploadEnabled = conf.getBoolean(FAST_UPLOAD, DEFAULT_FAST_UPLOAD);
boolean blockUploadEnabled = conf.getBoolean(FAST_UPLOAD, true);
if (blockUploadEnabled) {
if (!blockUploadEnabled) {
LOG.warn("The \"slow\" output stream is no longer supported");
}
blockOutputBuffer = conf.getTrimmed(FAST_UPLOAD_BUFFER,
DEFAULT_FAST_UPLOAD_BUFFER);
partSize = ensureOutputParameterInRange(MULTIPART_SIZE, partSize);
@ -293,9 +294,6 @@ public class S3AFileSystem extends FileSystem {
LOG.debug("Using S3ABlockOutputStream with buffer = {}; block={};" +
" queue limit={}",
blockOutputBuffer, partSize, blockOutputActiveBlocks);
} else {
LOG.debug("Using S3AOutputStream");
}
metadataStore = S3Guard.getMetadataStore(this);
allowAuthoritative = conf.getBoolean(METADATASTORE_AUTHORITATIVE,
@ -644,9 +642,7 @@ public class S3AFileSystem extends FileSystem {
}
instrumentation.fileCreated();
FSDataOutputStream output;
if (blockUploadEnabled) {
output = new FSDataOutputStream(
return new FSDataOutputStream(
new S3ABlockOutputStream(this,
key,
new SemaphoredDelegatingExecutor(boundedThreadPool,
@ -658,19 +654,6 @@ public class S3AFileSystem extends FileSystem {
new WriteOperationHelper(key)
),
null);
} else {
// We pass null to FSDataOutputStream so it won't count writes that
// are being buffered to a file
output = new FSDataOutputStream(
new S3AOutputStream(getConf(),
this,
key,
progress
),
null);
}
return output;
}
/**
@ -2471,7 +2454,9 @@ public class S3AFileSystem extends FileSystem {
sb.append(", cannedACL=").append(cannedACL.toString());
}
sb.append(", readAhead=").append(readAhead);
if (getConf() != null) {
sb.append(", blockSize=").append(getDefaultBlockSize());
}
sb.append(", multiPartThreshold=").append(multiPartThreshold);
if (serverSideEncryptionAlgorithm != null) {
sb.append(", serverSideEncryptionAlgorithm='")

View File

@ -1,143 +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.s3a;
import com.amazonaws.AmazonClientException;
import com.amazonaws.services.s3.model.ObjectMetadata;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.LocalDirAllocator;
import org.apache.hadoop.util.Progressable;
import org.slf4j.Logger;
import java.io.BufferedOutputStream;
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.io.OutputStream;
import java.util.concurrent.atomic.AtomicBoolean;
import static org.apache.hadoop.fs.s3a.S3AUtils.*;
/**
* Output stream to save data to S3.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class S3AOutputStream extends OutputStream {
private final OutputStream backupStream;
private final File backupFile;
private final AtomicBoolean closed = new AtomicBoolean(false);
private final String key;
private final Progressable progress;
private final S3AFileSystem fs;
public static final Logger LOG = S3AFileSystem.LOG;
public S3AOutputStream(Configuration conf,
S3AFileSystem fs,
String key,
Progressable progress)
throws IOException {
this.key = key;
this.progress = progress;
this.fs = fs;
backupFile = fs.createTmpFileForWrite("output-",
LocalDirAllocator.SIZE_UNKNOWN, conf);
LOG.debug("OutputStream for key '{}' writing to tempfile: {}",
key, backupFile);
this.backupStream = new BufferedOutputStream(
new FileOutputStream(backupFile));
}
/**
* Check for the filesystem being open.
* @throws IOException if the filesystem is closed.
*/
void checkOpen() throws IOException {
if (closed.get()) {
throw new IOException("Output Stream closed");
}
}
@Override
public void flush() throws IOException {
checkOpen();
backupStream.flush();
}
@Override
public void close() throws IOException {
if (closed.getAndSet(true)) {
return;
}
backupStream.close();
LOG.debug("OutputStream for key '{}' closed. Now beginning upload", key);
try {
final ObjectMetadata om = fs.newObjectMetadata(backupFile.length());
UploadInfo info = fs.putObject(
fs.newPutObjectRequest(
key,
om,
backupFile));
ProgressableProgressListener listener =
new ProgressableProgressListener(fs, key, info.getUpload(), progress);
info.getUpload().addProgressListener(listener);
info.getUpload().waitForUploadResult();
listener.uploadCompleted();
// This will delete unnecessary fake parent directories, update any
// MetadataStore
fs.finishedWrite(key, info.getLength());
} catch (InterruptedException e) {
throw (InterruptedIOException) new InterruptedIOException(e.toString())
.initCause(e);
} catch (AmazonClientException e) {
throw translateException("saving output", key , e);
} finally {
if (!backupFile.delete()) {
LOG.warn("Could not delete temporary s3a file: {}", backupFile);
}
super.close();
}
LOG.debug("OutputStream for key '{}' upload complete", key);
}
@Override
public void write(int b) throws IOException {
checkOpen();
backupStream.write(b);
}
@Override
public void write(byte[] b, int off, int len) throws IOException {
checkOpen();
backupStream.write(b, off, len);
}
}

View File

@ -1,59 +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 basic metadata for a file stored in a {@link NativeFileSystemStore}.
* </p>
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
class FileMetadata {
private final String key;
private final long length;
private final long lastModified;
public FileMetadata(String key, long length, long lastModified) {
this.key = key;
this.length = length;
this.lastModified = lastModified;
}
public String getKey() {
return key;
}
public long getLength() {
return length;
}
public long getLastModified() {
return lastModified;
}
@Override
public String toString() {
return "FileMetadata[" + key + ", " + length + ", " + lastModified + "]";
}
}

View File

@ -1,481 +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 static org.apache.hadoop.fs.s3native.NativeS3FileSystem.PATH_DELIMITER;
import java.io.BufferedInputStream;
import java.io.ByteArrayInputStream;
import java.io.EOFException;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
import java.net.URI;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSExceptionMessages;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.security.AccessControlException;
import org.jets3t.service.S3Service;
import org.jets3t.service.S3ServiceException;
import org.jets3t.service.ServiceException;
import org.jets3t.service.StorageObjectsChunk;
import org.jets3t.service.impl.rest.HttpException;
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
import org.jets3t.service.model.MultipartPart;
import org.jets3t.service.model.MultipartUpload;
import org.jets3t.service.model.S3Bucket;
import org.jets3t.service.model.S3Object;
import org.jets3t.service.model.StorageObject;
import org.jets3t.service.security.AWSCredentials;
import org.jets3t.service.utils.MultipartUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@InterfaceAudience.Private
@InterfaceStability.Unstable
class Jets3tNativeFileSystemStore implements NativeFileSystemStore {
private S3Service s3Service;
private S3Bucket bucket;
private long multipartBlockSize;
private boolean multipartEnabled;
private long multipartCopyBlockSize;
static final long MAX_PART_SIZE = (long)5 * 1024 * 1024 * 1024;
private String serverSideEncryptionAlgorithm;
public static final Logger LOG =
LoggerFactory.getLogger(Jets3tNativeFileSystemStore.class);
@Override
public void initialize(URI uri, Configuration conf) throws IOException {
S3Credentials s3Credentials = new S3Credentials();
s3Credentials.initialize(uri, conf);
try {
AWSCredentials awsCredentials =
new AWSCredentials(s3Credentials.getAccessKey(),
s3Credentials.getSecretAccessKey());
this.s3Service = new RestS3Service(awsCredentials);
} catch (S3ServiceException e) {
handleException(e);
}
multipartEnabled =
conf.getBoolean("fs.s3n.multipart.uploads.enabled", false);
multipartBlockSize = Math.min(
conf.getLong("fs.s3n.multipart.uploads.block.size", 64 * 1024 * 1024),
MAX_PART_SIZE);
multipartCopyBlockSize = Math.min(
conf.getLong("fs.s3n.multipart.copy.block.size", MAX_PART_SIZE),
MAX_PART_SIZE);
serverSideEncryptionAlgorithm = conf.get("fs.s3n.server-side-encryption-algorithm");
bucket = new S3Bucket(uri.getHost());
}
@Override
public void storeFile(String key, File file, byte[] md5Hash)
throws IOException {
if (multipartEnabled && file.length() >= multipartBlockSize) {
storeLargeFile(key, file, md5Hash);
return;
}
BufferedInputStream in = null;
try {
in = new BufferedInputStream(new FileInputStream(file));
S3Object object = new S3Object(key);
object.setDataInputStream(in);
object.setContentType("binary/octet-stream");
object.setContentLength(file.length());
object.setServerSideEncryptionAlgorithm(serverSideEncryptionAlgorithm);
if (md5Hash != null) {
object.setMd5Hash(md5Hash);
}
s3Service.putObject(bucket, object);
} catch (ServiceException e) {
handleException(e, key);
} finally {
IOUtils.closeStream(in);
}
}
public void storeLargeFile(String key, File file, byte[] md5Hash)
throws IOException {
S3Object object = new S3Object(key);
object.setDataInputFile(file);
object.setContentType("binary/octet-stream");
object.setContentLength(file.length());
object.setServerSideEncryptionAlgorithm(serverSideEncryptionAlgorithm);
if (md5Hash != null) {
object.setMd5Hash(md5Hash);
}
List<StorageObject> objectsToUploadAsMultipart =
new ArrayList<StorageObject>();
objectsToUploadAsMultipart.add(object);
MultipartUtils mpUtils = new MultipartUtils(multipartBlockSize);
try {
mpUtils.uploadObjects(bucket.getName(), s3Service,
objectsToUploadAsMultipart, null);
} catch (Exception e) {
handleException(e, key);
}
}
@Override
public void storeEmptyFile(String key) throws IOException {
try {
S3Object object = new S3Object(key);
object.setDataInputStream(new ByteArrayInputStream(new byte[0]));
object.setContentType("binary/octet-stream");
object.setContentLength(0);
object.setServerSideEncryptionAlgorithm(serverSideEncryptionAlgorithm);
s3Service.putObject(bucket, object);
} catch (ServiceException e) {
handleException(e, key);
}
}
@Override
public FileMetadata retrieveMetadata(String key) throws IOException {
StorageObject object = null;
try {
LOG.debug("Getting metadata for key: {} from bucket: {}",
key, bucket.getName());
object = s3Service.getObjectDetails(bucket.getName(), key);
return new FileMetadata(key, object.getContentLength(),
object.getLastModifiedDate().getTime());
} catch (ServiceException e) {
try {
// process
handleException(e, key);
return null;
} catch (FileNotFoundException fnfe) {
// and downgrade missing files
return null;
}
} finally {
if (object != null) {
object.closeDataInputStream();
}
}
}
/**
* @param key
* The key is the object name that is being retrieved from the S3 bucket
* @return
* This method returns null if the key is not found
* @throws IOException
*/
@Override
public InputStream retrieve(String key) throws IOException {
try {
LOG.debug("Getting key: {} from bucket: {}",
key, bucket.getName());
S3Object object = s3Service.getObject(bucket.getName(), key);
return object.getDataInputStream();
} catch (ServiceException e) {
handleException(e, key);
return null; //return null if key not found
}
}
/**
*
* @param key
* The key is the object name that is being retrieved from the S3 bucket
* @return
* This method returns null if the key is not found
* @throws IOException
*/
@Override
public InputStream retrieve(String key, long byteRangeStart)
throws IOException {
try {
LOG.debug("Getting key: {} from bucket: {} with byteRangeStart: {}",
key, bucket.getName(), byteRangeStart);
S3Object object = s3Service.getObject(bucket, key, null, null, null,
null, byteRangeStart, null);
return object.getDataInputStream();
} catch (ServiceException e) {
handleException(e, key);
return null;
}
}
@Override
public PartialListing list(String prefix, int maxListingLength)
throws IOException {
return list(prefix, maxListingLength, null, false);
}
@Override
public PartialListing list(String prefix, int maxListingLength, String priorLastKey,
boolean recurse) throws IOException {
return list(prefix, recurse ? null : PATH_DELIMITER, maxListingLength, priorLastKey);
}
/**
* list objects
* @param prefix prefix
* @param delimiter delimiter
* @param maxListingLength max no. of entries
* @param priorLastKey last key in any previous search
* @return a list of matches
* @throws IOException on any reported failure
*/
private PartialListing list(String prefix, String delimiter,
int maxListingLength, String priorLastKey) throws IOException {
try {
if (!prefix.isEmpty() && !prefix.endsWith(PATH_DELIMITER)) {
prefix += PATH_DELIMITER;
}
StorageObjectsChunk chunk = s3Service.listObjectsChunked(bucket.getName(),
prefix, delimiter, maxListingLength, priorLastKey);
FileMetadata[] fileMetadata =
new FileMetadata[chunk.getObjects().length];
for (int i = 0; i < fileMetadata.length; i++) {
StorageObject object = chunk.getObjects()[i];
fileMetadata[i] = new FileMetadata(object.getKey(),
object.getContentLength(), object.getLastModifiedDate().getTime());
}
return new PartialListing(chunk.getPriorLastKey(), fileMetadata,
chunk.getCommonPrefixes());
} catch (ServiceException e) {
handleException(e, prefix);
return null; // never returned - keep compiler happy
}
}
@Override
public void delete(String key) throws IOException {
try {
LOG.debug("Deleting key: {} from bucket: {}",
key, bucket.getName());
s3Service.deleteObject(bucket, key);
} catch (ServiceException e) {
handleException(e, key);
}
}
public void rename(String srcKey, String dstKey) throws IOException {
try {
s3Service.renameObject(bucket.getName(), srcKey, new S3Object(dstKey));
} catch (ServiceException e) {
handleException(e, srcKey);
}
}
@Override
public void copy(String srcKey, String dstKey) throws IOException {
try {
if(LOG.isDebugEnabled()) {
LOG.debug("Copying srcKey: " + srcKey + "to dstKey: " + dstKey + "in bucket: " + bucket.getName());
}
if (multipartEnabled) {
S3Object object = s3Service.getObjectDetails(bucket, srcKey, null,
null, null, null);
if (multipartCopyBlockSize > 0 &&
object.getContentLength() > multipartCopyBlockSize) {
copyLargeFile(object, dstKey);
return;
}
}
S3Object dstObject = new S3Object(dstKey);
dstObject.setServerSideEncryptionAlgorithm(serverSideEncryptionAlgorithm);
s3Service.copyObject(bucket.getName(), srcKey, bucket.getName(),
dstObject, false);
} catch (ServiceException e) {
handleException(e, srcKey);
}
}
public void copyLargeFile(S3Object srcObject, String dstKey) throws IOException {
try {
long partCount = srcObject.getContentLength() / multipartCopyBlockSize +
(srcObject.getContentLength() % multipartCopyBlockSize > 0 ? 1 : 0);
MultipartUpload multipartUpload = s3Service.multipartStartUpload
(bucket.getName(), dstKey, srcObject.getMetadataMap());
List<MultipartPart> listedParts = new ArrayList<MultipartPart>();
for (int i = 0; i < partCount; i++) {
long byteRangeStart = i * multipartCopyBlockSize;
long byteLength;
if (i < partCount - 1) {
byteLength = multipartCopyBlockSize;
} else {
byteLength = srcObject.getContentLength() % multipartCopyBlockSize;
if (byteLength == 0) {
byteLength = multipartCopyBlockSize;
}
}
MultipartPart copiedPart = s3Service.multipartUploadPartCopy
(multipartUpload, i + 1, bucket.getName(), srcObject.getKey(),
null, null, null, null, byteRangeStart,
byteRangeStart + byteLength - 1, null);
listedParts.add(copiedPart);
}
Collections.reverse(listedParts);
s3Service.multipartCompleteUpload(multipartUpload, listedParts);
} catch (ServiceException e) {
handleException(e, srcObject.getKey());
}
}
@Override
public void purge(String prefix) throws IOException {
String key = "";
try {
S3Object[] objects =
s3Service.listObjects(bucket.getName(), prefix, null);
for (S3Object object : objects) {
key = object.getKey();
s3Service.deleteObject(bucket, key);
}
} catch (S3ServiceException e) {
handleException(e, key);
}
}
@Override
public void dump() throws IOException {
StringBuilder sb = new StringBuilder("S3 Native Filesystem, ");
sb.append(bucket.getName()).append("\n");
try {
S3Object[] objects = s3Service.listObjects(bucket.getName());
for (S3Object object : objects) {
sb.append(object.getKey()).append("\n");
}
} catch (S3ServiceException e) {
handleException(e);
}
System.out.println(sb);
}
/**
* Handle any service exception by translating it into an IOException
* @param e exception
* @throws IOException exception -always
*/
private void handleException(Exception e) throws IOException {
throw processException(e, e, "");
}
/**
* Handle any service exception by translating it into an IOException
* @param e exception
* @param key key sought from object store
* @throws IOException exception -always
*/
private void handleException(Exception e, String key) throws IOException {
throw processException(e, e, key);
}
/**
* Handle any service exception by translating it into an IOException
* @param thrown exception
* @param original original exception -thrown if no other translation could
* be made
* @param key key sought from object store or "" for undefined
* @return an exception to throw. If isProcessingCause==true this may be null.
*/
private IOException processException(Throwable thrown, Throwable original,
String key) {
IOException result;
if (thrown.getCause() != null) {
// recurse down
result = processException(thrown.getCause(), original, key);
} else if (thrown instanceof HttpException) {
// nested HttpException - examine error code and react
HttpException httpException = (HttpException) thrown;
String responseMessage = httpException.getResponseMessage();
int responseCode = httpException.getResponseCode();
String bucketName = "s3n://" + bucket.getName();
String text = String.format("%s : %03d : %s",
bucketName,
responseCode,
responseMessage);
String filename = !key.isEmpty() ? (bucketName + "/" + key) : text;
IOException ioe;
switch (responseCode) {
case 404:
result = new FileNotFoundException(filename);
break;
case 416: // invalid range
result = new EOFException(FSExceptionMessages.CANNOT_SEEK_PAST_EOF
+": " + filename);
break;
case 403: //forbidden
result = new AccessControlException("Permission denied"
+": " + filename);
break;
default:
result = new IOException(text);
}
result.initCause(thrown);
} else if (thrown instanceof S3ServiceException) {
S3ServiceException se = (S3ServiceException) thrown;
LOG.debug(
"S3ServiceException: {}: {} : {}",
se.getS3ErrorCode(), se.getS3ErrorMessage(), se, se);
if ("InvalidRange".equals(se.getS3ErrorCode())) {
result = new EOFException(FSExceptionMessages.CANNOT_SEEK_PAST_EOF);
} else {
result = new S3Exception(se);
}
} else if (thrown instanceof ServiceException) {
ServiceException se = (ServiceException) thrown;
LOG.debug("S3ServiceException: {}: {} : {}",
se.getErrorCode(), se.toString(), se, se);
result = new S3Exception(se);
} else if (thrown instanceof IOException) {
result = (IOException) thrown;
} else {
// here there is no exception derived yet.
// this means no inner cause, and no translation made yet.
// convert the original to an IOException -rather than just the
// exception at the base of the tree
result = new S3Exception(original);
}
return result;
}
}

View File

@ -1,67 +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.File;
import java.io.IOException;
import java.io.InputStream;
import java.net.URI;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
/**
* <p>
* An abstraction for a key-based {@link File} store.
* </p>
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
interface NativeFileSystemStore {
void initialize(URI uri, Configuration conf) throws IOException;
void storeFile(String key, File file, byte[] md5Hash) throws IOException;
void storeEmptyFile(String key) throws IOException;
FileMetadata retrieveMetadata(String key) throws IOException;
InputStream retrieve(String key) throws IOException;
InputStream retrieve(String key, long byteRangeStart) throws IOException;
PartialListing list(String prefix, int maxListingLength) throws IOException;
PartialListing list(String prefix, int maxListingLength, String priorLastKey, boolean recursive)
throws IOException;
void delete(String key) throws IOException;
void copy(String srcKey, String dstKey) throws IOException;
/**
* Delete all keys with the given prefix. Used for testing.
* @throws IOException
*/
void purge(String prefix) throws IOException;
/**
* Diagnostic method to dump state to the console.
* @throws IOException
*/
void dump() throws IOException;
}

View File

@ -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.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;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.util.Progressable;
/**
* 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
* Message in thrown exceptions: {@value}.
*/
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);
}
@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;
private static final String UNSUPPORTED =
"The s3n:// client to Amazon S3 is no longer available:"
+ " please migrate to the s3a:// client";
public NativeS3FileSystem() {
// set store in initialize()
}
public NativeS3FileSystem(NativeFileSystemStore store) {
this.store = store;
}
/**
@ -336,504 +72,77 @@ public class NativeS3FileSystem extends FileSystem {
return "s3n";
}
/**
* Always fail to initialize.
* @throws IOException always.
*/
@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
*/
@Override
protected void checkPath(Path path) {
S3xLoginHelper.checkPath(getConf(), getUri(), path, getDefaultPort());
}
@Override
protected URI canonicalizeUri(URI rawUri) {
return S3xLoginHelper.canonicalizeUri(rawUri, getDefaultPort());
}
/** 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");
}
@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;
throw new IOException(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 + "'");
throw new UnsupportedOperationException(UNSUPPORTED);
}
@Override
public URI getUri() {
return uri;
}
/**
* <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()));
}
@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;
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));
throw new UnsupportedOperationException(UNSUPPORTED);
}
// 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);
}
}
@Override
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 FSDataOutputStream append(Path f,
int bufferSize,
Progressable progress) 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;
throw new UnsupportedOperationException(UNSUPPORTED);
}
@Override
public long getDefaultBlockSize() {
return getConf().getLong("fs.s3n.block.size", 64 * 1024 * 1024);
public boolean delete(Path f, boolean recursive) throws IOException {
throw new UnsupportedOperationException(UNSUPPORTED);
}
/**
* Set the working directory to the given directory.
*/
@Override
public void setWorkingDirectory(Path newDir) {
workingDir = newDir;
public FileStatus[] listStatus(Path f)
throws IOException {
throw new UnsupportedOperationException(UNSUPPORTED);
}
@Override
public void setWorkingDirectory(Path new_dir) {
throw new UnsupportedOperationException(UNSUPPORTED);
}
@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);
}
}

View File

@ -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;
}
}

View File

@ -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;
}
}

View File

@ -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);
}
}

View File

@ -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)
});
}
}

View File

@ -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>

View File

@ -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.

View File

@ -107,7 +107,6 @@ It is recommended that you leave the default setting here:
<name>fs.s3a.metadatastore.authoritative</name>
<value>false</value>
</property>
```
Setting this to `true` is currently an experimental feature. When true, the
@ -510,7 +509,6 @@ log4j.logger.com.amazonaws.services.dynamodbv2.AmazonDynamoDB
# Log all HTTP requests made; includes S3 interaction. This may
# include sensitive information such as account IDs in HTTP headers.
log4j.logger.com.amazonaws.request=DEBUG
```
If all else fails, S3Guard is designed to allow for easy recovery by deleting
@ -538,7 +536,7 @@ S3Guard tables are created with a version marker, an entry with the primary
key and child entry of `../VERSION`; the use of a relative path guarantees
that it will not be resolved.
#### Versioning policy.
*Versioning policy*
1. The version number of an S3Guard table will only be incremented when
an incompatible change is made to the table structure —that is, the structure
@ -557,7 +555,7 @@ in an incompatible manner. The version marker in tables exists to support
such an option if it ever becomes necessary, by ensuring that all S3Guard
client can recognise any version mismatch.
### Security
## Security
All users of the DynamoDB table must have write access to it. This
effectively means they must have write access to the entire object store.
@ -569,9 +567,9 @@ are only made after successful file creation, deletion and rename, the
store is *unlikely* to get out of sync, it is still something which
merits more testing before it could be considered reliable.
### Troubleshooting
## Troubleshooting
#### Error: `S3Guard table lacks version marker.`
### Error: `S3Guard table lacks version marker.`
The table which was intended to be used as a S3guard metadata store
does not have any version marker indicating that it is a S3Guard table.
@ -581,7 +579,7 @@ It may be that this is not a S3Guard table.
* Make sure that this is the correct table name.
* Delete the table, so it can be rebuilt.
#### Error: `Database table is from an incompatible S3Guard version`
### Error: `Database table is from an incompatible S3Guard version`
This indicates that the version of S3Guard which created (or possibly updated)
the database table is from a different version that that expected by the S3A
@ -596,7 +594,7 @@ bucket. Upgrade the application/library.
If the expected version is higher than the actual version, then the table
itself will need upgrading.
#### Error `"DynamoDB table TABLE does not exist in region REGION; auto-creation is turned off"`
### Error `"DynamoDB table TABLE does not exist in region REGION; auto-creation is turned off"`
S3Guard could not find the DynamoDB table for the Metadata Store,
and it was not configured to create it. Either the table was missing,
@ -608,3 +606,8 @@ or the configuration is preventing S3Guard from finding the table.
1. If the region is not set, verify that the table exists in the same
region as the bucket being used.
1. Create the table if necessary.
## Other Topis
For details on how to test S3Guard, see [Testing S3Guard](./testing.html#s3guard)

View File

@ -0,0 +1,52 @@
<!---
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.
-->
# The S3N Client
<!-- MACRO{toc|fromDepth=0|toDepth=5} -->
S3N was a Hadoop filesystem client which can read or write data stored
in Amazon S3. It uses URLs with the schema `s3n://`.
- - -
**Hadoop's S3N client for Amazon S3 has been superceded by
the S3A connector**
**Please upgrade to S3A for a supported, higher-performance S3 Client**
- - -
## <a name="migrating"></a> How to migrate to to the S3A client
1. Keep the `hadoop-aws` JAR on your classpath.
1. Add the `aws-java-sdk-bundle.jar` JAR which Hadoop ships
with to your classpath.
1. Change the authentication keys
| old key | new key |
|---------|---------|
| `fs.s3n.awsAccessKeyId` | `fs.s3a.access.key` |
| `fs.s3n.awsSecretAccessKey` | `fs.s3a.secret.key` |
Do make sure the property names are correct. For S3A, they are
`fs.s3a.access.key` and `fs.s3a.secret.key` —you cannot just copy the S3N
properties and replace `s3n` with `s3a`.
1. Replace URLs which began with `s3n://` with `s3a://`
1. You may now remove the `jets3t` JAR, as it is no longer needed.

View File

@ -12,9 +12,9 @@
limitations under the License. See accompanying LICENSE file.
-->
# Testing the S3 filesystem clients
# Testing the S3A filesystem client and its features, including S3Guard
<!-- MACRO{toc|fromDepth=0|toDepth=5} -->
<!-- MACRO{toc|fromDepth=0|toDepth=3} -->
This module includes both unit tests, which can run in isolation without
connecting to the S3 service, and integration tests, which require a working
@ -26,7 +26,7 @@ Due to eventual consistency, integration tests may fail without reason.
Transient failures, which no longer occur upon rerunning the test, should thus
be ignored.
## Policy for submitting patches which affect the `hadoop-aws` module.
## <a name="policy"></a> Policy for submitting patches which affect the `hadoop-aws` module.
The Apache Jenkins infrastucture does not run any S3 integration tests,
due to the need to keep credentials secure.
@ -74,7 +74,7 @@ in the production code, that could be a sign of a problem which may surface
over long-haul connections. Please help us identify and fix these problems
&mdash; especially as you are the one best placed to verify the fixes work.
## Setting up the tests
## <a name="setting-up"></a> Setting up the tests
To integration test the S3* filesystem clients, you need to provide
`auth-keys.xml` which passes in authentication details to the test runner.
@ -101,16 +101,11 @@ The XML file must contain all the ID/key information needed to connect
each of the filesystem clients to the object stores, and a URL for
each filesystem for its testing.
1. `test.fs.s3n.name` : the URL of the bucket for S3n tests
1. `test.fs.s3a.name` : the URL of the bucket for S3a tests
1. `fs.contract.test.fs.s3n` : the URL of the bucket for S3n filesystem contract tests
1. `fs.contract.test.fs.s3a` : the URL of the bucket for S3a filesystem contract tests
*Note* that running s3a and s3n tests in parallel mode, against the same bucket
is unreliable. We recommend using separate buckets or testing one connector
at a time.
The contents of each bucket will be destroyed during the test process:
The contents of the bucket will be destroyed during the test process:
do not use the bucket for any purpose other than testing. Furthermore, for
s3a, all in-progress multi-part uploads to the bucket will be aborted at the
start of a test (by forcing `fs.s3a.multipart.purge=true`) to clean up the
@ -121,26 +116,6 @@ Example:
```xml
<configuration>
<property>
<name>test.fs.s3n.name</name>
<value>s3n://test-aws-s3n/</value>
</property>
<property>
<name>fs.contract.test.fs.s3n</name>
<value>${test.fs.s3n.name}</value>
</property>
<property>
<name>fs.s3n.awsAccessKeyId</name>
<value>DONOTPCOMMITTHISKEYTOSCM</value>
</property>
<property>
<name>fs.s3n.awsSecretAccessKey</name>
<value>DONOTEVERSHARETHISSECRETKEY!</value>
</property>
<property>
<name>test.fs.s3a.name</name>
<value>s3a://test-aws-s3a/</value>
@ -172,7 +147,7 @@ Example:
</configuration>
```
### Configuring S3a Encryption
### <a name="encryption"></a> Configuring S3a Encryption
For S3a encryption tests to run correctly, the
`fs.s3a.server-side-encryption.key` must be configured in the s3a contract xml
@ -192,7 +167,7 @@ You can also force all the tests to run with a specific SSE encryption method
by configuring the property `fs.s3a.server-side-encryption-algorithm` in the s3a
contract file.
## Running the Tests
## <a name="running"></a> Running the Tests
After completing the configuration, execute the test run through Maven.
@ -251,7 +226,7 @@ combination with `test` or `it.test`. If you know that you are specifying only
tests that can run safely in parallel, then it will work. For wide patterns,
like `ITestS3A*` shown above, it may cause unpredictable test failures.
### Testing against different regions
### <a name="regions"></a> Testing against different regions
S3A can connect to different regions —the tests support this. Simply
define the target region in `auth-keys.xml`.
@ -265,7 +240,7 @@ define the target region in `auth-keys.xml`.
This is used for all tests expect for scale tests using a Public CSV.gz file
(see below)
### CSV Data source Tests
### <a name="csv"></a> CSV Data Tests
The `TestS3AInputStreamPerformance` tests require read access to a multi-MB
text file. The default file for these tests is one published by amazon,
@ -303,7 +278,7 @@ For the default test dataset, hosted in the `landsat-pds` bucket, this is:
</property>
```
### Viewing Integration Test Reports
## <a name="reporting"></a> Viewing Integration Test Reports
Integration test results and logs are stored in `target/failsafe-reports/`.
@ -313,7 +288,7 @@ plugin:
```bash
mvn surefire-report:failsafe-report-only
```
### Scale Tests
## <a name="scale"></a> Scale Tests
There are a set of tests designed to measure the scalability and performance
at scale of the S3A tests, *Scale Tests*. Tests include: creating
@ -325,7 +300,7 @@ By their very nature they are slow. And, as their execution time is often
limited by bandwidth between the computer running the tests and the S3 endpoint,
parallel execution does not speed these tests up.
#### Enabling the Scale Tests
### <a name="enabling-scale"></a> Enabling the Scale Tests
The tests are enabled if the `scale` property is set in the maven build
this can be done regardless of whether or not the parallel test profile
@ -342,7 +317,7 @@ sequentially; those which are slow due to HTTPS setup costs or server-side
actionsare included in the set of parallelized tests.
#### Maven build tuning options
### <a name="tuning_scale"></a> Tuning scale optins from Maven
Some of the tests can be tuned from the maven build or from the
@ -373,7 +348,7 @@ Only a few properties can be set this way; more will be added.
The file and partition sizes are numeric values with a k/m/g/t/p suffix depending
on the desired size. For example: 128M, 128m, 2G, 2G, 4T or even 1P.
#### Scale test configuration options
### <a name="scale-config"></a> Scale test configuration options
Some scale tests perform multiple operations (such as creating many directories).
@ -418,7 +393,7 @@ smaller to achieve faster test runs.
S3A specific scale test properties are
##### `fs.s3a.scale.test.huge.filesize`: size in MB for "Huge file tests".
*`fs.s3a.scale.test.huge.filesize`: size in MB for "Huge file tests".*
The Huge File tests validate S3A's ability to handle large files —the property
`fs.s3a.scale.test.huge.filesize` declares the file size to use.
@ -452,13 +427,11 @@ Otherwise, set a large timeout in `fs.s3a.scale.test.timeout`
</property>
```
The tests are executed in an order to only clean up created files after
the end of all the tests. If the tests are interrupted, the test data will remain.
## Testing against non AWS S3 endpoints.
## <a name="alternate_s3"></a> Testing against non AWS S3 endpoints.
The S3A filesystem is designed to work with storage endpoints which implement
the S3 protocols to the extent that the amazon S3 SDK is capable of talking
@ -527,7 +500,7 @@ An alternate endpoint may be defined in `test.fs.s3a.sts.endpoint`.
The default is ""; meaning "use the amazon default value".
## Debugging Test failures
## <a name="debugging"></a> Debugging Test failures
Logging at debug level is the standard way to provide more diagnostics output;
after setting this rerun the tests
@ -550,7 +523,7 @@ setting the `fs.s3a.user.agent.prefix` to a unique prefix for a specific
test run, which will enable the specific log entries to be more easily
located.
## Adding new tests
## <a name="new_tests"></a> Adding new tests
New tests are always welcome. Bear in mind that we need to keep costs
and test time down, which is done by
@ -593,7 +566,7 @@ fail with meaningful diagnostics, so any new problems can be easily debugged
from test logs.
### Requirements of new Tests
## <a name="requirements"></a> Requirements of new Tests
This is what we expect from new tests; they're an extension of the normal
@ -602,7 +575,7 @@ use requires the presence of secret credentials, where tests may be slow,
and where finding out why something failed from nothing but the test output
is critical.
#### Subclasses Existing Shared Base Classes
### Subclasses Existing Shared Base Classes
Extend `AbstractS3ATestBase` or `AbstractSTestS3AHugeFiles` unless justifiable.
These set things up for testing against the object stores, provide good threadnames,
@ -619,12 +592,12 @@ defined in `fs.s3a.contract.test`
Having shared base classes may help reduce future maintenance too. Please
use them/
#### Secure
### Secure
Don't ever log credentials. The credential tests go out of their way to
not provide meaningful logs or assertion messages precisely to avoid this.
#### Efficient of Time and Money
### Efficient of Time and Money
This means efficient in test setup/teardown, and, ideally, making use of
existing public datasets to save setup time and tester cost.
@ -650,7 +623,7 @@ against other regions, or with third party S3 implementations. Thus the
URL can be overridden for testing elsewhere.
#### Works With Other S3 Endpoints
### Works With Other S3 Endpoints
Don't assume AWS S3 US-East only, do allow for working with external S3 implementations.
Those may be behind the latest S3 API features, not support encryption, session
@ -678,7 +651,7 @@ adds some newlines so as to be easier to spot.
1. Use `ContractTestUtils.NanoTimer` to measure the duration of operations,
and log the output.
#### Fails Meaningfully
### Fails Meaningfully
The `ContractTestUtils` class contains a whole set of assertions for making
statements about the expected state of a filesystem, e.g.
@ -705,7 +678,7 @@ get called.
We really appreciate this &mdash; you will too.
## Tips
## <a name="tips"></a> Tips
### How to keep your credentials really safe
@ -725,7 +698,7 @@ using an absolute XInclude reference to it.
</configuration>
```
# Failure Injection
# <a name="failure-injection"></a>Failure Injection
**Warning do not enable any type of failure injection in production. The
following settings are for testing only.**
@ -858,7 +831,10 @@ The inconsistent client is shipped in the `hadoop-aws` JAR, so it can
be used in applications which work with S3 to see how they handle
inconsistent directory listings.
## Testing S3Guard
##<a name="s3guard"></a> Testing S3Guard
[S3Guard](./s3guard.html) is an extension to S3A which adds consistent metadata
listings to the S3A client. As it is part of S3A, it also needs to be tested.
The basic strategy for testing S3Guard correctness consists of:
@ -934,13 +910,6 @@ If the `s3guard` profile *is* set,
overwrite any previously set in the configuration files.
1. DynamoDB will be configured to create any missing tables.
### Warning About Concurrent Tests
You must not run S3A and S3N tests in parallel on the same bucket. This is
especially true when S3Guard is enabled. S3Guard requires that all clients
that are modifying the bucket have S3Guard enabled, so having S3N
integration tests running in parallel with S3A tests will cause strange
failures.
### Scale Testing MetadataStore Directly

View File

@ -12,27 +12,628 @@
limitations under the License. See accompanying LICENSE file.
-->
# Troubleshooting S3A
# Troubleshooting
Here are some lower level details and hints on troubleshooting and tuning
the S3A client.
<!-- MACRO{toc|fromDepth=0|toDepth=5} -->
## Logging at lower levels
##<a name="introduction"></a> Introduction
The AWS SDK and the Apache HTTP components can be configured to log at
more detail, as can S3A itself.
Common problems working with S3 are
```properties
log4j.logger.org.apache.hadoop.fs.s3a=DEBUG
log4j.logger.com.amazonaws.request=DEBUG
log4j.logger.org.apache.http=DEBUG
log4j.logger.org.apache.http.wire=ERROR
1. Classpath setup
1. Authentication
1. S3 Inconsistency side-effects
Classpath is usually the first problem. For the S3x filesystem clients,
you need the Hadoop-specific filesystem clients, third party S3 client libraries
compatible with the Hadoop code, and any dependent libraries compatible with
Hadoop and the specific JVM.
The classpath must be set up for the process talking to S3: if this is code
running in the Hadoop cluster, the JARs must be on that classpath. That
includes `distcp` and the `hadoop fs` command.
<!-- MACRO{toc|fromDepth=0|toDepth=2} -->
## <a name="classpath"></a> Classpath Setup
Note that for security reasons, the S3A client does not provide much detail
on the authentication process (i.e. the secrets used to authenticate).
### `ClassNotFoundException: org.apache.hadoop.fs.s3a.S3AFileSystem`
These is Hadoop filesytem client classes, found in the `hadoop-aws` JAR.
An exception reporting this class as missing means that this JAR is not on
the classpath.
### `ClassNotFoundException: com.amazonaws.services.s3.AmazonS3Client`
(or other `com.amazonaws` class.)
This means that the `aws-java-sdk-bundle.jar` JAR is not on the classpath:
add it.
### Missing method in `com.amazonaws` class
This can be triggered by incompatibilities between the AWS SDK on the classpath
and the version which Hadoop was compiled with.
The AWS SDK JARs change their signature enough between releases that the only
way to safely update the AWS SDK version is to recompile Hadoop against the later
version.
The sole fix is to use the same version of the AWS SDK with which Hadoop
was built.
## <a name="authentication"></a> Authentication Failure
If Hadoop cannot authenticate with the S3 service endpoint,
the client retries a number of times before eventually failing.
When it finally gives up, it will report a message about signature mismatch:
```
com.amazonaws.services.s3.model.AmazonS3Exception:
The request signature we calculated does not match the signature you provided.
Check your key and signing method.
(Service: Amazon S3; Status Code: 403; Error Code: SignatureDoesNotMatch,
```
Be aware that logging HTTP headers may leak sensitive AWS account information,
so should not be shared.
The likely cause is that you either have the wrong credentials or somehow
the credentials were not readable on the host attempting to read or write
the S3 Bucket.
## Advanced: network performance
Enabling debug logging for the package `org.apache.hadoop.fs.s3a`
can help provide more information.
The most common cause is that you have the wrong credentials for any of the current
authentication mechanism(s) —or somehow
the credentials were not readable on the host attempting to read or write
the S3 Bucket. However, there are a couple of system configuration problems
(JVM version, system clock) which also need to be checked.
Most common: there's an error in the configuration properties.
1. Make sure that the name of the bucket is the correct one.
That is: check the URL.
1. If using a private S3 server, make sure endpoint in `fs.s3a.endpoint` has
been set to this server -and that the client is not accidentally trying to
authenticate with the public Amazon S3 service.
1. Make sure the property names are correct. For S3A, they are
`fs.s3a.access.key` and `fs.s3a.secret.key` —you cannot just copy the S3N
properties and replace `s3n` with `s3a`.
1. Make sure the properties are visible to the process attempting to
talk to the object store. Placing them in `core-site.xml` is the standard
mechanism.
1. If using session authentication, the session may have expired.
Generate a new session token and secret.
1. If using environement variable-based authentication, make sure that the
relevant variables are set in the environment in which the process is running.
The standard first step is: try to use the AWS command line tools with the same
credentials, through a command such as:
hadoop fs -ls s3a://my-bucket/
Note the trailing "/" here; without that the shell thinks you are trying to list
your home directory under the bucket, which will only exist if explicitly created.
Attempting to list a bucket using inline credentials is a
means of verifying that the key and secret can access a bucket;
hadoop fs -ls s3a://key:secret@my-bucket/
Do escape any `+` or `/` symbols in the secret, as discussed below, and never
share the URL, logs generated using it, or use such an inline authentication
mechanism in production.
Finally, if you set the environment variables, you can take advantage of S3A's
support of environment-variable authentication by attempting the same ls operation.
That is: unset the `fs.s3a` secrets and rely on the environment variables.
### Authentication failure due to clock skew
The timestamp is used in signing to S3, so as to
defend against replay attacks. If the system clock is too far behind *or ahead*
of Amazon's, requests will be rejected.
This can surface as the situation where
read requests are allowed, but operations which write to the bucket are denied.
Check the system clock.
### Authentication failure when using URLs with embedded secrets
If using the (strongly discouraged) mechanism of including the
AWS Key and secret in a URL, then both "+" and "/" symbols need
to encoded in the URL. As many AWS secrets include these characters,
encoding problems are not uncommon.
| symbol | encoded value|
|-----------|-------------|
| `+` | `%2B` |
| `/` | `%2F` |
As an example, a URL for `bucket` with AWS ID `user1` and secret `a+b/c` would
be represented as
```
s3a://user1:a%2Bb%2Fc@bucket/
```
This technique is only needed when placing secrets in the URL. Again,
this is something users are strongly advised against using.
### <a name="bad_request"></a> "Bad Request" exception when working with AWS S3 Frankfurt, Seoul, or other "V4" endpoint
S3 Frankfurt and Seoul *only* support
[the V4 authentication API](http://docs.aws.amazon.com/AmazonS3/latest/API/sig-v4-authenticating-requests.html).
Requests using the V2 API will be rejected with 400 `Bad Request`
```
$ bin/hadoop fs -ls s3a://frankfurt/
WARN s3a.S3AFileSystem: Client: Amazon S3 error 400: 400 Bad Request; Bad Request (retryable)
com.amazonaws.services.s3.model.AmazonS3Exception: Bad Request (Service: Amazon S3; Status Code: 400; Error Code: 400 Bad Request; Request ID: 923C5D9E75E44C06), S3 Extended Request ID: HDwje6k+ANEeDsM6aJ8+D5gUmNAMguOk2BvZ8PH3g9z0gpH+IuwT7N19oQOnIr5CIx7Vqb/uThE=
at com.amazonaws.http.AmazonHttpClient.handleErrorResponse(AmazonHttpClient.java:1182)
at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:770)
at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
at com.amazonaws.services.s3.AmazonS3Client.headBucket(AmazonS3Client.java:1107)
at com.amazonaws.services.s3.AmazonS3Client.doesBucketExist(AmazonS3Client.java:1070)
at org.apache.hadoop.fs.s3a.S3AFileSystem.verifyBucketExists(S3AFileSystem.java:307)
at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:284)
at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2793)
at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:101)
at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2830)
at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2812)
at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:389)
at org.apache.hadoop.fs.Path.getFileSystem(Path.java:356)
at org.apache.hadoop.fs.shell.PathData.expandAsGlob(PathData.java:325)
at org.apache.hadoop.fs.shell.Command.expandArgument(Command.java:235)
at org.apache.hadoop.fs.shell.Command.expandArguments(Command.java:218)
at org.apache.hadoop.fs.shell.FsCommand.processRawArguments(FsCommand.java:103)
at org.apache.hadoop.fs.shell.Command.run(Command.java:165)
at org.apache.hadoop.fs.FsShell.run(FsShell.java:315)
at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76)
at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:90)
at org.apache.hadoop.fs.FsShell.main(FsShell.java:373)
ls: doesBucketExist on frankfurt-new: com.amazonaws.services.s3.model.AmazonS3Exception:
Bad Request (Service: Amazon S3; Status Code: 400; Error Code: 400 Bad Request;
```
This happens when trying to work with any S3 service which only supports the
"V4" signing API —but the client is configured to use the default S3 service
endpoint.
The S3A client needs to be given the endpoint to use via the `fs.s3a.endpoint`
property.
As an example, the endpoint for S3 Frankfurt is `s3.eu-central-1.amazonaws.com`:
```xml
<property>
<name>fs.s3a.endpoint</name>
<value>s3.eu-central-1.amazonaws.com</value>
</property>
```
## <a name="connectivity"></a> Connectivity Problems
### <a name="bad_endpoint"></a> Error message "The bucket you are attempting to access must be addressed using the specified endpoint"
This surfaces when `fs.s3a.endpoint` is configured to use an S3 service endpoint
which is neither the original AWS one, `s3.amazonaws.com` , nor the one where
the bucket is hosted. The error message contains the redirect target returned
by S3, which can be used to determine the correct value for `fs.s3a.endpoint`.
```
org.apache.hadoop.fs.s3a.AWSS3IOException: Received permanent redirect response
to bucket.s3-us-west-2.amazonaws.com. This likely indicates that the S3
endpoint configured in fs.s3a.endpoint does not match the AWS region
containing the bucket.: The bucket you are attempting to access must be
addressed using the specified endpoint. Please send all future requests to
this endpoint. (Service: Amazon S3; Status Code: 301;
Error Code: PermanentRedirect; Request ID: 7D39EC1021C61B11)
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:132)
at org.apache.hadoop.fs.s3a.S3AFileSystem.initMultipartUploads(S3AFileSystem.java:287)
at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:203)
at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2895)
at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:102)
at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2932)
at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2914)
at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:390)
```
1. Use the [Specific endpoint of the bucket's S3 service](http://docs.aws.amazon.com/general/latest/gr/rande.html#s3_region)
1. If not using "V4" authentication (see above), the original S3 endpoint
can be used:
```xml
<property>
<name>fs.s3a.endpoint</name>
<value>s3.amazonaws.com</value>
</property>
```
Using the explicit endpoint for the region is recommended for speed and
to use the V4 signing API.
### <a name="timeout"></a> "Timeout waiting for connection from pool" when writing data
This happens when using the output stream thread pool runs out of capacity.
```
[s3a-transfer-shared-pool1-t20] INFO http.AmazonHttpClient (AmazonHttpClient.java:executeHelper(496)) - Unable to execute HTTP request: Timeout waiting for connection from poolorg.apache.http.conn.ConnectionPoolTimeoutException: Timeout waiting for connection from pool
at org.apache.http.impl.conn.PoolingClientConnectionManager.leaseConnection(PoolingClientConnectionManager.java:230)
at org.apache.http.impl.conn.PoolingClientConnectionManager$1.getConnection(PoolingClientConnectionManager.java:199)
at sun.reflect.GeneratedMethodAccessor13.invoke(Unknown Source)
at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at com.amazonaws.http.conn.ClientConnectionRequestFactory$Handler.invoke(ClientConnectionRequestFactory.java:70)
at com.amazonaws.http.conn.$Proxy10.getConnection(Unknown Source)
at org.apache.http.impl.client.DefaultRequestDirector.execute(DefaultRequestDirector.java:424)
at org.apache.http.impl.client.AbstractHttpClient.doExecute(AbstractHttpClient.java:884)
at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:82)
at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:55)
at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:728)
at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
at com.amazonaws.services.s3.AmazonS3Client.doUploadPart(AmazonS3Client.java:2921)
at com.amazonaws.services.s3.AmazonS3Client.uploadPart(AmazonS3Client.java:2906)
at org.apache.hadoop.fs.s3a.S3AFileSystem.uploadPart(S3AFileSystem.java:1025)
at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload$1.call(S3ABlockOutputStream.java:360)
at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload$1.call(S3ABlockOutputStream.java:355)
at org.apache.hadoop.fs.s3a.BlockingThreadPoolExecutorService$CallableWithPermitRelease.call(BlockingThreadPoolExecutorService.java:239)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:745)
```
Make sure that `fs.s3a.connection.maximum` is at least larger
than `fs.s3a.threads.max`.
```xml
<property>
<name>fs.s3a.threads.max</name>
<value>20</value>
</property>
<property>
<name>fs.s3a.connection.maximum</name>
<value>30</value>
</property>
```
### "Timeout waiting for connection from pool" when reading data
This happens when more threads are trying to read from an S3A system than
the maximum number of allocated HTTP connections.
Set `fs.s3a.connection.maximum` to a larger value (and at least as large as
`fs.s3a.threads.max`)
### Out of heap memory when writing with via Fast Upload
This can happen when using the upload buffering mechanism
uses memory (either `fs.s3a.fast.upload.buffer=array` or
`fs.s3a.fast.upload.buffer=bytebuffer`).
More data is being generated than in the JVM than it can upload to S3 —and
so much data has been buffered that the JVM has run out of memory.
1. Consult [S3A Fast Upload Thread Tuning](./index.html#fast_upload_thread_tuning) for
detail on this issue and options to address it.
1. Switch to buffering to disk, rather than memory.
This surfaces if, while a multipart upload was taking place, all outstanding multipart
uploads were garbage collected. The upload operation cannot complete because
the data uploaded has been deleted.
Consult [Cleaning up After Incremental Upload Failures](./index.html#multipart_purge) for
details on how the multipart purge timeout can be set. If multipart uploads
are failing with the message above, it may be a sign that this value is too low.
### `MultiObjectDeleteException` during delete or rename of files
```
Exception in thread "main" com.amazonaws.services.s3.model.MultiObjectDeleteException:
Status Code: 0, AWS Service: null, AWS Request ID: null, AWS Error Code: null,
AWS Error Message: One or more objects could not be deleted, S3 Extended Request ID: null
at com.amazonaws.services.s3.AmazonS3Client.deleteObjects(AmazonS3Client.java:1745)
```
This happens when trying to delete multiple objects, and one of the objects
could not be deleted. It *should not occur* just because the object is missing.
More specifically: at the time this document was written, we could not create
such a failure.
It will occur if the caller lacks the permission to delete any of the objects.
Consult the log to see the specifics of which objects could not be deleted.
Do you have permission to do so?
If this operation is failing for reasons other than the caller lacking
permissions:
1. Try setting `fs.s3a.multiobjectdelete.enable` to `false`.
1. Consult [HADOOP-11572](https://issues.apache.org/jira/browse/HADOOP-11572)
for up to date advice.
### "Failed to Sanitize XML document"
```
org.apache.hadoop.fs.s3a.AWSClientIOException: getFileStatus on test/testname/streaming/:
com.amazonaws.AmazonClientException: Failed to sanitize XML document
destined for handler class com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser$ListBucketHandler:
Failed to sanitize XML document destined for handler class com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser$ListBucketHandler
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:105)
at org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:1462)
at org.apache.hadoop.fs.s3a.S3AFileSystem.innerListStatus(S3AFileSystem.java:1227)
at org.apache.hadoop.fs.s3a.S3AFileSystem.listStatus(S3AFileSystem.java:1203)
at org.apache.hadoop.fs.s3a.S3AGlobber.listStatus(S3AGlobber.java:69)
at org.apache.hadoop.fs.s3a.S3AGlobber.doGlob(S3AGlobber.java:210)
at org.apache.hadoop.fs.s3a.S3AGlobber.glob(S3AGlobber.java:125)
at org.apache.hadoop.fs.s3a.S3AFileSystem.globStatus(S3AFileSystem.java:1853)
at org.apache.hadoop.fs.s3a.S3AFileSystem.globStatus(S3AFileSystem.java:1841)
```
We believe this is caused by the connection to S3 being broken.
See [HADOOP-13811](https://issues.apache.org/jira/browse/HADOOP-13811).
It may go away if the operation is retried.
### JSON Parse Error from AWS SDK
Sometimes a JSON Parse error is reported with the stack trace in the `com.amazonaws`,
Again, we believe this is caused by the connection to S3 being broken.
It may go away if the operation is retried.
## Miscellaneous Errors
### When writing data: "java.io.FileNotFoundException: Completing multi-part upload"
```
java.io.FileNotFoundException: Completing multi-part upload on fork-5/test/multipart/1c397ca6-9dfb-4ac1-9cf7-db666673246b: com.amazonaws.services.s3.model.AmazonS3Exception: The specified upload does not exist. The upload ID may be invalid, or the upload may have been aborted or completed. (Service: Amazon S3; Status Code: 404; Error Code: NoSuchUpload; Request ID: 84FF8057174D9369), S3 Extended Request ID: Ij5Yn6Eq/qIERH4Z6Io3YL2t9/qNZ7z9gjPb1FrTtTovZ8k1MXqh+zCYYjqmfJ/fCY6E1+JR9jA=
at com.amazonaws.http.AmazonHttpClient.handleErrorResponse(AmazonHttpClient.java:1182)
at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:770)
at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
at com.amazonaws.services.s3.AmazonS3Client.completeMultipartUpload(AmazonS3Client.java:2705)
at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.complete(S3ABlockOutputStream.java:473)
at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.access$200(S3ABlockOutputStream.java:382)
at org.apache.hadoop.fs.s3a.S3ABlockOutputStream.close(S3ABlockOutputStream.java:272)
at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72)
at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106)
```
### Issue: when writing data, HTTP Exceptions logged at info from `AmazonHttpClient`
```
[s3a-transfer-shared-pool4-t6] INFO http.AmazonHttpClient (AmazonHttpClient.java:executeHelper(496)) - Unable to execute HTTP request: hwdev-steve-ireland-new.s3.amazonaws.com:443 failed to respond
org.apache.http.NoHttpResponseException: bucket.s3.amazonaws.com:443 failed to respond
at org.apache.http.impl.conn.DefaultHttpResponseParser.parseHead(DefaultHttpResponseParser.java:143)
at org.apache.http.impl.conn.DefaultHttpResponseParser.parseHead(DefaultHttpResponseParser.java:57)
at org.apache.http.impl.io.AbstractMessageParser.parse(AbstractMessageParser.java:261)
at org.apache.http.impl.AbstractHttpClientConnection.receiveResponseHeader(AbstractHttpClientConnection.java:283)
at org.apache.http.impl.conn.DefaultClientConnection.receiveResponseHeader(DefaultClientConnection.java:259)
at org.apache.http.impl.conn.ManagedClientConnectionImpl.receiveResponseHeader(ManagedClientConnectionImpl.java:209)
at org.apache.http.protocol.HttpRequestExecutor.doReceiveResponse(HttpRequestExecutor.java:272)
at com.amazonaws.http.protocol.SdkHttpRequestExecutor.doReceiveResponse(SdkHttpRequestExecutor.java:66)
at org.apache.http.protocol.HttpRequestExecutor.execute(HttpRequestExecutor.java:124)
at org.apache.http.impl.client.DefaultRequestDirector.tryExecute(DefaultRequestDirector.java:686)
at org.apache.http.impl.client.DefaultRequestDirector.execute(DefaultRequestDirector.java:488)
at org.apache.http.impl.client.AbstractHttpClient.doExecute(AbstractHttpClient.java:884)
at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:82)
at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:55)
at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:728)
at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
at com.amazonaws.services.s3.AmazonS3Client.copyPart(AmazonS3Client.java:1731)
at com.amazonaws.services.s3.transfer.internal.CopyPartCallable.call(CopyPartCallable.java:41)
at com.amazonaws.services.s3.transfer.internal.CopyPartCallable.call(CopyPartCallable.java:28)
at org.apache.hadoop.fs.s3a.BlockingThreadPoolExecutorService$CallableWithPermitRelease.call(BlockingThreadPoolExecutorService.java:239)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:745)
```
These are HTTP I/O exceptions caught and logged inside the AWS SDK. The client
will attempt to retry the operation; it may just be a transient event. If there
are many such exceptions in logs, it may be a symptom of connectivity or network
problems.
## File System Semantics
These are the issues where S3 does not appear to behave the way a filesystem
"should".
### Visible S3 Inconsistency
Amazon S3 is *an eventually consistent object store*. That is: not a filesystem.
To reduce visible inconsistencies, use the [S3Guard](./s3guard.html) consistency
cache.
By default, Amazon S3 offers read-after-create consistency: a newly created file
is immediately visible.
There is a small quirk: a negative GET may be cached, such
that even if an object is immediately created, the fact that there "wasn't"
an object is still remembered.
That means the following sequence on its own will be consistent
```
touch(path) -> getFileStatus(path)
```
But this sequence *may* be inconsistent.
```
getFileStatus(path) -> touch(path) -> getFileStatus(path)
```
A common source of visible inconsistencies is that the S3 metadata
database —the part of S3 which serves list requests— is updated asynchronously.
Newly added or deleted files may not be visible in the index, even though direct
operations on the object (`HEAD` and `GET`) succeed.
That means the `getFileStatus()` and `open()` operations are more likely
to be consistent with the state of the object store, but without S3Guard enabled,
directory list operations such as `listStatus()`, `listFiles()`, `listLocatedStatus()`,
and `listStatusIterator()` may not see newly created files, and still list
old files.
### `FileNotFoundException` even though the file was just written.
This can be a sign of consistency problems. It may also surface if there is some
asynchronous file write operation still in progress in the client: the operation
has returned, but the write has not yet completed. While the S3A client code
does block during the `close()` operation, we suspect that asynchronous writes
may be taking place somewhere in the stack —this could explain why parallel tests
fail more often than serialized tests.
### File not found in a directory listing, even though `getFileStatus()` finds it
(Similarly: deleted file found in listing, though `getFileStatus()` reports
that it is not there)
This is a visible sign of updates to the metadata server lagging
behind the state of the underlying filesystem.
Fix: Use S3Guard
### File not visible/saved
The files in an object store are not visible until the write has been completed.
In-progress writes are simply saved to a local file/cached in RAM and only uploaded.
at the end of a write operation. If a process terminated unexpectedly, or failed
to call the `close()` method on an output stream, the pending data will have
been lost.
### File `flush()`, `hsync` and `hflush()` calls do not save data to S3
Again, this is due to the fact that the data is cached locally until the
`close()` operation. The S3A filesystem cannot be used as a store of data
if it is required that the data is persisted durably after every
`Syncable.hflush()` or `Syncable.hsync()` call.
This includes resilient logging, HBase-style journalling
and the like. The standard strategy here is to save to HDFS and then copy to S3.
## <a name="encryption"></a> S3 Server Side Encryption
### Using SSE-KMS "Invalid arn"
When performing file operations, the user may run into an issue where the KMS
key arn is invalid.
```
com.amazonaws.services.s3.model.AmazonS3Exception:
Invalid arn (Service: Amazon S3; Status Code: 400; Error Code: KMS.NotFoundException; Request ID: 708284CF60EE233F),
S3 Extended Request ID: iHUUtXUSiNz4kv3Bdk/hf9F+wjPt8GIVvBHx/HEfCBYkn7W6zmpvbA3XT7Y5nTzcZtfuhcqDunw=:
Invalid arn (Service: Amazon S3; Status Code: 400; Error Code: KMS.NotFoundException; Request ID: 708284CF60EE233F)
```
This is due to either, the KMS key id is entered incorrectly, or the KMS key id
is in a different region than the S3 bucket being used.
### Using SSE-C "Bad Request"
When performing file operations the user may run into an unexpected 400/403
error such as
```
org.apache.hadoop.fs.s3a.AWSS3IOException: getFileStatus on fork-4/: com.amazonaws.services.s3.model.AmazonS3Exception:
Bad Request (Service: Amazon S3; Status Code: 400;
Error Code: 400 Bad Request; Request ID: 42F9A1987CB49A99),
S3 Extended Request ID: jU2kcwaXnWj5APB14Cgb1IKkc449gu2+dhIsW/+7x9J4D+VUkKvu78mBo03oh9jnOT2eoTLdECU=:
Bad Request (Service: Amazon S3; Status Code: 400; Error Code: 400 Bad Request; Request ID: 42F9A1987CB49A99)
```
This can happen in the cases of not specifying the correct SSE-C encryption key.
Such cases can be as follows:
1. An object is encrypted using SSE-C on S3 and either the wrong encryption type
is used, no encryption is specified, or the SSE-C specified is incorrect.
2. A directory is encrypted with a SSE-C keyA and the user is trying to move a
file using configured SSE-C keyB into that structure.
## <a name="performance"></a> Performance
S3 is slower to read data than HDFS, even on virtual clusters running on
Amazon EC2.
* HDFS replicates data for faster query performance.
* HDFS stores the data on the local hard disks, avoiding network traffic
if the code can be executed on that host. As EC2 hosts often have their
network bandwidth throttled, this can make a tangible difference.
* HDFS is significantly faster for many "metadata" operations: listing
the contents of a directory, calling `getFileStatus()` on path,
creating or deleting directories. (S3Guard reduces but does not eliminate
the speed gap).
* On HDFS, Directory renames and deletes are `O(1)` operations. On
S3 renaming is a very expensive `O(data)` operation which may fail partway through
in which case the final state depends on where the copy+ delete sequence was when it failed.
All the objects are copied, then the original set of objects are deleted, so
a failure should not lose data —it may result in duplicate datasets.
* Unless fast upload enabled, the write only begins on a `close()` operation.
This can take so long that some applications can actually time out.
* File IO involving many seek calls/positioned read calls will encounter
performance problems due to the size of the HTTP requests made. Enable the
"random" fadvise policy to alleviate this at the
expense of sequential read performance and bandwidth.
The slow performance of `rename()` surfaces during the commit phase of work,
including
* The MapReduce `FileOutputCommitter`. This also used by Apache Spark.
* DistCp's rename-after-copy operation.
* The `hdfs fs -rm` command renaming the file under `.Trash` rather than
deleting it. Use `-skipTrash` to eliminate that step.
These operations can be significantly slower when S3 is the destination
compared to HDFS or other "real" filesystem.
*Improving S3 load-balancing behavior*
Amazon S3 uses a set of front-end servers to provide access to the underlying data.
The choice of which front-end server to use is handled via load-balancing DNS
service: when the IP address of an S3 bucket is looked up, the choice of which
IP address to return to the client is made based on the the current load
of the front-end servers.
Over time, the load across the front-end changes, so those servers considered
"lightly loaded" will change. If the DNS value is cached for any length of time,
your application may end up talking to an overloaded server. Or, in the case
of failures, trying to talk to a server that is no longer there.
And by default, for historical security reasons in the era of applets,
the DNS TTL of a JVM is "infinity".
To work with AWS better, set the DNS time-to-live of an application which
works with S3 to something lower. See [AWS documentation](http://docs.aws.amazon.com/AWSSdkDocsJava/latest/DeveloperGuide/java-dg-jvm-ttl.html).
## <a name="network_performance"></a>Troubleshooting network performance
An example of this is covered in [HADOOP-13871](https://issues.apache.org/jira/browse/HADOOP-13871).
@ -49,4 +650,74 @@ Consider reducing the connection timeout of the s3a connection.
<value>15000</value>
</property>
```
This *may* cause the client to react faster to network pauses.
This *may* cause the client to react faster to network pauses, so display
stack traces fast. At the same time, it may be less resilient to
connectivity problems.
## Other Issues
### <a name="logging"></a> Enabling low-level logging
The AWS SDK and the Apache S3 components can be configured to log at
more detail, as can S3A itself.
```properties
log4j.logger.org.apache.hadoop.fs.s3a=DEBUG
log4j.logger.com.amazonaws.request=DEBUG
log4j.logger.com.amazonaws.thirdparty.apache.http=DEBUG
```
If using the "unshaded" JAR, then the Apache HttpClient can be directly configured:
```properties
log4j.logger.org.apache.http=DEBUG
```
This produces a log such as this, wich is for a V4-authenticated PUT of a 0-byte file used
as an empty directory marker
```
execchain.MainClientExec (MainClientExec.java:execute(255)) - Executing request PUT /test/ HTTP/1.1
execchain.MainClientExec (MainClientExec.java:execute(266)) - Proxy auth state: UNCHALLENGED
http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(135)) - http-outgoing-0 >> PUT /test/ HTTP/1.1
http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(138)) - http-outgoing-0 >> Host: ireland-new.s3-eu-west-1.amazonaws.com
http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(138)) - http-outgoing-0 >> x-amz-content-sha256: UNSIGNED-PAYLOAD
http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(138)) - http-outgoing-0 >> Authorization: AWS4-HMAC-SHA256 Credential=AKIAIYZ5JEEEER/20170904/eu-west-1/s3/aws4_request, ...
http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(138)) - http-outgoing-0 >> X-Amz-Date: 20170904T172929Z
http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(138)) - http-outgoing-0 >> User-Agent: Hadoop 3.0.0-beta-1, aws-sdk-java/1.11.134 ...
http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(138)) - http-outgoing-0 >> amz-sdk-invocation-id: 75b530f8-ad31-1ad3-13db-9bd53666b30d
http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(138)) - http-outgoing-0 >> amz-sdk-retry: 0/0/500
http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(138)) - http-outgoing-0 >> Content-Type: application/octet-stream
http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(138)) - http-outgoing-0 >> Content-Length: 0
http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(138)) - http-outgoing-0 >> Connection: Keep-Alive
http.wire (Wire.java:wire(72)) - http-outgoing-0 >> "PUT /test/ HTTP/1.1[\r][\n]"
http.wire (Wire.java:wire(72)) - http-outgoing-0 >> "Host: ireland-new.s3-eu-west-1.amazonaws.com[\r][\n]"
http.wire (Wire.java:wire(72)) - http-outgoing-0 >> "x-amz-content-sha256: UNSIGNED-PAYLOAD[\r][\n]"
http.wire (Wire.java:wire(72)) - http-outgoing-0 >> "Authorization: AWS4-HMAC-SHA256 Credential=AKIAIYZ5JEEEER/20170904/eu-west-1/s3/aws4_request, ,,,
http.wire (Wire.java:wire(72)) - http-outgoing-0 >> "X-Amz-Date: 20170904T172929Z[\r][\n]"
http.wire (Wire.java:wire(72)) - http-outgoing-0 >> "User-Agent: 3.0.0-beta-1, aws-sdk-java/1.11.134 ...
http.wire (Wire.java:wire(72)) - http-outgoing-0 >> "amz-sdk-invocation-id: 75b530f8-ad31-1ad3-13db-9bd53666b30d[\r][\n]"
http.wire (Wire.java:wire(72)) - http-outgoing-0 >> "amz-sdk-retry: 0/0/500[\r][\n]"
http.wire (Wire.java:wire(72)) - http-outgoing-0 >> "Content-Type: application/octet-stream[\r][\n]"
http.wire (Wire.java:wire(72)) - http-outgoing-0 >> "Content-Length: 0[\r][\n]"
http.wire (Wire.java:wire(72)) - http-outgoing-0 >> "Connection: Keep-Alive[\r][\n]"
http.wire (Wire.java:wire(72)) - http-outgoing-0 >> "[\r][\n]"
http.wire (Wire.java:wire(72)) - http-outgoing-0 << "HTTP/1.1 200 OK[\r][\n]"
http.wire (Wire.java:wire(72)) - http-outgoing-0 << "x-amz-id-2: mad9GqKztzlL0cdnCKAj9GJOAs+DUjbSC5jRkO7W1E7Nk2BUmFvt81bhSNPGdZmyyKqQI9i/B/A=[\r][\n]"
http.wire (Wire.java:wire(72)) - http-outgoing-0 << "x-amz-request-id: C953D2FE4ABF5C51[\r][\n]"
http.wire (Wire.java:wire(72)) - http-outgoing-0 << "Date: Mon, 04 Sep 2017 17:29:30 GMT[\r][\n]"
http.wire (Wire.java:wire(72)) - http-outgoing-0 << "ETag: "d41d8cd98f00b204e9800998ecf8427e"[\r][\n]"
http.wire (Wire.java:wire(72)) - http-outgoing-0 << "Content-Length: 0[\r][\n]"
http.wire (Wire.java:wire(72)) - http-outgoing-0 << "Server: AmazonS3[\r][\n]"
http.wire (Wire.java:wire(72)) - http-outgoing-0 << "[\r][\n]"
http.headers (LoggingManagedHttpClientConnection.java:onResponseReceived(124)) - http-outgoing-0 << HTTP/1.1 200 OK
http.headers (LoggingManagedHttpClientConnection.java:onResponseReceived(127)) - http-outgoing-0 << x-amz-id-2: mad9GqKztzlL0cdnCKAj9GJOAs+DUjbSC5jRkO7W1E7Nk2BUmFvt81bhSNPGdZmyyKqQI9i/B/A=
http.headers (LoggingManagedHttpClientConnection.java:onResponseReceived(127)) - http-outgoing-0 << x-amz-request-id: C953D2FE4ABF5C51
http.headers (LoggingManagedHttpClientConnection.java:onResponseReceived(127)) - http-outgoing-0 << Date: Mon, 04 Sep 2017 17:29:30 GMT
http.headers (LoggingManagedHttpClientConnection.java:onResponseReceived(127)) - http-outgoing-0 << ETag: "d41d8cd98f00b204e9800998ecf8427e"
http.headers (LoggingManagedHttpClientConnection.java:onResponseReceived(127)) - http-outgoing-0 << Content-Length: 0
http.headers (LoggingManagedHttpClientConnection.java:onResponseReceived(127)) - http-outgoing-0 << Server: AmazonS3
execchain.MainClientExec (MainClientExec.java:execute(284)) - Connection can be kept alive for 60000 MILLISECONDS
```

View File

@ -47,7 +47,6 @@ public class ITestS3AContractDistCp extends AbstractContractDistCpTest {
protected Configuration createConfiguration() {
Configuration newConf = super.createConfiguration();
newConf.setLong(MULTIPART_SIZE, MULTIPART_SETTING);
newConf.setBoolean(FAST_UPLOAD, true);
newConf.set(FAST_UPLOAD_BUFFER, FAST_UPLOAD_BUFFER_DISK);
// patch in S3Guard options
maybeEnableS3Guard(newConf);

View File

@ -1,41 +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.contract.s3n;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.contract.AbstractContractCreateTest;
import org.apache.hadoop.fs.contract.AbstractFSContract;
import org.apache.hadoop.fs.contract.ContractTestUtils;
/**
* S3N contract tests creating files.
*/
public class ITestS3NContractCreate extends AbstractContractCreateTest {
@Override
protected AbstractFSContract createContract(Configuration conf) {
return new NativeS3Contract(conf);
}
@Override
public void testOverwriteEmptyDirectory() throws Throwable {
ContractTestUtils.skip(
"blobstores can't distinguish empty directories from files");
}
}

View File

@ -1,34 +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.contract.s3n;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.contract.AbstractContractDeleteTest;
import org.apache.hadoop.fs.contract.AbstractFSContract;
/**
* S3A contract tests covering deletes.
*/
public class ITestS3NContractDelete extends AbstractContractDeleteTest {
@Override
protected AbstractFSContract createContract(Configuration conf) {
return new NativeS3Contract(conf);
}
}

View File

@ -1,34 +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.contract.s3n;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.contract.AbstractContractMkdirTest;
import org.apache.hadoop.fs.contract.AbstractFSContract;
/**
* Test dir operations on S3.
*/
public class ITestS3NContractMkdir extends AbstractContractMkdirTest {
@Override
protected AbstractFSContract createContract(Configuration conf) {
return new NativeS3Contract(conf);
}
}

View File

@ -1,34 +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.contract.s3n;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.contract.AbstractContractOpenTest;
import org.apache.hadoop.fs.contract.AbstractFSContract;
/**
* S3N contract tests opening files.
*/
public class ITestS3NContractOpen extends AbstractContractOpenTest {
@Override
protected AbstractFSContract createContract(Configuration conf) {
return new NativeS3Contract(conf);
}
}

View File

@ -1,35 +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.contract.s3n;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
import org.apache.hadoop.fs.contract.AbstractFSContract;
/**
* S3N contract tests covering rename.
*/
public class ITestS3NContractRename extends AbstractContractRenameTest {
@Override
protected AbstractFSContract createContract(Configuration conf) {
return new NativeS3Contract(conf);
}
}

View File

@ -1,35 +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.contract.s3n;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest;
import org.apache.hadoop.fs.contract.AbstractFSContract;
/**
* Root dir operations against an S3 bucket.
*/
public class ITestS3NContractRootDir extends
AbstractContractRootDirectoryTest {
@Override
protected AbstractFSContract createContract(Configuration conf) {
return new NativeS3Contract(conf);
}
}

View File

@ -1,34 +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.contract.s3n;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.contract.AbstractContractSeekTest;
import org.apache.hadoop.fs.contract.AbstractFSContract;
/**
* S3N contract tests covering file seek.
*/
public class ITestS3NContractSeek extends AbstractContractSeekTest {
@Override
protected AbstractFSContract createContract(Configuration conf) {
return new NativeS3Contract(conf);
}
}

View File

@ -1,50 +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.contract.s3n;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.contract.AbstractBondedFSContract;
/**
* The contract of S3N: only enabled if the test bucket is provided.
*/
public class NativeS3Contract extends AbstractBondedFSContract {
public static final String CONTRACT_XML = "contract/s3n.xml";
public NativeS3Contract(Configuration conf) {
super(conf);
//insert the base features
addConfResource(CONTRACT_XML);
}
@Override
public String getScheme() {
return "s3n";
}
@Override
public Path getTestPath() {
String testUniqueForkId = System.getProperty("test.unique.fork.id");
return testUniqueForkId == null ? super.getTestPath() :
new Path("/" + testUniqueForkId, "test");
}
}

View File

@ -56,7 +56,6 @@ public class ITestS3ABlockOutputArray extends AbstractS3ATestBase {
S3ATestUtils.disableFilesystemCaching(conf);
conf.setLong(MIN_MULTIPART_THRESHOLD, MULTIPART_MIN_SIZE);
conf.setInt(MULTIPART_SIZE, MULTIPART_MIN_SIZE);
conf.setBoolean(Constants.FAST_UPLOAD, true);
conf.set(FAST_UPLOAD_BUFFER, getBlockOutputBufferName());
return conf;
}

View File

@ -31,7 +31,6 @@ public class ITestS3AEncryptionSSECBlockOutputStream
@Override
protected Configuration createConfiguration() {
Configuration conf = super.createConfiguration();
conf.setBoolean(Constants.FAST_UPLOAD, true);
conf.set(Constants.FAST_UPLOAD_BUFFER,
Constants.FAST_UPLOAD_BYTEBUFFER);
conf.set(Constants.SERVER_SIDE_ENCRYPTION_KEY,

View File

@ -39,9 +39,7 @@ public class ITestS3AEncryptionSSEKMSUserDefinedKeyBlockOutputStream
skip(Constants.SERVER_SIDE_ENCRYPTION_KEY+ " is not set for " +
S3AEncryptionMethods.SSE_KMS.getMethod());
}
conf.setBoolean(Constants.FAST_UPLOAD, true);
conf.set(Constants.FAST_UPLOAD_BUFFER,
Constants.FAST_UPLOAD_BYTEBUFFER);
conf.set(Constants.FAST_UPLOAD_BUFFER, Constants.FAST_UPLOAD_BYTEBUFFER);
return conf;
}

View File

@ -29,7 +29,6 @@ public class ITestS3AEncryptionSSES3BlockOutputStream
@Override
protected Configuration createConfiguration() {
Configuration conf = super.createConfiguration();
conf.setBoolean(Constants.FAST_UPLOAD, true);
conf.set(Constants.FAST_UPLOAD_BUFFER,
Constants.FAST_UPLOAD_BYTEBUFFER);
//must specify encryption key as empty because SSE-S3 does not allow it,

View File

@ -98,7 +98,6 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
conf.setLong(MIN_MULTIPART_THRESHOLD, partitionSize);
conf.setInt(MULTIPART_SIZE, partitionSize);
conf.set(USER_AGENT_PREFIX, "STestS3AHugeFileCreate");
conf.setBoolean(FAST_UPLOAD, true);
conf.set(FAST_UPLOAD_BUFFER, getBlockOutputBufferName());
return conf;
}

View File

@ -1,41 +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.s3a.scale;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.s3a.Constants;
/**
* Use classic output for writing things; tweaks the configuration to do
* this after it has been set up in the superclass.
* The generator test has been copied and re
*/
public class ITestS3AHugeFilesClassicOutput extends AbstractSTestS3AHugeFiles {
@Override
protected Configuration createScaleConfiguration() {
final Configuration conf = super.createScaleConfiguration();
conf.setBoolean(Constants.FAST_UPLOAD, false);
return conf;
}
protected String getBlockOutputBufferName() {
return "classic";
}
}

View File

@ -1,33 +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;
/**
* S3N basic contract tests through mock in-memory S3 implementation.
*/
public class ITestInMemoryNativeS3FileSystemContract
extends NativeS3FileSystemContractBaseTest {
@Override
NativeFileSystemStore getNativeFileSystemStore() throws IOException {
return new InMemoryNativeFileSystemStore();
}
}

View File

@ -1,133 +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.conf.Configuration;
import org.apache.hadoop.fs.Path;
import static org.junit.Assert.*;
import static org.junit.Assume.*;
import org.junit.Before;
import org.junit.After;
import org.junit.BeforeClass;
import org.junit.Test;
import java.io.BufferedInputStream;
import java.io.BufferedOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.net.URI;
import java.security.DigestInputStream;
import java.security.DigestOutputStream;
import java.security.MessageDigest;
import java.security.NoSuchAlgorithmException;
/**
* S3N tests through live S3 service.
*/
public class ITestJets3tNativeFileSystemStore {
private Configuration conf;
private Jets3tNativeFileSystemStore store;
private NativeS3FileSystem fs;
@Before
public void setUp() throws Exception {
conf = new Configuration();
store = new Jets3tNativeFileSystemStore();
fs = new NativeS3FileSystem(store);
conf.setBoolean("fs.s3n.multipart.uploads.enabled", true);
conf.setLong("fs.s3n.multipart.uploads.block.size", 64 * 1024 * 1024);
fs.initialize(URI.create(conf.get("test.fs.s3n.name")), conf);
}
@After
public void tearDown() throws Exception {
try {
store.purge("test");
} catch (Exception e) {}
}
@BeforeClass
public static void checkSettings() throws Exception {
Configuration conf = new Configuration();
assumeNotNull(conf.get("fs.s3n.awsAccessKeyId"));
assumeNotNull(conf.get("fs.s3n.awsSecretAccessKey"));
assumeNotNull(conf.get("test.fs.s3n.name"));
}
protected void writeRenameReadCompare(Path path, long len)
throws IOException, NoSuchAlgorithmException {
// If len > fs.s3n.multipart.uploads.block.size,
// we'll use a multipart upload copy
MessageDigest digest = MessageDigest.getInstance("MD5");
OutputStream out = new BufferedOutputStream(
new DigestOutputStream(fs.create(path, false), digest));
for (long i = 0; i < len; i++) {
out.write('Q');
}
out.flush();
out.close();
assertTrue("Exists", fs.exists(path));
// Depending on if this file is over 5 GB or not,
// rename will cause a multipart upload copy
Path copyPath = path.suffix(".copy");
fs.rename(path, copyPath);
assertTrue("Copy exists", fs.exists(copyPath));
// Download file from S3 and compare the digest against the original
MessageDigest digest2 = MessageDigest.getInstance("MD5");
InputStream in = new BufferedInputStream(
new DigestInputStream(fs.open(copyPath), digest2));
long copyLen = 0;
while (in.read() != -1) {
copyLen++;
}
in.close();
assertEquals("Copy length matches original", len, copyLen);
assertArrayEquals("Digests match", digest.digest(), digest2.digest());
}
@Test
public void testSmallUpload() throws IOException, NoSuchAlgorithmException {
// Regular upload, regular copy
writeRenameReadCompare(new Path("/test/small"), 16384);
}
@Test
public void testMediumUpload() throws IOException, NoSuchAlgorithmException {
// Multipart upload, regular copy
writeRenameReadCompare(new Path("/test/medium"), 33554432); // 100 MB
}
/*
Enable Multipart upload to run this test
@Test
public void testExtraLargeUpload()
throws IOException, NoSuchAlgorithmException {
// Multipart upload, multipart copy
writeRenameReadCompare(new Path("/test/xlarge"), 5368709121L); // 5GB+1byte
}
*/
}

View File

@ -1,33 +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;
/**
* S3N basic contract tests through live S3 service.
*/
public class ITestJets3tNativeS3FileSystemContract
extends NativeS3FileSystemContractBaseTest {
@Override
NativeFileSystemStore getNativeFileSystemStore() throws IOException {
return new Jets3tNativeFileSystemStore();
}
}

View File

@ -1,213 +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 static org.apache.hadoop.fs.s3native.NativeS3FileSystem.PATH_DELIMITER;
import static org.apache.hadoop.fs.s3native.S3NativeFileSystemConfigKeys.S3_NATIVE_BUFFER_DIR_KEY;
import static org.apache.hadoop.fs.s3native.S3NativeFileSystemConfigKeys.addDeprecatedConfigKeys;
import java.io.BufferedInputStream;
import java.io.BufferedOutputStream;
import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.net.URI;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.SortedMap;
import java.util.SortedSet;
import java.util.TreeMap;
import java.util.TreeSet;
import java.util.Map.Entry;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.util.Time;
/**
* <p>
* A stub implementation of {@link NativeFileSystemStore} for testing
* {@link NativeS3FileSystem} without actually connecting to S3.
* </p>
*/
public class InMemoryNativeFileSystemStore implements NativeFileSystemStore {
static {
// Add the deprecated config keys
addDeprecatedConfigKeys();
}
private Configuration conf;
private SortedMap<String, FileMetadata> metadataMap =
new TreeMap<String, FileMetadata>();
private SortedMap<String, byte[]> dataMap = new TreeMap<String, byte[]>();
@Override
public void initialize(URI uri, Configuration conf) throws IOException {
this.conf = conf;
}
@Override
public void storeEmptyFile(String key) throws IOException {
metadataMap.put(key, new FileMetadata(key, 0, Time.now()));
dataMap.put(key, new byte[0]);
}
@Override
public void storeFile(String key, File file, byte[] md5Hash)
throws IOException {
ByteArrayOutputStream out = new ByteArrayOutputStream();
byte[] buf = new byte[8192];
int numRead;
BufferedInputStream in = null;
try {
in = new BufferedInputStream(new FileInputStream(file));
while ((numRead = in.read(buf)) >= 0) {
out.write(buf, 0, numRead);
}
} finally {
if (in != null) {
in.close();
}
}
metadataMap.put(key,
new FileMetadata(key, file.length(), Time.now()));
dataMap.put(key, out.toByteArray());
}
@Override
public InputStream retrieve(String key) throws IOException {
return retrieve(key, 0);
}
@Override
public InputStream retrieve(String key, long byteRangeStart)
throws IOException {
byte[] data = dataMap.get(key);
File file = createTempFile();
BufferedOutputStream out = null;
try {
out = new BufferedOutputStream(new FileOutputStream(file));
out.write(data, (int) byteRangeStart,
data.length - (int) byteRangeStart);
} finally {
if (out != null) {
out.close();
}
}
return new FileInputStream(file);
}
private File createTempFile() throws IOException {
File dir = new File(conf.get(S3_NATIVE_BUFFER_DIR_KEY));
if (!dir.exists() && !dir.mkdirs()) {
throw new IOException("Cannot create S3 buffer directory: " + dir);
}
File result = File.createTempFile("test-", ".tmp", dir);
result.deleteOnExit();
return result;
}
@Override
public FileMetadata retrieveMetadata(String key) throws IOException {
return metadataMap.get(key);
}
@Override
public PartialListing list(String prefix, int maxListingLength)
throws IOException {
return list(prefix, maxListingLength, null, false);
}
@Override
public PartialListing list(String prefix, int maxListingLength,
String priorLastKey, boolean recursive) throws IOException {
return list(prefix, recursive ? null : PATH_DELIMITER, maxListingLength, priorLastKey);
}
private PartialListing list(String prefix, String delimiter,
int maxListingLength, String priorLastKey) throws IOException {
if (prefix.length() > 0 && !prefix.endsWith(PATH_DELIMITER)) {
prefix += PATH_DELIMITER;
}
List<FileMetadata> metadata = new ArrayList<FileMetadata>();
SortedSet<String> commonPrefixes = new TreeSet<String>();
for (String key : dataMap.keySet()) {
if (key.startsWith(prefix)) {
if (delimiter == null) {
metadata.add(retrieveMetadata(key));
} else {
int delimIndex = key.indexOf(delimiter, prefix.length());
if (delimIndex == -1) {
metadata.add(retrieveMetadata(key));
} else {
String commonPrefix = key.substring(0, delimIndex);
commonPrefixes.add(commonPrefix);
}
}
}
if (metadata.size() + commonPrefixes.size() == maxListingLength) {
new PartialListing(key, metadata.toArray(new FileMetadata[0]),
commonPrefixes.toArray(new String[0]));
}
}
return new PartialListing(null, metadata.toArray(new FileMetadata[0]),
commonPrefixes.toArray(new String[0]));
}
@Override
public void delete(String key) throws IOException {
metadataMap.remove(key);
dataMap.remove(key);
}
@Override
public void copy(String srcKey, String dstKey) throws IOException {
metadataMap.put(dstKey, metadataMap.get(srcKey));
dataMap.put(dstKey, dataMap.get(srcKey));
}
@Override
public void purge(String prefix) throws IOException {
Iterator<Entry<String, FileMetadata>> i =
metadataMap.entrySet().iterator();
while (i.hasNext()) {
Entry<String, FileMetadata> entry = i.next();
if (entry.getKey().startsWith(prefix)) {
dataMap.remove(entry.getKey());
i.remove();
}
}
}
@Override
public void dump() throws IOException {
System.out.println(metadataMap.values());
System.out.println(dataMap.keySet());
}
}

View File

@ -1,266 +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.io.InputStream;
import java.net.URI;
import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystemContractBaseTest;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3native.NativeS3FileSystem.NativeS3FsInputStream;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.internal.AssumptionViolatedException;
import static org.junit.Assert.*;
public abstract class NativeS3FileSystemContractBaseTest
extends FileSystemContractBaseTest {
public static final String KEY_TEST_FS = "test.fs.s3n.name";
private NativeFileSystemStore store;
abstract NativeFileSystemStore getNativeFileSystemStore() throws IOException;
@Before
public void setUp() throws Exception {
Configuration conf = new Configuration();
String fsname = conf.get(KEY_TEST_FS);
if (StringUtils.isEmpty(fsname)) {
throw new AssumptionViolatedException(
"No test FS defined in :" + KEY_TEST_FS);
}
store = getNativeFileSystemStore();
fs = new NativeS3FileSystem(store);
fs.initialize(URI.create(fsname), conf);
}
@After
public void tearDown() throws Exception {
if (store != null) {
store.purge("test");
}
}
@Test
public void testCanonicalName() throws Exception {
assertNull("s3n doesn't support security token and shouldn't have canonical name",
fs.getCanonicalServiceName());
}
@Test
public void testListStatusForRoot() throws Exception {
FileStatus[] paths = fs.listStatus(path("/"));
assertEquals("Root directory is not empty; ", 0, paths.length);
Path testDir = path("/test");
assertTrue(fs.mkdirs(testDir));
paths = fs.listStatus(path("/"));
assertEquals(1, paths.length);
assertEquals(path("/test"), paths[0].getPath());
}
@Test
public void testNoTrailingBackslashOnBucket() throws Exception {
assertTrue(fs.getFileStatus(new Path(fs.getUri().toString())).isDirectory());
}
private void createTestFiles(String base) throws IOException {
store.storeEmptyFile(base + "/file1");
store.storeEmptyFile(base + "/dir/file2");
store.storeEmptyFile(base + "/dir/file3");
}
@Test
public void testDirWithDifferentMarkersWorks() throws Exception {
for (int i = 0; i <= 3; i++) {
String base = "test/hadoop" + i;
Path path = path("/" + base);
createTestFiles(base);
if (i == 0 ) {
//do nothing, we are testing correctness with no markers
}
else if (i == 1) {
// test for _$folder$ marker
store.storeEmptyFile(base + "_$folder$");
store.storeEmptyFile(base + "/dir_$folder$");
}
else if (i == 2) {
// test the end slash file marker
store.storeEmptyFile(base + "/");
store.storeEmptyFile(base + "/dir/");
}
else if (i == 3) {
// test both markers
store.storeEmptyFile(base + "_$folder$");
store.storeEmptyFile(base + "/dir_$folder$");
store.storeEmptyFile(base + "/");
store.storeEmptyFile(base + "/dir/");
}
assertTrue(fs.getFileStatus(path).isDirectory());
assertEquals(2, fs.listStatus(path).length);
}
}
@Test
public void testDeleteWithNoMarker() throws Exception {
String base = "test/hadoop";
Path path = path("/" + base);
createTestFiles(base);
fs.delete(path, true);
path = path("/test");
assertTrue(fs.getFileStatus(path).isDirectory());
assertEquals(0, fs.listStatus(path).length);
}
@Test
public void testRenameWithNoMarker() throws Exception {
String base = "test/hadoop";
Path dest = path("/test/hadoop2");
createTestFiles(base);
fs.rename(path("/" + base), dest);
Path path = path("/test");
assertTrue(fs.getFileStatus(path).isDirectory());
assertEquals(1, fs.listStatus(path).length);
assertTrue(fs.getFileStatus(dest).isDirectory());
assertEquals(2, fs.listStatus(dest).length);
}
@Test
public void testEmptyFile() throws Exception {
store.storeEmptyFile("test/hadoop/file1");
fs.open(path("/test/hadoop/file1")).close();
}
@Test
public void testBlockSize() throws Exception {
Path file = path("/test/hadoop/file");
createFile(file);
assertEquals("Default block size", fs.getDefaultBlockSize(file),
fs.getFileStatus(file).getBlockSize());
// Block size is determined at read time
long newBlockSize = fs.getDefaultBlockSize(file) * 2;
fs.getConf().setLong("fs.s3n.block.size", newBlockSize);
assertEquals("Double default block size", newBlockSize,
fs.getFileStatus(file).getBlockSize());
}
@Test
public void testRetryOnIoException() throws Exception {
class TestInputStream extends InputStream {
boolean shouldThrow = true;
int throwCount = 0;
int pos = 0;
byte[] bytes;
boolean threwException = false;
public TestInputStream() {
bytes = new byte[256];
for (int i = pos; i < 256; i++) {
bytes[i] = (byte)i;
}
}
@Override
public int read() throws IOException {
shouldThrow = !shouldThrow;
if (shouldThrow) {
throwCount++;
threwException = true;
throw new IOException();
}
assertFalse("IOException was thrown. InputStream should be reopened", threwException);
return pos++;
}
@Override
public int read(byte[] b, int off, int len) throws IOException {
shouldThrow = !shouldThrow;
if (shouldThrow) {
throwCount++;
threwException = true;
throw new IOException();
}
assertFalse("IOException was thrown. InputStream should be reopened", threwException);
int sizeToRead = Math.min(len, 256 - pos);
for (int i = 0; i < sizeToRead; i++) {
b[i] = bytes[pos + i];
}
pos += sizeToRead;
return sizeToRead;
}
public void reopenAt(long byteRangeStart) {
threwException = false;
pos = Long.valueOf(byteRangeStart).intValue();
}
}
final TestInputStream is = new TestInputStream();
class MockNativeFileSystemStore extends Jets3tNativeFileSystemStore {
@Override
public InputStream retrieve(String key, long byteRangeStart) throws IOException {
is.reopenAt(byteRangeStart);
return is;
}
}
NativeS3FsInputStream stream = new NativeS3FsInputStream(new MockNativeFileSystemStore(), null, is, "");
// Test reading methods.
byte[] result = new byte[256];
for (int i = 0; i < 128; i++) {
result[i] = (byte)stream.read();
}
for (int i = 128; i < 256; i += 8) {
byte[] temp = new byte[8];
int read = stream.read(temp, 0, 8);
assertEquals(8, read);
System.arraycopy(temp, 0, result, i, 8);
}
// Assert correct
for (int i = 0; i < 256; i++) {
assertEquals((byte)i, result[i]);
}
// Test to make sure the throw path was exercised.
// every read should have thrown 1 IOException except for the first read
// 144 = 128 - 1 + (128 / 8)
assertEquals(143, ((TestInputStream)is).throwCount);
}
}

View File

@ -1,32 +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.fs.s3native.NativeS3FileSystem;
import org.apache.hadoop.fs.s3native.InMemoryNativeFileSystemStore;
/**
* A helper implementation of {@link NativeS3FileSystem}
* without actually connecting to S3 for unit testing.
*/
public class S3NInMemoryFileSystem extends NativeS3FileSystem {
public S3NInMemoryFileSystem() {
super(new InMemoryNativeFileSystemStore());
}
}

View File

@ -1,129 +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.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.ProviderUtils;
import org.apache.hadoop.security.alias.CredentialProvider;
import org.apache.hadoop.security.alias.CredentialProviderFactory;
import java.io.File;
import java.net.URI;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import org.junit.rules.TestName;
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;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
/**
* This is to test the {@link S3Credentials} class for extracting AWS
* credentials.
*/
public class TestS3Credentials {
public static final Log LOG = LogFactory.getLog(TestS3Credentials.class);
@Rule
public final TestName test = new TestName();
@Before
public void announce() {
LOG.info("Running test " + test.getMethodName());
}
private static final String EXAMPLE_ID = "AKASOMEACCESSKEY";
private static final String EXAMPLE_KEY =
"RGV0cm9pdCBSZ/WQgY2xl/YW5lZCB1cAEXAMPLE";
@Test
public void testInvalidHostnameWithUnderscores() throws Exception {
S3Credentials s3Credentials = new S3Credentials();
try {
s3Credentials.initialize(new URI("s3n://a:b@c_d"), new Configuration());
fail("Should throw IllegalArgumentException");
} catch (IllegalArgumentException e) {
assertEquals("Invalid hostname in URI s3n://a:b@c_d", e.getMessage());
}
}
@Test
public void testPlaintextConfigPassword() throws Exception {
S3Credentials s3Credentials = new S3Credentials();
Configuration conf = new Configuration();
conf.set(S3_NATIVE_AWS_ACCESS_KEY_ID, EXAMPLE_ID);
conf.set(S3_NATIVE_AWS_SECRET_ACCESS_KEY, EXAMPLE_KEY);
s3Credentials.initialize(new URI("s3n://foobar"), conf);
assertEquals("Could not retrieve proper access key", EXAMPLE_ID,
s3Credentials.getAccessKey());
assertEquals("Could not retrieve proper secret", EXAMPLE_KEY,
s3Credentials.getSecretAccessKey());
}
@Test
public void testPlaintextConfigPasswordWithWhitespace() throws Exception {
S3Credentials s3Credentials = new S3Credentials();
Configuration conf = new Configuration();
conf.set(S3_NATIVE_AWS_ACCESS_KEY_ID, "\r\n " + EXAMPLE_ID +
" \r\n");
conf.set(S3_NATIVE_AWS_SECRET_ACCESS_KEY, "\r\n " + EXAMPLE_KEY +
" \r\n");
s3Credentials.initialize(new URI("s3n://foobar"), conf);
assertEquals("Could not retrieve proper access key", EXAMPLE_ID,
s3Credentials.getAccessKey());
assertEquals("Could not retrieve proper secret", EXAMPLE_KEY,
s3Credentials.getSecretAccessKey());
}
@Rule
public final TemporaryFolder tempDir = new TemporaryFolder();
@Test
public void testCredentialProvider() throws Exception {
// set up conf to have a cred provider
final Configuration conf = new Configuration();
final File file = tempDir.newFile("test.jks");
final URI jks = ProviderUtils.nestURIForLocalJavaKeyStoreProvider(
file.toURI());
conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH,
jks.toString());
// add our creds to the provider
final CredentialProvider provider =
CredentialProviderFactory.getProviders(conf).get(0);
provider.createCredentialEntry(S3_NATIVE_AWS_SECRET_ACCESS_KEY,
EXAMPLE_KEY.toCharArray());
provider.flush();
// make sure S3Creds can retrieve things.
S3Credentials s3Credentials = new S3Credentials();
conf.set(S3_NATIVE_AWS_ACCESS_KEY_ID, EXAMPLE_ID);
s3Credentials.initialize(new URI("s3n://foobar"), conf);
assertEquals("Could not retrieve proper access key", EXAMPLE_ID,
s3Credentials.getAccessKey());
assertEquals("Could not retrieve proper secret", EXAMPLE_KEY,
s3Credentials.getSecretAccessKey());
}
}

View File

@ -1,69 +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.BufferedReader;
import java.io.IOException;
import java.io.InputStreamReader;
import java.net.URI;
import junit.framework.TestCase;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
public class TestS3NInMemoryFileSystem extends TestCase {
private static final String TEST_PATH = "s3n://test/data.txt";
private static final String TEST_DATA = "Sample data for testing.";
private S3NInMemoryFileSystem fs;
@Override
public void setUp() throws IOException {
fs = new S3NInMemoryFileSystem();
fs.initialize(URI.create("s3n://test/"), new Configuration());
}
public void testBasicReadWriteIO() throws IOException {
FSDataOutputStream writeData = fs.create(new Path(TEST_PATH));
writeData.write(TEST_DATA.getBytes());
writeData.flush();
writeData.close();
FSDataInputStream readData = fs.open(new Path(TEST_PATH));
BufferedReader br = new BufferedReader(new InputStreamReader(readData));
String line = "";
StringBuffer stringBuffer = new StringBuffer();
while ((line = br.readLine()) != null) {
stringBuffer.append(line);
}
br.close();
assert(TEST_DATA.equals(stringBuffer.toString()));
}
@Override
public void tearDown() throws IOException {
fs.close();
}
}

View File

@ -1,110 +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.
-->
<configuration>
<!--
S3N is a blobstore, with very different behavior than a
classic filesystem.
-->
<property>
<name>fs.contract.test.root-tests-enabled</name>
<value>true</value>
</property>
<property>
<name>fs.contract.test.random-seek-count</name>
<value>10</value>
</property>
<property>
<name>fs.contract.is-blobstore</name>
<value>true</value>
</property>
<property>
<name>fs.contract.create-overwrites-directory</name>
<value>true</value>
</property>
<property>
<name>fs.contract.create-visibility-delayed</name>
<value>true</value>
</property>
<property>
<name>fs.contract.is-case-sensitive</name>
<value>true</value>
</property>
<property>
<name>fs.contract.rename-returns-false-if-source-missing</name>
<value>true</value>
</property>
<property>
<name>fs.contract.supports-append</name>
<value>false</value>
</property>
<property>
<name>fs.contract.supports-atomic-directory-delete</name>
<value>false</value>
</property>
<property>
<name>fs.contract.supports-atomic-rename</name>
<value>false</value>
</property>
<property>
<name>fs.contract.supports-block-locality</name>
<value>false</value>
</property>
<property>
<name>fs.contract.supports-concat</name>
<value>false</value>
</property>
<property>
<name>fs.contract.supports-seek</name>
<value>true</value>
</property>
<property>
<name>fs.contract.supports-seek-on-closed-file</name>
<value>true</value>
</property>
<property>
<name>fs.contract.rejects-seek-past-eof</name>
<value>true</value>
</property>
<property>
<name>fs.contract.supports-strict-exceptions</name>
<value>true</value>
</property>
<property>
<name>fs.contract.supports-unix-permissions</name>
<value>false</value>
</property>
</configuration>

View File

@ -32,3 +32,6 @@ log4j.logger.org.apache.hadoop.util.NativeCodeLoader=ERROR
# Log all HTTP requests made; includes S3 interaction. This may
# include sensitive information such as account IDs in HTTP headers.
#log4j.logger.com.amazonaws.request=DEBUG
# Turn on low level HTTP protocol debugging
#log4j.logger.com.amazonaws.thirdparty.apache.http=DEBUG