Bladeren bron

HADOOP-13560. S3ABlockOutputStream to support huge (many GB) file writes. Contributed by Steve Loughran

Steve Loughran 8 jaren geleden
bovenliggende
commit
6c348c5691
37 gewijzigde bestanden met toevoegingen van 4615 en 874 verwijderingen
  1. 57 17
      hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
  2. 13 3
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
  3. 56 2
      hadoop-tools/hadoop-aws/pom.xml
  4. 33 135
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BlockingThreadPoolExecutorService.java
  5. 69 2
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
  6. 703 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
  7. 821 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java
  8. 0 410
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java
  9. 361 47
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
  10. 245 3
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
  11. 28 29
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
  12. 39 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
  13. 230 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SemaphoredDelegatingExecutor.java
  14. 30 2
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
  15. 621 47
      hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
  16. 6 4
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java
  17. 1 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java
  18. 38 10
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestBlockingThreadPoolExecutorService.java
  19. 90 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java
  20. 30 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputByteBuffer.java
  21. 30 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputDisk.java
  22. 2 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockingThreadPool.java
  23. 29 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
  24. 4 3
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionBlockOutputStream.java
  25. 0 74
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFastOutputStream.java
  26. 98 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ATestUtils.java
  27. 73 2
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java
  28. 145 3
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
  29. 124 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestDataBlocks.java
  30. 1 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java
  31. 412 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java
  32. 4 15
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADeleteManyFiles.java
  33. 31 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesArrayBlocks.java
  34. 34 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesByteBufferBlocks.java
  35. 41 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesClassicOutput.java
  36. 31 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesDiskBlocks.java
  37. 85 66
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java

+ 57 - 17
hadoop-common-project/hadoop-common/src/main/resources/core-default.xml

@@ -994,8 +994,8 @@
 <property>
 <property>
   <name>fs.s3a.threads.max</name>
   <name>fs.s3a.threads.max</name>
   <value>10</value>
   <value>10</value>
-  <description> Maximum number of concurrent active (part)uploads,
-    which each use a thread from the threadpool.</description>
+  <description>The total number of threads available in the filesystem for data
+    uploads *or any other queued filesystem operation*.</description>
 </property>
 </property>
 
 
 <property>
 <property>
@@ -1008,8 +1008,7 @@
 <property>
 <property>
   <name>fs.s3a.max.total.tasks</name>
   <name>fs.s3a.max.total.tasks</name>
   <value>5</value>
   <value>5</value>
-  <description>Number of (part)uploads allowed to the queue before
-    blocking additional uploads.</description>
+  <description>The number of operations which can be queued for execution</description>
 </property>
 </property>
 
 
 <property>
 <property>
@@ -1047,13 +1046,21 @@
   <name>fs.s3a.multipart.purge</name>
   <name>fs.s3a.multipart.purge</name>
   <value>false</value>
   <value>false</value>
   <description>True if you want to purge existing multipart uploads that may not have been
   <description>True if you want to purge existing multipart uploads that may not have been
-     completed/aborted correctly</description>
+    completed/aborted correctly. The corresponding purge age is defined in
+    fs.s3a.multipart.purge.age.
+    If set, when the filesystem is instantiated then all outstanding uploads
+    older than the purge age will be terminated -across the entire bucket.
+    This will impact multipart uploads by other applications and users. so should
+    be used sparingly, with an age value chosen to stop failed uploads, without
+    breaking ongoing operations.
+  </description>
 </property>
 </property>
 
 
 <property>
 <property>
   <name>fs.s3a.multipart.purge.age</name>
   <name>fs.s3a.multipart.purge.age</name>
   <value>86400</value>
   <value>86400</value>
-  <description>Minimum age in seconds of multipart uploads to purge</description>
+  <description>Minimum age in seconds of multipart uploads to purge.
+  </description>
 </property>
 </property>
 
 
 <property>
 <property>
@@ -1086,10 +1093,50 @@
 <property>
 <property>
   <name>fs.s3a.fast.upload</name>
   <name>fs.s3a.fast.upload</name>
   <value>false</value>
   <value>false</value>
-  <description>Upload directly from memory instead of buffering to
-    disk first. Memory usage and parallelism can be controlled as up to
-    fs.s3a.multipart.size memory is consumed for each (part)upload actively
-    uploading (fs.s3a.threads.max) or queueing (fs.s3a.max.total.tasks)</description>
+  <description>
+    Use the incremental block-based fast upload mechanism with
+    the buffering mechanism set in fs.s3a.fast.upload.buffer.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.fast.upload.buffer</name>
+  <value>disk</value>
+  <description>
+    The buffering mechanism to use when using S3A fast upload
+    (fs.s3a.fast.upload=true). Values: disk, array, bytebuffer.
+    This configuration option has no effect if fs.s3a.fast.upload is false.
+
+    "disk" will use the directories listed in fs.s3a.buffer.dir as
+    the location(s) to save data prior to being uploaded.
+
+    "array" uses arrays in the JVM heap
+
+    "bytebuffer" uses off-heap memory within the JVM.
+
+    Both "array" and "bytebuffer" will consume memory in a single stream up to the number
+    of blocks set by:
+
+        fs.s3a.multipart.size * fs.s3a.fast.upload.active.blocks.
+
+    If using either of these mechanisms, keep this value low
+
+    The total number of threads performing work across all threads is set by
+    fs.s3a.threads.max, with fs.s3a.max.total.tasks values setting the number of queued
+    work items.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.fast.upload.active.blocks</name>
+  <value>4</value>
+  <description>
+    Maximum Number of blocks a single output stream can have
+    active (uploading, or queued to the central FileSystem
+    instance's pool of queued operations.
+
+    This stops a single stream overloading the shared thread pool.
+  </description>
 </property>
 </property>
 
 
 <property>
 <property>
@@ -1100,13 +1147,6 @@
   any call to setReadahead() is made to an open stream.</description>
   any call to setReadahead() is made to an open stream.</description>
 </property>
 </property>
 
 
-<property>
-  <name>fs.s3a.fast.buffer.size</name>
-  <value>1048576</value>
-  <description>Size of initial memory buffer in bytes allocated for an
-    upload. No effect if fs.s3a.fast.upload is false.</description>
-</property>
-
 <property>
 <property>
   <name>fs.s3a.user.agent.prefix</name>
   <name>fs.s3a.user.agent.prefix</name>
   <value></value>
   <value></value>

+ 13 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java

@@ -965,7 +965,7 @@ public class ContractTestUtils extends Assert {
    * @return the number of megabytes/second of the recorded operation
    * @return the number of megabytes/second of the recorded operation
    */
    */
   public static double bandwidthMBs(long bytes, long durationNS) {
   public static double bandwidthMBs(long bytes, long durationNS) {
-    return (bytes * 1000.0) / durationNS;
+    return bytes / (1024.0 * 1024) * 1.0e9 / durationNS;
   }
   }
 
 
   /**
   /**
@@ -1415,6 +1415,14 @@ public class ContractTestUtils extends Assert {
       return endTime - startTime;
       return endTime - startTime;
     }
     }
 
 
+    /**
+     * Intermediate duration of the operation.
+     * @return how much time has passed since the start (in nanos).
+     */
+    public long elapsedTime() {
+      return now() - startTime;
+    }
+
     public double bandwidth(long bytes) {
     public double bandwidth(long bytes) {
       return bandwidthMBs(bytes, duration());
       return bandwidthMBs(bytes, duration());
     }
     }
@@ -1422,10 +1430,12 @@ public class ContractTestUtils extends Assert {
     /**
     /**
      * Bandwidth as bytes per second.
      * Bandwidth as bytes per second.
      * @param bytes bytes in
      * @param bytes bytes in
-     * @return the number of bytes per second this operation timed.
+     * @return the number of bytes per second this operation.
+     *         0 if duration == 0.
      */
      */
     public double bandwidthBytes(long bytes) {
     public double bandwidthBytes(long bytes) {
-      return (bytes * 1.0) / duration();
+      double duration = duration();
+      return duration > 0 ? bytes / duration : 0;
     }
     }
 
 
     /**
     /**

+ 56 - 2
hadoop-tools/hadoop-aws/pom.xml

@@ -35,6 +35,15 @@
     <file.encoding>UTF-8</file.encoding>
     <file.encoding>UTF-8</file.encoding>
     <downloadSources>true</downloadSources>
     <downloadSources>true</downloadSources>
     <hadoop.tmp.dir>${project.build.directory}/test</hadoop.tmp.dir>
     <hadoop.tmp.dir>${project.build.directory}/test</hadoop.tmp.dir>
+
+    <!-- are scale tests enabled ? -->
+    <fs.s3a.scale.test.enabled>unset</fs.s3a.scale.test.enabled>
+    <!-- Size in MB of huge files. -->
+    <fs.s3a.scale.test.huge.filesize>unset</fs.s3a.scale.test.huge.filesize>
+    <!-- Size in MB of the partion size in huge file uploads. -->
+    <fs.s3a.scale.test.huge.partitionsize>unset</fs.s3a.scale.test.huge.partitionsize>
+    <!-- Timeout in seconds for scale tests.-->
+    <fs.s3a.scale.test.timeout>3600</fs.s3a.scale.test.timeout>
   </properties>
   </properties>
 
 
   <profiles>
   <profiles>
@@ -115,6 +124,11 @@
                 <!-- substitution.  Putting a prefix in front of it like -->
                 <!-- substitution.  Putting a prefix in front of it like -->
                 <!-- "fork-" makes it work. -->
                 <!-- "fork-" makes it work. -->
                 <test.unique.fork.id>fork-${surefire.forkNumber}</test.unique.fork.id>
                 <test.unique.fork.id>fork-${surefire.forkNumber}</test.unique.fork.id>
+                <!-- Propagate scale parameters -->
+                <fs.s3a.scale.test.enabled>${fs.s3a.scale.test.enabled}</fs.s3a.scale.test.enabled>
+                <fs.s3a.scale.test.huge.filesize>${fs.s3a.scale.test.huge.filesize}</fs.s3a.scale.test.huge.filesize>
+                <fs.s3a.scale.test.huge.huge.partitionsize>${fs.s3a.scale.test.huge.partitionsize}</fs.s3a.scale.test.huge.huge.partitionsize>
+                <fs.s3a.scale.test.timeout>${fs.s3a.scale.test.timeout}</fs.s3a.scale.test.timeout>
               </systemPropertyVariables>
               </systemPropertyVariables>
             </configuration>
             </configuration>
           </plugin>
           </plugin>
@@ -132,7 +146,10 @@
                   <forkCount>${testsThreadCount}</forkCount>
                   <forkCount>${testsThreadCount}</forkCount>
                   <reuseForks>false</reuseForks>
                   <reuseForks>false</reuseForks>
                   <argLine>${maven-surefire-plugin.argLine} -DminiClusterDedicatedDirs=true</argLine>
                   <argLine>${maven-surefire-plugin.argLine} -DminiClusterDedicatedDirs=true</argLine>
+                  <forkedProcessTimeoutInSeconds>${fs.s3a.scale.test.timeout}</forkedProcessTimeoutInSeconds>
                   <systemPropertyVariables>
                   <systemPropertyVariables>
+                    <!-- Tell tests that they are being executed in parallel -->
+                    <test.parallel.execution>true</test.parallel.execution>
                     <test.build.data>${test.build.data}/${surefire.forkNumber}</test.build.data>
                     <test.build.data>${test.build.data}/${surefire.forkNumber}</test.build.data>
                     <test.build.dir>${test.build.dir}/${surefire.forkNumber}</test.build.dir>
                     <test.build.dir>${test.build.dir}/${surefire.forkNumber}</test.build.dir>
                     <hadoop.tmp.dir>${hadoop.tmp.dir}/${surefire.forkNumber}</hadoop.tmp.dir>
                     <hadoop.tmp.dir>${hadoop.tmp.dir}/${surefire.forkNumber}</hadoop.tmp.dir>
@@ -142,6 +159,11 @@
                     <!-- substitution.  Putting a prefix in front of it like -->
                     <!-- substitution.  Putting a prefix in front of it like -->
                     <!-- "fork-" makes it work. -->
                     <!-- "fork-" makes it work. -->
                     <test.unique.fork.id>fork-${surefire.forkNumber}</test.unique.fork.id>
                     <test.unique.fork.id>fork-${surefire.forkNumber}</test.unique.fork.id>
+                    <!-- Propagate scale parameters -->
+                    <fs.s3a.scale.test.enabled>${fs.s3a.scale.test.enabled}</fs.s3a.scale.test.enabled>
+                    <fs.s3a.scale.test.huge.filesize>${fs.s3a.scale.test.huge.filesize}</fs.s3a.scale.test.huge.filesize>
+                    <fs.s3a.scale.test.huge.huge.partitionsize>${fs.s3a.scale.test.huge.partitionsize}</fs.s3a.scale.test.huge.huge.partitionsize>
+                    <fs.s3a.scale.test.timeout>${fs.s3a.scale.test.timeout}</fs.s3a.scale.test.timeout>
                   </systemPropertyVariables>
                   </systemPropertyVariables>
                   <!-- Some tests cannot run in parallel.  Tests that cover -->
                   <!-- Some tests cannot run in parallel.  Tests that cover -->
                   <!-- access to the root directory must run in isolation -->
                   <!-- access to the root directory must run in isolation -->
@@ -160,10 +182,11 @@
                   <excludes>
                   <excludes>
                     <exclude>**/ITestJets3tNativeS3FileSystemContract.java</exclude>
                     <exclude>**/ITestJets3tNativeS3FileSystemContract.java</exclude>
                     <exclude>**/ITestS3ABlockingThreadPool.java</exclude>
                     <exclude>**/ITestS3ABlockingThreadPool.java</exclude>
-                    <exclude>**/ITestS3AFastOutputStream.java</exclude>
                     <exclude>**/ITestS3AFileSystemContract.java</exclude>
                     <exclude>**/ITestS3AFileSystemContract.java</exclude>
                     <exclude>**/ITestS3AMiniYarnCluster.java</exclude>
                     <exclude>**/ITestS3AMiniYarnCluster.java</exclude>
                     <exclude>**/ITest*Root*.java</exclude>
                     <exclude>**/ITest*Root*.java</exclude>
+                    <exclude>**/ITestS3AFileContextStatistics.java</exclude>
+                    <include>**/ITestS3AHuge*.java</include>
                   </excludes>
                   </excludes>
                 </configuration>
                 </configuration>
               </execution>
               </execution>
@@ -174,6 +197,16 @@
                   <goal>verify</goal>
                   <goal>verify</goal>
                 </goals>
                 </goals>
                 <configuration>
                 <configuration>
+                  <forkedProcessTimeoutInSeconds>${fs.s3a.scale.test.timeout}</forkedProcessTimeoutInSeconds>
+                  <systemPropertyVariables>
+                    <!-- Tell tests that they are being executed sequentially -->
+                    <test.parallel.execution>false</test.parallel.execution>
+                    <!-- Propagate scale parameters -->
+                    <fs.s3a.scale.test.enabled>${fs.s3a.scale.test.enabled}</fs.s3a.scale.test.enabled>
+                    <fs.s3a.scale.test.huge.filesize>${fs.s3a.scale.test.huge.filesize}</fs.s3a.scale.test.huge.filesize>
+                    <fs.s3a.scale.test.huge.huge.partitionsize>${fs.s3a.scale.test.huge.partitionsize}</fs.s3a.scale.test.huge.huge.partitionsize>
+                    <fs.s3a.scale.test.timeout>${fs.s3a.scale.test.timeout}</fs.s3a.scale.test.timeout>
+                  </systemPropertyVariables>
                   <!-- Do a sequential run for tests that cannot handle -->
                   <!-- Do a sequential run for tests that cannot handle -->
                   <!-- parallel execution. -->
                   <!-- parallel execution. -->
                   <includes>
                   <includes>
@@ -183,6 +216,8 @@
                     <include>**/ITestS3AFileSystemContract.java</include>
                     <include>**/ITestS3AFileSystemContract.java</include>
                     <include>**/ITestS3AMiniYarnCluster.java</include>
                     <include>**/ITestS3AMiniYarnCluster.java</include>
                     <include>**/ITest*Root*.java</include>
                     <include>**/ITest*Root*.java</include>
+                    <include>**/ITestS3AFileContextStatistics.java</include>
+                    <include>**/ITestS3AHuge*.java</include>
                   </includes>
                   </includes>
                 </configuration>
                 </configuration>
               </execution>
               </execution>
@@ -210,7 +245,13 @@
                   <goal>verify</goal>
                   <goal>verify</goal>
                 </goals>
                 </goals>
                 <configuration>
                 <configuration>
-                  <forkedProcessTimeoutInSeconds>3600</forkedProcessTimeoutInSeconds>
+                  <systemPropertyVariables>
+                    <!-- Propagate scale parameters -->
+                    <fs.s3a.scale.test.enabled>${fs.s3a.scale.test.enabled}</fs.s3a.scale.test.enabled>
+                    <fs.s3a.scale.test.huge.filesize>${fs.s3a.scale.test.huge.filesize}</fs.s3a.scale.test.huge.filesize>
+                    <fs.s3a.scale.test.timeout>${fs.s3a.scale.test.timeout}</fs.s3a.scale.test.timeout>
+                  </systemPropertyVariables>
+                  <forkedProcessTimeoutInSeconds>${fs.s3a.scale.test.timeout}</forkedProcessTimeoutInSeconds>
                 </configuration>
                 </configuration>
               </execution>
               </execution>
             </executions>
             </executions>
@@ -218,6 +259,19 @@
         </plugins>
         </plugins>
       </build>
       </build>
     </profile>
     </profile>
+
+    <!-- Turn on scale tests-->
+    <profile>
+      <id>scale</id>
+      <activation>
+        <property>
+          <name>scale</name>
+        </property>
+      </activation>
+      <properties >
+        <fs.s3a.scale.test.enabled>true</fs.s3a.scale.test.enabled>
+      </properties>
+    </profile>
   </profiles>
   </profiles>
 
 
   <build>
   <build>

+ 33 - 135
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BlockingThreadPoolExecutorService.java

@@ -18,30 +18,21 @@
 
 
 package org.apache.hadoop.fs.s3a;
 package org.apache.hadoop.fs.s3a;
 
 
-import java.util.Collection;
-import java.util.List;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.RejectedExecutionHandler;
 import java.util.concurrent.RejectedExecutionHandler;
-import java.util.concurrent.Semaphore;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicInteger;
 
 
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
-import com.google.common.util.concurrent.ForwardingListeningExecutorService;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
 import com.google.common.util.concurrent.MoreExecutors;
 
 
+import org.apache.hadoop.classification.InterfaceAudience;
+
 /**
 /**
  * This ExecutorService blocks the submission of new tasks when its queue is
  * This ExecutorService blocks the submission of new tasks when its queue is
  * already full by using a semaphore. Task submissions require permits, task
  * already full by using a semaphore. Task submissions require permits, task
@@ -50,17 +41,17 @@ import com.google.common.util.concurrent.MoreExecutors;
  * This is inspired by <a href="https://github.com/apache/incubator-s4/blob/master/subprojects/s4-comm/src/main/java/org/apache/s4/comm/staging/BlockingThreadPoolExecutorService.java">
  * This is inspired by <a href="https://github.com/apache/incubator-s4/blob/master/subprojects/s4-comm/src/main/java/org/apache/s4/comm/staging/BlockingThreadPoolExecutorService.java">
  * this s4 threadpool</a>
  * this s4 threadpool</a>
  */
  */
-public class BlockingThreadPoolExecutorService
-    extends ForwardingListeningExecutorService {
+@InterfaceAudience.Private
+final class BlockingThreadPoolExecutorService
+    extends SemaphoredDelegatingExecutor {
 
 
   private static Logger LOG = LoggerFactory
   private static Logger LOG = LoggerFactory
       .getLogger(BlockingThreadPoolExecutorService.class);
       .getLogger(BlockingThreadPoolExecutorService.class);
 
 
-  private Semaphore queueingPermits;
-  private ListeningExecutorService executorDelegatee;
-
   private static final AtomicInteger POOLNUMBER = new AtomicInteger(1);
   private static final AtomicInteger POOLNUMBER = new AtomicInteger(1);
 
 
+  private final ThreadPoolExecutor eventProcessingExecutor;
+
   /**
   /**
    * Returns a {@link java.util.concurrent.ThreadFactory} that names each
    * Returns a {@link java.util.concurrent.ThreadFactory} that names each
    * created thread uniquely,
    * created thread uniquely,
@@ -69,7 +60,7 @@ public class BlockingThreadPoolExecutorService
    * @param prefix The prefix of every created Thread's name
    * @param prefix The prefix of every created Thread's name
    * @return a {@link java.util.concurrent.ThreadFactory} that names threads
    * @return a {@link java.util.concurrent.ThreadFactory} that names threads
    */
    */
-  public static ThreadFactory getNamedThreadFactory(final String prefix) {
+  static ThreadFactory getNamedThreadFactory(final String prefix) {
     SecurityManager s = System.getSecurityManager();
     SecurityManager s = System.getSecurityManager();
     final ThreadGroup threadGroup = (s != null) ? s.getThreadGroup() :
     final ThreadGroup threadGroup = (s != null) ? s.getThreadGroup() :
         Thread.currentThread().getThreadGroup();
         Thread.currentThread().getThreadGroup();
@@ -113,6 +104,12 @@ public class BlockingThreadPoolExecutorService
     };
     };
   }
   }
 
 
+  private BlockingThreadPoolExecutorService(int permitCount,
+      ThreadPoolExecutor eventProcessingExecutor) {
+    super(MoreExecutors.listeningDecorator(eventProcessingExecutor),
+        permitCount, false);
+    this.eventProcessingExecutor = eventProcessingExecutor;
+  }
 
 
   /**
   /**
    * A thread pool that that blocks clients submitting additional tasks if
    * A thread pool that that blocks clients submitting additional tasks if
@@ -125,10 +122,12 @@ public class BlockingThreadPoolExecutorService
    * @param unit time unit
    * @param unit time unit
    * @param prefixName prefix of name for threads
    * @param prefixName prefix of name for threads
    */
    */
-  public BlockingThreadPoolExecutorService(int activeTasks, int waitingTasks,
-      long keepAliveTime, TimeUnit unit, String prefixName) {
-    super();
-    queueingPermits = new Semaphore(waitingTasks + activeTasks, false);
+  public static BlockingThreadPoolExecutorService newInstance(
+      int activeTasks,
+      int waitingTasks,
+      long keepAliveTime, TimeUnit unit,
+      String prefixName) {
+
     /* Although we generally only expect up to waitingTasks tasks in the
     /* Although we generally only expect up to waitingTasks tasks in the
     queue, we need to be able to buffer all tasks in case dequeueing is
     queue, we need to be able to buffer all tasks in case dequeueing is
     slower than enqueueing. */
     slower than enqueueing. */
@@ -147,126 +146,25 @@ public class BlockingThreadPoolExecutorService
               }
               }
             });
             });
     eventProcessingExecutor.allowCoreThreadTimeOut(true);
     eventProcessingExecutor.allowCoreThreadTimeOut(true);
-    executorDelegatee =
-        MoreExecutors.listeningDecorator(eventProcessingExecutor);
-
-  }
-
-  @Override
-  protected ListeningExecutorService delegate() {
-    return executorDelegatee;
-  }
-
-  @Override
-  public <T> ListenableFuture<T> submit(Callable<T> task) {
-    try {
-      queueingPermits.acquire();
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-      return Futures.immediateFailedCheckedFuture(e);
-    }
-    return super.submit(new CallableWithPermitRelease<T>(task));
-  }
-
-  @Override
-  public <T> ListenableFuture<T> submit(Runnable task, T result) {
-    try {
-      queueingPermits.acquire();
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-      return Futures.immediateFailedCheckedFuture(e);
-    }
-    return super.submit(new RunnableWithPermitRelease(task), result);
-  }
-
-  @Override
-  public ListenableFuture<?> submit(Runnable task) {
-    try {
-      queueingPermits.acquire();
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-      return Futures.immediateFailedCheckedFuture(e);
-    }
-    return super.submit(new RunnableWithPermitRelease(task));
-  }
-
-  @Override
-  public void execute(Runnable command) {
-    try {
-      queueingPermits.acquire();
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-    }
-    super.execute(new RunnableWithPermitRelease(command));
+    return new BlockingThreadPoolExecutorService(waitingTasks + activeTasks,
+        eventProcessingExecutor);
   }
   }
 
 
   /**
   /**
-   * Releases a permit after the task is executed.
+   * Get the actual number of active threads.
+   * @return the active thread count
    */
    */
-  class RunnableWithPermitRelease implements Runnable {
-
-    private Runnable delegatee;
-
-    public RunnableWithPermitRelease(Runnable delegatee) {
-      this.delegatee = delegatee;
-    }
-
-    @Override
-    public void run() {
-      try {
-        delegatee.run();
-      } finally {
-        queueingPermits.release();
-      }
-
-    }
-  }
-
-  /**
-   * Releases a permit after the task is completed.
-   */
-  class CallableWithPermitRelease<T> implements Callable<T> {
-
-    private Callable<T> delegatee;
-
-    public CallableWithPermitRelease(Callable<T> delegatee) {
-      this.delegatee = delegatee;
-    }
-
-    @Override
-    public T call() throws Exception {
-      try {
-        return delegatee.call();
-      } finally {
-        queueingPermits.release();
-      }
-    }
-
-  }
-
-  @Override
-  public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks)
-      throws InterruptedException {
-    throw new RuntimeException("Not implemented");
+  int getActiveCount() {
+    return eventProcessingExecutor.getActiveCount();
   }
   }
 
 
   @Override
   @Override
-  public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks,
-      long timeout, TimeUnit unit) throws InterruptedException {
-    throw new RuntimeException("Not implemented");
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "BlockingThreadPoolExecutorService{");
+    sb.append(super.toString());
+    sb.append(", activeCount=").append(getActiveCount());
+    sb.append('}');
+    return sb.toString();
   }
   }
-
-  @Override
-  public <T> T invokeAny(Collection<? extends Callable<T>> tasks)
-      throws InterruptedException, ExecutionException {
-    throw new RuntimeException("Not implemented");
-  }
-
-  @Override
-  public <T> T invokeAny(Collection<? extends Callable<T>> tasks, long timeout,
-      TimeUnit unit)
-      throws InterruptedException, ExecutionException, TimeoutException {
-    throw new RuntimeException("Not implemented");
-  }
-
 }
 }

+ 69 - 2
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java

