You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by cn...@apache.org on 2016/10/10 21:03:59 UTC
[1/7] hadoop git commit: HADOOP-13208. S3A listFiles(recursive=true)
to do a bulk listObjects instead of walking the pseudo-tree of directories.
Contributed by Steve Loughran.
Repository: hadoop
Updated Branches:
refs/heads/branch-2.8 def48f522 -> 67d8301e5
HADOOP-13208. S3A listFiles(recursive=true) to do a bulk listObjects instead of walking the pseudo-tree of directories. Contributed by Steve Loughran.
(cherry picked from commit 822d661b8fcc42bec6eea958d9fd02ef1aaa4b6c)
(cherry picked from commit 986162f9776837b1343073d721f89be8f3362085)
Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/d29dc397
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d29dc397
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d29dc397
Branch: refs/heads/branch-2.8
Commit: d29dc39791192b943631ab078f1078297b93a709
Parents: def48f5
Author: Chris Nauroth <cn...@apache.org>
Authored: Wed Aug 17 14:54:54 2016 -0700
Committer: Chris Nauroth <cn...@apache.org>
Committed: Mon Oct 10 14:01:07 2016 -0700
----------------------------------------------------------------------
.../java/org/apache/hadoop/fs/s3a/Listing.java | 594 +++++++++++++++++++
.../org/apache/hadoop/fs/s3a/S3AFileSystem.java | 372 +++++++-----
.../hadoop/fs/s3a/S3AInstrumentation.java | 1 +
.../java/org/apache/hadoop/fs/s3a/S3AUtils.java | 12 +
.../org/apache/hadoop/fs/s3a/Statistic.java | 2 +
.../s3a/TestS3AContractGetFileStatus.java | 16 +
.../org/apache/hadoop/fs/s3a/S3ATestUtils.java | 8 +-
.../s3a/scale/TestS3ADirectoryPerformance.java | 58 +-
8 files changed, 897 insertions(+), 166 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d29dc397/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java
new file mode 100644
index 0000000..4120b20
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java
@@ -0,0 +1,594 @@
+/*
+ * 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.ListObjectsRequest;
+import com.amazonaws.services.s3.model.ObjectListing;
+import com.amazonaws.services.s3.model.S3ObjectSummary;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.slf4j.Logger;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.NoSuchElementException;
+
+import static org.apache.hadoop.fs.s3a.Constants.S3N_FOLDER_SUFFIX;
+import static org.apache.hadoop.fs.s3a.S3AUtils.createFileStatus;
+import static org.apache.hadoop.fs.s3a.S3AUtils.objectRepresentsDirectory;
+import static org.apache.hadoop.fs.s3a.S3AUtils.stringify;
+import static org.apache.hadoop.fs.s3a.S3AUtils.translateException;
+
+/**
+ * Place for the S3A listing classes; keeps all the small classes under control.
+ */
+public class Listing {
+
+ private final S3AFileSystem owner;
+ private static final Logger LOG = S3AFileSystem.LOG;
+
+ public Listing(S3AFileSystem owner) {
+ this.owner = owner;
+ }
+
+ /**
+ * Create a FileStatus iterator against a path, with a given
+ * list object request.
+ * @param listPath path of the listing
+ * @param request initial request to make
+ * @param filter the filter on which paths to accept
+ * @param acceptor the class/predicate to decide which entries to accept
+ * in the listing based on the full file status.
+ * @return the iterator
+ * @throws IOException IO Problems
+ */
+ FileStatusListingIterator createFileStatusListingIterator(
+ Path listPath,
+ ListObjectsRequest request,
+ PathFilter filter,
+ Listing.FileStatusAcceptor acceptor) throws IOException {
+ return new FileStatusListingIterator(
+ new ObjectListingIterator(listPath, request),
+ filter,
+ acceptor);
+ }
+
+ /**
+ * Create a located status iterator over a file status iterator.
+ * @param statusIterator an iterator over the remote status entries
+ * @return a new remote iterator
+ */
+ LocatedFileStatusIterator createLocatedFileStatusIterator(
+ RemoteIterator<FileStatus> statusIterator) {
+ return new LocatedFileStatusIterator(statusIterator);
+ }
+
+ /**
+ * Interface to implement by the logic deciding whether to accept a summary
+ * entry or path as a valid file or directory.
+ */
+ interface FileStatusAcceptor {
+
+ /**
+ * Predicate to decide whether or not to accept a summary entry.
+ * @param keyPath qualified path to the entry
+ * @param summary summary entry
+ * @return true if the entry is accepted (i.e. that a status entry
+ * should be generated.
+ */
+ boolean accept(Path keyPath, S3ObjectSummary summary);
+
+ /**
+ * Predicate to decide whether or not to accept a prefix.
+ * @param keyPath qualified path to the entry
+ * @param commonPrefix the prefix
+ * @return true if the entry is accepted (i.e. that a status entry
+ * should be generated.)
+ */
+ boolean accept(Path keyPath, String commonPrefix);
+ }
+
+ /**
+ * A remote iterator which only iterates over a single `LocatedFileStatus`
+ * value.
+ *
+ * If the status value is null, the iterator declares that it has no data.
+ * This iterator is used to handle {@link listStatus()} calls where the path
+ * handed in refers to a file, not a directory: this is the iterator
+ * returned.
+ */
+ static final class SingleStatusRemoteIterator
+ implements RemoteIterator<LocatedFileStatus> {
+
+ /**
+ * The status to return; set to null after the first iteration.
+ */
+ private LocatedFileStatus status;
+
+ /**
+ * Constructor.
+ * @param status status value: may be null, in which case
+ * the iterator is empty.
+ */
+ public SingleStatusRemoteIterator(LocatedFileStatus status) {
+ this.status = status;
+ }
+
+ /**
+ * {@inheritDoc}
+ * @return true if there is a file status to return: this is always false
+ * for the second iteration, and may be false for the first.
+ * @throws IOException never
+ */
+ @Override
+ public boolean hasNext() throws IOException {
+ return status != null;
+ }
+
+ /**
+ * {@inheritDoc}
+ * @return the non-null status element passed in when the instance was
+ * constructed, if it ha not already been retrieved.
+ * @throws IOException never
+ * @throws NoSuchElementException if this is the second call, or it is
+ * the first call and a null {@link LocatedFileStatus} entry was passed
+ * to the constructor.
+ */
+ @Override
+ public LocatedFileStatus next() throws IOException {
+ if (hasNext()) {
+ LocatedFileStatus s = this.status;
+ status = null;
+ return s;
+ } else {
+ throw new NoSuchElementException();
+ }
+ }
+ }
+
+ /**
+ * Wraps up object listing into a remote iterator which will ask for more
+ * listing data if needed.
+ *
+ * This is a complex operation, especially the process to determine
+ * if there are more entries remaining. If there are no more results
+ * remaining in the (filtered) results of the current listing request, then
+ * another request is made <i>and those results filtered</i> before the
+ * iterator can declare that there is more data available.
+ *
+ * The need to filter the results precludes the iterator from simply
+ * declaring that if the {@link S3AFileSystem.ObjectListingIterator#hasNext()}
+ * is true then there are more results. Instead the next batch of results must
+ * be retrieved and filtered.
+ *
+ * What does this mean? It means that remote requests to retrieve new
+ * batches of object listings are made in the {@link #hasNext()} call;
+ * the {@link #next()} call simply returns the filtered results of the last
+ * listing processed. However, do note that {@link #next()} calls
+ * {@link #hasNext()} during its operation. This is critical to ensure
+ * that a listing obtained through a sequence of {@link #next()} will
+ * complete with the same set of results as a classic
+ * {@code while(it.hasNext()} loop.
+ *
+ * Thread safety: None.
+ */
+ class FileStatusListingIterator
+ implements RemoteIterator<FileStatus> {
+
+ /** Source of objects. */
+ private final ObjectListingIterator source;
+ /** Filter of paths from API call. */
+ private final PathFilter filter;
+ /** Filter of entries from file status. */
+ private final FileStatusAcceptor acceptor;
+ /** request batch size. */
+ private int batchSize;
+ /** Iterator over the current set of results. */
+ private ListIterator<FileStatus> statusBatchIterator;
+
+ /**
+ * Create an iterator over file status entries.
+ * @param source the listing iterator from a listObjects call.
+ * @param filter the filter on which paths to accept
+ * @param acceptor the class/predicate to decide which entries to accept
+ * in the listing based on the full file status.
+ * @throws IOException IO Problems
+ */
+ FileStatusListingIterator(ObjectListingIterator source,
+ PathFilter filter,
+ FileStatusAcceptor acceptor) throws IOException {
+ this.source = source;
+ this.filter = filter;
+ this.acceptor = acceptor;
+ // build the first set of results. This will not trigger any
+ // remote IO, assuming the source iterator is in its initial
+ // iteration
+ requestNextBatch();
+ }
+
+ /**
+ * Report whether or not there is new data available.
+ * If there is data in the local filtered list, return true.
+ * Else: request more data util that condition is met, or there
+ * is no more remote listing data.
+ * @return true if a call to {@link #next()} will succeed.
+ * @throws IOException
+ */
+ @Override
+ public boolean hasNext() throws IOException {
+ return statusBatchIterator.hasNext() || requestNextBatch();
+ }
+
+ @Override
+ public FileStatus next() throws IOException {
+ if (!hasNext()) {
+ throw new NoSuchElementException();
+ }
+ return statusBatchIterator.next();
+ }
+
+ /**
+ * Try to retrieve another batch.
+ * Note that for the initial batch,
+ * {@link S3AFileSystem.ObjectListingIterator} does not generate a request;
+ * it simply returns the initial set.
+ *
+ * @return true if a new batch was created.
+ * @throws IOException IO problems
+ */
+ private boolean requestNextBatch() throws IOException {
+ // look for more object listing batches being available
+ while (source.hasNext()) {
+ // if available, retrieve it and build the next status
+ if (buildNextStatusBatch(source.next())) {
+ // this batch successfully generated entries matching the filters/
+ // acceptors; declare that the request was successful
+ return true;
+ } else {
+ LOG.debug("All entries in batch were filtered...continuing");
+ }
+ }
+ // if this code is reached, it means that all remaining
+ // object lists have been retrieved, and there are no new entries
+ // to return.
+ return false;
+ }
+
+ /**
+ * Build the next status batch from a listing.
+ * @param objects the next object listing
+ * @return true if this added any entries after filtering
+ */
+ private boolean buildNextStatusBatch(ObjectListing objects) {
+ // counters for debug logs
+ int added = 0, ignored = 0;
+ // list to fill in with results. Initial size will be list maximum.
+ List<FileStatus> stats = new ArrayList<>(
+ objects.getObjectSummaries().size() +
+ objects.getCommonPrefixes().size());
+ // objects
+ for (S3ObjectSummary summary : objects.getObjectSummaries()) {
+ String key = summary.getKey();
+ Path keyPath = owner.keyToQualifiedPath(key);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("{}: {}", keyPath, stringify(summary));
+ }
+ // Skip over keys that are ourselves and old S3N _$folder$ files
+ if (acceptor.accept(keyPath, summary) && filter.accept(keyPath)) {
+ FileStatus status = createFileStatus(keyPath, summary,
+ owner.getDefaultBlockSize(keyPath));
+ LOG.debug("Adding: {}", status);
+ stats.add(status);
+ added++;
+ } else {
+ LOG.debug("Ignoring: {}", keyPath);
+ ignored++;
+ }
+ }
+
+ // prefixes: always directories
+ for (String prefix : objects.getCommonPrefixes()) {
+ Path keyPath = owner.keyToQualifiedPath(prefix);
+ if (acceptor.accept(keyPath, prefix) && filter.accept(keyPath)) {
+ FileStatus status = new S3AFileStatus(true, false, keyPath);
+ LOG.debug("Adding directory: {}", status);
+ added++;
+ stats.add(status);
+ } else {
+ LOG.debug("Ignoring directory: {}", keyPath);
+ ignored++;
+ }
+ }
+
+ // finish up
+ batchSize = stats.size();
+ statusBatchIterator = stats.listIterator();
+ boolean hasNext = statusBatchIterator.hasNext();
+ LOG.debug("Added {} entries; ignored {}; hasNext={}; hasMoreObjects={}",
+ added, ignored, hasNext, objects.isTruncated());
+ return hasNext;
+ }
+
+ /**
+ * Get the number of entries in the current batch.
+ * @return a number, possibly zero.
+ */
+ public int getBatchSize() {
+ return batchSize;
+ }
+ }
+
+ /**
+ * Wraps up AWS `ListObjects` requests in a remote iterator
+ * which will ask for more listing data if needed.
+ *
+ * That is:
+ *
+ * 1. The first invocation of the {@link #next()} call will return the results
+ * of the first request, the one created during the construction of the
+ * instance.
+ *
+ * 2. Second and later invocations will continue the ongoing listing,
+ * calling {@link #continueListObjects(ObjectListing)} to request the next
+ * batch of results.
+ *
+ * 3. The {@link #hasNext()} predicate returns true for the initial call,
+ * where {@link #next()} will return the initial results. It declares
+ * that it has future results iff the last executed request was truncated.
+ *
+ * Thread safety: none.
+ */
+ class ObjectListingIterator implements RemoteIterator<ObjectListing> {
+
+ /** The path listed. */
+ private final Path listPath;
+
+ /** The most recent listing results. */
+ private ObjectListing objects;
+
+ /** Indicator that this is the first listing. */
+ private boolean firstListing = true;
+
+ /**
+ * Count of how many listings have been requested
+ * (including initial result).
+ */
+ private int listingCount = 1;
+
+ /**
+ * Maximum keys in a request.
+ */
+ private int maxKeys;
+
+ /**
+ * Constructor -calls `listObjects()` on the request to populate the
+ * initial set of results/fail if there was a problem talking to the bucket.
+ * @param listPath path of the listing
+ * @param request initial request to make
+ * */
+ ObjectListingIterator(
+ Path listPath,
+ ListObjectsRequest request) {
+ this.listPath = listPath;
+ this.maxKeys = owner.getMaxKeys();
+ this.objects = owner.listObjects(request);
+ }
+
+ /**
+ * Declare that the iterator has data if it is either is the initial
+ * iteration or it is a later one and the last listing obtained was
+ * incomplete.
+ * @throws IOException never: there is no IO in this operation.
+ */
+ @Override
+ public boolean hasNext() throws IOException {
+ return firstListing || objects.isTruncated();
+ }
+
+ /**
+ * Ask for the next listing.
+ * For the first invocation, this returns the initial set, with no
+ * remote IO. For later requests, S3 will be queried, hence the calls
+ * may block or fail.
+ * @return the next object listing.
+ * @throws IOException if a query made of S3 fails.
+ * @throws NoSuchElementException if there is no more data to list.
+ */
+ @Override
+ public ObjectListing next() throws IOException {
+ if (firstListing) {
+ // on the first listing, don't request more data.
+ // Instead just clear the firstListing flag so that it future calls
+ // will request new data.
+ firstListing = false;
+ } else {
+ try {
+ if (!objects.isTruncated()) {
+ // nothing more to request: fail.
+ throw new NoSuchElementException("No more results in listing of "
+ + listPath);
+ }
+ // need to request a new set of objects.
+ LOG.debug("[{}], Requesting next {} objects under {}",
+ listingCount, maxKeys, listPath);
+ objects = owner.continueListObjects(objects);
+ listingCount++;
+ LOG.debug("New listing status: {}", this);
+ } catch (AmazonClientException e) {
+ throw translateException("listObjects()", listPath, e);
+ }
+ }
+ return objects;
+ }
+
+ @Override
+ public String toString() {
+ return "Object listing iterator against " + listPath
+ + "; listing count "+ listingCount
+ + "; isTruncated=" + objects.isTruncated();
+ }
+
+ /**
+ * Get the path listed.
+ * @return the path used in this listing.
+ */
+ public Path getListPath() {
+ return listPath;
+ }
+
+ /**
+ * Get the count of listing requests.
+ * @return the counter of requests made (including the initial lookup).
+ */
+ public int getListingCount() {
+ return listingCount;
+ }
+ }
+
+ /**
+ * Accept all entries except the base path and those which map to S3N
+ * pseudo directory markers.
+ */
+ static class AcceptFilesOnly implements FileStatusAcceptor {
+ private final Path qualifiedPath;
+
+ public AcceptFilesOnly(Path qualifiedPath) {
+ this.qualifiedPath = qualifiedPath;
+ }
+
+ /**
+ * Reject a summary entry if the key path is the qualified Path, or
+ * it ends with {@code "_$folder$"}.
+ * @param keyPath key path of the entry
+ * @param summary summary entry
+ * @return true if the entry is accepted (i.e. that a status entry
+ * should be generated.
+ */
+ @Override
+ public boolean accept(Path keyPath, S3ObjectSummary summary) {
+ return !keyPath.equals(qualifiedPath)
+ && !summary.getKey().endsWith(S3N_FOLDER_SUFFIX)
+ && !objectRepresentsDirectory(summary.getKey(), summary.getSize());
+ }
+
+ /**
+ * Accept no directory paths.
+ * @param keyPath qualified path to the entry
+ * @param prefix common prefix in listing.
+ * @return false, always.
+ */
+ @Override
+ public boolean accept(Path keyPath, String prefix) {
+ return false;
+ }
+ }
+
+ /**
+ * Take a remote iterator over a set of {@link FileStatus} instances and
+ * return a remote iterator of {@link LocatedFileStatus} instances.
+ */
+ class LocatedFileStatusIterator
+ implements RemoteIterator<LocatedFileStatus> {
+ private final RemoteIterator<FileStatus> statusIterator;
+
+ /**
+ * Constructor.
+ * @param statusIterator an iterator over the remote status entries
+ */
+ LocatedFileStatusIterator(RemoteIterator<FileStatus> statusIterator) {
+ this.statusIterator = statusIterator;
+ }
+
+ @Override
+ public boolean hasNext() throws IOException {
+ return statusIterator.hasNext();
+ }
+
+ @Override
+ public LocatedFileStatus next() throws IOException {
+ return owner.toLocatedFileStatus(statusIterator.next());
+ }
+ }
+
+ /**
+ * Accept all entries except the base path and those which map to S3N
+ * pseudo directory markers.
+ */
+ static class AcceptAllButSelfAndS3nDirs implements FileStatusAcceptor {
+
+ /** Base path. */
+ private final Path qualifiedPath;
+
+ /**
+ * Constructor.
+ * @param qualifiedPath an already-qualified path.
+ */
+ public AcceptAllButSelfAndS3nDirs(Path qualifiedPath) {
+ this.qualifiedPath = qualifiedPath;
+ }
+
+ /**
+ * Reject a summary entry if the key path is the qualified Path, or
+ * it ends with {@code "_$folder$"}.
+ * @param keyPath key path of the entry
+ * @param summary summary entry
+ * @return true if the entry is accepted (i.e. that a status entry
+ * should be generated.)
+ */
+ @Override
+ public boolean accept(Path keyPath, S3ObjectSummary summary) {
+ return !keyPath.equals(qualifiedPath) &&
+ !summary.getKey().endsWith(S3N_FOLDER_SUFFIX);
+ }
+
+ /**
+ * Accept all prefixes except the one for the base path, "self".
+ * @param keyPath qualified path to the entry
+ * @param prefix common prefix in listing.
+ * @return true if the entry is accepted (i.e. that a status entry
+ * should be generated.
+ */
+ @Override
+ public boolean accept(Path keyPath, String prefix) {
+ return !keyPath.equals(qualifiedPath);
+ }
+ }
+
+ /**
+ * A Path filter which accepts all filenames.
+ */
+ static final PathFilter ACCEPT_ALL = new PathFilter() {
+ @Override
+ public boolean accept(Path file) {
+ return true;
+ }
+
+ @Override
+ public String toString() {
+ return "ACCEPT_ALL";
+ }
+ };
+
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d29dc397/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
index 2ad0431..24a8d64 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
@@ -87,6 +87,7 @@ import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.VersionInfo;
import static org.apache.hadoop.fs.s3a.Constants.*;
+import static org.apache.hadoop.fs.s3a.Listing.ACCEPT_ALL;
import static org.apache.hadoop.fs.s3a.S3AUtils.*;
import static org.apache.hadoop.fs.s3a.Statistic.*;
@@ -118,6 +119,7 @@ public class S3AFileSystem extends FileSystem {
private AmazonS3Client s3;
private String bucket;
private int maxKeys;
+ private Listing listing;
private long partSize;
private boolean enableMultiObjectsDelete;
private TransferManager transfers;
@@ -187,6 +189,7 @@ public class S3AFileSystem extends FileSystem {
initAmazonS3Client(conf, credentials, awsConf);
maxKeys = intOption(conf, MAX_PAGING_KEYS, DEFAULT_MAX_PAGING_KEYS, 1);
+ listing = new Listing(this);
partSize = conf.getLong(MULTIPART_SIZE, DEFAULT_MULTIPART_SIZE);
if (partSize < 5 * 1024 * 1024) {
LOG.error(MULTIPART_SIZE + " must be at least 5 MB");
@@ -476,7 +479,11 @@ public class S3AFileSystem extends FileSystem {
super();
}
- /* Turns a path (relative or otherwise) into an S3 key
+ /**
+ * Turns a path (relative or otherwise) into an S3 key.
+ *
+ * @param path input path, may be relative to the working dir
+ * @return a key excluding the leading "/", or, if it is the root path, ""
*/
private String pathToKey(Path path) {
if (!path.isAbsolute()) {
@@ -490,11 +497,50 @@ public class S3AFileSystem extends FileSystem {
return path.toUri().getPath().substring(1);
}
+ /**
+ * Turns a path (relative or otherwise) into an S3 key, adding a trailing
+ * "/" if the path is not the root <i>and</i> does not already have a "/"
+ * at the end.
+ *
+ * @param key s3 key or ""
+ * @return the with a trailing "/", or, if it is the root key, "",
+ */
+ private String maybeAddTrailingSlash(String key) {
+ if (!key.isEmpty() && !key.endsWith("/")) {
+ return key + '/';
+ } else {
+ return key;
+ }
+ }
+
+ /**
+ * Convert a path back to a key.
+ * @param key input key
+ * @return the path from this key
+ */
private Path keyToPath(String key) {
return new Path("/" + key);
}
/**
+ * Convert a key to a fully qualified path.
+ * @param key input key
+ * @return the fully qualified path including URI scheme and bucket name.
+ */
+ Path keyToQualifiedPath(String key) {
+ return qualify(keyToPath(key));
+ }
+
+ /**
+ * Qualify a path.
+ * @param path path to qualify
+ * @return a qualified path.
+ */
+ Path qualify(Path path) {
+ return path.makeQualified(uri, workingDir);
+ }
+
+ /**
* Check that a Path belongs to this FileSystem.
* Unlike the superclass, this version does not look at authority,
* only hostnames.
@@ -512,10 +558,10 @@ public class S3AFileSystem extends FileSystem {
}
/**
- * Opens an FSDataInputStream at the indicated Path.
- * @param f the file name to open
- * @param bufferSize the size of the buffer to be used.
- */
+ * Opens an FSDataInputStream at the indicated Path.
+ * @param f the file name to open
+ * @param bufferSize the size of the buffer to be used.
+ */
public FSDataInputStream open(Path f, int bufferSize)
throws IOException {
@@ -862,7 +908,7 @@ public class S3AFileSystem extends FileSystem {
* @return the next result object
*/
protected ObjectListing continueListObjects(ObjectListing objects) {
- incrementStatistic(OBJECT_LIST_REQUESTS);
+ incrementStatistic(OBJECT_CONTINUE_LIST_REQUESTS);
incrementReadOperations();
return s3.listNextBatchOfObjects(objects);
}
@@ -1146,12 +1192,7 @@ public class S3AFileSystem extends FileSystem {
} else {
LOG.debug("Getting objects for directory prefix {} to delete", key);
- ListObjectsRequest request = new ListObjectsRequest();
- request.setBucketName(bucket);
- request.setPrefix(key);
- // Hopefully not setting a delimiter will cause this to find everything
- //request.setDelimiter("/");
- request.setMaxKeys(maxKeys);
+ ListObjectsRequest request = createListObjectsRequest(key, null);
ObjectListing objects = listObjects(request);
List<DeleteObjectsRequest.KeyVersion> keys =
@@ -1254,66 +1295,57 @@ public class S3AFileSystem extends FileSystem {
*/
public FileStatus[] innerListStatus(Path f) throws FileNotFoundException,
IOException, AmazonClientException {
- String key = pathToKey(f);
- LOG.debug("List status for path: {}", f);
+ Path path = qualify(f);
+ String key = pathToKey(path);
+ LOG.debug("List status for path: {}", path);
incrementStatistic(INVOCATION_LIST_STATUS);
- final List<FileStatus> result = new ArrayList<FileStatus>();
- final FileStatus fileStatus = getFileStatus(f);
+ List<FileStatus> result;
+ final FileStatus fileStatus = getFileStatus(path);
if (fileStatus.isDirectory()) {
if (!key.isEmpty()) {
- key = key + "/";
+ key = key + '/';
}
- ListObjectsRequest request = new ListObjectsRequest();
- request.setBucketName(bucket);
- request.setPrefix(key);
- request.setDelimiter("/");
- request.setMaxKeys(maxKeys);
-
+ ListObjectsRequest request = createListObjectsRequest(key, "/");
LOG.debug("listStatus: doing listObjects for directory {}", key);
- ObjectListing objects = listObjects(request);
-
- Path fQualified = f.makeQualified(uri, workingDir);
-
- while (true) {
- for (S3ObjectSummary summary : objects.getObjectSummaries()) {
- Path keyPath = keyToPath(summary.getKey()).makeQualified(uri, workingDir);
- // Skip over keys that are ourselves and old S3N _$folder$ files
- if (keyPath.equals(fQualified) ||
- summary.getKey().endsWith(S3N_FOLDER_SUFFIX)) {
- LOG.debug("Ignoring: {}", keyPath);
- } else {
- S3AFileStatus status = createFileStatus(keyPath, summary,
- getDefaultBlockSize(keyPath));
- result.add(status);
- LOG.debug("Adding: {}", status);
- }
- }
-
- for (String prefix : objects.getCommonPrefixes()) {
- Path keyPath = keyToPath(prefix).makeQualified(uri, workingDir);
- if (!keyPath.equals(f)) {
- result.add(new S3AFileStatus(true, false, keyPath));
- LOG.debug("Adding: rd: {}", keyPath);
- }
- }
-
- if (objects.isTruncated()) {
- LOG.debug("listStatus: list truncated - getting next batch");
- objects = continueListObjects(objects);
- } else {
- break;
- }
+ Listing.FileStatusListingIterator files =
+ listing.createFileStatusListingIterator(path,
+ request,
+ ACCEPT_ALL,
+ new Listing.AcceptAllButSelfAndS3nDirs(path));
+ result = new ArrayList<>(files.getBatchSize());
+ while (files.hasNext()) {
+ result.add(files.next());
}
+ return result.toArray(new FileStatus[result.size()]);
} else {
- LOG.debug("Adding: rd (not a dir): {}", f);
- result.add(fileStatus);
+ LOG.debug("Adding: rd (not a dir): {}", path);
+ FileStatus[] stats = new FileStatus[1];
+ stats[0]= fileStatus;
+ return stats;
}
+ }
- return result.toArray(new FileStatus[result.size()]);
+ /**
+ * Create a {@code ListObjectsRequest} request against this bucket,
+ * with the maximum keys returned in a query set by {@link #maxKeys}.
+ * @param key key for request
+ * @param delimiter any delimiter
+ * @return the request
+ */
+ private ListObjectsRequest createListObjectsRequest(String key,
+ String delimiter) {
+ ListObjectsRequest request = new ListObjectsRequest();
+ request.setBucketName(bucket);
+ request.setMaxKeys(maxKeys);
+ request.setPrefix(key);
+ if (delimiter != null) {
+ request.setDelimiter(delimiter);
+ }
+ return request;
}
/**
@@ -1414,11 +1446,11 @@ public class S3AFileSystem extends FileSystem {
* @throws java.io.FileNotFoundException when the path does not exist;
* @throws IOException on other problems.
*/
- public S3AFileStatus getFileStatus(Path f) throws IOException {
- String key = pathToKey(f);
+ public S3AFileStatus getFileStatus(final Path f) throws IOException {
incrementStatistic(INVOCATION_GET_FILE_STATUS);
- LOG.debug("Getting path status for {} ({})", f , key);
-
+ final Path path = qualify(f);
+ String key = pathToKey(path);
+ LOG.debug("Getting path status for {} ({})", path , key);
if (!key.isEmpty()) {
try {
ObjectMetadata meta = getObjectMetadata(key);
@@ -1426,20 +1458,20 @@ public class S3AFileSystem extends FileSystem {
if (objectRepresentsDirectory(key, meta.getContentLength())) {
LOG.debug("Found exact file: fake directory");
return new S3AFileStatus(true, true,
- f.makeQualified(uri, workingDir));
+ path);
} else {
LOG.debug("Found exact file: normal file");
return new S3AFileStatus(meta.getContentLength(),
dateToLong(meta.getLastModified()),
- f.makeQualified(uri, workingDir),
- getDefaultBlockSize(f.makeQualified(uri, workingDir)));
+ path,
+ getDefaultBlockSize(path));
}
} catch (AmazonServiceException e) {
if (e.getStatusCode() != 404) {
- throw translateException("getFileStatus", f, e);
+ throw translateException("getFileStatus", path, e);
}
} catch (AmazonClientException e) {
- throw translateException("getFileStatus", f, e);
+ throw translateException("getFileStatus", path, e);
}
// Necessary?
@@ -1450,14 +1482,14 @@ public class S3AFileSystem extends FileSystem {
if (objectRepresentsDirectory(newKey, meta.getContentLength())) {
LOG.debug("Found file (with /): fake directory");
- return new S3AFileStatus(true, true, f.makeQualified(uri, workingDir));
+ return new S3AFileStatus(true, true, path);
} else {
LOG.warn("Found file (with /): real file? should not happen: {}", key);
return new S3AFileStatus(meta.getContentLength(),
dateToLong(meta.getLastModified()),
- f.makeQualified(uri, workingDir),
- getDefaultBlockSize(f.makeQualified(uri, workingDir)));
+ path,
+ getDefaultBlockSize(path));
}
} catch (AmazonServiceException e) {
if (e.getStatusCode() != 404) {
@@ -1470,9 +1502,7 @@ public class S3AFileSystem extends FileSystem {
}
try {
- if (!key.isEmpty() && !key.endsWith("/")) {
- key = key + "/";
- }
+ key = maybeAddTrailingSlash(key);
ListObjectsRequest request = new ListObjectsRequest();
request.setBucketName(bucket);
request.setPrefix(key);
@@ -1496,11 +1526,10 @@ public class S3AFileSystem extends FileSystem {
}
}
- return new S3AFileStatus(true, false,
- f.makeQualified(uri, workingDir));
+ return new S3AFileStatus(true, false, path);
} else if (key.isEmpty()) {
LOG.debug("Found root directory");
- return new S3AFileStatus(true, true, f.makeQualified(uri, workingDir));
+ return new S3AFileStatus(true, true, path);
}
} catch (AmazonServiceException e) {
if (e.getStatusCode() != 404) {
@@ -1510,8 +1539,8 @@ public class S3AFileSystem extends FileSystem {
throw translateException("getFileStatus", key, e);
}
- LOG.debug("Not Found: {}", f);
- throw new FileNotFoundException("No such file or directory: " + f);
+ LOG.debug("Not Found: {}", path);
+ throw new FileNotFoundException("No such file or directory: " + path);
}
/**
@@ -1869,7 +1898,15 @@ public class S3AFileSystem extends FileSystem {
}
/**
- * Override superclass so as to add statistic collection.
+ * Get the maximum key count.
+ * @return a value, valid after initialization
+ */
+ int getMaxKeys() {
+ return maxKeys;
+ }
+
+ /**
+ * Increments the statistic {@link Statistic#INVOCATION_GLOB_STATUS}.
* {@inheritDoc}
*/
@Override
@@ -1894,24 +1931,6 @@ public class S3AFileSystem extends FileSystem {
* {@inheritDoc}
*/
@Override
- public RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f)
- throws FileNotFoundException, IOException {
- incrementStatistic(INVOCATION_LIST_LOCATED_STATUS);
- return super.listLocatedStatus(f);
- }
-
- @Override
- public RemoteIterator<LocatedFileStatus> listFiles(Path f,
- boolean recursive) throws FileNotFoundException, IOException {
- incrementStatistic(INVOCATION_LIST_FILES);
- return super.listFiles(f, recursive);
- }
-
- /**
- * Override superclass so as to add statistic collection.
- * {@inheritDoc}
- */
- @Override
public boolean exists(Path f) throws IOException {
incrementStatistic(INVOCATION_EXISTS);
return super.exists(f);
@@ -1938,6 +1957,129 @@ public class S3AFileSystem extends FileSystem {
}
/**
+ * {@inheritDoc}.
+ *
+ * This implementation is optimized for S3, which can do a bulk listing
+ * off all entries under a path in one single operation. Thus there is
+ * no need to recursively walk the directory tree.
+ *
+ * Instead a {@link ListObjectsRequest} is created requesting a (windowed)
+ * listing of all entries under the given path. This is used to construct
+ * an {@code ObjectListingIterator} instance, iteratively returning the
+ * sequence of lists of elements under the path. This is then iterated
+ * over in a {@code FileStatusListingIterator}, which generates
+ * {@link S3AFileStatus} instances, one per listing entry.
+ * These are then translated into {@link LocatedFileStatus} instances.
+ *
+ * This is essentially a nested and wrapped set of iterators, with some
+ * generator classes; an architecture which may become less convoluted
+ * using lambda-expressions.
+ * @param f a path
+ * @param recursive if the subdirectories need to be traversed recursively
+ *
+ * @return an iterator that traverses statuses of the files/directories
+ * in the given path
+ * @throws FileNotFoundException if {@code path} does not exist
+ * @throws IOException if any I/O error occurred
+ */
+ @Override
+ public RemoteIterator<LocatedFileStatus> listFiles(Path f,
+ boolean recursive) throws FileNotFoundException, IOException {
+ incrementStatistic(INVOCATION_LIST_FILES);
+ Path path = qualify(f);
+ LOG.debug("listFiles({}, {})", path, recursive);
+ try {
+ // lookup dir triggers existence check
+ final FileStatus fileStatus = getFileStatus(path);
+ if (fileStatus.isFile()) {
+ // simple case: File
+ LOG.debug("Path is a file");
+ return new Listing.SingleStatusRemoteIterator(
+ toLocatedFileStatus(fileStatus));
+ } else {
+ // directory: do a bulk operation
+ String key = maybeAddTrailingSlash(pathToKey(path));
+ String delimiter = recursive ? null : "/";
+ LOG.debug("Requesting all entries under {} with delimiter '{}'",
+ key, delimiter);
+ return listing.createLocatedFileStatusIterator(
+ listing.createFileStatusListingIterator(path,
+ createListObjectsRequest(key, delimiter),
+ ACCEPT_ALL,
+ new Listing.AcceptFilesOnly(path)));
+ }
+ } catch (AmazonClientException e) {
+ throw translateException("listFiles", path, e);
+ }
+ }
+
+ /**
+ * Override superclass so as to add statistic collection.
+ * {@inheritDoc}
+ */
+ @Override
+ public RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f)
+ throws FileNotFoundException, IOException {
+ return listLocatedStatus(f, ACCEPT_ALL);
+ }
+
+ /**
+ * {@inheritDoc}.
+ *
+ * S3 Optimized directory listing. The initial operation performs the
+ * first bulk listing; extra listings will take place
+ * when all the current set of results are used up.
+ * @param f a path
+ * @param filter a path filter
+ * @return an iterator that traverses statuses of the files/directories
+ * in the given path
+ * @throws FileNotFoundException if {@code path} does not exist
+ * @throws IOException if any I/O error occurred
+ */
+ @Override
+ public RemoteIterator<LocatedFileStatus> listLocatedStatus(final Path f,
+ final PathFilter filter)
+ throws FileNotFoundException, IOException {
+ incrementStatistic(INVOCATION_LIST_LOCATED_STATUS);
+ Path path = qualify(f);
+ LOG.debug("listLocatedStatus({}, {}", path, filter);
+ try {
+ // lookup dir triggers existence check
+ final FileStatus fileStatus = getFileStatus(path);
+ if (fileStatus.isFile()) {
+ // simple case: File
+ LOG.debug("Path is a file");
+ return new Listing.SingleStatusRemoteIterator(
+ filter.accept(path) ? toLocatedFileStatus(fileStatus) : null);
+ } else {
+ // directory: trigger a lookup
+ String key = maybeAddTrailingSlash(pathToKey(path));
+ return listing.createLocatedFileStatusIterator(
+ listing.createFileStatusListingIterator(path,
+ createListObjectsRequest(key, "/"),
+ filter,
+ new Listing.AcceptAllButSelfAndS3nDirs(path)));
+ }
+ } catch (AmazonClientException e) {
+ throw translateException("listLocatedStatus", path, e);
+ }
+ }
+
+ /**
+ * Build a {@link LocatedFileStatus} from a {@link FileStatus} instance.
+ * @param status file status
+ * @return a located status with block locations set up from this FS.
+ * @throws IOException IO Problems.
+ */
+ LocatedFileStatus toLocatedFileStatus(FileStatus status)
+ throws IOException {
+ return new LocatedFileStatus(status,
+ status.isFile() ?
+ getFileBlockLocations(status, 0, status.getLen())
+ : null);
+ }
+
+ /**
* Get a integer option >= the minimum allowed value.
* @param conf configuration
* @param key key to look up
@@ -1974,38 +2116,4 @@ public class S3AFileSystem extends FileSystem {
return v;
}
- /**
- * This is a simple encapsulation of the
- * S3 access key and secret.
- */
- static class AWSAccessKeys {
- private String accessKey = null;
- private String accessSecret = null;
-
- /**
- * Constructor.
- * @param key - AWS access key
- * @param secret - AWS secret key
- */
- public AWSAccessKeys(String key, String secret) {
- accessKey = key;
- accessSecret = secret;
- }
-
- /**
- * Return the AWS access key.
- * @return key
- */
- public String getAccessKey() {
- return accessKey;
- }
-
- /**
- * Return the AWS secret key.
- * @return secret
- */
- public String getAccessSecret() {
- return accessSecret;
- }
- }
}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d29dc397/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
index 20c77c2..26b5b51 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
@@ -96,6 +96,7 @@ public class S3AInstrumentation {
OBJECT_COPY_REQUESTS,
OBJECT_DELETE_REQUESTS,
OBJECT_LIST_REQUESTS,
+ OBJECT_CONTINUE_LIST_REQUESTS,
OBJECT_METADATA_REQUESTS,
OBJECT_MULTIPART_UPLOAD_ABORTED,
OBJECT_PUT_BYTES,
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d29dc397/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
index 4b7db39..40d0b1b 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
@@ -408,4 +408,16 @@ public final class S3AUtils {
throw new IOException("Cannot find password option " + key, ioe);
}
}
+
+ /**
+ * String information about a summary entry for debug messages.
+ * @param summary summary object
+ * @return string value
+ */
+ public static String stringify(S3ObjectSummary summary) {
+ StringBuilder builder = new StringBuilder(summary.getKey().length() + 100);
+ builder.append(summary.getKey()).append(' ');
+ builder.append("size=").append(summary.getSize());
+ return builder.toString();
+ }
}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d29dc397/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
index 67b2e80..d84a355 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
@@ -73,6 +73,8 @@ public enum Statistic {
OBJECT_DELETE_REQUESTS("object_delete_requests", "Object delete requests"),
OBJECT_LIST_REQUESTS("object_list_requests",
"Number of object listings made"),
+ OBJECT_CONTINUE_LIST_REQUESTS("object_continue_list_requests",
+ "Number of continued object listings made"),
OBJECT_METADATA_REQUESTS("object_metadata_requests",
"Number of requests for object metadata"),
OBJECT_MULTIPART_UPLOAD_ABORTED("object_multipart_aborted",
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d29dc397/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractGetFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractGetFileStatus.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractGetFileStatus.java
index d7b8fe3..5937d49 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractGetFileStatus.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractGetFileStatus.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.fs.contract.s3a;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.contract.AbstractFSContract;
import org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.apache.hadoop.fs.s3a.S3ATestUtils;
public class TestS3AContractGetFileStatus extends AbstractContractGetFileStatusTest {
@@ -28,4 +30,18 @@ public class TestS3AContractGetFileStatus extends AbstractContractGetFileStatusT
return new S3AContract(conf);
}
+ @Override
+ public void teardown() throws Exception {
+ getLog().info("FS details {}", getFileSystem());
+ super.teardown();
+ }
+
+ @Override
+ protected Configuration createConfiguration() {
+ Configuration conf = super.createConfiguration();
+ S3ATestUtils.disableFilesystemCaching(conf);
+ // aggressively low page size forces tests to go multipage
+ conf.setInt(Constants.MAX_PAGING_KEYS, 2);
+ return conf;
+ }
}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d29dc397/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
index 8f484d8..95f6d4b 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
@@ -124,7 +124,7 @@ public class S3ATestUtils {
try {
callback.call();
return;
- } catch (FailFastException e) {
+ } catch (InterruptedException | FailFastException e) {
throw e;
} catch (Exception e) {
lastException = e;
@@ -330,7 +330,7 @@ public class S3ATestUtils {
* @return true if the value is {@code ==} the other's
*/
public boolean diffEquals(MetricDiff that) {
- return this.currentValue() == that.currentValue();
+ return this.diff() == that.diff();
}
/**
@@ -339,7 +339,7 @@ public class S3ATestUtils {
* @return true if the value is {@code <} the other's
*/
public boolean diffLessThan(MetricDiff that) {
- return this.currentValue() < that.currentValue();
+ return this.diff() < that.diff();
}
/**
@@ -348,7 +348,7 @@ public class S3ATestUtils {
* @return true if the value is {@code <=} the other's
*/
public boolean diffLessThanOrEquals(MetricDiff that) {
- return this.currentValue() <= that.currentValue();
+ return this.diff() <= that.diff();
}
/**
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d29dc397/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3ADirectoryPerformance.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3ADirectoryPerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3ADirectoryPerformance.java
index 7ece394..35ea3ad 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3ADirectoryPerformance.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3ADirectoryPerformance.java
@@ -18,7 +18,6 @@
package org.apache.hadoop.fs.s3a.scale;
-import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.Statistic;
import org.junit.Test;
@@ -51,6 +50,8 @@ public class TestS3ADirectoryPerformance extends S3AScaleTestBase {
int files = scale;
MetricDiff metadataRequests = new MetricDiff(fs, OBJECT_METADATA_REQUESTS);
MetricDiff listRequests = new MetricDiff(fs, OBJECT_LIST_REQUESTS);
+ MetricDiff listContinueRequests =
+ new MetricDiff(fs, OBJECT_CONTINUE_LIST_REQUESTS);
MetricDiff listStatusCalls = new MetricDiff(fs, INVOCATION_LIST_FILES);
MetricDiff getFileStatusCalls =
new MetricDiff(fs, INVOCATION_GET_FILE_STATUS);
@@ -69,34 +70,29 @@ public class TestS3ADirectoryPerformance extends S3AScaleTestBase {
printThenReset(LOG,
metadataRequests,
listRequests,
+ listContinueRequests,
listStatusCalls,
getFileStatusCalls);
+ describe("Listing files via treewalk");
try {
// Scan the directory via an explicit tree walk.
// This is the baseline for any listing speedups.
- MetricDiff treewalkMetadataRequests =
- new MetricDiff(fs, OBJECT_METADATA_REQUESTS);
- MetricDiff treewalkListRequests = new MetricDiff(fs,
- OBJECT_LIST_REQUESTS);
- MetricDiff treewalkListStatusCalls = new MetricDiff(fs,
- INVOCATION_LIST_FILES);
- MetricDiff treewalkGetFileStatusCalls =
- new MetricDiff(fs, INVOCATION_GET_FILE_STATUS);
NanoTimer treeWalkTimer = new NanoTimer();
TreeScanResults treewalkResults = treeWalk(fs, listDir);
- treeWalkTimer.end("List status via treewalk");
+ treeWalkTimer.end("List status via treewalk of %s", created);
- print(LOG,
- treewalkMetadataRequests,
- treewalkListRequests,
- treewalkListStatusCalls,
- treewalkGetFileStatusCalls);
+ printThenReset(LOG,
+ metadataRequests,
+ listRequests,
+ listContinueRequests,
+ listStatusCalls,
+ getFileStatusCalls);
assertEquals("Files found in listFiles(recursive=true) " +
" created=" + created + " listed=" + treewalkResults,
created.getFileCount(), treewalkResults.getFileCount());
-
+ describe("Listing files via listFiles(recursive=true)");
// listFiles() does the recursion internally
NanoTimer listFilesRecursiveTimer = new NanoTimer();
@@ -108,31 +104,33 @@ public class TestS3ADirectoryPerformance extends S3AScaleTestBase {
" created=" + created + " listed=" + listFilesResults,
created.getFileCount(), listFilesResults.getFileCount());
- treewalkListRequests.assertDiffEquals(listRequests);
- printThenReset(LOG,
- metadataRequests, listRequests,
- listStatusCalls, getFileStatusCalls);
-
- NanoTimer globStatusTimer = new NanoTimer();
- FileStatus[] globStatusFiles = fs.globStatus(listDir);
- globStatusTimer.end("Time to globStatus() %s", globStatusTimer);
- LOG.info("Time for glob status {} entries: {}",
- globStatusFiles.length,
- toHuman(createTimer.duration()));
- printThenReset(LOG,
+ // only two list operations should have taken place
+ print(LOG,
metadataRequests,
listRequests,
+ listContinueRequests,
listStatusCalls,
getFileStatusCalls);
+ assertEquals(listRequests.toString(), 2, listRequests.diff());
+ reset(metadataRequests,
+ listRequests,
+ listContinueRequests,
+ listStatusCalls,
+ getFileStatusCalls);
+
} finally {
+ describe("deletion");
// deletion at the end of the run
NanoTimer deleteTimer = new NanoTimer();
fs.delete(listDir, true);
deleteTimer.end("Deleting directory tree");
printThenReset(LOG,
- metadataRequests, listRequests,
- listStatusCalls, getFileStatusCalls);
+ metadataRequests,
+ listRequests,
+ listContinueRequests,
+ listStatusCalls,
+ getFileStatusCalls);
}
}
---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org
[5/7] hadoop git commit: HADOOP-13446. Support running isolated unit
tests separate from AWS integration tests. Contributed by Chris Nauroth.
Posted by cn...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractCreate.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractCreate.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractCreate.java
new file mode 100644
index 0000000..502cf5a
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractCreate.java
@@ -0,0 +1,41 @@
+/*
+ * 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");
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractDelete.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractDelete.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractDelete.java
new file mode 100644
index 0000000..675f979
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractDelete.java
@@ -0,0 +1,34 @@
+/*
+ * 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);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractMkdir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractMkdir.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractMkdir.java
new file mode 100644
index 0000000..3c566f3
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractMkdir.java
@@ -0,0 +1,34 @@
+/*
+ * 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);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractOpen.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractOpen.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractOpen.java
new file mode 100644
index 0000000..7ebfc4e
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractOpen.java
@@ -0,0 +1,34 @@
+/*
+ * 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);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractRename.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractRename.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractRename.java
new file mode 100644
index 0000000..effe9eb
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractRename.java
@@ -0,0 +1,35 @@
+/*
+ * 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);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractRootDir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractRootDir.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractRootDir.java
new file mode 100644
index 0000000..3fdf868
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractRootDir.java
@@ -0,0 +1,35 @@
+/*
+ * 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);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractSeek.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractSeek.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractSeek.java
new file mode 100644
index 0000000..9e1ce73
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractSeek.java
@@ -0,0 +1,34 @@
+/*
+ * 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);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/TestS3NContractCreate.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/TestS3NContractCreate.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/TestS3NContractCreate.java
deleted file mode 100644
index e44e2b1..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/TestS3NContractCreate.java
+++ /dev/null
@@ -1,38 +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;
-
-public class TestS3NContractCreate 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");
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/TestS3NContractDelete.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/TestS3NContractDelete.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/TestS3NContractDelete.java
deleted file mode 100644
index 1b79d27..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/TestS3NContractDelete.java
+++ /dev/null
@@ -1,31 +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;
-
-public class TestS3NContractDelete extends AbstractContractDeleteTest {
-
- @Override
- protected AbstractFSContract createContract(Configuration conf) {
- return new NativeS3Contract(conf);
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/TestS3NContractMkdir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/TestS3NContractMkdir.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/TestS3NContractMkdir.java
deleted file mode 100644
index 527a31d..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/TestS3NContractMkdir.java
+++ /dev/null
@@ -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 TestS3NContractMkdir extends AbstractContractMkdirTest {
-
- @Override
- protected AbstractFSContract createContract(Configuration conf) {
- return new NativeS3Contract(conf);
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/TestS3NContractOpen.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/TestS3NContractOpen.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/TestS3NContractOpen.java
deleted file mode 100644
index 2186f28..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/TestS3NContractOpen.java
+++ /dev/null
@@ -1,31 +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;
-
-public class TestS3NContractOpen extends AbstractContractOpenTest {
-
- @Override
- protected AbstractFSContract createContract(Configuration conf) {
- return new NativeS3Contract(conf);
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/TestS3NContractRename.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/TestS3NContractRename.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/TestS3NContractRename.java
deleted file mode 100644
index d673416..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/TestS3NContractRename.java
+++ /dev/null
@@ -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.contract.s3n;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-
-public class TestS3NContractRename extends AbstractContractRenameTest {
-
- @Override
- protected AbstractFSContract createContract(Configuration conf) {
- return new NativeS3Contract(conf);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/TestS3NContractRootDir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/TestS3NContractRootDir.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/TestS3NContractRootDir.java
deleted file mode 100644
index 94f8483..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/TestS3NContractRootDir.java
+++ /dev/null
@@ -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 TestS3NContractRootDir extends
- AbstractContractRootDirectoryTest {
-
- @Override
- protected AbstractFSContract createContract(Configuration conf) {
- return new NativeS3Contract(conf);
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/TestS3NContractSeek.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/TestS3NContractSeek.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/TestS3NContractSeek.java
deleted file mode 100644
index 6d04fff..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/TestS3NContractSeek.java
+++ /dev/null
@@ -1,31 +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;
-
-public class TestS3NContractSeek extends AbstractContractSeekTest {
-
- @Override
- protected AbstractFSContract createContract(Configuration conf) {
- return new NativeS3Contract(conf);
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3/ITestInMemoryS3FileSystemContract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3/ITestInMemoryS3FileSystemContract.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3/ITestInMemoryS3FileSystemContract.java
new file mode 100644
index 0000000..d69858b
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3/ITestInMemoryS3FileSystemContract.java
@@ -0,0 +1,35 @@
+/**
+ * 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.s3;
+
+import java.io.IOException;
+
+/**
+ * S3 basic contract tests through mock in-memory S3 implementation.
+ */
+@Deprecated
+public class ITestInMemoryS3FileSystemContract
+ extends S3FileSystemContractBaseTest {
+
+ @Override
+ FileSystemStore getFileSystemStore() throws IOException {
+ return new InMemoryFileSystemStore();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3/TestInMemoryS3FileSystemContract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3/TestInMemoryS3FileSystemContract.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3/TestInMemoryS3FileSystemContract.java
deleted file mode 100644
index f94c283..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3/TestInMemoryS3FileSystemContract.java
+++ /dev/null
@@ -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.s3;
-
-import java.io.IOException;
-
-@Deprecated
-public class TestInMemoryS3FileSystemContract
- extends S3FileSystemContractBaseTest {
-
- @Override
- FileSystemStore getFileSystemStore() throws IOException {
- return new InMemoryFileSystemStore();
- }
-
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestBlockingThreadPoolExecutorService.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestBlockingThreadPoolExecutorService.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestBlockingThreadPoolExecutorService.java
new file mode 100644
index 0000000..b0b8a65
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestBlockingThreadPoolExecutorService.java
@@ -0,0 +1,182 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.google.common.util.concurrent.ListenableFuture;
+import org.apache.hadoop.util.StopWatch;
+import org.junit.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+/**
+ * Basic unit test for S3A's blocking executor service.
+ */
+public class ITestBlockingThreadPoolExecutorService {
+
+ private static final Logger LOG = LoggerFactory.getLogger(
+ BlockingThreadPoolExecutorService.class);
+
+ private static final int NUM_ACTIVE_TASKS = 4;
+ private static final int NUM_WAITING_TASKS = 2;
+ private static final int TASK_SLEEP_MSEC = 100;
+ private static final int SHUTDOWN_WAIT_MSEC = 200;
+ private static final int SHUTDOWN_WAIT_TRIES = 5;
+ private static final int BLOCKING_THRESHOLD_MSEC = 50;
+
+ private static final Integer SOME_VALUE = 1337;
+
+ private static BlockingThreadPoolExecutorService tpe = null;
+
+ @AfterClass
+ public static void afterClass() throws Exception {
+ ensureDestroyed();
+ }
+
+ /**
+ * Basic test of running one trivial task.
+ */
+ @Test
+ public void testSubmitCallable() throws Exception {
+ ensureCreated();
+ ListenableFuture<Integer> f = tpe.submit(callableSleeper);
+ Integer v = f.get();
+ assertEquals(SOME_VALUE, v);
+ }
+
+ /**
+ * More involved test, including detecting blocking when at capacity.
+ */
+ @Test
+ public void testSubmitRunnable() throws Exception {
+ ensureCreated();
+ int totalTasks = NUM_ACTIVE_TASKS + NUM_WAITING_TASKS;
+ StopWatch stopWatch = new StopWatch().start();
+ for (int i = 0; i < totalTasks; i++) {
+ tpe.submit(sleeper);
+ assertDidntBlock(stopWatch);
+ }
+ tpe.submit(sleeper);
+ assertDidBlock(stopWatch);
+ }
+
+ @Test
+ public void testShutdown() throws Exception {
+ // Cover create / destroy, regardless of when this test case runs
+ ensureCreated();
+ ensureDestroyed();
+
+ // Cover create, execute, destroy, regardless of when test case runs
+ ensureCreated();
+ testSubmitRunnable();
+ ensureDestroyed();
+ }
+
+ // Helper functions, etc.
+
+ private void assertDidntBlock(StopWatch sw) {
+ try {
+ assertFalse("Non-blocking call took too long.",
+ sw.now(TimeUnit.MILLISECONDS) > BLOCKING_THRESHOLD_MSEC);
+ } finally {
+ sw.reset().start();
+ }
+ }
+
+ private void assertDidBlock(StopWatch sw) {
+ try {
+ if (sw.now(TimeUnit.MILLISECONDS) < BLOCKING_THRESHOLD_MSEC) {
+ throw new RuntimeException("Blocking call returned too fast.");
+ }
+ } finally {
+ sw.reset().start();
+ }
+ }
+
+ private Runnable sleeper = new Runnable() {
+ @Override
+ public void run() {
+ String name = Thread.currentThread().getName();
+ try {
+ Thread.sleep(TASK_SLEEP_MSEC);
+ } catch (InterruptedException e) {
+ LOG.info("Thread {} interrupted.", name);
+ Thread.currentThread().interrupt();
+ }
+ }
+ };
+
+ private Callable<Integer> callableSleeper = new Callable<Integer>() {
+ @Override
+ public Integer call() throws Exception {
+ sleeper.run();
+ return SOME_VALUE;
+ }
+ };
+
+ /**
+ * Helper function to create thread pool under test.
+ */
+ private static void ensureCreated() throws Exception {
+ if (tpe == null) {
+ LOG.debug("Creating thread pool");
+ tpe = new BlockingThreadPoolExecutorService(NUM_ACTIVE_TASKS,
+ NUM_WAITING_TASKS, 1, TimeUnit.SECONDS, "btpetest");
+ }
+ }
+
+ /**
+ * Helper function to terminate thread pool under test, asserting that
+ * shutdown -> terminate works as expected.
+ */
+ private static void ensureDestroyed() throws Exception {
+ if (tpe == null) {
+ return;
+ }
+ int shutdownTries = SHUTDOWN_WAIT_TRIES;
+
+ tpe.shutdown();
+ if (!tpe.isShutdown()) {
+ throw new RuntimeException("Shutdown had no effect.");
+ }
+
+ while (!tpe.awaitTermination(SHUTDOWN_WAIT_MSEC,
+ TimeUnit.MILLISECONDS)) {
+ LOG.info("Waiting for thread pool shutdown.");
+ if (shutdownTries-- <= 0) {
+ LOG.error("Failed to terminate thread pool gracefully.");
+ break;
+ }
+ }
+ if (!tpe.isTerminated()) {
+ tpe.shutdownNow();
+ if (!tpe.awaitTermination(SHUTDOWN_WAIT_MSEC,
+ TimeUnit.MILLISECONDS)) {
+ throw new RuntimeException(
+ "Failed to terminate thread pool in timely manner.");
+ }
+ }
+ tpe = null;
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAWSCredentialsProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAWSCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAWSCredentialsProvider.java
new file mode 100644
index 0000000..cf8783c
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAWSCredentialsProvider.java
@@ -0,0 +1,250 @@
+/**
+ * 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 java.io.IOException;
+import java.net.URI;
+import java.nio.file.AccessDeniedException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.rules.Timeout;
+
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSCredentialsProviderChain;
+import com.amazonaws.auth.BasicAWSCredentials;
+import com.amazonaws.auth.InstanceProfileCredentialsProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.hadoop.fs.s3a.Constants.*;
+import static org.apache.hadoop.fs.s3a.S3ATestConstants.*;
+import static org.apache.hadoop.fs.s3a.S3AUtils.*;
+import static org.junit.Assert.*;
+
+/**
+ * Tests for {@link Constants#AWS_CREDENTIALS_PROVIDER} logic.
+ *
+ */
+public class ITestS3AAWSCredentialsProvider {
+ private static final Logger LOG =
+ LoggerFactory.getLogger(ITestS3AAWSCredentialsProvider.class);
+
+ @Rule
+ public Timeout testTimeout = new Timeout(1 * 60 * 1000);
+
+ @Rule
+ public ExpectedException exception = ExpectedException.none();
+
+ /**
+ * Declare what exception to raise, and the text which must be found
+ * in it.
+ * @param exceptionClass class of exception
+ * @param text text in exception
+ */
+ private void expectException(Class<? extends Throwable> exceptionClass,
+ String text) {
+ exception.expect(exceptionClass);
+ exception.expectMessage(text);
+ }
+
+ @Test
+ public void testBadConfiguration() throws IOException {
+ Configuration conf = new Configuration();
+ conf.set(AWS_CREDENTIALS_PROVIDER, "no.such.class");
+ try {
+ createFailingFS(conf);
+ } catch (IOException e) {
+ if (!(e.getCause() instanceof ClassNotFoundException)) {
+ LOG.error("Unexpected nested cause: {} in {}", e.getCause(), e, e);
+ throw e;
+ }
+ }
+ }
+
+ /**
+ * Create a filesystem, expect it to fail by raising an IOException.
+ * Raises an assertion exception if in fact the FS does get instantiated.
+ * @param conf configuration
+ * @throws IOException an expected exception.
+ */
+ private void createFailingFS(Configuration conf) throws IOException {
+ S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf);
+ fs.listStatus(new Path("/"));
+ fail("Expected exception - got " + fs);
+ }
+
+ static class BadCredentialsProvider implements AWSCredentialsProvider {
+
+ @SuppressWarnings("unused")
+ public BadCredentialsProvider(URI name, Configuration conf) {
+ }
+
+ @Override
+ public AWSCredentials getCredentials() {
+ return new BasicAWSCredentials("bad_key", "bad_secret");
+ }
+
+ @Override
+ public void refresh() {
+ }
+ }
+
+ @Test
+ public void testBadCredentials() throws Exception {
+ Configuration conf = new Configuration();
+ conf.set(AWS_CREDENTIALS_PROVIDER, BadCredentialsProvider.class.getName());
+ try {
+ createFailingFS(conf);
+ } catch (AccessDeniedException e) {
+ // expected
+ }
+ }
+
+ static class GoodCredentialsProvider extends AWSCredentialsProviderChain {
+
+ @SuppressWarnings("unused")
+ public GoodCredentialsProvider(URI name, Configuration conf) {
+ super(new BasicAWSCredentialsProvider(conf.get(ACCESS_KEY),
+ conf.get(SECRET_KEY)), new InstanceProfileCredentialsProvider());
+ }
+ }
+
+ @Test
+ public void testGoodProvider() throws Exception {
+ Configuration conf = new Configuration();
+ conf.set(AWS_CREDENTIALS_PROVIDER, GoodCredentialsProvider.class.getName());
+ S3ATestUtils.createTestFileSystem(conf);
+ }
+
+ @Test
+ public void testAnonymousProvider() throws Exception {
+ Configuration conf = new Configuration();
+ conf.set(AWS_CREDENTIALS_PROVIDER,
+ AnonymousAWSCredentialsProvider.class.getName());
+ Path testFile = new Path(
+ conf.getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE));
+ S3ATestUtils.useCSVDataEndpoint(conf);
+ FileSystem fs = FileSystem.newInstance(testFile.toUri(), conf);
+ assertNotNull(fs);
+ assertTrue(fs instanceof S3AFileSystem);
+ FileStatus stat = fs.getFileStatus(testFile);
+ assertNotNull(stat);
+ assertEquals(testFile, stat.getPath());
+ }
+
+ /**
+ * A credential provider whose constructor signature doesn't match.
+ */
+ static class ConstructorSignatureErrorProvider
+ implements AWSCredentialsProvider {
+
+ @SuppressWarnings("unused")
+ public ConstructorSignatureErrorProvider(String str) {
+ }
+
+ @Override
+ public AWSCredentials getCredentials() {
+ return null;
+ }
+
+ @Override
+ public void refresh() {
+ }
+ }
+
+ /**
+ * A credential provider whose constructor raises an NPE.
+ */
+ static class ConstructorFailureProvider
+ implements AWSCredentialsProvider {
+
+ @SuppressWarnings("unused")
+ public ConstructorFailureProvider() {
+ throw new NullPointerException("oops");
+ }
+
+ @Override
+ public AWSCredentials getCredentials() {
+ return null;
+ }
+
+ @Override
+ public void refresh() {
+ }
+ }
+
+ @Test
+ public void testProviderWrongClass() throws Exception {
+ expectProviderInstantiationFailure(this.getClass().getName(),
+ NOT_AWS_PROVIDER);
+ }
+
+ @Test
+ public void testProviderNotAClass() throws Exception {
+ expectProviderInstantiationFailure("NoSuchClass",
+ "ClassNotFoundException");
+ }
+
+ private void expectProviderInstantiationFailure(String option,
+ String expectedErrorText) throws IOException {
+ Configuration conf = new Configuration();
+ conf.set(AWS_CREDENTIALS_PROVIDER, option);
+ Path testFile = new Path(
+ conf.getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE));
+ expectException(IOException.class, expectedErrorText);
+ URI uri = testFile.toUri();
+ S3AUtils.createAWSCredentialProviderSet(uri, conf, uri);
+ }
+
+ @Test
+ public void testProviderConstructorError() throws Exception {
+ expectProviderInstantiationFailure(
+ ConstructorSignatureErrorProvider.class.getName(),
+ CONSTRUCTOR_EXCEPTION);
+ }
+
+ @Test
+ public void testProviderFailureError() throws Exception {
+ expectProviderInstantiationFailure(
+ ConstructorFailureProvider.class.getName(),
+ INSTANTIATION_EXCEPTION);
+ }
+
+ @Test
+ public void testInstantiationChain() throws Throwable {
+ Configuration conf = new Configuration();
+ conf.set(AWS_CREDENTIALS_PROVIDER,
+ TemporaryAWSCredentialsProvider.NAME
+ + ", \t" + SimpleAWSCredentialsProvider.NAME
+ + " ,\n " + AnonymousAWSCredentialsProvider.NAME);
+ Path testFile = new Path(
+ conf.getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE));
+
+ URI uri = testFile.toUri();
+ S3AUtils.createAWSCredentialProviderSet(uri, conf, uri);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockingThreadPool.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockingThreadPool.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockingThreadPool.java
new file mode 100644
index 0000000..4444d0c
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockingThreadPool.java
@@ -0,0 +1,80 @@
+/**
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+/**
+ * Demonstrate that the threadpool blocks additional client requests if
+ * its queue is full (rather than throwing an exception) by initiating an
+ * upload consisting of 4 parts with 2 threads and 1 spot in the queue. The
+ * 4th part should not trigger an exception as it would with a
+ * non-blocking threadpool.
+ */
+public class ITestS3ABlockingThreadPool {
+
+ private Configuration conf;
+ private S3AFileSystem fs;
+
+ @Rule
+ public Timeout testTimeout = new Timeout(30 * 60 * 1000);
+
+ protected Path getTestPath() {
+ return new Path("/tests3a");
+ }
+
+ @Before
+ public void setUp() throws Exception {
+ conf = new Configuration();
+ conf.setLong(Constants.MIN_MULTIPART_THRESHOLD, 5 * 1024 * 1024);
+ conf.setLong(Constants.MULTIPART_SIZE, 5 * 1024 * 1024);
+ conf.setInt(Constants.MAX_THREADS, 2);
+ conf.setInt(Constants.MAX_TOTAL_TASKS, 1);
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ if (fs != null) {
+ fs.delete(getTestPath(), true);
+ }
+ }
+
+ @Test
+ public void testRegularMultiPartUpload() throws Exception {
+ fs = S3ATestUtils.createTestFileSystem(conf);
+ ContractTestUtils.createAndVerifyFile(fs, getTestPath(), 16 * 1024 *
+ 1024);
+ }
+
+ @Test
+ public void testFastMultiPartUpload() throws Exception {
+ conf.setBoolean(Constants.FAST_UPLOAD, true);
+ fs = S3ATestUtils.createTestFileSystem(conf);
+ ContractTestUtils.createAndVerifyFile(fs, getTestPath(), 16 * 1024 *
+ 1024);
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlocksize.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlocksize.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlocksize.java
new file mode 100644
index 0000000..9a6dae7
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlocksize.java
@@ -0,0 +1,96 @@
+/**
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.AbstractFSContractTestBase;
+import org.apache.hadoop.fs.contract.s3a.S3AContract;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.fileStatsToString;
+
+/**
+ * S3A tests for configuring block size.
+ */
+public class ITestS3ABlocksize extends AbstractFSContractTestBase {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(ITestS3ABlocksize.class);
+
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new S3AContract(conf);
+ }
+
+ @Rule
+ public Timeout testTimeout = new Timeout(30 * 60 * 1000);
+
+ @Test
+ @SuppressWarnings("deprecation")
+ public void testBlockSize() throws Exception {
+ FileSystem fs = getFileSystem();
+ long defaultBlockSize = fs.getDefaultBlockSize();
+ assertEquals("incorrect blocksize",
+ S3AFileSystem.DEFAULT_BLOCKSIZE, defaultBlockSize);
+ long newBlockSize = defaultBlockSize * 2;
+ fs.getConf().setLong(Constants.FS_S3A_BLOCK_SIZE, newBlockSize);
+
+ Path dir = path("testBlockSize");
+ Path file = new Path(dir, "file");
+ createFile(fs, file, true, dataset(1024, 'a', 'z' - 'a'));
+ FileStatus fileStatus = fs.getFileStatus(file);
+ assertEquals("Double default block size in stat(): " + fileStatus,
+ newBlockSize,
+ fileStatus.getBlockSize());
+
+ // check the listing & assert that the block size is picked up by
+ // this route too.
+ boolean found = false;
+ FileStatus[] listing = fs.listStatus(dir);
+ for (FileStatus stat : listing) {
+ LOG.info("entry: {}", stat);
+ if (file.equals(stat.getPath())) {
+ found = true;
+ assertEquals("Double default block size in ls(): " + stat,
+ newBlockSize,
+ stat.getBlockSize());
+ }
+ }
+ assertTrue("Did not find " + fileStatsToString(listing, ", "), found);
+ }
+
+ @Test
+ public void testRootFileStatusHasBlocksize() throws Throwable {
+ FileSystem fs = getFileSystem();
+ FileStatus status = fs.getFileStatus(new Path("/"));
+ assertTrue("Invalid root blocksize",
+ status.getBlockSize() >= 0);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
new file mode 100644
index 0000000..9fadc1f
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
@@ -0,0 +1,441 @@
+/**
+ * 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.ClientConfiguration;
+import com.amazonaws.services.s3.AmazonS3Client;
+import com.amazonaws.services.s3.S3ClientOptions;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang.reflect.FieldUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.s3native.S3xLoginHelper;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.net.URI;
+
+import org.apache.hadoop.security.ProviderUtils;
+import org.apache.hadoop.security.alias.CredentialProvider;
+import org.apache.hadoop.security.alias.CredentialProviderFactory;
+import org.apache.hadoop.util.VersionInfo;
+import org.apache.http.HttpStatus;
+import org.junit.rules.TemporaryFolder;
+
+/**
+ * S3A tests for configuration.
+ */
+public class ITestS3AConfiguration {
+ private static final String EXAMPLE_ID = "AKASOMEACCESSKEY";
+ private static final String EXAMPLE_KEY =
+ "RGV0cm9pdCBSZ/WQgY2xl/YW5lZCB1cAEXAMPLE";
+
+ private Configuration conf;
+ private S3AFileSystem fs;
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(ITestS3AConfiguration.class);
+
+ private static final String TEST_ENDPOINT = "test.fs.s3a.endpoint";
+
+ @Rule
+ public Timeout testTimeout = new Timeout(30 * 60 * 1000);
+
+ @Rule
+ public final TemporaryFolder tempDir = new TemporaryFolder();
+
+ /**
+ * Test if custom endpoint is picked up.
+ * <p/>
+ * The test expects TEST_ENDPOINT to be defined in the Configuration
+ * describing the endpoint of the bucket to which TEST_FS_S3A_NAME points
+ * (f.i. "s3-eu-west-1.amazonaws.com" if the bucket is located in Ireland).
+ * Evidently, the bucket has to be hosted in the region denoted by the
+ * endpoint for the test to succeed.
+ * <p/>
+ * More info and the list of endpoint identifiers:
+ * http://docs.aws.amazon.com/general/latest/gr/rande.html#s3_region
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testEndpoint() throws Exception {
+ conf = new Configuration();
+ String endpoint = conf.getTrimmed(TEST_ENDPOINT, "");
+ if (endpoint.isEmpty()) {
+ LOG.warn("Custom endpoint test skipped as " + TEST_ENDPOINT + "config " +
+ "setting was not detected");
+ } else {
+ conf.set(Constants.ENDPOINT, endpoint);
+ fs = S3ATestUtils.createTestFileSystem(conf);
+ AmazonS3Client s3 = fs.getAmazonS3Client();
+ String endPointRegion = "";
+ // Differentiate handling of "s3-" and "s3." based endpoint identifiers
+ String[] endpointParts = StringUtils.split(endpoint, '.');
+ if (endpointParts.length == 3) {
+ endPointRegion = endpointParts[0].substring(3);
+ } else if (endpointParts.length == 4) {
+ endPointRegion = endpointParts[1];
+ } else {
+ fail("Unexpected endpoint");
+ }
+ assertEquals("Endpoint config setting and bucket location differ: ",
+ endPointRegion, s3.getBucketLocation(fs.getUri().getHost()));
+ }
+ }
+
+ @Test
+ public void testProxyConnection() throws Exception {
+ conf = new Configuration();
+ conf.setInt(Constants.MAX_ERROR_RETRIES, 2);
+ conf.set(Constants.PROXY_HOST, "127.0.0.1");
+ conf.setInt(Constants.PROXY_PORT, 1);
+ String proxy =
+ conf.get(Constants.PROXY_HOST) + ":" + conf.get(Constants.PROXY_PORT);
+ try {
+ fs = S3ATestUtils.createTestFileSystem(conf);
+ fail("Expected a connection error for proxy server at " + proxy);
+ } catch (AWSClientIOException e) {
+ // expected
+ }
+ }
+
+ @Test
+ public void testProxyPortWithoutHost() throws Exception {
+ conf = new Configuration();
+ conf.setInt(Constants.MAX_ERROR_RETRIES, 2);
+ conf.setInt(Constants.PROXY_PORT, 1);
+ try {
+ fs = S3ATestUtils.createTestFileSystem(conf);
+ fail("Expected a proxy configuration error");
+ } catch (IllegalArgumentException e) {
+ String msg = e.toString();
+ if (!msg.contains(Constants.PROXY_HOST) &&
+ !msg.contains(Constants.PROXY_PORT)) {
+ throw e;
+ }
+ }
+ }
+
+ @Test
+ public void testAutomaticProxyPortSelection() throws Exception {
+ conf = new Configuration();
+ conf.setInt(Constants.MAX_ERROR_RETRIES, 2);
+ conf.set(Constants.PROXY_HOST, "127.0.0.1");
+ conf.set(Constants.SECURE_CONNECTIONS, "true");
+ try {
+ fs = S3ATestUtils.createTestFileSystem(conf);
+ fail("Expected a connection error for proxy server");
+ } catch (AWSClientIOException e) {
+ // expected
+ }
+ conf.set(Constants.SECURE_CONNECTIONS, "false");
+ try {
+ fs = S3ATestUtils.createTestFileSystem(conf);
+ fail("Expected a connection error for proxy server");
+ } catch (AWSClientIOException e) {
+ // expected
+ }
+ }
+
+ @Test
+ public void testUsernameInconsistentWithPassword() throws Exception {
+ conf = new Configuration();
+ conf.setInt(Constants.MAX_ERROR_RETRIES, 2);
+ conf.set(Constants.PROXY_HOST, "127.0.0.1");
+ conf.setInt(Constants.PROXY_PORT, 1);
+ conf.set(Constants.PROXY_USERNAME, "user");
+ try {
+ fs = S3ATestUtils.createTestFileSystem(conf);
+ fail("Expected a connection error for proxy server");
+ } catch (IllegalArgumentException e) {
+ String msg = e.toString();
+ if (!msg.contains(Constants.PROXY_USERNAME) &&
+ !msg.contains(Constants.PROXY_PASSWORD)) {
+ throw e;
+ }
+ }
+ conf = new Configuration();
+ conf.setInt(Constants.MAX_ERROR_RETRIES, 2);
+ conf.set(Constants.PROXY_HOST, "127.0.0.1");
+ conf.setInt(Constants.PROXY_PORT, 1);
+ conf.set(Constants.PROXY_PASSWORD, "password");
+ try {
+ fs = S3ATestUtils.createTestFileSystem(conf);
+ fail("Expected a connection error for proxy server");
+ } catch (IllegalArgumentException e) {
+ String msg = e.toString();
+ if (!msg.contains(Constants.PROXY_USERNAME) &&
+ !msg.contains(Constants.PROXY_PASSWORD)) {
+ throw e;
+ }
+ }
+ }
+
+ @Test
+ public void testCredsFromCredentialProvider() 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());
+
+ provisionAccessKeys(conf);
+
+ conf.set(Constants.ACCESS_KEY, EXAMPLE_ID + "LJM");
+ S3xLoginHelper.Login creds =
+ S3AUtils.getAWSAccessKeys(new URI("s3a://foobar"), conf);
+ assertEquals("AccessKey incorrect.", EXAMPLE_ID, creds.getUser());
+ assertEquals("SecretKey incorrect.", EXAMPLE_KEY, creds.getPassword());
+ }
+
+ void provisionAccessKeys(final Configuration conf) throws Exception {
+ // add our creds to the provider
+ final CredentialProvider provider =
+ CredentialProviderFactory.getProviders(conf).get(0);
+ provider.createCredentialEntry(Constants.ACCESS_KEY,
+ EXAMPLE_ID.toCharArray());
+ provider.createCredentialEntry(Constants.SECRET_KEY,
+ EXAMPLE_KEY.toCharArray());
+ provider.flush();
+ }
+
+ @Test
+ public void testCredsFromUserInfo() 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());
+
+ provisionAccessKeys(conf);
+
+ conf.set(Constants.ACCESS_KEY, EXAMPLE_ID + "LJM");
+ URI uriWithUserInfo = new URI("s3a://123:456@foobar");
+ S3xLoginHelper.Login creds =
+ S3AUtils.getAWSAccessKeys(uriWithUserInfo, conf);
+ assertEquals("AccessKey incorrect.", "123", creds.getUser());
+ assertEquals("SecretKey incorrect.", "456", creds.getPassword());
+ }
+
+ @Test
+ public void testIDFromUserInfoSecretFromCredentialProvider()
+ 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());
+
+ provisionAccessKeys(conf);
+
+ conf.set(Constants.ACCESS_KEY, EXAMPLE_ID + "LJM");
+ URI uriWithUserInfo = new URI("s3a://123@foobar");
+ S3xLoginHelper.Login creds =
+ S3AUtils.getAWSAccessKeys(uriWithUserInfo, conf);
+ assertEquals("AccessKey incorrect.", "123", creds.getUser());
+ assertEquals("SecretKey incorrect.", EXAMPLE_KEY, creds.getPassword());
+ }
+
+ @Test
+ public void testSecretFromCredentialProviderIDFromConfig() 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(Constants.SECRET_KEY,
+ EXAMPLE_KEY.toCharArray());
+ provider.flush();
+
+ conf.set(Constants.ACCESS_KEY, EXAMPLE_ID);
+ S3xLoginHelper.Login creds =
+ S3AUtils.getAWSAccessKeys(new URI("s3a://foobar"), conf);
+ assertEquals("AccessKey incorrect.", EXAMPLE_ID, creds.getUser());
+ assertEquals("SecretKey incorrect.", EXAMPLE_KEY, creds.getPassword());
+ }
+
+ @Test
+ public void testIDFromCredentialProviderSecretFromConfig() 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(Constants.ACCESS_KEY,
+ EXAMPLE_ID.toCharArray());
+ provider.flush();
+
+ conf.set(Constants.SECRET_KEY, EXAMPLE_KEY);
+ S3xLoginHelper.Login creds =
+ S3AUtils.getAWSAccessKeys(new URI("s3a://foobar"), conf);
+ assertEquals("AccessKey incorrect.", EXAMPLE_ID, creds.getUser());
+ assertEquals("SecretKey incorrect.", EXAMPLE_KEY, creds.getPassword());
+ }
+
+ @Test
+ public void testExcludingS3ACredentialProvider() 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,
+ "jceks://s3a/foobar," + jks.toString());
+
+ // first make sure that the s3a based provider is removed
+ Configuration c = ProviderUtils.excludeIncompatibleCredentialProviders(
+ conf, S3AFileSystem.class);
+ String newPath = conf.get(
+ CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH);
+ assertFalse("Provider Path incorrect", newPath.contains("s3a://"));
+
+ // now let's make sure the new path is created by the S3AFileSystem
+ // and the integration still works. Let's provision the keys through
+ // the altered configuration instance and then try and access them
+ // using the original config with the s3a provider in the path.
+ provisionAccessKeys(c);
+
+ conf.set(Constants.ACCESS_KEY, EXAMPLE_ID + "LJM");
+ URI uriWithUserInfo = new URI("s3a://123:456@foobar");
+ S3xLoginHelper.Login creds =
+ S3AUtils.getAWSAccessKeys(uriWithUserInfo, conf);
+ assertEquals("AccessKey incorrect.", "123", creds.getUser());
+ assertEquals("SecretKey incorrect.", "456", creds.getPassword());
+
+ }
+
+ @Test
+ public void shouldBeAbleToSwitchOnS3PathStyleAccessViaConfigProperty()
+ throws Exception {
+
+ conf = new Configuration();
+ conf.set(Constants.PATH_STYLE_ACCESS, Boolean.toString(true));
+ assertTrue(conf.getBoolean(Constants.PATH_STYLE_ACCESS, false));
+
+ try {
+ fs = S3ATestUtils.createTestFileSystem(conf);
+ assertNotNull(fs);
+ AmazonS3Client s3 = fs.getAmazonS3Client();
+ assertNotNull(s3);
+ S3ClientOptions clientOptions = getField(s3, S3ClientOptions.class,
+ "clientOptions");
+ assertTrue("Expected to find path style access to be switched on!",
+ clientOptions.isPathStyleAccess());
+ byte[] file = ContractTestUtils.toAsciiByteArray("test file");
+ ContractTestUtils.writeAndRead(fs,
+ new Path("/path/style/access/testFile"), file, file.length,
+ conf.getInt(Constants.FS_S3A_BLOCK_SIZE, file.length), false, true);
+ } catch (final AWSS3IOException e) {
+ LOG.error("Caught exception: ", e);
+ // Catch/pass standard path style access behaviour when live bucket
+ // isn't in the same region as the s3 client default. See
+ // http://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html
+ assertEquals(e.getStatusCode(), HttpStatus.SC_MOVED_PERMANENTLY);
+ }
+ }
+
+ @Test
+ public void testDefaultUserAgent() throws Exception {
+ conf = new Configuration();
+ fs = S3ATestUtils.createTestFileSystem(conf);
+ assertNotNull(fs);
+ AmazonS3Client s3 = fs.getAmazonS3Client();
+ assertNotNull(s3);
+ ClientConfiguration awsConf = getField(s3, ClientConfiguration.class,
+ "clientConfiguration");
+ assertEquals("Hadoop " + VersionInfo.getVersion(), awsConf.getUserAgent());
+ }
+
+ @Test
+ public void testCustomUserAgent() throws Exception {
+ conf = new Configuration();
+ conf.set(Constants.USER_AGENT_PREFIX, "MyApp");
+ fs = S3ATestUtils.createTestFileSystem(conf);
+ assertNotNull(fs);
+ AmazonS3Client s3 = fs.getAmazonS3Client();
+ assertNotNull(s3);
+ ClientConfiguration awsConf = getField(s3, ClientConfiguration.class,
+ "clientConfiguration");
+ assertEquals("MyApp, Hadoop " + VersionInfo.getVersion(),
+ awsConf.getUserAgent());
+ }
+
+ @Test
+ public void testCloseIdempotent() throws Throwable {
+ conf = new Configuration();
+ fs = S3ATestUtils.createTestFileSystem(conf);
+ fs.close();
+ fs.close();
+ }
+
+ /**
+ * Reads and returns a field from an object using reflection. If the field
+ * cannot be found, is null, or is not the expected type, then this method
+ * fails the test.
+ *
+ * @param target object to read
+ * @param fieldType type of field to read, which will also be the return type
+ * @param fieldName name of field to read
+ * @return field that was read
+ * @throws IllegalAccessException if access not allowed
+ */
+ private static <T> T getField(Object target, Class<T> fieldType,
+ String fieldName) throws IllegalAccessException {
+ Object obj = FieldUtils.readField(target, fieldName, true);
+ assertNotNull(String.format(
+ "Could not read field named %s in object with class %s.", fieldName,
+ target.getClass().getName()), obj);
+ assertTrue(String.format(
+ "Unexpected type found for field named %s, expected %s, actual %s.",
+ fieldName, fieldType.getName(), obj.getClass().getName()),
+ fieldType.isAssignableFrom(obj.getClass()));
+ return fieldType.cast(obj);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACredentialsInURL.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACredentialsInURL.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACredentialsInURL.java
new file mode 100644
index 0000000..b3d7abf
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACredentialsInURL.java
@@ -0,0 +1,155 @@
+/**
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.URI;
+import java.net.URLEncoder;
+import java.nio.file.AccessDeniedException;
+
+import static org.apache.hadoop.fs.s3a.S3ATestConstants.TEST_FS_S3A_NAME;
+
+/**
+ * Tests that credentials can go into the URL. This includes a valid
+ * set, and a check that an invalid set do at least get stripped out
+ * of the final URI
+ */
+public class ITestS3ACredentialsInURL extends Assert {
+ private S3AFileSystem fs;
+ private static final Logger LOG =
+ LoggerFactory.getLogger(ITestS3ACredentialsInURL.class);
+ @Rule
+ public Timeout testTimeout = new Timeout(30 * 60 * 1000);
+
+ @After
+ public void teardown() {
+ IOUtils.closeStream(fs);
+ }
+
+ /**
+ * Test instantiation.
+ * @throws Throwable
+ */
+ @Test
+ public void testInstantiateFromURL() throws Throwable {
+
+ Configuration conf = new Configuration();
+ String accessKey = conf.get(Constants.ACCESS_KEY);
+ String secretKey = conf.get(Constants.SECRET_KEY);
+ String fsname = conf.getTrimmed(TEST_FS_S3A_NAME, "");
+ Assume.assumeNotNull(fsname, accessKey, secretKey);
+ URI original = new URI(fsname);
+ URI secretsURI = createUriWithEmbeddedSecrets(original,
+ accessKey, secretKey);
+ if (secretKey.contains("/")) {
+ assertTrue("test URI encodes the / symbol", secretsURI.toString().
+ contains("%252F"));
+ }
+ if (secretKey.contains("+")) {
+ assertTrue("test URI encodes the + symbol", secretsURI.toString().
+ contains("%252B"));
+ }
+ assertFalse("Does not contain secrets", original.equals(secretsURI));
+
+ conf.set(TEST_FS_S3A_NAME, secretsURI.toString());
+ conf.unset(Constants.ACCESS_KEY);
+ conf.unset(Constants.SECRET_KEY);
+ fs = S3ATestUtils.createTestFileSystem(conf);
+ String fsURI = fs.getUri().toString();
+ assertFalse("FS URI contains a @ symbol", fsURI.contains("@"));
+ assertFalse("FS URI contains a % symbol", fsURI.contains("%"));
+ if (!original.toString().startsWith(fsURI)) {
+ fail("Filesystem URI does not match original");
+ }
+ validate("original path", new Path(original));
+ validate("bare path", new Path("/"));
+ validate("secrets path", new Path(secretsURI));
+ }
+
+ private void validate(String text, Path path) throws IOException {
+ try {
+ fs.canonicalizeUri(path.toUri());
+ fs.checkPath(path);
+ assertTrue(text + " Not a directory",
+ fs.getFileStatus(new Path("/")).isDirectory());
+ fs.globStatus(path);
+ } catch (AssertionError e) {
+ throw e;
+ } catch (Exception e) {
+ LOG.debug("{} failure: {}", text, e, e);
+ fail(text + " Test failed");
+ }
+ }
+
+ /**
+ * Set up some invalid credentials, verify login is rejected.
+ * @throws Throwable
+ */
+ @Test
+ public void testInvalidCredentialsFail() throws Throwable {
+ Configuration conf = new Configuration();
+ String fsname = conf.getTrimmed(TEST_FS_S3A_NAME, "");
+ Assume.assumeNotNull(fsname);
+ URI original = new URI(fsname);
+ URI testURI = createUriWithEmbeddedSecrets(original, "user", "//");
+
+ conf.set(TEST_FS_S3A_NAME, testURI.toString());
+ fs = S3ATestUtils.createTestFileSystem(conf);
+ try {
+ S3AFileStatus status = fs.getFileStatus(new Path("/"));
+ fail("Expected an AccessDeniedException, got " + status);
+ } catch (AccessDeniedException e) {
+ // expected
+ }
+
+ }
+
+ private URI createUriWithEmbeddedSecrets(URI original,
+ String accessKey,
+ String secretKey) throws UnsupportedEncodingException {
+ String encodedSecretKey = URLEncoder.encode(secretKey, "UTF-8");
+ String formattedString = String.format("%s://%s:%s@%s/%s/",
+ original.getScheme(),
+ accessKey,
+ encodedSecretKey,
+ original.getHost(),
+ original.getPath());
+ URI testURI;
+ try {
+ testURI = new Path(formattedString).toUri();
+ } catch (IllegalArgumentException e) {
+ // inner cause is stripped to keep any secrets out of stack traces
+ throw new IllegalArgumentException("Could not encode Path");
+ }
+ return testURI;
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryption.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryption.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryption.java
new file mode 100644
index 0000000..4543278
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryption.java
@@ -0,0 +1,104 @@
+/*
+ * 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.services.s3.model.ObjectMetadata;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.io.IOUtils;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
+
+/**
+ * Test whether or not encryption works by turning it on. Some checks
+ * are made for different file sizes as there have been reports that the
+ * file length may be rounded up to match word boundaries.
+ */
+public class ITestS3AEncryption extends AbstractS3ATestBase {
+ private static final String AES256 = Constants.SERVER_SIDE_ENCRYPTION_AES256;
+
+ @Override
+ protected Configuration createConfiguration() {
+ Configuration conf = super.createConfiguration();
+ S3ATestUtils.disableFilesystemCaching(conf);
+ conf.set(Constants.SERVER_SIDE_ENCRYPTION_ALGORITHM,
+ AES256);
+ return conf;
+ }
+
+ private static final int[] SIZES = {
+ 0, 1, 2, 3, 4, 5, 254, 255, 256, 257, 2 ^ 10 - 3, 2 ^ 11 - 2, 2 ^ 12 - 1
+ };
+
+ @Override
+ public void teardown() throws Exception {
+ super.teardown();
+ IOUtils.closeStream(getFileSystem());
+ }
+
+ @Test
+ public void testEncryption() throws Throwable {
+ for (int size: SIZES) {
+ validateEncryptionForFilesize(size);
+ }
+ }
+
+ @Test
+ public void testEncryptionOverRename() throws Throwable {
+ skipIfEncryptionTestsDisabled(getConfiguration());
+ Path src = path(createFilename(1024));
+ byte[] data = dataset(1024, 'a', 'z');
+ S3AFileSystem fs = getFileSystem();
+ writeDataset(fs, src, data, data.length, 1024 * 1024, true);
+ ContractTestUtils.verifyFileContents(fs, src, data);
+ Path dest = path(src.getName() + "-copy");
+ fs.rename(src, dest);
+ ContractTestUtils.verifyFileContents(fs, dest, data);
+ assertEncrypted(dest);
+ }
+
+ protected void validateEncryptionForFilesize(int len) throws IOException {
+ skipIfEncryptionTestsDisabled(getConfiguration());
+ describe("Create an encrypted file of size " + len);
+ String src = createFilename(len);
+ Path path = writeThenReadFile(src, len);
+ assertEncrypted(path);
+ rm(getFileSystem(), path, false, false);
+ }
+
+ private String createFilename(int len) {
+ return String.format("%s-%04x", methodName.getMethodName(), len);
+ }
+
+ /**
+ * Assert that at path references an encrypted blob.
+ * @param path path
+ * @throws IOException on a failure
+ */
+ private void assertEncrypted(Path path) throws IOException {
+ ObjectMetadata md = getFileSystem().getObjectMetadata(path);
+ assertEquals(AES256, md.getSSEAlgorithm());
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionAlgorithmPropagation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionAlgorithmPropagation.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionAlgorithmPropagation.java
new file mode 100644
index 0000000..81578c2
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionAlgorithmPropagation.java
@@ -0,0 +1,83 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
+
+/**
+ * Test whether or not encryption settings propagate by choosing an invalid
+ * one. We expect the write to fail with a 400 bad request error
+ */
+public class ITestS3AEncryptionAlgorithmPropagation
+ extends AbstractS3ATestBase {
+
+ @Override
+ protected Configuration createConfiguration() {
+ Configuration conf = super.createConfiguration();
+ S3ATestUtils.disableFilesystemCaching(conf);
+ conf.set(Constants.SERVER_SIDE_ENCRYPTION_ALGORITHM,
+ "DES");
+ return conf;
+ }
+
+ @Override
+ public void teardown() throws Exception {
+ super.teardown();
+ IOUtils.closeStream(getFileSystem());
+ }
+
+ @Test
+ public void testEncrypt0() throws Throwable {
+ writeThenReadFileToFailure(0);
+ }
+
+ @Test
+ public void testEncrypt256() throws Throwable {
+ writeThenReadFileToFailure(256);
+ }
+
+ /**
+ * Make this a no-op so test setup doesn't fail.
+ * @param path path path
+ * @throws IOException on any failure
+ */
+ @Override
+ protected void mkdirs(Path path) throws IOException {
+
+ }
+
+ protected void writeThenReadFileToFailure(int len) throws IOException {
+ skipIfEncryptionTestsDisabled(getConfiguration());
+ describe("Create an encrypted file of size " + len);
+ try {
+ writeThenReadFile(methodName.getMethodName() + '-' + len, len);
+ fail("Expected an exception about an illegal encryption algorithm");
+ } catch (AWSS3IOException e) {
+ assertStatusCode(e, 400);
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionFastOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionFastOutputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionFastOutputStream.java
new file mode 100644
index 0000000..c06fed1
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionFastOutputStream.java
@@ -0,0 +1,35 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+
+/**
+ * Run the encryption tests against the Fast output stream.
+ * This verifies that both file writing paths can encrypt their data.
+ */
+public class ITestS3AEncryptionFastOutputStream extends ITestS3AEncryption {
+
+ @Override
+ protected Configuration createConfiguration() {
+ Configuration conf = super.createConfiguration();
+ conf.setBoolean(Constants.FAST_UPLOAD, true);
+ return conf;
+ }
+}
---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org
[2/7] hadoop git commit: HADOOP-13446. Support running isolated unit
tests separate from AWS integration tests. Contributed by Chris Nauroth.
Posted by cn...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3ADeleteFilesOneByOne.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3ADeleteFilesOneByOne.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3ADeleteFilesOneByOne.java
deleted file mode 100644
index 77c85a9..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3ADeleteFilesOneByOne.java
+++ /dev/null
@@ -1,40 +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;
-import org.junit.Test;
-
-import java.io.IOException;
-
-public class TestS3ADeleteFilesOneByOne extends TestS3ADeleteManyFiles {
-
- @Override
- protected Configuration createConfiguration() {
- Configuration configuration = super.createConfiguration();
- configuration.setBoolean(Constants.ENABLE_MULTI_DELETE, false);
- return configuration;
- }
-
- @Test
- public void testOpenCreate() throws IOException {
-
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3ADeleteManyFiles.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3ADeleteManyFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3ADeleteManyFiles.java
deleted file mode 100644
index bb2cf04..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3ADeleteManyFiles.java
+++ /dev/null
@@ -1,126 +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.fs.Path;
-import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorCompletionService;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-
-/**
- * Test some scalable operations related to file renaming and deletion.
- */
-public class TestS3ADeleteManyFiles extends S3AScaleTestBase {
- private static final Logger LOG =
- LoggerFactory.getLogger(TestS3ADeleteManyFiles.class);
-
- @Test
- public void testBulkRenameAndDelete() throws Throwable {
- final Path scaleTestDir = getTestPath();
- final Path srcDir = new Path(scaleTestDir, "src");
- final Path finalDir = new Path(scaleTestDir, "final");
- final long count = getOperationCount();
- ContractTestUtils.rm(fs, scaleTestDir, true, false);
-
- fs.mkdirs(srcDir);
- fs.mkdirs(finalDir);
-
- int testBufferSize = fs.getConf()
- .getInt(ContractTestUtils.IO_CHUNK_BUFFER_SIZE,
- ContractTestUtils.DEFAULT_IO_CHUNK_BUFFER_SIZE);
- // use Executor to speed up file creation
- ExecutorService exec = Executors.newFixedThreadPool(16);
- final ExecutorCompletionService<Boolean> completionService =
- new ExecutorCompletionService<>(exec);
- try {
- final byte[] data = ContractTestUtils.dataset(testBufferSize, 'a', 'z');
-
- for (int i = 0; i < count; ++i) {
- final String fileName = "foo-" + i;
- completionService.submit(new Callable<Boolean>() {
- @Override
- public Boolean call() throws IOException {
- ContractTestUtils.createFile(fs, new Path(srcDir, fileName),
- false, data);
- return fs.exists(new Path(srcDir, fileName));
- }
- });
- }
- for (int i = 0; i < count; ++i) {
- final Future<Boolean> future = completionService.take();
- try {
- if (!future.get()) {
- LOG.warn("cannot create file");
- }
- } catch (ExecutionException e) {
- LOG.warn("Error while uploading file", e.getCause());
- throw e;
- }
- }
- } finally {
- exec.shutdown();
- }
-
- int nSrcFiles = fs.listStatus(srcDir).length;
- fs.rename(srcDir, finalDir);
- assertEquals(nSrcFiles, fs.listStatus(finalDir).length);
- ContractTestUtils.assertPathDoesNotExist(fs, "not deleted after rename",
- new Path(srcDir, "foo-" + 0));
- ContractTestUtils.assertPathDoesNotExist(fs, "not deleted after rename",
- new Path(srcDir, "foo-" + count / 2));
- ContractTestUtils.assertPathDoesNotExist(fs, "not deleted after rename",
- new Path(srcDir, "foo-" + (count - 1)));
- ContractTestUtils.assertPathExists(fs, "not renamed to dest dir",
- new Path(finalDir, "foo-" + 0));
- ContractTestUtils.assertPathExists(fs, "not renamed to dest dir",
- new Path(finalDir, "foo-" + count/2));
- ContractTestUtils.assertPathExists(fs, "not renamed to dest dir",
- new Path(finalDir, "foo-" + (count-1)));
-
- ContractTestUtils.assertDeleted(fs, finalDir, true, false);
- }
-
- @Test
- public void testOpenCreate() throws IOException {
- Path dir = new Path("/tests3a");
- ContractTestUtils.createAndVerifyFile(fs, dir, 1024);
- ContractTestUtils.createAndVerifyFile(fs, dir, 5 * 1024 * 1024);
- ContractTestUtils.createAndVerifyFile(fs, dir, 20 * 1024 * 1024);
-
-
- /*
- Enable to test the multipart upload
- try {
- ContractTestUtils.createAndVerifyFile(fs, dir,
- (long)6 * 1024 * 1024 * 1024);
- } catch (IOException e) {
- fail(e.getMessage());
- }
- */
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3ADirectoryPerformance.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3ADirectoryPerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3ADirectoryPerformance.java
deleted file mode 100644
index 35ea3ad..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3ADirectoryPerformance.java
+++ /dev/null
@@ -1,187 +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.fs.Path;
-import org.apache.hadoop.fs.s3a.Statistic;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-
-import static org.apache.hadoop.fs.s3a.Statistic.*;
-import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
-import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
-
-/**
- * Test the performance of listing files/directories.
- */
-public class TestS3ADirectoryPerformance extends S3AScaleTestBase {
- private static final Logger LOG = LoggerFactory.getLogger(
- TestS3ADirectoryPerformance.class);
-
- @Test
- public void testListOperations() throws Throwable {
- describe("Test recursive list operations");
- final Path scaleTestDir = getTestPath();
- final Path listDir = new Path(scaleTestDir, "lists");
-
- // scale factor.
- int scale = getConf().getInt(KEY_DIRECTORY_COUNT, DEFAULT_DIRECTORY_COUNT);
- int width = scale;
- int depth = scale;
- int files = scale;
- MetricDiff metadataRequests = new MetricDiff(fs, OBJECT_METADATA_REQUESTS);
- MetricDiff listRequests = new MetricDiff(fs, OBJECT_LIST_REQUESTS);
- MetricDiff listContinueRequests =
- new MetricDiff(fs, OBJECT_CONTINUE_LIST_REQUESTS);
- MetricDiff listStatusCalls = new MetricDiff(fs, INVOCATION_LIST_FILES);
- MetricDiff getFileStatusCalls =
- new MetricDiff(fs, INVOCATION_GET_FILE_STATUS);
- NanoTimer createTimer = new NanoTimer();
- TreeScanResults created =
- createSubdirs(fs, listDir, depth, width, files, 0);
- // add some empty directories
- int emptyDepth = 1 * scale;
- int emptyWidth = 3 * scale;
-
- created.add(createSubdirs(fs, listDir, emptyDepth, emptyWidth, 0,
- 0, "empty", "f-", ""));
- createTimer.end("Time to create %s", created);
- LOG.info("Time per operation: {}",
- toHuman(createTimer.nanosPerOperation(created.totalCount())));
- printThenReset(LOG,
- metadataRequests,
- listRequests,
- listContinueRequests,
- listStatusCalls,
- getFileStatusCalls);
-
- describe("Listing files via treewalk");
- try {
- // Scan the directory via an explicit tree walk.
- // This is the baseline for any listing speedups.
- NanoTimer treeWalkTimer = new NanoTimer();
- TreeScanResults treewalkResults = treeWalk(fs, listDir);
- treeWalkTimer.end("List status via treewalk of %s", created);
-
- printThenReset(LOG,
- metadataRequests,
- listRequests,
- listContinueRequests,
- listStatusCalls,
- getFileStatusCalls);
- assertEquals("Files found in listFiles(recursive=true) " +
- " created=" + created + " listed=" + treewalkResults,
- created.getFileCount(), treewalkResults.getFileCount());
-
- describe("Listing files via listFiles(recursive=true)");
- // listFiles() does the recursion internally
- NanoTimer listFilesRecursiveTimer = new NanoTimer();
-
- TreeScanResults listFilesResults = new TreeScanResults(
- fs.listFiles(listDir, true));
-
- listFilesRecursiveTimer.end("listFiles(recursive=true) of %s", created);
- assertEquals("Files found in listFiles(recursive=true) " +
- " created=" + created + " listed=" + listFilesResults,
- created.getFileCount(), listFilesResults.getFileCount());
-
- // only two list operations should have taken place
- print(LOG,
- metadataRequests,
- listRequests,
- listContinueRequests,
- listStatusCalls,
- getFileStatusCalls);
- assertEquals(listRequests.toString(), 2, listRequests.diff());
- reset(metadataRequests,
- listRequests,
- listContinueRequests,
- listStatusCalls,
- getFileStatusCalls);
-
-
- } finally {
- describe("deletion");
- // deletion at the end of the run
- NanoTimer deleteTimer = new NanoTimer();
- fs.delete(listDir, true);
- deleteTimer.end("Deleting directory tree");
- printThenReset(LOG,
- metadataRequests,
- listRequests,
- listContinueRequests,
- listStatusCalls,
- getFileStatusCalls);
- }
- }
-
- @Test
- public void testTimeToStatEmptyDirectory() throws Throwable {
- describe("Time to stat an empty directory");
- Path path = new Path(getTestPath(), "empty");
- fs.mkdirs(path);
- timeToStatPath(path);
- }
-
- @Test
- public void testTimeToStatNonEmptyDirectory() throws Throwable {
- describe("Time to stat a non-empty directory");
- Path path = new Path(getTestPath(), "dir");
- fs.mkdirs(path);
- touch(fs, new Path(path, "file"));
- timeToStatPath(path);
- }
-
- @Test
- public void testTimeToStatFile() throws Throwable {
- describe("Time to stat a simple file");
- Path path = new Path(getTestPath(), "file");
- touch(fs, path);
- timeToStatPath(path);
- }
-
- @Test
- public void testTimeToStatRoot() throws Throwable {
- describe("Time to stat the root path");
- timeToStatPath(new Path("/"));
- }
-
- private void timeToStatPath(Path path) throws IOException {
- describe("Timing getFileStatus(\"%s\")", path);
- MetricDiff metadataRequests =
- new MetricDiff(fs, Statistic.OBJECT_METADATA_REQUESTS);
- MetricDiff listRequests =
- new MetricDiff(fs, Statistic.OBJECT_LIST_REQUESTS);
- long attempts = getOperationCount();
- NanoTimer timer = new NanoTimer();
- for (long l = 0; l < attempts; l++) {
- fs.getFileStatus(path);
- }
- timer.end("Time to execute %d getFileStatusCalls", attempts);
- LOG.info("Time per call: {}", toHuman(timer.nanosPerOperation(attempts)));
- LOG.info("metadata: {}", metadataRequests);
- LOG.info("metadata per operation {}", metadataRequests.diff() / attempts);
- LOG.info("listObjects: {}", listRequests);
- LOG.info("listObjects: per operation {}", listRequests.diff() / attempts);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3AInputStreamPerformance.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3AInputStreamPerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3AInputStreamPerformance.java
deleted file mode 100644
index d6d9d66..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3AInputStreamPerformance.java
+++ /dev/null
@@ -1,534 +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.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.apache.hadoop.fs.s3a.S3AFileStatus;
-import org.apache.hadoop.fs.s3a.S3AFileSystem;
-import org.apache.hadoop.fs.s3a.S3AInputPolicy;
-import org.apache.hadoop.fs.s3a.S3AInputStream;
-import org.apache.hadoop.fs.s3a.S3AInstrumentation;
-import org.apache.hadoop.fs.s3a.S3ATestUtils;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.CompressionCodecFactory;
-import org.apache.hadoop.util.LineReader;
-import org.junit.After;
-import org.junit.Assume;
-import org.junit.Before;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.EOFException;
-import java.io.IOException;
-
-import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
-import static org.apache.hadoop.fs.s3a.Constants.*;
-
-/**
- * Look at the performance of S3a operations.
- */
-public class TestS3AInputStreamPerformance extends S3AScaleTestBase {
- private static final Logger LOG = LoggerFactory.getLogger(
- TestS3AInputStreamPerformance.class);
-
- private S3AFileSystem s3aFS;
- private Path testData;
- private S3AFileStatus testDataStatus;
- private FSDataInputStream in;
- private S3AInstrumentation.InputStreamStatistics streamStatistics;
- public static final int BLOCK_SIZE = 32 * 1024;
- public static final int BIG_BLOCK_SIZE = 256 * 1024;
-
- /** Tests only run if the there is a named test file that can be read. */
- private boolean testDataAvailable = true;
- private String assumptionMessage = "test file";
-
- /**
- * Open the FS and the test data. The input stream is always set up here.
- * @throws IOException IO Problems.
- */
- @Before
- public void openFS() throws IOException {
- Configuration conf = getConf();
- conf.setInt(SOCKET_SEND_BUFFER, 16 * 1024);
- conf.setInt(SOCKET_RECV_BUFFER, 16 * 1024);
- String testFile = conf.getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE);
- if (testFile.isEmpty()) {
- assumptionMessage = "Empty test property: " + KEY_CSVTEST_FILE;
- testDataAvailable = false;
- } else {
- S3ATestUtils.useCSVDataEndpoint(conf);
- testData = new Path(testFile);
- Path path = this.testData;
- bindS3aFS(path);
- try {
- testDataStatus = s3aFS.getFileStatus(this.testData);
- } catch (IOException e) {
- LOG.warn("Failed to read file {} specified in {}",
- testFile, KEY_CSVTEST_FILE, e);
- throw e;
- }
- }
- }
-
- private void bindS3aFS(Path path) throws IOException {
- s3aFS = (S3AFileSystem) FileSystem.newInstance(path.toUri(), getConf());
- }
-
- /**
- * Cleanup: close the stream, close the FS.
- */
- @After
- public void cleanup() {
- describe("cleanup");
- IOUtils.closeStream(in);
- IOUtils.closeStream(s3aFS);
- }
-
- /**
- * Declare that the test requires the CSV test dataset.
- */
- private void requireCSVTestData() {
- Assume.assumeTrue(assumptionMessage, testDataAvailable);
- }
-
- /**
- * Open the test file with the read buffer specified in the setting.
- * {@link #KEY_READ_BUFFER_SIZE}; use the {@code Normal} policy
- * @return the stream, wrapping an S3a one
- * @throws IOException IO problems
- */
- FSDataInputStream openTestFile() throws IOException {
- return openTestFile(S3AInputPolicy.Normal, 0);
- }
-
- /**
- * Open the test file with the read buffer specified in the setting
- * {@link #KEY_READ_BUFFER_SIZE}.
- * This includes the {@link #requireCSVTestData()} assumption; so
- * if called before any FS op, will automatically skip the test
- * if the CSV file is absent.
- *
- * @param inputPolicy input policy to use
- * @param readahead readahead/buffer size
- * @return the stream, wrapping an S3a one
- * @throws IOException IO problems
- */
- FSDataInputStream openTestFile(S3AInputPolicy inputPolicy, long readahead)
- throws IOException {
- requireCSVTestData();
- return openDataFile(s3aFS, this.testData, inputPolicy, readahead);
- }
-
- /**
- * Open a test file with the read buffer specified in the setting
- * {@link #KEY_READ_BUFFER_SIZE}.
- *
- * @param path path to open
- * @param inputPolicy input policy to use
- * @param readahead readahead/buffer size
- * @return the stream, wrapping an S3a one
- * @throws IOException IO problems
- */
- private FSDataInputStream openDataFile(S3AFileSystem fs,
- Path path,
- S3AInputPolicy inputPolicy,
- long readahead) throws IOException {
- int bufferSize = getConf().getInt(KEY_READ_BUFFER_SIZE,
- DEFAULT_READ_BUFFER_SIZE);
- S3AInputPolicy policy = fs.getInputPolicy();
- fs.setInputPolicy(inputPolicy);
- try {
- FSDataInputStream stream = fs.open(path, bufferSize);
- if (readahead >= 0) {
- stream.setReadahead(readahead);
- }
- streamStatistics = getInputStreamStatistics(stream);
- return stream;
- } finally {
- fs.setInputPolicy(policy);
- }
- }
-
- /**
- * Assert that the stream was only ever opened once.
- */
- protected void assertStreamOpenedExactlyOnce() {
- assertOpenOperationCount(1);
- }
-
- /**
- * Make an assertion count about the number of open operations.
- * @param expected the expected number
- */
- private void assertOpenOperationCount(long expected) {
- assertEquals("open operations in\n" + in,
- expected, streamStatistics.openOperations);
- }
-
- /**
- * Log how long an IOP took, by dividing the total time by the
- * count of operations, printing in a human-readable form.
- * @param operation operation being measured
- * @param timer timing data
- * @param count IOP count.
- */
- protected void logTimePerIOP(String operation,
- NanoTimer timer,
- long count) {
- LOG.info("Time per {}: {} nS",
- operation, toHuman(timer.duration() / count));
- }
-
- @Test
- public void testTimeToOpenAndReadWholeFileBlocks() throws Throwable {
- requireCSVTestData();
- int blockSize = _1MB;
- describe("Open the test file %s and read it in blocks of size %d",
- testData, blockSize);
- long len = testDataStatus.getLen();
- in = openTestFile();
- byte[] block = new byte[blockSize];
- NanoTimer timer2 = new NanoTimer();
- long count = 0;
- // implicitly rounding down here
- long blockCount = len / blockSize;
- for (long i = 0; i < blockCount; i++) {
- int offset = 0;
- int remaining = blockSize;
- NanoTimer blockTimer = new NanoTimer();
- int reads = 0;
- while (remaining > 0) {
- int bytesRead = in.read(block, offset, remaining);
- reads ++;
- if (bytesRead == 1) {
- break;
- }
- remaining -= bytesRead;
- offset += bytesRead;
- count += bytesRead;
- }
- blockTimer.end("Reading block %d in %d reads", i, reads);
- }
- timer2.end("Time to read %d bytes in %d blocks", len, blockCount );
- bandwidth(timer2, count);
- logStreamStatistics();
- }
-
- @Test
- public void testLazySeekEnabled() throws Throwable {
- describe("Verify that seeks do not trigger any IO");
- in = openTestFile();
- long len = testDataStatus.getLen();
- NanoTimer timer = new NanoTimer();
- long blockCount = len / BLOCK_SIZE;
- for (long i = 0; i < blockCount; i++) {
- in.seek(in.getPos() + BLOCK_SIZE - 1);
- }
- in.seek(0);
- blockCount++;
- timer.end("Time to execute %d seeks", blockCount);
- logTimePerIOP("seek()", timer, blockCount);
- logStreamStatistics();
- assertOpenOperationCount(0);
- assertEquals("bytes read", 0, streamStatistics.bytesRead);
- }
-
- @Test
- public void testReadaheadOutOfRange() throws Throwable {
- try {
- in = openTestFile();
- in.setReadahead(-1L);
- fail("Stream should have rejected the request "+ in);
- } catch (IllegalArgumentException e) {
- // expected
- }
- }
-
- @Test
- public void testReadWithNormalPolicy() throws Throwable {
- describe("Read big blocks with a big readahead");
- executeSeekReadSequence(BIG_BLOCK_SIZE, BIG_BLOCK_SIZE * 2,
- S3AInputPolicy.Normal);
- assertStreamOpenedExactlyOnce();
- }
-
- @Test
- public void testDecompressionSequential128K() throws Throwable {
- describe("Decompress with a 128K readahead");
- executeDecompression(128 * 1024, S3AInputPolicy.Sequential);
- assertStreamOpenedExactlyOnce();
- }
-
- /**
- * Execute a decompression + line read with the given input policy.
- * @param readahead byte readahead
- * @param inputPolicy read policy
- * @throws IOException IO Problems
- */
- private void executeDecompression(long readahead,
- S3AInputPolicy inputPolicy) throws IOException {
- CompressionCodecFactory factory
- = new CompressionCodecFactory(getConf());
- CompressionCodec codec = factory.getCodec(testData);
- long bytesRead = 0;
- int lines = 0;
-
- FSDataInputStream objectIn = openTestFile(inputPolicy, readahead);
- ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
- try (LineReader lineReader = new LineReader(
- codec.createInputStream(objectIn), getConf())) {
- Text line = new Text();
- int read;
- while ((read = lineReader.readLine(line)) > 0) {
- bytesRead += read;
- lines++;
- }
- } catch (EOFException eof) {
- // done
- }
- timer.end("Time to read %d lines [%d bytes expanded, %d raw]" +
- " with readahead = %d",
- lines,
- bytesRead,
- testDataStatus.getLen(),
- readahead);
- logTimePerIOP("line read", timer, lines);
- logStreamStatistics();
- }
-
- private void logStreamStatistics() {
- LOG.info(String.format("Stream Statistics%n{}"), streamStatistics);
- }
-
- /**
- * Execute a seek+read sequence.
- * @param blockSize block size for seeks
- * @param readahead what the readahead value of the stream should be
- * @throws IOException IO problems
- */
- protected void executeSeekReadSequence(long blockSize,
- long readahead,
- S3AInputPolicy policy) throws IOException {
- in = openTestFile(policy, readahead);
- long len = testDataStatus.getLen();
- NanoTimer timer = new NanoTimer();
- long blockCount = len / blockSize;
- LOG.info("Reading {} blocks, readahead = {}",
- blockCount, readahead);
- for (long i = 0; i < blockCount; i++) {
- in.seek(in.getPos() + blockSize - 1);
- // this is the read
- assertTrue(in.read() >= 0);
- }
- timer.end("Time to execute %d seeks of distance %d with readahead = %d",
- blockCount,
- blockSize,
- readahead);
- logTimePerIOP("seek(pos + " + blockCount+"); read()", timer, blockCount);
- LOG.info("Effective bandwidth {} MB/S",
- timer.bandwidthDescription(streamStatistics.bytesRead -
- streamStatistics.bytesSkippedOnSeek));
- logStreamStatistics();
- }
-
- public static final int _4K = 4 * 1024;
- public static final int _8K = 8 * 1024;
- public static final int _16K = 16 * 1024;
- public static final int _32K = 32 * 1024;
- public static final int _64K = 64 * 1024;
- public static final int _128K = 128 * 1024;
- public static final int _256K = 256 * 1024;
- public static final int _1MB = 1024 * 1024;
- public static final int _2MB = 2 * _1MB;
- public static final int _10MB = _1MB * 10;
- public static final int _5MB = _1MB * 5;
-
- private static final int[][] RANDOM_IO_SEQUENCE = {
- {_2MB, _128K},
- {_128K, _128K},
- {_5MB, _64K},
- {_1MB, _1MB},
- };
-
- @Test
- public void testRandomIORandomPolicy() throws Throwable {
- executeRandomIO(S3AInputPolicy.Random, (long) RANDOM_IO_SEQUENCE.length);
- assertEquals("streams aborted in " + streamStatistics,
- 0, streamStatistics.aborted);
- }
-
- @Test
- public void testRandomIONormalPolicy() throws Throwable {
- long expectedOpenCount = RANDOM_IO_SEQUENCE.length;
- executeRandomIO(S3AInputPolicy.Normal, expectedOpenCount);
- assertEquals("streams aborted in " + streamStatistics,
- 4, streamStatistics.aborted);
- }
-
- /**
- * Execute the random IO {@code readFully(pos, bytes[])} sequence defined by
- * {@link #RANDOM_IO_SEQUENCE}. The stream is closed afterwards; that's used
- * in the timing too
- * @param policy read policy
- * @param expectedOpenCount expected number of stream openings
- * @throws IOException IO problems
- * @return the timer
- */
- private ContractTestUtils.NanoTimer executeRandomIO(S3AInputPolicy policy,
- long expectedOpenCount)
- throws IOException {
- describe("Random IO with policy \"%s\"", policy);
- byte[] buffer = new byte[_1MB];
- long totalBytesRead = 0;
-
- in = openTestFile(policy, 0);
- ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
- for (int[] action : RANDOM_IO_SEQUENCE) {
- int position = action[0];
- int range = action[1];
- in.readFully(position, buffer, 0, range);
- totalBytesRead += range;
- }
- int reads = RANDOM_IO_SEQUENCE.length;
- timer.end("Time to execute %d reads of total size %d bytes",
- reads,
- totalBytesRead);
- in.close();
- assertOpenOperationCount(expectedOpenCount);
- logTimePerIOP("byte read", timer, totalBytesRead);
- LOG.info("Effective bandwidth {} MB/S",
- timer.bandwidthDescription(streamStatistics.bytesRead -
- streamStatistics.bytesSkippedOnSeek));
- logStreamStatistics();
- return timer;
- }
-
- S3AInputStream getS3aStream() {
- return (S3AInputStream) in.getWrappedStream();
- }
-
- @Test
- public void testRandomReadOverBuffer() throws Throwable {
- describe("read over a buffer, making sure that the requests" +
- " spans readahead ranges");
- int datasetLen = _32K;
- Path dataFile = new Path(getTestPath(), "testReadOverBuffer.bin");
- byte[] sourceData = dataset(datasetLen, 0, 64);
- // relies on the field 'fs' referring to the R/W FS
- writeDataset(fs, dataFile, sourceData, datasetLen, _16K, true);
- byte[] buffer = new byte[datasetLen];
- int readahead = _8K;
- int halfReadahead = _4K;
- in = openDataFile(fs, dataFile, S3AInputPolicy.Random, readahead);
-
- LOG.info("Starting initial reads");
- S3AInputStream s3aStream = getS3aStream();
- assertEquals(readahead, s3aStream.getReadahead());
- byte[] oneByte = new byte[1];
- assertEquals(1, in.read(0, oneByte, 0, 1));
- // make some assertions about the current state
- assertEquals("remaining in\n" + in,
- readahead - 1, s3aStream.remainingInCurrentRequest());
- assertEquals("range start in\n" + in,
- 0, s3aStream.getContentRangeStart());
- assertEquals("range finish in\n" + in,
- readahead, s3aStream.getContentRangeFinish());
-
- assertStreamOpenedExactlyOnce();
-
- describe("Starting sequence of positioned read calls over\n%s", in);
- NanoTimer readTimer = new NanoTimer();
- int currentPos = halfReadahead;
- int offset = currentPos;
- int bytesRead = 0;
- int readOps = 0;
-
- // make multiple read() calls
- while (bytesRead < halfReadahead) {
- int length = buffer.length - offset;
- int read = in.read(currentPos, buffer, offset, length);
- bytesRead += read;
- offset += read;
- readOps++;
- assertEquals("open operations on request #" + readOps
- + " after reading " + bytesRead
- + " current position in stream " + currentPos
- + " in\n" + fs
- + "\n " + in,
- 1, streamStatistics.openOperations);
- for (int i = currentPos; i < currentPos + read; i++) {
- assertEquals("Wrong value from byte " + i,
- sourceData[i], buffer[i]);
- }
- currentPos += read;
- }
- assertStreamOpenedExactlyOnce();
- // assert at the end of the original block
- assertEquals(readahead, currentPos);
- readTimer.end("read %d in %d operations", bytesRead, readOps);
- bandwidth(readTimer, bytesRead);
- LOG.info("Time per byte(): {} nS",
- toHuman(readTimer.nanosPerOperation(bytesRead)));
- LOG.info("Time per read(): {} nS",
- toHuman(readTimer.nanosPerOperation(readOps)));
-
- describe("read last byte");
- // read one more
- int read = in.read(currentPos, buffer, bytesRead, 1);
- assertTrue("-1 from last read", read >= 0);
- assertOpenOperationCount(2);
- assertEquals("Wrong value from read ", sourceData[currentPos],
- (int) buffer[currentPos]);
- currentPos++;
-
-
- // now scan all the way to the end of the file, using single byte read()
- // calls
- describe("read() to EOF over \n%s", in);
- long readCount = 0;
- NanoTimer timer = new NanoTimer();
- LOG.info("seeking");
- in.seek(currentPos);
- LOG.info("reading");
- while(currentPos < datasetLen) {
- int r = in.read();
- assertTrue("Negative read() at position " + currentPos + " in\n" + in,
- r >= 0);
- buffer[currentPos] = (byte)r;
- assertEquals("Wrong value from read from\n" + in,
- sourceData[currentPos], r);
- currentPos++;
- readCount++;
- }
- timer.end("read %d bytes", readCount);
- bandwidth(timer, readCount);
- LOG.info("Time per read(): {} nS",
- toHuman(timer.nanosPerOperation(readCount)));
-
- assertEquals("last read in " + in, -1, in.read());
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/yarn/ITestS3A.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/yarn/ITestS3A.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/yarn/ITestS3A.java
new file mode 100644
index 0000000..ca57da6
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/yarn/ITestS3A.java
@@ -0,0 +1,88 @@
+/**
+ * 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.yarn;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FsStatus;
+import org.apache.hadoop.fs.Path;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import java.util.EnumSet;
+import org.apache.hadoop.fs.s3a.S3ATestUtils;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * S3A tests through the {@link FileContext} API.
+ */
+public class ITestS3A {
+ private FileContext fc;
+
+ @Rule
+ public final Timeout testTimeout = new Timeout(90000);
+
+ @Before
+ public void setUp() throws Exception {
+ Configuration conf = new Configuration();
+ fc = S3ATestUtils.createTestFileContext(conf);
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ if (fc != null) {
+ fc.delete(getTestPath(), true);
+ }
+ }
+
+ protected Path getTestPath() {
+ return new Path("/tests3afc");
+ }
+
+ @Test
+ public void testS3AStatus() throws Exception {
+ FsStatus fsStatus = fc.getFsStatus(null);
+ assertNotNull(fsStatus);
+ assertTrue("Used capacity should be positive: " + fsStatus.getUsed(),
+ fsStatus.getUsed() >= 0);
+ assertTrue("Remaining capacity should be positive: " + fsStatus
+ .getRemaining(),
+ fsStatus.getRemaining() >= 0);
+ assertTrue("Capacity should be positive: " + fsStatus.getCapacity(),
+ fsStatus.getCapacity() >= 0);
+ }
+
+ @Test
+ public void testS3ACreateFileInSubDir() throws Exception {
+ Path dirPath = getTestPath();
+ fc.mkdir(dirPath, FileContext.DIR_DEFAULT_PERM, true);
+ Path filePath = new Path(dirPath, "file");
+ try (FSDataOutputStream file = fc.create(filePath, EnumSet.of(CreateFlag
+ .CREATE))) {
+ file.write(666);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/yarn/ITestS3AMiniYarnCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/yarn/ITestS3AMiniYarnCluster.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/yarn/ITestS3AMiniYarnCluster.java
new file mode 100644
index 0000000..772d8c7
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/yarn/ITestS3AMiniYarnCluster.java
@@ -0,0 +1,148 @@
+/**
+ * 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.yarn;
+
+import java.io.IOException;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.examples.WordCount;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.S3ATestUtils;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.MiniYARNCluster;
+
+import org.junit.After;
+import static org.junit.Assert.assertTrue;
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests that S3A is usable through a YARN application.
+ */
+public class ITestS3AMiniYarnCluster {
+
+ private final Configuration conf = new YarnConfiguration();
+ private S3AFileSystem fs;
+ private MiniYARNCluster yarnCluster;
+ private final String rootPath = "/tests/MiniClusterWordCount/";
+
+ @Before
+ public void beforeTest() throws IOException {
+ fs = S3ATestUtils.createTestFileSystem(conf);
+ fs.mkdirs(new Path(rootPath + "input/"));
+
+ yarnCluster = new MiniYARNCluster("MiniClusterWordCount", // testName
+ 1, // number of node managers
+ 1, // number of local log dirs per node manager
+ 1); // number of hdfs dirs per node manager
+ yarnCluster.init(conf);
+ yarnCluster.start();
+ }
+
+ @After
+ public void afterTest() throws IOException {
+ fs.delete(new Path(rootPath), true);
+ yarnCluster.stop();
+ }
+
+ @Test
+ public void testWithMiniCluster() throws Exception {
+ Path input = new Path(rootPath + "input/in.txt");
+ input = input.makeQualified(fs.getUri(), fs.getWorkingDirectory());
+ Path output = new Path(rootPath + "output/");
+ output = output.makeQualified(fs.getUri(), fs.getWorkingDirectory());
+
+ writeStringToFile(input, "first line\nsecond line\nthird line");
+
+ Job job = Job.getInstance(conf, "word count");
+ job.setJarByClass(WordCount.class);
+ job.setMapperClass(WordCount.TokenizerMapper.class);
+ job.setCombinerClass(WordCount.IntSumReducer.class);
+ job.setReducerClass(WordCount.IntSumReducer.class);
+ job.setOutputKeyClass(Text.class);
+ job.setOutputValueClass(IntWritable.class);
+ FileInputFormat.addInputPath(job, input);
+ FileOutputFormat.setOutputPath(job, output);
+
+ int exitCode = (job.waitForCompletion(true) ? 0 : 1);
+ assertEquals("Returned error code.", 0, exitCode);
+
+ assertTrue(fs.exists(new Path(output, "_SUCCESS")));
+ String outputAsStr = readStringFromFile(new Path(output, "part-r-00000"));
+ Map<String, Integer> resAsMap = getResultAsMap(outputAsStr);
+
+ assertEquals(4, resAsMap.size());
+ assertEquals(1, (int) resAsMap.get("first"));
+ assertEquals(1, (int) resAsMap.get("second"));
+ assertEquals(1, (int) resAsMap.get("third"));
+ assertEquals(3, (int) resAsMap.get("line"));
+ }
+
+ /**
+ * helper method.
+ */
+ private Map<String, Integer> getResultAsMap(String outputAsStr)
+ throws IOException {
+ Map<String, Integer> result = new HashMap<>();
+ for (String line : outputAsStr.split("\n")) {
+ String[] tokens = line.split("\t");
+ result.put(tokens[0], Integer.parseInt(tokens[1]));
+ }
+ return result;
+ }
+
+ /**
+ * helper method.
+ */
+ private void writeStringToFile(Path path, String string) throws IOException {
+ FileContext fc = S3ATestUtils.createTestFileContext(conf);
+ try (FSDataOutputStream file = fc.create(path,
+ EnumSet.of(CreateFlag.CREATE))) {
+ file.write(string.getBytes());
+ }
+ }
+
+ /**
+ * helper method.
+ */
+ private String readStringFromFile(Path path) {
+ try (FSDataInputStream in = fs.open(path)) {
+ long bytesLen = fs.getFileStatus(path).getLen();
+ byte[] buffer = new byte[(int) bytesLen];
+ IOUtils.readFully(in, buffer, 0, buffer.length);
+ return new String(buffer);
+ } catch (IOException e) {
+ throw new RuntimeException("Failed to read from [" + path + "]", e);
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/yarn/TestS3A.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/yarn/TestS3A.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/yarn/TestS3A.java
deleted file mode 100644
index a22dd28..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/yarn/TestS3A.java
+++ /dev/null
@@ -1,85 +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.yarn;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileContext;
-import org.apache.hadoop.fs.FsStatus;
-import org.apache.hadoop.fs.Path;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.Timeout;
-
-import java.util.EnumSet;
-import org.apache.hadoop.fs.s3a.S3ATestUtils;
-
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-
-public class TestS3A {
- private FileContext fc;
-
- @Rule
- public final Timeout testTimeout = new Timeout(90000);
-
- @Before
- public void setUp() throws Exception {
- Configuration conf = new Configuration();
- fc = S3ATestUtils.createTestFileContext(conf);
- }
-
- @After
- public void tearDown() throws Exception {
- if (fc != null) {
- fc.delete(getTestPath(), true);
- }
- }
-
- protected Path getTestPath() {
- return new Path("/tests3afc");
- }
-
- @Test
- public void testS3AStatus() throws Exception {
- FsStatus fsStatus = fc.getFsStatus(null);
- assertNotNull(fsStatus);
- assertTrue("Used capacity should be positive: " + fsStatus.getUsed(),
- fsStatus.getUsed() >= 0);
- assertTrue("Remaining capacity should be positive: " + fsStatus
- .getRemaining(),
- fsStatus.getRemaining() >= 0);
- assertTrue("Capacity should be positive: " + fsStatus.getCapacity(),
- fsStatus.getCapacity() >= 0);
- }
-
- @Test
- public void testS3ACreateFileInSubDir() throws Exception {
- Path dirPath = getTestPath();
- fc.mkdir(dirPath,FileContext.DIR_DEFAULT_PERM,true);
- Path filePath = new Path(dirPath, "file");
- try (FSDataOutputStream file = fc.create(filePath, EnumSet.of(CreateFlag
- .CREATE))) {
- file.write(666);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/yarn/TestS3AMiniYarnCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/yarn/TestS3AMiniYarnCluster.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/yarn/TestS3AMiniYarnCluster.java
deleted file mode 100644
index 990d79f..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/yarn/TestS3AMiniYarnCluster.java
+++ /dev/null
@@ -1,144 +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.yarn;
-
-import java.io.IOException;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.examples.WordCount;
-import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileContext;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.s3a.S3AFileSystem;
-import org.apache.hadoop.fs.s3a.S3ATestUtils;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.server.MiniYARNCluster;
-
-import org.junit.After;
-import static org.junit.Assert.assertTrue;
-import org.junit.Before;
-import org.junit.Test;
-import static org.junit.Assert.assertEquals;
-
-public class TestS3AMiniYarnCluster {
-
- private final Configuration conf = new YarnConfiguration();
- private S3AFileSystem fs;
- private MiniYARNCluster yarnCluster;
- private final String rootPath = "/tests/MiniClusterWordCount/";
-
- @Before
- public void beforeTest() throws IOException {
- fs = S3ATestUtils.createTestFileSystem(conf);
- fs.mkdirs(new Path(rootPath + "input/"));
-
- yarnCluster = new MiniYARNCluster("MiniClusterWordCount", // testName
- 1, // number of node managers
- 1, // number of local log dirs per node manager
- 1); // number of hdfs dirs per node manager
- yarnCluster.init(conf);
- yarnCluster.start();
- }
-
- @After
- public void afterTest() throws IOException {
- fs.delete(new Path(rootPath), true);
- yarnCluster.stop();
- }
-
- @Test
- public void testWithMiniCluster() throws Exception {
- Path input = new Path(rootPath + "input/in.txt");
- input = input.makeQualified(fs.getUri(), fs.getWorkingDirectory());
- Path output = new Path(rootPath + "output/");
- output = output.makeQualified(fs.getUri(), fs.getWorkingDirectory());
-
- writeStringToFile(input, "first line\nsecond line\nthird line");
-
- Job job = Job.getInstance(conf, "word count");
- job.setJarByClass(WordCount.class);
- job.setMapperClass(WordCount.TokenizerMapper.class);
- job.setCombinerClass(WordCount.IntSumReducer.class);
- job.setReducerClass(WordCount.IntSumReducer.class);
- job.setOutputKeyClass(Text.class);
- job.setOutputValueClass(IntWritable.class);
- FileInputFormat.addInputPath(job, input);
- FileOutputFormat.setOutputPath(job, output);
-
- int exitCode = (job.waitForCompletion(true) ? 0 : 1);
- assertEquals("Returned error code.", 0, exitCode);
-
- assertTrue(fs.exists(new Path(output, "_SUCCESS")));
- String outputAsStr = readStringFromFile(new Path(output, "part-r-00000"));
- Map<String, Integer> resAsMap = getResultAsMap(outputAsStr);
-
- assertEquals(4, resAsMap.size());
- assertEquals(1, (int) resAsMap.get("first"));
- assertEquals(1, (int) resAsMap.get("second"));
- assertEquals(1, (int) resAsMap.get("third"));
- assertEquals(3, (int) resAsMap.get("line"));
- }
-
- /**
- * helper method
- */
- private Map<String, Integer> getResultAsMap(String outputAsStr) throws IOException {
- Map<String, Integer> result = new HashMap<>();
- for (String line : outputAsStr.split("\n")) {
- String[] tokens = line.split("\t");
- result.put(tokens[0], Integer.parseInt(tokens[1]));
- }
- return result;
- }
-
- /**
- * helper method
- */
- private void writeStringToFile(Path path, String string) throws IOException {
- FileContext fc = S3ATestUtils.createTestFileContext(conf);
- try (FSDataOutputStream file = fc.create(path,
- EnumSet.of(CreateFlag.CREATE))) {
- file.write(string.getBytes());
- }
- }
-
- /**
- * helper method
- */
- private String readStringFromFile(Path path) {
- try (FSDataInputStream in = fs.open(path)) {
- long bytesLen = fs.getFileStatus(path).getLen();
- byte[] buffer = new byte[(int) bytesLen];
- IOUtils.readFully(in, buffer, 0, buffer.length);
- return new String(buffer);
- } catch (IOException e) {
- throw new RuntimeException("Failed to read from [" + path + "]", e);
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/ITestInMemoryNativeS3FileSystemContract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/ITestInMemoryNativeS3FileSystemContract.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/ITestInMemoryNativeS3FileSystemContract.java
new file mode 100644
index 0000000..adbf950
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/ITestInMemoryNativeS3FileSystemContract.java
@@ -0,0 +1,33 @@
+/**
+ * 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();
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/ITestJets3tNativeFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/ITestJets3tNativeFileSystemStore.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/ITestJets3tNativeFileSystemStore.java
new file mode 100644
index 0000000..cfe622c
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/ITestJets3tNativeFileSystemStore.java
@@ -0,0 +1,133 @@
+/**
+ * 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
+ }
+ */
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/ITestJets3tNativeS3FileSystemContract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/ITestJets3tNativeS3FileSystemContract.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/ITestJets3tNativeS3FileSystemContract.java
new file mode 100644
index 0000000..e51eaf6
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/ITestJets3tNativeS3FileSystemContract.java
@@ -0,0 +1,33 @@
+/**
+ * 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();
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/TestInMemoryNativeS3FileSystemContract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/TestInMemoryNativeS3FileSystemContract.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/TestInMemoryNativeS3FileSystemContract.java
deleted file mode 100644
index 664d39e..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/TestInMemoryNativeS3FileSystemContract.java
+++ /dev/null
@@ -1,30 +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;
-
-public class TestInMemoryNativeS3FileSystemContract
- extends NativeS3FileSystemContractBaseTest {
-
- @Override
- NativeFileSystemStore getNativeFileSystemStore() throws IOException {
- return new InMemoryNativeFileSystemStore();
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/TestJets3tNativeFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/TestJets3tNativeFileSystemStore.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/TestJets3tNativeFileSystemStore.java
deleted file mode 100644
index dbd476e..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/TestJets3tNativeFileSystemStore.java
+++ /dev/null
@@ -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.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;
-
-
-public class TestJets3tNativeFileSystemStore {
- 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
- }
- */
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/TestJets3tNativeS3FileSystemContract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/TestJets3tNativeS3FileSystemContract.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/TestJets3tNativeS3FileSystemContract.java
deleted file mode 100644
index 42d6f06..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/TestJets3tNativeS3FileSystemContract.java
+++ /dev/null
@@ -1,30 +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;
-
-public class TestJets3tNativeS3FileSystemContract
- extends NativeS3FileSystemContractBaseTest {
-
- @Override
- NativeFileSystemStore getNativeFileSystemStore() throws IOException {
- return new Jets3tNativeFileSystemStore();
- }
-}
---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org
[3/7] hadoop git commit: HADOOP-13446. Support running isolated unit
tests separate from AWS integration tests. Contributed by Chris Nauroth.
Posted by cn...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AFailureHandling.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AFailureHandling.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AFailureHandling.java
deleted file mode 100644
index 83337f0..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AFailureHandling.java
+++ /dev/null
@@ -1,139 +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 org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.InvalidRequestException;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.apache.hadoop.fs.contract.AbstractFSContractTestBase;
-import org.apache.hadoop.fs.contract.s3a.S3AContract;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.EOFException;
-import java.io.FileNotFoundException;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-
-import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
-import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
-import static org.apache.hadoop.fs.s3a.S3AUtils.*;
-
-/**
- * Test S3A Failure translation, including a functional test
- * generating errors during stream IO.
- */
-public class TestS3AFailureHandling extends AbstractFSContractTestBase {
- private static final Logger LOG =
- LoggerFactory.getLogger(TestS3AFailureHandling.class);
-
- @Override
- protected AbstractFSContract createContract(Configuration conf) {
- return new S3AContract(conf);
- }
-
- @Test
- public void testReadFileChanged() throws Throwable {
- describe("overwrite a file with a shorter one during a read, seek");
- final int fullLength = 8192;
- final byte[] fullDataset = dataset(fullLength, 'a', 32);
- final int shortLen = 4096;
- final byte[] shortDataset = dataset(shortLen, 'A', 32);
- final FileSystem fs = getFileSystem();
- final Path testpath = path("readFileToChange.txt");
- // initial write
- writeDataset(fs, testpath, fullDataset, fullDataset.length, 1024, false);
- try(FSDataInputStream instream = fs.open(testpath)) {
- instream.seek(fullLength - 16);
- assertTrue("no data to read", instream.read() >= 0);
- // overwrite
- writeDataset(fs, testpath, shortDataset, shortDataset.length, 1024, true);
- // here the file length is less. Probe the file to see if this is true,
- // with a spin and wait
- eventually(30 *1000, new Callable<Void>() {
- @Override
- public Void call() throws Exception {
- assertEquals(shortLen, fs.getFileStatus(testpath).getLen());
- return null;
- }
- });
- // here length is shorter. Assuming it has propagated to all replicas,
- // the position of the input stream is now beyond the EOF.
- // An attempt to seek backwards to a position greater than the
- // short length will raise an exception from AWS S3, which must be
- // translated into an EOF
-
- instream.seek(shortLen + 1024);
- int c = instream.read();
- assertIsEOF("read()", c);
-
- byte[] buf = new byte[256];
-
- assertIsEOF("read(buffer)", instream.read(buf));
- assertIsEOF("read(offset)",
- instream.read(instream.getPos(), buf, 0, buf.length));
-
- // now do a block read fully, again, backwards from the current pos
- try {
- instream.readFully(shortLen + 512, buf);
- fail("Expected readFully to fail");
- } catch (EOFException expected) {
- LOG.debug("Expected EOF: ", expected);
- }
-
- assertIsEOF("read(offset)",
- instream.read(shortLen + 510, buf, 0, buf.length));
-
- // seek somewhere useful
- instream.seek(shortLen - 256);
-
- // delete the file. Reads must fail
- fs.delete(testpath, false);
-
- try {
- int r = instream.read();
- fail("Expected an exception, got " + r);
- } catch (FileNotFoundException e) {
- // expected
- }
-
- try {
- instream.readFully(2048, buf);
- fail("Expected readFully to fail");
- } catch (FileNotFoundException e) {
- // expected
- }
-
- }
- }
-
- /**
- * Assert that a read operation returned an EOF value.
- * @param operation specific operation
- * @param readResult result
- */
- private void assertIsEOF(String operation, int readResult) {
- assertEquals("Expected EOF from "+ operation
- + "; got char " + (char) readResult, -1, readResult);
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AFastOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AFastOutputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AFastOutputStream.java
deleted file mode 100644
index e507cf6..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AFastOutputStream.java
+++ /dev/null
@@ -1,74 +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 org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.Timeout;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-
-import java.io.IOException;
-
-/**
- * Tests regular and multi-part upload functionality for S3AFastOutputStream.
- * File sizes are kept small to reduce test duration on slow connections
- */
-public class TestS3AFastOutputStream {
- private FileSystem fs;
-
-
- @Rule
- public Timeout testTimeout = new Timeout(30 * 60 * 1000);
-
- @Before
- public void setUp() throws Exception {
- Configuration conf = new Configuration();
- conf.setLong(Constants.MIN_MULTIPART_THRESHOLD, 5 * 1024 * 1024);
- conf.setInt(Constants.MULTIPART_SIZE, 5 * 1024 * 1024);
- conf.setBoolean(Constants.FAST_UPLOAD, true);
- fs = S3ATestUtils.createTestFileSystem(conf);
- }
-
- @After
- public void tearDown() throws Exception {
- if (fs != null) {
- fs.delete(getTestPath(), true);
- }
- }
-
- protected Path getTestPath() {
- return new Path("/tests3a");
- }
-
- @Test
- public void testRegularUpload() throws IOException {
- ContractTestUtils.createAndVerifyFile(fs, getTestPath(), 1024 * 1024);
- }
-
- @Test
- public void testMultiPartUpload() throws IOException {
- ContractTestUtils.createAndVerifyFile(fs, getTestPath(), 6 * 1024 *
- 1024);
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AFileOperationCost.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AFileOperationCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AFileOperationCost.java
deleted file mode 100644
index 179eb88..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AFileOperationCost.java
+++ /dev/null
@@ -1,277 +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 org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.apache.hadoop.fs.contract.AbstractFSContractTestBase;
-import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.apache.hadoop.fs.contract.s3a.S3AContract;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.net.URI;
-
-import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
-import static org.apache.hadoop.fs.s3a.Statistic.*;
-import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
-import static org.apache.hadoop.fs.s3a.S3ATestUtils.MetricDiff;
-
-/**
- * Use metrics to assert about the cost of file status queries.
- * {@link S3AFileSystem#getFileStatus(Path)}.
- */
-public class TestS3AFileOperationCost extends AbstractFSContractTestBase {
-
- private MetricDiff metadataRequests;
- private MetricDiff listRequests;
-
- private static final Logger LOG =
- LoggerFactory.getLogger(TestS3AFileOperationCost.class);
-
- @Override
- protected AbstractFSContract createContract(Configuration conf) {
- return new S3AContract(conf);
- }
-
- @Override
- public S3AFileSystem getFileSystem() {
- return (S3AFileSystem) super.getFileSystem();
- }
-
- @Override
- public void setup() throws Exception {
- super.setup();
- S3AFileSystem fs = getFileSystem();
- metadataRequests = new MetricDiff(fs, OBJECT_METADATA_REQUESTS);
- listRequests = new MetricDiff(fs, OBJECT_LIST_REQUESTS);
- }
-
- @Test
- public void testCostOfGetFileStatusOnFile() throws Throwable {
- describe("performing getFileStatus on a file");
- Path simpleFile = path("simple.txt");
- S3AFileSystem fs = getFileSystem();
- touch(fs, simpleFile);
- resetMetricDiffs();
- S3AFileStatus status = fs.getFileStatus(simpleFile);
- assertTrue("not a file: " + status, status.isFile());
- metadataRequests.assertDiffEquals(1);
- listRequests.assertDiffEquals(0);
- }
-
- private void resetMetricDiffs() {
- reset(metadataRequests, listRequests);
- }
-
- @Test
- public void testCostOfGetFileStatusOnEmptyDir() throws Throwable {
- describe("performing getFileStatus on an empty directory");
- S3AFileSystem fs = getFileSystem();
- Path dir = path("empty");
- fs.mkdirs(dir);
- resetMetricDiffs();
- S3AFileStatus status = fs.getFileStatus(dir);
- assertTrue("not empty: " + status, status.isEmptyDirectory());
- metadataRequests.assertDiffEquals(2);
- listRequests.assertDiffEquals(0);
- }
-
- @Test
- public void testCostOfGetFileStatusOnMissingFile() throws Throwable {
- describe("performing getFileStatus on a missing file");
- S3AFileSystem fs = getFileSystem();
- Path path = path("missing");
- resetMetricDiffs();
- try {
- S3AFileStatus status = fs.getFileStatus(path);
- fail("Got a status back from a missing file path " + status);
- } catch (FileNotFoundException expected) {
- // expected
- }
- metadataRequests.assertDiffEquals(2);
- listRequests.assertDiffEquals(1);
- }
-
- @Test
- public void testCostOfGetFileStatusOnMissingSubPath() throws Throwable {
- describe("performing getFileStatus on a missing file");
- S3AFileSystem fs = getFileSystem();
- Path path = path("missingdir/missingpath");
- resetMetricDiffs();
- try {
- S3AFileStatus status = fs.getFileStatus(path);
- fail("Got a status back from a missing file path " + status);
- } catch (FileNotFoundException expected) {
- // expected
- }
- metadataRequests.assertDiffEquals(2);
- listRequests.assertDiffEquals(1);
- }
-
- @Test
- public void testCostOfGetFileStatusOnNonEmptyDir() throws Throwable {
- describe("performing getFileStatus on a non-empty directory");
- S3AFileSystem fs = getFileSystem();
- Path dir = path("empty");
- fs.mkdirs(dir);
- Path simpleFile = new Path(dir, "simple.txt");
- touch(fs, simpleFile);
- resetMetricDiffs();
- S3AFileStatus status = fs.getFileStatus(dir);
- if (status.isEmptyDirectory()) {
- // erroneous state
- String fsState = fs.toString();
- fail("FileStatus says directory isempty: " + status
- + "\n" + ContractTestUtils.ls(fs, dir)
- + "\n" + fsState);
- }
- metadataRequests.assertDiffEquals(2);
- listRequests.assertDiffEquals(1);
- }
-
- @Test
- public void testCostOfCopyFromLocalFile() throws Throwable {
- describe("testCostOfCopyFromLocalFile");
- String testDirProp = System.getProperty("test.build.data",
- "target" + File.separator + "test" + File.separator + "data");
- File localTestDir = new File(testDirProp, "tmp").getAbsoluteFile();
- localTestDir.mkdirs();
- File tmpFile = File.createTempFile("tests3acost", ".txt",
- localTestDir);
- tmpFile.delete();
- try {
- URI localFileURI = tmpFile.toURI();
- FileSystem localFS = FileSystem.get(localFileURI,
- getFileSystem().getConf());
- Path localPath = new Path(localFileURI);
- int len = 10 * 1024;
- byte[] data = dataset(len, 'A', 'Z');
- writeDataset(localFS, localPath, data, len, 1024, true);
- S3AFileSystem s3a = getFileSystem();
- MetricDiff copyLocalOps = new MetricDiff(s3a,
- INVOCATION_COPY_FROM_LOCAL_FILE);
- MetricDiff putRequests = new MetricDiff(s3a,
- OBJECT_PUT_REQUESTS);
- MetricDiff putBytes = new MetricDiff(s3a,
- OBJECT_PUT_BYTES);
-
- Path remotePath = path("copied");
- s3a.copyFromLocalFile(false, true, localPath, remotePath);
- verifyFileContents(s3a, remotePath, data);
- copyLocalOps.assertDiffEquals(1);
- putRequests.assertDiffEquals(1);
- putBytes.assertDiffEquals(len);
- // print final stats
- LOG.info("Filesystem {}", s3a);
- } finally {
- tmpFile.delete();
- }
- }
-
- private void reset(MetricDiff... diffs) {
- for (MetricDiff diff : diffs) {
- diff.reset();
- }
- }
-
- @Test
- public void testFakeDirectoryDeletion() throws Throwable {
- describe("Verify whether create file works after renaming a file. "
- + "In S3, rename deletes any fake directories as a part of "
- + "clean up activity");
- S3AFileSystem fs = getFileSystem();
- Path srcBaseDir = path("src");
- mkdirs(srcBaseDir);
- MetricDiff deleteRequests =
- new MetricDiff(fs, Statistic.OBJECT_DELETE_REQUESTS);
- MetricDiff directoriesDeleted =
- new MetricDiff(fs, Statistic.DIRECTORIES_DELETED);
- MetricDiff fakeDirectoriesDeleted =
- new MetricDiff(fs, Statistic.FAKE_DIRECTORIES_DELETED);
- MetricDiff directoriesCreated =
- new MetricDiff(fs, Statistic.DIRECTORIES_CREATED);
-
- Path srcDir = new Path(srcBaseDir, "1/2/3/4/5/6");
- Path srcFilePath = new Path(srcDir, "source.txt");
- int srcDirDepth = directoriesInPath(srcDir);
- // one dir created, one removed
- mkdirs(srcDir);
- String state = "after mkdir(srcDir)";
- directoriesCreated.assertDiffEquals(state, 1);
-/* TODO: uncomment once HADOOP-13222 is in
- deleteRequests.assertDiffEquals(state, 1);
- directoriesDeleted.assertDiffEquals(state, 0);
- fakeDirectoriesDeleted.assertDiffEquals(state, srcDirDepth);
-*/
- reset(deleteRequests, directoriesCreated, directoriesDeleted,
- fakeDirectoriesDeleted);
-
- // creating a file should trigger demise of the src dir
- touch(fs, srcFilePath);
- state = "after touch(fs, srcFilePath)";
- deleteRequests.assertDiffEquals(state, 1);
- directoriesCreated.assertDiffEquals(state, 0);
- directoriesDeleted.assertDiffEquals(state, 0);
- fakeDirectoriesDeleted.assertDiffEquals(state, srcDirDepth);
-
- reset(deleteRequests, directoriesCreated, directoriesDeleted,
- fakeDirectoriesDeleted);
-
- Path destBaseDir = path("dest");
- Path destDir = new Path(destBaseDir, "1/2/3/4/5/6");
- Path destFilePath = new Path(destDir, "dest.txt");
- mkdirs(destDir);
- state = "after mkdir(destDir)";
-
- int destDirDepth = directoriesInPath(destDir);
- directoriesCreated.assertDiffEquals(state, 1);
-/* TODO: uncomment once HADOOP-13222 is in
- deleteRequests.assertDiffEquals(state,1);
- directoriesDeleted.assertDiffEquals(state,0);
- fakeDirectoriesDeleted.assertDiffEquals(state,destDirDepth);
-*/
- reset(deleteRequests, directoriesCreated, directoriesDeleted,
- fakeDirectoriesDeleted);
-
- fs.rename(srcFilePath, destFilePath);
- state = "after rename(srcFilePath, destFilePath)";
- directoriesCreated.assertDiffEquals(state, 1);
- // one for the renamed file, one for the parent
- deleteRequests.assertDiffEquals(state, 2);
- directoriesDeleted.assertDiffEquals(state, 0);
- fakeDirectoriesDeleted.assertDiffEquals(state, destDirDepth);
-
- reset(deleteRequests, directoriesCreated, directoriesDeleted,
- fakeDirectoriesDeleted);
-
- assertIsFile(destFilePath);
- assertIsDirectory(srcDir);
- }
-
- private int directoriesInPath(Path path) {
- return path.isRoot() ? 0 : 1 + directoriesInPath(path.getParent());
- }
-
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AFileSystemContract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AFileSystemContract.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AFileSystemContract.java
deleted file mode 100644
index 6315a80..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AFileSystemContract.java
+++ /dev/null
@@ -1,104 +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 org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystemContractBaseTest;
-import org.apache.hadoop.fs.Path;
-
-/**
- * Tests a live S3 system. If your keys and bucket aren't specified, all tests
- * are marked as passed.
- *
- * This uses BlockJUnit4ClassRunner because FileSystemContractBaseTest from
- * TestCase which uses the old Junit3 runner that doesn't ignore assumptions
- * properly making it impossible to skip the tests if we don't have a valid
- * bucket.
- **/
-public class TestS3AFileSystemContract extends FileSystemContractBaseTest {
-
- protected static final Logger LOG =
- LoggerFactory.getLogger(TestS3AFileSystemContract.class);
-
- @Override
- public void setUp() throws Exception {
- Configuration conf = new Configuration();
-
- fs = S3ATestUtils.createTestFileSystem(conf);
- super.setUp();
- }
-
- @Override
- protected void tearDown() throws Exception {
- if (fs != null) {
- fs.delete(path("test"), true);
- }
- super.tearDown();
- }
-
- @Override
- public void testMkdirsWithUmask() throws Exception {
- // not supported
- }
-
- @Override
- public void testRenameFileAsExistingFile() throws Exception {
- if (!renameSupported()) return;
-
- Path src = path("/test/hadoop/file");
- createFile(src);
- Path dst = path("/test/new/newfile");
- createFile(dst);
- // s3 doesn't support rename option
- // rename-overwrites-dest is always allowed.
- rename(src, dst, true, false, true);
- }
-
- @Override
- public void testRenameDirectoryAsExistingDirectory() throws Exception {
- if (!renameSupported()) {
- return;
- }
-
- Path src = path("/test/hadoop/dir");
- fs.mkdirs(src);
- createFile(path("/test/hadoop/dir/file1"));
- createFile(path("/test/hadoop/dir/subdir/file2"));
-
- Path dst = path("/test/new/newdir");
- fs.mkdirs(dst);
- rename(src, dst, true, false, true);
- assertFalse("Nested file1 exists",
- fs.exists(path("/test/hadoop/dir/file1")));
- assertFalse("Nested file2 exists",
- fs.exists(path("/test/hadoop/dir/subdir/file2")));
- assertTrue("Renamed nested file1 exists",
- fs.exists(path("/test/new/newdir/file1")));
- assertTrue("Renamed nested exists",
- fs.exists(path("/test/new/newdir/subdir/file2")));
- }
-
-// @Override
- public void testMoveDirUnderParent() throws Throwable {
- // not support because
- // Fails if dst is a directory that is not empty.
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ATemporaryCredentials.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ATemporaryCredentials.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ATemporaryCredentials.java
deleted file mode 100644
index 957dfb9..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ATemporaryCredentials.java
+++ /dev/null
@@ -1,148 +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 java.io.IOException;
-import java.net.URI;
-
-import com.amazonaws.auth.AWSCredentials;
-import com.amazonaws.auth.AWSCredentialsProvider;
-import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient;
-import com.amazonaws.services.securitytoken.model.GetSessionTokenRequest;
-import com.amazonaws.services.securitytoken.model.GetSessionTokenResult;
-import com.amazonaws.services.securitytoken.model.Credentials;
-
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.apache.hadoop.fs.contract.AbstractFSContractTestBase;
-import org.apache.hadoop.fs.contract.s3a.S3AContract;
-import org.apache.hadoop.fs.s3native.S3xLoginHelper;
-import org.apache.hadoop.conf.Configuration;
-
-import org.junit.Test;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
-import static org.apache.hadoop.fs.s3a.Constants.*;
-
-/**
- * Tests use of temporary credentials (for example, AWS STS & S3).
- * This test extends a class that "does things to the root directory", and
- * should only be used against transient filesystems where you don't care about
- * the data.
- */
-public class TestS3ATemporaryCredentials extends AbstractFSContractTestBase {
- public static final String TEST_STS_ENABLED = "test.fs.s3a.sts.enabled";
- public static final String TEST_STS_ENDPOINT = "test.fs.s3a.sts.endpoint";
-
- private static final Logger LOG =
- LoggerFactory.getLogger(TestS3ATemporaryCredentials.class);
-
- private static final String PROVIDER_CLASS
- = TemporaryAWSCredentialsProvider.NAME;
-
- private static final long TEST_FILE_SIZE = 1024;
-
- @Override
- protected AbstractFSContract createContract(Configuration conf) {
- return new S3AContract(conf);
- }
-
- /**
- * Test use of STS for requesting temporary credentials.
- *
- * The property test.sts.endpoint can be set to point this at different
- * STS endpoints. This test will use the AWS credentials (if provided) for
- * S3A tests to request temporary credentials, then attempt to use those
- * credentials instead.
- *
- * @throws IOException
- */
- @Test
- public void testSTS() throws IOException {
- Configuration conf = getContract().getConf();
- if (!conf.getBoolean(TEST_STS_ENABLED, true)) {
- skip("STS functional tests disabled");
- }
-
- S3xLoginHelper.Login login = S3AUtils.getAWSAccessKeys(
- URI.create("s3a://foobar"), conf);
- if (!login.hasLogin()) {
- skip("testSTS disabled because AWS credentials not configured");
- }
- AWSCredentialsProvider parentCredentials = new BasicAWSCredentialsProvider(
- login.getUser(), login.getPassword());
-
- String stsEndpoint = conf.getTrimmed(TEST_STS_ENDPOINT, "");
- AWSSecurityTokenServiceClient stsClient;
- stsClient = new AWSSecurityTokenServiceClient(parentCredentials);
- if (!stsEndpoint.isEmpty()) {
- LOG.debug("STS Endpoint ={}", stsEndpoint);
- stsClient.setEndpoint(stsEndpoint);
- }
- GetSessionTokenRequest sessionTokenRequest = new GetSessionTokenRequest();
- sessionTokenRequest.setDurationSeconds(900);
- GetSessionTokenResult sessionTokenResult;
- sessionTokenResult = stsClient.getSessionToken(sessionTokenRequest);
- Credentials sessionCreds = sessionTokenResult.getCredentials();
-
- String childAccessKey = sessionCreds.getAccessKeyId();
- conf.set(ACCESS_KEY, childAccessKey);
- String childSecretKey = sessionCreds.getSecretAccessKey();
- conf.set(SECRET_KEY, childSecretKey);
- String sessionToken = sessionCreds.getSessionToken();
- conf.set(SESSION_TOKEN, sessionToken);
-
- conf.set(AWS_CREDENTIALS_PROVIDER, PROVIDER_CLASS);
-
- try(S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) {
- createAndVerifyFile(fs, path("testSTS"), TEST_FILE_SIZE);
- }
-
- // now create an invalid set of credentials by changing the session
- // token
- conf.set(SESSION_TOKEN, "invalid-" + sessionToken);
- try (S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) {
- createAndVerifyFile(fs, path("testSTSInvalidToken"), TEST_FILE_SIZE);
- fail("Expected an access exception, but file access to "
- + fs.getUri() + " was allowed: " + fs);
- } catch (AWSS3IOException ex) {
- LOG.info("Expected Exception: {}", ex.toString());
- LOG.debug("Expected Exception: {}", ex, ex);
- }
- }
-
- @Test
- public void testTemporaryCredentialValidation() throws Throwable {
- Configuration conf = new Configuration();
- conf.set(ACCESS_KEY, "accesskey");
- conf.set(SECRET_KEY, "secretkey");
- conf.set(SESSION_TOKEN, "");
- TemporaryAWSCredentialsProvider provider
- = new TemporaryAWSCredentialsProvider(getFileSystem().getUri(), conf);
- try {
- AWSCredentials credentials = provider.getCredentials();
- fail("Expected a CredentialInitializationException,"
- + " got " + credentials);
- } catch (CredentialInitializationException expected) {
- // expected
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContext.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContext.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContext.java
new file mode 100644
index 0000000..7e4273a
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContext.java
@@ -0,0 +1,23 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.fs.s3a.fileContext;
+
+import org.apache.hadoop.fs.TestFileContext;
+
+/**
+ * Implementation of TestFileContext for S3a.
+ */
+public class ITestS3AFileContext extends TestFileContext{
+
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextCreateMkdir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextCreateMkdir.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextCreateMkdir.java
new file mode 100644
index 0000000..4b8d4bb
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextCreateMkdir.java
@@ -0,0 +1,35 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.fs.s3a.fileContext;
+
+import java.io.IOException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContextCreateMkdirBaseTest;
+import org.apache.hadoop.fs.s3a.S3ATestUtils;
+import org.junit.Before;
+
+/**
+ * Extends FileContextCreateMkdirBaseTest for a S3a FileContext.
+ */
+public class ITestS3AFileContextCreateMkdir
+ extends FileContextCreateMkdirBaseTest {
+
+ @Before
+ public void setUp() throws IOException, Exception {
+ Configuration conf = new Configuration();
+ fc = S3ATestUtils.createTestFileContext(conf);
+ super.setUp();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextMainOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextMainOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextMainOperations.java
new file mode 100644
index 0000000..7247c16
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextMainOperations.java
@@ -0,0 +1,60 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.fs.s3a.fileContext;
+
+import java.io.IOException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContextMainOperationsBaseTest;
+import org.apache.hadoop.fs.s3a.S3ATestUtils;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+
+/**
+ * S3A implementation of FileContextMainOperationsBaseTest.
+ */
+public class ITestS3AFileContextMainOperations
+ extends FileContextMainOperationsBaseTest {
+
+ @Before
+ public void setUp() throws IOException, Exception {
+ Configuration conf = new Configuration();
+ fc = S3ATestUtils.createTestFileContext(conf);
+ super.setUp();
+ }
+
+ @Override
+ protected boolean listCorruptedBlocksSupported() {
+ return false;
+ }
+
+ @Test
+ @Ignore
+ public void testCreateFlagAppendExistingFile() throws IOException {
+ //append not supported, so test removed
+ }
+
+ @Test
+ @Ignore
+ public void testCreateFlagCreateAppendExistingFile() throws IOException {
+ //append not supported, so test removed
+ }
+
+ @Test
+ @Ignore
+ public void testSetVerifyChecksum() throws IOException {
+ //checksums ignored, so test removed
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java
new file mode 100644
index 0000000..5e88aba
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java
@@ -0,0 +1,61 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.fs.s3a.fileContext;
+
+import java.net.URI;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FCStatisticsBaseTest;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.s3a.S3ATestUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+
+/**
+ * S3a implementation of FCStatisticsBaseTest.
+ */
+public class ITestS3AFileContextStatistics extends FCStatisticsBaseTest {
+
+ @Before
+ public void setUp() throws Exception {
+ Configuration conf = new Configuration();
+ fc = S3ATestUtils.createTestFileContext(conf);
+ fc.mkdir(fileContextTestHelper.getTestRootPath(fc, "test"),
+ FileContext.DEFAULT_PERM, true);
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ fc.delete(fileContextTestHelper.getTestRootPath(fc, "test"), true);
+ }
+
+ @Override
+ protected void verifyReadBytes(FileSystem.Statistics stats) {
+ // one blockSize for read, one for pread
+ Assert.assertEquals(2 * blockSize, stats.getBytesRead());
+ }
+
+ @Override
+ protected void verifyWrittenBytes(FileSystem.Statistics stats) {
+ //No extra bytes are written
+ Assert.assertEquals(blockSize, stats.getBytesWritten());
+ }
+
+ @Override
+ protected URI getFsUri() {
+ return fc.getHomeDirectory().toUri();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextURI.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextURI.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextURI.java
new file mode 100644
index 0000000..fff1fcb
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextURI.java
@@ -0,0 +1,44 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.fs.s3a.fileContext;
+
+import java.io.IOException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContextURIBase;
+import org.apache.hadoop.fs.s3a.S3ATestUtils;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+
+/**
+ * S3a implementation of FileContextURIBase.
+ */
+public class ITestS3AFileContextURI extends FileContextURIBase {
+
+ @Before
+ public void setUp() throws IOException, Exception {
+ Configuration conf = new Configuration();
+ fc1 = S3ATestUtils.createTestFileContext(conf);
+ fc2 = S3ATestUtils.createTestFileContext(conf); //different object, same FS
+ super.setUp();
+ }
+
+ @Test
+ @Ignore
+ public void testFileStatus() throws IOException {
+ // test disabled
+ // (the statistics tested with this method are not relevant for an S3FS)
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextUtil.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextUtil.java
new file mode 100644
index 0000000..d0312ba
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextUtil.java
@@ -0,0 +1,34 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.fs.s3a.fileContext;
+
+import java.io.IOException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContextUtilBase;
+import org.apache.hadoop.fs.s3a.S3ATestUtils;
+import org.junit.Before;
+
+/**
+ * S3A implementation of FileContextUtilBase.
+ */
+public class ITestS3AFileContextUtil extends FileContextUtilBase {
+
+ @Before
+ public void setUp() throws IOException, Exception {
+ Configuration conf = new Configuration();
+ fc = S3ATestUtils.createTestFileContext(conf);
+ super.setUp();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/TestS3AFileContext.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/TestS3AFileContext.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/TestS3AFileContext.java
deleted file mode 100644
index 34d78a5..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/TestS3AFileContext.java
+++ /dev/null
@@ -1,23 +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.
- */
-package org.apache.hadoop.fs.s3a.fileContext;
-
-import org.apache.hadoop.fs.TestFileContext;
-
-/**
- * Implementation of TestFileContext for S3a
- */
-public class TestS3AFileContext extends TestFileContext{
-
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/TestS3AFileContextCreateMkdir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/TestS3AFileContextCreateMkdir.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/TestS3AFileContextCreateMkdir.java
deleted file mode 100644
index b0c4d84..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/TestS3AFileContextCreateMkdir.java
+++ /dev/null
@@ -1,35 +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.
- */
-package org.apache.hadoop.fs.s3a.fileContext;
-
-import java.io.IOException;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileContextCreateMkdirBaseTest;
-import org.apache.hadoop.fs.s3a.S3ATestUtils;
-import org.junit.Before;
-
-/**
- * Extends FileContextCreateMkdirBaseTest for a S3a FileContext
- */
-public class TestS3AFileContextCreateMkdir
- extends FileContextCreateMkdirBaseTest {
-
- @Before
- public void setUp() throws IOException, Exception {
- Configuration conf = new Configuration();
- fc = S3ATestUtils.createTestFileContext(conf);
- super.setUp();
- }
-
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/TestS3AFileContextMainOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/TestS3AFileContextMainOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/TestS3AFileContextMainOperations.java
deleted file mode 100644
index 4d200d1..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/TestS3AFileContextMainOperations.java
+++ /dev/null
@@ -1,60 +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.
- */
-package org.apache.hadoop.fs.s3a.fileContext;
-
-import java.io.IOException;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileContextMainOperationsBaseTest;
-import org.apache.hadoop.fs.s3a.S3ATestUtils;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-
-/**
- * S3A implementation of FileContextMainOperationsBaseTest
- */
-public class TestS3AFileContextMainOperations
- extends FileContextMainOperationsBaseTest {
-
- @Before
- public void setUp() throws IOException, Exception {
- Configuration conf = new Configuration();
- fc = S3ATestUtils.createTestFileContext(conf);
- super.setUp();
- }
-
- @Override
- protected boolean listCorruptedBlocksSupported() {
- return false;
- }
-
- @Test
- @Ignore
- public void testCreateFlagAppendExistingFile() throws IOException {
- //append not supported, so test removed
- }
-
- @Test
- @Ignore
- public void testCreateFlagCreateAppendExistingFile() throws IOException {
- //append not supported, so test removed
- }
-
- @Test
- @Ignore
- public void testSetVerifyChecksum() throws IOException {
- //checksums ignored, so test removed
- }
-
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/TestS3AFileContextStatistics.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/TestS3AFileContextStatistics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/TestS3AFileContextStatistics.java
deleted file mode 100644
index a9f4848..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/TestS3AFileContextStatistics.java
+++ /dev/null
@@ -1,60 +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.
- */
-package org.apache.hadoop.fs.s3a.fileContext;
-
-import java.net.URI;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FCStatisticsBaseTest;
-import org.apache.hadoop.fs.FileContext;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.s3a.S3ATestUtils;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-
-/**
- * S3a implementation of FCStatisticsBaseTest
- */
-public class TestS3AFileContextStatistics extends FCStatisticsBaseTest {
-
- @Before
- public void setUp() throws Exception {
- Configuration conf = new Configuration();
- fc = S3ATestUtils.createTestFileContext(conf);
- fc.mkdir(fileContextTestHelper.getTestRootPath(fc, "test"), FileContext.DEFAULT_PERM, true);
- }
-
- @After
- public void tearDown() throws Exception {
- fc.delete(fileContextTestHelper.getTestRootPath(fc, "test"), true);
- }
-
- @Override
- protected void verifyReadBytes(FileSystem.Statistics stats) {
- // one blockSize for read, one for pread
- Assert.assertEquals(2 * blockSize, stats.getBytesRead());
- }
-
- @Override
- protected void verifyWrittenBytes(FileSystem.Statistics stats) {
- //No extra bytes are written
- Assert.assertEquals(blockSize, stats.getBytesWritten());
- }
-
- @Override
- protected URI getFsUri() {
- return fc.getHomeDirectory().toUri();
- }
-
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/TestS3AFileContextURI.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/TestS3AFileContextURI.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/TestS3AFileContextURI.java
deleted file mode 100644
index 3da7b19..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/TestS3AFileContextURI.java
+++ /dev/null
@@ -1,43 +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.
- */
-package org.apache.hadoop.fs.s3a.fileContext;
-
-import java.io.IOException;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileContextURIBase;
-import org.apache.hadoop.fs.s3a.S3ATestUtils;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-
-/**
- * S3a implementation of FileContextURIBase
- */
-public class TestS3AFileContextURI extends FileContextURIBase {
-
- @Before
- public void setUp() throws IOException, Exception {
- Configuration conf = new Configuration();
- fc1 = S3ATestUtils.createTestFileContext(conf);
- fc2 = S3ATestUtils.createTestFileContext(conf); //different object, same FS
- super.setUp();
- }
-
- @Test
- @Ignore
- public void testFileStatus() throws IOException {
- //test disabled (the statistics tested with this method are not relevant for an S3FS)
- }
-
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/TestS3AFileContextUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/TestS3AFileContextUtil.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/TestS3AFileContextUtil.java
deleted file mode 100644
index 666f4c2..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/TestS3AFileContextUtil.java
+++ /dev/null
@@ -1,34 +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.
- */
-package org.apache.hadoop.fs.s3a.fileContext;
-
-import java.io.IOException;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileContextUtilBase;
-import org.apache.hadoop.fs.s3a.S3ATestUtils;
-import org.junit.Before;
-
-/**
- * S3A implementation of FileContextUtilBase
- */
-public class TestS3AFileContextUtil extends FileContextUtilBase {
-
- @Before
- public void setUp() throws IOException, Exception {
- Configuration conf = new Configuration();
- fc = S3ATestUtils.createTestFileContext(conf);
- super.setUp();
- }
-
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADeleteFilesOneByOne.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADeleteFilesOneByOne.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADeleteFilesOneByOne.java
new file mode 100644
index 0000000..a375664
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADeleteFilesOneByOne.java
@@ -0,0 +1,44 @@
+/*
+ * 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;
+import org.junit.Test;
+
+import java.io.IOException;
+
+/**
+ * Tests file deletion with multi-delete disabled.
+ */
+public class ITestS3ADeleteFilesOneByOne extends ITestS3ADeleteManyFiles {
+
+ @Override
+ protected Configuration createConfiguration() {
+ Configuration configuration = super.createConfiguration();
+ configuration.setBoolean(Constants.ENABLE_MULTI_DELETE, false);
+ return configuration;
+ }
+
+ @Override
+ @Test
+ public void testOpenCreate() throws IOException {
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADeleteManyFiles.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADeleteManyFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADeleteManyFiles.java
new file mode 100644
index 0000000..542dc12
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADeleteManyFiles.java
@@ -0,0 +1,126 @@
+/*
+ * 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.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+/**
+ * Test some scalable operations related to file renaming and deletion.
+ */
+public class ITestS3ADeleteManyFiles extends S3AScaleTestBase {
+ private static final Logger LOG =
+ LoggerFactory.getLogger(ITestS3ADeleteManyFiles.class);
+
+ @Test
+ public void testBulkRenameAndDelete() throws Throwable {
+ final Path scaleTestDir = getTestPath();
+ final Path srcDir = new Path(scaleTestDir, "src");
+ final Path finalDir = new Path(scaleTestDir, "final");
+ final long count = getOperationCount();
+ ContractTestUtils.rm(fs, scaleTestDir, true, false);
+
+ fs.mkdirs(srcDir);
+ fs.mkdirs(finalDir);
+
+ int testBufferSize = fs.getConf()
+ .getInt(ContractTestUtils.IO_CHUNK_BUFFER_SIZE,
+ ContractTestUtils.DEFAULT_IO_CHUNK_BUFFER_SIZE);
+ // use Executor to speed up file creation
+ ExecutorService exec = Executors.newFixedThreadPool(16);
+ final ExecutorCompletionService<Boolean> completionService =
+ new ExecutorCompletionService<>(exec);
+ try {
+ final byte[] data = ContractTestUtils.dataset(testBufferSize, 'a', 'z');
+
+ for (int i = 0; i < count; ++i) {
+ final String fileName = "foo-" + i;
+ completionService.submit(new Callable<Boolean>() {
+ @Override
+ public Boolean call() throws IOException {
+ ContractTestUtils.createFile(fs, new Path(srcDir, fileName),
+ false, data);
+ return fs.exists(new Path(srcDir, fileName));
+ }
+ });
+ }
+ for (int i = 0; i < count; ++i) {
+ final Future<Boolean> future = completionService.take();
+ try {
+ if (!future.get()) {
+ LOG.warn("cannot create file");
+ }
+ } catch (ExecutionException e) {
+ LOG.warn("Error while uploading file", e.getCause());
+ throw e;
+ }
+ }
+ } finally {
+ exec.shutdown();
+ }
+
+ int nSrcFiles = fs.listStatus(srcDir).length;
+ fs.rename(srcDir, finalDir);
+ assertEquals(nSrcFiles, fs.listStatus(finalDir).length);
+ ContractTestUtils.assertPathDoesNotExist(fs, "not deleted after rename",
+ new Path(srcDir, "foo-" + 0));
+ ContractTestUtils.assertPathDoesNotExist(fs, "not deleted after rename",
+ new Path(srcDir, "foo-" + count / 2));
+ ContractTestUtils.assertPathDoesNotExist(fs, "not deleted after rename",
+ new Path(srcDir, "foo-" + (count - 1)));
+ ContractTestUtils.assertPathExists(fs, "not renamed to dest dir",
+ new Path(finalDir, "foo-" + 0));
+ ContractTestUtils.assertPathExists(fs, "not renamed to dest dir",
+ new Path(finalDir, "foo-" + count/2));
+ ContractTestUtils.assertPathExists(fs, "not renamed to dest dir",
+ new Path(finalDir, "foo-" + (count-1)));
+
+ ContractTestUtils.assertDeleted(fs, finalDir, true, false);
+ }
+
+ @Test
+ public void testOpenCreate() throws IOException {
+ Path dir = new Path("/tests3a");
+ ContractTestUtils.createAndVerifyFile(fs, dir, 1024);
+ ContractTestUtils.createAndVerifyFile(fs, dir, 5 * 1024 * 1024);
+ ContractTestUtils.createAndVerifyFile(fs, dir, 20 * 1024 * 1024);
+
+
+ /*
+ Enable to test the multipart upload
+ try {
+ ContractTestUtils.createAndVerifyFile(fs, dir,
+ (long)6 * 1024 * 1024 * 1024);
+ } catch (IOException e) {
+ fail(e.getMessage());
+ }
+ */
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java
new file mode 100644
index 0000000..b5f4eb3
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java
@@ -0,0 +1,187 @@
+/*
+ * 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.fs.Path;
+import org.apache.hadoop.fs.s3a.Statistic;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.s3a.Statistic.*;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
+
+/**
+ * Test the performance of listing files/directories.
+ */
+public class ITestS3ADirectoryPerformance extends S3AScaleTestBase {
+ private static final Logger LOG = LoggerFactory.getLogger(
+ ITestS3ADirectoryPerformance.class);
+
+ @Test
+ public void testListOperations() throws Throwable {
+ describe("Test recursive list operations");
+ final Path scaleTestDir = getTestPath();
+ final Path listDir = new Path(scaleTestDir, "lists");
+
+ // scale factor.
+ int scale = getConf().getInt(KEY_DIRECTORY_COUNT, DEFAULT_DIRECTORY_COUNT);
+ int width = scale;
+ int depth = scale;
+ int files = scale;
+ MetricDiff metadataRequests = new MetricDiff(fs, OBJECT_METADATA_REQUESTS);
+ MetricDiff listRequests = new MetricDiff(fs, OBJECT_LIST_REQUESTS);
+ MetricDiff listContinueRequests =
+ new MetricDiff(fs, OBJECT_CONTINUE_LIST_REQUESTS);
+ MetricDiff listStatusCalls = new MetricDiff(fs, INVOCATION_LIST_FILES);
+ MetricDiff getFileStatusCalls =
+ new MetricDiff(fs, INVOCATION_GET_FILE_STATUS);
+ NanoTimer createTimer = new NanoTimer();
+ TreeScanResults created =
+ createSubdirs(fs, listDir, depth, width, files, 0);
+ // add some empty directories
+ int emptyDepth = 1 * scale;
+ int emptyWidth = 3 * scale;
+
+ created.add(createSubdirs(fs, listDir, emptyDepth, emptyWidth, 0,
+ 0, "empty", "f-", ""));
+ createTimer.end("Time to create %s", created);
+ LOG.info("Time per operation: {}",
+ toHuman(createTimer.nanosPerOperation(created.totalCount())));
+ printThenReset(LOG,
+ metadataRequests,
+ listRequests,
+ listContinueRequests,
+ listStatusCalls,
+ getFileStatusCalls);
+
+ describe("Listing files via treewalk");
+ try {
+ // Scan the directory via an explicit tree walk.
+ // This is the baseline for any listing speedups.
+ NanoTimer treeWalkTimer = new NanoTimer();
+ TreeScanResults treewalkResults = treeWalk(fs, listDir);
+ treeWalkTimer.end("List status via treewalk of %s", created);
+
+ printThenReset(LOG,
+ metadataRequests,
+ listRequests,
+ listContinueRequests,
+ listStatusCalls,
+ getFileStatusCalls);
+ assertEquals("Files found in listFiles(recursive=true) " +
+ " created=" + created + " listed=" + treewalkResults,
+ created.getFileCount(), treewalkResults.getFileCount());
+
+ describe("Listing files via listFiles(recursive=true)");
+ // listFiles() does the recursion internally
+ NanoTimer listFilesRecursiveTimer = new NanoTimer();
+
+ TreeScanResults listFilesResults = new TreeScanResults(
+ fs.listFiles(listDir, true));
+
+ listFilesRecursiveTimer.end("listFiles(recursive=true) of %s", created);
+ assertEquals("Files found in listFiles(recursive=true) " +
+ " created=" + created + " listed=" + listFilesResults,
+ created.getFileCount(), listFilesResults.getFileCount());
+
+ // only two list operations should have taken place
+ print(LOG,
+ metadataRequests,
+ listRequests,
+ listContinueRequests,
+ listStatusCalls,
+ getFileStatusCalls);
+ assertEquals(listRequests.toString(), 2, listRequests.diff());
+ reset(metadataRequests,
+ listRequests,
+ listContinueRequests,
+ listStatusCalls,
+ getFileStatusCalls);
+
+
+ } finally {
+ describe("deletion");
+ // deletion at the end of the run
+ NanoTimer deleteTimer = new NanoTimer();
+ fs.delete(listDir, true);
+ deleteTimer.end("Deleting directory tree");
+ printThenReset(LOG,
+ metadataRequests,
+ listRequests,
+ listContinueRequests,
+ listStatusCalls,
+ getFileStatusCalls);
+ }
+ }
+
+ @Test
+ public void testTimeToStatEmptyDirectory() throws Throwable {
+ describe("Time to stat an empty directory");
+ Path path = new Path(getTestPath(), "empty");
+ fs.mkdirs(path);
+ timeToStatPath(path);
+ }
+
+ @Test
+ public void testTimeToStatNonEmptyDirectory() throws Throwable {
+ describe("Time to stat a non-empty directory");
+ Path path = new Path(getTestPath(), "dir");
+ fs.mkdirs(path);
+ touch(fs, new Path(path, "file"));
+ timeToStatPath(path);
+ }
+
+ @Test
+ public void testTimeToStatFile() throws Throwable {
+ describe("Time to stat a simple file");
+ Path path = new Path(getTestPath(), "file");
+ touch(fs, path);
+ timeToStatPath(path);
+ }
+
+ @Test
+ public void testTimeToStatRoot() throws Throwable {
+ describe("Time to stat the root path");
+ timeToStatPath(new Path("/"));
+ }
+
+ private void timeToStatPath(Path path) throws IOException {
+ describe("Timing getFileStatus(\"%s\")", path);
+ MetricDiff metadataRequests =
+ new MetricDiff(fs, Statistic.OBJECT_METADATA_REQUESTS);
+ MetricDiff listRequests =
+ new MetricDiff(fs, Statistic.OBJECT_LIST_REQUESTS);
+ long attempts = getOperationCount();
+ NanoTimer timer = new NanoTimer();
+ for (long l = 0; l < attempts; l++) {
+ fs.getFileStatus(path);
+ }
+ timer.end("Time to execute %d getFileStatusCalls", attempts);
+ LOG.info("Time per call: {}", toHuman(timer.nanosPerOperation(attempts)));
+ LOG.info("metadata: {}", metadataRequests);
+ LOG.info("metadata per operation {}", metadataRequests.diff() / attempts);
+ LOG.info("listObjects: {}", listRequests);
+ LOG.info("listObjects: per operation {}", listRequests.diff() / attempts);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
new file mode 100644
index 0000000..e2163c5
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
@@ -0,0 +1,534 @@
+/*
+ * 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.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.s3a.S3AFileStatus;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.S3AInputPolicy;
+import org.apache.hadoop.fs.s3a.S3AInputStream;
+import org.apache.hadoop.fs.s3a.S3AInstrumentation;
+import org.apache.hadoop.fs.s3a.S3ATestUtils;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.apache.hadoop.util.LineReader;
+import org.junit.After;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.EOFException;
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
+import static org.apache.hadoop.fs.s3a.Constants.*;
+
+/**
+ * Look at the performance of S3a operations.
+ */
+public class ITestS3AInputStreamPerformance extends S3AScaleTestBase {
+ private static final Logger LOG = LoggerFactory.getLogger(
+ ITestS3AInputStreamPerformance.class);
+
+ private S3AFileSystem s3aFS;
+ private Path testData;
+ private S3AFileStatus testDataStatus;
+ private FSDataInputStream in;
+ private S3AInstrumentation.InputStreamStatistics streamStatistics;
+ public static final int BLOCK_SIZE = 32 * 1024;
+ public static final int BIG_BLOCK_SIZE = 256 * 1024;
+
+ /** Tests only run if the there is a named test file that can be read. */
+ private boolean testDataAvailable = true;
+ private String assumptionMessage = "test file";
+
+ /**
+ * Open the FS and the test data. The input stream is always set up here.
+ * @throws IOException IO Problems.
+ */
+ @Before
+ public void openFS() throws IOException {
+ Configuration conf = getConf();
+ conf.setInt(SOCKET_SEND_BUFFER, 16 * 1024);
+ conf.setInt(SOCKET_RECV_BUFFER, 16 * 1024);
+ String testFile = conf.getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE);
+ if (testFile.isEmpty()) {
+ assumptionMessage = "Empty test property: " + KEY_CSVTEST_FILE;
+ testDataAvailable = false;
+ } else {
+ S3ATestUtils.useCSVDataEndpoint(conf);
+ testData = new Path(testFile);
+ Path path = this.testData;
+ bindS3aFS(path);
+ try {
+ testDataStatus = s3aFS.getFileStatus(this.testData);
+ } catch (IOException e) {
+ LOG.warn("Failed to read file {} specified in {}",
+ testFile, KEY_CSVTEST_FILE, e);
+ throw e;
+ }
+ }
+ }
+
+ private void bindS3aFS(Path path) throws IOException {
+ s3aFS = (S3AFileSystem) FileSystem.newInstance(path.toUri(), getConf());
+ }
+
+ /**
+ * Cleanup: close the stream, close the FS.
+ */
+ @After
+ public void cleanup() {
+ describe("cleanup");
+ IOUtils.closeStream(in);
+ IOUtils.closeStream(s3aFS);
+ }
+
+ /**
+ * Declare that the test requires the CSV test dataset.
+ */
+ private void requireCSVTestData() {
+ Assume.assumeTrue(assumptionMessage, testDataAvailable);
+ }
+
+ /**
+ * Open the test file with the read buffer specified in the setting.
+ * {@link #KEY_READ_BUFFER_SIZE}; use the {@code Normal} policy
+ * @return the stream, wrapping an S3a one
+ * @throws IOException IO problems
+ */
+ FSDataInputStream openTestFile() throws IOException {
+ return openTestFile(S3AInputPolicy.Normal, 0);
+ }
+
+ /**
+ * Open the test file with the read buffer specified in the setting
+ * {@link #KEY_READ_BUFFER_SIZE}.
+ * This includes the {@link #requireCSVTestData()} assumption; so
+ * if called before any FS op, will automatically skip the test
+ * if the CSV file is absent.
+ *
+ * @param inputPolicy input policy to use
+ * @param readahead readahead/buffer size
+ * @return the stream, wrapping an S3a one
+ * @throws IOException IO problems
+ */
+ FSDataInputStream openTestFile(S3AInputPolicy inputPolicy, long readahead)
+ throws IOException {
+ requireCSVTestData();
+ return openDataFile(s3aFS, this.testData, inputPolicy, readahead);
+ }
+
+ /**
+ * Open a test file with the read buffer specified in the setting
+ * {@link #KEY_READ_BUFFER_SIZE}.
+ *
+ * @param path path to open
+ * @param inputPolicy input policy to use
+ * @param readahead readahead/buffer size
+ * @return the stream, wrapping an S3a one
+ * @throws IOException IO problems
+ */
+ private FSDataInputStream openDataFile(S3AFileSystem fs,
+ Path path,
+ S3AInputPolicy inputPolicy,
+ long readahead) throws IOException {
+ int bufferSize = getConf().getInt(KEY_READ_BUFFER_SIZE,
+ DEFAULT_READ_BUFFER_SIZE);
+ S3AInputPolicy policy = fs.getInputPolicy();
+ fs.setInputPolicy(inputPolicy);
+ try {
+ FSDataInputStream stream = fs.open(path, bufferSize);
+ if (readahead >= 0) {
+ stream.setReadahead(readahead);
+ }
+ streamStatistics = getInputStreamStatistics(stream);
+ return stream;
+ } finally {
+ fs.setInputPolicy(policy);
+ }
+ }
+
+ /**
+ * Assert that the stream was only ever opened once.
+ */
+ protected void assertStreamOpenedExactlyOnce() {
+ assertOpenOperationCount(1);
+ }
+
+ /**
+ * Make an assertion count about the number of open operations.
+ * @param expected the expected number
+ */
+ private void assertOpenOperationCount(long expected) {
+ assertEquals("open operations in\n" + in,
+ expected, streamStatistics.openOperations);
+ }
+
+ /**
+ * Log how long an IOP took, by dividing the total time by the
+ * count of operations, printing in a human-readable form.
+ * @param operation operation being measured
+ * @param timer timing data
+ * @param count IOP count.
+ */
+ protected void logTimePerIOP(String operation,
+ NanoTimer timer,
+ long count) {
+ LOG.info("Time per {}: {} nS",
+ operation, toHuman(timer.duration() / count));
+ }
+
+ @Test
+ public void testTimeToOpenAndReadWholeFileBlocks() throws Throwable {
+ requireCSVTestData();
+ int blockSize = _1MB;
+ describe("Open the test file %s and read it in blocks of size %d",
+ testData, blockSize);
+ long len = testDataStatus.getLen();
+ in = openTestFile();
+ byte[] block = new byte[blockSize];
+ NanoTimer timer2 = new NanoTimer();
+ long count = 0;
+ // implicitly rounding down here
+ long blockCount = len / blockSize;
+ for (long i = 0; i < blockCount; i++) {
+ int offset = 0;
+ int remaining = blockSize;
+ NanoTimer blockTimer = new NanoTimer();
+ int reads = 0;
+ while (remaining > 0) {
+ int bytesRead = in.read(block, offset, remaining);
+ reads++;
+ if (bytesRead == 1) {
+ break;
+ }
+ remaining -= bytesRead;
+ offset += bytesRead;
+ count += bytesRead;
+ }
+ blockTimer.end("Reading block %d in %d reads", i, reads);
+ }
+ timer2.end("Time to read %d bytes in %d blocks", len, blockCount);
+ bandwidth(timer2, count);
+ logStreamStatistics();
+ }
+
+ @Test
+ public void testLazySeekEnabled() throws Throwable {
+ describe("Verify that seeks do not trigger any IO");
+ in = openTestFile();
+ long len = testDataStatus.getLen();
+ NanoTimer timer = new NanoTimer();
+ long blockCount = len / BLOCK_SIZE;
+ for (long i = 0; i < blockCount; i++) {
+ in.seek(in.getPos() + BLOCK_SIZE - 1);
+ }
+ in.seek(0);
+ blockCount++;
+ timer.end("Time to execute %d seeks", blockCount);
+ logTimePerIOP("seek()", timer, blockCount);
+ logStreamStatistics();
+ assertOpenOperationCount(0);
+ assertEquals("bytes read", 0, streamStatistics.bytesRead);
+ }
+
+ @Test
+ public void testReadaheadOutOfRange() throws Throwable {
+ try {
+ in = openTestFile();
+ in.setReadahead(-1L);
+ fail("Stream should have rejected the request "+ in);
+ } catch (IllegalArgumentException e) {
+ // expected
+ }
+ }
+
+ @Test
+ public void testReadWithNormalPolicy() throws Throwable {
+ describe("Read big blocks with a big readahead");
+ executeSeekReadSequence(BIG_BLOCK_SIZE, BIG_BLOCK_SIZE * 2,
+ S3AInputPolicy.Normal);
+ assertStreamOpenedExactlyOnce();
+ }
+
+ @Test
+ public void testDecompressionSequential128K() throws Throwable {
+ describe("Decompress with a 128K readahead");
+ executeDecompression(128 * 1024, S3AInputPolicy.Sequential);
+ assertStreamOpenedExactlyOnce();
+ }
+
+ /**
+ * Execute a decompression + line read with the given input policy.
+ * @param readahead byte readahead
+ * @param inputPolicy read policy
+ * @throws IOException IO Problems
+ */
+ private void executeDecompression(long readahead,
+ S3AInputPolicy inputPolicy) throws IOException {
+ CompressionCodecFactory factory
+ = new CompressionCodecFactory(getConf());
+ CompressionCodec codec = factory.getCodec(testData);
+ long bytesRead = 0;
+ int lines = 0;
+
+ FSDataInputStream objectIn = openTestFile(inputPolicy, readahead);
+ ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
+ try (LineReader lineReader = new LineReader(
+ codec.createInputStream(objectIn), getConf())) {
+ Text line = new Text();
+ int read;
+ while ((read = lineReader.readLine(line)) > 0) {
+ bytesRead += read;
+ lines++;
+ }
+ } catch (EOFException eof) {
+ // done
+ }
+ timer.end("Time to read %d lines [%d bytes expanded, %d raw]" +
+ " with readahead = %d",
+ lines,
+ bytesRead,
+ testDataStatus.getLen(),
+ readahead);
+ logTimePerIOP("line read", timer, lines);
+ logStreamStatistics();
+ }
+
+ private void logStreamStatistics() {
+ LOG.info(String.format("Stream Statistics%n{}"), streamStatistics);
+ }
+
+ /**
+ * Execute a seek+read sequence.
+ * @param blockSize block size for seeks
+ * @param readahead what the readahead value of the stream should be
+ * @throws IOException IO problems
+ */
+ protected void executeSeekReadSequence(long blockSize,
+ long readahead,
+ S3AInputPolicy policy) throws IOException {
+ in = openTestFile(policy, readahead);
+ long len = testDataStatus.getLen();
+ NanoTimer timer = new NanoTimer();
+ long blockCount = len / blockSize;
+ LOG.info("Reading {} blocks, readahead = {}",
+ blockCount, readahead);
+ for (long i = 0; i < blockCount; i++) {
+ in.seek(in.getPos() + blockSize - 1);
+ // this is the read
+ assertTrue(in.read() >= 0);
+ }
+ timer.end("Time to execute %d seeks of distance %d with readahead = %d",
+ blockCount,
+ blockSize,
+ readahead);
+ logTimePerIOP("seek(pos + " + blockCount+"); read()", timer, blockCount);
+ LOG.info("Effective bandwidth {} MB/S",
+ timer.bandwidthDescription(streamStatistics.bytesRead -
+ streamStatistics.bytesSkippedOnSeek));
+ logStreamStatistics();
+ }
+
+ public static final int _4K = 4 * 1024;
+ public static final int _8K = 8 * 1024;
+ public static final int _16K = 16 * 1024;
+ public static final int _32K = 32 * 1024;
+ public static final int _64K = 64 * 1024;
+ public static final int _128K = 128 * 1024;
+ public static final int _256K = 256 * 1024;
+ public static final int _1MB = 1024 * 1024;
+ public static final int _2MB = 2 * _1MB;
+ public static final int _10MB = _1MB * 10;
+ public static final int _5MB = _1MB * 5;
+
+ private static final int[][] RANDOM_IO_SEQUENCE = {
+ {_2MB, _128K},
+ {_128K, _128K},
+ {_5MB, _64K},
+ {_1MB, _1MB},
+ };
+
+ @Test
+ public void testRandomIORandomPolicy() throws Throwable {
+ executeRandomIO(S3AInputPolicy.Random, (long) RANDOM_IO_SEQUENCE.length);
+ assertEquals("streams aborted in " + streamStatistics,
+ 0, streamStatistics.aborted);
+ }
+
+ @Test
+ public void testRandomIONormalPolicy() throws Throwable {
+ long expectedOpenCount = RANDOM_IO_SEQUENCE.length;
+ executeRandomIO(S3AInputPolicy.Normal, expectedOpenCount);
+ assertEquals("streams aborted in " + streamStatistics,
+ 4, streamStatistics.aborted);
+ }
+
+ /**
+ * Execute the random IO {@code readFully(pos, bytes[])} sequence defined by
+ * {@link #RANDOM_IO_SEQUENCE}. The stream is closed afterwards; that's used
+ * in the timing too
+ * @param policy read policy
+ * @param expectedOpenCount expected number of stream openings
+ * @throws IOException IO problems
+ * @return the timer
+ */
+ private ContractTestUtils.NanoTimer executeRandomIO(S3AInputPolicy policy,
+ long expectedOpenCount)
+ throws IOException {
+ describe("Random IO with policy \"%s\"", policy);
+ byte[] buffer = new byte[_1MB];
+ long totalBytesRead = 0;
+
+ in = openTestFile(policy, 0);
+ ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
+ for (int[] action : RANDOM_IO_SEQUENCE) {
+ int position = action[0];
+ int range = action[1];
+ in.readFully(position, buffer, 0, range);
+ totalBytesRead += range;
+ }
+ int reads = RANDOM_IO_SEQUENCE.length;
+ timer.end("Time to execute %d reads of total size %d bytes",
+ reads,
+ totalBytesRead);
+ in.close();
+ assertOpenOperationCount(expectedOpenCount);
+ logTimePerIOP("byte read", timer, totalBytesRead);
+ LOG.info("Effective bandwidth {} MB/S",
+ timer.bandwidthDescription(streamStatistics.bytesRead -
+ streamStatistics.bytesSkippedOnSeek));
+ logStreamStatistics();
+ return timer;
+ }
+
+ S3AInputStream getS3aStream() {
+ return (S3AInputStream) in.getWrappedStream();
+ }
+
+ @Test
+ public void testRandomReadOverBuffer() throws Throwable {
+ describe("read over a buffer, making sure that the requests" +
+ " spans readahead ranges");
+ int datasetLen = _32K;
+ Path dataFile = new Path(getTestPath(), "testReadOverBuffer.bin");
+ byte[] sourceData = dataset(datasetLen, 0, 64);
+ // relies on the field 'fs' referring to the R/W FS
+ writeDataset(fs, dataFile, sourceData, datasetLen, _16K, true);
+ byte[] buffer = new byte[datasetLen];
+ int readahead = _8K;
+ int halfReadahead = _4K;
+ in = openDataFile(fs, dataFile, S3AInputPolicy.Random, readahead);
+
+ LOG.info("Starting initial reads");
+ S3AInputStream s3aStream = getS3aStream();
+ assertEquals(readahead, s3aStream.getReadahead());
+ byte[] oneByte = new byte[1];
+ assertEquals(1, in.read(0, oneByte, 0, 1));
+ // make some assertions about the current state
+ assertEquals("remaining in\n" + in,
+ readahead - 1, s3aStream.remainingInCurrentRequest());
+ assertEquals("range start in\n" + in,
+ 0, s3aStream.getContentRangeStart());
+ assertEquals("range finish in\n" + in,
+ readahead, s3aStream.getContentRangeFinish());
+
+ assertStreamOpenedExactlyOnce();
+
+ describe("Starting sequence of positioned read calls over\n%s", in);
+ NanoTimer readTimer = new NanoTimer();
+ int currentPos = halfReadahead;
+ int offset = currentPos;
+ int bytesRead = 0;
+ int readOps = 0;
+
+ // make multiple read() calls
+ while (bytesRead < halfReadahead) {
+ int length = buffer.length - offset;
+ int read = in.read(currentPos, buffer, offset, length);
+ bytesRead += read;
+ offset += read;
+ readOps++;
+ assertEquals("open operations on request #" + readOps
+ + " after reading " + bytesRead
+ + " current position in stream " + currentPos
+ + " in\n" + fs
+ + "\n " + in,
+ 1, streamStatistics.openOperations);
+ for (int i = currentPos; i < currentPos + read; i++) {
+ assertEquals("Wrong value from byte " + i,
+ sourceData[i], buffer[i]);
+ }
+ currentPos += read;
+ }
+ assertStreamOpenedExactlyOnce();
+ // assert at the end of the original block
+ assertEquals(readahead, currentPos);
+ readTimer.end("read %d in %d operations", bytesRead, readOps);
+ bandwidth(readTimer, bytesRead);
+ LOG.info("Time per byte(): {} nS",
+ toHuman(readTimer.nanosPerOperation(bytesRead)));
+ LOG.info("Time per read(): {} nS",
+ toHuman(readTimer.nanosPerOperation(readOps)));
+
+ describe("read last byte");
+ // read one more
+ int read = in.read(currentPos, buffer, bytesRead, 1);
+ assertTrue("-1 from last read", read >= 0);
+ assertOpenOperationCount(2);
+ assertEquals("Wrong value from read ", sourceData[currentPos],
+ (int) buffer[currentPos]);
+ currentPos++;
+
+
+ // now scan all the way to the end of the file, using single byte read()
+ // calls
+ describe("read() to EOF over \n%s", in);
+ long readCount = 0;
+ NanoTimer timer = new NanoTimer();
+ LOG.info("seeking");
+ in.seek(currentPos);
+ LOG.info("reading");
+ while(currentPos < datasetLen) {
+ int r = in.read();
+ assertTrue("Negative read() at position " + currentPos + " in\n" + in,
+ r >= 0);
+ buffer[currentPos] = (byte)r;
+ assertEquals("Wrong value from read from\n" + in,
+ sourceData[currentPos], r);
+ currentPos++;
+ readCount++;
+ }
+ timer.end("read %d bytes", readCount);
+ bandwidth(timer, readCount);
+ LOG.info("Time per read(): {} nS",
+ toHuman(timer.nanosPerOperation(readCount)));
+
+ assertEquals("last read in " + in, -1, in.read());
+ }
+}
---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org
[4/7] hadoop git commit: HADOOP-13446. Support running isolated unit
tests separate from AWS integration tests. Contributed by Chris Nauroth.
Posted by cn...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java
new file mode 100644
index 0000000..0686488
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java
@@ -0,0 +1,137 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.AbstractFSContractTestBase;
+import org.apache.hadoop.fs.contract.s3a.S3AContract;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.EOFException;
+import java.io.FileNotFoundException;
+import java.util.concurrent.Callable;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
+import static org.apache.hadoop.fs.s3a.S3AUtils.*;
+
+/**
+ * Test S3A Failure translation, including a functional test
+ * generating errors during stream IO.
+ */
+public class ITestS3AFailureHandling extends AbstractFSContractTestBase {
+ private static final Logger LOG =
+ LoggerFactory.getLogger(ITestS3AFailureHandling.class);
+
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new S3AContract(conf);
+ }
+
+ @Test
+ public void testReadFileChanged() throws Throwable {
+ describe("overwrite a file with a shorter one during a read, seek");
+ final int fullLength = 8192;
+ final byte[] fullDataset = dataset(fullLength, 'a', 32);
+ final int shortLen = 4096;
+ final byte[] shortDataset = dataset(shortLen, 'A', 32);
+ final FileSystem fs = getFileSystem();
+ final Path testpath = path("readFileToChange.txt");
+ // initial write
+ writeDataset(fs, testpath, fullDataset, fullDataset.length, 1024, false);
+ try(FSDataInputStream instream = fs.open(testpath)) {
+ instream.seek(fullLength - 16);
+ assertTrue("no data to read", instream.read() >= 0);
+ // overwrite
+ writeDataset(fs, testpath, shortDataset, shortDataset.length, 1024, true);
+ // here the file length is less. Probe the file to see if this is true,
+ // with a spin and wait
+ eventually(30 *1000, new Callable<Void>() {
+ @Override
+ public Void call() throws Exception {
+ assertEquals(shortLen, fs.getFileStatus(testpath).getLen());
+ return null;
+ }
+ });
+ // here length is shorter. Assuming it has propagated to all replicas,
+ // the position of the input stream is now beyond the EOF.
+ // An attempt to seek backwards to a position greater than the
+ // short length will raise an exception from AWS S3, which must be
+ // translated into an EOF
+
+ instream.seek(shortLen + 1024);
+ int c = instream.read();
+ assertIsEOF("read()", c);
+
+ byte[] buf = new byte[256];
+
+ assertIsEOF("read(buffer)", instream.read(buf));
+ assertIsEOF("read(offset)",
+ instream.read(instream.getPos(), buf, 0, buf.length));
+
+ // now do a block read fully, again, backwards from the current pos
+ try {
+ instream.readFully(shortLen + 512, buf);
+ fail("Expected readFully to fail");
+ } catch (EOFException expected) {
+ LOG.debug("Expected EOF: ", expected);
+ }
+
+ assertIsEOF("read(offset)",
+ instream.read(shortLen + 510, buf, 0, buf.length));
+
+ // seek somewhere useful
+ instream.seek(shortLen - 256);
+
+ // delete the file. Reads must fail
+ fs.delete(testpath, false);
+
+ try {
+ int r = instream.read();
+ fail("Expected an exception, got " + r);
+ } catch (FileNotFoundException e) {
+ // expected
+ }
+
+ try {
+ instream.readFully(2048, buf);
+ fail("Expected readFully to fail");
+ } catch (FileNotFoundException e) {
+ // expected
+ }
+
+ }
+ }
+
+ /**
+ * Assert that a read operation returned an EOF value.
+ * @param operation specific operation
+ * @param readResult result
+ */
+ private void assertIsEOF(String operation, int readResult) {
+ assertEquals("Expected EOF from "+ operation
+ + "; got char " + (char) readResult, -1, readResult);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFastOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFastOutputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFastOutputStream.java
new file mode 100644
index 0000000..b5fa1c3
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFastOutputStream.java
@@ -0,0 +1,74 @@
+/**
+ * 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 org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+
+import java.io.IOException;
+
+/**
+ * Tests regular and multi-part upload functionality for S3AFastOutputStream.
+ * File sizes are kept small to reduce test duration on slow connections
+ */
+public class ITestS3AFastOutputStream {
+ private FileSystem fs;
+
+
+ @Rule
+ public Timeout testTimeout = new Timeout(30 * 60 * 1000);
+
+ @Before
+ public void setUp() throws Exception {
+ Configuration conf = new Configuration();
+ conf.setLong(Constants.MIN_MULTIPART_THRESHOLD, 5 * 1024 * 1024);
+ conf.setInt(Constants.MULTIPART_SIZE, 5 * 1024 * 1024);
+ conf.setBoolean(Constants.FAST_UPLOAD, true);
+ fs = S3ATestUtils.createTestFileSystem(conf);
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ if (fs != null) {
+ fs.delete(getTestPath(), true);
+ }
+ }
+
+ protected Path getTestPath() {
+ return new Path("/tests3a");
+ }
+
+ @Test
+ public void testRegularUpload() throws IOException {
+ ContractTestUtils.createAndVerifyFile(fs, getTestPath(), 1024 * 1024);
+ }
+
+ @Test
+ public void testMultiPartUpload() throws IOException {
+ ContractTestUtils.createAndVerifyFile(fs, getTestPath(), 6 * 1024 *
+ 1024);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java
new file mode 100644
index 0000000..4dcd836
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java
@@ -0,0 +1,277 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.AbstractFSContractTestBase;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.contract.s3a.S3AContract;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.net.URI;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
+import static org.apache.hadoop.fs.s3a.Statistic.*;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.MetricDiff;
+
+/**
+ * Use metrics to assert about the cost of file status queries.
+ * {@link S3AFileSystem#getFileStatus(Path)}.
+ */
+public class ITestS3AFileOperationCost extends AbstractFSContractTestBase {
+
+ private MetricDiff metadataRequests;
+ private MetricDiff listRequests;
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(ITestS3AFileOperationCost.class);
+
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new S3AContract(conf);
+ }
+
+ @Override
+ public S3AFileSystem getFileSystem() {
+ return (S3AFileSystem) super.getFileSystem();
+ }
+
+ @Override
+ public void setup() throws Exception {
+ super.setup();
+ S3AFileSystem fs = getFileSystem();
+ metadataRequests = new MetricDiff(fs, OBJECT_METADATA_REQUESTS);
+ listRequests = new MetricDiff(fs, OBJECT_LIST_REQUESTS);
+ }
+
+ @Test
+ public void testCostOfGetFileStatusOnFile() throws Throwable {
+ describe("performing getFileStatus on a file");
+ Path simpleFile = path("simple.txt");
+ S3AFileSystem fs = getFileSystem();
+ touch(fs, simpleFile);
+ resetMetricDiffs();
+ S3AFileStatus status = fs.getFileStatus(simpleFile);
+ assertTrue("not a file: " + status, status.isFile());
+ metadataRequests.assertDiffEquals(1);
+ listRequests.assertDiffEquals(0);
+ }
+
+ private void resetMetricDiffs() {
+ reset(metadataRequests, listRequests);
+ }
+
+ @Test
+ public void testCostOfGetFileStatusOnEmptyDir() throws Throwable {
+ describe("performing getFileStatus on an empty directory");
+ S3AFileSystem fs = getFileSystem();
+ Path dir = path("empty");
+ fs.mkdirs(dir);
+ resetMetricDiffs();
+ S3AFileStatus status = fs.getFileStatus(dir);
+ assertTrue("not empty: " + status, status.isEmptyDirectory());
+ metadataRequests.assertDiffEquals(2);
+ listRequests.assertDiffEquals(0);
+ }
+
+ @Test
+ public void testCostOfGetFileStatusOnMissingFile() throws Throwable {
+ describe("performing getFileStatus on a missing file");
+ S3AFileSystem fs = getFileSystem();
+ Path path = path("missing");
+ resetMetricDiffs();
+ try {
+ S3AFileStatus status = fs.getFileStatus(path);
+ fail("Got a status back from a missing file path " + status);
+ } catch (FileNotFoundException expected) {
+ // expected
+ }
+ metadataRequests.assertDiffEquals(2);
+ listRequests.assertDiffEquals(1);
+ }
+
+ @Test
+ public void testCostOfGetFileStatusOnMissingSubPath() throws Throwable {
+ describe("performing getFileStatus on a missing file");
+ S3AFileSystem fs = getFileSystem();
+ Path path = path("missingdir/missingpath");
+ resetMetricDiffs();
+ try {
+ S3AFileStatus status = fs.getFileStatus(path);
+ fail("Got a status back from a missing file path " + status);
+ } catch (FileNotFoundException expected) {
+ // expected
+ }
+ metadataRequests.assertDiffEquals(2);
+ listRequests.assertDiffEquals(1);
+ }
+
+ @Test
+ public void testCostOfGetFileStatusOnNonEmptyDir() throws Throwable {
+ describe("performing getFileStatus on a non-empty directory");
+ S3AFileSystem fs = getFileSystem();
+ Path dir = path("empty");
+ fs.mkdirs(dir);
+ Path simpleFile = new Path(dir, "simple.txt");
+ touch(fs, simpleFile);
+ resetMetricDiffs();
+ S3AFileStatus status = fs.getFileStatus(dir);
+ if (status.isEmptyDirectory()) {
+ // erroneous state
+ String fsState = fs.toString();
+ fail("FileStatus says directory isempty: " + status
+ + "\n" + ContractTestUtils.ls(fs, dir)
+ + "\n" + fsState);
+ }
+ metadataRequests.assertDiffEquals(2);
+ listRequests.assertDiffEquals(1);
+ }
+
+ @Test
+ public void testCostOfCopyFromLocalFile() throws Throwable {
+ describe("testCostOfCopyFromLocalFile");
+ String testDirProp = System.getProperty("test.build.data",
+ "target" + File.separator + "test" + File.separator + "data");
+ File localTestDir = new File(testDirProp, "tmp").getAbsoluteFile();
+ localTestDir.mkdirs();
+ File tmpFile = File.createTempFile("tests3acost", ".txt",
+ localTestDir);
+ tmpFile.delete();
+ try {
+ URI localFileURI = tmpFile.toURI();
+ FileSystem localFS = FileSystem.get(localFileURI,
+ getFileSystem().getConf());
+ Path localPath = new Path(localFileURI);
+ int len = 10 * 1024;
+ byte[] data = dataset(len, 'A', 'Z');
+ writeDataset(localFS, localPath, data, len, 1024, true);
+ S3AFileSystem s3a = getFileSystem();
+ MetricDiff copyLocalOps = new MetricDiff(s3a,
+ INVOCATION_COPY_FROM_LOCAL_FILE);
+ MetricDiff putRequests = new MetricDiff(s3a,
+ OBJECT_PUT_REQUESTS);
+ MetricDiff putBytes = new MetricDiff(s3a,
+ OBJECT_PUT_BYTES);
+
+ Path remotePath = path("copied");
+ s3a.copyFromLocalFile(false, true, localPath, remotePath);
+ verifyFileContents(s3a, remotePath, data);
+ copyLocalOps.assertDiffEquals(1);
+ putRequests.assertDiffEquals(1);
+ putBytes.assertDiffEquals(len);
+ // print final stats
+ LOG.info("Filesystem {}", s3a);
+ } finally {
+ tmpFile.delete();
+ }
+ }
+
+ private void reset(MetricDiff... diffs) {
+ for (MetricDiff diff : diffs) {
+ diff.reset();
+ }
+ }
+
+ @Test
+ public void testFakeDirectoryDeletion() throws Throwable {
+ describe("Verify whether create file works after renaming a file. "
+ + "In S3, rename deletes any fake directories as a part of "
+ + "clean up activity");
+ S3AFileSystem fs = getFileSystem();
+ Path srcBaseDir = path("src");
+ mkdirs(srcBaseDir);
+ MetricDiff deleteRequests =
+ new MetricDiff(fs, Statistic.OBJECT_DELETE_REQUESTS);
+ MetricDiff directoriesDeleted =
+ new MetricDiff(fs, Statistic.DIRECTORIES_DELETED);
+ MetricDiff fakeDirectoriesDeleted =
+ new MetricDiff(fs, Statistic.FAKE_DIRECTORIES_DELETED);
+ MetricDiff directoriesCreated =
+ new MetricDiff(fs, Statistic.DIRECTORIES_CREATED);
+
+ Path srcDir = new Path(srcBaseDir, "1/2/3/4/5/6");
+ Path srcFilePath = new Path(srcDir, "source.txt");
+ int srcDirDepth = directoriesInPath(srcDir);
+ // one dir created, one removed
+ mkdirs(srcDir);
+ String state = "after mkdir(srcDir)";
+ directoriesCreated.assertDiffEquals(state, 1);
+/* TODO: uncomment once HADOOP-13222 is in
+ deleteRequests.assertDiffEquals(state, 1);
+ directoriesDeleted.assertDiffEquals(state, 0);
+ fakeDirectoriesDeleted.assertDiffEquals(state, srcDirDepth);
+*/
+ reset(deleteRequests, directoriesCreated, directoriesDeleted,
+ fakeDirectoriesDeleted);
+
+ // creating a file should trigger demise of the src dir
+ touch(fs, srcFilePath);
+ state = "after touch(fs, srcFilePath)";
+ deleteRequests.assertDiffEquals(state, 1);
+ directoriesCreated.assertDiffEquals(state, 0);
+ directoriesDeleted.assertDiffEquals(state, 0);
+ fakeDirectoriesDeleted.assertDiffEquals(state, srcDirDepth);
+
+ reset(deleteRequests, directoriesCreated, directoriesDeleted,
+ fakeDirectoriesDeleted);
+
+ Path destBaseDir = path("dest");
+ Path destDir = new Path(destBaseDir, "1/2/3/4/5/6");
+ Path destFilePath = new Path(destDir, "dest.txt");
+ mkdirs(destDir);
+ state = "after mkdir(destDir)";
+
+ int destDirDepth = directoriesInPath(destDir);
+ directoriesCreated.assertDiffEquals(state, 1);
+/* TODO: uncomment once HADOOP-13222 is in
+ deleteRequests.assertDiffEquals(state,1);
+ directoriesDeleted.assertDiffEquals(state,0);
+ fakeDirectoriesDeleted.assertDiffEquals(state,destDirDepth);
+*/
+ reset(deleteRequests, directoriesCreated, directoriesDeleted,
+ fakeDirectoriesDeleted);
+
+ fs.rename(srcFilePath, destFilePath);
+ state = "after rename(srcFilePath, destFilePath)";
+ directoriesCreated.assertDiffEquals(state, 1);
+ // one for the renamed file, one for the parent
+ deleteRequests.assertDiffEquals(state, 2);
+ directoriesDeleted.assertDiffEquals(state, 0);
+ fakeDirectoriesDeleted.assertDiffEquals(state, destDirDepth);
+
+ reset(deleteRequests, directoriesCreated, directoriesDeleted,
+ fakeDirectoriesDeleted);
+
+ assertIsFile(destFilePath);
+ assertIsDirectory(srcDir);
+ }
+
+ private int directoriesInPath(Path path) {
+ return path.isRoot() ? 0 : 1 + directoriesInPath(path.getParent());
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemContract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemContract.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemContract.java
new file mode 100644
index 0000000..858ac22
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemContract.java
@@ -0,0 +1,106 @@
+/**
+ * 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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystemContractBaseTest;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Tests a live S3 system. If your keys and bucket aren't specified, all tests
+ * are marked as passed.
+ *
+ * This uses BlockJUnit4ClassRunner because FileSystemContractBaseTest from
+ * TestCase which uses the old Junit3 runner that doesn't ignore assumptions
+ * properly making it impossible to skip the tests if we don't have a valid
+ * bucket.
+ **/
+public class ITestS3AFileSystemContract extends FileSystemContractBaseTest {
+
+ protected static final Logger LOG =
+ LoggerFactory.getLogger(ITestS3AFileSystemContract.class);
+
+ @Override
+ public void setUp() throws Exception {
+ Configuration conf = new Configuration();
+
+ fs = S3ATestUtils.createTestFileSystem(conf);
+ super.setUp();
+ }
+
+ @Override
+ protected void tearDown() throws Exception {
+ if (fs != null) {
+ fs.delete(path("test"), true);
+ }
+ super.tearDown();
+ }
+
+ @Override
+ public void testMkdirsWithUmask() throws Exception {
+ // not supported
+ }
+
+ @Override
+ public void testRenameFileAsExistingFile() throws Exception {
+ if (!renameSupported()) {
+ return;
+ }
+
+ Path src = path("/test/hadoop/file");
+ createFile(src);
+ Path dst = path("/test/new/newfile");
+ createFile(dst);
+ // s3 doesn't support rename option
+ // rename-overwrites-dest is always allowed.
+ rename(src, dst, true, false, true);
+ }
+
+ @Override
+ public void testRenameDirectoryAsExistingDirectory() throws Exception {
+ if (!renameSupported()) {
+ return;
+ }
+
+ Path src = path("/test/hadoop/dir");
+ fs.mkdirs(src);
+ createFile(path("/test/hadoop/dir/file1"));
+ createFile(path("/test/hadoop/dir/subdir/file2"));
+
+ Path dst = path("/test/new/newdir");
+ fs.mkdirs(dst);
+ rename(src, dst, true, false, true);
+ assertFalse("Nested file1 exists",
+ fs.exists(path("/test/hadoop/dir/file1")));
+ assertFalse("Nested file2 exists",
+ fs.exists(path("/test/hadoop/dir/subdir/file2")));
+ assertTrue("Renamed nested file1 exists",
+ fs.exists(path("/test/new/newdir/file1")));
+ assertTrue("Renamed nested exists",
+ fs.exists(path("/test/new/newdir/subdir/file2")));
+ }
+
+// @Override
+ public void testMoveDirUnderParent() throws Throwable {
+ // not support because
+ // Fails if dst is a directory that is not empty.
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ATemporaryCredentials.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ATemporaryCredentials.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ATemporaryCredentials.java
new file mode 100644
index 0000000..360a151
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ATemporaryCredentials.java
@@ -0,0 +1,148 @@
+/**
+ * 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 java.io.IOException;
+import java.net.URI;
+
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient;
+import com.amazonaws.services.securitytoken.model.GetSessionTokenRequest;
+import com.amazonaws.services.securitytoken.model.GetSessionTokenResult;
+import com.amazonaws.services.securitytoken.model.Credentials;
+
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.AbstractFSContractTestBase;
+import org.apache.hadoop.fs.contract.s3a.S3AContract;
+import org.apache.hadoop.fs.s3native.S3xLoginHelper;
+import org.apache.hadoop.conf.Configuration;
+
+import org.junit.Test;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
+import static org.apache.hadoop.fs.s3a.Constants.*;
+
+/**
+ * Tests use of temporary credentials (for example, AWS STS & S3).
+ * This test extends a class that "does things to the root directory", and
+ * should only be used against transient filesystems where you don't care about
+ * the data.
+ */
+public class ITestS3ATemporaryCredentials extends AbstractFSContractTestBase {
+ public static final String TEST_STS_ENABLED = "test.fs.s3a.sts.enabled";
+ public static final String TEST_STS_ENDPOINT = "test.fs.s3a.sts.endpoint";
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(ITestS3ATemporaryCredentials.class);
+
+ private static final String PROVIDER_CLASS
+ = TemporaryAWSCredentialsProvider.NAME;
+
+ private static final long TEST_FILE_SIZE = 1024;
+
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new S3AContract(conf);
+ }
+
+ /**
+ * Test use of STS for requesting temporary credentials.
+ *
+ * The property test.sts.endpoint can be set to point this at different
+ * STS endpoints. This test will use the AWS credentials (if provided) for
+ * S3A tests to request temporary credentials, then attempt to use those
+ * credentials instead.
+ *
+ * @throws IOException
+ */
+ @Test
+ public void testSTS() throws IOException {
+ Configuration conf = getContract().getConf();
+ if (!conf.getBoolean(TEST_STS_ENABLED, true)) {
+ skip("STS functional tests disabled");
+ }
+
+ S3xLoginHelper.Login login = S3AUtils.getAWSAccessKeys(
+ URI.create("s3a://foobar"), conf);
+ if (!login.hasLogin()) {
+ skip("testSTS disabled because AWS credentials not configured");
+ }
+ AWSCredentialsProvider parentCredentials = new BasicAWSCredentialsProvider(
+ login.getUser(), login.getPassword());
+
+ String stsEndpoint = conf.getTrimmed(TEST_STS_ENDPOINT, "");
+ AWSSecurityTokenServiceClient stsClient;
+ stsClient = new AWSSecurityTokenServiceClient(parentCredentials);
+ if (!stsEndpoint.isEmpty()) {
+ LOG.debug("STS Endpoint ={}", stsEndpoint);
+ stsClient.setEndpoint(stsEndpoint);
+ }
+ GetSessionTokenRequest sessionTokenRequest = new GetSessionTokenRequest();
+ sessionTokenRequest.setDurationSeconds(900);
+ GetSessionTokenResult sessionTokenResult;
+ sessionTokenResult = stsClient.getSessionToken(sessionTokenRequest);
+ Credentials sessionCreds = sessionTokenResult.getCredentials();
+
+ String childAccessKey = sessionCreds.getAccessKeyId();
+ conf.set(ACCESS_KEY, childAccessKey);
+ String childSecretKey = sessionCreds.getSecretAccessKey();
+ conf.set(SECRET_KEY, childSecretKey);
+ String sessionToken = sessionCreds.getSessionToken();
+ conf.set(SESSION_TOKEN, sessionToken);
+
+ conf.set(AWS_CREDENTIALS_PROVIDER, PROVIDER_CLASS);
+
+ try(S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) {
+ createAndVerifyFile(fs, path("testSTS"), TEST_FILE_SIZE);
+ }
+
+ // now create an invalid set of credentials by changing the session
+ // token
+ conf.set(SESSION_TOKEN, "invalid-" + sessionToken);
+ try (S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) {
+ createAndVerifyFile(fs, path("testSTSInvalidToken"), TEST_FILE_SIZE);
+ fail("Expected an access exception, but file access to "
+ + fs.getUri() + " was allowed: " + fs);
+ } catch (AWSS3IOException ex) {
+ LOG.info("Expected Exception: {}", ex.toString());
+ LOG.debug("Expected Exception: {}", ex, ex);
+ }
+ }
+
+ @Test
+ public void testTemporaryCredentialValidation() throws Throwable {
+ Configuration conf = new Configuration();
+ conf.set(ACCESS_KEY, "accesskey");
+ conf.set(SECRET_KEY, "secretkey");
+ conf.set(SESSION_TOKEN, "");
+ TemporaryAWSCredentialsProvider provider
+ = new TemporaryAWSCredentialsProvider(getFileSystem().getUri(), conf);
+ try {
+ AWSCredentials credentials = provider.getCredentials();
+ fail("Expected a CredentialInitializationException,"
+ + " got " + credentials);
+ } catch (CredentialInitializationException expected) {
+ // expected
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestBlockingThreadPoolExecutorService.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestBlockingThreadPoolExecutorService.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestBlockingThreadPoolExecutorService.java
deleted file mode 100644
index 25a8958..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestBlockingThreadPoolExecutorService.java
+++ /dev/null
@@ -1,182 +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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * 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.google.common.util.concurrent.ListenableFuture;
-import org.apache.hadoop.util.StopWatch;
-import org.junit.*;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.Callable;
-import java.util.concurrent.TimeUnit;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-
-/**
- * Basic unit test for S3A's blocking executor service.
- */
-public class TestBlockingThreadPoolExecutorService {
-
- private static final Logger LOG = LoggerFactory.getLogger(
- BlockingThreadPoolExecutorService.class);
-
- private static final int NUM_ACTIVE_TASKS = 4;
- private static final int NUM_WAITING_TASKS = 2;
- private static final int TASK_SLEEP_MSEC = 100;
- private static final int SHUTDOWN_WAIT_MSEC = 200;
- private static final int SHUTDOWN_WAIT_TRIES = 5;
- private static final int BLOCKING_THRESHOLD_MSEC = 50;
-
- private static final Integer SOME_VALUE = 1337;
-
- private static BlockingThreadPoolExecutorService tpe = null;
-
- @AfterClass
- public static void afterClass() throws Exception {
- ensureDestroyed();
- }
-
- /**
- * Basic test of running one trivial task.
- */
- @Test
- public void testSubmitCallable() throws Exception {
- ensureCreated();
- ListenableFuture<Integer> f = tpe.submit(callableSleeper);
- Integer v = f.get();
- assertEquals(SOME_VALUE, v);
- }
-
- /**
- * More involved test, including detecting blocking when at capacity.
- */
- @Test
- public void testSubmitRunnable() throws Exception {
- ensureCreated();
- int totalTasks = NUM_ACTIVE_TASKS + NUM_WAITING_TASKS;
- StopWatch stopWatch = new StopWatch().start();
- for (int i = 0; i < totalTasks; i++) {
- tpe.submit(sleeper);
- assertDidntBlock(stopWatch);
- }
- tpe.submit(sleeper);
- assertDidBlock(stopWatch);
- }
-
- @Test
- public void testShutdown() throws Exception {
- // Cover create / destroy, regardless of when this test case runs
- ensureCreated();
- ensureDestroyed();
-
- // Cover create, execute, destroy, regardless of when test case runs
- ensureCreated();
- testSubmitRunnable();
- ensureDestroyed();
- }
-
- // Helper functions, etc.
-
- private void assertDidntBlock(StopWatch sw) {
- try {
- assertFalse("Non-blocking call took too long.",
- sw.now(TimeUnit.MILLISECONDS) > BLOCKING_THRESHOLD_MSEC);
- } finally {
- sw.reset().start();
- }
- }
-
- private void assertDidBlock(StopWatch sw) {
- try {
- if (sw.now(TimeUnit.MILLISECONDS) < BLOCKING_THRESHOLD_MSEC) {
- throw new RuntimeException("Blocking call returned too fast.");
- }
- } finally {
- sw.reset().start();
- }
- }
-
- private Runnable sleeper = new Runnable() {
- @Override
- public void run() {
- String name = Thread.currentThread().getName();
- try {
- Thread.sleep(TASK_SLEEP_MSEC);
- } catch (InterruptedException e) {
- LOG.info("Thread {} interrupted.", name);
- Thread.currentThread().interrupt();
- }
- }
- };
-
- private Callable<Integer> callableSleeper = new Callable<Integer>() {
- @Override
- public Integer call() throws Exception {
- sleeper.run();
- return SOME_VALUE;
- }
- };
-
- /**
- * Helper function to create thread pool under test.
- */
- private static void ensureCreated() throws Exception {
- if (tpe == null) {
- LOG.debug("Creating thread pool");
- tpe = new BlockingThreadPoolExecutorService(NUM_ACTIVE_TASKS,
- NUM_WAITING_TASKS, 1, TimeUnit.SECONDS, "btpetest");
- }
- }
-
- /**
- * Helper function to terminate thread pool under test, asserting that
- * shutdown -> terminate works as expected.
- */
- private static void ensureDestroyed() throws Exception {
- if (tpe == null) {
- return;
- }
- int shutdownTries = SHUTDOWN_WAIT_TRIES;
-
- tpe.shutdown();
- if (!tpe.isShutdown()) {
- throw new RuntimeException("Shutdown had no effect.");
- }
-
- while (!tpe.awaitTermination(SHUTDOWN_WAIT_MSEC,
- TimeUnit.MILLISECONDS)) {
- LOG.info("Waiting for thread pool shutdown.");
- if (shutdownTries-- <= 0) {
- LOG.error("Failed to terminate thread pool gracefully.");
- break;
- }
- }
- if (!tpe.isTerminated()) {
- tpe.shutdownNow();
- if (!tpe.awaitTermination(SHUTDOWN_WAIT_MSEC,
- TimeUnit.MILLISECONDS)) {
- throw new RuntimeException(
- "Failed to terminate thread pool in timely manner.");
- }
- }
- tpe = null;
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java
deleted file mode 100644
index 5ba1871..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java
+++ /dev/null
@@ -1,250 +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 java.io.IOException;
-import java.net.URI;
-import java.nio.file.AccessDeniedException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.rules.Timeout;
-
-import com.amazonaws.auth.AWSCredentials;
-import com.amazonaws.auth.AWSCredentialsProvider;
-import com.amazonaws.auth.AWSCredentialsProviderChain;
-import com.amazonaws.auth.BasicAWSCredentials;
-import com.amazonaws.auth.InstanceProfileCredentialsProvider;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.apache.hadoop.fs.s3a.Constants.*;
-import static org.apache.hadoop.fs.s3a.S3ATestConstants.*;
-import static org.apache.hadoop.fs.s3a.S3AUtils.*;
-import static org.junit.Assert.*;
-
-/**
- * Tests for {@link Constants#AWS_CREDENTIALS_PROVIDER} logic.
- *
- */
-public class TestS3AAWSCredentialsProvider {
- private static final Logger LOG =
- LoggerFactory.getLogger(TestS3AAWSCredentialsProvider.class);
-
- @Rule
- public Timeout testTimeout = new Timeout(1 * 60 * 1000);
-
- @Rule
- public ExpectedException exception = ExpectedException.none();
-
- /**
- * Declare what exception to raise, and the text which must be found
- * in it.
- * @param exceptionClass class of exception
- * @param text text in exception
- */
- private void expectException(Class<? extends Throwable> exceptionClass,
- String text) {
- exception.expect(exceptionClass);
- exception.expectMessage(text);
- }
-
- @Test
- public void testBadConfiguration() throws IOException {
- Configuration conf = new Configuration();
- conf.set(AWS_CREDENTIALS_PROVIDER, "no.such.class");
- try {
- createFailingFS(conf);
- } catch (IOException e) {
- if (!(e.getCause() instanceof ClassNotFoundException)) {
- LOG.error("Unexpected nested cause: {} in {}", e.getCause(), e, e);
- throw e;
- }
- }
- }
-
- /**
- * Create a filesystem, expect it to fail by raising an IOException.
- * Raises an assertion exception if in fact the FS does get instantiated.
- * @param conf configuration
- * @throws IOException an expected exception.
- */
- private void createFailingFS(Configuration conf) throws IOException {
- S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf);
- fs.listStatus(new Path("/"));
- fail("Expected exception - got " + fs);
- }
-
- static class BadCredentialsProvider implements AWSCredentialsProvider {
-
- @SuppressWarnings("unused")
- public BadCredentialsProvider(URI name, Configuration conf) {
- }
-
- @Override
- public AWSCredentials getCredentials() {
- return new BasicAWSCredentials("bad_key", "bad_secret");
- }
-
- @Override
- public void refresh() {
- }
- }
-
- @Test
- public void testBadCredentials() throws Exception {
- Configuration conf = new Configuration();
- conf.set(AWS_CREDENTIALS_PROVIDER, BadCredentialsProvider.class.getName());
- try {
- createFailingFS(conf);
- } catch (AccessDeniedException e) {
- // expected
- }
- }
-
- static class GoodCredentialsProvider extends AWSCredentialsProviderChain {
-
- @SuppressWarnings("unused")
- public GoodCredentialsProvider(URI name, Configuration conf) {
- super(new BasicAWSCredentialsProvider(conf.get(ACCESS_KEY),
- conf.get(SECRET_KEY)), new InstanceProfileCredentialsProvider());
- }
- }
-
- @Test
- public void testGoodProvider() throws Exception {
- Configuration conf = new Configuration();
- conf.set(AWS_CREDENTIALS_PROVIDER, GoodCredentialsProvider.class.getName());
- S3ATestUtils.createTestFileSystem(conf);
- }
-
- @Test
- public void testAnonymousProvider() throws Exception {
- Configuration conf = new Configuration();
- conf.set(AWS_CREDENTIALS_PROVIDER,
- AnonymousAWSCredentialsProvider.class.getName());
- Path testFile = new Path(
- conf.getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE));
- S3ATestUtils.useCSVDataEndpoint(conf);
- FileSystem fs = FileSystem.newInstance(testFile.toUri(), conf);
- assertNotNull(fs);
- assertTrue(fs instanceof S3AFileSystem);
- FileStatus stat = fs.getFileStatus(testFile);
- assertNotNull(stat);
- assertEquals(testFile, stat.getPath());
- }
-
- /**
- * A credential provider whose constructor signature doesn't match.
- */
- static class ConstructorSignatureErrorProvider
- implements AWSCredentialsProvider {
-
- @SuppressWarnings("unused")
- public ConstructorSignatureErrorProvider(String str) {
- }
-
- @Override
- public AWSCredentials getCredentials() {
- return null;
- }
-
- @Override
- public void refresh() {
- }
- }
-
- /**
- * A credential provider whose constructor raises an NPE.
- */
- static class ConstructorFailureProvider
- implements AWSCredentialsProvider {
-
- @SuppressWarnings("unused")
- public ConstructorFailureProvider() {
- throw new NullPointerException("oops");
- }
-
- @Override
- public AWSCredentials getCredentials() {
- return null;
- }
-
- @Override
- public void refresh() {
- }
- }
-
- @Test
- public void testProviderWrongClass() throws Exception {
- expectProviderInstantiationFailure(this.getClass().getName(),
- NOT_AWS_PROVIDER);
- }
-
- @Test
- public void testProviderNotAClass() throws Exception {
- expectProviderInstantiationFailure("NoSuchClass",
- "ClassNotFoundException");
- }
-
- private void expectProviderInstantiationFailure(String option,
- String expectedErrorText) throws IOException {
- Configuration conf = new Configuration();
- conf.set(AWS_CREDENTIALS_PROVIDER, option);
- Path testFile = new Path(
- conf.getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE));
- expectException(IOException.class, expectedErrorText);
- URI uri = testFile.toUri();
- S3AUtils.createAWSCredentialProviderSet(uri, conf, uri);
- }
-
- @Test
- public void testProviderConstructorError() throws Exception {
- expectProviderInstantiationFailure(
- ConstructorSignatureErrorProvider.class.getName(),
- CONSTRUCTOR_EXCEPTION);
- }
-
- @Test
- public void testProviderFailureError() throws Exception {
- expectProviderInstantiationFailure(
- ConstructorFailureProvider.class.getName(),
- INSTANTIATION_EXCEPTION);
- }
-
- @Test
- public void testInstantiationChain() throws Throwable {
- Configuration conf = new Configuration();
- conf.set(AWS_CREDENTIALS_PROVIDER,
- TemporaryAWSCredentialsProvider.NAME
- + ", \t" + SimpleAWSCredentialsProvider.NAME
- + " ,\n " + AnonymousAWSCredentialsProvider.NAME);
- Path testFile = new Path(
- conf.getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE));
-
- URI uri = testFile.toUri();
- S3AUtils.createAWSCredentialProviderSet(uri, conf, uri);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockingThreadPool.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockingThreadPool.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockingThreadPool.java
deleted file mode 100644
index bd738b2..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockingThreadPool.java
+++ /dev/null
@@ -1,80 +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 org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.Timeout;
-
-/**
- * Demonstrate that the threadpool blocks additional client requests if
- * its queue is full (rather than throwing an exception) by initiating an
- * upload consisting of 4 parts with 2 threads and 1 spot in the queue. The
- * 4th part should not trigger an exception as it would with a
- * non-blocking threadpool.
- */
-public class TestS3ABlockingThreadPool {
-
- private Configuration conf;
- private S3AFileSystem fs;
-
- @Rule
- public Timeout testTimeout = new Timeout(30 * 60 * 1000);
-
- protected Path getTestPath() {
- return new Path("/tests3a");
- }
-
- @Before
- public void setUp() throws Exception {
- conf = new Configuration();
- conf.setLong(Constants.MIN_MULTIPART_THRESHOLD, 5 * 1024 * 1024);
- conf.setLong(Constants.MULTIPART_SIZE, 5 * 1024 * 1024);
- conf.setInt(Constants.MAX_THREADS, 2);
- conf.setInt(Constants.MAX_TOTAL_TASKS, 1);
- }
-
- @After
- public void tearDown() throws Exception {
- if (fs != null) {
- fs.delete(getTestPath(), true);
- }
- }
-
- @Test
- public void testRegularMultiPartUpload() throws Exception {
- fs = S3ATestUtils.createTestFileSystem(conf);
- ContractTestUtils.createAndVerifyFile(fs, getTestPath(), 16 * 1024 *
- 1024);
- }
-
- @Test
- public void testFastMultiPartUpload() throws Exception {
- conf.setBoolean(Constants.FAST_UPLOAD, true);
- fs = S3ATestUtils.createTestFileSystem(conf);
- ContractTestUtils.createAndVerifyFile(fs, getTestPath(), 16 * 1024 *
- 1024);
-
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlocksize.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlocksize.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlocksize.java
deleted file mode 100644
index 76fbf99..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlocksize.java
+++ /dev/null
@@ -1,93 +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 org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.apache.hadoop.fs.contract.AbstractFSContractTestBase;
-import org.apache.hadoop.fs.contract.s3a.S3AContract;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.Timeout;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
-import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
-import static org.apache.hadoop.fs.contract.ContractTestUtils.fileStatsToString;
-
-public class TestS3ABlocksize extends AbstractFSContractTestBase {
-
- private static final Logger LOG =
- LoggerFactory.getLogger(TestS3ABlocksize.class);
-
- @Override
- protected AbstractFSContract createContract(Configuration conf) {
- return new S3AContract(conf);
- }
-
- @Rule
- public Timeout testTimeout = new Timeout(30 * 60 * 1000);
-
- @Test
- @SuppressWarnings("deprecation")
- public void testBlockSize() throws Exception {
- FileSystem fs = getFileSystem();
- long defaultBlockSize = fs.getDefaultBlockSize();
- assertEquals("incorrect blocksize",
- S3AFileSystem.DEFAULT_BLOCKSIZE, defaultBlockSize);
- long newBlockSize = defaultBlockSize * 2;
- fs.getConf().setLong(Constants.FS_S3A_BLOCK_SIZE, newBlockSize);
-
- Path dir = path("testBlockSize");
- Path file = new Path(dir, "file");
- createFile(fs, file, true, dataset(1024, 'a', 'z' - 'a'));
- FileStatus fileStatus = fs.getFileStatus(file);
- assertEquals("Double default block size in stat(): " + fileStatus,
- newBlockSize,
- fileStatus.getBlockSize());
-
- // check the listing & assert that the block size is picked up by
- // this route too.
- boolean found = false;
- FileStatus[] listing = fs.listStatus(dir);
- for (FileStatus stat : listing) {
- LOG.info("entry: {}", stat);
- if (file.equals(stat.getPath())) {
- found = true;
- assertEquals("Double default block size in ls(): " + stat,
- newBlockSize,
- stat.getBlockSize());
- }
- }
- assertTrue("Did not find " + fileStatsToString(listing, ", "), found);
- }
-
- @Test
- public void testRootFileStatusHasBlocksize() throws Throwable {
- FileSystem fs = getFileSystem();
- FileStatus status = fs.getFileStatus(new Path("/"));
- assertTrue("Invalid root blocksize",
- status.getBlockSize() >= 0);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AConfiguration.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AConfiguration.java
deleted file mode 100644
index 99934a5..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AConfiguration.java
+++ /dev/null
@@ -1,435 +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.ClientConfiguration;
-import com.amazonaws.services.s3.AmazonS3Client;
-import com.amazonaws.services.s3.S3ClientOptions;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.commons.lang.reflect.FieldUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.apache.hadoop.fs.s3native.S3xLoginHelper;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.Timeout;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import java.io.File;
-import java.net.URI;
-
-import org.apache.hadoop.security.ProviderUtils;
-import org.apache.hadoop.security.alias.CredentialProvider;
-import org.apache.hadoop.security.alias.CredentialProviderFactory;
-import org.apache.hadoop.util.VersionInfo;
-import org.apache.http.HttpStatus;
-import org.junit.rules.TemporaryFolder;
-
-public class TestS3AConfiguration {
- private static final String EXAMPLE_ID = "AKASOMEACCESSKEY";
- private static final String EXAMPLE_KEY =
- "RGV0cm9pdCBSZ/WQgY2xl/YW5lZCB1cAEXAMPLE";
-
- private Configuration conf;
- private S3AFileSystem fs;
-
- private static final Logger LOG =
- LoggerFactory.getLogger(TestS3AConfiguration.class);
-
- private static final String TEST_ENDPOINT = "test.fs.s3a.endpoint";
-
- @Rule
- public Timeout testTimeout = new Timeout(30 * 60 * 1000);
-
- @Rule
- public final TemporaryFolder tempDir = new TemporaryFolder();
-
- /**
- * Test if custom endpoint is picked up.
- * <p/>
- * The test expects TEST_ENDPOINT to be defined in the Configuration
- * describing the endpoint of the bucket to which TEST_FS_S3A_NAME points
- * (f.i. "s3-eu-west-1.amazonaws.com" if the bucket is located in Ireland).
- * Evidently, the bucket has to be hosted in the region denoted by the
- * endpoint for the test to succeed.
- * <p/>
- * More info and the list of endpoint identifiers:
- * http://docs.aws.amazon.com/general/latest/gr/rande.html#s3_region
- *
- * @throws Exception
- */
- @Test
- public void testEndpoint() throws Exception {
- conf = new Configuration();
- String endpoint = conf.getTrimmed(TEST_ENDPOINT, "");
- if (endpoint.isEmpty()) {
- LOG.warn("Custom endpoint test skipped as " + TEST_ENDPOINT + "config " +
- "setting was not detected");
- } else {
- conf.set(Constants.ENDPOINT, endpoint);
- fs = S3ATestUtils.createTestFileSystem(conf);
- AmazonS3Client s3 = fs.getAmazonS3Client();
- String endPointRegion = "";
- // Differentiate handling of "s3-" and "s3." based endpoint identifiers
- String[] endpointParts = StringUtils.split(endpoint, '.');
- if (endpointParts.length == 3) {
- endPointRegion = endpointParts[0].substring(3);
- } else if (endpointParts.length == 4) {
- endPointRegion = endpointParts[1];
- } else {
- fail("Unexpected endpoint");
- }
- assertEquals("Endpoint config setting and bucket location differ: ",
- endPointRegion, s3.getBucketLocation(fs.getUri().getHost()));
- }
- }
-
- @Test
- public void testProxyConnection() throws Exception {
- conf = new Configuration();
- conf.setInt(Constants.MAX_ERROR_RETRIES, 2);
- conf.set(Constants.PROXY_HOST, "127.0.0.1");
- conf.setInt(Constants.PROXY_PORT, 1);
- String proxy =
- conf.get(Constants.PROXY_HOST) + ":" + conf.get(Constants.PROXY_PORT);
- try {
- fs = S3ATestUtils.createTestFileSystem(conf);
- fail("Expected a connection error for proxy server at " + proxy);
- } catch (AWSClientIOException e) {
- // expected
- }
- }
-
- @Test
- public void testProxyPortWithoutHost() throws Exception {
- conf = new Configuration();
- conf.setInt(Constants.MAX_ERROR_RETRIES, 2);
- conf.setInt(Constants.PROXY_PORT, 1);
- try {
- fs = S3ATestUtils.createTestFileSystem(conf);
- fail("Expected a proxy configuration error");
- } catch (IllegalArgumentException e) {
- String msg = e.toString();
- if (!msg.contains(Constants.PROXY_HOST) &&
- !msg.contains(Constants.PROXY_PORT)) {
- throw e;
- }
- }
- }
-
- @Test
- public void testAutomaticProxyPortSelection() throws Exception {
- conf = new Configuration();
- conf.setInt(Constants.MAX_ERROR_RETRIES, 2);
- conf.set(Constants.PROXY_HOST, "127.0.0.1");
- conf.set(Constants.SECURE_CONNECTIONS, "true");
- try {
- fs = S3ATestUtils.createTestFileSystem(conf);
- fail("Expected a connection error for proxy server");
- } catch (AWSClientIOException e) {
- // expected
- }
- conf.set(Constants.SECURE_CONNECTIONS, "false");
- try {
- fs = S3ATestUtils.createTestFileSystem(conf);
- fail("Expected a connection error for proxy server");
- } catch (AWSClientIOException e) {
- // expected
- }
- }
-
- @Test
- public void testUsernameInconsistentWithPassword() throws Exception {
- conf = new Configuration();
- conf.setInt(Constants.MAX_ERROR_RETRIES, 2);
- conf.set(Constants.PROXY_HOST, "127.0.0.1");
- conf.setInt(Constants.PROXY_PORT, 1);
- conf.set(Constants.PROXY_USERNAME, "user");
- try {
- fs = S3ATestUtils.createTestFileSystem(conf);
- fail("Expected a connection error for proxy server");
- } catch (IllegalArgumentException e) {
- String msg = e.toString();
- if (!msg.contains(Constants.PROXY_USERNAME) &&
- !msg.contains(Constants.PROXY_PASSWORD)) {
- throw e;
- }
- }
- conf = new Configuration();
- conf.setInt(Constants.MAX_ERROR_RETRIES, 2);
- conf.set(Constants.PROXY_HOST, "127.0.0.1");
- conf.setInt(Constants.PROXY_PORT, 1);
- conf.set(Constants.PROXY_PASSWORD, "password");
- try {
- fs = S3ATestUtils.createTestFileSystem(conf);
- fail("Expected a connection error for proxy server");
- } catch (IllegalArgumentException e) {
- String msg = e.toString();
- if (!msg.contains(Constants.PROXY_USERNAME) &&
- !msg.contains(Constants.PROXY_PASSWORD)) {
- throw e;
- }
- }
- }
-
- @Test
- public void testCredsFromCredentialProvider() 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());
-
- provisionAccessKeys(conf);
-
- conf.set(Constants.ACCESS_KEY, EXAMPLE_ID + "LJM");
- S3xLoginHelper.Login creds =
- S3AUtils.getAWSAccessKeys(new URI("s3a://foobar"), conf);
- assertEquals("AccessKey incorrect.", EXAMPLE_ID, creds.getUser());
- assertEquals("SecretKey incorrect.", EXAMPLE_KEY, creds.getPassword());
- }
-
- void provisionAccessKeys(final Configuration conf) throws Exception {
- // add our creds to the provider
- final CredentialProvider provider =
- CredentialProviderFactory.getProviders(conf).get(0);
- provider.createCredentialEntry(Constants.ACCESS_KEY,
- EXAMPLE_ID.toCharArray());
- provider.createCredentialEntry(Constants.SECRET_KEY,
- EXAMPLE_KEY.toCharArray());
- provider.flush();
- }
-
- @Test
- public void testCredsFromUserInfo() 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());
-
- provisionAccessKeys(conf);
-
- conf.set(Constants.ACCESS_KEY, EXAMPLE_ID + "LJM");
- URI uriWithUserInfo = new URI("s3a://123:456@foobar");
- S3xLoginHelper.Login creds =
- S3AUtils.getAWSAccessKeys(uriWithUserInfo, conf);
- assertEquals("AccessKey incorrect.", "123", creds.getUser());
- assertEquals("SecretKey incorrect.", "456", creds.getPassword());
- }
-
- @Test
- public void testIDFromUserInfoSecretFromCredentialProvider()
- 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());
-
- provisionAccessKeys(conf);
-
- conf.set(Constants.ACCESS_KEY, EXAMPLE_ID + "LJM");
- URI uriWithUserInfo = new URI("s3a://123@foobar");
- S3xLoginHelper.Login creds =
- S3AUtils.getAWSAccessKeys(uriWithUserInfo, conf);
- assertEquals("AccessKey incorrect.", "123", creds.getUser());
- assertEquals("SecretKey incorrect.", EXAMPLE_KEY, creds.getPassword());
- }
-
- @Test
- public void testSecretFromCredentialProviderIDFromConfig() 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(Constants.SECRET_KEY,
- EXAMPLE_KEY.toCharArray());
- provider.flush();
-
- conf.set(Constants.ACCESS_KEY, EXAMPLE_ID);
- S3xLoginHelper.Login creds =
- S3AUtils.getAWSAccessKeys(new URI("s3a://foobar"), conf);
- assertEquals("AccessKey incorrect.", EXAMPLE_ID, creds.getUser());
- assertEquals("SecretKey incorrect.", EXAMPLE_KEY, creds.getPassword());
- }
-
- @Test
- public void testIDFromCredentialProviderSecretFromConfig() 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(Constants.ACCESS_KEY,
- EXAMPLE_ID.toCharArray());
- provider.flush();
-
- conf.set(Constants.SECRET_KEY, EXAMPLE_KEY);
- S3xLoginHelper.Login creds =
- S3AUtils.getAWSAccessKeys(new URI("s3a://foobar"), conf);
- assertEquals("AccessKey incorrect.", EXAMPLE_ID, creds.getUser());
- assertEquals("SecretKey incorrect.", EXAMPLE_KEY, creds.getPassword());
- }
-
- @Test
- public void testExcludingS3ACredentialProvider() 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,
- "jceks://s3a/foobar," + jks.toString());
-
- // first make sure that the s3a based provider is removed
- Configuration c = ProviderUtils.excludeIncompatibleCredentialProviders(
- conf, S3AFileSystem.class);
- String newPath = conf.get(
- CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH);
- assertFalse("Provider Path incorrect", newPath.contains("s3a://"));
-
- // now let's make sure the new path is created by the S3AFileSystem
- // and the integration still works. Let's provision the keys through
- // the altered configuration instance and then try and access them
- // using the original config with the s3a provider in the path.
- provisionAccessKeys(c);
-
- conf.set(Constants.ACCESS_KEY, EXAMPLE_ID + "LJM");
- URI uriWithUserInfo = new URI("s3a://123:456@foobar");
- S3xLoginHelper.Login creds =
- S3AUtils.getAWSAccessKeys(uriWithUserInfo, conf);
- assertEquals("AccessKey incorrect.", "123", creds.getUser());
- assertEquals("SecretKey incorrect.", "456", creds.getPassword());
-
- }
-
- @Test
- public void shouldBeAbleToSwitchOnS3PathStyleAccessViaConfigProperty() throws Exception {
-
- conf = new Configuration();
- conf.set(Constants.PATH_STYLE_ACCESS, Boolean.toString(true));
- assertTrue(conf.getBoolean(Constants.PATH_STYLE_ACCESS, false));
-
- try {
- fs = S3ATestUtils.createTestFileSystem(conf);
- assertNotNull(fs);
- AmazonS3Client s3 = fs.getAmazonS3Client();
- assertNotNull(s3);
- S3ClientOptions clientOptions = getField(s3, S3ClientOptions.class,
- "clientOptions");
- assertTrue("Expected to find path style access to be switched on!",
- clientOptions.isPathStyleAccess());
- byte[] file = ContractTestUtils.toAsciiByteArray("test file");
- ContractTestUtils.writeAndRead(fs, new Path("/path/style/access/testFile"), file, file.length, conf.getInt(Constants.FS_S3A_BLOCK_SIZE, file.length), false, true);
- } catch (final AWSS3IOException e) {
- LOG.error("Caught exception: ", e);
- // Catch/pass standard path style access behaviour when live bucket
- // isn't in the same region as the s3 client default. See
- // http://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html
- assertEquals(e.getStatusCode(), HttpStatus.SC_MOVED_PERMANENTLY);
- }
- }
-
- @Test
- public void testDefaultUserAgent() throws Exception {
- conf = new Configuration();
- fs = S3ATestUtils.createTestFileSystem(conf);
- assertNotNull(fs);
- AmazonS3Client s3 = fs.getAmazonS3Client();
- assertNotNull(s3);
- ClientConfiguration awsConf = getField(s3, ClientConfiguration.class,
- "clientConfiguration");
- assertEquals("Hadoop " + VersionInfo.getVersion(), awsConf.getUserAgent());
- }
-
- @Test
- public void testCustomUserAgent() throws Exception {
- conf = new Configuration();
- conf.set(Constants.USER_AGENT_PREFIX, "MyApp");
- fs = S3ATestUtils.createTestFileSystem(conf);
- assertNotNull(fs);
- AmazonS3Client s3 = fs.getAmazonS3Client();
- assertNotNull(s3);
- ClientConfiguration awsConf = getField(s3, ClientConfiguration.class,
- "clientConfiguration");
- assertEquals("MyApp, Hadoop " + VersionInfo.getVersion(),
- awsConf.getUserAgent());
- }
-
- @Test
- public void testCloseIdempotent() throws Throwable {
- conf = new Configuration();
- fs = S3ATestUtils.createTestFileSystem(conf);
- fs.close();
- fs.close();
- }
-
- /**
- * Reads and returns a field from an object using reflection. If the field
- * cannot be found, is null, or is not the expected type, then this method
- * fails the test.
- *
- * @param target object to read
- * @param fieldType type of field to read, which will also be the return type
- * @param fieldName name of field to read
- * @return field that was read
- * @throws IllegalAccessException if access not allowed
- */
- private static <T> T getField(Object target, Class<T> fieldType,
- String fieldName) throws IllegalAccessException {
- Object obj = FieldUtils.readField(target, fieldName, true);
- assertNotNull(String.format(
- "Could not read field named %s in object with class %s.", fieldName,
- target.getClass().getName()), obj);
- assertTrue(String.format(
- "Unexpected type found for field named %s, expected %s, actual %s.",
- fieldName, fieldType.getName(), obj.getClass().getName()),
- fieldType.isAssignableFrom(obj.getClass()));
- return fieldType.cast(obj);
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ACredentialsInURL.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ACredentialsInURL.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ACredentialsInURL.java
deleted file mode 100644
index 8cb7c0f..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ACredentialsInURL.java
+++ /dev/null
@@ -1,155 +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 org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IOUtils;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Assume;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.Timeout;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.net.URI;
-import java.net.URLEncoder;
-import java.nio.file.AccessDeniedException;
-
-import static org.apache.hadoop.fs.s3a.S3ATestConstants.TEST_FS_S3A_NAME;
-
-/**
- * Tests that credentials can go into the URL. This includes a valid
- * set, and a check that an invalid set do at least get stripped out
- * of the final URI
- */
-public class TestS3ACredentialsInURL extends Assert {
- private S3AFileSystem fs;
- private static final Logger LOG =
- LoggerFactory.getLogger(TestS3ACredentialsInURL.class);
- @Rule
- public Timeout testTimeout = new Timeout(30 * 60 * 1000);
-
- @After
- public void teardown() {
- IOUtils.closeStream(fs);
- }
-
- /**
- * Test instantiation.
- * @throws Throwable
- */
- @Test
- public void testInstantiateFromURL() throws Throwable {
-
- Configuration conf = new Configuration();
- String accessKey = conf.get(Constants.ACCESS_KEY);
- String secretKey = conf.get(Constants.SECRET_KEY);
- String fsname = conf.getTrimmed(TEST_FS_S3A_NAME, "");
- Assume.assumeNotNull(fsname, accessKey, secretKey);
- URI original = new URI(fsname);
- URI secretsURI = createUriWithEmbeddedSecrets(original,
- accessKey, secretKey);
- if (secretKey.contains("/")) {
- assertTrue("test URI encodes the / symbol", secretsURI.toString().
- contains("%252F"));
- }
- if (secretKey.contains("+")) {
- assertTrue("test URI encodes the + symbol", secretsURI.toString().
- contains("%252B"));
- }
- assertFalse("Does not contain secrets", original.equals(secretsURI));
-
- conf.set(TEST_FS_S3A_NAME, secretsURI.toString());
- conf.unset(Constants.ACCESS_KEY);
- conf.unset(Constants.SECRET_KEY);
- fs = S3ATestUtils.createTestFileSystem(conf);
- String fsURI = fs.getUri().toString();
- assertFalse("FS URI contains a @ symbol", fsURI.contains("@"));
- assertFalse("FS URI contains a % symbol", fsURI.contains("%"));
- if (!original.toString().startsWith(fsURI)) {
- fail("Filesystem URI does not match original");
- }
- validate("original path", new Path(original));
- validate("bare path", new Path("/"));
- validate("secrets path", new Path(secretsURI));
- }
-
- private void validate(String text, Path path) throws IOException {
- try {
- fs.canonicalizeUri(path.toUri());
- fs.checkPath(path);
- assertTrue(text + " Not a directory",
- fs.getFileStatus(new Path("/")).isDirectory());
- fs.globStatus(path);
- } catch (AssertionError e) {
- throw e;
- } catch (Exception e) {
- LOG.debug("{} failure: {}", text, e, e);
- fail(text + " Test failed");
- }
- }
-
- /**
- * Set up some invalid credentials, verify login is rejected.
- * @throws Throwable
- */
- @Test
- public void testInvalidCredentialsFail() throws Throwable {
- Configuration conf = new Configuration();
- String fsname = conf.getTrimmed(TEST_FS_S3A_NAME, "");
- Assume.assumeNotNull(fsname);
- URI original = new URI(fsname);
- URI testURI = createUriWithEmbeddedSecrets(original, "user", "//");
-
- conf.set(TEST_FS_S3A_NAME, testURI.toString());
- fs = S3ATestUtils.createTestFileSystem(conf);
- try {
- S3AFileStatus status = fs.getFileStatus(new Path("/"));
- fail("Expected an AccessDeniedException, got " + status);
- } catch (AccessDeniedException e) {
- // expected
- }
-
- }
-
- private URI createUriWithEmbeddedSecrets(URI original,
- String accessKey,
- String secretKey) throws UnsupportedEncodingException {
- String encodedSecretKey = URLEncoder.encode(secretKey, "UTF-8");
- String formattedString = String.format("%s://%s:%s@%s/%s/",
- original.getScheme(),
- accessKey,
- encodedSecretKey,
- original.getHost(),
- original.getPath());
- URI testURI;
- try {
- testURI = new Path(formattedString).toUri();
- } catch (IllegalArgumentException e) {
- // inner cause is stripped to keep any secrets out of stack traces
- throw new IllegalArgumentException("Could not encode Path");
- }
- return testURI;
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AEncryption.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AEncryption.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AEncryption.java
deleted file mode 100644
index 43a26ff..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AEncryption.java
+++ /dev/null
@@ -1,104 +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.services.s3.model.ObjectMetadata;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.apache.hadoop.io.IOUtils;
-import org.junit.Test;
-
-import java.io.IOException;
-
-import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
-import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
-
-/**
- * Test whether or not encryption works by turning it on. Some checks
- * are made for different file sizes as there have been reports that the
- * file length may be rounded up to match word boundaries.
- */
-public class TestS3AEncryption extends AbstractS3ATestBase {
- private static final String AES256 = Constants.SERVER_SIDE_ENCRYPTION_AES256;
-
- @Override
- protected Configuration createConfiguration() {
- Configuration conf = super.createConfiguration();
- S3ATestUtils.disableFilesystemCaching(conf);
- conf.set(Constants.SERVER_SIDE_ENCRYPTION_ALGORITHM,
- AES256);
- return conf;
- }
-
- private static final int[] SIZES = {
- 0, 1, 2, 3, 4, 5, 254, 255, 256, 257, 2 ^ 10 - 3, 2 ^ 11 - 2, 2 ^ 12 - 1
- };
-
- @Override
- public void teardown() throws Exception {
- super.teardown();
- IOUtils.closeStream(getFileSystem());
- }
-
- @Test
- public void testEncryption() throws Throwable {
- for (int size: SIZES) {
- validateEncryptionForFilesize(size);
- }
- }
-
- @Test
- public void testEncryptionOverRename() throws Throwable {
- skipIfEncryptionTestsDisabled(getConfiguration());
- Path src = path(createFilename(1024));
- byte[] data = dataset(1024, 'a', 'z');
- S3AFileSystem fs = getFileSystem();
- writeDataset(fs, src, data, data.length, 1024 * 1024, true);
- ContractTestUtils.verifyFileContents(fs, src, data);
- Path dest = path(src.getName() + "-copy");
- fs.rename(src, dest);
- ContractTestUtils.verifyFileContents(fs, dest, data);
- assertEncrypted(dest);
- }
-
- protected void validateEncryptionForFilesize(int len) throws IOException {
- skipIfEncryptionTestsDisabled(getConfiguration());
- describe("Create an encrypted file of size " + len);
- String src = createFilename(len);
- Path path = writeThenReadFile(src, len);
- assertEncrypted(path);
- rm(getFileSystem(), path, false, false);
- }
-
- private String createFilename(int len) {
- return String.format("%s-%04x", methodName.getMethodName(), len);
- }
-
- /**
- * Assert that at path references an encrypted blob.
- * @param path path
- * @throws IOException on a failure
- */
- private void assertEncrypted(Path path) throws IOException {
- ObjectMetadata md = getFileSystem().getObjectMetadata(path);
- assertEquals(AES256, md.getSSEAlgorithm());
- }
-
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AEncryptionAlgorithmPropagation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AEncryptionAlgorithmPropagation.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AEncryptionAlgorithmPropagation.java
deleted file mode 100644
index 04ca830..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AEncryptionAlgorithmPropagation.java
+++ /dev/null
@@ -1,82 +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 org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IOUtils;
-import org.junit.Test;
-
-import java.io.IOException;
-
-import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
-
-/**
- * Test whether or not encryption settings propagate by choosing an invalid
- * one. We expect the write to fail with a 400 bad request error
- */
-public class TestS3AEncryptionAlgorithmPropagation extends AbstractS3ATestBase {
-
- @Override
- protected Configuration createConfiguration() {
- Configuration conf = super.createConfiguration();
- S3ATestUtils.disableFilesystemCaching(conf);
- conf.set(Constants.SERVER_SIDE_ENCRYPTION_ALGORITHM,
- "DES");
- return conf;
- }
-
- @Override
- public void teardown() throws Exception {
- super.teardown();
- IOUtils.closeStream(getFileSystem());
- }
-
- @Test
- public void testEncrypt0() throws Throwable {
- writeThenReadFileToFailure(0);
- }
-
- @Test
- public void testEncrypt256() throws Throwable {
- writeThenReadFileToFailure(256);
- }
-
- /**
- * Make this a no-op so test setup doesn't fail.
- * @param path path path
- * @throws IOException on any failure
- */
- @Override
- protected void mkdirs(Path path) throws IOException {
-
- }
-
- protected void writeThenReadFileToFailure(int len) throws IOException {
- skipIfEncryptionTestsDisabled(getConfiguration());
- describe("Create an encrypted file of size " + len);
- try {
- writeThenReadFile(methodName.getMethodName() + '-' + len, len);
- fail("Expected an exception about an illegal encryption algorithm");
- } catch (AWSS3IOException e) {
- assertStatusCode(e, 400);
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AEncryptionFastOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AEncryptionFastOutputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AEncryptionFastOutputStream.java
deleted file mode 100644
index 1fa8486..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AEncryptionFastOutputStream.java
+++ /dev/null
@@ -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.s3a;
-
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * Run the encryption tests against the Fast output stream.
- * This verifies that both file writing paths can encrypt their data.
- */
-public class TestS3AEncryptionFastOutputStream extends TestS3AEncryption {
-
- @Override
- protected Configuration createConfiguration() {
- Configuration conf = super.createConfiguration();
- conf.setBoolean(Constants.FAST_UPLOAD, true);
- return conf;
- }
-}
---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org
[7/7] hadoop git commit: HADOOP-13447. Refactor S3AFileSystem to
support introduction of separate metadata repository and tests. Contributed
by Chris Nauroth.
Posted by cn...@apache.org.
HADOOP-13447. Refactor S3AFileSystem to support introduction of separate metadata repository and tests. Contributed by Chris Nauroth.
(cherry picked from commit d152557cf7f4d2288524c222fcbaf152bdc038b0)
(cherry picked from commit e28930a38b3868bc701aeef229831292320b4a3a)
Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/67d8301e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/67d8301e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/67d8301e
Branch: refs/heads/branch-2.8
Commit: 67d8301e55c19057957173fdfc28e77e070f2787
Parents: 027d76f
Author: Chris Nauroth <cn...@apache.org>
Authored: Tue Sep 6 09:36:21 2016 -0700
Committer: Chris Nauroth <cn...@apache.org>
Committed: Mon Oct 10 14:01:19 2016 -0700
----------------------------------------------------------------------
hadoop-tools/hadoop-aws/pom.xml | 5 +
.../org/apache/hadoop/fs/s3a/Constants.java | 29 ++-
.../hadoop/fs/s3a/S3AFastOutputStream.java | 6 +-
.../org/apache/hadoop/fs/s3a/S3AFileSystem.java | 192 ++-------------
.../apache/hadoop/fs/s3a/S3AInputStream.java | 6 +-
.../java/org/apache/hadoop/fs/s3a/S3AUtils.java | 40 ++++
.../apache/hadoop/fs/s3a/S3ClientFactory.java | 232 +++++++++++++++++++
.../hadoop/fs/s3native/S3xLoginHelper.java | 3 +
.../hadoop/fs/s3a/AbstractS3AMockTest.java | 70 ++++++
.../hadoop/fs/s3a/ITestS3AConfiguration.java | 10 +-
.../hadoop/fs/s3a/MockS3ClientFactory.java | 40 ++++
.../hadoop/fs/s3a/TestS3AGetFileStatus.java | 126 ++++++++++
12 files changed, 567 insertions(+), 192 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hadoop/blob/67d8301e/hadoop-tools/hadoop-aws/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml
index c91081d..e3befd4 100644
--- a/hadoop-tools/hadoop-aws/pom.xml
+++ b/hadoop-tools/hadoop-aws/pom.xml
@@ -295,6 +295,11 @@
<scope>test</scope>
</dependency>
<dependency>
+ <groupId>org.mockito</groupId>
+ <artifactId>mockito-all</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-mapreduce-client-jobclient</artifactId>
<scope>test</scope>
http://git-wip-us.apache.org/repos/asf/hadoop/blob/67d8301e/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
index 1508675..cf97c35 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
@@ -53,7 +53,8 @@ public final class Constants {
public static final int DEFAULT_MAXIMUM_CONNECTIONS = 15;
// connect to s3 over ssl?
- public static final String SECURE_CONNECTIONS = "fs.s3a.connection.ssl.enabled";
+ public static final String SECURE_CONNECTIONS =
+ "fs.s3a.connection.ssl.enabled";
public static final boolean DEFAULT_SECURE_CONNECTIONS = true;
//use a custom endpoint?
@@ -75,7 +76,8 @@ public final class Constants {
public static final int DEFAULT_MAX_ERROR_RETRIES = 20;
// seconds until we give up trying to establish a connection to s3
- public static final String ESTABLISH_TIMEOUT = "fs.s3a.connection.establish.timeout";
+ public static final String ESTABLISH_TIMEOUT =
+ "fs.s3a.connection.establish.timeout";
public static final int DEFAULT_ESTABLISH_TIMEOUT = 50000;
// seconds until we give up on a connection to s3
@@ -116,11 +118,13 @@ public final class Constants {
public static final long DEFAULT_MULTIPART_SIZE = 104857600; // 100 MB
// minimum size in bytes before we start a multipart uploads or copy
- public static final String MIN_MULTIPART_THRESHOLD = "fs.s3a.multipart.threshold";
+ public static final String MIN_MULTIPART_THRESHOLD =
+ "fs.s3a.multipart.threshold";
public static final long DEFAULT_MIN_MULTIPART_THRESHOLD = Integer.MAX_VALUE;
//enable multiobject-delete calls?
- public static final String ENABLE_MULTI_DELETE = "fs.s3a.multiobjectdelete.enable";
+ public static final String ENABLE_MULTI_DELETE =
+ "fs.s3a.multiobjectdelete.enable";
// comma separated list of directories
public static final String BUFFER_DIR = "fs.s3a.buffer.dir";
@@ -139,11 +143,13 @@ public final class Constants {
public static final String DEFAULT_CANNED_ACL = "";
// should we try to purge old multipart uploads when starting up
- public static final String PURGE_EXISTING_MULTIPART = "fs.s3a.multipart.purge";
+ public static final String PURGE_EXISTING_MULTIPART =
+ "fs.s3a.multipart.purge";
public static final boolean DEFAULT_PURGE_EXISTING_MULTIPART = false;
// purge any multipart uploads older than this number of seconds
- public static final String PURGE_EXISTING_MULTIPART_AGE = "fs.s3a.multipart.purge.age";
+ public static final String PURGE_EXISTING_MULTIPART_AGE =
+ "fs.s3a.multipart.purge.age";
public static final long DEFAULT_PURGE_EXISTING_MULTIPART_AGE = 14400;
// s3 server-side encryption
@@ -203,4 +209,15 @@ public final class Constants {
*/
@InterfaceStability.Unstable
public static final String INPUT_FADV_RANDOM = "random";
+
+ @InterfaceAudience.Private
+ @InterfaceStability.Unstable
+ public static final String S3_CLIENT_FACTORY_IMPL =
+ "fs.s3a.s3.client.factory.impl";
+
+ @InterfaceAudience.Private
+ @InterfaceStability.Unstable
+ public static final Class<? extends S3ClientFactory>
+ DEFAULT_S3_CLIENT_FACTORY_IMPL =
+ S3ClientFactory.DefaultS3ClientFactory.class;
}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/67d8301e/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java
index 5509d36..c25d0fb 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java
@@ -21,7 +21,7 @@ package org.apache.hadoop.fs.s3a;
import com.amazonaws.AmazonClientException;
import com.amazonaws.event.ProgressEvent;
import com.amazonaws.event.ProgressListener;
-import com.amazonaws.services.s3.AmazonS3Client;
+import com.amazonaws.services.s3.AmazonS3;
import com.amazonaws.services.s3.model.AbortMultipartUploadRequest;
import com.amazonaws.services.s3.model.CannedAccessControlList;
import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest;
@@ -71,7 +71,7 @@ public class S3AFastOutputStream extends OutputStream {
private static final Logger LOG = S3AFileSystem.LOG;
private final String key;
private final String bucket;
- private final AmazonS3Client client;
+ private final AmazonS3 client;
private final int partSize;
private final int multiPartThreshold;
private final S3AFileSystem fs;
@@ -102,7 +102,7 @@ public class S3AFastOutputStream extends OutputStream {
* @param threadPoolExecutor thread factory
* @throws IOException on any problem
*/
- public S3AFastOutputStream(AmazonS3Client client,
+ public S3AFastOutputStream(AmazonS3 client,
S3AFileSystem fs,
String bucket,
String key,
http://git-wip-us.apache.org/repos/asf/hadoop/blob/67d8301e/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
index 24a8d64..15bd23a 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
@@ -36,11 +36,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
import com.amazonaws.AmazonClientException;
import com.amazonaws.AmazonServiceException;
-import com.amazonaws.ClientConfiguration;
-import com.amazonaws.Protocol;
-import com.amazonaws.auth.AWSCredentialsProvider;
-import com.amazonaws.services.s3.AmazonS3Client;
-import com.amazonaws.services.s3.S3ClientOptions;
+import com.amazonaws.services.s3.AmazonS3;
import com.amazonaws.services.s3.model.AmazonS3Exception;
import com.amazonaws.services.s3.model.CannedAccessControlList;
import com.amazonaws.services.s3.model.DeleteObjectsRequest;
@@ -59,7 +55,6 @@ import com.amazonaws.services.s3.transfer.Upload;
import com.amazonaws.event.ProgressListener;
import com.amazonaws.event.ProgressEvent;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.classification.InterfaceAudience;
@@ -84,7 +79,7 @@ import org.apache.hadoop.fs.StorageStatistics;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.s3native.S3xLoginHelper;
import org.apache.hadoop.util.Progressable;
-import org.apache.hadoop.util.VersionInfo;
+import org.apache.hadoop.util.ReflectionUtils;
import static org.apache.hadoop.fs.s3a.Constants.*;
import static org.apache.hadoop.fs.s3a.Listing.ACCEPT_ALL;
@@ -116,7 +111,7 @@ public class S3AFileSystem extends FileSystem {
public static final int DEFAULT_BLOCKSIZE = 32 * 1024 * 1024;
private URI uri;
private Path workingDir;
- private AmazonS3Client s3;
+ private AmazonS3 s3;
private String bucket;
private int maxKeys;
private Listing listing;
@@ -156,37 +151,11 @@ public class S3AFileSystem extends FileSystem {
bucket = name.getHost();
- AWSCredentialsProvider credentials =
- createAWSCredentialProviderSet(name, conf, uri);
-
- ClientConfiguration awsConf = new ClientConfiguration();
- awsConf.setMaxConnections(intOption(conf, MAXIMUM_CONNECTIONS,
- DEFAULT_MAXIMUM_CONNECTIONS, 1));
- boolean secureConnections = conf.getBoolean(SECURE_CONNECTIONS,
- DEFAULT_SECURE_CONNECTIONS);
- awsConf.setProtocol(secureConnections ? Protocol.HTTPS : Protocol.HTTP);
- awsConf.setMaxErrorRetry(intOption(conf, MAX_ERROR_RETRIES,
- DEFAULT_MAX_ERROR_RETRIES, 0));
- awsConf.setConnectionTimeout(intOption(conf, ESTABLISH_TIMEOUT,
- DEFAULT_ESTABLISH_TIMEOUT, 0));
- awsConf.setSocketTimeout(intOption(conf, SOCKET_TIMEOUT,
- DEFAULT_SOCKET_TIMEOUT, 0));
- int sockSendBuffer = intOption(conf, SOCKET_SEND_BUFFER,
- DEFAULT_SOCKET_SEND_BUFFER, 2048);
- int sockRecvBuffer = intOption(conf, SOCKET_RECV_BUFFER,
- DEFAULT_SOCKET_RECV_BUFFER, 2048);
- awsConf.setSocketBufferSizeHints(sockSendBuffer, sockRecvBuffer);
- String signerOverride = conf.getTrimmed(SIGNING_ALGORITHM, "");
- if (!signerOverride.isEmpty()) {
- LOG.debug("Signer override = {}", signerOverride);
- awsConf.setSignerOverride(signerOverride);
- }
-
- initProxySupport(conf, awsConf, secureConnections);
-
- initUserAgent(conf, awsConf);
-
- initAmazonS3Client(conf, credentials, awsConf);
+ Class<? extends S3ClientFactory> s3ClientFactoryClass = conf.getClass(
+ S3_CLIENT_FACTORY_IMPL, DEFAULT_S3_CLIENT_FACTORY_IMPL,
+ S3ClientFactory.class);
+ s3 = ReflectionUtils.newInstance(s3ClientFactoryClass, conf)
+ .createS3Client(name, uri);
maxKeys = intOption(conf, MAX_PAGING_KEYS, DEFAULT_MAX_PAGING_KEYS, 1);
listing = new Listing(this);
@@ -282,50 +251,6 @@ public class S3AFileSystem extends FileSystem {
}
}
- void initProxySupport(Configuration conf, ClientConfiguration awsConf,
- boolean secureConnections) throws IllegalArgumentException {
- String proxyHost = conf.getTrimmed(PROXY_HOST, "");
- int proxyPort = conf.getInt(PROXY_PORT, -1);
- if (!proxyHost.isEmpty()) {
- awsConf.setProxyHost(proxyHost);
- if (proxyPort >= 0) {
- awsConf.setProxyPort(proxyPort);
- } else {
- if (secureConnections) {
- LOG.warn("Proxy host set without port. Using HTTPS default 443");
- awsConf.setProxyPort(443);
- } else {
- LOG.warn("Proxy host set without port. Using HTTP default 80");
- awsConf.setProxyPort(80);
- }
- }
- String proxyUsername = conf.getTrimmed(PROXY_USERNAME);
- String proxyPassword = conf.getTrimmed(PROXY_PASSWORD);
- if ((proxyUsername == null) != (proxyPassword == null)) {
- String msg = "Proxy error: " + PROXY_USERNAME + " or " +
- PROXY_PASSWORD + " set without the other.";
- LOG.error(msg);
- throw new IllegalArgumentException(msg);
- }
- awsConf.setProxyUsername(proxyUsername);
- awsConf.setProxyPassword(proxyPassword);
- awsConf.setProxyDomain(conf.getTrimmed(PROXY_DOMAIN));
- awsConf.setProxyWorkstation(conf.getTrimmed(PROXY_WORKSTATION));
- if (LOG.isDebugEnabled()) {
- LOG.debug("Using proxy server {}:{} as user {} with password {} on " +
- "domain {} as workstation {}", awsConf.getProxyHost(),
- awsConf.getProxyPort(),
- String.valueOf(awsConf.getProxyUsername()),
- awsConf.getProxyPassword(), awsConf.getProxyDomain(),
- awsConf.getProxyWorkstation());
- }
- } else if (proxyPort >= 0) {
- String msg = "Proxy error: " + PROXY_PORT + " set without " + PROXY_HOST;
- LOG.error(msg);
- throw new IllegalArgumentException(msg);
- }
- }
-
/**
* Get S3A Instrumentation. For test purposes.
* @return this instance's instrumentation.
@@ -334,53 +259,9 @@ public class S3AFileSystem extends FileSystem {
return instrumentation;
}
- /**
- * Initializes the User-Agent header to send in HTTP requests to the S3
- * back-end. We always include the Hadoop version number. The user also may
- * set an optional custom prefix to put in front of the Hadoop version number.
- * The AWS SDK interally appends its own information, which seems to include
- * the AWS SDK version, OS and JVM version.
- *
- * @param conf Hadoop configuration
- * @param awsConf AWS SDK configuration
- */
- private void initUserAgent(Configuration conf, ClientConfiguration awsConf) {
- String userAgent = "Hadoop " + VersionInfo.getVersion();
- String userAgentPrefix = conf.getTrimmed(USER_AGENT_PREFIX, "");
- if (!userAgentPrefix.isEmpty()) {
- userAgent = userAgentPrefix + ", " + userAgent;
- }
- LOG.debug("Using User-Agent: {}", userAgent);
- awsConf.setUserAgent(userAgent);
- }
-
- private void initAmazonS3Client(Configuration conf,
- AWSCredentialsProvider credentials, ClientConfiguration awsConf)
- throws IllegalArgumentException {
- s3 = new AmazonS3Client(credentials, awsConf);
- String endPoint = conf.getTrimmed(ENDPOINT, "");
- if (!endPoint.isEmpty()) {
- try {
- s3.setEndpoint(endPoint);
- } catch (IllegalArgumentException e) {
- String msg = "Incorrect endpoint: " + e.getMessage();
- LOG.error(msg);
- throw new IllegalArgumentException(msg, e);
- }
- }
- enablePathStyleAccessIfRequired(conf);
- }
-
- private void enablePathStyleAccessIfRequired(Configuration conf) {
- final boolean pathStyleAccess = conf.getBoolean(PATH_STYLE_ACCESS, false);
- if (pathStyleAccess) {
- LOG.debug("Enabling path style access!");
- s3.setS3ClientOptions(new S3ClientOptions().withPathStyleAccess(true));
- }
- }
-
private void initTransferManager() {
- TransferManagerConfiguration transferConfiguration = new TransferManagerConfiguration();
+ TransferManagerConfiguration transferConfiguration =
+ new TransferManagerConfiguration();
transferConfiguration.setMinimumUploadPartSize(partSize);
transferConfiguration.setMultipartUploadThreshold(multiPartThreshold);
transferConfiguration.setMultipartCopyPartSize(partSize);
@@ -451,7 +332,7 @@ public class S3AFileSystem extends FileSystem {
* @return AmazonS3Client
*/
@VisibleForTesting
- AmazonS3Client getAmazonS3Client() {
+ AmazonS3 getAmazonS3Client() {
return s3;
}
@@ -475,10 +356,6 @@ public class S3AFileSystem extends FileSystem {
this.inputPolicy = inputPolicy;
}
- public S3AFileSystem() {
- super();
- }
-
/**
* Turns a path (relative or otherwise) into an S3 key.
*
@@ -822,8 +699,10 @@ public class S3AFileSystem extends FileSystem {
while (true) {
for (S3ObjectSummary summary : objects.getObjectSummaries()) {
- keysToDelete.add(new DeleteObjectsRequest.KeyVersion(summary.getKey()));
- String newDstKey = dstKey + summary.getKey().substring(srcKey.length());
+ keysToDelete.add(
+ new DeleteObjectsRequest.KeyVersion(summary.getKey()));
+ String newDstKey =
+ dstKey + summary.getKey().substring(srcKey.length());
copyFile(summary.getKey(), newDstKey, summary.getSize());
if (keysToDelete.size() == MAX_ENTRIES_TO_DELETE) {
@@ -1484,7 +1363,8 @@ public class S3AFileSystem extends FileSystem {
LOG.debug("Found file (with /): fake directory");
return new S3AFileStatus(true, true, path);
} else {
- LOG.warn("Found file (with /): real file? should not happen: {}", key);
+ LOG.warn("Found file (with /): real file? should not happen: {}",
+ key);
return new S3AFileStatus(meta.getContentLength(),
dateToLong(meta.getLastModified()),
@@ -2078,42 +1958,4 @@ public class S3AFileSystem extends FileSystem {
getFileBlockLocations(status, 0, status.getLen())
: null);
}
-
- /**
- * Get a integer option >= the minimum allowed value.
- * @param conf configuration
- * @param key key to look up
- * @param defVal default value
- * @param min minimum value
- * @return the value
- * @throws IllegalArgumentException if the value is below the minimum
- */
- static int intOption(Configuration conf, String key, int defVal, int min) {
- int v = conf.getInt(key, defVal);
- Preconditions.checkArgument(v >= min,
- String.format("Value of %s: %d is below the minimum value %d",
- key, v, min));
- return v;
- }
-
- /**
- * Get a long option >= the minimum allowed value.
- * @param conf configuration
- * @param key key to look up
- * @param defVal default value
- * @param min minimum value
- * @return the value
- * @throws IllegalArgumentException if the value is below the minimum
- */
- static long longOption(Configuration conf,
- String key,
- long defVal,
- long min) {
- long v = conf.getLong(key, defVal);
- Preconditions.checkArgument(v >= min,
- String.format("Value of %s: %d is below the minimum value %d",
- key, v, min));
- return v;
- }
-
}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/67d8301e/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
index ccb9726..dd6cdd7 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
@@ -19,7 +19,7 @@
package org.apache.hadoop.fs.s3a;
import com.amazonaws.AmazonClientException;
-import com.amazonaws.services.s3.AmazonS3Client;
+import com.amazonaws.services.s3.AmazonS3;
import com.amazonaws.services.s3.model.GetObjectRequest;
import com.amazonaws.services.s3.model.S3ObjectInputStream;
import com.google.common.base.Preconditions;
@@ -71,7 +71,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
private volatile boolean closed;
private S3ObjectInputStream wrappedStream;
private final FileSystem.Statistics stats;
- private final AmazonS3Client client;
+ private final AmazonS3 client;
private final String bucket;
private final String key;
private final long contentLength;
@@ -101,7 +101,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
public S3AInputStream(String bucket,
String key,
long contentLength,
- AmazonS3Client client,
+ AmazonS3 client,
FileSystem.Statistics stats,
S3AInstrumentation instrumentation,
long readahead,
http://git-wip-us.apache.org/repos/asf/hadoop/blob/67d8301e/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
index 40d0b1b..93d819b 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
@@ -25,6 +25,9 @@ import com.amazonaws.auth.EnvironmentVariableCredentialsProvider;
import com.amazonaws.auth.InstanceProfileCredentialsProvider;
import com.amazonaws.services.s3.model.AmazonS3Exception;
import com.amazonaws.services.s3.model.S3ObjectSummary;
+
+import com.google.common.base.Preconditions;
+
import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@@ -420,4 +423,41 @@ public final class S3AUtils {
builder.append("size=").append(summary.getSize());
return builder.toString();
}
+
+ /**
+ * Get a integer option >= the minimum allowed value.
+ * @param conf configuration
+ * @param key key to look up
+ * @param defVal default value
+ * @param min minimum value
+ * @return the value
+ * @throws IllegalArgumentException if the value is below the minimum
+ */
+ static int intOption(Configuration conf, String key, int defVal, int min) {
+ int v = conf.getInt(key, defVal);
+ Preconditions.checkArgument(v >= min,
+ String.format("Value of %s: %d is below the minimum value %d",
+ key, v, min));
+ return v;
+ }
+
+ /**
+ * Get a long option >= the minimum allowed value.
+ * @param conf configuration
+ * @param key key to look up
+ * @param defVal default value
+ * @param min minimum value
+ * @return the value
+ * @throws IllegalArgumentException if the value is below the minimum
+ */
+ static long longOption(Configuration conf,
+ String key,
+ long defVal,
+ long min) {
+ long v = conf.getLong(key, defVal);
+ Preconditions.checkArgument(v >= min,
+ String.format("Value of %s: %d is below the minimum value %d",
+ key, v, min));
+ return v;
+ }
}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/67d8301e/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java
new file mode 100644
index 0000000..0a4dd02
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java
@@ -0,0 +1,232 @@
+/**
+ * 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 static org.apache.hadoop.fs.s3a.Constants.*;
+import static org.apache.hadoop.fs.s3a.S3AUtils.*;
+
+import java.io.IOException;
+import java.net.URI;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.Protocol;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.s3.AmazonS3;
+import com.amazonaws.services.s3.AmazonS3Client;
+import com.amazonaws.services.s3.S3ClientOptions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.util.VersionInfo;
+
+import org.slf4j.Logger;
+
+/**
+ * Factory for creation of S3 client instances to be used by {@link S3Store}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+interface S3ClientFactory {
+
+ /**
+ * Creates a new {@link AmazonS3} client. This method accepts the S3A file
+ * system URI both in raw input form and validated form as separate arguments,
+ * because both values may be useful in logging.
+ *
+ * @param name raw input S3A file system URI
+ * @param uri validated form of S3A file system URI
+ * @return S3 client
+ * @throws IOException IO problem
+ */
+ AmazonS3 createS3Client(URI name, URI uri) throws IOException;
+
+ /**
+ * The default factory implementation, which calls the AWS SDK to configure
+ * and create an {@link AmazonS3Client} that communicates with the S3 service.
+ */
+ static class DefaultS3ClientFactory extends Configured
+ implements S3ClientFactory {
+
+ private static final Logger LOG = S3AFileSystem.LOG;
+
+ @Override
+ public AmazonS3 createS3Client(URI name, URI uri) throws IOException {
+ Configuration conf = getConf();
+ AWSCredentialsProvider credentials =
+ createAWSCredentialProviderSet(name, conf, uri);
+ ClientConfiguration awsConf = new ClientConfiguration();
+ initConnectionSettings(conf, awsConf);
+ initProxySupport(conf, awsConf);
+ initUserAgent(conf, awsConf);
+ return createAmazonS3Client(conf, credentials, awsConf);
+ }
+
+ /**
+ * Initializes all AWS SDK settings related to connection management.
+ *
+ * @param conf Hadoop configuration
+ * @param awsConf AWS SDK configuration
+ */
+ private static void initConnectionSettings(Configuration conf,
+ ClientConfiguration awsConf) {
+ awsConf.setMaxConnections(intOption(conf, MAXIMUM_CONNECTIONS,
+ DEFAULT_MAXIMUM_CONNECTIONS, 1));
+ boolean secureConnections = conf.getBoolean(SECURE_CONNECTIONS,
+ DEFAULT_SECURE_CONNECTIONS);
+ awsConf.setProtocol(secureConnections ? Protocol.HTTPS : Protocol.HTTP);
+ awsConf.setMaxErrorRetry(intOption(conf, MAX_ERROR_RETRIES,
+ DEFAULT_MAX_ERROR_RETRIES, 0));
+ awsConf.setConnectionTimeout(intOption(conf, ESTABLISH_TIMEOUT,
+ DEFAULT_ESTABLISH_TIMEOUT, 0));
+ awsConf.setSocketTimeout(intOption(conf, SOCKET_TIMEOUT,
+ DEFAULT_SOCKET_TIMEOUT, 0));
+ int sockSendBuffer = intOption(conf, SOCKET_SEND_BUFFER,
+ DEFAULT_SOCKET_SEND_BUFFER, 2048);
+ int sockRecvBuffer = intOption(conf, SOCKET_RECV_BUFFER,
+ DEFAULT_SOCKET_RECV_BUFFER, 2048);
+ awsConf.setSocketBufferSizeHints(sockSendBuffer, sockRecvBuffer);
+ String signerOverride = conf.getTrimmed(SIGNING_ALGORITHM, "");
+ if (!signerOverride.isEmpty()) {
+ LOG.debug("Signer override = {}", signerOverride);
+ awsConf.setSignerOverride(signerOverride);
+ }
+ }
+
+ /**
+ * Initializes AWS SDK proxy support if configured.
+ *
+ * @param conf Hadoop configuration
+ * @param awsConf AWS SDK configuration
+ * @throws IllegalArgumentException if misconfigured
+ */
+ private static void initProxySupport(Configuration conf,
+ ClientConfiguration awsConf) throws IllegalArgumentException {
+ String proxyHost = conf.getTrimmed(PROXY_HOST, "");
+ int proxyPort = conf.getInt(PROXY_PORT, -1);
+ if (!proxyHost.isEmpty()) {
+ awsConf.setProxyHost(proxyHost);
+ if (proxyPort >= 0) {
+ awsConf.setProxyPort(proxyPort);
+ } else {
+ if (conf.getBoolean(SECURE_CONNECTIONS, DEFAULT_SECURE_CONNECTIONS)) {
+ LOG.warn("Proxy host set without port. Using HTTPS default 443");
+ awsConf.setProxyPort(443);
+ } else {
+ LOG.warn("Proxy host set without port. Using HTTP default 80");
+ awsConf.setProxyPort(80);
+ }
+ }
+ String proxyUsername = conf.getTrimmed(PROXY_USERNAME);
+ String proxyPassword = conf.getTrimmed(PROXY_PASSWORD);
+ if ((proxyUsername == null) != (proxyPassword == null)) {
+ String msg = "Proxy error: " + PROXY_USERNAME + " or " +
+ PROXY_PASSWORD + " set without the other.";
+ LOG.error(msg);
+ throw new IllegalArgumentException(msg);
+ }
+ awsConf.setProxyUsername(proxyUsername);
+ awsConf.setProxyPassword(proxyPassword);
+ awsConf.setProxyDomain(conf.getTrimmed(PROXY_DOMAIN));
+ awsConf.setProxyWorkstation(conf.getTrimmed(PROXY_WORKSTATION));
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Using proxy server {}:{} as user {} with password {} on " +
+ "domain {} as workstation {}", awsConf.getProxyHost(),
+ awsConf.getProxyPort(),
+ String.valueOf(awsConf.getProxyUsername()),
+ awsConf.getProxyPassword(), awsConf.getProxyDomain(),
+ awsConf.getProxyWorkstation());
+ }
+ } else if (proxyPort >= 0) {
+ String msg =
+ "Proxy error: " + PROXY_PORT + " set without " + PROXY_HOST;
+ LOG.error(msg);
+ throw new IllegalArgumentException(msg);
+ }
+ }
+
+ /**
+ * Initializes the User-Agent header to send in HTTP requests to the S3
+ * back-end. We always include the Hadoop version number. The user also
+ * may set an optional custom prefix to put in front of the Hadoop version
+ * number. The AWS SDK interally appends its own information, which seems
+ * to include the AWS SDK version, OS and JVM version.
+ *
+ * @param conf Hadoop configuration
+ * @param awsConf AWS SDK configuration
+ */
+ private static void initUserAgent(Configuration conf,
+ ClientConfiguration awsConf) {
+ String userAgent = "Hadoop " + VersionInfo.getVersion();
+ String userAgentPrefix = conf.getTrimmed(USER_AGENT_PREFIX, "");
+ if (!userAgentPrefix.isEmpty()) {
+ userAgent = userAgentPrefix + ", " + userAgent;
+ }
+ LOG.debug("Using User-Agent: {}", userAgent);
+ awsConf.setUserAgent(userAgent);
+ }
+
+ /**
+ * Creates an {@link AmazonS3Client} from the established configuration.
+ *
+ * @param conf Hadoop configuration
+ * @param credentials AWS credentials
+ * @param awsConf AWS SDK configuration
+ * @return S3 client
+ * @throws IllegalArgumentException if misconfigured
+ */
+ private static AmazonS3 createAmazonS3Client(Configuration conf,
+ AWSCredentialsProvider credentials, ClientConfiguration awsConf)
+ throws IllegalArgumentException {
+ AmazonS3 s3 = new AmazonS3Client(credentials, awsConf);
+ String endPoint = conf.getTrimmed(ENDPOINT, "");
+ if (!endPoint.isEmpty()) {
+ try {
+ s3.setEndpoint(endPoint);
+ } catch (IllegalArgumentException e) {
+ String msg = "Incorrect endpoint: " + e.getMessage();
+ LOG.error(msg);
+ throw new IllegalArgumentException(msg, e);
+ }
+ }
+ enablePathStyleAccessIfRequired(s3, conf);
+ return s3;
+ }
+
+ /**
+ * Enables path-style access to S3 buckets if configured. By default, the
+ * behavior is to use virtual hosted-style access with URIs of the form
+ * http://bucketname.s3.amazonaws.com. Enabling path-style access and a
+ * region-specific endpoint switches the behavior to use URIs of the form
+ * http://s3-eu-west-1.amazonaws.com/bucketname.
+ *
+ * @param s3 S3 client
+ * @param conf Hadoop configuration
+ */
+ private static void enablePathStyleAccessIfRequired(AmazonS3 s3,
+ Configuration conf) {
+ final boolean pathStyleAccess = conf.getBoolean(PATH_STYLE_ACCESS, false);
+ if (pathStyleAccess) {
+ LOG.debug("Enabling path style access!");
+ s3.setS3ClientOptions(new S3ClientOptions().withPathStyleAccess(true));
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/67d8301e/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java
index bc8c2e6..97ece37 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java
@@ -132,6 +132,9 @@ public final class S3xLoginHelper {
*
* This strips out login information.
*
+ * @param uri the URI to canonicalize
+ * @param defaultPort default port to use in canonicalized URI if the input
+ * URI has no port and this value is greater than 0
* @return a new, canonicalized URI.
*/
public static URI canonicalizeUri(URI uri, int defaultPort) {
http://git-wip-us.apache.org/repos/asf/hadoop/blob/67d8301e/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java
new file mode 100644
index 0000000..6734947
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java
@@ -0,0 +1,70 @@
+/**
+ * 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 static org.apache.hadoop.fs.s3a.Constants.*;
+
+import com.amazonaws.AmazonServiceException;
+import com.amazonaws.services.s3.AmazonS3;
+
+import java.net.URI;
+
+import org.apache.hadoop.conf.Configuration;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.rules.ExpectedException;
+
+/**
+ * Abstract base class for S3A unit tests using a mock S3 client.
+ */
+public abstract class AbstractS3AMockTest {
+
+ protected static final String BUCKET = "mock-bucket";
+ protected static final AmazonServiceException NOT_FOUND;
+ static {
+ NOT_FOUND = new AmazonServiceException("Not Found");
+ NOT_FOUND.setStatusCode(404);
+ }
+
+ @Rule
+ public ExpectedException exception = ExpectedException.none();
+
+ protected S3AFileSystem fs;
+ protected AmazonS3 s3;
+
+ @Before
+ public void setup() throws Exception {
+ Configuration conf = new Configuration();
+ conf.setClass(S3_CLIENT_FACTORY_IMPL, MockS3ClientFactory.class,
+ S3ClientFactory.class);
+ fs = new S3AFileSystem();
+ URI uri = URI.create(FS_S3A + "://" + BUCKET);
+ fs.initialize(uri, conf);
+ s3 = fs.getAmazonS3Client();
+ }
+
+ @After
+ public void teardown() throws Exception {
+ if (fs != null) {
+ fs.close();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/67d8301e/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
index 9fadc1f..4404c60 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
@@ -19,7 +19,7 @@
package org.apache.hadoop.fs.s3a;
import com.amazonaws.ClientConfiguration;
-import com.amazonaws.services.s3.AmazonS3Client;
+import com.amazonaws.services.s3.AmazonS3;
import com.amazonaws.services.s3.S3ClientOptions;
import org.apache.commons.lang.StringUtils;
@@ -96,7 +96,7 @@ public class ITestS3AConfiguration {
} else {
conf.set(Constants.ENDPOINT, endpoint);
fs = S3ATestUtils.createTestFileSystem(conf);
- AmazonS3Client s3 = fs.getAmazonS3Client();
+ AmazonS3 s3 = fs.getAmazonS3Client();
String endPointRegion = "";
// Differentiate handling of "s3-" and "s3." based endpoint identifiers
String[] endpointParts = StringUtils.split(endpoint, '.');
@@ -362,7 +362,7 @@ public class ITestS3AConfiguration {
try {
fs = S3ATestUtils.createTestFileSystem(conf);
assertNotNull(fs);
- AmazonS3Client s3 = fs.getAmazonS3Client();
+ AmazonS3 s3 = fs.getAmazonS3Client();
assertNotNull(s3);
S3ClientOptions clientOptions = getField(s3, S3ClientOptions.class,
"clientOptions");
@@ -386,7 +386,7 @@ public class ITestS3AConfiguration {
conf = new Configuration();
fs = S3ATestUtils.createTestFileSystem(conf);
assertNotNull(fs);
- AmazonS3Client s3 = fs.getAmazonS3Client();
+ AmazonS3 s3 = fs.getAmazonS3Client();
assertNotNull(s3);
ClientConfiguration awsConf = getField(s3, ClientConfiguration.class,
"clientConfiguration");
@@ -399,7 +399,7 @@ public class ITestS3AConfiguration {
conf.set(Constants.USER_AGENT_PREFIX, "MyApp");
fs = S3ATestUtils.createTestFileSystem(conf);
assertNotNull(fs);
- AmazonS3Client s3 = fs.getAmazonS3Client();
+ AmazonS3 s3 = fs.getAmazonS3Client();
assertNotNull(s3);
ClientConfiguration awsConf = getField(s3, ClientConfiguration.class,
"clientConfiguration");
http://git-wip-us.apache.org/repos/asf/hadoop/blob/67d8301e/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java
new file mode 100644
index 0000000..41f04ee
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java
@@ -0,0 +1,40 @@
+/**
+ * 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 static org.mockito.Mockito.*;
+
+import java.net.URI;
+
+import com.amazonaws.services.s3.AmazonS3;
+
+/**
+ * An {@link S3ClientFactory} that returns Mockito mocks of the {@link AmazonS3}
+ * interface suitable for unit testing.
+ */
+public class MockS3ClientFactory implements S3ClientFactory {
+
+ @Override
+ public AmazonS3 createS3Client(URI name, URI uri) {
+ String bucket = name.getHost();
+ AmazonS3 s3 = mock(AmazonS3.class);
+ when(s3.doesBucketExist(bucket)).thenReturn(true);
+ return s3;
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/67d8301e/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AGetFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AGetFileStatus.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AGetFileStatus.java
new file mode 100644
index 0000000..f9e9c6b
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AGetFileStatus.java
@@ -0,0 +1,126 @@
+/**
+ * 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 static org.junit.Assert.*;
+import static org.mockito.Matchers.*;
+import static org.mockito.Mockito.*;
+
+import java.io.FileNotFoundException;
+import java.util.Collections;
+import java.util.Date;
+
+import com.amazonaws.services.s3.model.ListObjectsRequest;
+import com.amazonaws.services.s3.model.ObjectListing;
+import com.amazonaws.services.s3.model.ObjectMetadata;
+import com.amazonaws.services.s3.model.S3ObjectSummary;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+
+import org.junit.Test;
+
+/**
+ * S3A tests for getFileStatus using mock S3 client.
+ */
+public class TestS3AGetFileStatus extends AbstractS3AMockTest {
+
+ @Test
+ public void testFile() throws Exception {
+ Path path = new Path("/file");
+ String key = path.toUri().getPath().substring(1);
+ ObjectMetadata meta = new ObjectMetadata();
+ meta.setContentLength(1L);
+ meta.setLastModified(new Date(2L));
+ when(s3.getObjectMetadata(BUCKET, key)).thenReturn(meta);
+ FileStatus stat = fs.getFileStatus(path);
+ assertNotNull(stat);
+ assertEquals(fs.makeQualified(path), stat.getPath());
+ assertTrue(stat.isFile());
+ assertEquals(meta.getContentLength(), stat.getLen());
+ assertEquals(meta.getLastModified().getTime(), stat.getModificationTime());
+ }
+
+ @Test
+ public void testFakeDirectory() throws Exception {
+ Path path = new Path("/dir");
+ String key = path.toUri().getPath().substring(1);
+ when(s3.getObjectMetadata(BUCKET, key)).thenThrow(NOT_FOUND);
+ ObjectMetadata meta = new ObjectMetadata();
+ meta.setContentLength(0L);
+ when(s3.getObjectMetadata(BUCKET, key + "/")).thenReturn(meta);
+ FileStatus stat = fs.getFileStatus(path);
+ assertNotNull(stat);
+ assertEquals(fs.makeQualified(path), stat.getPath());
+ assertTrue(stat.isDirectory());
+ }
+
+ @Test
+ public void testImplicitDirectory() throws Exception {
+ Path path = new Path("/dir");
+ String key = path.toUri().getPath().substring(1);
+ when(s3.getObjectMetadata(BUCKET, key)).thenThrow(NOT_FOUND);
+ when(s3.getObjectMetadata(BUCKET, key + "/")).thenThrow(NOT_FOUND);
+ ObjectListing objects = mock(ObjectListing.class);
+ when(objects.getCommonPrefixes()).thenReturn(
+ Collections.singletonList("dir/"));
+ when(objects.getObjectSummaries()).thenReturn(
+ Collections.<S3ObjectSummary>emptyList());
+ when(s3.listObjects(any(ListObjectsRequest.class))).thenReturn(objects);
+ FileStatus stat = fs.getFileStatus(path);
+ assertNotNull(stat);
+ assertEquals(fs.makeQualified(path), stat.getPath());
+ assertTrue(stat.isDirectory());
+ }
+
+ @Test
+ public void testRoot() throws Exception {
+ Path path = new Path("/");
+ String key = path.toUri().getPath().substring(1);
+ when(s3.getObjectMetadata(BUCKET, key)).thenThrow(NOT_FOUND);
+ when(s3.getObjectMetadata(BUCKET, key + "/")).thenThrow(NOT_FOUND);
+ ObjectListing objects = mock(ObjectListing.class);
+ when(objects.getCommonPrefixes()).thenReturn(
+ Collections.<String>emptyList());
+ when(objects.getObjectSummaries()).thenReturn(
+ Collections.<S3ObjectSummary>emptyList());
+ when(s3.listObjects(any(ListObjectsRequest.class))).thenReturn(objects);
+ FileStatus stat = fs.getFileStatus(path);
+ assertNotNull(stat);
+ assertEquals(fs.makeQualified(path), stat.getPath());
+ assertTrue(stat.isDirectory());
+ assertTrue(stat.getPath().isRoot());
+ }
+
+ @Test
+ public void testNotFound() throws Exception {
+ Path path = new Path("/dir");
+ String key = path.toUri().getPath().substring(1);
+ when(s3.getObjectMetadata(BUCKET, key)).thenThrow(NOT_FOUND);
+ when(s3.getObjectMetadata(BUCKET, key + "/")).thenThrow(NOT_FOUND);
+ ObjectListing objects = mock(ObjectListing.class);
+ when(objects.getCommonPrefixes()).thenReturn(
+ Collections.<String>emptyList());
+ when(objects.getObjectSummaries()).thenReturn(
+ Collections.<S3ObjectSummary>emptyList());
+ when(s3.listObjects(any(ListObjectsRequest.class))).thenReturn(objects);
+ exception.expect(FileNotFoundException.class);
+ fs.getFileStatus(path);
+ }
+}
---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org
[6/7] hadoop git commit: HADOOP-13446. Support running isolated unit
tests separate from AWS integration tests. Contributed by Chris Nauroth.
Posted by cn...@apache.org.
HADOOP-13446. Support running isolated unit tests separate from AWS integration tests. Contributed by Chris Nauroth.
(cherry picked from commit cda68de9b913878210118a6637a1d0eb950c26ad)
Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/027d76fa
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/027d76fa
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/027d76fa
Branch: refs/heads/branch-2.8
Commit: 027d76fa8c92c7dda0612a06d9f29172f2e63dc3
Parents: d29dc39
Author: Chris Nauroth <cn...@apache.org>
Authored: Tue Aug 23 07:19:20 2016 -0700
Committer: Chris Nauroth <cn...@apache.org>
Committed: Mon Oct 10 14:01:13 2016 -0700
----------------------------------------------------------------------
hadoop-project/pom.xml | 5 +
hadoop-tools/hadoop-aws/pom.xml | 95 +++-
.../src/site/markdown/tools/hadoop-aws/index.md | 67 ++-
.../fs/contract/s3/ITestS3ContractCreate.java | 35 ++
.../fs/contract/s3/ITestS3ContractDelete.java | 35 ++
.../fs/contract/s3/ITestS3ContractMkdir.java | 35 ++
.../fs/contract/s3/ITestS3ContractOpen.java | 35 ++
.../fs/contract/s3/ITestS3ContractRename.java | 36 ++
.../fs/contract/s3/ITestS3ContractRootDir.java | 57 ++
.../fs/contract/s3/ITestS3ContractSeek.java | 44 ++
.../fs/contract/s3/TestS3ContractCreate.java | 33 --
.../fs/contract/s3/TestS3ContractDelete.java | 32 --
.../fs/contract/s3/TestS3ContractMkdir.java | 33 --
.../fs/contract/s3/TestS3ContractOpen.java | 33 --
.../fs/contract/s3/TestS3ContractRename.java | 33 --
.../fs/contract/s3/TestS3ContractRootDir.java | 57 --
.../fs/contract/s3/TestS3ContractSeek.java | 42 --
.../fs/contract/s3a/ITestS3AContractCreate.java | 35 ++
.../fs/contract/s3a/ITestS3AContractDelete.java | 34 ++
.../fs/contract/s3a/ITestS3AContractDistCp.java | 46 ++
.../s3a/ITestS3AContractGetFileStatus.java | 51 ++
.../fs/contract/s3a/ITestS3AContractMkdir.java | 34 ++
.../fs/contract/s3a/ITestS3AContractOpen.java | 34 ++
.../fs/contract/s3a/ITestS3AContractRename.java | 62 +++
.../contract/s3a/ITestS3AContractRootDir.java | 72 +++
.../fs/contract/s3a/ITestS3AContractSeek.java | 34 ++
.../fs/contract/s3a/TestS3AContractCreate.java | 33 --
.../fs/contract/s3a/TestS3AContractDelete.java | 31 --
.../fs/contract/s3a/TestS3AContractDistCp.java | 46 --
.../s3a/TestS3AContractGetFileStatus.java | 47 --
.../fs/contract/s3a/TestS3AContractMkdir.java | 34 --
.../fs/contract/s3a/TestS3AContractOpen.java | 31 --
.../fs/contract/s3a/TestS3AContractRename.java | 61 ---
.../fs/contract/s3a/TestS3AContractRootDir.java | 72 ---
.../fs/contract/s3a/TestS3AContractSeek.java | 31 --
.../fs/contract/s3n/ITestS3NContractCreate.java | 41 ++
.../fs/contract/s3n/ITestS3NContractDelete.java | 34 ++
.../fs/contract/s3n/ITestS3NContractMkdir.java | 34 ++
.../fs/contract/s3n/ITestS3NContractOpen.java | 34 ++
.../fs/contract/s3n/ITestS3NContractRename.java | 35 ++
.../contract/s3n/ITestS3NContractRootDir.java | 35 ++
.../fs/contract/s3n/ITestS3NContractSeek.java | 34 ++
.../fs/contract/s3n/TestS3NContractCreate.java | 38 --
.../fs/contract/s3n/TestS3NContractDelete.java | 31 --
.../fs/contract/s3n/TestS3NContractMkdir.java | 34 --
.../fs/contract/s3n/TestS3NContractOpen.java | 31 --
.../fs/contract/s3n/TestS3NContractRename.java | 32 --
.../fs/contract/s3n/TestS3NContractRootDir.java | 35 --
.../fs/contract/s3n/TestS3NContractSeek.java | 31 --
.../s3/ITestInMemoryS3FileSystemContract.java | 35 ++
.../fs/s3/TestInMemoryS3FileSystemContract.java | 32 --
.../ITestBlockingThreadPoolExecutorService.java | 182 +++++++
.../fs/s3a/ITestS3AAWSCredentialsProvider.java | 250 +++++++++
.../fs/s3a/ITestS3ABlockingThreadPool.java | 80 +++
.../apache/hadoop/fs/s3a/ITestS3ABlocksize.java | 96 ++++
.../hadoop/fs/s3a/ITestS3AConfiguration.java | 441 +++++++++++++++
.../hadoop/fs/s3a/ITestS3ACredentialsInURL.java | 155 ++++++
.../hadoop/fs/s3a/ITestS3AEncryption.java | 104 ++++
.../ITestS3AEncryptionAlgorithmPropagation.java | 83 +++
.../s3a/ITestS3AEncryptionFastOutputStream.java | 35 ++
.../hadoop/fs/s3a/ITestS3AFailureHandling.java | 137 +++++
.../hadoop/fs/s3a/ITestS3AFastOutputStream.java | 74 +++
.../fs/s3a/ITestS3AFileOperationCost.java | 277 ++++++++++
.../fs/s3a/ITestS3AFileSystemContract.java | 106 ++++
.../fs/s3a/ITestS3ATemporaryCredentials.java | 148 +++++
.../TestBlockingThreadPoolExecutorService.java | 182 -------
.../fs/s3a/TestS3AAWSCredentialsProvider.java | 250 ---------
.../fs/s3a/TestS3ABlockingThreadPool.java | 80 ---
.../apache/hadoop/fs/s3a/TestS3ABlocksize.java | 93 ----
.../hadoop/fs/s3a/TestS3AConfiguration.java | 435 ---------------
.../hadoop/fs/s3a/TestS3ACredentialsInURL.java | 155 ------
.../apache/hadoop/fs/s3a/TestS3AEncryption.java | 104 ----
.../TestS3AEncryptionAlgorithmPropagation.java | 82 ---
.../s3a/TestS3AEncryptionFastOutputStream.java | 35 --
.../hadoop/fs/s3a/TestS3AFailureHandling.java | 139 -----
.../hadoop/fs/s3a/TestS3AFastOutputStream.java | 74 ---
.../hadoop/fs/s3a/TestS3AFileOperationCost.java | 277 ----------
.../fs/s3a/TestS3AFileSystemContract.java | 104 ----
.../fs/s3a/TestS3ATemporaryCredentials.java | 148 -----
.../fs/s3a/fileContext/ITestS3AFileContext.java | 23 +
.../ITestS3AFileContextCreateMkdir.java | 35 ++
.../ITestS3AFileContextMainOperations.java | 60 +++
.../ITestS3AFileContextStatistics.java | 61 +++
.../s3a/fileContext/ITestS3AFileContextURI.java | 44 ++
.../fileContext/ITestS3AFileContextUtil.java | 34 ++
.../fs/s3a/fileContext/TestS3AFileContext.java | 23 -
.../TestS3AFileContextCreateMkdir.java | 35 --
.../TestS3AFileContextMainOperations.java | 60 ---
.../TestS3AFileContextStatistics.java | 60 ---
.../s3a/fileContext/TestS3AFileContextURI.java | 43 --
.../s3a/fileContext/TestS3AFileContextUtil.java | 34 --
.../s3a/scale/ITestS3ADeleteFilesOneByOne.java | 44 ++
.../fs/s3a/scale/ITestS3ADeleteManyFiles.java | 126 +++++
.../s3a/scale/ITestS3ADirectoryPerformance.java | 187 +++++++
.../scale/ITestS3AInputStreamPerformance.java | 534 +++++++++++++++++++
.../s3a/scale/TestS3ADeleteFilesOneByOne.java | 40 --
.../fs/s3a/scale/TestS3ADeleteManyFiles.java | 126 -----
.../s3a/scale/TestS3ADirectoryPerformance.java | 187 -------
.../scale/TestS3AInputStreamPerformance.java | 534 -------------------
.../org/apache/hadoop/fs/s3a/yarn/ITestS3A.java | 88 +++
.../fs/s3a/yarn/ITestS3AMiniYarnCluster.java | 148 +++++
.../org/apache/hadoop/fs/s3a/yarn/TestS3A.java | 85 ---
.../fs/s3a/yarn/TestS3AMiniYarnCluster.java | 144 -----
...ITestInMemoryNativeS3FileSystemContract.java | 33 ++
.../ITestJets3tNativeFileSystemStore.java | 133 +++++
.../ITestJets3tNativeS3FileSystemContract.java | 33 ++
.../TestInMemoryNativeS3FileSystemContract.java | 30 --
.../TestJets3tNativeFileSystemStore.java | 129 -----
.../TestJets3tNativeS3FileSystemContract.java | 30 --
109 files changed, 4843 insertions(+), 4667 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index dfb2224..a8d0024 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -1050,6 +1050,11 @@
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-failsafe-plugin</artifactId>
+ <version>${maven-failsafe-plugin.version}</version>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-install-plugin</artifactId>
<version>${maven-install-plugin.version}</version>
</plugin>
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml
index ebf32be..c91081d 100644
--- a/hadoop-tools/hadoop-aws/pom.xml
+++ b/hadoop-tools/hadoop-aws/pom.xml
@@ -46,7 +46,7 @@
</file>
</activation>
<properties>
- <maven.test.skip>true</maven.test.skip>
+ <skipITs>true</skipITs>
</properties>
</profile>
<profile>
@@ -57,11 +57,16 @@
</file>
</activation>
<properties>
- <maven.test.skip>false</maven.test.skip>
+ <skipITs>false</skipITs>
</properties>
</profile>
<profile>
<id>parallel-tests</id>
+ <activation>
+ <property>
+ <name>parallel-tests</name>
+ </property>
+ </activation>
<build>
<plugins>
<plugin>
@@ -96,12 +101,32 @@
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
+ <configuration>
+ <forkCount>${testsThreadCount}</forkCount>
+ <reuseForks>false</reuseForks>
+ <argLine>${maven-surefire-plugin.argLine} -DminiClusterDedicatedDirs=true</argLine>
+ <systemPropertyVariables>
+ <test.build.data>${test.build.data}/${surefire.forkNumber}</test.build.data>
+ <test.build.dir>${test.build.dir}/${surefire.forkNumber}</test.build.dir>
+ <hadoop.tmp.dir>${hadoop.tmp.dir}/${surefire.forkNumber}</hadoop.tmp.dir>
+
+ <!-- Due to a Maven quirk, setting this to just -->
+ <!-- surefire.forkNumber won't do the parameter -->
+ <!-- substitution. Putting a prefix in front of it like -->
+ <!-- "fork-" makes it work. -->
+ <test.unique.fork.id>fork-${surefire.forkNumber}</test.unique.fork.id>
+ </systemPropertyVariables>
+ </configuration>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-failsafe-plugin</artifactId>
<executions>
<execution>
- <id>default-test</id>
- <phase>test</phase>
+ <id>default-integration-test</id>
<goals>
- <goal>test</goal>
+ <goal>integration-test</goal>
+ <goal>verify</goal>
</goals>
<configuration>
<forkCount>${testsThreadCount}</forkCount>
@@ -129,32 +154,35 @@
<!-- Exclude all of these tests from parallel execution, -->
<!-- and instead run them sequentially in a separate -->
<!-- Surefire execution step later. -->
+ <includes>
+ <include>**/ITest*.java</include>
+ </includes>
<excludes>
- <exclude>**/TestJets3tNativeS3FileSystemContract.java</exclude>
- <exclude>**/TestS3ABlockingThreadPool.java</exclude>
- <exclude>**/TestS3AFastOutputStream.java</exclude>
- <exclude>**/TestS3AFileSystemContract.java</exclude>
- <exclude>**/TestS3AMiniYarnCluster.java</exclude>
- <exclude>**/Test*Root*.java</exclude>
+ <exclude>**/ITestJets3tNativeS3FileSystemContract.java</exclude>
+ <exclude>**/ITestS3ABlockingThreadPool.java</exclude>
+ <exclude>**/ITestS3AFastOutputStream.java</exclude>
+ <exclude>**/ITestS3AFileSystemContract.java</exclude>
+ <exclude>**/ITestS3AMiniYarnCluster.java</exclude>
+ <exclude>**/ITest*Root*.java</exclude>
</excludes>
</configuration>
</execution>
<execution>
- <id>sequential-tests</id>
- <phase>test</phase>
+ <id>sequential-integration-tests</id>
<goals>
- <goal>test</goal>
+ <goal>integration-test</goal>
+ <goal>verify</goal>
</goals>
<configuration>
<!-- Do a sequential run for tests that cannot handle -->
<!-- parallel execution. -->
<includes>
- <include>**/TestJets3tNativeS3FileSystemContract.java</include>
- <include>**/TestS3ABlockingThreadPool.java</include>
- <include>**/TestS3AFastOutputStream.java</include>
- <include>**/TestS3AFileSystemContract.java</include>
- <include>**/TestS3AMiniYarnCluster.java</include>
- <include>**/Test*Root*.java</include>
+ <include>**/ITestJets3tNativeS3FileSystemContract.java</include>
+ <include>**/ITestS3ABlockingThreadPool.java</include>
+ <include>**/ITestS3AFastOutputStream.java</include>
+ <include>**/ITestS3AFileSystemContract.java</include>
+ <include>**/ITestS3AMiniYarnCluster.java</include>
+ <include>**/ITest*Root*.java</include>
</includes>
</configuration>
</execution>
@@ -163,6 +191,33 @@
</plugins>
</build>
</profile>
+ <profile>
+ <id>sequential-tests</id>
+ <activation>
+ <property>
+ <name>!parallel-tests</name>
+ </property>
+ </activation>
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-failsafe-plugin</artifactId>
+ <executions>
+ <execution>
+ <goals>
+ <goal>integration-test</goal>
+ <goal>verify</goal>
+ </goals>
+ <configuration>
+ <forkedProcessTimeoutInSeconds>3600</forkedProcessTimeoutInSeconds>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+ </profile>
</profiles>
<build>
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
index 5270dfa..67972ca 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
@@ -1375,11 +1375,18 @@ works with S3 to something lower. See [AWS documentation](http://docs.aws.amazon
## Testing the S3 filesystem clients
-Due to eventual consistency, tests may fail without reason. Transient
-failures, which no longer occur upon rerunning the test, should thus be ignored.
+This module includes both unit tests, which can run in isolation without
+connecting to the S3 service, and integration tests, which require a working
+connection to S3 to interact with a bucket. Unit test suites follow the naming
+convention `Test*.java`. Integration tests follow the naming convention
+`ITest*.java`.
-To test the S3* filesystem clients, you need to provide two files
-which pass in authentication details to the test runner
+Due to eventual consistency, integration tests may fail without reason.
+Transient failures, which no longer occur upon rerunning the test, should thus
+be ignored.
+
+To integration test the S3* filesystem clients, you need to provide two files
+which pass in authentication details to the test runner.
1. `auth-keys.xml`
1. `core-site.xml`
@@ -1399,7 +1406,8 @@ need to apply a specific, non-default property change during the tests.
The presence of this file triggers the testing of the S3 classes.
-Without this file, *none of the tests in this module will be executed*
+Without this file, *none of the integration tests in this module will be
+executed*.
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
@@ -1586,23 +1594,50 @@ source code tree, it is not going to get accidentally committed.
After completing the configuration, execute the test run through Maven.
- mvn clean test
+ mvn clean verify
-It's also possible to execute multiple test suites in parallel by enabling the
-`parallel-tests` Maven profile. The tests spend most of their time blocked on
-network I/O with the S3 service, so running in parallel tends to complete full
-test runs faster.
+It's also possible to execute multiple test suites in parallel by passing the
+`parallel-tests` property on the command line. The tests spend most of their
+time blocked on network I/O with the S3 service, so running in parallel tends to
+complete full test runs faster.
- mvn -Pparallel-tests clean test
+ mvn -Dparallel-tests clean verify
Some tests must run with exclusive access to the S3 bucket, so even with the
-`parallel-tests` profile enabled, several test suites will run in serial in a
-separate Maven execution step after the parallel tests.
+`parallel-tests` property, several test suites will run in serial in a separate
+Maven execution step after the parallel tests.
+
+By default, `parallel-tests` runs 4 test suites concurrently. This can be tuned
+by passing the `testsThreadCount` property.
+
+ mvn -Dparallel-tests -DtestsThreadCount=8 clean verify
+
+To run just unit tests, which do not require S3 connectivity or AWS credentials,
+use any of the above invocations, but switch the goal to `test` instead of
+`verify`.
+
+ mvn clean test
+
+ mvn -Dparallel-tests clean test
+
+ mvn -Dparallel-tests -DtestsThreadCount=8 clean test
+
+To run only a specific named subset of tests, pass the `test` property for unit
+tests or the `it.test` property for integration tests.
+
+ mvn clean test -Dtest=TestS3AInputPolicies
+
+ mvn clean verify -Dit.test=ITestS3AFileContextStatistics
-By default, the `parallel-tests` profile runs 4 test suites concurrently. This
-can be tuned by passing the `testsThreadCount` argument.
+ mvn clean verify -Dtest=TestS3A* -Dit.test=ITestS3A*
- mvn -Pparallel-tests -DtestsThreadCount=8 clean test
+Note that when running a specific subset of tests, the patterns passed in `test`
+and `it.test` override the configuration of which tests need to run in isolation
+in a separate serial phase (mentioned above). This can cause unpredictable
+results, so the recommendation is to avoid passing `parallel-tests` in
+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
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/ITestS3ContractCreate.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/ITestS3ContractCreate.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/ITestS3ContractCreate.java
new file mode 100644
index 0000000..e003f28
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/ITestS3ContractCreate.java
@@ -0,0 +1,35 @@
+/*
+ * 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.s3;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractCreateTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * S3 contract tests creating files.
+ */
+@Deprecated
+public class ITestS3ContractCreate extends AbstractContractCreateTest {
+
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new S3Contract(conf);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/ITestS3ContractDelete.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/ITestS3ContractDelete.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/ITestS3ContractDelete.java
new file mode 100644
index 0000000..307e521
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/ITestS3ContractDelete.java
@@ -0,0 +1,35 @@
+/*
+ * 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.s3;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractDeleteTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * S3 contract tests covering deletes.
+ */
+@Deprecated
+public class ITestS3ContractDelete extends AbstractContractDeleteTest {
+
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new S3Contract(conf);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/ITestS3ContractMkdir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/ITestS3ContractMkdir.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/ITestS3ContractMkdir.java
new file mode 100644
index 0000000..3368393
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/ITestS3ContractMkdir.java
@@ -0,0 +1,35 @@
+/*
+ * 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.s3;
+
+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.
+ */
+@Deprecated
+public class ITestS3ContractMkdir extends AbstractContractMkdirTest {
+
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new S3Contract(conf);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/ITestS3ContractOpen.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/ITestS3ContractOpen.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/ITestS3ContractOpen.java
new file mode 100644
index 0000000..c8cc998
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/ITestS3ContractOpen.java
@@ -0,0 +1,35 @@
+/*
+ * 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.s3;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractOpenTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * S3 contract tests opening files.
+ */
+@Deprecated
+public class ITestS3ContractOpen extends AbstractContractOpenTest {
+
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new S3Contract(conf);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/ITestS3ContractRename.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/ITestS3ContractRename.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/ITestS3ContractRename.java
new file mode 100644
index 0000000..e4dcb13
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/ITestS3ContractRename.java
@@ -0,0 +1,36 @@
+/*
+ * 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.s3;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * S3 contract tests covering rename.
+ */
+@Deprecated
+public class ITestS3ContractRename extends AbstractContractRenameTest {
+
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new S3Contract(conf);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/ITestS3ContractRootDir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/ITestS3ContractRootDir.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/ITestS3ContractRootDir.java
new file mode 100644
index 0000000..7003640
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/ITestS3ContractRootDir.java
@@ -0,0 +1,57 @@
+/*
+ * 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.s3;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+import org.junit.Ignore;
+import org.junit.Test;
+
+/**
+ * root dir operations against an S3 bucket.
+ */
+@Deprecated
+public class ITestS3ContractRootDir extends AbstractContractRootDirectoryTest {
+
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new S3Contract(conf);
+ }
+
+ @Override
+ @Test
+ @Ignore
+ public void testRmEmptyRootDirRecursive() throws Throwable {
+
+ }
+
+ @Override
+ @Test
+ @Ignore
+ public void testRmEmptyRootDirNonRecursive() {
+ }
+
+ @Override
+ @Test
+ @Ignore
+ public void testRmRootRecursive() {
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/ITestS3ContractSeek.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/ITestS3ContractSeek.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/ITestS3ContractSeek.java
new file mode 100644
index 0000000..67b16b9
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/ITestS3ContractSeek.java
@@ -0,0 +1,44 @@
+/*
+ * 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.s3;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractSeekTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+import org.junit.Ignore;
+import org.junit.Test;
+
+/**
+ * S3 contract tests covering file seek.
+ */
+@Deprecated
+public class ITestS3ContractSeek extends AbstractContractSeekTest {
+
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new S3Contract(conf);
+ }
+
+ @Override
+ @Test
+ @Ignore
+ public void testReadFullyZeroByteFile() {
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/TestS3ContractCreate.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/TestS3ContractCreate.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/TestS3ContractCreate.java
deleted file mode 100644
index 4e98db9..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/TestS3ContractCreate.java
+++ /dev/null
@@ -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.contract.s3;
-
-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;
-
-@Deprecated
-public class TestS3ContractCreate extends AbstractContractCreateTest {
-
- @Override
- protected AbstractFSContract createContract(Configuration conf) {
- return new S3Contract(conf);
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/TestS3ContractDelete.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/TestS3ContractDelete.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/TestS3ContractDelete.java
deleted file mode 100644
index 37b4ed2..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/TestS3ContractDelete.java
+++ /dev/null
@@ -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.contract.s3;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractDeleteTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-
-@Deprecated
-public class TestS3ContractDelete extends AbstractContractDeleteTest {
-
- @Override
- protected AbstractFSContract createContract(Configuration conf) {
- return new S3Contract(conf);
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/TestS3ContractMkdir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/TestS3ContractMkdir.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/TestS3ContractMkdir.java
deleted file mode 100644
index 1e6ac7f..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/TestS3ContractMkdir.java
+++ /dev/null
@@ -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.contract.s3;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractMkdirTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.apache.hadoop.fs.contract.ContractTestUtils;
-
-@Deprecated
-public class TestS3ContractMkdir extends AbstractContractMkdirTest {
-
- @Override
- protected AbstractFSContract createContract(Configuration conf) {
- return new S3Contract(conf);
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/TestS3ContractOpen.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/TestS3ContractOpen.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/TestS3ContractOpen.java
deleted file mode 100644
index 2af61de..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/TestS3ContractOpen.java
+++ /dev/null
@@ -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.contract.s3;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractOpenTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.apache.hadoop.fs.contract.ContractTestUtils;
-
-@Deprecated
-public class TestS3ContractOpen extends AbstractContractOpenTest {
-
- @Override
- protected AbstractFSContract createContract(Configuration conf) {
- return new S3Contract(conf);
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/TestS3ContractRename.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/TestS3ContractRename.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/TestS3ContractRename.java
deleted file mode 100644
index ba45478..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/TestS3ContractRename.java
+++ /dev/null
@@ -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.contract.s3;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-
-@Deprecated
-public class TestS3ContractRename extends AbstractContractRenameTest {
-
- @Override
- protected AbstractFSContract createContract(Configuration conf) {
- return new S3Contract(conf);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/TestS3ContractRootDir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/TestS3ContractRootDir.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/TestS3ContractRootDir.java
deleted file mode 100644
index 452862d..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/TestS3ContractRootDir.java
+++ /dev/null
@@ -1,57 +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.s3;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-
-import org.junit.Ignore;
-import org.junit.Test;
-
-/**
- * root dir operations against an S3 bucket
- */
-@Deprecated
-public class TestS3ContractRootDir extends AbstractContractRootDirectoryTest {
-
- @Override
- protected AbstractFSContract createContract(Configuration conf) {
- return new S3Contract(conf);
- }
-
- @Override
- @Test
- @Ignore
- public void testRmEmptyRootDirRecursive() throws Throwable {
-
- }
-
- @Override
- @Test
- @Ignore
- public void testRmEmptyRootDirNonRecursive() {
- }
-
- @Override
- @Test
- @Ignore
- public void testRmRootRecursive() {
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/TestS3ContractSeek.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/TestS3ContractSeek.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/TestS3ContractSeek.java
deleted file mode 100644
index b15c0a2..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3/TestS3ContractSeek.java
+++ /dev/null
@@ -1,42 +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.s3;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractSeekTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.apache.hadoop.fs.contract.ContractTestUtils;
-
-import org.junit.Ignore;
-import org.junit.Test;
-
-@Deprecated
-public class TestS3ContractSeek extends AbstractContractSeekTest {
-
- @Override
- protected AbstractFSContract createContract(Configuration conf) {
- return new S3Contract(conf);
- }
-
- @Override
- @Test
- @Ignore
- public void testReadFullyZeroByteFile() {
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractCreate.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractCreate.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractCreate.java
new file mode 100644
index 0000000..d2a858f
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractCreate.java
@@ -0,0 +1,35 @@
+/*
+ * 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.s3a;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractCreateTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * S3A contract tests creating files.
+ */
+public class ITestS3AContractCreate extends AbstractContractCreateTest {
+
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new S3AContract(conf);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDelete.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDelete.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDelete.java
new file mode 100644
index 0000000..a47dcaef
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDelete.java
@@ -0,0 +1,34 @@
+/*
+ * 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.s3a;
+
+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 ITestS3AContractDelete extends AbstractContractDeleteTest {
+
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new S3AContract(conf);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java
new file mode 100644
index 0000000..28278fe
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java
@@ -0,0 +1,46 @@
+/**
+ * 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.s3a;
+
+import static org.apache.hadoop.fs.s3a.Constants.MIN_MULTIPART_THRESHOLD;
+import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_SIZE;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.tools.contract.AbstractContractDistCpTest;
+
+/**
+ * Contract test suite covering S3A integration with DistCp.
+ */
+public class ITestS3AContractDistCp extends AbstractContractDistCpTest {
+
+ private static final long MULTIPART_SETTING = 8 * 1024 * 1024; // 8 MB
+
+ @Override
+ protected Configuration createConfiguration() {
+ Configuration newConf = super.createConfiguration();
+ newConf.setLong(MIN_MULTIPART_THRESHOLD, MULTIPART_SETTING);
+ newConf.setLong(MULTIPART_SIZE, MULTIPART_SETTING);
+ return newConf;
+ }
+
+ @Override
+ protected S3AContract createContract(Configuration conf) {
+ return new S3AContract(conf);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractGetFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractGetFileStatus.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractGetFileStatus.java
new file mode 100644
index 0000000..c7ed5a3
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractGetFileStatus.java
@@ -0,0 +1,51 @@
+/**
+ * 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.s3a;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.apache.hadoop.fs.s3a.S3ATestUtils;
+
+/**
+ * S3A contract tests covering getFileStatus.
+ */
+public class ITestS3AContractGetFileStatus
+ extends AbstractContractGetFileStatusTest {
+
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new S3AContract(conf);
+ }
+
+ @Override
+ public void teardown() throws Exception {
+ getLog().info("FS details {}", getFileSystem());
+ super.teardown();
+ }
+
+ @Override
+ protected Configuration createConfiguration() {
+ Configuration conf = super.createConfiguration();
+ S3ATestUtils.disableFilesystemCaching(conf);
+ // aggressively low page size forces tests to go multipage
+ conf.setInt(Constants.MAX_PAGING_KEYS, 2);
+ return conf;
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractMkdir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractMkdir.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractMkdir.java
new file mode 100644
index 0000000..d953e7e
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractMkdir.java
@@ -0,0 +1,34 @@
+/*
+ * 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.s3a;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractMkdirTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Test dir operations on S3A.
+ */
+public class ITestS3AContractMkdir extends AbstractContractMkdirTest {
+
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new S3AContract(conf);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractOpen.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractOpen.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractOpen.java
new file mode 100644
index 0000000..a7bdc0d
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractOpen.java
@@ -0,0 +1,34 @@
+/*
+ * 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.s3a;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractOpenTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * S3A contract tests opening files.
+ */
+public class ITestS3AContractOpen extends AbstractContractOpenTest {
+
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new S3AContract(conf);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRename.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRename.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRename.java
new file mode 100644
index 0000000..5dba03d
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRename.java
@@ -0,0 +1,62 @@
+/*
+ * 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.s3a;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
+
+/**
+ * S3A contract tests covering rename.
+ */
+public class ITestS3AContractRename extends AbstractContractRenameTest {
+
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new S3AContract(conf);
+ }
+
+ @Override
+ public void testRenameDirIntoExistingDir() throws Throwable {
+ describe("Verify renaming a dir into an existing dir puts the files"
+ +" from the source dir into the existing dir"
+ +" and leaves existing files alone");
+ FileSystem fs = getFileSystem();
+ String sourceSubdir = "source";
+ Path srcDir = path(sourceSubdir);
+ Path srcFilePath = new Path(srcDir, "source-256.txt");
+ byte[] srcDataset = dataset(256, 'a', 'z');
+ writeDataset(fs, srcFilePath, srcDataset, srcDataset.length, 1024, false);
+ Path destDir = path("dest");
+
+ Path destFilePath = new Path(destDir, "dest-512.txt");
+ byte[] destDateset = dataset(512, 'A', 'Z');
+ writeDataset(fs, destFilePath, destDateset, destDateset.length, 1024,
+ false);
+ assertIsFile(destFilePath);
+
+ boolean rename = fs.rename(srcDir, destDir);
+ assertFalse("s3a doesn't support rename to non-empty directory", rename);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRootDir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRootDir.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRootDir.java
new file mode 100644
index 0000000..8383a77
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRootDir.java
@@ -0,0 +1,72 @@
+/*
+ * 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.s3a;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * root dir operations against an S3 bucket.
+ */
+public class ITestS3AContractRootDir extends
+ AbstractContractRootDirectoryTest {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(ITestS3AContractRootDir.class);
+
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new S3AContract(conf);
+ }
+
+ @Override
+ public void testListEmptyRootDirectory() throws IOException {
+ for (int attempt = 1, maxAttempts = 10; attempt <= maxAttempts; ++attempt) {
+ try {
+ super.testListEmptyRootDirectory();
+ break;
+ } catch (AssertionError | FileNotFoundException e) {
+ if (attempt < maxAttempts) {
+ LOG.info("Attempt {} of {} for empty root directory test failed. "
+ + "This is likely caused by eventual consistency of S3 "
+ + "listings. Attempting retry.", attempt, maxAttempts);
+ try {
+ Thread.sleep(1000);
+ } catch (InterruptedException e2) {
+ Thread.currentThread().interrupt();
+ fail("Test interrupted.");
+ break;
+ }
+ } else {
+ LOG.error(
+ "Empty root directory test failed {} attempts. Failing test.",
+ maxAttempts);
+ throw e;
+ }
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java
new file mode 100644
index 0000000..1572fbc
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java
@@ -0,0 +1,34 @@
+/*
+ * 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.s3a;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractSeekTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * S3A contract tests covering file seek.
+ */
+public class ITestS3AContractSeek extends AbstractContractSeekTest {
+
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new S3AContract(conf);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractCreate.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractCreate.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractCreate.java
deleted file mode 100644
index 035252b..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractCreate.java
+++ /dev/null
@@ -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.contract.s3a;
-
-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;
-
-public class TestS3AContractCreate extends AbstractContractCreateTest {
-
- @Override
- protected AbstractFSContract createContract(Configuration conf) {
- return new S3AContract(conf);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractDelete.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractDelete.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractDelete.java
deleted file mode 100644
index 733a517..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractDelete.java
+++ /dev/null
@@ -1,31 +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.s3a;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractDeleteTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-
-public class TestS3AContractDelete extends AbstractContractDeleteTest {
-
- @Override
- protected AbstractFSContract createContract(Configuration conf) {
- return new S3AContract(conf);
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractDistCp.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractDistCp.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractDistCp.java
deleted file mode 100644
index 7eb0afa..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractDistCp.java
+++ /dev/null
@@ -1,46 +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.s3a;
-
-import static org.apache.hadoop.fs.s3a.Constants.MIN_MULTIPART_THRESHOLD;
-import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_SIZE;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.tools.contract.AbstractContractDistCpTest;
-
-/**
- * Contract test suite covering S3A integration with DistCp.
- */
-public class TestS3AContractDistCp extends AbstractContractDistCpTest {
-
- private static final long MULTIPART_SETTING = 8 * 1024 * 1024; // 8 MB
-
- @Override
- protected Configuration createConfiguration() {
- Configuration newConf = super.createConfiguration();
- newConf.setLong(MIN_MULTIPART_THRESHOLD, MULTIPART_SETTING);
- newConf.setLong(MULTIPART_SIZE, MULTIPART_SETTING);
- return newConf;
- }
-
- @Override
- protected S3AContract createContract(Configuration conf) {
- return new S3AContract(conf);
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractGetFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractGetFileStatus.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractGetFileStatus.java
deleted file mode 100644
index 5937d49..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractGetFileStatus.java
+++ /dev/null
@@ -1,47 +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.s3a;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest;
-import org.apache.hadoop.fs.s3a.Constants;
-import org.apache.hadoop.fs.s3a.S3ATestUtils;
-
-public class TestS3AContractGetFileStatus extends AbstractContractGetFileStatusTest {
-
- @Override
- protected AbstractFSContract createContract(Configuration conf) {
- return new S3AContract(conf);
- }
-
- @Override
- public void teardown() throws Exception {
- getLog().info("FS details {}", getFileSystem());
- super.teardown();
- }
-
- @Override
- protected Configuration createConfiguration() {
- Configuration conf = super.createConfiguration();
- S3ATestUtils.disableFilesystemCaching(conf);
- // aggressively low page size forces tests to go multipage
- conf.setInt(Constants.MAX_PAGING_KEYS, 2);
- return conf;
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractMkdir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractMkdir.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractMkdir.java
deleted file mode 100644
index a312782..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractMkdir.java
+++ /dev/null
@@ -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.s3a;
-
-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 TestS3AContractMkdir extends AbstractContractMkdirTest {
-
- @Override
- protected AbstractFSContract createContract(Configuration conf) {
- return new S3AContract(conf);
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractOpen.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractOpen.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractOpen.java
deleted file mode 100644
index f735deb..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractOpen.java
+++ /dev/null
@@ -1,31 +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.s3a;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractOpenTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-
-public class TestS3AContractOpen extends AbstractContractOpenTest {
-
- @Override
- protected AbstractFSContract createContract(Configuration conf) {
- return new S3AContract(conf);
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractRename.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractRename.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractRename.java
deleted file mode 100644
index af1ed37..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractRename.java
+++ /dev/null
@@ -1,61 +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.s3a;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.junit.Test;
-
-import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
-import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
-
-public class TestS3AContractRename extends AbstractContractRenameTest {
-
- @Override
- protected AbstractFSContract createContract(Configuration conf) {
- return new S3AContract(conf);
- }
-
- @Override
- public void testRenameDirIntoExistingDir() throws Throwable {
- describe("Verify renaming a dir into an existing dir puts the files"
- +" from the source dir into the existing dir"
- +" and leaves existing files alone");
- FileSystem fs = getFileSystem();
- String sourceSubdir = "source";
- Path srcDir = path(sourceSubdir);
- Path srcFilePath = new Path(srcDir, "source-256.txt");
- byte[] srcDataset = dataset(256, 'a', 'z');
- writeDataset(fs, srcFilePath, srcDataset, srcDataset.length, 1024, false);
- Path destDir = path("dest");
-
- Path destFilePath = new Path(destDir, "dest-512.txt");
- byte[] destDateset = dataset(512, 'A', 'Z');
- writeDataset(fs, destFilePath, destDateset, destDateset.length, 1024,
- false);
- assertIsFile(destFilePath);
-
- boolean rename = fs.rename(srcDir, destDir);
- assertFalse("s3a doesn't support rename to non-empty directory", rename);
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractRootDir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractRootDir.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractRootDir.java
deleted file mode 100644
index 1cf1eb8..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractRootDir.java
+++ /dev/null
@@ -1,72 +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.s3a;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * root dir operations against an S3 bucket.
- */
-public class TestS3AContractRootDir extends
- AbstractContractRootDirectoryTest {
-
- private static final Logger LOG =
- LoggerFactory.getLogger(TestS3AContractRootDir.class);
-
- @Override
- protected AbstractFSContract createContract(Configuration conf) {
- return new S3AContract(conf);
- }
-
- @Override
- public void testListEmptyRootDirectory() throws IOException {
- for (int attempt = 1, maxAttempts = 10; attempt <= maxAttempts; ++attempt) {
- try {
- super.testListEmptyRootDirectory();
- break;
- } catch (AssertionError | FileNotFoundException e) {
- if (attempt < maxAttempts) {
- LOG.info("Attempt {} of {} for empty root directory test failed. "
- + "This is likely caused by eventual consistency of S3 "
- + "listings. Attempting retry.", attempt, maxAttempts);
- try {
- Thread.sleep(1000);
- } catch (InterruptedException e2) {
- Thread.currentThread().interrupt();
- fail("Test interrupted.");
- break;
- }
- } else {
- LOG.error(
- "Empty root directory test failed {} attempts. Failing test.",
- maxAttempts);
- throw e;
- }
- }
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/027d76fa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractSeek.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractSeek.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractSeek.java
deleted file mode 100644
index d677ec4..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/TestS3AContractSeek.java
+++ /dev/null
@@ -1,31 +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.s3a;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractSeekTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-
-public class TestS3AContractSeek extends AbstractContractSeekTest {
-
- @Override
- protected AbstractFSContract createContract(Configuration conf) {
- return new S3AContract(conf);
- }
-}
---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org