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