@@ -35,6 +35,9 @@ public final class Constants {
   private Constants() {
   private Constants() {
   }
   }
 
 
+  /** The minimum multipart size which S3 supports. */
+  public static final int MULTIPART_MIN_SIZE = 5 * 1024 * 1024;
+
   // s3 access key
   // s3 access key
   public static final String ACCESS_KEY = "fs.s3a.access.key";
   public static final String ACCESS_KEY = "fs.s3a.access.key";
 
 
@@ -124,14 +127,72 @@ public final class Constants {
   // comma separated list of directories
   // comma separated list of directories
   public static final String BUFFER_DIR = "fs.s3a.buffer.dir";
   public static final String BUFFER_DIR = "fs.s3a.buffer.dir";
 
 
-  // should we upload directly from memory rather than using a file buffer
+  // switch to the fast block-by-block upload mechanism
   public static final String FAST_UPLOAD = "fs.s3a.fast.upload";
   public static final String FAST_UPLOAD = "fs.s3a.fast.upload";
   public static final boolean DEFAULT_FAST_UPLOAD = false;
   public static final boolean DEFAULT_FAST_UPLOAD = false;
 
 
   //initial size of memory buffer for a fast upload
   //initial size of memory buffer for a fast upload
+  @Deprecated
   public static final String FAST_BUFFER_SIZE = "fs.s3a.fast.buffer.size";
   public static final String FAST_BUFFER_SIZE = "fs.s3a.fast.buffer.size";
   public static final int DEFAULT_FAST_BUFFER_SIZE = 1048576; //1MB
   public static final int DEFAULT_FAST_BUFFER_SIZE = 1048576; //1MB
 
 
+  /**
+   * What buffer to use.
+   * Default is {@link #FAST_UPLOAD_BUFFER_DISK}
+   * Value: {@value}
+   */
+  @InterfaceStability.Unstable
+  public static final String FAST_UPLOAD_BUFFER =
+      "fs.s3a.fast.upload.buffer";
+
+  /**
+   * Buffer blocks to disk: {@value}.
+   * Capacity is limited to available disk space.
+   */
+
+  @InterfaceStability.Unstable
+  public static final String FAST_UPLOAD_BUFFER_DISK = "disk";
+
+  /**
+   * Use an in-memory array. Fast but will run of heap rapidly: {@value}.
+   */
+  @InterfaceStability.Unstable
+  public static final String FAST_UPLOAD_BUFFER_ARRAY = "array";
+
+  /**
+   * Use a byte buffer. May be more memory efficient than the
+   * {@link #FAST_UPLOAD_BUFFER_ARRAY}: {@value}.
+   */
+  @InterfaceStability.Unstable
+  public static final String FAST_UPLOAD_BYTEBUFFER = "bytebuffer";
+
+  /**
+   * Default buffer option: {@value}.
+   */
+  @InterfaceStability.Unstable
+  public static final String DEFAULT_FAST_UPLOAD_BUFFER =
+      FAST_UPLOAD_BUFFER_DISK;
+
+  /**
+   * Maximum Number of blocks a single output stream can have
+   * active (uploading, or queued to the central FileSystem
+   * instance's pool of queued operations.
+   * This stops a single stream overloading the shared thread pool.
+   * {@value}
+   * <p>
+   * Default is {@link #DEFAULT_FAST_UPLOAD_ACTIVE_BLOCKS}
+   */
+  @InterfaceStability.Unstable
+  public static final String FAST_UPLOAD_ACTIVE_BLOCKS =
+      "fs.s3a.fast.upload.active.blocks";
+
+  /**
+   * Limit of queued block upload operations before writes
+   * block. Value: {@value}
+   */
+  @InterfaceStability.Unstable
+  public static final int DEFAULT_FAST_UPLOAD_ACTIVE_BLOCKS = 4;
+
   // Private | PublicRead | PublicReadWrite | AuthenticatedRead |
   // Private | PublicRead | PublicReadWrite | AuthenticatedRead |
   // LogDeliveryWrite | BucketOwnerRead | BucketOwnerFullControl
   // LogDeliveryWrite | BucketOwnerRead | BucketOwnerFullControl
   public static final String CANNED_ACL = "fs.s3a.acl.default";
   public static final String CANNED_ACL = "fs.s3a.acl.default";
@@ -145,7 +206,7 @@ public final class Constants {
   // purge any multipart uploads older than this number of seconds
   // purge any multipart uploads older than this number of seconds
   public static final String PURGE_EXISTING_MULTIPART_AGE =
   public static final String PURGE_EXISTING_MULTIPART_AGE =
       "fs.s3a.multipart.purge.age";
       "fs.s3a.multipart.purge.age";
-  public static final long DEFAULT_PURGE_EXISTING_MULTIPART_AGE = 14400;
+  public static final long DEFAULT_PURGE_EXISTING_MULTIPART_AGE = 86400;
 
 
   // s3 server-side encryption
   // s3 server-side encryption
   public static final String SERVER_SIDE_ENCRYPTION_ALGORITHM =
   public static final String SERVER_SIDE_ENCRYPTION_ALGORITHM =
@@ -215,4 +276,10 @@ public final class Constants {
   public static final Class<? extends S3ClientFactory>
   public static final Class<? extends S3ClientFactory>
       DEFAULT_S3_CLIENT_FACTORY_IMPL =
       DEFAULT_S3_CLIENT_FACTORY_IMPL =
           S3ClientFactory.DefaultS3ClientFactory.class;
           S3ClientFactory.DefaultS3ClientFactory.class;
+
+  /**
+   * Maximum number of partitions in a multipart upload: {@value}.
+   */
+  @InterfaceAudience.Private
+  public static final int MAX_MULTIPART_COUNT = 10000;
 }
 }

+ 703 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java

@@ -0,0 +1,703 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import com.amazonaws.AmazonClientException;
+import com.amazonaws.event.ProgressEvent;
+import com.amazonaws.event.ProgressEventType;
+import com.amazonaws.event.ProgressListener;
+import com.amazonaws.services.s3.model.CompleteMultipartUploadResult;
+import com.amazonaws.services.s3.model.PartETag;
+import com.amazonaws.services.s3.model.PutObjectRequest;
+import com.amazonaws.services.s3.model.PutObjectResult;
+import com.amazonaws.services.s3.model.UploadPartRequest;
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.util.Progressable;
+
+import static org.apache.hadoop.fs.s3a.S3AUtils.*;
+import static org.apache.hadoop.fs.s3a.Statistic.*;
+
+/**
+ * Upload files/parts directly via different buffering mechanisms:
+ * including memory and disk.
+ *
+ * If the stream is closed and no update has started, then the upload
+ * is instead done as a single PUT operation.
+ *
+ * Unstable: statistics and error handling might evolve.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+class S3ABlockOutputStream extends OutputStream {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(S3ABlockOutputStream.class);
+
+  /** Owner FileSystem. */
+  private final S3AFileSystem fs;
+
+  /** Object being uploaded. */
+  private final String key;
+
+  /** Size of all blocks. */
+  private final int blockSize;
+
+  /** Callback for progress. */
+  private final ProgressListener progressListener;
+  private final ListeningExecutorService executorService;
+
+  /**
+   * Retry policy for multipart commits; not all AWS SDK versions retry that.
+   */
+  private final RetryPolicy retryPolicy =
+      RetryPolicies.retryUpToMaximumCountWithProportionalSleep(
+          5,
+          2000,
+          TimeUnit.MILLISECONDS);
+  /**
+   * Factory for blocks.
+   */
+  private final S3ADataBlocks.BlockFactory blockFactory;
+
+  /** Preallocated byte buffer for writing single characters. */
+  private final byte[] singleCharWrite = new byte[1];
+
+  /** Multipart upload details; null means none started. */
+  private MultiPartUpload multiPartUpload;
+
+  /** Closed flag. */
+  private final AtomicBoolean closed = new AtomicBoolean(false);
+
+  /** Current data block. Null means none currently active */
+  private S3ADataBlocks.DataBlock activeBlock;
+
+  /** Count of blocks uploaded. */
+  private long blockCount = 0;
+
+  /** Statistics to build up. */
+  private final S3AInstrumentation.OutputStreamStatistics statistics;
+
+  /**
+   * Write operation helper; encapsulation of the filesystem operations.
+   */
+  private final S3AFileSystem.WriteOperationHelper writeOperationHelper;
+
+  /**
+   * An S3A output stream which uploads partitions in a separate pool of
+   * threads; different {@link S3ADataBlocks.BlockFactory}
+   * instances can control where data is buffered.
+   *
+   * @param fs S3AFilesystem
+   * @param key S3 object to work on.
+   * @param executorService the executor service to use to schedule work
+   * @param progress report progress in order to prevent timeouts. If
+   * this object implements {@code ProgressListener} then it will be
+   * directly wired up to the AWS client, so receive detailed progress
+   * information.
+   * @param blockSize size of a single block.
+   * @param blockFactory factory for creating stream destinations
+   * @param statistics stats for this stream
+   * @param writeOperationHelper state of the write operation.
+   * @throws IOException on any problem
+   */
+  S3ABlockOutputStream(S3AFileSystem fs,
+      String key,
+      ExecutorService executorService,
+      Progressable progress,
+      long blockSize,
+      S3ADataBlocks.BlockFactory blockFactory,
+      S3AInstrumentation.OutputStreamStatistics statistics,
+      S3AFileSystem.WriteOperationHelper writeOperationHelper)
+      throws IOException {
+    this.fs = fs;
+    this.key = key;
+    this.blockFactory = blockFactory;
+    this.blockSize = (int) blockSize;
+    this.statistics = statistics;
+    this.writeOperationHelper = writeOperationHelper;
+    Preconditions.checkArgument(blockSize >= Constants.MULTIPART_MIN_SIZE,
+        "Block size is too small: %d", blockSize);
+    this.executorService = MoreExecutors.listeningDecorator(executorService);
+    this.multiPartUpload = null;
+    this.progressListener = (progress instanceof ProgressListener) ?
+        (ProgressListener) progress
+        : new ProgressableListener(progress);
+    // create that first block. This guarantees that an open + close sequence
+    // writes a 0-byte entry.
+    createBlockIfNeeded();
+    LOG.debug("Initialized S3ABlockOutputStream for {}" +
+        " output to {}", writeOperationHelper, activeBlock);
+  }
+
+  /**
+   * Demand create a destination block.
+   * @return the active block; null if there isn't one.
+   * @throws IOException on any failure to create
+   */
+  private synchronized S3ADataBlocks.DataBlock createBlockIfNeeded()
+      throws IOException {
+    if (activeBlock == null) {
+      blockCount++;
+      if (blockCount>= Constants.MAX_MULTIPART_COUNT) {
+        LOG.error("Number of partitions in stream exceeds limit for S3: " +
+             + Constants.MAX_MULTIPART_COUNT +  " write may fail.");
+      }
+      activeBlock = blockFactory.create(this.blockSize);
+    }
+    return activeBlock;
+  }
+
+  /**
+   * Synchronized accessor to the active block.
+   * @return the active block; null if there isn't one.
+   */
+  private synchronized S3ADataBlocks.DataBlock getActiveBlock() {
+    return activeBlock;
+  }
+
+  /**
+   * Predicate to query whether or not there is an active block.
+   * @return true if there is an active block.
+   */
+  private synchronized boolean hasActiveBlock() {
+    return activeBlock != null;
+  }
+
+  /**
+   * Clear the active block.
+   */
+  private void clearActiveBlock() {
+    LOG.debug("Clearing active block");
+    synchronized (this) {
+      activeBlock = null;
+    }
+  }
+
+  /**
+   * Check for the filesystem being open.
+   * @throws IOException if the filesystem is closed.
+   */
+  void checkOpen() throws IOException {
+    if (closed.get()) {
+      throw new IOException("Filesystem " + writeOperationHelper + " closed");
+    }
+  }
+
+  /**
+   * The flush operation does not trigger an upload; that awaits
+   * the next block being full. What it does do is call {@code flush() }
+   * on the current block, leaving it to choose how to react.
+   * @throws IOException Any IO problem.
+   */
+  @Override
+  public synchronized void flush() throws IOException {
+    checkOpen();
+    S3ADataBlocks.DataBlock dataBlock = getActiveBlock();
+    if (dataBlock != null) {
+      dataBlock.flush();
+    }
+  }
+
+  /**
+   * Writes a byte to the destination. If this causes the buffer to reach
+   * its limit, the actual upload is submitted to the threadpool.
+   * @param b the int of which the lowest byte is written
+   * @throws IOException on any problem
+   */
+  @Override
+  public synchronized void write(int b) throws IOException {
+    singleCharWrite[0] = (byte)b;
+    write(singleCharWrite, 0, 1);
+  }
+
+  /**
+   * Writes a range of bytes from to the memory buffer. If this causes the
+   * buffer to reach its limit, the actual upload is submitted to the
+   * threadpool and the remainder of the array is written to memory
+   * (recursively).
+   * @param source byte array containing
+   * @param offset offset in array where to start
+   * @param len number of bytes to be written
+   * @throws IOException on any problem
+   */
+  @Override
+  public synchronized void write(byte[] source, int offset, int len)
+      throws IOException {
+
+    S3ADataBlocks.validateWriteArgs(source, offset, len);
+    checkOpen();
+    if (len == 0) {
+      return;
+    }
+    S3ADataBlocks.DataBlock block = createBlockIfNeeded();
+    int written = block.write(source, offset, len);
+    int remainingCapacity = block.remainingCapacity();
+    if (written < len) {
+      // not everything was written —the block has run out
+      // of capacity
+      // Trigger an upload then process the remainder.
+      LOG.debug("writing more data than block has capacity -triggering upload");
+      uploadCurrentBlock();
+      // tail recursion is mildly expensive, but given buffer sizes must be MB.
+      // it's unlikely to recurse very deeply.
+      this.write(source, offset + written, len - written);
+    } else {
+      if (remainingCapacity == 0) {
+        // the whole buffer is done, trigger an upload
+        uploadCurrentBlock();
+      }
+    }
+  }
+
+  /**
+   * Start an asynchronous upload of the current block.
+   * @throws IOException Problems opening the destination for upload
+   * or initializing the upload.
+   */
+  private synchronized void uploadCurrentBlock() throws IOException {
+    Preconditions.checkState(hasActiveBlock(), "No active block");
+    LOG.debug("Writing block # {}", blockCount);
+    if (multiPartUpload == null) {
+      LOG.debug("Initiating Multipart upload");
+      multiPartUpload = new MultiPartUpload();
+    }
+    try {
+      multiPartUpload.uploadBlockAsync(getActiveBlock());
+    } finally {
+      // set the block to null, so the next write will create a new block.
+      clearActiveBlock();
+    }
+  }
+
+  /**
+   * Close the stream.
+   *
+   * This will not return until the upload is complete
+   * or the attempt to perform the upload has failed.
+   * Exceptions raised in this method are indicative that the write has
+   * failed and data is at risk of being lost.
+   * @throws IOException on any failure.
+   */
+  @Override
+  public void close() throws IOException {
+    if (closed.getAndSet(true)) {
+      // already closed
+      LOG.debug("Ignoring close() as stream is already closed");
+      return;
+    }
+    S3ADataBlocks.DataBlock block = getActiveBlock();
+    boolean hasBlock = hasActiveBlock();
+    LOG.debug("{}: Closing block #{}: current block= {}",
+        this,
+        blockCount,
+        hasBlock ? block : "(none)");
+    try {
+      if (multiPartUpload == null) {
+        if (hasBlock) {
+          // no uploads of data have taken place, put the single block up.
+          // This must happen even if there is no data, so that 0 byte files
+          // are created.
+          putObject();
+        }
+      } else {
+        // there has already been at least one block scheduled for upload;
+        // put up the current then wait
+        if (hasBlock && block.hasData()) {
+          //send last part
+          uploadCurrentBlock();
+        }
+        // wait for the partial uploads to finish
+        final List<PartETag> partETags =
+            multiPartUpload.waitForAllPartUploads();
+        // then complete the operation
+        multiPartUpload.complete(partETags);
+      }
+      LOG.debug("Upload complete for {}", writeOperationHelper);
+    } catch (IOException ioe) {
+      writeOperationHelper.writeFailed(ioe);
+      throw ioe;
+    } finally {
+      LOG.debug("Closing block and factory");
+      IOUtils.closeStream(block);
+      IOUtils.closeStream(blockFactory);
+      LOG.debug("Statistics: {}", statistics);
+      IOUtils.closeStream(statistics);
+      clearActiveBlock();
+    }
+    // All end of write operations, including deleting fake parent directories
+    writeOperationHelper.writeSuccessful();
+  }
+
+  /**
+   * Upload the current block as a single PUT request; if the buffer
+   * is empty a 0-byte PUT will be invoked, as it is needed to create an
+   * entry at the far end.
+   * @throws IOException any problem.
+   */
+  private void putObject() throws IOException {
+    LOG.debug("Executing regular upload for {}", writeOperationHelper);
+
+    final S3ADataBlocks.DataBlock block = getActiveBlock();
+    int size = block.dataSize();
+    final PutObjectRequest putObjectRequest =
+        writeOperationHelper.newPutRequest(
+            block.startUpload(),
+            size);
+    long transferQueueTime = now();
+    BlockUploadProgress callback =
+        new BlockUploadProgress(
+            block, progressListener, transferQueueTime);
+    putObjectRequest.setGeneralProgressListener(callback);
+    statistics.blockUploadQueued(size);
+    ListenableFuture<PutObjectResult> putObjectResult =
+        executorService.submit(new Callable<PutObjectResult>() {
+          @Override
+          public PutObjectResult call() throws Exception {
+            PutObjectResult result = fs.putObjectDirect(putObjectRequest);
+            block.close();
+            return result;
+          }
+        });
+    clearActiveBlock();
+    //wait for completion
+    try {
+      putObjectResult.get();
+    } catch (InterruptedException ie) {
+      LOG.warn("Interrupted object upload", ie);
+      Thread.currentThread().interrupt();
+    } catch (ExecutionException ee) {
+      throw extractException("regular upload", key, ee);
+    }
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "S3ABlockOutputStream{");
+    sb.append(writeOperationHelper.toString());
+    sb.append(", blockSize=").append(blockSize);
+    // unsynced access; risks consistency in exchange for no risk of deadlock.
+    S3ADataBlocks.DataBlock block = activeBlock;
+    if (block != null) {
+      sb.append(", activeBlock=").append(block);
+    }
+    sb.append('}');
+    return sb.toString();
+  }
+
+  private void incrementWriteOperations() {
+    fs.incrementWriteOperations();
+  }
+
+  /**
+   * Current time in milliseconds.
+   * @return time
+   */
+  private long now() {
+    return System.currentTimeMillis();
+  }
+
+  /**
+   * Multiple partition upload.
+   */
+  private class MultiPartUpload {
+    private final String uploadId;
+    private final List<ListenableFuture<PartETag>> partETagsFutures;
+
+    public MultiPartUpload() throws IOException {
+      this.uploadId = writeOperationHelper.initiateMultiPartUpload();
+      this.partETagsFutures = new ArrayList<>(2);
+      LOG.debug("Initiated multi-part upload for {} with " +
+          "id '{}'", writeOperationHelper, uploadId);
+    }
+
+    /**
+     * Upload a block of data.
+     * This will take the block
+     * @param block block to upload
+     * @throws IOException upload failure
+     */
+    private void uploadBlockAsync(final S3ADataBlocks.DataBlock block)
+        throws IOException {
+      LOG.debug("Queueing upload of {}", block);
+      final int size = block.dataSize();
+      final InputStream uploadStream = block.startUpload();
+      final int currentPartNumber = partETagsFutures.size() + 1;
+      final UploadPartRequest request =
+          writeOperationHelper.newUploadPartRequest(
+              uploadId,
+              uploadStream,
+              currentPartNumber,
+              size);
+      long transferQueueTime = now();
+      BlockUploadProgress callback =
+          new BlockUploadProgress(
+              block, progressListener, transferQueueTime);
+      request.setGeneralProgressListener(callback);
+      statistics.blockUploadQueued(block.dataSize());
+      ListenableFuture<PartETag> partETagFuture =
+          executorService.submit(new Callable<PartETag>() {
+            @Override
+            public PartETag call() throws Exception {
+              // this is the queued upload operation
+              LOG.debug("Uploading part {} for id '{}'", currentPartNumber,
+                  uploadId);
+              // do the upload
+              PartETag partETag = fs.uploadPart(request).getPartETag();
+              LOG.debug("Completed upload of {}", block);
+              LOG.debug("Stream statistics of {}", statistics);
+
+              // close the block
+              block.close();
+              return partETag;
+            }
+          });
+      partETagsFutures.add(partETagFuture);
+    }
+
+    /**
+     * Block awaiting all outstanding uploads to complete.
+     * @return list of results
+     * @throws IOException IO Problems
+     */
+    private List<PartETag> waitForAllPartUploads() throws IOException {
+      LOG.debug("Waiting for {} uploads to complete", partETagsFutures.size());
+      try {
+        return Futures.allAsList(partETagsFutures).get();
+      } catch (InterruptedException ie) {
+        LOG.warn("Interrupted partUpload", ie);
+        Thread.currentThread().interrupt();
+        return null;
+      } catch (ExecutionException ee) {
+        //there is no way of recovering so abort
+        //cancel all partUploads
+        LOG.debug("While waiting for upload completion", ee);
+        LOG.debug("Cancelling futures");
+        for (ListenableFuture<PartETag> future : partETagsFutures) {
+          future.cancel(true);
+        }
+        //abort multipartupload
+        this.abort();
+        throw extractException("Multi-part upload with id '" + uploadId
+                + "' to " + key, key, ee);
+      }
+    }
+
+    /**
+     * This completes a multipart upload.
+     * Sometimes it fails; here retries are handled to avoid losing all data
+     * on a transient failure.
+     * @param partETags list of partial uploads
+     * @throws IOException on any problem
+     */
+    private CompleteMultipartUploadResult complete(List<PartETag> partETags)
+        throws IOException {
+      int retryCount = 0;
+      AmazonClientException lastException;
+      String operation =
+          String.format("Completing multi-part upload for key '%s'," +
+                  " id '%s' with %s partitions ",
+              key, uploadId, partETags.size());
+      do {
+        try {
+          LOG.debug(operation);
+          return writeOperationHelper.completeMultipartUpload(
+                  uploadId,
+                  partETags);
+        } catch (AmazonClientException e) {
+          lastException = e;
+          statistics.exceptionInMultipartComplete();
+        }
+      } while (shouldRetry(operation, lastException, retryCount++));
+      // this point is only reached if the operation failed more than
+      // the allowed retry count
+      throw translateException(operation, key, lastException);
+    }
+
+    /**
+     * Abort a multi-part upload. Retries are attempted on failures.
+     * IOExceptions are caught; this is expected to be run as a cleanup process.
+     */
+    public void abort() {
+      int retryCount = 0;
+      AmazonClientException lastException;
+      fs.incrementStatistic(OBJECT_MULTIPART_UPLOAD_ABORTED);
+      String operation =
+          String.format("Aborting multi-part upload for '%s', id '%s",
+              writeOperationHelper, uploadId);
+      do {
+        try {
+          LOG.debug(operation);
+          writeOperationHelper.abortMultipartUpload(uploadId);
+          return;
+        } catch (AmazonClientException e) {
+          lastException = e;
+          statistics.exceptionInMultipartAbort();
+        }
+      } while (shouldRetry(operation, lastException, retryCount++));
+      // this point is only reached if the operation failed more than
+      // the allowed retry count
+      LOG.warn("Unable to abort multipart upload, you may need to purge  " +
+          "uploaded parts", lastException);
+    }
+
+    /**
+     * Predicate to determine whether a failed operation should
+     * be attempted again.
+     * If a retry is advised, the exception is automatically logged and
+     * the filesystem statistic {@link Statistic#IGNORED_ERRORS} incremented.
+     * The method then sleeps for the sleep time suggested by the sleep policy;
+     * if the sleep is interrupted then {@code Thread.interrupted()} is set
+     * to indicate the thread was interrupted; then false is returned.
+     *
+     * @param operation operation for log message
+     * @param e exception raised.
+     * @param retryCount  number of retries already attempted
+     * @return true if another attempt should be made
+     */
+    private boolean shouldRetry(String operation,
+        AmazonClientException e,
+        int retryCount) {
+      try {
+        RetryPolicy.RetryAction retryAction =
+            retryPolicy.shouldRetry(e, retryCount, 0, true);
+        boolean retry = retryAction == RetryPolicy.RetryAction.RETRY;
+        if (retry) {
+          fs.incrementStatistic(IGNORED_ERRORS);
+          LOG.info("Retrying {} after exception ", operation, e);
+          Thread.sleep(retryAction.delayMillis);
+        }
+        return retry;
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+        return false;
+      } catch (Exception ignored) {
+        return false;
+      }
+    }
+
+  }
+
+  /**
+   * The upload progress listener registered for events returned
+   * during the upload of a single block.
+   * It updates statistics and handles the end of the upload.
+   * Transfer failures are logged at WARN.
+   */
+  private final class BlockUploadProgress implements ProgressListener {
+    private final S3ADataBlocks.DataBlock block;
+    private final ProgressListener nextListener;
+    private final long transferQueueTime;
+    private long transferStartTime;
+
+    /**
+     * Track the progress of a single block upload.
+     * @param block block to monitor
+     * @param nextListener optional next progress listener
+     * @param transferQueueTime time the block was transferred
+     * into the queue
+     */
+    private BlockUploadProgress(S3ADataBlocks.DataBlock block,
+        ProgressListener nextListener,
+        long transferQueueTime) {
+      this.block = block;
+      this.transferQueueTime = transferQueueTime;
+      this.nextListener = nextListener;
+    }
+
+    @Override
+    public void progressChanged(ProgressEvent progressEvent) {
+      ProgressEventType eventType = progressEvent.getEventType();
+      long bytesTransferred = progressEvent.getBytesTransferred();
+
+      int size = block.dataSize();
+      switch (eventType) {
+
+      case REQUEST_BYTE_TRANSFER_EVENT:
+        // bytes uploaded
+        statistics.bytesTransferred(bytesTransferred);
+        break;
+
+      case TRANSFER_PART_STARTED_EVENT:
+        transferStartTime = now();
+        statistics.blockUploadStarted(transferStartTime - transferQueueTime,
+            size);
+        incrementWriteOperations();
+        break;
+
+      case TRANSFER_PART_COMPLETED_EVENT:
+        statistics.blockUploadCompleted(now() - transferStartTime, size);
+        break;
+
+      case TRANSFER_PART_FAILED_EVENT:
+        statistics.blockUploadFailed(now() - transferStartTime, size);
+        LOG.warn("Transfer failure of block {}", block);
+        break;
+
+      default:
+        // nothing
+      }
+
+      if (nextListener != null) {
+        nextListener.progressChanged(progressEvent);
+      }
+    }
+  }
+
+  /**
+   * Bridge from AWS {@code ProgressListener} to Hadoop {@link Progressable}.
+   */
+  private static class ProgressableListener implements ProgressListener {
+    private final Progressable progress;
+
+    public ProgressableListener(Progressable progress) {
+      this.progress = progress;
+    }
+
+    public void progressChanged(ProgressEvent progressEvent) {
+      if (progress != null) {
+        progress.progress();
+      }
+    }
+  }
+
+}

+ 821 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java

@@ -0,0 +1,821 @@
+/*
+ * 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.BufferedOutputStream;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.Closeable;
+import java.io.EOFException;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.FilterInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.util.DirectBufferPool;
+
+import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.*;
+
+/**
+ * Set of classes to support output streaming into blocks which are then
+ * uploaded as partitions.
+ */
+final class S3ADataBlocks {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(S3ADataBlocks.class);
+
+  private S3ADataBlocks() {
+  }
+
+  /**
+   * Validate args to a write command. These are the same validation checks
+   * expected for any implementation of {@code OutputStream.write()}.
+   * @param b byte array containing data
+   * @param off offset in array where to start
+   * @param len number of bytes to be written
+   * @throws NullPointerException for a null buffer
+   * @throws IndexOutOfBoundsException if indices are out of range
+   */
+  static void validateWriteArgs(byte[] b, int off, int len)
+      throws IOException {
+    Preconditions.checkNotNull(b);
+    if ((off < 0) || (off > b.length) || (len < 0) ||
+        ((off + len) > b.length) || ((off + len) < 0)) {
+      throw new IndexOutOfBoundsException(
+          "write (b[" + b.length + "], " + off + ", " + len + ')');
+    }
+  }
+
+  /**
+   * Create a factory.
+   * @param owner factory owner
+   * @param name factory name -the option from {@link Constants}.
+   * @return the factory, ready to be initialized.
+   * @throws IllegalArgumentException if the name is unknown.
+   */
+  static BlockFactory createFactory(S3AFileSystem owner,
+      String name) {
+    switch (name) {
+    case Constants.FAST_UPLOAD_BUFFER_ARRAY:
+      return new ArrayBlockFactory(owner);
+    case Constants.FAST_UPLOAD_BUFFER_DISK:
+      return new DiskBlockFactory(owner);
+    case Constants.FAST_UPLOAD_BYTEBUFFER:
+      return new ByteBufferBlockFactory(owner);
+    default:
+      throw new IllegalArgumentException("Unsupported block buffer" +
+          " \"" + name + '"');
+    }
+  }
+
+  /**
+   * Base class for block factories.
+   */
+  static abstract class BlockFactory implements Closeable {
+
+    private final S3AFileSystem owner;
+
+    protected BlockFactory(S3AFileSystem owner) {
+      this.owner = owner;
+    }
+
+
+    /**
+     * Create a block.
+     * @param limit limit of the block.
+     * @return a new block.
+     */
+    abstract DataBlock create(int limit) throws IOException;
+
+    /**
+     * Implement any close/cleanup operation.
+     * Base class is a no-op
+     * @throws IOException -ideally, it shouldn't.
+     */
+    @Override
+    public void close() throws IOException {
+    }
+
+    /**
+     * Owner.
+     */
+    protected S3AFileSystem getOwner() {
+      return owner;
+    }
+  }
+
+  /**
+   * This represents a block being uploaded.
+   */
+  static abstract class DataBlock implements Closeable {
+
+    enum DestState {Writing, Upload, Closed}
+
+    private volatile DestState state = Writing;
+
+    /**
+     * Atomically enter a state, verifying current state.
+     * @param current current state. null means "no check"
+     * @param next next state
+     * @throws IllegalStateException if the current state is not as expected
+     */
+    protected synchronized final void enterState(DestState current,
+        DestState next)
+        throws IllegalStateException {
+      verifyState(current);
+      LOG.debug("{}: entering state {}", this, next);
+      state = next;
+    }
+
+    /**
+     * Verify that the block is in the declared state.
+     * @param expected expected state.
+     * @throws IllegalStateException if the DataBlock is in the wrong state
+     */
+    protected final void verifyState(DestState expected)
+        throws IllegalStateException {
+      if (expected != null && state != expected) {
+        throw new IllegalStateException("Expected stream state " + expected
+            + " -but actual state is " + state + " in " + this);
+      }
+    }
+
+    /**
+     * Current state.
+     * @return the current state.
+     */
+    final DestState getState() {
+      return state;
+    }
+
+    /**
+     * Return the current data size.
+     * @return the size of the data
+     */
+    abstract int dataSize();
+
+    /**
+     * Predicate to verify that the block has the capacity to write
+     * the given set of bytes.
+     * @param bytes number of bytes desired to be written.
+     * @return true if there is enough space.
+     */
+    abstract boolean hasCapacity(long bytes);
+
+    /**
+     * Predicate to check if there is data in the block.
+     * @return true if there is
+     */
+    boolean hasData() {
+      return dataSize() > 0;
+    }
+
+    /**
+     * The remaining capacity in the block before it is full.
+     * @return the number of bytes remaining.
+     */
+    abstract int remainingCapacity();
+
+    /**
+     * Write a series of bytes from the buffer, from the offset.
+     * Returns the number of bytes written.
+     * Only valid in the state {@code Writing}.
+     * Base class verifies the state but does no writing.
+     * @param buffer buffer
+     * @param offset offset
+     * @param length length of write
+     * @return number of bytes written
+     * @throws IOException trouble
+     */
+    int write(byte[] buffer, int offset, int length) throws IOException {
+      verifyState(Writing);
+      Preconditions.checkArgument(buffer != null, "Null buffer");
+      Preconditions.checkArgument(length >= 0, "length is negative");
+      Preconditions.checkArgument(offset >= 0, "offset is negative");
+      Preconditions.checkArgument(
+          !(buffer.length - offset < length),
+          "buffer shorter than amount of data to write");
+      return 0;
+    }
+
+    /**
+     * Flush the output.
+     * Only valid in the state {@code Writing}.
+     * In the base class, this is a no-op
+     * @throws IOException any IO problem.
+     */
+    void flush() throws IOException {
+      verifyState(Writing);
+    }
+
+    /**
+     * Switch to the upload state and return a stream for uploading.
+     * Base class calls {@link #enterState(DestState, DestState)} to
+     * manage the state machine.
+     * @return the stream
+     * @throws IOException trouble
+     */
+    InputStream startUpload() throws IOException {
+      LOG.debug("Start datablock upload");
+      enterState(Writing, Upload);
+      return null;
+    }
+
+    /**
+     * Enter the closed state.
+     * @return true if the class was in any other state, implying that
+     * the subclass should do its close operations
+     */
+    protected synchronized boolean enterClosedState() {
+      if (!state.equals(Closed)) {
+        enterState(null, Closed);
+        return true;
+      } else {
+        return false;
+      }
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (enterClosedState()) {
+        LOG.debug("Closed {}", this);
+        innerClose();
+      }
+    }
+
+    /**
+     * Inner close logic for subclasses to implement.
+     */
+    protected void innerClose() throws IOException {
+
+    }
+
+  }
+
+  // ====================================================================
+
+  /**
+   * Use byte arrays on the heap for storage.
+   */
+  static class ArrayBlockFactory extends BlockFactory {
+
+    ArrayBlockFactory(S3AFileSystem owner) {
+      super(owner);
+    }
+
+    @Override
+    DataBlock create(int limit) throws IOException {
+      return new ByteArrayBlock(limit);
+    }
+
+  }
+
+  /**
+   * Stream to memory via a {@code ByteArrayOutputStream}.
+   *
+   * This was taken from {@code S3AFastOutputStream} and has the
+   * same problem which surfaced there: it can consume a lot of heap space
+   * proportional to the mismatch between writes to the stream and
+   * the JVM-wide upload bandwidth to the S3 endpoint.
+   * The memory consumption can be limited by tuning the filesystem settings
+   * to restrict the number of queued/active uploads.
+   */
+
+  static class ByteArrayBlock extends DataBlock {
+    private ByteArrayOutputStream buffer;
+    private final int limit;
+    // cache data size so that it is consistent after the buffer is reset.
+    private Integer dataSize;
+
+    ByteArrayBlock(int limit) {
+      this.limit = limit;
+      buffer = new ByteArrayOutputStream();
+    }
+
+    /**
+     * Get the amount of data; if there is no buffer then the size is 0.
+     * @return the amount of data available to upload.
+     */
+    @Override
+    int dataSize() {
+      return dataSize != null ? dataSize : buffer.size();
+    }
+
+    @Override
+    InputStream startUpload() throws IOException {
+      super.startUpload();
+      dataSize = buffer.size();
+      ByteArrayInputStream bufferData = new ByteArrayInputStream(
+          buffer.toByteArray());
+      buffer = null;
+      return bufferData;
+    }
+
+    @Override
+    boolean hasCapacity(long bytes) {
+      return dataSize() + bytes <= limit;
+    }
+
+    @Override
+    int remainingCapacity() {
+      return limit - dataSize();
+    }
+
+    @Override
+    int write(byte[] b, int offset, int len) throws IOException {
+      super.write(b, offset, len);
+      int written = Math.min(remainingCapacity(), len);
+      buffer.write(b, offset, written);
+      return written;
+    }
+
+    @Override
+    protected void innerClose() {
+      buffer = null;
+    }
+
+    @Override
+    public String toString() {
+      return "ByteArrayBlock{" +
+          "state=" + getState() +
+          ", limit=" + limit +
+          ", dataSize=" + dataSize +
+          '}';
+    }
+  }
+
+  // ====================================================================
+
+  /**
+   * Stream via Direct ByteBuffers; these are allocated off heap
+   * via {@link DirectBufferPool}.
+   * This is actually the most complex of all the block factories,
+   * due to the need to explicitly recycle buffers; in comparison, the
+   * {@link DiskBlock} buffer delegates the work of deleting files to
+   * the {@link DiskBlock.FileDeletingInputStream}. Here the
+   * input stream {@link ByteBufferInputStream} has a similar task, along
+   * with the foundational work of streaming data from a byte array.
+   */
+
+  static class ByteBufferBlockFactory extends BlockFactory {
+
+    private final DirectBufferPool bufferPool = new DirectBufferPool();
+    private final AtomicInteger buffersOutstanding = new AtomicInteger(0);
+
+    ByteBufferBlockFactory(S3AFileSystem owner) {
+      super(owner);
+    }
+
+    @Override
+    ByteBufferBlock create(int limit) throws IOException {
+      return new ByteBufferBlock(limit);
+    }
+
+    private ByteBuffer requestBuffer(int limit) {
+      LOG.debug("Requesting buffer of size {}", limit);
+      buffersOutstanding.incrementAndGet();
+      return bufferPool.getBuffer(limit);
+    }
+
+    private void releaseBuffer(ByteBuffer buffer) {
+      LOG.debug("Releasing buffer");
+      bufferPool.returnBuffer(buffer);
+      buffersOutstanding.decrementAndGet();
+    }
+
+    /**
+     * Get count of outstanding buffers.
+     * @return the current buffer count
+     */
+    public int getOutstandingBufferCount() {
+      return buffersOutstanding.get();
+    }
+
+    @Override
+    public String toString() {
+      return "ByteBufferBlockFactory{"
+          + "buffersOutstanding=" + buffersOutstanding +
+          '}';
+    }
+
+    /**
+     * A DataBlock which requests a buffer from pool on creation; returns
+     * it when the output stream is closed.
+     */
+    class ByteBufferBlock extends DataBlock {
+      private ByteBuffer buffer;
+      private final int bufferSize;
+      // cache data size so that it is consistent after the buffer is reset.
+      private Integer dataSize;
+
+      /**
+       * Instantiate. This will request a ByteBuffer of the desired size.
+       * @param bufferSize buffer size
+       */
+      ByteBufferBlock(int bufferSize) {
+        this.bufferSize = bufferSize;
+        buffer = requestBuffer(bufferSize);
+      }
+
+      /**
+       * Get the amount of data; if there is no buffer then the size is 0.
+       * @return the amount of data available to upload.
+       */
+      @Override
+      int dataSize() {
+        return dataSize != null ? dataSize : bufferCapacityUsed();
+      }
+
+      @Override
+      ByteBufferInputStream startUpload() throws IOException {
+        super.startUpload();
+        dataSize = bufferCapacityUsed();
+        // set the buffer up from reading from the beginning
+        buffer.limit(buffer.position());
+        buffer.position(0);
+        return new ByteBufferInputStream(dataSize, buffer);
+      }
+
+      @Override
+      public boolean hasCapacity(long bytes) {
+        return bytes <= remainingCapacity();
+      }
+
+      @Override
+      public int remainingCapacity() {
+        return buffer != null ? buffer.remaining() : 0;
+      }
+
+      private int bufferCapacityUsed() {
+        return buffer.capacity() - buffer.remaining();
+      }
+
+      @Override
+      int write(byte[] b, int offset, int len) throws IOException {
+        super.write(b, offset, len);
+        int written = Math.min(remainingCapacity(), len);
+        buffer.put(b, offset, written);
+        return written;
+      }
+
+      @Override
+      protected void innerClose() {
+        buffer = null;
+      }
+
+      @Override
+      public String toString() {
+        return "ByteBufferBlock{"
+            + "state=" + getState() +
+            ", dataSize=" + dataSize() +
+            ", limit=" + bufferSize +
+            ", remainingCapacity=" + remainingCapacity() +
+            '}';
+      }
+
+    }
+
+    /**
+     * Provide an input stream from a byte buffer; supporting
+     * {@link #mark(int)}, which is required to enable replay of failed
+     * PUT attempts.
+     * This input stream returns the buffer to the pool afterwards.
+     */
+    class ByteBufferInputStream extends InputStream {
+
+      private final int size;
+      private ByteBuffer byteBuffer;
+
+      ByteBufferInputStream(int size, ByteBuffer byteBuffer) {
+        LOG.debug("Creating ByteBufferInputStream of size {}", size);
+        this.size = size;
+        this.byteBuffer = byteBuffer;
+      }
+
+      /**
+       * Return the buffer to the pool after the stream is closed.
+       */
+      @Override
+      public synchronized void close() {
+        if (byteBuffer != null) {
+          LOG.debug("releasing buffer");
+          releaseBuffer(byteBuffer);
+          byteBuffer = null;
+        }
+      }
+
+      /**
+       * Verify that the stream is open.
+       * @throws IOException if the stream is closed
+       */
+      private void verifyOpen() throws IOException {
+        if (byteBuffer == null) {
+          throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
+        }
+      }
+
+      public synchronized int read() throws IOException {
+        if (available() > 0) {
+          return byteBuffer.get() & 0xFF;
+        } else {
+          return -1;
+        }
+      }
+
+      @Override
+      public synchronized long skip(long offset) throws IOException {
+        verifyOpen();
+        long newPos = position() + offset;
+        if (newPos < 0) {
+          throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK);
+        }
+        if (newPos > size) {
+          throw new EOFException(FSExceptionMessages.CANNOT_SEEK_PAST_EOF);
+        }
+        byteBuffer.position((int) newPos);
+        return newPos;
+      }
+
+      @Override
+      public synchronized int available() {
+        Preconditions.checkState(byteBuffer != null,
+            FSExceptionMessages.STREAM_IS_CLOSED);
+        return byteBuffer.remaining();
+      }
+
+      /**
+       * Get the current buffer position.
+       * @return the buffer position
+       */
+      public synchronized int position() {
+        return byteBuffer.position();
+      }
+
+      /**
+       * Check if there is data left.
+       * @return true if there is data remaining in the buffer.
+       */
+      public synchronized boolean hasRemaining() {
+        return byteBuffer.hasRemaining();
+      }
+
+      @Override
+      public synchronized void mark(int readlimit) {
+        LOG.debug("mark at {}", position());
+        byteBuffer.mark();
+      }
+
+      @Override
+      public synchronized void reset() throws IOException {
+        LOG.debug("reset");
+        byteBuffer.reset();
+      }
+
+      @Override
+      public boolean markSupported() {
+        return true;
+      }
+
+      /**
+       * Read in data.
+       * @param buffer destination buffer
+       * @param offset offset within the buffer
+       * @param length length of bytes to read
+       * @throws EOFException if the position is negative
+       * @throws IndexOutOfBoundsException if there isn't space for the
+       * amount of data requested.
+       * @throws IllegalArgumentException other arguments are invalid.
+       */
+      @SuppressWarnings("NullableProblems")
+      public synchronized int read(byte[] buffer, int offset, int length)
+          throws IOException {
+        Preconditions.checkArgument(length >= 0, "length is negative");
+        Preconditions.checkArgument(buffer != null, "Null buffer");
+        if (buffer.length - offset < length) {
+          throw new IndexOutOfBoundsException(
+              FSExceptionMessages.TOO_MANY_BYTES_FOR_DEST_BUFFER
+                  + ": request length =" + length
+                  + ", with offset =" + offset
+                  + "; buffer capacity =" + (buffer.length - offset));
+        }
+        verifyOpen();
+        if (!hasRemaining()) {
+          return -1;
+        }
+
+        int toRead = Math.min(length, available());
+        byteBuffer.get(buffer, offset, toRead);
+        return toRead;
+      }
+
+      @Override
+      public String toString() {
+        final StringBuilder sb = new StringBuilder(
+            "ByteBufferInputStream{");
+        sb.append("size=").append(size);
+        ByteBuffer buffer = this.byteBuffer;
+        if (buffer != null) {
+          sb.append(", available=").append(buffer.remaining());
+        }
+        sb.append('}');
+        return sb.toString();
+      }
+    }
+  }
+
+  // ====================================================================
+
+  /**
+   * Buffer blocks to disk.
+   */
+  static class DiskBlockFactory extends BlockFactory {
+
+    DiskBlockFactory(S3AFileSystem owner) {
+      super(owner);
+    }
+
+    /**
+     * Create a temp file and a block which writes to it.
+     * @param limit limit of the block.
+     * @return the new block
+     * @throws IOException IO problems
+     */
+    @Override
+    DataBlock create(int limit) throws IOException {
+      File destFile = getOwner()
+          .createTmpFileForWrite("s3ablock", limit, getOwner().getConf());
+      return new DiskBlock(destFile, limit);
+    }
+  }
+
+  /**
+   * Stream to a file.
+   * This will stop at the limit; the caller is expected to create a new block
+   */
+  static class DiskBlock extends DataBlock {
+
+    private int bytesWritten;
+    private final File bufferFile;
+    private final int limit;
+    private BufferedOutputStream out;
+    private InputStream uploadStream;
+
+    DiskBlock(File bufferFile, int limit)
+        throws FileNotFoundException {
+      this.limit = limit;
+      this.bufferFile = bufferFile;
+      out = new BufferedOutputStream(new FileOutputStream(bufferFile));
+    }
+
+    @Override
+    int dataSize() {
+      return bytesWritten;
+    }
+
+    @Override
+    boolean hasCapacity(long bytes) {
+      return dataSize() + bytes <= limit;
+    }
+
+    @Override
+    int remainingCapacity() {
+      return limit - bytesWritten;
+    }
+
+    @Override
+    int write(byte[] b, int offset, int len) throws IOException {
+      super.write(b, offset, len);
+      int written = Math.min(remainingCapacity(), len);
+      out.write(b, offset, written);
+      bytesWritten += written;
+      return written;
+    }
+
+    @Override
+    InputStream startUpload() throws IOException {
+      super.startUpload();
+      try {
+        out.flush();
+      } finally {
+        out.close();
+        out = null;
+      }
+      uploadStream = new FileInputStream(bufferFile);
+      return new FileDeletingInputStream(uploadStream);
+    }
+
+    /**
+     * The close operation will delete the destination file if it still
+     * exists.
+     * @throws IOException IO problems
+     */
+    @Override
+    protected void innerClose() throws IOException {
+      final DestState state = getState();
+      LOG.debug("Closing {}", this);
+      switch (state) {
+      case Writing:
+        if (bufferFile.exists()) {
+          // file was not uploaded
+          LOG.debug("Deleting buffer file as upload did not start");
+          boolean deleted = bufferFile.delete();
+          if (!deleted && bufferFile.exists()) {
+            LOG.warn("Failed to delete buffer file {}", bufferFile);
+          }
+        }
+        break;
+
+      case Upload:
+        LOG.debug("Buffer file {} exists —close upload stream", bufferFile);
+        break;
+
+      case Closed:
+        // no-op
+        break;
+
+      default:
+        // this state can never be reached, but checkstyle complains, so
+        // it is here.
+      }
+    }
+
+    /**
+     * Flush operation will flush to disk.
+     * @throws IOException IOE raised on FileOutputStream
+     */
+    @Override
+    void flush() throws IOException {
+      super.flush();
+      out.flush();
+    }
+
+    @Override
+    public String toString() {
+      String sb = "FileBlock{"
+          + "destFile=" + bufferFile +
+          ", state=" + getState() +
+          ", dataSize=" + dataSize() +
+          ", limit=" + limit +
+          '}';
+      return sb;
+    }
+
+    /**
+     * An input stream which deletes the buffer file when closed.
+     */
+    private final class FileDeletingInputStream extends FilterInputStream {
+      private final AtomicBoolean closed = new AtomicBoolean(false);
+
+      FileDeletingInputStream(InputStream source) {
+        super(source);
+      }
+
+      /**
+       * Delete the input file when closed.
+       * @throws IOException IO problem
+       */
+      @Override
+      public void close() throws IOException {
+        try {
+          super.close();
+        } finally {
+          if (!closed.getAndSet(true)) {
+            if (!bufferFile.delete()) {
+              LOG.warn("delete({}) returned false",
+                  bufferFile.getAbsoluteFile());
+            }
+          }
+        }
+      }
+    }
+  }
+
+}

+ 0 - 410
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java

@@ -1,410 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.s3a;
-
-import com.amazonaws.AmazonClientException;
-import com.amazonaws.event.ProgressEvent;
-import com.amazonaws.event.ProgressListener;
-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;
-import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
-import com.amazonaws.services.s3.model.ObjectMetadata;
-import com.amazonaws.services.s3.model.PartETag;
-import com.amazonaws.services.s3.model.PutObjectRequest;
-import com.amazonaws.services.s3.model.PutObjectResult;
-import com.amazonaws.services.s3.model.UploadPartRequest;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.ListeningExecutorService;
-import com.google.common.util.concurrent.MoreExecutors;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.util.Progressable;
-import org.slf4j.Logger;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.List;
-
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-
-import static org.apache.hadoop.fs.s3a.S3AUtils.*;
-import static org.apache.hadoop.fs.s3a.Statistic.*;
-
-/**
- * Upload files/parts asap directly from a memory buffer (instead of buffering
- * to a file).
- * <p>
- * Uploads are managed low-level rather than through the AWS TransferManager.
- * This allows for uploading each part of a multi-part upload as soon as
- * the bytes are in memory, rather than waiting until the file is closed.
- * <p>
- * Unstable: statistics and error handling might evolve
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class S3AFastOutputStream extends OutputStream {
-
-  private static final Logger LOG = S3AFileSystem.LOG;
-  private final String key;
-  private final String bucket;
-  private final AmazonS3 client;
-  private final int partSize;
-  private final int multiPartThreshold;
-  private final S3AFileSystem fs;
-  private final CannedAccessControlList cannedACL;
-  private final ProgressListener progressListener;
-  private final ListeningExecutorService executorService;
-  private MultiPartUpload multiPartUpload;
-  private boolean closed;
-  private ByteArrayOutputStream buffer;
-  private int bufferLimit;
-
-
-  /**
-   * Creates a fast OutputStream that uploads to S3 from memory.
-   * For MultiPartUploads, as soon as sufficient bytes have been written to
-   * the stream a part is uploaded immediately (by using the low-level
-   * multi-part upload API on the AmazonS3Client).
-   *
-   * @param client AmazonS3Client used for S3 calls
-   * @param fs S3AFilesystem
-   * @param bucket S3 bucket name
-   * @param key S3 key name
-   * @param progress report progress in order to prevent timeouts
-   * @param cannedACL used CannedAccessControlList
-   * @param partSize size of a single part in a multi-part upload (except
-   * last part)
-   * @param multiPartThreshold files at least this size use multi-part upload
-   * @param threadPoolExecutor thread factory
-   * @throws IOException on any problem
-   */
-  public S3AFastOutputStream(AmazonS3 client,
-      S3AFileSystem fs,
-      String bucket,
-      String key,
-      Progressable progress,
-      CannedAccessControlList cannedACL,
-      long partSize,
-      long multiPartThreshold,
-      ExecutorService threadPoolExecutor)
-      throws IOException {
-    this.bucket = bucket;
-    this.key = key;
-    this.client = client;
-    this.fs = fs;
-    this.cannedACL = cannedACL;
-    //Ensure limit as ByteArrayOutputStream size cannot exceed Integer.MAX_VALUE
-    if (partSize > Integer.MAX_VALUE) {
-      this.partSize = Integer.MAX_VALUE;
-      LOG.warn("s3a: MULTIPART_SIZE capped to ~2.14GB (maximum allowed size " +
-          "when using 'FAST_UPLOAD = true')");
-    } else {
-      this.partSize = (int) partSize;
-    }
-    if (multiPartThreshold > Integer.MAX_VALUE) {
-      this.multiPartThreshold = Integer.MAX_VALUE;
-      LOG.warn("s3a: MIN_MULTIPART_THRESHOLD capped to ~2.14GB (maximum " +
-          "allowed size when using 'FAST_UPLOAD = true')");
-    } else {
-      this.multiPartThreshold = (int) multiPartThreshold;
-    }
-    this.bufferLimit = this.multiPartThreshold;
-    this.closed = false;
-    int initialBufferSize = this.fs.getConf()
-        .getInt(Constants.FAST_BUFFER_SIZE, Constants.DEFAULT_FAST_BUFFER_SIZE);
-    if (initialBufferSize < 0) {
-      LOG.warn("s3a: FAST_BUFFER_SIZE should be a positive number. Using " +
-          "default value");
-      initialBufferSize = Constants.DEFAULT_FAST_BUFFER_SIZE;
-    } else if (initialBufferSize > this.bufferLimit) {
-      LOG.warn("s3a: automatically adjusting FAST_BUFFER_SIZE to not " +
-          "exceed MIN_MULTIPART_THRESHOLD");
-      initialBufferSize = this.bufferLimit;
-    }
-    this.buffer = new ByteArrayOutputStream(initialBufferSize);
-    this.executorService = MoreExecutors.listeningDecorator(threadPoolExecutor);
-    this.multiPartUpload = null;
-    this.progressListener = new ProgressableListener(progress);
-    LOG.debug("Initialized S3AFastOutputStream for bucket '{}' key '{}'",
-        bucket, key);
-  }
-
-  /**
-   * Writes a byte to the memory buffer. If this causes the buffer to reach
-   * its limit, the actual upload is submitted to the threadpool.
-   * @param b the int of which the lowest byte is written
-   * @throws IOException on any problem
-   */
-  @Override
-  public synchronized void write(int b) throws IOException {
-    buffer.write(b);
-    if (buffer.size() == bufferLimit) {
-      uploadBuffer();
-    }
-  }
-
-  /**
-   * Writes a range of bytes from to the memory buffer. If this causes the
-   * buffer to reach its limit, the actual upload is submitted to the
-   * threadpool and the remainder of the array is written to memory
-   * (recursively).
-   * @param b byte array containing
-   * @param off offset in array where to start
-   * @param len number of bytes to be written
-   * @throws IOException on any problem
-   */
-  @Override
-  public synchronized void write(byte[] b, int off, int len)
-      throws IOException {
-    if (b == null) {
-      throw new NullPointerException();
-    } else if ((off < 0) || (off > b.length) || (len < 0) ||
-        ((off + len) > b.length) || ((off + len) < 0)) {
-      throw new IndexOutOfBoundsException();
-    } else if (len == 0) {
-      return;
-    }
-    if (buffer.size() + len < bufferLimit) {
-      buffer.write(b, off, len);
-    } else {
-      int firstPart = bufferLimit - buffer.size();
-      buffer.write(b, off, firstPart);
-      uploadBuffer();
-      this.write(b, off + firstPart, len - firstPart);
-    }
-  }
-
-  private synchronized void uploadBuffer() throws IOException {
-    if (multiPartUpload == null) {
-      multiPartUpload = initiateMultiPartUpload();
-       /* Upload the existing buffer if it exceeds partSize. This possibly
-       requires multiple parts! */
-      final byte[] allBytes = buffer.toByteArray();
-      buffer = null; //earlier gc?
-      LOG.debug("Total length of initial buffer: {}", allBytes.length);
-      int processedPos = 0;
-      while ((multiPartThreshold - processedPos) >= partSize) {
-        LOG.debug("Initial buffer: processing from byte {} to byte {}",
-            processedPos, (processedPos + partSize - 1));
-        multiPartUpload.uploadPartAsync(new ByteArrayInputStream(allBytes,
-            processedPos, partSize), partSize);
-        processedPos += partSize;
-      }
-      //resize and reset stream
-      bufferLimit = partSize;
-      buffer = new ByteArrayOutputStream(bufferLimit);
-      buffer.write(allBytes, processedPos, multiPartThreshold - processedPos);
-    } else {
-      //upload next part
-      multiPartUpload.uploadPartAsync(new ByteArrayInputStream(buffer
-          .toByteArray()), partSize);
-      buffer.reset();
-    }
-  }
-
-  /**
-   * Close the stream. This will not return until the upload is complete
-   * or the attempt to perform the upload has failed.
-   * Exceptions raised in this method are indicative that the write has
-   * failed and data is at risk of being lost.
-   * @throws IOException on any failure.
-   */
-  @Override
-  public synchronized void close() throws IOException {
-    if (closed) {
-      return;
-    }
-    closed = true;
-    try {
-      if (multiPartUpload == null) {
-        putObject();
-      } else {
-        int size = buffer.size();
-        if (size > 0) {
-          fs.incrementPutStartStatistics(size);
-          //send last part
-          multiPartUpload.uploadPartAsync(new ByteArrayInputStream(buffer
-              .toByteArray()), size);
-        }
-        final List<PartETag> partETags = multiPartUpload
-            .waitForAllPartUploads();
-        multiPartUpload.complete(partETags);
-      }
-      // This will delete unnecessary fake parent directories
-      fs.finishedWrite(key);
-      LOG.debug("Upload complete for bucket '{}' key '{}'", bucket, key);
-    } finally {
-      buffer = null;
-      super.close();
-    }
-  }
-
-  /**
-   * Create the default metadata for a multipart upload operation.
-   * @return the metadata to use/extend.
-   */
-  private ObjectMetadata createDefaultMetadata() {
-    return fs.newObjectMetadata();
-  }
-
-  private MultiPartUpload initiateMultiPartUpload() throws IOException {
-    final InitiateMultipartUploadRequest initiateMPURequest =
-        new InitiateMultipartUploadRequest(bucket,
-            key,
-            createDefaultMetadata());
-    initiateMPURequest.setCannedACL(cannedACL);
-    try {
-      return new MultiPartUpload(
-          client.initiateMultipartUpload(initiateMPURequest).getUploadId());
-    } catch (AmazonClientException ace) {
-      throw translateException("initiate MultiPartUpload", key, ace);
-    }
-  }
-
-  private void putObject() throws IOException {
-    LOG.debug("Executing regular upload for bucket '{}' key '{}'",
-        bucket, key);
-    final ObjectMetadata om = createDefaultMetadata();
-    final int size = buffer.size();
-    om.setContentLength(size);
-    final PutObjectRequest putObjectRequest =
-        fs.newPutObjectRequest(key,
-            om,
-            new ByteArrayInputStream(buffer.toByteArray()));
-    putObjectRequest.setGeneralProgressListener(progressListener);
-    ListenableFuture<PutObjectResult> putObjectResult =
-        executorService.submit(new Callable<PutObjectResult>() {
-          @Override
-          public PutObjectResult call() throws Exception {
-            fs.incrementPutStartStatistics(size);
-            return client.putObject(putObjectRequest);
-          }
-        });
-    //wait for completion
-    try {
-      putObjectResult.get();
-    } catch (InterruptedException ie) {
-      LOG.warn("Interrupted object upload: {}", ie, ie);
-      Thread.currentThread().interrupt();
-    } catch (ExecutionException ee) {
-      throw extractException("regular upload", key, ee);
-    }
-  }
-
-
-  private class MultiPartUpload {
-    private final String uploadId;
-    private final List<ListenableFuture<PartETag>> partETagsFutures;
-
-    public MultiPartUpload(String uploadId) {
-      this.uploadId = uploadId;
-      this.partETagsFutures = new ArrayList<ListenableFuture<PartETag>>();
-      LOG.debug("Initiated multi-part upload for bucket '{}' key '{}' with " +
-          "id '{}'", bucket, key, uploadId);
-    }
-
-    private void uploadPartAsync(ByteArrayInputStream inputStream,
-        int partSize) {
-      final int currentPartNumber = partETagsFutures.size() + 1;
-      final UploadPartRequest request =
-          new UploadPartRequest().withBucketName(bucket).withKey(key)
-              .withUploadId(uploadId).withInputStream(inputStream)
-              .withPartNumber(currentPartNumber).withPartSize(partSize);
-      request.setGeneralProgressListener(progressListener);
-      ListenableFuture<PartETag> partETagFuture =
-          executorService.submit(new Callable<PartETag>() {
-            @Override
-            public PartETag call() throws Exception {
-              LOG.debug("Uploading part {} for id '{}'", currentPartNumber,
-                  uploadId);
-              return fs.uploadPart(request).getPartETag();
-            }
-          });
-      partETagsFutures.add(partETagFuture);
-    }
-
-    private List<PartETag> waitForAllPartUploads() throws IOException {
-      try {
-        return Futures.allAsList(partETagsFutures).get();
-      } catch (InterruptedException ie) {
-        LOG.warn("Interrupted partUpload: {}", ie, ie);
-        Thread.currentThread().interrupt();
-        return null;
-      } catch (ExecutionException ee) {
-        //there is no way of recovering so abort
-        //cancel all partUploads
-        for (ListenableFuture<PartETag> future : partETagsFutures) {
-          future.cancel(true);
-        }
-        //abort multipartupload
-        this.abort();
-        throw extractException("Multi-part upload with id '" + uploadId + "'",
-            key, ee);
-      }
-    }
-
-    private void complete(List<PartETag> partETags) throws IOException {
-      try {
-        LOG.debug("Completing multi-part upload for key '{}', id '{}'",
-            key, uploadId);
-        client.completeMultipartUpload(
-            new CompleteMultipartUploadRequest(bucket,
-                key,
-                uploadId,
-                partETags));
-      } catch (AmazonClientException e) {
-        throw translateException("Completing multi-part upload", key, e);
-      }
-    }
-
-    public void abort() {
-      LOG.warn("Aborting multi-part upload with id '{}'", uploadId);
-      try {
-        fs.incrementStatistic(OBJECT_MULTIPART_UPLOAD_ABORTED);
-        client.abortMultipartUpload(new AbortMultipartUploadRequest(bucket,
-            key, uploadId));
-      } catch (Exception e2) {
-        LOG.warn("Unable to abort multipart upload, you may need to purge  " +
-            "uploaded parts: {}", e2, e2);
-      }
-    }
-  }
-
-  private static class ProgressableListener implements ProgressListener {
-    private final Progressable progress;
-
-    public ProgressableListener(Progressable progress) {
-      this.progress = progress;
-    }
-
-    public void progressChanged(ProgressEvent progressEvent) {
-      if (progress != null) {
-        progress.progress();
-      }
-    }
-  }
-}

+ 361 - 47
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java

@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
  * distributed with this work for additional information
@@ -37,14 +37,20 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import com.amazonaws.AmazonClientException;
 import com.amazonaws.AmazonClientException;
 import com.amazonaws.AmazonServiceException;
 import com.amazonaws.AmazonServiceException;
 import com.amazonaws.services.s3.AmazonS3;
 import com.amazonaws.services.s3.AmazonS3;
+import com.amazonaws.services.s3.model.AbortMultipartUploadRequest;
 import com.amazonaws.services.s3.model.AmazonS3Exception;
 import com.amazonaws.services.s3.model.AmazonS3Exception;
 import com.amazonaws.services.s3.model.CannedAccessControlList;
 import com.amazonaws.services.s3.model.CannedAccessControlList;
+import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest;
+import com.amazonaws.services.s3.model.CompleteMultipartUploadResult;
+import com.amazonaws.services.s3.model.CopyObjectRequest;
 import com.amazonaws.services.s3.model.DeleteObjectsRequest;
 import com.amazonaws.services.s3.model.DeleteObjectsRequest;
+import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
 import com.amazonaws.services.s3.model.ListObjectsRequest;
 import com.amazonaws.services.s3.model.ListObjectsRequest;
 import com.amazonaws.services.s3.model.ObjectListing;
 import com.amazonaws.services.s3.model.ObjectListing;
 import com.amazonaws.services.s3.model.ObjectMetadata;
 import com.amazonaws.services.s3.model.ObjectMetadata;
+import com.amazonaws.services.s3.model.PartETag;
 import com.amazonaws.services.s3.model.PutObjectRequest;
 import com.amazonaws.services.s3.model.PutObjectRequest;
-import com.amazonaws.services.s3.model.CopyObjectRequest;
+import com.amazonaws.services.s3.model.PutObjectResult;
 import com.amazonaws.services.s3.model.S3ObjectSummary;
 import com.amazonaws.services.s3.model.S3ObjectSummary;
 import com.amazonaws.services.s3.model.UploadPartRequest;
 import com.amazonaws.services.s3.model.UploadPartRequest;
 import com.amazonaws.services.s3.model.UploadPartResult;
 import com.amazonaws.services.s3.model.UploadPartResult;
@@ -55,6 +61,8 @@ import com.amazonaws.services.s3.transfer.Upload;
 import com.amazonaws.event.ProgressListener;
 import com.amazonaws.event.ProgressListener;
 import com.amazonaws.event.ProgressEvent;
 import com.amazonaws.event.ProgressEvent;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.ListeningExecutorService;
 
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -68,6 +76,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.GlobalStorageStatistics;
 import org.apache.hadoop.fs.GlobalStorageStatistics;
 import org.apache.hadoop.fs.InvalidRequestException;
 import org.apache.hadoop.fs.InvalidRequestException;
+import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
@@ -118,9 +127,12 @@ public class S3AFileSystem extends FileSystem {
   private long partSize;
   private long partSize;
   private boolean enableMultiObjectsDelete;
   private boolean enableMultiObjectsDelete;
   private TransferManager transfers;
   private TransferManager transfers;
-  private ExecutorService threadPoolExecutor;
+  private ListeningExecutorService threadPoolExecutor;
   private long multiPartThreshold;
   private long multiPartThreshold;
   public static final Logger LOG = LoggerFactory.getLogger(S3AFileSystem.class);
   public static final Logger LOG = LoggerFactory.getLogger(S3AFileSystem.class);
+  private static final Logger PROGRESS =
+      LoggerFactory.getLogger("org.apache.hadoop.fs.s3a.S3AFileSystem.Progress");
+  private LocalDirAllocator directoryAllocator;
   private CannedAccessControlList cannedACL;
   private CannedAccessControlList cannedACL;
   private String serverSideEncryptionAlgorithm;
   private String serverSideEncryptionAlgorithm;
   private S3AInstrumentation instrumentation;
   private S3AInstrumentation instrumentation;
@@ -131,6 +143,10 @@ public class S3AFileSystem extends FileSystem {
 
 
   // The maximum number of entries that can be deleted in any call to s3
   // The maximum number of entries that can be deleted in any call to s3
   private static final int MAX_ENTRIES_TO_DELETE = 1000;
   private static final int MAX_ENTRIES_TO_DELETE = 1000;
+  private boolean blockUploadEnabled;
+  private String blockOutputBuffer;
+  private S3ADataBlocks.BlockFactory blockFactory;
+  private int blockOutputActiveBlocks;
 
 
   /** Called after a new FileSystem instance is constructed.
   /** Called after a new FileSystem instance is constructed.
    * @param name a uri whose authority section names the host, port, etc.
    * @param name a uri whose authority section names the host, port, etc.
@@ -157,18 +173,11 @@ public class S3AFileSystem extends FileSystem {
 
 
       maxKeys = intOption(conf, MAX_PAGING_KEYS, DEFAULT_MAX_PAGING_KEYS, 1);
       maxKeys = intOption(conf, MAX_PAGING_KEYS, DEFAULT_MAX_PAGING_KEYS, 1);
       listing = new Listing(this);
       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");
-        partSize = 5 * 1024 * 1024;
-      }
+      partSize = getMultipartSizeProperty(conf,
+          MULTIPART_SIZE, DEFAULT_MULTIPART_SIZE);
+      multiPartThreshold = getMultipartSizeProperty(conf,
+          MIN_MULTIPART_THRESHOLD, DEFAULT_MIN_MULTIPART_THRESHOLD);
 
 
-      multiPartThreshold = conf.getLong(MIN_MULTIPART_THRESHOLD,
-          DEFAULT_MIN_MULTIPART_THRESHOLD);
-      if (multiPartThreshold < 5 * 1024 * 1024) {
-        LOG.error(MIN_MULTIPART_THRESHOLD + " must be at least 5 MB");
-        multiPartThreshold = 5 * 1024 * 1024;
-      }
       //check but do not store the block size
       //check but do not store the block size
       longOption(conf, FS_S3A_BLOCK_SIZE, DEFAULT_BLOCKSIZE, 1);
       longOption(conf, FS_S3A_BLOCK_SIZE, DEFAULT_BLOCKSIZE, 1);
       enableMultiObjectsDelete = conf.getBoolean(ENABLE_MULTI_DELETE, true);
       enableMultiObjectsDelete = conf.getBoolean(ENABLE_MULTI_DELETE, true);
@@ -189,14 +198,14 @@ public class S3AFileSystem extends FileSystem {
         LOG.warn(MAX_THREADS + " must be at least 2: forcing to 2.");
         LOG.warn(MAX_THREADS + " must be at least 2: forcing to 2.");
         maxThreads = 2;
         maxThreads = 2;
       }
       }
-      int totalTasks = conf.getInt(MAX_TOTAL_TASKS, DEFAULT_MAX_TOTAL_TASKS);
-      if (totalTasks < 1) {
-        LOG.warn(MAX_TOTAL_TASKS + "must be at least 1: forcing to 1.");
-        totalTasks = 1;
-      }
-      long keepAliveTime = conf.getLong(KEEPALIVE_TIME, DEFAULT_KEEPALIVE_TIME);
-      threadPoolExecutor = new BlockingThreadPoolExecutorService(maxThreads,
-          maxThreads + totalTasks, keepAliveTime, TimeUnit.SECONDS,
+      int totalTasks = intOption(conf,
+          MAX_TOTAL_TASKS, DEFAULT_MAX_TOTAL_TASKS, 1);
+      long keepAliveTime = longOption(conf, KEEPALIVE_TIME,
+          DEFAULT_KEEPALIVE_TIME, 0);
+      threadPoolExecutor = BlockingThreadPoolExecutorService.newInstance(
+          maxThreads,
+          maxThreads + totalTasks,
+          keepAliveTime, TimeUnit.SECONDS,
           "s3a-transfer-shared");
           "s3a-transfer-shared");
 
 
       initTransferManager();
       initTransferManager();
@@ -209,8 +218,25 @@ public class S3AFileSystem extends FileSystem {
 
 
       serverSideEncryptionAlgorithm =
       serverSideEncryptionAlgorithm =
           conf.getTrimmed(SERVER_SIDE_ENCRYPTION_ALGORITHM);
           conf.getTrimmed(SERVER_SIDE_ENCRYPTION_ALGORITHM);
+      LOG.debug("Using encryption {}", serverSideEncryptionAlgorithm);
       inputPolicy = S3AInputPolicy.getPolicy(
       inputPolicy = S3AInputPolicy.getPolicy(
           conf.getTrimmed(INPUT_FADVISE, INPUT_FADV_NORMAL));
           conf.getTrimmed(INPUT_FADVISE, INPUT_FADV_NORMAL));
+
+      blockUploadEnabled = conf.getBoolean(FAST_UPLOAD, DEFAULT_FAST_UPLOAD);
+
+      if (blockUploadEnabled) {
+        blockOutputBuffer = conf.getTrimmed(FAST_UPLOAD_BUFFER,
+            DEFAULT_FAST_UPLOAD_BUFFER);
+        partSize = ensureOutputParameterInRange(MULTIPART_SIZE, partSize);
+        blockFactory = S3ADataBlocks.createFactory(this, blockOutputBuffer);
+        blockOutputActiveBlocks = intOption(conf,
+            FAST_UPLOAD_ACTIVE_BLOCKS, DEFAULT_FAST_UPLOAD_ACTIVE_BLOCKS, 1);
+        LOG.debug("Using S3ABlockOutputStream with buffer = {}; block={};" +
+                " queue limit={}",
+            blockOutputBuffer, partSize, blockOutputActiveBlocks);
+      } else {
+        LOG.debug("Using S3AOutputStream");
+      }
     } catch (AmazonClientException e) {
     } catch (AmazonClientException e) {
       throw translateException("initializing ", new Path(name), e);
       throw translateException("initializing ", new Path(name), e);
     }
     }
@@ -336,6 +362,33 @@ public class S3AFileSystem extends FileSystem {
     return inputPolicy;
     return inputPolicy;
   }
   }
 
 
+  /**
+   * Demand create the directory allocator, then create a temporary file.
+   * {@link LocalDirAllocator#createTmpFileForWrite(String, long, Configuration)}.
+   *  @param pathStr prefix for the temporary file
+   *  @param size the size of the file that is going to be written
+   *  @param conf the Configuration object
+   *  @return a unique temporary file
+   *  @throws IOException IO problems
+   */
+  synchronized File createTmpFileForWrite(String pathStr, long size,
+      Configuration conf) throws IOException {
+    if (directoryAllocator == null) {
+      String bufferDir = conf.get(BUFFER_DIR) != null
+          ? BUFFER_DIR : "hadoop.tmp.dir";
+      directoryAllocator = new LocalDirAllocator(bufferDir);
+    }
+    return directoryAllocator.createTmpFileForWrite(pathStr, size, conf);
+  }
+
+  /**
+   * Get the bucket of this filesystem.
+   * @return the bucket
+   */
+  public String getBucket() {
+    return bucket;
+  }
+
   /**
   /**
    * Change the input policy for this FS.
    * Change the input policy for this FS.
    * @param inputPolicy new policy
    * @param inputPolicy new policy
@@ -460,6 +513,7 @@ public class S3AFileSystem extends FileSystem {
    * @see #setPermission(Path, FsPermission)
    * @see #setPermission(Path, FsPermission)
    */
    */
   @Override
   @Override
+  @SuppressWarnings("IOResourceOpenedButNotSafelyClosed")
   public FSDataOutputStream create(Path f, FsPermission permission,
   public FSDataOutputStream create(Path f, FsPermission permission,
       boolean overwrite, int bufferSize, short replication, long blockSize,
       boolean overwrite, int bufferSize, short replication, long blockSize,
       Progressable progress) throws IOException {
       Progressable progress) throws IOException {
@@ -484,28 +538,33 @@ public class S3AFileSystem extends FileSystem {
 
 
     }
     }
     instrumentation.fileCreated();
     instrumentation.fileCreated();
-    if (getConf().getBoolean(FAST_UPLOAD, DEFAULT_FAST_UPLOAD)) {
-      return new FSDataOutputStream(
-          new S3AFastOutputStream(s3,
-              this,
-              bucket,
+    FSDataOutputStream output;
+    if (blockUploadEnabled) {
+      output = new FSDataOutputStream(
+          new S3ABlockOutputStream(this,
               key,
               key,
+              new SemaphoredDelegatingExecutor(threadPoolExecutor,
+                  blockOutputActiveBlocks, true),
               progress,
               progress,
-              cannedACL,
               partSize,
               partSize,
-              multiPartThreshold,
-              threadPoolExecutor),
-          statistics);
+              blockFactory,
+              instrumentation.newOutputStreamStatistics(),
+              new WriteOperationHelper(key)
+          ),
+          null);
+    } else {
+
+      // We pass null to FSDataOutputStream so it won't count writes that
+      // are being buffered to a file
+      output = new FSDataOutputStream(
+          new S3AOutputStream(getConf(),
+              this,
+              key,
+              progress
+          ),
+          null);
     }
     }
-    // We pass null to FSDataOutputStream so it won't count writes that
-    // are being buffered to a file
-    return new FSDataOutputStream(
-        new S3AOutputStream(getConf(),
-            this,
-            key,
-            progress
-        ),
-        null);
+    return output;
   }
   }
 
 
   /**
   /**
@@ -749,6 +808,33 @@ public class S3AFileSystem extends FileSystem {
     storageStatistics.incrementCounter(statistic, count);
     storageStatistics.incrementCounter(statistic, count);
   }
   }
 
 
+  /**
+   * Decrement a gauge by a specific value.
+   * @param statistic The operation to decrement
+   * @param count the count to decrement
+   */
+  protected void decrementGauge(Statistic statistic, long count) {
+    instrumentation.decrementGauge(statistic, count);
+  }
+
+  /**
+   * Increment a gauge by a specific value.
+   * @param statistic The operation to increment
+   * @param count the count to increment
+   */
+  protected void incrementGauge(Statistic statistic, long count) {
+    instrumentation.incrementGauge(statistic, count);
+  }
+
+  /**
+   * Get the storage statistics of this filesystem.
+   * @return the storage statistics
+   */
+  @Override
+  public S3AStorageStatistics getStorageStatistics() {
+    return storageStatistics;
+  }
+
   /**
   /**
    * Request object metadata; increments counters in the process.
    * Request object metadata; increments counters in the process.
    * @param key key
    * @param key key
@@ -896,7 +982,9 @@ public class S3AFileSystem extends FileSystem {
    */
    */
   public ObjectMetadata newObjectMetadata(long length) {
   public ObjectMetadata newObjectMetadata(long length) {
     final ObjectMetadata om = newObjectMetadata();
     final ObjectMetadata om = newObjectMetadata();
-    om.setContentLength(length);
+    if (length >= 0) {
+      om.setContentLength(length);
+    }
     return om;
     return om;
   }
   }
 
 
@@ -918,7 +1006,41 @@ public class S3AFileSystem extends FileSystem {
       len = putObjectRequest.getMetadata().getContentLength();
       len = putObjectRequest.getMetadata().getContentLength();
     }
     }
     incrementPutStartStatistics(len);
     incrementPutStartStatistics(len);
-    return transfers.upload(putObjectRequest);
+    try {
+      Upload upload = transfers.upload(putObjectRequest);
+      incrementPutCompletedStatistics(true, len);
+      return upload;
+    } catch (AmazonClientException e) {
+      incrementPutCompletedStatistics(false, len);
+      throw e;
+    }
+  }
+
+  /**
+   * PUT an object directly (i.e. not via the transfer manager).
+   * Byte length is calculated from the file length, or, if there is no
+   * file, from the content length of the header.
+   * @param putObjectRequest the request
+   * @return the upload initiated
+   * @throws AmazonClientException on problems
+   */
+  public PutObjectResult putObjectDirect(PutObjectRequest putObjectRequest)
+      throws AmazonClientException {
+    long len;
+    if (putObjectRequest.getFile() != null) {
+      len = putObjectRequest.getFile().length();
+    } else {
+      len = putObjectRequest.getMetadata().getContentLength();
+    }
+    incrementPutStartStatistics(len);
+    try {
+      PutObjectResult result = s3.putObject(putObjectRequest);
+      incrementPutCompletedStatistics(true, len);
+      return result;
+    } catch (AmazonClientException e) {
+      incrementPutCompletedStatistics(false, len);
+      throw e;
+    }
   }
   }
 
 
   /**
   /**
@@ -926,10 +1048,20 @@ public class S3AFileSystem extends FileSystem {
    * Increments the write and put counters
    * Increments the write and put counters
    * @param request request
    * @param request request
    * @return the result of the operation.
    * @return the result of the operation.
+   * @throws AmazonClientException on problems
    */
    */
-  public UploadPartResult uploadPart(UploadPartRequest request) {
-    incrementPutStartStatistics(request.getPartSize());
-    return s3.uploadPart(request);
+  public UploadPartResult uploadPart(UploadPartRequest request)
+      throws AmazonClientException {
+    long len = request.getPartSize();
+    incrementPutStartStatistics(len);
+    try {
+      UploadPartResult uploadPartResult = s3.uploadPart(request);
+      incrementPutCompletedStatistics(true, len);
+      return uploadPartResult;
+    } catch (AmazonClientException e) {
+      incrementPutCompletedStatistics(false, len);
+      throw e;
+    }
   }
   }
 
 
   /**
   /**
@@ -942,9 +1074,28 @@ public class S3AFileSystem extends FileSystem {
     LOG.debug("PUT start {} bytes", bytes);
     LOG.debug("PUT start {} bytes", bytes);
     incrementWriteOperations();
     incrementWriteOperations();
     incrementStatistic(OBJECT_PUT_REQUESTS);
     incrementStatistic(OBJECT_PUT_REQUESTS);
+    incrementGauge(OBJECT_PUT_REQUESTS_ACTIVE, 1);
+    if (bytes > 0) {
+      incrementGauge(OBJECT_PUT_BYTES_PENDING, bytes);
+    }
+  }
+
+  /**
+   * At the end of a put/multipart upload operation, update the
+   * relevant counters and gauges.
+   *
+   * @param success did the operation succeed?
+   * @param bytes bytes in the request.
+   */
+  public void incrementPutCompletedStatistics(boolean success, long bytes) {
+    LOG.debug("PUT completed success={}; {} bytes", success, bytes);
+    incrementWriteOperations();
     if (bytes > 0) {
     if (bytes > 0) {
       incrementStatistic(OBJECT_PUT_BYTES, bytes);
       incrementStatistic(OBJECT_PUT_BYTES, bytes);
+      decrementGauge(OBJECT_PUT_BYTES_PENDING, bytes);
     }
     }
+    incrementStatistic(OBJECT_PUT_REQUESTS_COMPLETED);
+    decrementGauge(OBJECT_PUT_REQUESTS_ACTIVE, 1);
   }
   }
 
 
   /**
   /**
@@ -955,7 +1106,7 @@ public class S3AFileSystem extends FileSystem {
    * @param bytes bytes successfully uploaded.
    * @param bytes bytes successfully uploaded.
    */
    */
   public void incrementPutProgressStatistics(String key, long bytes) {
   public void incrementPutProgressStatistics(String key, long bytes) {
-    LOG.debug("PUT {}: {} bytes", key, bytes);
+    PROGRESS.debug("PUT {}: {} bytes", key, bytes);
     incrementWriteOperations();
     incrementWriteOperations();
     if (bytes > 0) {
     if (bytes > 0) {
       statistics.incrementBytesWritten(bytes);
       statistics.incrementBytesWritten(bytes);
@@ -1475,7 +1626,7 @@ public class S3AFileSystem extends FileSystem {
     LocalFileSystem local = getLocal(getConf());
     LocalFileSystem local = getLocal(getConf());
     File srcfile = local.pathToFile(src);
     File srcfile = local.pathToFile(src);
 
 
-    final ObjectMetadata om = newObjectMetadata();
+    final ObjectMetadata om = newObjectMetadata(srcfile.length());
     PutObjectRequest putObjectRequest = newPutObjectRequest(key, om, srcfile);
     PutObjectRequest putObjectRequest = newPutObjectRequest(key, om, srcfile);
     Upload up = putObject(putObjectRequest);
     Upload up = putObject(putObjectRequest);
     ProgressableProgressListener listener = new ProgressableProgressListener(
     ProgressableProgressListener listener = new ProgressableProgressListener(
@@ -1743,6 +1894,10 @@ public class S3AFileSystem extends FileSystem {
           .append(serverSideEncryptionAlgorithm)
           .append(serverSideEncryptionAlgorithm)
           .append('\'');
           .append('\'');
     }
     }
+    if (blockFactory != null) {
+      sb.append(", blockFactory=").append(blockFactory);
+    }
+    sb.append(", executor=").append(threadPoolExecutor);
     sb.append(", statistics {")
     sb.append(", statistics {")
         .append(statistics)
         .append(statistics)
         .append("}");
         .append("}");
@@ -1950,4 +2105,163 @@ public class S3AFileSystem extends FileSystem {
           getFileBlockLocations(status, 0, status.getLen())
           getFileBlockLocations(status, 0, status.getLen())
           : null);
           : null);
   }
   }
+
+  /**
+   * Helper for an ongoing write operation.
+   * <p>
+   * It hides direct access to the S3 API from the output stream,
+   * and is a location where the object upload process can be evolved/enhanced.
+   * <p>
+   * Features
+   * <ul>
+   *   <li>Methods to create and submit requests to S3, so avoiding
+   *   all direct interaction with the AWS APIs.</li>
+   *   <li>Some extra preflight checks of arguments, so failing fast on
+   *   errors.</li>
+   *   <li>Callbacks to let the FS know of events in the output stream
+   *   upload process.</li>
+   * </ul>
+   *
+   * Each instance of this state is unique to a single output stream.
+   */
+  final class WriteOperationHelper {
+    private final String key;
+
+    private WriteOperationHelper(String key) {
+      this.key = key;
+    }
+
+    /**
+     * Create a {@link PutObjectRequest} request.
+     * The metadata is assumed to have been configured with the size of the
+     * operation.
+     * @param inputStream source data.
+     * @param length size, if known. Use -1 for not known
+     * @return the request
+     */
+    PutObjectRequest newPutRequest(InputStream inputStream, long length) {
+      return newPutObjectRequest(key, newObjectMetadata(length), inputStream);
+    }
+
+    /**
+     * Callback on a successful write.
+     */
+    void writeSuccessful() {
+      finishedWrite(key);
+    }
+
+    /**
+     * Callback on a write failure.
+     * @param e Any exception raised which triggered the failure.
+     */
+    void writeFailed(Exception e) {
+      LOG.debug("Write to {} failed", this, e);
+    }
+
+    /**
+     * Create a new object metadata instance.
+     * Any standard metadata headers are added here, for example:
+     * encryption.
+     * @param length size, if known. Use -1 for not known
+     * @return a new metadata instance
+     */
+    public ObjectMetadata newObjectMetadata(long length) {
+      return S3AFileSystem.this.newObjectMetadata(length);
+    }
+
+    /**
+     * Start the multipart upload process.
+     * @return the upload result containing the ID
+     * @throws IOException IO problem
+     */
+    String initiateMultiPartUpload() throws IOException {
+      LOG.debug("Initiating Multipart upload");
+      final InitiateMultipartUploadRequest initiateMPURequest =
+          new InitiateMultipartUploadRequest(bucket,
+              key,
+              newObjectMetadata(-1));
+      initiateMPURequest.setCannedACL(cannedACL);
+      try {
+        return s3.initiateMultipartUpload(initiateMPURequest)
+            .getUploadId();
+      } catch (AmazonClientException ace) {
+        throw translateException("initiate MultiPartUpload", key, ace);
+      }
+    }
+
+    /**
+     * Complete a multipart upload operation.
+     * @param uploadId multipart operation Id
+     * @param partETags list of partial uploads
+     * @return the result
+     * @throws AmazonClientException on problems.
+     */
+    CompleteMultipartUploadResult completeMultipartUpload(String uploadId,
+        List<PartETag> partETags) throws AmazonClientException {
+      Preconditions.checkNotNull(uploadId);
+      Preconditions.checkNotNull(partETags);
+      Preconditions.checkArgument(!partETags.isEmpty(),
+          "No partitions have been uploaded");
+      return s3.completeMultipartUpload(
+          new CompleteMultipartUploadRequest(bucket,
+              key,
+              uploadId,
+              partETags));
+    }
+
+    /**
+     * Abort a multipart upload operation.
+     * @param uploadId multipart operation Id
+     * @return the result
+     * @throws AmazonClientException on problems.
+     */
+    void abortMultipartUpload(String uploadId) throws AmazonClientException {
+      s3.abortMultipartUpload(
+          new AbortMultipartUploadRequest(bucket, key, uploadId));
+    }
+
+    /**
+     * Create and initialize a part request of a multipart upload.
+     * @param uploadId ID of ongoing upload
+     * @param uploadStream source of data to upload
+     * @param partNumber current part number of the upload
+     * @param size amount of data
+     * @return the request.
+     */
+    UploadPartRequest newUploadPartRequest(String uploadId,
+        InputStream uploadStream,
+        int partNumber,
+        int size) {
+      Preconditions.checkNotNull(uploadId);
+      Preconditions.checkNotNull(uploadStream);
+      Preconditions.checkArgument(size > 0, "Invalid partition size %s", size);
+      Preconditions.checkArgument(partNumber> 0 && partNumber <=10000,
+          "partNumber must be between 1 and 10000 inclusive, but is %s",
+          partNumber);
+
+      LOG.debug("Creating part upload request for {} #{} size {}",
+          uploadId, partNumber, size);
+      return new UploadPartRequest()
+          .withBucketName(bucket)
+          .withKey(key)
+          .withUploadId(uploadId)
+          .withInputStream(uploadStream)
+          .withPartNumber(partNumber)
+          .withPartSize(size);
+    }
+
+    /**
+     * The toString method is intended to be used in logging/toString calls.
+     * @return a string description.
+     */
+    @Override
+    public String toString() {
+      final StringBuilder sb = new StringBuilder(
+          "{bucket=").append(bucket);
+      sb.append(", key='").append(key).append('\'');
+      sb.append('}');
+      return sb.toString();
+    }
+  }
+
 }
 }

+ 245 - 3
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java

@@ -18,7 +18,9 @@
 
 
 package org.apache.hadoop.fs.s3a;
 package org.apache.hadoop.fs.s3a;
 
 
-import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.metrics2.MetricStringBuilder;
 import org.apache.hadoop.metrics2.MetricStringBuilder;
@@ -29,10 +31,12 @@ import org.apache.hadoop.metrics2.lib.MutableCounterLong;
 import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
 import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
 import org.apache.hadoop.metrics2.lib.MutableMetric;
 import org.apache.hadoop.metrics2.lib.MutableMetric;
 
 
+import java.io.Closeable;
 import java.net.URI;
 import java.net.URI;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Map;
 import java.util.UUID;
 import java.util.UUID;
+import java.util.concurrent.atomic.AtomicLong;
 
 
 import static org.apache.hadoop.fs.s3a.Statistic.*;
 import static org.apache.hadoop.fs.s3a.Statistic.*;
 
 
@@ -50,6 +54,9 @@ import static org.apache.hadoop.fs.s3a.Statistic.*;
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 @InterfaceStability.Evolving
 public class S3AInstrumentation {
 public class S3AInstrumentation {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      S3AInstrumentation.class);
+
   public static final String CONTEXT = "S3AFileSystem";
   public static final String CONTEXT = "S3AFileSystem";
   private final MetricsRegistry registry =
   private final MetricsRegistry registry =
       new MetricsRegistry("S3AFileSystem").setContext(CONTEXT);
       new MetricsRegistry("S3AFileSystem").setContext(CONTEXT);
@@ -100,7 +107,23 @@ public class S3AInstrumentation {
       OBJECT_METADATA_REQUESTS,
       OBJECT_METADATA_REQUESTS,
       OBJECT_MULTIPART_UPLOAD_ABORTED,
       OBJECT_MULTIPART_UPLOAD_ABORTED,
       OBJECT_PUT_BYTES,
       OBJECT_PUT_BYTES,
-      OBJECT_PUT_REQUESTS
+      OBJECT_PUT_REQUESTS,
+      OBJECT_PUT_REQUESTS_COMPLETED,
+      STREAM_WRITE_FAILURES,
+      STREAM_WRITE_BLOCK_UPLOADS,
+      STREAM_WRITE_BLOCK_UPLOADS_COMMITTED,
+      STREAM_WRITE_BLOCK_UPLOADS_ABORTED,
+      STREAM_WRITE_TOTAL_TIME,
+      STREAM_WRITE_TOTAL_DATA,
+  };
+
+
+  private static final Statistic[] GAUGES_TO_CREATE = {
+      OBJECT_PUT_REQUESTS_ACTIVE,
+      OBJECT_PUT_BYTES_PENDING,
+      STREAM_WRITE_BLOCK_UPLOADS_ACTIVE,
+      STREAM_WRITE_BLOCK_UPLOADS_PENDING,
+      STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING,
   };
   };
 
 
   public S3AInstrumentation(URI name) {
   public S3AInstrumentation(URI name) {
@@ -143,6 +166,9 @@ public class S3AInstrumentation {
     for (Statistic statistic : COUNTERS_TO_CREATE) {
     for (Statistic statistic : COUNTERS_TO_CREATE) {
       counter(statistic);
       counter(statistic);
     }
     }
+    for (Statistic statistic : GAUGES_TO_CREATE) {
+      gauge(statistic.getSymbol(), statistic.getDescription());
+    }
   }
   }
 
 
   /**
   /**
@@ -254,13 +280,13 @@ public class S3AInstrumentation {
    * Lookup a counter by name. Return null if it is not known.
    * Lookup a counter by name. Return null if it is not known.
    * @param name counter name
    * @param name counter name
    * @return the counter
    * @return the counter
+   * @throws IllegalStateException if the metric is not a counter
    */
    */
   private MutableCounterLong lookupCounter(String name) {
   private MutableCounterLong lookupCounter(String name) {
     MutableMetric metric = lookupMetric(name);
     MutableMetric metric = lookupMetric(name);
     if (metric == null) {
     if (metric == null) {
       return null;
       return null;
     }
     }
-    Preconditions.checkNotNull(metric, "not found: " + name);
     if (!(metric instanceof MutableCounterLong)) {
     if (!(metric instanceof MutableCounterLong)) {
       throw new IllegalStateException("Metric " + name
       throw new IllegalStateException("Metric " + name
           + " is not a MutableCounterLong: " + metric);
           + " is not a MutableCounterLong: " + metric);
@@ -268,6 +294,20 @@ public class S3AInstrumentation {
     return (MutableCounterLong) metric;
     return (MutableCounterLong) metric;
   }
   }
 
 
+  /**
+   * Look up a gauge.
+   * @param name gauge name
+   * @return the gauge or null
+   * @throws ClassCastException if the metric is not a Gauge.
+   */
+  public MutableGaugeLong lookupGauge(String name) {
+    MutableMetric metric = lookupMetric(name);
+    if (metric == null) {
+      LOG.debug("No gauge {}", name);
+    }
+    return (MutableGaugeLong) metric;
+  }
+
   /**
   /**
    * Look up a metric from both the registered set and the lighter weight
    * Look up a metric from both the registered set and the lighter weight
    * stream entries.
    * stream entries.
@@ -349,6 +389,47 @@ public class S3AInstrumentation {
       counter.incr(count);
       counter.incr(count);
     }
     }
   }
   }
+  /**
+   * Increment a specific counter.
+   * No-op if not defined.
+   * @param op operation
+   * @param count atomic long containing value
+   */
+  public void incrementCounter(Statistic op, AtomicLong count) {
+    incrementCounter(op, count.get());
+  }
+
+  /**
+   * Increment a specific gauge.
+   * No-op if not defined.
+   * @param op operation
+   * @param count increment value
+   * @throws ClassCastException if the metric is of the wrong type
+   */
+  public void incrementGauge(Statistic op, long count) {
+    MutableGaugeLong gauge = lookupGauge(op.getSymbol());
+    if (gauge != null) {
+      gauge.incr(count);
+    } else {
+      LOG.debug("No Gauge: "+ op);
+    }
+  }
+
+  /**
+   * Decrement a specific gauge.
+   * No-op if not defined.
+   * @param op operation
+   * @param count increment value
+   * @throws ClassCastException if the metric is of the wrong type
+   */
+  public void decrementGauge(Statistic op, long count) {
+    MutableGaugeLong gauge = lookupGauge(op.getSymbol());
+    if (gauge != null) {
+      gauge.decr(count);
+    } else {
+      LOG.debug("No Gauge: " + op);
+    }
+  }
 
 
   /**
   /**
    * Create a stream input statistics instance.
    * Create a stream input statistics instance.
@@ -553,4 +634,165 @@ public class S3AInstrumentation {
       return sb.toString();
       return sb.toString();
     }
     }
   }
   }
+
+  /**
+   * Create a stream output statistics instance.
+   * @return the new instance
+   */
+
+  OutputStreamStatistics newOutputStreamStatistics() {
+    return new OutputStreamStatistics();
+  }
+
+  /**
+   * Merge in the statistics of a single output stream into
+   * the filesystem-wide statistics.
+   * @param statistics stream statistics
+   */
+  private void mergeOutputStreamStatistics(OutputStreamStatistics statistics) {
+    incrementCounter(STREAM_WRITE_TOTAL_TIME, statistics.totalUploadDuration());
+    incrementCounter(STREAM_WRITE_QUEUE_DURATION, statistics.queueDuration);
+    incrementCounter(STREAM_WRITE_TOTAL_DATA, statistics.bytesUploaded);
+    incrementCounter(STREAM_WRITE_BLOCK_UPLOADS,
+        statistics.blockUploadsCompleted);
+  }
+
+  /**
+   * Statistics updated by an output stream during its actual operation.
+   * Some of these stats may be relayed. However, as block upload is
+   * spans multiple
+   */
+  @InterfaceAudience.Private
+  @InterfaceStability.Unstable
+  public final class OutputStreamStatistics implements Closeable {
+    private final AtomicLong blocksSubmitted = new AtomicLong(0);
+    private final AtomicLong blocksInQueue = new AtomicLong(0);
+    private final AtomicLong blocksActive = new AtomicLong(0);
+    private final AtomicLong blockUploadsCompleted = new AtomicLong(0);
+    private final AtomicLong blockUploadsFailed = new AtomicLong(0);
+    private final AtomicLong bytesPendingUpload = new AtomicLong(0);
+
+    private final AtomicLong bytesUploaded = new AtomicLong(0);
+    private final AtomicLong transferDuration = new AtomicLong(0);
+    private final AtomicLong queueDuration = new AtomicLong(0);
+    private final AtomicLong exceptionsInMultipartFinalize = new AtomicLong(0);
+
+    /**
+     * Block is queued for upload.
+     */
+    void blockUploadQueued(int blockSize) {
+      blocksSubmitted.incrementAndGet();
+      blocksInQueue.incrementAndGet();
+      bytesPendingUpload.addAndGet(blockSize);
+      incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_PENDING, 1);
+      incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING, blockSize);
+    }
+
+    /** Queued block has been scheduled for upload. */
+    void blockUploadStarted(long duration, int blockSize) {
+      queueDuration.addAndGet(duration);
+      blocksInQueue.decrementAndGet();
+      blocksActive.incrementAndGet();
+      incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_PENDING, -1);
+      incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_ACTIVE, 1);
+    }
+
+    /** A block upload has completed. */
+    void blockUploadCompleted(long duration, int blockSize) {
+      this.transferDuration.addAndGet(duration);
+      incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_ACTIVE, -1);
+      blocksActive.decrementAndGet();
+      blockUploadsCompleted.incrementAndGet();
+    }
+
+    /**
+     *  A block upload has failed.
+     *  A final transfer completed event is still expected, so this
+     *  does not decrement the active block counter.
+     */
+    void blockUploadFailed(long duration, int blockSize) {
+      blockUploadsFailed.incrementAndGet();
+    }
+
+    /** Intermediate report of bytes uploaded. */
+    void bytesTransferred(long byteCount) {
+      bytesUploaded.addAndGet(byteCount);
+      bytesPendingUpload.addAndGet(-byteCount);
+      incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING, -byteCount);
+    }
+
+    /**
+     * Note an exception in a multipart complete.
+     */
+    void exceptionInMultipartComplete() {
+      exceptionsInMultipartFinalize.incrementAndGet();
+    }
+
+    /**
+     * Note an exception in a multipart abort.
+     */
+    void exceptionInMultipartAbort() {
+      exceptionsInMultipartFinalize.incrementAndGet();
+    }
+
+    /**
+     * Get the number of bytes pending upload.
+     * @return the number of bytes in the pending upload state.
+     */
+    public long getBytesPendingUpload() {
+      return bytesPendingUpload.get();
+    }
+
+    /**
+     * Output stream has closed.
+     * Trigger merge in of all statistics not updated during operation.
+     */
+    @Override
+    public void close() {
+      if (bytesPendingUpload.get() > 0) {
+        LOG.warn("Closing output stream statistics while data is still marked" +
+            " as pending upload in {}", this);
+      }
+      mergeOutputStreamStatistics(this);
+    }
+
+    long averageQueueTime() {
+      return blocksSubmitted.get() > 0 ?
+          (queueDuration.get() / blocksSubmitted.get()) : 0;
+    }
+
+    double effectiveBandwidth() {
+      double duration = totalUploadDuration() / 1000.0;
+      return duration > 0 ?
+          (bytesUploaded.get() / duration) : 0;
+    }
+
+    long totalUploadDuration() {
+      return queueDuration.get() + transferDuration.get();
+    }
+
+    @Override
+    public String toString() {
+      final StringBuilder sb = new StringBuilder(
+          "OutputStreamStatistics{");
+      sb.append("blocksSubmitted=").append(blocksSubmitted);
+      sb.append(", blocksInQueue=").append(blocksInQueue);
+      sb.append(", blocksActive=").append(blocksActive);
+      sb.append(", blockUploadsCompleted=").append(blockUploadsCompleted);
+      sb.append(", blockUploadsFailed=").append(blockUploadsFailed);
+      sb.append(", bytesPendingUpload=").append(bytesPendingUpload);
+      sb.append(", bytesUploaded=").append(bytesUploaded);
+      sb.append(", exceptionsInMultipartFinalize=").append(
+          exceptionsInMultipartFinalize);
+      sb.append(", transferDuration=").append(transferDuration).append(" ms");
+      sb.append(", queueDuration=").append(queueDuration).append(" ms");
+      sb.append(", averageQueueTime=").append(averageQueueTime()).append(" ms");
+      sb.append(", totalUploadDuration=").append(totalUploadDuration())
+          .append(" ms");
+      sb.append(", effectiveBandwidth=").append(effectiveBandwidth())
+          .append(" bytes/s");
+      sb.append('}');
+      return sb.toString();
+    }
+  }
 }
 }

+ 28 - 29
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java

@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
  * distributed with this work for additional information
@@ -35,8 +35,8 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.io.InterruptedIOException;
 import java.io.OutputStream;
 import java.io.OutputStream;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 
-import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.S3AUtils.*;
 import static org.apache.hadoop.fs.s3a.S3AUtils.*;
 
 
 /**
 /**
@@ -45,37 +45,27 @@ import static org.apache.hadoop.fs.s3a.S3AUtils.*;
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 @InterfaceStability.Evolving
 public class S3AOutputStream extends OutputStream {
 public class S3AOutputStream extends OutputStream {
-  private OutputStream backupStream;
-  private File backupFile;
-  private boolean closed;
-  private String key;
-  private Progressable progress;
-  private long partSize;
-  private long partSizeThreshold;
-  private S3AFileSystem fs;
-  private LocalDirAllocator lDirAlloc;
+  private final OutputStream backupStream;
+  private final File backupFile;
+  private final AtomicBoolean closed = new AtomicBoolean(false);
+  private final String key;
+  private final Progressable progress;
+  private final S3AFileSystem fs;
 
 
   public static final Logger LOG = S3AFileSystem.LOG;
   public static final Logger LOG = S3AFileSystem.LOG;
 
 
   public S3AOutputStream(Configuration conf,
   public S3AOutputStream(Configuration conf,
-      S3AFileSystem fs, String key, Progressable progress)
+      S3AFileSystem fs,
+      String key,
+      Progressable progress)
       throws IOException {
       throws IOException {
     this.key = key;
     this.key = key;
     this.progress = progress;
     this.progress = progress;
     this.fs = fs;
     this.fs = fs;
 
 
-    partSize = fs.getPartitionSize();
-    partSizeThreshold = fs.getMultiPartThreshold();
-
-    if (conf.get(BUFFER_DIR, null) != null) {
-      lDirAlloc = new LocalDirAllocator(BUFFER_DIR);
-    } else {
-      lDirAlloc = new LocalDirAllocator("${hadoop.tmp.dir}/s3a");
-    }
 
 
-    backupFile = lDirAlloc.createTmpFileForWrite("output-",
+    backupFile = fs.createTmpFileForWrite("output-",
         LocalDirAllocator.SIZE_UNKNOWN, conf);
         LocalDirAllocator.SIZE_UNKNOWN, conf);
-    closed = false;
 
 
     LOG.debug("OutputStream for key '{}' writing to tempfile: {}",
     LOG.debug("OutputStream for key '{}' writing to tempfile: {}",
         key, backupFile);
         key, backupFile);
@@ -84,25 +74,33 @@ public class S3AOutputStream extends OutputStream {
         new FileOutputStream(backupFile));
         new FileOutputStream(backupFile));
   }
   }
 
 
+  /**
+   * Check for the filesystem being open.
+   * @throws IOException if the filesystem is closed.
+   */
+  void checkOpen() throws IOException {
+    if (closed.get()) {
+      throw new IOException("Output Stream closed");
+    }
+  }
+
   @Override
   @Override
   public void flush() throws IOException {
   public void flush() throws IOException {
+    checkOpen();
     backupStream.flush();
     backupStream.flush();
   }
   }
 
 
   @Override
   @Override
-  public synchronized void close() throws IOException {
-    if (closed) {
+  public void close() throws IOException {
+    if (closed.getAndSet(true)) {
       return;
       return;
     }
     }
 
 
     backupStream.close();
     backupStream.close();
     LOG.debug("OutputStream for key '{}' closed. Now beginning upload", key);
     LOG.debug("OutputStream for key '{}' closed. Now beginning upload", key);
-    LOG.debug("Minimum upload part size: {} threshold {}" , partSize,
-        partSizeThreshold);
-
 
 
     try {
     try {
-      final ObjectMetadata om = fs.newObjectMetadata();
+      final ObjectMetadata om = fs.newObjectMetadata(backupFile.length());
       Upload upload = fs.putObject(
       Upload upload = fs.putObject(
           fs.newPutObjectRequest(
           fs.newPutObjectRequest(
               key,
               key,
@@ -126,18 +124,19 @@ public class S3AOutputStream extends OutputStream {
         LOG.warn("Could not delete temporary s3a file: {}", backupFile);
         LOG.warn("Could not delete temporary s3a file: {}", backupFile);
       }
       }
       super.close();
       super.close();
-      closed = true;
     }
     }
     LOG.debug("OutputStream for key '{}' upload complete", key);
     LOG.debug("OutputStream for key '{}' upload complete", key);
   }
   }
 
 
   @Override
   @Override
   public void write(int b) throws IOException {
   public void write(int b) throws IOException {
+    checkOpen();
     backupStream.write(b);
     backupStream.write(b);
   }
   }
 
 
   @Override
   @Override
   public void write(byte[] b, int off, int len) throws IOException {
   public void write(byte[] b, int off, int len) throws IOException {
+    checkOpen();
     backupStream.write(b, off, len);
     backupStream.write(b, off, len);
   }
   }
 
 

+ 39 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java

@@ -49,6 +49,7 @@ import java.util.concurrent.ExecutionException;
 import static org.apache.hadoop.fs.s3a.Constants.ACCESS_KEY;
 import static org.apache.hadoop.fs.s3a.Constants.ACCESS_KEY;
 import static org.apache.hadoop.fs.s3a.Constants.AWS_CREDENTIALS_PROVIDER;
 import static org.apache.hadoop.fs.s3a.Constants.AWS_CREDENTIALS_PROVIDER;
 import static org.apache.hadoop.fs.s3a.Constants.ENDPOINT;
 import static org.apache.hadoop.fs.s3a.Constants.ENDPOINT;
+import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_MIN_SIZE;
 import static org.apache.hadoop.fs.s3a.Constants.SECRET_KEY;
 import static org.apache.hadoop.fs.s3a.Constants.SECRET_KEY;
 
 
 /**
 /**
@@ -460,4 +461,42 @@ public final class S3AUtils {
             key, v, min));
             key, v, min));
     return v;
     return v;
   }
   }
+
+  /**
+   * Get a size property from the configuration: this property must
+   * be at least equal to {@link Constants#MULTIPART_MIN_SIZE}.
+   * If it is too small, it is rounded up to that minimum, and a warning
+   * printed.
+   * @param conf configuration
+   * @param property property name
+   * @param defVal default value
+   * @return the value, guaranteed to be above the minimum size
+   */
+  public static long getMultipartSizeProperty(Configuration conf,
+      String property, long defVal) {
+    long partSize = conf.getLong(property, defVal);
+    if (partSize < MULTIPART_MIN_SIZE) {
+      LOG.warn("{} must be at least 5 MB; configured value is {}",
+          property, partSize);
+      partSize = MULTIPART_MIN_SIZE;
+    }
+    return partSize;
+  }
+
+  /**
+   * Ensure that the long value is in the range of an integer.
+   * @param name property name for error messages
+   * @param size original size
+   * @return the size, guaranteed to be less than or equal to the max
+   * value of an integer.
+   */
+  public static int ensureOutputParameterInRange(String name, long size) {
+    if (size > Integer.MAX_VALUE) {
+      LOG.warn("s3a: {} capped to ~2.14GB" +
+          " (maximum allowed size with current output mechanism)", name);
+      return Integer.MAX_VALUE;
+    } else {
+      return (int)size;
+    }
+  }
 }
 }

+ 230 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SemaphoredDelegatingExecutor.java

@@ -0,0 +1,230 @@
+/*
+ * 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.google.common.util.concurrent.ForwardingListeningExecutorService;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * This ExecutorService blocks the submission of new tasks when its queue is
+ * already full by using a semaphore. Task submissions require permits, task
+ * completions release permits.
+ * <p>
+ * This is a refactoring of {@link BlockingThreadPoolExecutorService}; that code
+ * contains the thread pool logic, whereas this isolates the semaphore
+ * and submit logic for use with other thread pools and delegation models.
+ * In particular, it <i>permits multiple per stream executors to share a
+ * single per-FS-instance executor; the latter to throttle overall
+ * load from the the FS, the others to limit the amount of load which
+ * a single output stream can generate.</i>
+ * <p>
+ * This is inspired by <a href="https://github.com/apache/incubator-s4/blob/master/subprojects/s4-comm/src/main/java/org/apache/s4/comm/staging/BlockingThreadPoolExecutorService.java">
+ * this s4 threadpool</a>
+ */
+@SuppressWarnings("NullableProblems")
+@InterfaceAudience.Private
+class SemaphoredDelegatingExecutor extends
+    ForwardingListeningExecutorService {
+
+  private final Semaphore queueingPermits;
+  private final ListeningExecutorService executorDelegatee;
+  private final int permitCount;
+
+  /**
+   * Instantiate.
+   * @param executorDelegatee Executor to delegate to
+   * @param permitCount number of permits into the queue permitted
+   * @param fair should the semaphore be "fair"
+   */
+  SemaphoredDelegatingExecutor(ListeningExecutorService executorDelegatee,
+      int permitCount,
+      boolean fair) {
+    this.permitCount = permitCount;
+    queueingPermits = new Semaphore(permitCount, fair);
+    this.executorDelegatee = executorDelegatee;
+  }
+
+  @Override
+  protected ListeningExecutorService delegate() {
+    return executorDelegatee;
+  }
+
+
+  @Override
+  public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks)
+      throws InterruptedException {
+    throw new RuntimeException("Not implemented");
+  }
+
+  @Override
+  public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks,
+      long timeout, TimeUnit unit) throws InterruptedException {
+    throw new RuntimeException("Not implemented");
+  }
+
+  @Override
+  public <T> T invokeAny(Collection<? extends Callable<T>> tasks)
+      throws InterruptedException, ExecutionException {
+    throw new RuntimeException("Not implemented");
+  }
+
+  @Override
+  public <T> T invokeAny(Collection<? extends Callable<T>> tasks, long timeout,
+      TimeUnit unit)
+      throws InterruptedException, ExecutionException, TimeoutException {
+    throw new RuntimeException("Not implemented");
+  }
+
+  @Override
+  public <T> ListenableFuture<T> submit(Callable<T> task) {
+    try {
+      queueingPermits.acquire();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      return Futures.immediateFailedCheckedFuture(e);
+    }
+    return super.submit(new CallableWithPermitRelease<>(task));
+  }
+
+  @Override
+  public <T> ListenableFuture<T> submit(Runnable task, T result) {
+    try {
+      queueingPermits.acquire();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      return Futures.immediateFailedCheckedFuture(e);
+    }
+    return super.submit(new RunnableWithPermitRelease(task), result);
+  }
+
+  @Override
+  public ListenableFuture<?> submit(Runnable task) {
+    try {
+      queueingPermits.acquire();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      return Futures.immediateFailedCheckedFuture(e);
+    }
+    return super.submit(new RunnableWithPermitRelease(task));
+  }
+
+  @Override
+  public void execute(Runnable command) {
+    try {
+      queueingPermits.acquire();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+    }
+    super.execute(new RunnableWithPermitRelease(command));
+  }
+
+  /**
+   * Get the number of permits available; guaranteed to be
+   * {@code 0 <= availablePermits <= size}.
+   * @return the number of permits available at the time of invocation.
+   */
+  public int getAvailablePermits() {
+    return queueingPermits.availablePermits();
+  }
+
+  /**
+   * Get the number of threads waiting to acquire a permit.
+   * @return snapshot of the length of the queue of blocked threads.
+   */
+  public int getWaitingCount() {
+    return queueingPermits.getQueueLength();
+  }
+
+  /**
+   * Total number of permits.
+   * @return the number of permits as set in the constructor
+   */
+  public int getPermitCount() {
+    return permitCount;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "SemaphoredDelegatingExecutor{");
+    sb.append("permitCount=").append(getPermitCount());
+    sb.append(", available=").append(getAvailablePermits());
+    sb.append(", waiting=").append(getWaitingCount());
+    sb.append('}');
+    return sb.toString();
+  }
+
+  /**
+   * Releases a permit after the task is executed.
+   */
+  class RunnableWithPermitRelease implements Runnable {
+
+    private Runnable delegatee;
+
+    public RunnableWithPermitRelease(Runnable delegatee) {
+      this.delegatee = delegatee;
+    }
+
+    @Override
+    public void run() {
+      try {
+        delegatee.run();
+      } finally {
+        queueingPermits.release();
+      }
+
+    }
+  }
+
+  /**
+   * Releases a permit after the task is completed.
+   */
+  class CallableWithPermitRelease<T> implements Callable<T> {
+
+    private Callable<T> delegatee;
+
+    public CallableWithPermitRelease(Callable<T> delegatee) {
+      this.delegatee = delegatee;
+    }
+
+    @Override
+    public T call() throws Exception {
+      try {
+        return delegatee.call();
+      } finally {
+        queueingPermits.release();
+      }
+    }
+
+  }
+
+}

+ 30 - 2
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java

@@ -81,10 +81,16 @@ public enum Statistic {
       "Object multipart upload aborted"),
       "Object multipart upload aborted"),
   OBJECT_PUT_REQUESTS("object_put_requests",
   OBJECT_PUT_REQUESTS("object_put_requests",
       "Object put/multipart upload count"),
       "Object put/multipart upload count"),
+  OBJECT_PUT_REQUESTS_COMPLETED("object_put_requests_completed",
+      "Object put/multipart upload completed count"),
+  OBJECT_PUT_REQUESTS_ACTIVE("object_put_requests_active",
+      "Current number of active put requests"),
   OBJECT_PUT_BYTES("object_put_bytes", "number of bytes uploaded"),
   OBJECT_PUT_BYTES("object_put_bytes", "number of bytes uploaded"),
+  OBJECT_PUT_BYTES_PENDING("object_put_bytes_pending",
+      "number of bytes queued for upload/being actively uploaded"),
   STREAM_ABORTED("stream_aborted",
   STREAM_ABORTED("stream_aborted",
       "Count of times the TCP stream was aborted"),
       "Count of times the TCP stream was aborted"),
-  STREAM_BACKWARD_SEEK_OPERATIONS("stream_backward_seek_pperations",
+  STREAM_BACKWARD_SEEK_OPERATIONS("stream_backward_seek_operations",
       "Number of executed seek operations which went backwards in a stream"),
       "Number of executed seek operations which went backwards in a stream"),
   STREAM_CLOSED("streamClosed", "Count of times the TCP stream was closed"),
   STREAM_CLOSED("streamClosed", "Count of times the TCP stream was closed"),
   STREAM_CLOSE_OPERATIONS("stream_close_operations",
   STREAM_CLOSE_OPERATIONS("stream_close_operations",
@@ -112,7 +118,29 @@ public enum Statistic {
   STREAM_CLOSE_BYTES_READ("stream_bytes_read_in_close",
   STREAM_CLOSE_BYTES_READ("stream_bytes_read_in_close",
       "Count of bytes read when closing streams during seek operations."),
       "Count of bytes read when closing streams during seek operations."),
   STREAM_ABORT_BYTES_DISCARDED("stream_bytes_discarded_in_abort",
   STREAM_ABORT_BYTES_DISCARDED("stream_bytes_discarded_in_abort",
-      "Count of bytes discarded by aborting the stream");
+      "Count of bytes discarded by aborting the stream"),
+  STREAM_WRITE_FAILURES("stream_write_failures",
+      "Count of stream write failures reported"),
+  STREAM_WRITE_BLOCK_UPLOADS("stream_write_block_uploads",
+      "Count of block/partition uploads completed"),
+  STREAM_WRITE_BLOCK_UPLOADS_ACTIVE("stream_write_block_uploads_active",
+      "Count of block/partition uploads completed"),
+  STREAM_WRITE_BLOCK_UPLOADS_COMMITTED("stream_write_block_uploads_committed",
+      "Count of number of block uploads committed"),
+  STREAM_WRITE_BLOCK_UPLOADS_ABORTED("stream_write_block_uploads_aborted",
+      "Count of number of block uploads aborted"),
+
+  STREAM_WRITE_BLOCK_UPLOADS_PENDING("stream_write_block_uploads_pending",
+      "Gauge of block/partitions uploads queued to be written"),
+  STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING(
+      "stream_write_block_uploads_data_pending",
+      "Gauge of block/partitions data uploads queued to be written"),
+  STREAM_WRITE_TOTAL_TIME("stream_write_total_time",
+      "Count of total time taken for uploads to complete"),
+  STREAM_WRITE_TOTAL_DATA("stream_write_total_data",
+      "Count of total data uploaded in block output"),
+  STREAM_WRITE_QUEUE_DURATION("stream_write_queue_duration",
+      "Total queue duration of all block uploads");
 
 
   private static final Map<String, Statistic> SYMBOL_MAP =
   private static final Map<String, Statistic> SYMBOL_MAP =
       new HashMap<>(Statistic.values().length);
       new HashMap<>(Statistic.values().length);

+ 621 - 47
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md

@@ -1,3 +1,4 @@
+
 <!---
 <!---
   Licensed under the Apache License, Version 2.0 (the "License");
   Licensed under the Apache License, Version 2.0 (the "License");
   you may not use this file except in compliance with the License.
   you may not use this file except in compliance with the License.
@@ -852,40 +853,361 @@ Seoul
 If the wrong endpoint is used, the request may fail. This may be reported as a 301/redirect error,
 If the wrong endpoint is used, the request may fail. This may be reported as a 301/redirect error,
 or as a 400 Bad Request.
 or as a 400 Bad Request.
 
 
-### S3AFastOutputStream
- **Warning: NEW in hadoop 2.7. UNSTABLE, EXPERIMENTAL: use at own risk**
 
 
-    <property>
-      <name>fs.s3a.fast.upload</name>
-      <value>false</value>
-      <description>Upload directly from memory instead of buffering to
-      disk first. Memory usage and parallelism can be controlled as up to
-      fs.s3a.multipart.size memory is consumed for each (part)upload actively
-      uploading (fs.s3a.threads.max) or queueing (fs.s3a.max.total.tasks)</description>
-    </property>
 
 
-    <property>
-      <name>fs.s3a.fast.buffer.size</name>
-      <value>1048576</value>
-      <description>Size (in bytes) of initial memory buffer allocated for an
-      upload. No effect if fs.s3a.fast.upload is false.</description>
-    </property>
+### <a name="s3a_fast_upload"></a>Stabilizing: S3A Fast Upload
+
+
+**New in Hadoop 2.7; significantly enhanced in Hadoop 2.9**
+
+
+Because of the nature of the S3 object store, data written to an S3A `OutputStream`
+is not written incrementally —instead, by default, it is buffered to disk
+until the stream is closed in its `close()` method.
+
+This can make output slow:
+
+* The execution time for `OutputStream.close()` is proportional to the amount of data
+buffered and inversely proportional to the bandwidth. That is `O(data/bandwidth)`.
+* The bandwidth is that available from the host to S3: other work in the same
+process, server or network at the time of upload may increase the upload time,
+hence the duration of the `close()` call.
+* If a process uploading data fails before `OutputStream.close()` is called,
+all data is lost.
+* The disks hosting temporary directories defined in `fs.s3a.buffer.dir` must
+have the capacity to store the entire buffered file.
+
+Put succinctly: the further the process is from the S3 endpoint, or the smaller
+the EC-hosted VM is, the longer it will take work to complete.
+
+This can create problems in application code:
+
+* Code often assumes that the `close()` call is fast;
+ the delays can create bottlenecks in operations.
+* Very slow uploads sometimes cause applications to time out. (generally,
+threads blocking during the upload stop reporting progress, so trigger timeouts)
+* Streaming very large amounts of data may consume all disk space before the upload begins.
+
+
+Work to addess this began in Hadoop 2.7 with the `S3AFastOutputStream`
+[HADOOP-11183](https://issues.apache.org/jira/browse/HADOOP-11183), and
+has continued with ` S3ABlockOutputStream`
+[HADOOP-13560](https://issues.apache.org/jira/browse/HADOOP-13560).
+
+
+This adds an alternative output stream, "S3a Fast Upload" which:
+
+1.  Always uploads large files as blocks with the size set by
+    `fs.s3a.multipart.size`. That is: the threshold at which multipart uploads
+    begin and the size of each upload are identical.
+1.  Buffers blocks to disk (default) or in on-heap or off-heap memory.
+1.  Uploads blocks in parallel in background threads.
+1.  Begins uploading blocks as soon as the buffered data exceeds this partition
+    size.
+1.  When buffering data to disk, uses the directory/directories listed in
+    `fs.s3a.buffer.dir`. The size of data which can be buffered is limited
+    to the available disk space.
+1.  Generates output statistics as metrics on the filesystem, including
+    statistics of active and pending block uploads.
+1.  Has the time to `close()` set by the amount of remaning data to upload, rather
+    than the total size of the file.
+
+With incremental writes of blocks, "S3A fast upload" offers an upload
+time at least as fast as the "classic" mechanism, with significant benefits
+on long-lived output streams, and when very large amounts of data are generated.
+The in memory buffering mechanims may also  offer speedup when running adjacent to
+S3 endpoints, as disks are not used for intermediate data storage.
+
+
+```xml
+<property>
+  <name>fs.s3a.fast.upload</name>
+  <value>true</value>
+  <description>
+    Use the incremental block upload mechanism with
+    the buffering mechanism set in fs.s3a.fast.upload.buffer.
+    The number of threads performing uploads in the filesystem is defined
+    by fs.s3a.threads.max; the queue of waiting uploads limited by
+    fs.s3a.max.total.tasks.
+    The size of each buffer is set by fs.s3a.multipart.size.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.fast.upload.buffer</name>
+  <value>disk</value>
+  <description>
+    The buffering mechanism to use when using S3A fast upload
+    (fs.s3a.fast.upload=true). Values: disk, array, bytebuffer.
+    This configuration option has no effect if fs.s3a.fast.upload is false.
+
+    "disk" will use the directories listed in fs.s3a.buffer.dir as
+    the location(s) to save data prior to being uploaded.
+
+    "array" uses arrays in the JVM heap
+
+    "bytebuffer" uses off-heap memory within the JVM.
+
+    Both "array" and "bytebuffer" will consume memory in a single stream up to the number
+    of blocks set by:
+
+        fs.s3a.multipart.size * fs.s3a.fast.upload.active.blocks.
+
+    If using either of these mechanisms, keep this value low
+
+    The total number of threads performing work across all threads is set by
+    fs.s3a.threads.max, with fs.s3a.max.total.tasks values setting the number of queued
+    work items.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.multipart.size</name>
+  <value>104857600</value>
+  <description>
+  How big (in bytes) to split upload or copy operations up into.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.fast.upload.active.blocks</name>
+  <value>8</value>
+  <description>
+    Maximum Number of blocks a single output stream can have
+    active (uploading, or queued to the central FileSystem
+    instance's pool of queued operations.
+
+    This stops a single stream overloading the shared thread pool.
+  </description>
+</property>
+```
+
+**Notes**
+
+* If the amount of data written to a stream is below that set in `fs.s3a.multipart.size`,
+the upload is performed in the `OutputStream.close()` operation —as with
+the original output stream.
+
+* The published Hadoop metrics monitor include live queue length and
+upload operation counts, so identifying when there is a backlog of work/
+a mismatch between data generation rates and network bandwidth. Per-stream
+statistics can also be logged by calling `toString()` on the current stream.
+
+* Incremental writes are not visible; the object can only be listed
+or read when the multipart operation completes in the `close()` call, which
+will block until the upload is completed.
+
+
+#### <a name="s3a_fast_upload_disk"></a>Fast Upload with Disk Buffers `fs.s3a.fast.upload.buffer=disk`
+
+When `fs.s3a.fast.upload.buffer` is set to `disk`, all data is buffered
+to local hard disks prior to upload. This minimizes the amount of memory
+consumed, and so eliminates heap size as the limiting factor in queued uploads
+—exactly as the original "direct to disk" buffering used when
+`fs.s3a.fast.upload=false`.
+
+
+```xml
+<property>
+  <name>fs.s3a.fast.upload</name>
+  <value>true</value>
+</property>
+
+<property>
+  <name>fs.s3a.fast.upload.buffer</name>
+  <value>disk</value>
+</property>
+
+```
+
+
+#### <a name="s3a_fast_upload_bytebuffer"></a>Fast Upload with ByteBuffers: `fs.s3a.fast.upload.buffer=bytebuffer`
+
+When `fs.s3a.fast.upload.buffer` is set to `bytebuffer`, all data is buffered
+in "Direct" ByteBuffers prior to upload. This *may* be faster than buffering to disk,
+and, if disk space is small (for example, tiny EC2 VMs), there may not
+be much disk space to buffer with.
+
+The ByteBuffers are created in the memory of the JVM, but not in the Java Heap itself.
+The amount of data which can be buffered is
+limited by the Java runtime, the operating system, and, for YARN applications,
+the amount of memory requested for each container.
+
+The slower the write bandwidth to S3, the greater the risk of running out
+of memory —and so the more care is needed in
+[tuning the upload settings](#s3a_fast_upload_thread_tuning).
+
+
+```xml
+<property>
+  <name>fs.s3a.fast.upload</name>
+  <value>true</value>
+</property>
+
+<property>
+  <name>fs.s3a.fast.upload.buffer</name>
+  <value>bytebuffer</value>
+</property>
+```
+
+#### <a name="s3a_fast_upload_array"></a>Fast Upload with Arrays: `fs.s3a.fast.upload.buffer=array`
+
+When `fs.s3a.fast.upload.buffer` is set to `array`, all data is buffered
+in byte arrays in the JVM's heap prior to upload.
+This *may* be faster than buffering to disk.
+
+This `array` option is similar to the in-memory-only stream offered in
+Hadoop 2.7 with `fs.s3a.fast.upload=true`
+
+The amount of data which can be buffered is limited by the available
+size of the JVM heap heap. The slower the write bandwidth to S3, the greater
+the risk of heap overflows. This risk can be mitigated by
+[tuning the upload settings](#s3a_fast_upload_thread_tuning).
+
+```xml
+<property>
+  <name>fs.s3a.fast.upload</name>
+  <value>true</value>
+</property>
+
+<property>
+  <name>fs.s3a.fast.upload.buffer</name>
+  <value>array</value>
+</property>
+
+```
+#### <a name="s3a_fast_upload_thread_tuning"></a>S3A Fast Upload Thread Tuning
+
+Both the [Array](#s3a_fast_upload_array) and [Byte buffer](#s3a_fast_upload_bytebuffer)
+buffer mechanisms can consume very large amounts of memory, on-heap or
+off-heap respectively. The [disk buffer](#s3a_fast_upload_disk) mechanism
+does not use much memory up, but will consume hard disk capacity.
+
+If there are many output streams being written to in a single process, the
+amount of memory or disk used is the multiple of all stream's active memory/disk use.
+
+Careful tuning may be needed to reduce the risk of running out memory, especially
+if the data is buffered in memory.
+
+There are a number parameters which can be tuned:
+
+1. The total number of threads available in the filesystem for data
+uploads *or any other queued filesystem operation*. This is set in
+`fs.s3a.threads.max`
+
+1. The number of operations which can be queued for execution:, *awaiting
+a thread*: `fs.s3a.max.total.tasks`
+
+1. The number of blocks which a single output stream can have active,
+that is: being uploaded by a thread, or queued in the filesystem thread queue:
+`fs.s3a.fast.upload.active.blocks`
+
+1. How long an idle thread can stay in the thread pool before it is retired: `fs.s3a.threads.keepalivetime`
+
+
+When the maximum allowed number of active blocks of a single stream is reached,
+no more blocks can be uploaded from that stream until one or more of those active
+blocks' uploads completes. That is: a `write()` call which would trigger an upload
+of a now full datablock, will instead block until there is capacity in the queue.
+
+How does that come together?
+
+* As the pool of threads set in `fs.s3a.threads.max` is shared (and intended
+to be used across all threads), a larger number here can allow for more
+parallel operations. However, as uploads require network bandwidth, adding more
+threads does not guarantee speedup.
+
+* The extra queue of tasks for the thread pool (`fs.s3a.max.total.tasks`)
+covers all ongoing background S3A operations (future plans include: parallelized
+rename operations, asynchronous directory operations).
+
+* When using memory buffering, a small value of `fs.s3a.fast.upload.active.blocks`
+limits the amount of memory which can be consumed per stream.
+
+* When using disk buffering a larger value of `fs.s3a.fast.upload.active.blocks`
+does not consume much memory. But it may result in a large number of blocks to
+compete with other filesystem operations.
+
+
+We recommend a low value of `fs.s3a.fast.upload.active.blocks`; enough
+to start background upload without overloading other parts of the system,
+then experiment to see if higher values deliver more throughtput —especially
+from VMs running on EC2.
+
+```xml
+
+<property>
+  <name>fs.s3a.fast.upload.active.blocks</name>
+  <value>4</value>
+  <description>
+    Maximum Number of blocks a single output stream can have
+    active (uploading, or queued to the central FileSystem
+    instance's pool of queued operations.
+
+    This stops a single stream overloading the shared thread pool.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.threads.max</name>
+  <value>10</value>
+  <description>The total number of threads available in the filesystem for data
+    uploads *or any other queued filesystem operation*.</description>
+</property>
+
+<property>
+  <name>fs.s3a.max.total.tasks</name>
+  <value>5</value>
+  <description>The number of operations which can be queued for execution</description>
+</property>
+
+<property>
+  <name>fs.s3a.threads.keepalivetime</name>
+  <value>60</value>
+  <description>Number of seconds a thread can be idle before being
+    terminated.</description>
+</property>
+
+```
+
+
+#### <a name="s3a_multipart_purge"></a>Cleaning up After Incremental Upload Failures: `fs.s3a.multipart.purge`
+
+
+If an incremental streaming operation is interrupted, there may be
+intermediate partitions uploaded to S3 —data which will be billed for.
+
+These charges can be reduced by enabling `fs.s3a.multipart.purge`,
+and setting a purge time in seconds, such as 86400 seconds —24 hours.
+When an S3A FileSystem instance is instantiated with the purge time greater
+than zero, it will, on startup, delete all outstanding partition requests
+older than this time.
+
+```xml
+<property>
+  <name>fs.s3a.multipart.purge</name>
+  <value>true</value>
+  <description>True if you want to purge existing multipart uploads that may not have been
+     completed/aborted correctly</description>
+</property>
+
+<property>
+  <name>fs.s3a.multipart.purge.age</name>
+  <value>86400</value>
+  <description>Minimum age in seconds of multipart uploads to purge</description>
+</property>
+```
+
+If an S3A client is instantited with `fs.s3a.multipart.purge=true`,
+it will delete all out of date uploads *in the entire bucket*. That is: it will affect all
+multipart uploads to that bucket, from all applications.
 
 
-Writes are buffered in memory instead of to a file on local disk. This
-removes the throughput bottleneck of the local disk write and read cycle
-before starting the actual upload. Furthermore, it allows handling files that
-are larger than the remaining local disk space.
-
-However, non-trivial memory tuning is needed for optimal results and careless
-settings could cause memory overflow. Up to `fs.s3a.threads.max` parallel
-(part)uploads are active. Furthermore, up to `fs.s3a.max.total.tasks`
-additional part(uploads) can be waiting (and thus memory buffers are created).
-The memory buffer is uploaded as a single upload if it is not larger than
-`fs.s3a.multipart.threshold`. Else, a multi-part upload is initiated and
-parts of size `fs.s3a.multipart.size` are used to protect against overflowing
-the available memory. These settings should be tuned to the envisioned
-workflow (some large files, many small ones, ...) and the physical
-limitations of the machine and cluster (memory, network bandwidth).
+Leaving `fs.s3a.multipart.purge` to its default, `false`,
+means that the client will not make any attempt to reset or change the partition
+rate.
+
+The best practise for using this option is to disable multipart purges in
+normal use of S3A, enabling only in manual/scheduled housekeeping operations.
 
 
 ### S3A Experimental "fadvise" input policy support
 ### S3A Experimental "fadvise" input policy support
 
 
@@ -1221,7 +1543,143 @@ can be used:
 Using the explicit endpoint for the region is recommended for speed and the
 Using the explicit endpoint for the region is recommended for speed and the
 ability to use the V4 signing API.
 ability to use the V4 signing API.
 
 
-## Visible S3 Inconsistency
+
+### "Timeout waiting for connection from pool" when writing to S3A
+
+This happens when using the Block output stream, `fs.s3a.fast.upload=true` and
+the thread pool runs out of capacity.
+
+```
+[s3a-transfer-shared-pool1-t20] INFO  http.AmazonHttpClient (AmazonHttpClient.java:executeHelper(496)) - Unable to execute HTTP request: Timeout waiting for connection from poolorg.apache.http.conn.ConnectionPoolTimeoutException: Timeout waiting for connection from pool
+  at org.apache.http.impl.conn.PoolingClientConnectionManager.leaseConnection(PoolingClientConnectionManager.java:230)
+  at org.apache.http.impl.conn.PoolingClientConnectionManager$1.getConnection(PoolingClientConnectionManager.java:199)
+  at sun.reflect.GeneratedMethodAccessor13.invoke(Unknown Source)
+  at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
+  at java.lang.reflect.Method.invoke(Method.java:498)
+  at com.amazonaws.http.conn.ClientConnectionRequestFactory$Handler.invoke(ClientConnectionRequestFactory.java:70)
+  at com.amazonaws.http.conn.$Proxy10.getConnection(Unknown Source)
+  at org.apache.http.impl.client.DefaultRequestDirector.execute(DefaultRequestDirector.java:424)
+  at org.apache.http.impl.client.AbstractHttpClient.doExecute(AbstractHttpClient.java:884)
+  at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:82)
+  at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:55)
+  at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:728)
+  at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
+  at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
+  at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
+  at com.amazonaws.services.s3.AmazonS3Client.doUploadPart(AmazonS3Client.java:2921)
+  at com.amazonaws.services.s3.AmazonS3Client.uploadPart(AmazonS3Client.java:2906)
+  at org.apache.hadoop.fs.s3a.S3AFileSystem.uploadPart(S3AFileSystem.java:1025)
+  at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload$1.call(S3ABlockOutputStream.java:360)
+  at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload$1.call(S3ABlockOutputStream.java:355)
+  at org.apache.hadoop.fs.s3a.BlockingThreadPoolExecutorService$CallableWithPermitRelease.call(BlockingThreadPoolExecutorService.java:239)
+  at java.util.concurrent.FutureTask.run(FutureTask.java:266)
+  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
+  at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
+  at java.lang.Thread.run(Thread.java:745)
+```
+
+Make sure that `fs.s3a.connection.maximum` is at least larger
+than `fs.s3a.threads.max`.
+
+```xml
+<property>
+  <name>fs.s3a.threads.max</name>
+  <value>20</value>
+</property>
+
+<property>
+  <name>fs.s3a.connection.maximum</name>
+  <value>30</value>
+</property>
+```
+
+### "Timeout waiting for connection from pool" when reading from S3A
+
+This happens when more threads are trying to read from an S3A system than
+the maximum number of allocated HTTP connections.
+
+Set `fs.s3a.connection.maximum` to a larger value (and at least as large as
+`fs.s3a.threads.max`)
+
+### Out of heap memory when writing to S3A via Fast Upload
+
+This can happen when using the fast upload mechanism (`fs.s3a.fast.upload=true`)
+and in-memory buffering (either `fs.s3a.fast.upload.buffer=array` or
+`fs.s3a.fast.upload.buffer=bytebuffer`).
+
+More data is being generated than in the JVM than it can upload to S3 —and
+so much data has been buffered that the JVM has run out of memory.
+
+Consult [S3A Fast Upload Thread Tuning](#s3a_fast_upload_thread_tuning) for
+detail on this issue and options to address it. Consider also buffering to
+disk, rather than memory.
+
+
+### When writing to S3A: "java.io.FileNotFoundException: Completing multi-part upload"
+
+
+```
+java.io.FileNotFoundException: Completing multi-part upload on fork-5/test/multipart/1c397ca6-9dfb-4ac1-9cf7-db666673246b: com.amazonaws.services.s3.model.AmazonS3Exception: The specified upload does not exist. The upload ID may be invalid, or the upload may have been aborted or completed. (Service: Amazon S3; Status Code: 404; Error Code: NoSuchUpload; Request ID: 84FF8057174D9369), S3 Extended Request ID: Ij5Yn6Eq/qIERH4Z6Io3YL2t9/qNZ7z9gjPb1FrTtTovZ8k1MXqh+zCYYjqmfJ/fCY6E1+JR9jA=
+  at com.amazonaws.http.AmazonHttpClient.handleErrorResponse(AmazonHttpClient.java:1182)
+  at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:770)
+  at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
+  at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
+  at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
+  at com.amazonaws.services.s3.AmazonS3Client.completeMultipartUpload(AmazonS3Client.java:2705)
+  at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.complete(S3ABlockOutputStream.java:473)
+  at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.access$200(S3ABlockOutputStream.java:382)
+  at org.apache.hadoop.fs.s3a.S3ABlockOutputStream.close(S3ABlockOutputStream.java:272)
+  at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72)
+  at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106)
+```
+
+This surfaces if, while a multipart upload was taking place, all outstanding multipart
+uploads were garbage collected. The upload operation cannot complete because
+the data uploaded has been deleted.
+
+Consult [Cleaning up After Incremental Upload Failures](#s3a_multipart_purge) for
+details on how the multipart purge timeout can be set. If multipart uploads
+are failing with the message above, it may be a sign that this value is too low.
+
+### When writing to S3A, HTTP Exceptions logged at info from `AmazonHttpClient`
+
+```
+[s3a-transfer-shared-pool4-t6] INFO  http.AmazonHttpClient (AmazonHttpClient.java:executeHelper(496)) - Unable to execute HTTP request: hwdev-steve-ireland-new.s3.amazonaws.com:443 failed to respond
+org.apache.http.NoHttpResponseException: bucket.s3.amazonaws.com:443 failed to respond
+  at org.apache.http.impl.conn.DefaultHttpResponseParser.parseHead(DefaultHttpResponseParser.java:143)
+  at org.apache.http.impl.conn.DefaultHttpResponseParser.parseHead(DefaultHttpResponseParser.java:57)
+  at org.apache.http.impl.io.AbstractMessageParser.parse(AbstractMessageParser.java:261)
+  at org.apache.http.impl.AbstractHttpClientConnection.receiveResponseHeader(AbstractHttpClientConnection.java:283)
+  at org.apache.http.impl.conn.DefaultClientConnection.receiveResponseHeader(DefaultClientConnection.java:259)
+  at org.apache.http.impl.conn.ManagedClientConnectionImpl.receiveResponseHeader(ManagedClientConnectionImpl.java:209)
+  at org.apache.http.protocol.HttpRequestExecutor.doReceiveResponse(HttpRequestExecutor.java:272)
+  at com.amazonaws.http.protocol.SdkHttpRequestExecutor.doReceiveResponse(SdkHttpRequestExecutor.java:66)
+  at org.apache.http.protocol.HttpRequestExecutor.execute(HttpRequestExecutor.java:124)
+  at org.apache.http.impl.client.DefaultRequestDirector.tryExecute(DefaultRequestDirector.java:686)
+  at org.apache.http.impl.client.DefaultRequestDirector.execute(DefaultRequestDirector.java:488)
+  at org.apache.http.impl.client.AbstractHttpClient.doExecute(AbstractHttpClient.java:884)
+  at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:82)
+  at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:55)
+  at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:728)
+  at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
+  at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
+  at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
+  at com.amazonaws.services.s3.AmazonS3Client.copyPart(AmazonS3Client.java:1731)
+  at com.amazonaws.services.s3.transfer.internal.CopyPartCallable.call(CopyPartCallable.java:41)
+  at com.amazonaws.services.s3.transfer.internal.CopyPartCallable.call(CopyPartCallable.java:28)
+  at org.apache.hadoop.fs.s3a.BlockingThreadPoolExecutorService$CallableWithPermitRelease.call(BlockingThreadPoolExecutorService.java:239)
+  at java.util.concurrent.FutureTask.run(FutureTask.java:266)
+  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
+  at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
+  at java.lang.Thread.run(Thread.java:745)
+```
+
+These are HTTP I/O exceptions caught and logged inside the AWS SDK. The client
+will attempt to retry the operation; it may just be a transient event. If there
+are many such exceptions in logs, it may be a symptom of connectivity or network
+problems.
+
+### Visible S3 Inconsistency
 
 
 Amazon S3 is *an eventually consistent object store*. That is: not a filesystem.
 Amazon S3 is *an eventually consistent object store*. That is: not a filesystem.
 
 
@@ -1564,7 +2022,7 @@ tests or the `it.test` property for integration tests.
 
 
     mvn clean test -Dtest=TestS3AInputPolicies
     mvn clean test -Dtest=TestS3AInputPolicies
 
 
-    mvn clean verify -Dit.test=ITestS3AFileContextStatistics
+    mvn clean verify -Dit.test=ITestS3AFileContextStatistics -Dtest=none
 
 
     mvn clean verify -Dtest=TestS3A* -Dit.test=ITestS3A*
     mvn clean verify -Dtest=TestS3A* -Dit.test=ITestS3A*
 
 
@@ -1614,7 +2072,7 @@ An alternate endpoint may be defined in `test.fs.s3a.sts.endpoint`.
 
 
 The default is ""; meaning "use the amazon default value".
 The default is ""; meaning "use the amazon default value".
 
 
-#### CSV Data source Tests
+### CSV Data source Tests
 
 
 The `TestS3AInputStreamPerformance` tests require read access to a multi-MB
 The `TestS3AInputStreamPerformance` tests require read access to a multi-MB
 text file. The default file for these tests is one published by amazon,
 text file. The default file for these tests is one published by amazon,
@@ -1661,18 +2119,89 @@ endpoint:
   <value>s3.amazonaws.com</value>
   <value>s3.amazonaws.com</value>
 </property>
 </property>
 ```
 ```
+### Viewing Integration Test Reports
 
 
-#### Scale test operation count
+
+Integration test results and logs are stored in `target/failsafe-reports/`.
+An HTML report can be generated during site generation, or with the `surefire-report`
+plugin:
+
+```
+mvn surefire-report:failsafe-report-only
+```
+### Scale Tests
+
+There are a set of tests designed to measure the scalability and performance
+at scale of the S3A tests, *Scale Tests*. Tests include: creating
+and traversing directory trees, uploading large files, renaming them,
+deleting them, seeking through the files, performing random IO, and others.
+This makes them a foundational part of the benchmarking.
+
+By their very nature they are slow. And, as their execution time is often
+limited by bandwidth between the computer running the tests and the S3 endpoint,
+parallel execution does not speed these tests up.
+
+#### Enabling the Scale Tests
+
+The tests are enabled if the `scale` property is set in the maven build
+this can be done regardless of whether or not the parallel test profile
+is used
+
+```bash
+mvn verify -Dscale
+
+mvn verify -Dparallel-tests -Dscale -DtestsThreadCount=8
+```
+
+The most bandwidth intensive tests (those which upload data) always run
+sequentially; those which are slow due to HTTPS setup costs or server-side
+actionsare included in the set of parallelized tests.
+
+
+#### Maven build tuning options
+
+
+Some of the tests can be tuned from the maven build or from the
+configuration file used to run the tests.
+
+```bash
+mvn verify -Dscale -Dfs.s3a.scale.test.huge.filesize=128M
+```
+
+The algorithm is
+
+1. The value is queried from the configuration file, using a default value if
+it is not set.
+1. The value is queried from the JVM System Properties, where it is passed
+down by maven.
+1. If the system property is null, empty, or it has the value `unset`, then
+the configuration value is used. The `unset` option is used to
+[work round a quirk in maven property propagation](http://stackoverflow.com/questions/7773134/null-versus-empty-arguments-in-maven).
+
+Only a few properties can be set this way; more will be added.
+
+| Property | Meaninging |
+|-----------|-------------|
+| `fs.s3a.scale.test.timeout`| Timeout in seconds for scale tests |
+| `fs.s3a.scale.test.huge.filesize`| Size for huge file uploads |
+| `fs.s3a.scale.test.huge.huge.partitionsize`| Size for partitions in huge file uploads |
+
+The file and partition sizes are numeric values with a k/m/g/t/p suffix depending
+on the desired size. For example: 128M, 128m, 2G, 2G, 4T or even 1P.
+
+#### Scale test configuration options
 
 
 Some scale tests perform multiple operations (such as creating many directories).
 Some scale tests perform multiple operations (such as creating many directories).
 
 
 The exact number of operations to perform is configurable in the option
 The exact number of operations to perform is configurable in the option
 `scale.test.operation.count`
 `scale.test.operation.count`
 
 
-      <property>
-        <name>scale.test.operation.count</name>
-        <value>10</value>
-      </property>
+```xml
+<property>
+  <name>scale.test.operation.count</name>
+  <value>10</value>
+</property>
+```
 
 
 Larger values generate more load, and are recommended when testing locally,
 Larger values generate more load, and are recommended when testing locally,
 or in batch runs.
 or in batch runs.
@@ -1685,19 +2214,64 @@ the width and depth of tests creating recursive directories. Larger
 values create exponentially more directories, with consequent performance
 values create exponentially more directories, with consequent performance
 impact.
 impact.
 
 
-      <property>
-        <name>scale.test.directory.count</name>
-        <value>2</value>
-      </property>
+```xml
+<property>
+  <name>scale.test.directory.count</name>
+  <value>2</value>
+</property>
+```
 
 
 DistCp tests targeting S3A support a configurable file size.  The default is
 DistCp tests targeting S3A support a configurable file size.  The default is
 10 MB, but the configuration value is expressed in KB so that it can be tuned
 10 MB, but the configuration value is expressed in KB so that it can be tuned
 smaller to achieve faster test runs.
 smaller to achieve faster test runs.
 
 
-      <property>
-        <name>scale.test.distcp.file.size.kb</name>
-        <value>10240</value>
-      </property>
+```xml
+<property>
+  <name>scale.test.distcp.file.size.kb</name>
+  <value>10240</value>
+</property>
+```
+
+S3A specific scale test properties are
+
+##### `fs.s3a.scale.test.huge.filesize`: size in MB for "Huge file tests".
+
+The Huge File tests validate S3A's ability to handle large files —the property
+`fs.s3a.scale.test.huge.filesize` declares the file size to use.
+
+```xml
+<property>
+  <name>fs.s3a.scale.test.huge.filesize</name>
+  <value>200M</value>
+</property>
+```
+
+Amazon S3 handles files larger than 5GB differently than smaller ones.
+Setting the huge filesize to a number greater than that) validates support
+for huge files.
+
+```xml
+<property>
+  <name>fs.s3a.scale.test.huge.filesize</name>
+  <value>6G</value>
+</property>
+```
+
+Tests at this scale are slow: they are best executed from hosts running in
+the cloud infrastructure where the S3 endpoint is based.
+Otherwise, set a large timeout in `fs.s3a.scale.test.timeout`
+
+```xml
+<property>
+  <name>fs.s3a.scale.test.timeout</name>
+  <value>432000</value>
+</property>
+```
+
+
+The tests are executed in an order to only clean up created files after
+the end of all the tests. If the tests are interrupted, the test data will remain.
+
 
 
 
 
 ### Testing against non AWS S3 endpoints.
 ### Testing against non AWS S3 endpoints.

+ 6 - 4
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java

@@ -18,24 +18,26 @@
 
 
 package org.apache.hadoop.fs.contract.s3a;
 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 static org.apache.hadoop.fs.s3a.Constants.*;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.tools.contract.AbstractContractDistCpTest;
 import org.apache.hadoop.tools.contract.AbstractContractDistCpTest;
 
 
 /**
 /**
  * Contract test suite covering S3A integration with DistCp.
  * Contract test suite covering S3A integration with DistCp.
+ * Uses the block output stream, buffered to disk. This is the
+ * recommended output mechanism for DistCP due to its scalability.
  */
  */
 public class ITestS3AContractDistCp extends AbstractContractDistCpTest {
 public class ITestS3AContractDistCp extends AbstractContractDistCpTest {
 
 
-  private static final long MULTIPART_SETTING = 8 * 1024 * 1024; // 8 MB
+  private static final long MULTIPART_SETTING = MULTIPART_MIN_SIZE;
 
 
   @Override
   @Override
   protected Configuration createConfiguration() {
   protected Configuration createConfiguration() {
     Configuration newConf = super.createConfiguration();
     Configuration newConf = super.createConfiguration();
-    newConf.setLong(MIN_MULTIPART_THRESHOLD, MULTIPART_SETTING);
     newConf.setLong(MULTIPART_SIZE, MULTIPART_SETTING);
     newConf.setLong(MULTIPART_SIZE, MULTIPART_SETTING);
+    newConf.setBoolean(FAST_UPLOAD, true);
+    newConf.set(FAST_UPLOAD_BUFFER, FAST_UPLOAD_BUFFER_DISK);
     return newConf;
     return newConf;
   }
   }
 
 

+ 1 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java

@@ -48,6 +48,7 @@ public abstract class AbstractS3ATestBase extends AbstractFSContractTestBase
   @Override
   @Override
   public void teardown() throws Exception {
   public void teardown() throws Exception {
     super.teardown();
     super.teardown();
+    describe("closing file system");
     IOUtils.closeStream(getFileSystem());
     IOUtils.closeStream(getFileSystem());
   }
   }
 
 

+ 38 - 10
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestBlockingThreadPoolExecutorService.java

@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
  * distributed with this work for additional information
@@ -20,18 +20,23 @@ package org.apache.hadoop.fs.s3a;
 
 
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.ListenableFuture;
 import org.apache.hadoop.util.StopWatch;
 import org.apache.hadoop.util.StopWatch;
-import org.junit.*;
+
+import org.junit.AfterClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
 import java.util.concurrent.Callable;
 import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeUnit;
 
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertFalse;
 
 
 /**
 /**
- * Basic unit test for S3A's blocking executor service.
+ * Basic test for S3A's blocking executor service.
  */
  */
 public class ITestBlockingThreadPoolExecutorService {
 public class ITestBlockingThreadPoolExecutorService {
 
 
@@ -47,7 +52,10 @@ public class ITestBlockingThreadPoolExecutorService {
 
 
   private static final Integer SOME_VALUE = 1337;
   private static final Integer SOME_VALUE = 1337;
 
 
-  private static BlockingThreadPoolExecutorService tpe = null;
+  private static BlockingThreadPoolExecutorService tpe;
+
+  @Rule
+  public Timeout testTimeout = new Timeout(60 * 1000);
 
 
   @AfterClass
   @AfterClass
   public static void afterClass() throws Exception {
   public static void afterClass() throws Exception {
@@ -71,13 +79,23 @@ public class ITestBlockingThreadPoolExecutorService {
   @Test
   @Test
   public void testSubmitRunnable() throws Exception {
   public void testSubmitRunnable() throws Exception {
     ensureCreated();
     ensureCreated();
-    int totalTasks = NUM_ACTIVE_TASKS + NUM_WAITING_TASKS;
+    verifyQueueSize(tpe, NUM_ACTIVE_TASKS + NUM_WAITING_TASKS);
+  }
+
+  /**
+   * Verify the size of the executor's queue, by verifying that the first
+   * submission to block is {@code expectedQueueSize + 1}.
+   * @param executorService executor service to test
+   * @param expectedQueueSize size of queue
+   */
+  protected void verifyQueueSize(ExecutorService executorService,
+      int expectedQueueSize) {
     StopWatch stopWatch = new StopWatch().start();
     StopWatch stopWatch = new StopWatch().start();
-    for (int i = 0; i < totalTasks; i++) {
-      tpe.submit(sleeper);
+    for (int i = 0; i < expectedQueueSize; i++) {
+      executorService.submit(sleeper);
       assertDidntBlock(stopWatch);
       assertDidntBlock(stopWatch);
     }
     }
-    tpe.submit(sleeper);
+    executorService.submit(sleeper);
     assertDidBlock(stopWatch);
     assertDidBlock(stopWatch);
   }
   }
 
 
@@ -93,6 +111,15 @@ public class ITestBlockingThreadPoolExecutorService {
     ensureDestroyed();
     ensureDestroyed();
   }
   }
 
 
+  @Test
+  public void testChainedQueue() throws Throwable {
+    ensureCreated();
+    int size = 2;
+    ExecutorService wrapper = new SemaphoredDelegatingExecutor(tpe,
+        size, true);
+    verifyQueueSize(wrapper, size);
+  }
+
   // Helper functions, etc.
   // Helper functions, etc.
 
 
   private void assertDidntBlock(StopWatch sw) {
   private void assertDidntBlock(StopWatch sw) {
@@ -141,8 +168,9 @@ public class ITestBlockingThreadPoolExecutorService {
   private static void ensureCreated() throws Exception {
   private static void ensureCreated() throws Exception {
     if (tpe == null) {
     if (tpe == null) {
       LOG.debug("Creating thread pool");
       LOG.debug("Creating thread pool");
-      tpe = new BlockingThreadPoolExecutorService(NUM_ACTIVE_TASKS,
-          NUM_WAITING_TASKS, 1, TimeUnit.SECONDS, "btpetest");
+      tpe = BlockingThreadPoolExecutorService.newInstance(
+          NUM_ACTIVE_TASKS, NUM_WAITING_TASKS,
+          1, TimeUnit.SECONDS, "btpetest");
     }
     }
   }
   }
 
 

+ 90 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java

@@ -0,0 +1,90 @@
+/*
+ * 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.FSDataOutputStream;
+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.s3a.Constants.*;
+
+/**
+ * Tests small file upload functionality for
+ * {@link S3ABlockOutputStream} with the blocks buffered in byte arrays.
+ *
+ * File sizes are kept small to reduce test duration on slow connections;
+ * multipart tests are kept in scale tests.
+ */
+public class ITestS3ABlockOutputArray extends AbstractS3ATestBase {
+
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    S3ATestUtils.disableFilesystemCaching(conf);
+    conf.setLong(MIN_MULTIPART_THRESHOLD, MULTIPART_MIN_SIZE);
+    conf.setInt(MULTIPART_SIZE, MULTIPART_MIN_SIZE);
+    conf.setBoolean(Constants.FAST_UPLOAD, true);
+    conf.set(FAST_UPLOAD_BUFFER, getBlockOutputBufferName());
+    return conf;
+  }
+
+  protected String getBlockOutputBufferName() {
+    return FAST_UPLOAD_BUFFER_ARRAY;
+  }
+
+  @Test
+  public void testZeroByteUpload() throws IOException {
+    verifyUpload("0", 0);
+  }
+
+  @Test
+  public void testRegularUpload() throws IOException {
+    verifyUpload("regular", 1024);
+  }
+
+  @Test(expected = IOException.class)
+  public void testDoubleStreamClose() throws Throwable {
+    Path dest = path("testDoubleStreamClose");
+    describe(" testDoubleStreamClose");
+    FSDataOutputStream stream = getFileSystem().create(dest, true);
+    byte[] data = ContractTestUtils.dataset(16, 'a', 26);
+    try {
+      stream.write(data);
+      stream.close();
+      stream.write(data);
+    } finally {
+      IOUtils.closeStream(stream);
+    }
+  }
+
+  public void verifyUpload(String name, int fileSize) throws IOException {
+    Path dest = path(name);
+    describe(name + " upload to " + dest);
+    ContractTestUtils.createAndVerifyFile(
+        getFileSystem(),
+        dest,
+        fileSize);
+  }
+}

+ 30 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputByteBuffer.java

@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ * Use {@link Constants#FAST_UPLOAD_BYTEBUFFER} for buffering.
+ */
+public class ITestS3ABlockOutputByteBuffer extends ITestS3ABlockOutputArray {
+
+  protected String getBlockOutputBufferName() {
+    return Constants.FAST_UPLOAD_BYTEBUFFER;
+  }
+
+}

+ 30 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputDisk.java

@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ * Use {@link Constants#FAST_UPLOAD_BUFFER_DISK} for buffering.
+ */
+public class ITestS3ABlockOutputDisk extends ITestS3ABlockOutputArray {
+
+  protected String getBlockOutputBufferName() {
+    return Constants.FAST_UPLOAD_BUFFER_DISK;
+  }
+
+}

+ 2 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockingThreadPool.java

@@ -72,6 +72,8 @@ public class ITestS3ABlockingThreadPool {
   @Test
   @Test
   public void testFastMultiPartUpload() throws Exception {
   public void testFastMultiPartUpload() throws Exception {
     conf.setBoolean(Constants.FAST_UPLOAD, true);
     conf.setBoolean(Constants.FAST_UPLOAD, true);
+    conf.set(Constants.FAST_UPLOAD_BUFFER,
+        Constants.FAST_UPLOAD_BYTEBUFFER);
     fs = S3ATestUtils.createTestFileSystem(conf);
     fs = S3ATestUtils.createTestFileSystem(conf);
     ContractTestUtils.createAndVerifyFile(fs, getTestPath(), 16 * 1024 *
     ContractTestUtils.createAndVerifyFile(fs, getTestPath(), 16 * 1024 *
         1024);
         1024);

+ 29 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java

@@ -28,6 +28,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.s3native.S3xLoginHelper;
 import org.apache.hadoop.fs.s3native.S3xLoginHelper;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Rule;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.Test;
 import org.junit.rules.Timeout;
 import org.junit.rules.Timeout;
@@ -35,6 +36,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
@@ -417,6 +419,33 @@ public class ITestS3AConfiguration {
     fs.close();
     fs.close();
   }
   }
 
 
+  @Test
+  public void testDirectoryAllocatorDefval() throws Throwable {
+    conf = new Configuration();
+    conf.unset(Constants.BUFFER_DIR);
+    fs = S3ATestUtils.createTestFileSystem(conf);
+    File tmp = fs.createTmpFileForWrite("out-", 1024, conf);
+    assertTrue("not found: " + tmp, tmp.exists());
+    tmp.delete();
+  }
+
+  @Test
+  public void testDirectoryAllocatorRR() throws Throwable {
+    File dir1 = GenericTestUtils.getRandomizedTestDir();
+    File dir2 = GenericTestUtils.getRandomizedTestDir();
+    dir1.mkdirs();
+    dir2.mkdirs();
+    conf = new Configuration();
+    conf.set(Constants.BUFFER_DIR, dir1 +", " + dir2);
+    fs = S3ATestUtils.createTestFileSystem(conf);
+    File tmp1 = fs.createTmpFileForWrite("out-", 1024, conf);
+    tmp1.delete();
+    File tmp2 = fs.createTmpFileForWrite("out-", 1024, conf);
+    tmp2.delete();
+    assertNotEquals("round robin not working",
+        tmp1.getParent(), tmp2.getParent());
+  }
+
   /**
   /**
    * Reads and returns a field from an object using reflection.  If the field
    * 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
    * cannot be found, is null, or is not the expected type, then this method

+ 4 - 3
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionFastOutputStream.java → hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionBlockOutputStream.java

@@ -21,15 +21,16 @@ package org.apache.hadoop.fs.s3a;
 import org.apache.hadoop.conf.Configuration;
 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.
+ * Run the encryption tests against the block output stream.
  */
  */
-public class ITestS3AEncryptionFastOutputStream extends ITestS3AEncryption {
+public class ITestS3AEncryptionBlockOutputStream extends ITestS3AEncryption {
 
 
   @Override
   @Override
   protected Configuration createConfiguration() {
   protected Configuration createConfiguration() {
     Configuration conf = super.createConfiguration();
     Configuration conf = super.createConfiguration();
     conf.setBoolean(Constants.FAST_UPLOAD, true);
     conf.setBoolean(Constants.FAST_UPLOAD, true);
+    conf.set(Constants.FAST_UPLOAD_BUFFER,
+        Constants.FAST_UPLOAD_BYTEBUFFER);
     return conf;
     return conf;
   }
   }
 }
 }

+ 0 - 74
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFastOutputStream.java

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

+ 98 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ATestUtils.java

@@ -0,0 +1,98 @@
+/*
+ * 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.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
+
+/**
+ * Test the test utils. Why an integration test? it's needed to
+ * verify property pushdown.
+ */
+public class ITestS3ATestUtils extends Assert {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ITestS3ATestUtils.class);
+  public static final String KEY = "undefined.property";
+
+  @Before
+  public void clear() {
+    System.clearProperty(KEY);
+  }
+
+  @Test
+  public void testGetTestProperty() throws Throwable {
+    Configuration conf = new Configuration(false);
+    assertEquals("a", getTestProperty(conf, KEY, "a"));
+    conf.set(KEY, "\t b \n");
+    assertEquals("b", getTestProperty(conf, KEY, "a"));
+    System.setProperty(KEY, "c");
+    assertEquals("c", getTestProperty(conf, KEY, "a"));
+    unsetSysprop();
+    assertEquals("b", getTestProperty(conf, KEY, "a"));
+  }
+
+  @Test
+  public void testGetTestPropertyLong() throws Throwable {
+    Configuration conf = new Configuration(false);
+    assertEquals(1, getTestPropertyLong(conf, KEY, 1));
+    conf.setInt(KEY, 2);
+    assertEquals(2, getTestPropertyLong(conf, KEY, 1));
+    System.setProperty(KEY, "3");
+    assertEquals(3, getTestPropertyLong(conf, KEY, 1));
+  }
+
+  @Test
+  public void testGetTestPropertyInt() throws Throwable {
+    Configuration conf = new Configuration(false);
+    assertEquals(1, getTestPropertyInt(conf, KEY, 1));
+    conf.setInt(KEY, 2);
+    assertEquals(2, getTestPropertyInt(conf, KEY, 1));
+    System.setProperty(KEY, "3");
+    assertEquals(3, getTestPropertyInt(conf, KEY, 1));
+    conf.unset(KEY);
+    assertEquals(3, getTestPropertyInt(conf, KEY, 1));
+    unsetSysprop();
+    assertEquals(5, getTestPropertyInt(conf, KEY, 5));
+  }
+
+  @Test
+  public void testGetTestPropertyBool() throws Throwable {
+    Configuration conf = new Configuration(false);
+    assertTrue(getTestPropertyBool(conf, KEY, true));
+    conf.set(KEY, "\tfalse \n");
+    assertFalse(getTestPropertyBool(conf, KEY, true));
+    System.setProperty(KEY, "true");
+    assertTrue(getTestPropertyBool(conf, KEY, true));
+    unsetSysprop();
+    assertEquals("false", getTestProperty(conf, KEY, "true"));
+    conf.unset(KEY);
+    assertTrue(getTestPropertyBool(conf, KEY, true));
+  }
+
+  protected void unsetSysprop() {
+    System.setProperty(KEY, UNSET_PROPERTY);
+  }
+
+}

+ 73 - 2
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java

@@ -43,15 +43,36 @@ public interface S3ATestConstants {
    */
    */
   String TEST_FS_S3A_NAME = TEST_FS_S3A + "name";
   String TEST_FS_S3A_NAME = TEST_FS_S3A + "name";
 
 
+  /**
+   * Run the encryption tests?
+   */
+  String KEY_ENCRYPTION_TESTS = TEST_FS_S3A + "encryption.enabled";
+
+  /**
+   * Tell tests that they are being executed in parallel: {@value}.
+   */
+  String KEY_PARALLEL_TEST_EXECUTION = "test.parallel.execution";
+
+  /**
+   * A property set to true in maven if scale tests are enabled: {@value}.
+   */
+  String KEY_SCALE_TESTS_ENABLED = S3A_SCALE_TEST + "enabled";
+
   /**
   /**
    * The number of operations to perform: {@value}.
    * The number of operations to perform: {@value}.
    */
    */
   String KEY_OPERATION_COUNT = SCALE_TEST + "operation.count";
   String KEY_OPERATION_COUNT = SCALE_TEST + "operation.count";
 
 
+  /**
+   * The number of directory operations to perform: {@value}.
+   */
+  String KEY_DIRECTORY_COUNT = SCALE_TEST + "directory.count";
+
   /**
   /**
    * The readahead buffer: {@value}.
    * The readahead buffer: {@value}.
    */
    */
   String KEY_READ_BUFFER_SIZE = S3A_SCALE_TEST + "read.buffer.size";
   String KEY_READ_BUFFER_SIZE = S3A_SCALE_TEST + "read.buffer.size";
+
   int DEFAULT_READ_BUFFER_SIZE = 16384;
   int DEFAULT_READ_BUFFER_SIZE = 16384;
 
 
   /**
   /**
@@ -64,13 +85,63 @@ public interface S3ATestConstants {
    */
    */
   String DEFAULT_CSVTEST_FILE = "s3a://landsat-pds/scene_list.gz";
   String DEFAULT_CSVTEST_FILE = "s3a://landsat-pds/scene_list.gz";
 
 
+  /**
+   * Endpoint for the S3 CSV/scale tests. This defaults to
+   * being us-east.
+   */
+  String KEY_CSVTEST_ENDPOINT = S3A_SCALE_TEST + "csvfile.endpoint";
+
+  /**
+   * Endpoint for the S3 CSV/scale tests. This defaults to
+   * being us-east.
+   */
+  String DEFAULT_CSVTEST_ENDPOINT = "s3.amazonaws.com";
+
+  /**
+   * Name of the property to define the timeout for scale tests: {@value}.
+   * Measured in seconds.
+   */
+  String KEY_TEST_TIMEOUT = S3A_SCALE_TEST + "timeout";
+
+  /**
+   * Name of the property to define the file size for the huge file
+   * tests: {@value}.
+   * Measured in KB; a suffix like "M", or "G" will change the unit.
+   */
+  String KEY_HUGE_FILESIZE = S3A_SCALE_TEST + "huge.filesize";
+
+  /**
+   * Name of the property to define the partition size for the huge file
+   * tests: {@value}.
+   * Measured in KB; a suffix like "M", or "G" will change the unit.
+   */
+  String KEY_HUGE_PARTITION_SIZE = S3A_SCALE_TEST + "huge.partitionsize";
+
+  /**
+   * The default huge size is small —full 5GB+ scale tests are something
+   * to run in long test runs on EC2 VMs. {@value}.
+   */
+  String DEFAULT_HUGE_FILESIZE = "10M";
+
   /**
   /**
    * The default number of operations to perform: {@value}.
    * The default number of operations to perform: {@value}.
    */
    */
   long DEFAULT_OPERATION_COUNT = 2005;
   long DEFAULT_OPERATION_COUNT = 2005;
 
 
   /**
   /**
-   * Run the encryption tests?
+   * Default number of directories to create when performing
+   * directory performance/scale tests.
    */
    */
-  String KEY_ENCRYPTION_TESTS = TEST_FS_S3A + "encryption.enabled";
+  int DEFAULT_DIRECTORY_COUNT = 2;
+
+  /**
+   * Default scale test timeout in seconds: {@value}.
+   */
+  int DEFAULT_TEST_TIMEOUT = 30 * 60;
+
+  /**
+   * Default policy on scale tests: {@value}.
+   */
+  boolean DEFAULT_SCALE_TESTS_ENABLED = false;
+
 }
 }

+ 145 - 3
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java

@@ -39,6 +39,12 @@ import static org.apache.hadoop.fs.s3a.Constants.*;
  */
  */
 public class S3ATestUtils {
 public class S3ATestUtils {
 
 
+  /**
+   * Value to set a system property to (in maven) to declare that
+   * a property has been unset.
+   */
+  public static final String UNSET_PROPERTY = "unset";
+
   /**
   /**
    * Create the test filesystem.
    * Create the test filesystem.
    *
    *
@@ -53,8 +59,25 @@ public class S3ATestUtils {
    */
    */
   public static S3AFileSystem createTestFileSystem(Configuration conf)
   public static S3AFileSystem createTestFileSystem(Configuration conf)
       throws IOException {
       throws IOException {
-    String fsname = conf.getTrimmed(TEST_FS_S3A_NAME, "");
+    return createTestFileSystem(conf, true);
+  }
 
 
+  /**
+   * Create the test filesystem with or without multipart purging
+   *
+   * If the test.fs.s3a.name property is not set, this will
+   * trigger a JUnit failure.
+   * @param conf configuration
+   * @param purge flag to enable Multipart purging
+   * @return the FS
+   * @throws IOException IO Problems
+   * @throws AssumptionViolatedException if the FS is not named
+   */
+  public static S3AFileSystem createTestFileSystem(Configuration conf,
+      boolean purge)
+      throws IOException {
+
+    String fsname = conf.getTrimmed(TEST_FS_S3A_NAME, "");
 
 
     boolean liveTest = !StringUtils.isEmpty(fsname);
     boolean liveTest = !StringUtils.isEmpty(fsname);
     URI testURI = null;
     URI testURI = null;
@@ -70,8 +93,12 @@ public class S3ATestUtils {
     }
     }
     S3AFileSystem fs1 = new S3AFileSystem();
     S3AFileSystem fs1 = new S3AFileSystem();
     //enable purging in tests
     //enable purging in tests
-    conf.setBoolean(PURGE_EXISTING_MULTIPART, true);
-    conf.setInt(PURGE_EXISTING_MULTIPART_AGE, 0);
+    if (purge) {
+      conf.setBoolean(PURGE_EXISTING_MULTIPART, true);
+      // but a long delay so that parallel multipart tests don't
+      // suddenly start timing out
+      conf.setInt(PURGE_EXISTING_MULTIPART_AGE, 30 * 60);
+    }
     fs1.initialize(testURI, conf);
     fs1.initialize(testURI, conf);
     return fs1;
     return fs1;
   }
   }
@@ -148,6 +175,121 @@ public class S3ATestUtils {
     }
     }
   }
   }
 
 
+  /**
+   * Get a long test property.
+   * <ol>
+   *   <li>Look up configuration value (which can pick up core-default.xml),
+   *       using {@code defVal} as the default value (if conf != null).
+   *   </li>
+   *   <li>Fetch the system property.</li>
+   *   <li>If the system property is not empty or "(unset)":
+   *   it overrides the conf value.
+   *   </li>
+   * </ol>
+   * This puts the build properties in charge of everything. It's not a
+   * perfect design; having maven set properties based on a file, as ant let
+   * you do, is better for customization.
+   *
+   * As to why there's a special (unset) value, see
+   * {@link http://stackoverflow.com/questions/7773134/null-versus-empty-arguments-in-maven}
+   * @param conf config: may be null
+   * @param key key to look up
+   * @param defVal default value
+   * @return the evaluated test property.
+   */
+  public static long getTestPropertyLong(Configuration conf,
+      String key, long defVal) {
+    return Long.valueOf(
+        getTestProperty(conf, key, Long.toString(defVal)));
+  }
+  /**
+   * Get a test property value in bytes, using k, m, g, t, p, e suffixes.
+   * {@link org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix#string2long(String)}
+   * <ol>
+   *   <li>Look up configuration value (which can pick up core-default.xml),
+   *       using {@code defVal} as the default value (if conf != null).
+   *   </li>
+   *   <li>Fetch the system property.</li>
+   *   <li>If the system property is not empty or "(unset)":
+   *   it overrides the conf value.
+   *   </li>
+   * </ol>
+   * This puts the build properties in charge of everything. It's not a
+   * perfect design; having maven set properties based on a file, as ant let
+   * you do, is better for customization.
+   *
+   * As to why there's a special (unset) value, see
+   * {@link http://stackoverflow.com/questions/7773134/null-versus-empty-arguments-in-maven}
+   * @param conf config: may be null
+   * @param key key to look up
+   * @param defVal default value
+   * @return the evaluated test property.
+   */
+  public static long getTestPropertyBytes(Configuration conf,
+      String key, String defVal) {
+    return org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix
+        .string2long(getTestProperty(conf, key, defVal));
+  }
+
+  /**
+   * Get an integer test property; algorithm described in
+   * {@link #getTestPropertyLong(Configuration, String, long)}.
+   * @param key key to look up
+   * @param defVal default value
+   * @return the evaluated test property.
+   */
+  public static int getTestPropertyInt(Configuration conf,
+      String key, int defVal) {
+    return (int) getTestPropertyLong(conf, key, defVal);
+  }
+
+  /**
+   * Get a boolean test property; algorithm described in
+   * {@link #getTestPropertyLong(Configuration, String, long)}.
+   * @param key key to look up
+   * @param defVal default value
+   * @return the evaluated test property.
+   */
+  public static boolean getTestPropertyBool(Configuration conf,
+      String key,
+      boolean defVal) {
+    return Boolean.valueOf(
+        getTestProperty(conf, key, Boolean.toString(defVal)));
+  }
+
+  /**
+   * Get a string test property.
+   * <ol>
+   *   <li>Look up configuration value (which can pick up core-default.xml),
+   *       using {@code defVal} as the default value (if conf != null).
+   *   </li>
+   *   <li>Fetch the system property.</li>
+   *   <li>If the system property is not empty or "(unset)":
+   *   it overrides the conf value.
+   *   </li>
+   * </ol>
+   * This puts the build properties in charge of everything. It's not a
+   * perfect design; having maven set properties based on a file, as ant let
+   * you do, is better for customization.
+   *
+   * As to why there's a special (unset) value, see
+   * @see <a href="http://stackoverflow.com/questions/7773134/null-versus-empty-arguments-in-maven">
+   *   Stack Overflow</a>
+   * @param conf config: may be null
+   * @param key key to look up
+   * @param defVal default value
+   * @return the evaluated test property.
+   */
+
+  public static String getTestProperty(Configuration conf,
+      String key,
+      String defVal) {
+    String confVal = conf != null ? conf.getTrimmed(key, defVal) : defVal;
+    String propval = System.getProperty(key);
+    return StringUtils.isNotEmpty(propval) && !UNSET_PROPERTY.equals(propval)
+        ? propval : confVal;
+  }
+
   /**
   /**
    * The exception to raise so as to exit fast from
    * The exception to raise so as to exit fast from
    * {@link #eventually(int, Callable)}.
    * {@link #eventually(int, Callable)}.

+ 124 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestDataBlocks.java

@@ -0,0 +1,124 @@
+/*
+ * 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.contract.ContractTestUtils;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+/**
+ * Unit tests for {@link S3ADataBlocks}.
+ */
+public class TestDataBlocks extends Assert {
+
+  @Rule
+  public Timeout testTimeout = new Timeout(30 * 1000);
+
+  @Before
+  public void nameThread() {
+    Thread.currentThread().setName("JUnit");
+  }
+
+  /**
+   * Test the {@link S3ADataBlocks.ByteBufferBlockFactory}.
+   * That code implements an input stream over a ByteBuffer, and has to
+   * return the buffer to the pool after the read complete.
+   *
+   * This test verifies the basic contract of the process.
+   */
+  @Test
+  public void testByteBufferIO() throws Throwable {
+    try (S3ADataBlocks.ByteBufferBlockFactory factory =
+             new S3ADataBlocks.ByteBufferBlockFactory(null)) {
+      int limit = 128;
+      S3ADataBlocks.ByteBufferBlockFactory.ByteBufferBlock block
+          = factory.create(limit);
+      assertEquals("outstanding buffers in " + factory,
+          1, factory.getOutstandingBufferCount());
+
+      byte[] buffer = ContractTestUtils.toAsciiByteArray("test data");
+      int bufferLen = buffer.length;
+      block.write(buffer, 0, bufferLen);
+      assertEquals(bufferLen, block.dataSize());
+      assertEquals("capacity in " + block,
+          limit - bufferLen, block.remainingCapacity());
+      assertTrue("hasCapacity(64) in " + block, block.hasCapacity(64));
+      assertTrue("No capacity in " + block,
+          block.hasCapacity(limit - bufferLen));
+
+      // now start the write
+      S3ADataBlocks.ByteBufferBlockFactory.ByteBufferInputStream
+          stream = block.startUpload();
+      assertTrue("!hasRemaining() in " + stream, stream.hasRemaining());
+      int expected = bufferLen;
+      assertEquals("wrong available() in " + stream,
+          expected, stream.available());
+
+      assertEquals('t', stream.read());
+      expected--;
+      assertEquals("wrong available() in " + stream,
+          expected, stream.available());
+
+      // close the block. The buffer must remain outstanding here;
+      // the stream manages the lifecycle of it now
+      block.close();
+      assertEquals("outstanding buffers in " + factory,
+          1, factory.getOutstandingBufferCount());
+      block.close();
+
+      // read into a byte array with an offset
+      int offset = 5;
+      byte[] in = new byte[limit];
+      assertEquals(2, stream.read(in, offset, 2));
+      assertEquals('e', in[offset]);
+      assertEquals('s', in[offset + 1]);
+      expected -= 2;
+      assertEquals("wrong available() in " + stream,
+          expected, stream.available());
+
+      // read to end
+      byte[] remainder = new byte[limit];
+      int c;
+      int index = 0;
+      while ((c = stream.read()) >= 0) {
+        remainder[index++] = (byte) c;
+      }
+      assertEquals(expected, index);
+      assertEquals('a', remainder[--index]);
+
+      assertEquals("wrong available() in " + stream,
+          0, stream.available());
+      assertTrue("hasRemaining() in " + stream, !stream.hasRemaining());
+
+      // when the stream is closed, the data should be returned
+      stream.close();
+      assertEquals("outstanding buffers in " + factory,
+          0, factory.getOutstandingBufferCount());
+      stream.close();
+      assertEquals("outstanding buffers in " + factory,
+          0, factory.getOutstandingBufferCount());
+
+    }
+
+  }
+
+}

+ 1 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java

@@ -34,6 +34,7 @@ public class ITestS3AFileContextStatistics extends FCStatisticsBaseTest {
     fc = S3ATestUtils.createTestFileContext(conf);
     fc = S3ATestUtils.createTestFileContext(conf);
     fc.mkdir(fileContextTestHelper.getTestRootPath(fc, "test"),
     fc.mkdir(fileContextTestHelper.getTestRootPath(fc, "test"),
         FileContext.DEFAULT_PERM, true);
         FileContext.DEFAULT_PERM, true);
+    FileContext.clearStatistics();
   }
   }
 
 
   @After
   @After

+ 412 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java

@@ -0,0 +1,412 @@
+/*
+ * 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 java.io.IOException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import com.amazonaws.event.ProgressEvent;
+import com.amazonaws.event.ProgressEventType;
+import com.amazonaws.event.ProgressListener;
+import org.junit.FixMethodOrder;
+import org.junit.Test;
+import org.junit.runners.MethodSorters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StorageStatistics;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.s3a.S3AFileStatus;
+import org.apache.hadoop.fs.s3a.Statistic;
+import org.apache.hadoop.util.Progressable;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
+import static org.apache.hadoop.fs.s3a.Constants.*;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
+
+/**
+ * Scale test which creates a huge file.
+ *
+ * <b>Important:</b> the order in which these tests execute is fixed to
+ * alphabetical order. Test cases are numbered {@code test_123_} to impose
+ * an ordering based on the numbers.
+ *
+ * Having this ordering allows the tests to assume that the huge file
+ * exists. Even so: they should all have a {@link #assumeHugeFileExists()}
+ * check at the start, in case an individual test is executed.
+ */
+@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      AbstractSTestS3AHugeFiles.class);
+  public static final int DEFAULT_UPLOAD_BLOCKSIZE = 64 * _1KB;
+  public static final String DEFAULT_PARTITION_SIZE = "8M";
+  private Path scaleTestDir;
+  private Path hugefile;
+  private Path hugefileRenamed;
+
+  private int uploadBlockSize = DEFAULT_UPLOAD_BLOCKSIZE;
+  private int partitionSize;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+
+    final Path testPath = getTestPath();
+    scaleTestDir = new Path(testPath, "scale");
+    hugefile = new Path(scaleTestDir, "hugefile");
+    hugefileRenamed = new Path(scaleTestDir, "hugefileRenamed");
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    // do nothing. Specifically: do not delete the test dir
+  }
+
+  /**
+   * Note that this can get called before test setup.
+   * @return the configuration to use.
+   */
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    partitionSize = (int)getTestPropertyBytes(conf,
+        KEY_HUGE_PARTITION_SIZE,
+        DEFAULT_PARTITION_SIZE);
+    assertTrue("Partition size too small: " + partitionSize,
+        partitionSize > MULTIPART_MIN_SIZE);
+    conf.setLong(SOCKET_SEND_BUFFER, _1MB);
+    conf.setLong(SOCKET_RECV_BUFFER, _1MB);
+    conf.setLong(MIN_MULTIPART_THRESHOLD, partitionSize);
+    conf.setInt(MULTIPART_SIZE, partitionSize);
+    conf.set(USER_AGENT_PREFIX, "STestS3AHugeFileCreate");
+    conf.setBoolean(FAST_UPLOAD, true);
+    conf.set(FAST_UPLOAD_BUFFER, getBlockOutputBufferName());
+    return conf;
+  }
+
+  /**
+   * The name of the buffering mechanism to use.
+   * @return a buffering mechanism
+   */
+  protected abstract String getBlockOutputBufferName();
+
+  @Test
+  public void test_010_CreateHugeFile() throws IOException {
+    assertFalse("Please run this test sequentially to avoid timeouts" +
+            " and bandwidth problems", isParallelExecution());
+    long filesize = getTestPropertyBytes(getConf(), KEY_HUGE_FILESIZE,
+        DEFAULT_HUGE_FILESIZE);
+    long filesizeMB = filesize / _1MB;
+
+    // clean up from any previous attempts
+    deleteHugeFile();
+
+    describe("Creating file %s of size %d MB" +
+            " with partition size %d buffered by %s",
+        hugefile, filesizeMB, partitionSize, getBlockOutputBufferName());
+
+    // now do a check of available upload time, with a pessimistic bandwidth
+    // (that of remote upload tests). If the test times out then not only is
+    // the test outcome lost, as the follow-on tests continue, they will
+    // overlap with the ongoing upload test, for much confusion.
+    int timeout = getTestTimeoutSeconds();
+    // assume 1 MB/s upload bandwidth
+    int bandwidth = _1MB;
+    long uploadTime = filesize / bandwidth;
+    assertTrue(String.format("Timeout set in %s seconds is too low;" +
+            " estimating upload time of %d seconds at 1 MB/s." +
+            " Rerun tests with -D%s=%d",
+            timeout, uploadTime, KEY_TEST_TIMEOUT, uploadTime * 2),
+        uploadTime < timeout);
+    assertEquals("File size set in " + KEY_HUGE_FILESIZE + " = " + filesize
+            + " is not a multiple of " + uploadBlockSize,
+        0, filesize % uploadBlockSize);
+
+    byte[] data = new byte[uploadBlockSize];
+    for (int i = 0; i < uploadBlockSize; i++) {
+      data[i] = (byte) (i % 256);
+    }
+
+    long blocks = filesize / uploadBlockSize;
+    long blocksPerMB = _1MB / uploadBlockSize;
+
+    // perform the upload.
+    // there's lots of logging here, so that a tail -f on the output log
+    // can give a view of what is happening.
+    StorageStatistics storageStatistics = fs.getStorageStatistics();
+    String putRequests = Statistic.OBJECT_PUT_REQUESTS.getSymbol();
+    String putBytes = Statistic.OBJECT_PUT_BYTES.getSymbol();
+    Statistic putRequestsActive = Statistic.OBJECT_PUT_REQUESTS_ACTIVE;
+    Statistic putBytesPending = Statistic.OBJECT_PUT_BYTES_PENDING;
+
+    ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
+
+    long blocksPer10MB = blocksPerMB * 10;
+    ProgressCallback progress = new ProgressCallback(timer);
+    try (FSDataOutputStream out = fs.create(hugefile,
+        true,
+        uploadBlockSize,
+        progress)) {
+
+      for (long block = 1; block <= blocks; block++) {
+        out.write(data);
+        long written = block * uploadBlockSize;
+        // every 10 MB and on file upload @ 100%, print some stats
+        if (block % blocksPer10MB == 0 || written == filesize) {
+          long percentage = written * 100 / filesize;
+          double elapsedTime = timer.elapsedTime() / 1.0e9;
+          double writtenMB = 1.0 * written / _1MB;
+          LOG.info(String.format("[%02d%%] Buffered %.2f MB out of %d MB;" +
+                  " PUT %d bytes (%d pending) in %d operations (%d active);" +
+                  " elapsedTime=%.2fs; write to buffer bandwidth=%.2f MB/s",
+              percentage,
+              writtenMB,
+              filesizeMB,
+              storageStatistics.getLong(putBytes),
+              gaugeValue(putBytesPending),
+              storageStatistics.getLong(putRequests),
+              gaugeValue(putRequestsActive),
+              elapsedTime,
+              writtenMB / elapsedTime));
+        }
+      }
+      // now close the file
+      LOG.info("Closing file and completing write operation");
+      ContractTestUtils.NanoTimer closeTimer
+          = new ContractTestUtils.NanoTimer();
+      out.close();
+      closeTimer.end("time to close() output stream");
+    }
+
+    timer.end("time to write %d MB in blocks of %d",
+        filesizeMB, uploadBlockSize);
+    logFSState();
+    bandwidth(timer, filesize);
+    long putRequestCount = storageStatistics.getLong(putRequests);
+    Long putByteCount = storageStatistics.getLong(putBytes);
+    LOG.info("PUT {} bytes in {} operations; {} MB/operation",
+        putByteCount, putRequestCount,
+        putByteCount / (putRequestCount * _1MB));
+    LOG.info("Time per PUT {} nS",
+        toHuman(timer.nanosPerOperation(putRequestCount)));
+    assertEquals("active put requests in \n" + fs,
+        0, gaugeValue(putRequestsActive));
+    ContractTestUtils.assertPathExists(fs, "Huge file", hugefile);
+    S3AFileStatus status = fs.getFileStatus(hugefile);
+    ContractTestUtils.assertIsFile(hugefile, status);
+    assertEquals("File size in " + status, filesize, status.getLen());
+    progress.verifyNoFailures("Put file " + hugefile + " of size " + filesize);
+  }
+
+  /**
+   * Progress callback from AWS. Likely to come in on a different thread.
+   */
+  private final class ProgressCallback implements Progressable,
+      ProgressListener {
+    private AtomicLong bytesTransferred = new AtomicLong(0);
+    private AtomicInteger failures = new AtomicInteger(0);
+    private final ContractTestUtils.NanoTimer timer;
+
+    private ProgressCallback(NanoTimer timer) {
+      this.timer = timer;
+    }
+
+    @Override
+    public void progress() {
+    }
+
+    @Override
+    public void progressChanged(ProgressEvent progressEvent) {
+      ProgressEventType eventType = progressEvent.getEventType();
+      if (eventType.isByteCountEvent()) {
+        bytesTransferred.addAndGet(progressEvent.getBytesTransferred());
+      }
+      switch (eventType) {
+      case TRANSFER_PART_FAILED_EVENT:
+        // failure
+        failures.incrementAndGet();
+        LOG.warn("Transfer failure");
+        break;
+      case TRANSFER_PART_COMPLETED_EVENT:
+        // completion
+        long elapsedTime = timer.elapsedTime();
+        double elapsedTimeS = elapsedTime / 1.0e9;
+        long written = bytesTransferred.get();
+        long writtenMB = written / _1MB;
+        LOG.info(String.format(
+            "Event %s; total uploaded=%d MB in %.1fs;" +
+                " effective upload bandwidth = %.2f MB/s",
+            progressEvent,
+            writtenMB, elapsedTimeS, writtenMB / elapsedTimeS));
+        break;
+      default:
+        if (eventType.isByteCountEvent()) {
+          LOG.debug("Event {}", progressEvent);
+        } else {
+          LOG.info("Event {}", progressEvent);
+        }
+        break;
+      }
+    }
+
+    @Override
+    public String toString() {
+      String sb = "ProgressCallback{"
+          + "bytesTransferred=" + bytesTransferred +
+          ", failures=" + failures +
+          '}';
+      return sb;
+    }
+
+    private void verifyNoFailures(String operation) {
+      assertEquals("Failures in " + operation +": " + this, 0, failures.get());
+    }
+  }
+
+  void assumeHugeFileExists() throws IOException {
+    ContractTestUtils.assertPathExists(fs, "huge file not created", hugefile);
+    ContractTestUtils.assertIsFile(fs, hugefile);
+  }
+
+  private void logFSState() {
+    LOG.info("File System state after operation:\n{}", fs);
+  }
+
+  @Test
+  public void test_040_PositionedReadHugeFile() throws Throwable {
+    assumeHugeFileExists();
+    final String encryption = getConf().getTrimmed(
+        SERVER_SIDE_ENCRYPTION_ALGORITHM);
+    boolean encrypted = encryption != null;
+    if (encrypted) {
+      LOG.info("File is encrypted with algorithm {}", encryption);
+    }
+    String filetype = encrypted ? "encrypted file" : "file";
+    describe("Positioned reads of %s %s", filetype, hugefile);
+    S3AFileStatus status = fs.getFileStatus(hugefile);
+    long filesize = status.getLen();
+    int ops = 0;
+    final int bufferSize = 8192;
+    byte[] buffer = new byte[bufferSize];
+    long eof = filesize - 1;
+
+    ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
+    ContractTestUtils.NanoTimer readAtByte0, readAtByte0Again, readAtEOF;
+    try (FSDataInputStream in = fs.open(hugefile, uploadBlockSize)) {
+      readAtByte0 = new ContractTestUtils.NanoTimer();
+      in.readFully(0, buffer);
+      readAtByte0.end("time to read data at start of file");
+      ops++;
+
+      readAtEOF = new ContractTestUtils.NanoTimer();
+      in.readFully(eof - bufferSize, buffer);
+      readAtEOF.end("time to read data at end of file");
+      ops++;
+
+      readAtByte0Again = new ContractTestUtils.NanoTimer();
+      in.readFully(0, buffer);
+      readAtByte0Again.end("time to read data at start of file again");
+      ops++;
+      LOG.info("Final stream state: {}", in);
+    }
+    long mb = Math.max(filesize / _1MB, 1);
+
+    logFSState();
+    timer.end("time to performed positioned reads of %s of %d MB ",
+        filetype, mb);
+    LOG.info("Time per positioned read = {} nS",
+        toHuman(timer.nanosPerOperation(ops)));
+  }
+
+  @Test
+  public void test_050_readHugeFile() throws Throwable {
+    assumeHugeFileExists();
+    describe("Reading %s", hugefile);
+    S3AFileStatus status = fs.getFileStatus(hugefile);
+    long filesize = status.getLen();
+    long blocks = filesize / uploadBlockSize;
+    byte[] data = new byte[uploadBlockSize];
+
+    ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
+    try (FSDataInputStream in = fs.open(hugefile, uploadBlockSize)) {
+      for (long block = 0; block < blocks; block++) {
+        in.readFully(data);
+      }
+      LOG.info("Final stream state: {}", in);
+    }
+
+    long mb = Math.max(filesize / _1MB, 1);
+    timer.end("time to read file of %d MB ", mb);
+    LOG.info("Time per MB to read = {} nS",
+        toHuman(timer.nanosPerOperation(mb)));
+    bandwidth(timer, filesize);
+    logFSState();
+  }
+
+  @Test
+  public void test_100_renameHugeFile() throws Throwable {
+    assumeHugeFileExists();
+    describe("renaming %s to %s", hugefile, hugefileRenamed);
+    S3AFileStatus status = fs.getFileStatus(hugefile);
+    long filesize = status.getLen();
+    fs.delete(hugefileRenamed, false);
+    ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
+    fs.rename(hugefile, hugefileRenamed);
+    long mb = Math.max(filesize / _1MB, 1);
+    timer.end("time to rename file of %d MB", mb);
+    LOG.info("Time per MB to rename = {} nS",
+        toHuman(timer.nanosPerOperation(mb)));
+    bandwidth(timer, filesize);
+    logFSState();
+    S3AFileStatus destFileStatus = fs.getFileStatus(hugefileRenamed);
+    assertEquals(filesize, destFileStatus.getLen());
+
+    // rename back
+    ContractTestUtils.NanoTimer timer2 = new ContractTestUtils.NanoTimer();
+    fs.rename(hugefileRenamed, hugefile);
+    timer2.end("Renaming back");
+    LOG.info("Time per MB to rename = {} nS",
+        toHuman(timer2.nanosPerOperation(mb)));
+    bandwidth(timer2, filesize);
+  }
+
+  @Test
+  public void test_999_DeleteHugeFiles() throws IOException {
+    deleteHugeFile();
+    ContractTestUtils.NanoTimer timer2 = new ContractTestUtils.NanoTimer();
+
+    fs.delete(hugefileRenamed, false);
+    timer2.end("time to delete %s", hugefileRenamed);
+    ContractTestUtils.rm(fs, getTestPath(), true, true);
+  }
+
+  protected void deleteHugeFile() throws IOException {
+    describe("Deleting %s", hugefile);
+    NanoTimer timer = new NanoTimer();
+    fs.delete(hugefile, false);
+    timer.end("time to delete %s", hugefile);
+  }
+
+}

+ 4 - 15
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADeleteManyFiles.java

@@ -116,20 +116,9 @@ public class ITestS3ADeleteManyFiles extends S3AScaleTestBase {
 
 
   @Test
   @Test
   public void testOpenCreate() throws IOException {
   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());
-    }
-    */
+    final Path scaleTestDir = getTestPath();
+    final Path srcDir = new Path(scaleTestDir, "opencreate");
+    ContractTestUtils.createAndVerifyFile(fs, srcDir, 1024);
+    ContractTestUtils.createAndVerifyFile(fs, srcDir, 50 * 1024);
   }
   }
 }
 }

+ 31 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesArrayBlocks.java

@@ -0,0 +1,31 @@
+/*
+ * 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.s3a.Constants;
+
+/**
+ * Use {@link Constants#FAST_UPLOAD_BUFFER_ARRAY} for buffering.
+ */
+public class ITestS3AHugeFilesArrayBlocks extends AbstractSTestS3AHugeFiles {
+
+  protected String getBlockOutputBufferName() {
+    return Constants.FAST_UPLOAD_BUFFER_ARRAY;
+  }
+}

+ 34 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesByteBufferBlocks.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.s3a.scale;
+
+import org.apache.hadoop.fs.s3a.Constants;
+
+import static org.apache.hadoop.fs.s3a.Constants.FAST_UPLOAD_BYTEBUFFER;
+
+/**
+ * Use {@link Constants#FAST_UPLOAD_BYTEBUFFER} for buffering.
+ */
+public class ITestS3AHugeFilesByteBufferBlocks
+    extends AbstractSTestS3AHugeFiles {
+
+  protected String getBlockOutputBufferName() {
+    return FAST_UPLOAD_BYTEBUFFER;
+  }
+}

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

+ 31 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesDiskBlocks.java

@@ -0,0 +1,31 @@
+/*
+ * 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.s3a.Constants;
+
+/**
+ * Use {@link Constants#FAST_UPLOAD_BUFFER_DISK} for buffering.
+ */
+public class ITestS3AHugeFilesDiskBlocks extends AbstractSTestS3AHugeFiles {
+
+  protected String getBlockOutputBufferName() {
+    return Constants.FAST_UPLOAD_BUFFER_DISK;
+  }
+}

+ 85 - 66
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java

@@ -20,18 +20,18 @@ package org.apache.hadoop.fs.s3a.scale;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
-import org.apache.hadoop.fs.Path;
-
 import org.apache.hadoop.fs.s3a.S3AInputStream;
 import org.apache.hadoop.fs.s3a.S3AInputStream;
 import org.apache.hadoop.fs.s3a.S3AInstrumentation;
 import org.apache.hadoop.fs.s3a.S3AInstrumentation;
 import org.apache.hadoop.fs.s3a.S3ATestConstants;
 import org.apache.hadoop.fs.s3a.S3ATestConstants;
-import org.apache.hadoop.fs.s3a.S3ATestUtils;
+import org.apache.hadoop.fs.s3a.Statistic;
+import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Rule;
 import org.junit.rules.TestName;
 import org.junit.rules.TestName;
 import org.junit.rules.Timeout;
 import org.junit.rules.Timeout;
@@ -40,6 +40,8 @@ import org.slf4j.LoggerFactory;
 
 
 import java.io.InputStream;
 import java.io.InputStream;
 
 
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
+
 /**
 /**
  * Base class for scale tests; here is where the common scale configuration
  * Base class for scale tests; here is where the common scale configuration
  * keys are defined.
  * keys are defined.
@@ -47,71 +49,18 @@ import java.io.InputStream;
 public class S3AScaleTestBase extends Assert implements S3ATestConstants {
 public class S3AScaleTestBase extends Assert implements S3ATestConstants {
 
 
   @Rule
   @Rule
-  public TestName methodName = new TestName();
+  public final TestName methodName = new TestName();
 
 
   @Rule
   @Rule
-  public Timeout testTimeout = new Timeout(30 * 60 * 1000);
+  public Timeout testTimeout = createTestTimeout();
 
 
-  @BeforeClass
-  public static void nameThread() {
+  @Before
+  public void nameThread() {
     Thread.currentThread().setName("JUnit");
     Thread.currentThread().setName("JUnit");
   }
   }
 
 
-  /**
-   * The number of operations to perform: {@value}.
-   */
-  public static final String KEY_OPERATION_COUNT =
-      SCALE_TEST + "operation.count";
-
-  /**
-   * The number of directory operations to perform: {@value}.
-   */
-  public static final String KEY_DIRECTORY_COUNT =
-      SCALE_TEST + "directory.count";
-
-  /**
-   * The readahead buffer: {@value}.
-   */
-  public static final String KEY_READ_BUFFER_SIZE =
-      S3A_SCALE_TEST + "read.buffer.size";
-
-  public static final int DEFAULT_READ_BUFFER_SIZE = 16384;
-
-  /**
-   * Key for a multi MB test file: {@value}.
-   */
-  public static final String KEY_CSVTEST_FILE =
-      S3A_SCALE_TEST + "csvfile";
-  /**
-   * Default path for the multi MB test file: {@value}.
-   */
-  public static final String DEFAULT_CSVTEST_FILE
-      = "s3a://landsat-pds/scene_list.gz";
-
-  /**
-   * Endpoint for the S3 CSV/scale tests. This defaults to
-   * being us-east.
-   */
-  public static final String KEY_CSVTEST_ENDPOINT =
-      S3A_SCALE_TEST + "csvfile.endpoint";
-
-  /**
-   * Endpoint for the S3 CSV/scale tests. This defaults to
-   * being us-east.
-   */
-  public static final String DEFAULT_CSVTEST_ENDPOINT =
-      "s3.amazonaws.com";
-
-  /**
-   * The default number of operations to perform: {@value}.
-   */
-  public static final long DEFAULT_OPERATION_COUNT = 2005;
-
-  /**
-   * Default number of directories to create when performing
-   * directory performance/scale tests.
-   */
-  public static final int DEFAULT_DIRECTORY_COUNT = 2;
+  public static final int _1KB = 1024;
+  public static final int _1MB = _1KB * _1KB;
 
 
   protected S3AFileSystem fs;
   protected S3AFileSystem fs;
 
 
@@ -120,6 +69,8 @@ public class S3AScaleTestBase extends Assert implements S3ATestConstants {
 
 
   private Configuration conf;
   private Configuration conf;
 
 
+  private boolean enabled;
+
   /**
   /**
    * Configuration generator. May be overridden to inject
    * Configuration generator. May be overridden to inject
    * some custom options.
    * some custom options.
@@ -137,11 +88,33 @@ public class S3AScaleTestBase extends Assert implements S3ATestConstants {
     return conf;
     return conf;
   }
   }
 
 
+  /**
+   * Setup. This triggers creation of the configuration.
+   */
   @Before
   @Before
   public void setUp() throws Exception {
   public void setUp() throws Exception {
-    conf = createConfiguration();
+    demandCreateConfiguration();
     LOG.debug("Scale test operation count = {}", getOperationCount());
     LOG.debug("Scale test operation count = {}", getOperationCount());
-    fs = S3ATestUtils.createTestFileSystem(conf);
+    // multipart purges are disabled on the scale tests
+    fs = createTestFileSystem(conf, false);
+    // check for the test being enabled
+    enabled = getTestPropertyBool(
+        getConf(),
+        KEY_SCALE_TESTS_ENABLED,
+        DEFAULT_SCALE_TESTS_ENABLED);
+    Assume.assumeTrue("Scale test disabled: to enable set property " +
+        KEY_SCALE_TESTS_ENABLED, enabled);
+  }
+
+  /**
+   * Create the configuration if it is not already set up.
+   * @return the configuration.
+   */
+  private synchronized Configuration demandCreateConfiguration() {
+    if (conf == null) {
+      conf = createConfiguration();
+    }
+    return conf;
   }
   }
 
 
   @After
   @After
@@ -160,7 +133,27 @@ public class S3AScaleTestBase extends Assert implements S3ATestConstants {
   }
   }
 
 
   /**
   /**
-   * Describe a test in the logs
+   * Create the timeout for tests. Some large tests may need a larger value.
+   * @return the test timeout to use
+   */
+  protected Timeout createTestTimeout() {
+    demandCreateConfiguration();
+    return new Timeout(
+        getTestTimeoutSeconds() * 1000);
+  }
+
+  /**
+   * Get the test timeout in seconds.
+   * @return the test timeout as set in system properties or the default.
+   */
+  protected static int getTestTimeoutSeconds() {
+    return getTestPropertyInt(null,
+        KEY_TEST_TIMEOUT,
+        DEFAULT_TEST_TIMEOUT);
+  }
+
+  /**
+   * Describe a test in the logs.
    * @param text text to print
    * @param text text to print
    * @param args arguments to format in the printing
    * @param args arguments to format in the printing
    */
    */
@@ -189,4 +182,30 @@ public class S3AScaleTestBase extends Assert implements S3ATestConstants {
     }
     }
   }
   }
 
 
+  /**
+   * Get the gauge value of a statistic. Raises an assertion if
+   * there is no such gauge.
+   * @param statistic statistic to look up
+   * @return the value.
+   */
+  public long gaugeValue(Statistic statistic) {
+    S3AInstrumentation instrumentation = fs.getInstrumentation();
+    MutableGaugeLong gauge = instrumentation.lookupGauge(statistic.getSymbol());
+    assertNotNull("No gauge " + statistic
+        + " in " + instrumentation.dump("", " = ", "\n", true), gauge);
+    return gauge.value();
+  }
+
+  protected boolean isEnabled() {
+    return enabled;
+  }
+
+  /**
+   * Flag to indicate that this test is being used sequentially. This
+   * is used by some of the scale tests to validate test time expectations.
+   * @return true if the build indicates this test is being run in parallel.
+   */
+  protected boolean isParallelExecution() {
+    return Boolean.getBoolean(S3ATestConstants.KEY_PARALLEL_TEST_EXECUTION);
+  }
 }
 }