Forráskód Böngészése

HADOOP-19354. S3A: S3AInputStream to be created by factory under S3AStore (#7214)

S3 InputStreams are created by a factory class, with the
choice of factory dynamically chosen by the option

  fs.s3a.input.stream.type

Supported values: classic, prefetching, analytics, custom

Contributed by Steve Loughran
Steve Loughran 2 hónapja
szülő
commit
5067082dcd
66 módosított fájl, 3652 hozzáadás és 807 törlés
  1. 36 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ConfigurationHelper.java
  2. 68 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestConfigurationHelper.java
  3. 1 1
      hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
  4. 24 12
      hadoop-tools/hadoop-aws/pom.xml
  5. 47 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
  6. 140 201
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
  7. 99 289
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
  8. 7 37
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java
  9. 65 1
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java
  10. 89 15
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/VectoredIOContext.java
  11. 11 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditIntegration.java
  12. 7 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditManagerS3A.java
  13. 29 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditorFlags.java
  14. 7 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/OperationAuditor.java
  15. 79 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractOperationAuditor.java
  16. 7 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/ActiveAuditManagerS3A.java
  17. 4 10
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java
  18. 8 1
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditManagerS3A.java
  19. 11 5
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditor.java
  20. 3 6
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClientManager.java
  21. 22 38
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClientManagerImpl.java
  22. 123 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InputStreamCallbacksImpl.java
  23. 203 10
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java
  24. 96 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/AbstractObjectInputStreamFactory.java
  25. 73 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ClassicObjectInputStreamFactory.java
  26. 46 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/FactoryBindingParameters.java
  27. 125 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/InputStreamType.java
  28. 421 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStream.java
  29. 64 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStreamCallbacks.java
  30. 90 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStreamFactory.java
  31. 190 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectReadParameters.java
  32. 148 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/StreamFactoryRequirements.java
  33. 213 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/StreamIntegration.java
  34. 29 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/package-info.java
  35. 67 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/PrefetchOptions.java
  36. 109 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/PrefetchingInputStreamFactory.java
  37. 8 8
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ACachingInputStream.java
  38. 5 3
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3AInMemoryInputStream.java
  39. 44 23
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchingInputStream.java
  40. 7 7
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteInputStream.java
  41. 3 2
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObject.java
  42. 7 1
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
  43. 12 2
      hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/prefetching.md
  44. 228 0
      hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/reading.md
  45. 1 1
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java
  46. 14 4
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
  47. 27 17
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AInputStreamLeakage.java
  48. 2 3
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingCacheFiles.java
  49. 2 4
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingInputStream.java
  50. 5 6
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingLruEviction.java
  51. 11 7
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java
  52. 46 6
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
  53. 16 11
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java
  54. 10 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AccessCheckingAuditor.java
  55. 25 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AuditTestSupport.java
  56. 9 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditManager.java
  57. 3 4
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestConnectionTimeouts.java
  58. 339 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/streams/TestStreamFactories.java
  59. 3 4
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java
  60. 2 3
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestUnbufferDraining.java
  61. 4 4
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/MockS3ARemoteObject.java
  62. 23 20
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchFakes.java
  63. 11 8
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteInputStream.java
  64. 3 3
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteObject.java
  65. 17 30
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
  66. 4 0
      hadoop-tools/hadoop-aws/src/test/resources/log4j.properties

+ 36 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ConfigurationHelper.java

@@ -22,6 +22,7 @@ import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.Locale;
 import java.util.Map;
+import java.util.function.Function;
 import java.util.stream.Collectors;
 
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -123,4 +124,39 @@ public final class ConfigurationHelper {
     return mapping;
   }
 
+  /**
+   * Look up an enum from the configuration option and map it to
+   * a value in the supplied enum class.
+   * If no value is supplied or there is no match for the supplied value,
+   * the fallback function is invoked, passing in the trimmed and possibly
+   * empty string of the value.
+   * Extends {link {@link Configuration#getEnum(String, Enum)}}
+   * by adding case independence and a lambda expression for fallback,
+   * rather than a default value.
+   * @param conf configuration
+   * @param name property name
+   * @param enumClass classname to resolve
+   * @param fallback fallback supplier
+   * @param <E> enumeration type.
+   * @return an enum value
+   * @throws IllegalArgumentException If mapping is illegal for the type provided
+   */
+  public static <E extends Enum<E>> E resolveEnum(
+      Configuration conf,
+      String name,
+      Class<E> enumClass,
+      Function<String, E> fallback) {
+
+    final String val = conf.getTrimmed(name, "");
+
+    // build a map of lower case string to enum values.
+    final Map<String, E> mapping = mapEnumNamesToValues("", enumClass);
+    final E mapped = mapping.get(val.toLowerCase(Locale.ROOT));
+    if (mapped != null) {
+      return mapped;
+    } else {
+      // fallback handles it
+      return fallback.apply(val);
+    }
+  }
 }

+ 68 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestConfigurationHelper.java

@@ -31,6 +31,7 @@ import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 import static org.apache.hadoop.util.ConfigurationHelper.ERROR_MULTIPLE_ELEMENTS_MATCHING_TO_LOWER_CASE_VALUE;
 import static org.apache.hadoop.util.ConfigurationHelper.mapEnumNamesToValues;
 import static org.apache.hadoop.util.ConfigurationHelper.parseEnumSet;
+import static org.apache.hadoop.util.ConfigurationHelper.resolveEnum;
 
 /**
  * Test for {@link ConfigurationHelper}.
@@ -43,6 +44,12 @@ public class TestConfigurationHelper extends AbstractHadoopTestBase {
    */
   private enum SimpleEnum { a, b, c, i }
 
+  /**
+   * Upper case version of SimpleEnum.
+   * "i" is included for case tests, as it is special in turkey.
+   */
+  private enum UppercaseEnum { A, B, C, I }
+
 
   /**
    * Special case: an enum with no values.
@@ -171,4 +178,65 @@ public class TestConfigurationHelper extends AbstractHadoopTestBase {
         .containsExactly(SimpleEnum.a, SimpleEnum.b, SimpleEnum.c);
   }
 
+  @Test
+  public void testResolveEnumGood() throws Throwable {
+    assertEnumResolution("c", SimpleEnum.c);
+  }
+
+  @Test
+  public void testResolveEnumTrimmed() throws Throwable {
+    // strings are trimmed at each end
+    assertEnumResolution("\n i \n ", SimpleEnum.i);
+  }
+
+  @Test
+  public void testResolveEnumCaseConversion() throws Throwable {
+    assertEnumResolution("C", SimpleEnum.c);
+  }
+
+  @Test
+  public void testResolveEnumNoMatch() throws Throwable {
+    assertEnumResolution("other", null);
+  }
+
+  @Test
+  public void testResolveEnumEmpty() throws Throwable {
+    assertEnumResolution("", null);
+  }
+
+  @Test
+  public void testResolveEnumUpperCaseConversion() throws Throwable {
+    assertUpperEnumResolution("C", UppercaseEnum.C);
+  }
+
+  @Test
+  public void testResolveLowerToUpperCaseConversion() throws Throwable {
+    assertUpperEnumResolution("i", UppercaseEnum.I);
+  }
+
+  /**
+   * Assert that a string value in a configuration resolves to the expected
+   * value.
+   * @param value value to set
+   * @param expected expected outcome, set to null for no resolution.
+   */
+  private void assertEnumResolution(final String value, final SimpleEnum expected) {
+    Assertions.assertThat(resolveEnum(confWithKey(value),
+            "key", SimpleEnum.class, (v) -> null))
+        .describedAs("Resolution of %s", value)
+        .isEqualTo(expected);
+  }
+
+  /**
+   * Equivalent for Uppercase Enum.
+   * @param value value to set
+   * @param expected expected outcome, set to null for no resolution.
+   */
+  private void assertUpperEnumResolution(final String value, UppercaseEnum expected) {
+    Assertions.assertThat(resolveEnum(confWithKey(value),
+            "key", UppercaseEnum.class, (v) -> null))
+        .describedAs("Resolution of %s", value)
+        .isEqualTo(expected);
+  }
+
 }

+ 1 - 1
hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml

@@ -30,7 +30,7 @@
   </Match>
   <!-- we are using completable futures, so ignore the Future which submit() returns -->
   <Match>
-    <Class name="org.apache.hadoop.fs.s3a.S3AFileSystem$InputStreamCallbacksImpl" />
+    <Class name="org.apache.hadoop.fs.s3a.impl.InputStreamCallbacksImpl" />
     <Bug pattern="RV_RETURN_VALUE_IGNORED_BAD_PRACTICE" />
   </Match>
 

+ 24 - 12
hadoop-tools/hadoop-aws/pom.xml

@@ -48,9 +48,8 @@
     <!-- Set a longer timeout for integration test (in milliseconds) -->
     <test.integration.timeout>200000</test.integration.timeout>
 
-
-    <!-- Is prefetch enabled? -->
-    <fs.s3a.prefetch.enabled>unset</fs.s3a.prefetch.enabled>
+    <!-- stream type to use in tests; passed down in fs.s3a.input.stream.type  -->
+    <stream>classic</stream>
     <!-- Job ID; allows for parallel jobs on same bucket -->
     <!-- job.id is used to build the path for tests; default is 00.-->
     <job.id>00</job.id>
@@ -122,8 +121,8 @@
                 <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>
-                <!-- Prefetch -->
-                <fs.s3a.prefetch.enabled>${fs.s3a.prefetch.enabled}</fs.s3a.prefetch.enabled>
+                <!-- Stream Type -->
+                <fs.s3a.input.stream.type>${stream}</fs.s3a.input.stream.type>
               </systemPropertyVariables>
             </configuration>
           </plugin>
@@ -161,8 +160,8 @@
                     <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>
                     <test.default.timeout>${test.integration.timeout}</test.default.timeout>
-                    <!-- Prefetch -->
-                    <fs.s3a.prefetch.enabled>${fs.s3a.prefetch.enabled}</fs.s3a.prefetch.enabled>
+                    <!-- Stream Type -->
+                    <fs.s3a.input.stream.type>${stream}</fs.s3a.input.stream.type>
                     <!-- are root tests enabled. Set to false when running parallel jobs on same bucket -->
                     <fs.s3a.root.tests.enabled>${root.tests.enabled}</fs.s3a.root.tests.enabled>
 
@@ -212,8 +211,8 @@
                     <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>
-                    <!-- Prefetch -->
-                    <fs.s3a.prefetch.enabled>${fs.s3a.prefetch.enabled}</fs.s3a.prefetch.enabled>
+                    <!-- Stream Type -->
+                    <fs.s3a.input.stream.type>${stream}</fs.s3a.input.stream.type>
                     <!-- are root tests enabled. Set to false when running parallel jobs on same bucket -->
                     <fs.s3a.root.tests.enabled>${root.tests.enabled}</fs.s3a.root.tests.enabled>
                     <test.unique.fork.id>job-${job.id}</test.unique.fork.id>
@@ -273,8 +272,8 @@
                     <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>
-                    <!-- Prefetch -->
-                    <fs.s3a.prefetch.enabled>${fs.s3a.prefetch.enabled}</fs.s3a.prefetch.enabled>
+                    <!-- Stream Type -->
+                    <fs.s3a.input.stream.type>${stream}</fs.s3a.input.stream.type>
                     <test.unique.fork.id>job-${job.id}</test.unique.fork.id>
                   </systemPropertyVariables>
                   <forkedProcessTimeoutInSeconds>${fs.s3a.scale.test.timeout}</forkedProcessTimeoutInSeconds>
@@ -308,7 +307,20 @@
         </property>
       </activation>
       <properties>
-        <fs.s3a.prefetch.enabled>true</fs.s3a.prefetch.enabled>
+        <stream>prefetch</stream>
+      </properties>
+    </profile>
+
+    <!-- Switch to the analytics input stream-->
+    <profile>
+      <id>analytics</id>
+      <activation>
+        <property>
+          <name>analytics</name>
+        </property>
+      </activation>
+      <properties>
+        <stream>analytics</stream>
       </properties>
     </profile>
 

+ 47 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.fs.s3a;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.s3a.impl.streams.StreamIntegration;
 import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory;
 
 import java.time.Duration;
@@ -1580,14 +1581,60 @@ public final class Constants {
    */
   public static final String AWS_AUTH_CLASS_PREFIX = "com.amazonaws.auth";
 
+  /**
+   * Input stream type: {@value}.
+   */
+  public static final String INPUT_STREAM_TYPE = "fs.s3a.input.stream.type";
+
+  /**
+   * The classic input stream: {@value}.
+   */
+  public static final String INPUT_STREAM_TYPE_CLASSIC =
+      StreamIntegration.CLASSIC;
+
+  /**
+   * The prefetching input stream: {@value}.
+   */
+  public static final String INPUT_STREAM_TYPE_PREFETCH = StreamIntegration.PREFETCH;
+
+  /**
+   * The analytics input stream: {@value}.
+   */
+  public static final String INPUT_STREAM_TYPE_ANALYTICS =
+      StreamIntegration.ANALYTICS;
+
+  /**
+   * Request the default input stream,
+   * whatever it is for this release: {@value}.
+   */
+  public static final String INPUT_STREAM_TYPE_DEFAULT = StreamIntegration.DEFAULT;
+
+  /**
+   * The custom input stream type: {@value}".
+   * If set, the classname is loaded from
+   * {@link #INPUT_STREAM_CUSTOM_FACTORY}.
+   * <p>
+   * This option is primarily for testing as it can
+   * be used to generated failures.
+   */
+  public static final String INPUT_STREAM_TYPE_CUSTOM =
+      StreamIntegration.CUSTOM;
+
+  /**
+   * Classname of the factory to instantiate for custom streams: {@value}.
+   */
+  public static final String INPUT_STREAM_CUSTOM_FACTORY = "fs.s3a.input.stream.custom.factory";
+
   /**
    * Controls whether the prefetching input stream is enabled.
    */
+  @Deprecated
   public static final String PREFETCH_ENABLED_KEY = "fs.s3a.prefetch.enabled";
 
   /**
    * Default option as to whether the prefetching input stream is enabled.
    */
+  @Deprecated
   public static final boolean  PREFETCH_ENABLED_DEFAULT = false;
 
   // If the default values are used, each file opened for reading will consume

+ 140 - 201
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java

@@ -51,14 +51,11 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import javax.annotation.Nullable;
 
-import software.amazon.awssdk.core.ResponseInputStream;
 import software.amazon.awssdk.core.exception.SdkException;
 import software.amazon.awssdk.services.s3.S3Client;
 import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest;
 import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadResponse;
 import software.amazon.awssdk.services.s3.model.GetBucketLocationRequest;
-import software.amazon.awssdk.services.s3.model.GetObjectRequest;
-import software.amazon.awssdk.services.s3.model.GetObjectResponse;
 import software.amazon.awssdk.services.s3.model.HeadBucketRequest;
 import software.amazon.awssdk.services.s3.model.HeadBucketResponse;
 import software.amazon.awssdk.services.s3.model.MultipartUpload;
@@ -109,6 +106,7 @@ import org.apache.hadoop.fs.impl.OpenFileParameters;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.s3a.api.PerformanceFlagEnum;
 import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A;
+import org.apache.hadoop.fs.s3a.audit.AuditorFlags;
 import org.apache.hadoop.fs.s3a.auth.SignerManager;
 import org.apache.hadoop.fs.s3a.auth.delegation.DelegationOperations;
 import org.apache.hadoop.fs.s3a.auth.delegation.DelegationTokenProvider;
@@ -125,6 +123,7 @@ import org.apache.hadoop.fs.s3a.impl.ConfigurationHelper;
 import org.apache.hadoop.fs.s3a.impl.ContextAccessors;
 import org.apache.hadoop.fs.s3a.impl.CopyFromLocalOperation;
 import org.apache.hadoop.fs.s3a.impl.CreateFileBuilder;
+import org.apache.hadoop.fs.s3a.impl.InputStreamCallbacksImpl;
 import org.apache.hadoop.fs.s3a.impl.S3AFileSystemOperations;
 import org.apache.hadoop.fs.s3a.impl.CSEV1CompatibleS3AFileSystemOperations;
 import org.apache.hadoop.fs.s3a.impl.CSEMaterials;
@@ -148,7 +147,9 @@ import org.apache.hadoop.fs.s3a.impl.StoreContextBuilder;
 import org.apache.hadoop.fs.s3a.impl.StoreContextFactory;
 import org.apache.hadoop.fs.s3a.impl.UploadContentProviders;
 import org.apache.hadoop.fs.s3a.impl.CSEUtils;
-import org.apache.hadoop.fs.s3a.prefetch.S3APrefetchingInputStream;
+import org.apache.hadoop.fs.s3a.impl.streams.ObjectReadParameters;
+import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamCallbacks;
+import org.apache.hadoop.fs.s3a.impl.streams.StreamFactoryRequirements;
 import org.apache.hadoop.fs.s3a.tools.MarkerToolOperations;
 import org.apache.hadoop.fs.s3a.tools.MarkerToolOperationsImpl;
 import org.apache.hadoop.fs.statistics.DurationTracker;
@@ -157,7 +158,6 @@ import org.apache.hadoop.fs.statistics.FileSystemStatisticNames;
 import org.apache.hadoop.fs.statistics.IOStatistics;
 import org.apache.hadoop.fs.statistics.IOStatisticsSource;
 import org.apache.hadoop.fs.statistics.IOStatisticsContext;
-import org.apache.hadoop.fs.statistics.StreamStatisticNames;
 import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
 import org.apache.hadoop.fs.store.LogExactlyOnce;
 import org.apache.hadoop.fs.store.audit.AuditEntryPoint;
@@ -256,6 +256,7 @@ import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion;
 import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.logDnsLookup;
 import static org.apache.hadoop.fs.s3a.impl.S3ExpressStorage.STORE_CAPABILITY_S3_EXPRESS_STORAGE;
 import static org.apache.hadoop.fs.s3a.impl.S3ExpressStorage.isS3ExpressStore;
+import static org.apache.hadoop.fs.s3a.impl.streams.StreamFactoryRequirements.Requirements.ExpectUnauditedGetRequests;
 import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.checkNoS3Guard;
 import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.logIOStatisticsAtLevel;
 import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_CONTINUE_LIST_REQUEST;
@@ -334,18 +335,13 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   private ExecutorService boundedThreadPool;
   private ThreadPoolExecutor unboundedThreadPool;
 
-  // S3 reads are prefetched asynchronously using this future pool.
+  /**
+   * Future pool built on the bounded thread pool.
+   * S3 reads are prefetched asynchronously using this future pool if the
+   * Stream Factory requests it.
+   */
   private ExecutorServiceFuturePool futurePool;
 
-  // If true, the prefetching input stream is used for reads.
-  private boolean prefetchEnabled;
-
-  // Size in bytes of a single prefetch block.
-  private int prefetchBlockSize;
-
-  // Size of prefetch queue (in number of blocks).
-  private int prefetchBlockCount;
-
   private int executorCapacity;
   private long multiPartThreshold;
   public static final Logger LOG = LoggerFactory.getLogger(S3AFileSystem.class);
@@ -353,7 +349,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   /** Log to warn of storage class configuration problems. */
   private static final LogExactlyOnce STORAGE_CLASS_WARNING = new LogExactlyOnce(LOG);
 
-  private LocalDirAllocator directoryAllocator;
   private String cannedACL;
 
   /**
@@ -381,12 +376,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   /** Vectored IO context. */
   private VectoredIOContext vectoredIOContext;
 
-  /**
-   * Maximum number of active range read operation a single
-   * input stream can have.
-   */
-  private int vectoredActiveRangeReads;
-
   private long readAhead;
   private ChangeDetectionPolicy changeDetectionPolicy;
   private final AtomicBoolean closed = new AtomicBoolean(false);
@@ -540,7 +529,24 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     addDeprecatedKeys();
   }
 
-  /** Called after a new FileSystem instance is constructed.
+  /**
+   * Initialize the filesystem.
+   * <p>
+   * This is called after a new FileSystem instance is constructed -but
+   * within the filesystem cache creation process.
+   * A slow start here while multiple threads are calling
+   *  will result in multiple
+   * instances of the filesystem being created -and all but one deleted.
+   * <i>Keep this as fast as possible, and avoid network IO</i>.
+   * <p>
+   * This performs the majority of the filesystem setup, and as things
+   * are intermixed the ordering of operations is very sensitive.
+   * Be very careful when moving things.
+   * <p>
+   * To help identify where filesystem instances are created,
+   * the full stack is logged at TRACE.
+   * <p>
+   * Also, ignore checkstyle complaints about method length.
    * @param name a uri whose authority section names the host, port, etc.
    *   for this FileSystem
    * @param originalConf the configuration to use for the FS. The
@@ -661,22 +667,11 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       dirOperationsPurgeUploads = conf.getBoolean(DIRECTORY_OPERATIONS_PURGE_UPLOADS,
           s3ExpressStore);
 
-      this.prefetchEnabled = conf.getBoolean(PREFETCH_ENABLED_KEY, PREFETCH_ENABLED_DEFAULT);
-      long prefetchBlockSizeLong =
-          longBytesOption(conf, PREFETCH_BLOCK_SIZE_KEY, PREFETCH_BLOCK_DEFAULT_SIZE, 1);
-      if (prefetchBlockSizeLong > (long) Integer.MAX_VALUE) {
-        throw new IOException("S3A prefatch block size exceeds int limit");
-      }
-      this.prefetchBlockSize = (int) prefetchBlockSizeLong;
-      this.prefetchBlockCount =
-          intOption(conf, PREFETCH_BLOCK_COUNT_KEY, PREFETCH_BLOCK_DEFAULT_COUNT, 1);
       this.isMultipartUploadEnabled = conf.getBoolean(MULTIPART_UPLOADS_ENABLED,
           DEFAULT_MULTIPART_UPLOAD_ENABLED);
       // multipart copy and upload are the same; this just makes it explicit
       this.isMultipartCopyEnabled = isMultipartUploadEnabled;
 
-      initThreadPools(conf);
-
       int listVersion = conf.getInt(LIST_VERSION, DEFAULT_LIST_VERSION);
       if (listVersion < 1 || listVersion > 2) {
         LOG.warn("Configured fs.s3a.list.version {} is invalid, forcing " +
@@ -693,6 +688,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       signerManager.initCustomSigners();
 
       // start auditing
+      // extra configuration will be passed down later.
       initializeAuditService();
 
       // create the requestFactory.
@@ -706,6 +702,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       // the FS came with a DT
       // this may do some patching of the configuration (e.g. setting
       // the encryption algorithms)
+      // requires the audit manager to be initialized.
       ClientManager clientManager = createClientManager(name, delegationTokensEnabled);
 
       inputPolicy = S3AInputPolicy.getPolicy(
@@ -778,9 +775,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
           longBytesOption(conf, ASYNC_DRAIN_THRESHOLD,
                         DEFAULT_ASYNC_DRAIN_THRESHOLD, 0),
           inputPolicy);
-      vectoredActiveRangeReads = intOption(conf,
-              AWS_S3_VECTOR_ACTIVE_RANGE_READS, DEFAULT_AWS_S3_VECTOR_ACTIVE_RANGE_READS, 1);
-      vectoredIOContext = populateVectoredIOContext(conf);
       scheme = (this.uri != null && this.uri.getScheme() != null) ? this.uri.getScheme() : FS_S3A;
       optimizedCopyFromLocal = conf.getBoolean(OPTIMIZED_COPY_FROM_LOCAL,
           OPTIMIZED_COPY_FROM_LOCAL_DEFAULT);
@@ -788,12 +782,32 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       s3AccessGrantsEnabled = conf.getBoolean(AWS_S3_ACCESS_GRANTS_ENABLED, false);
 
       int rateLimitCapacity = intOption(conf, S3A_IO_RATE_LIMIT, DEFAULT_S3A_IO_RATE_LIMIT, 0);
-      // now create the store
+
+      // now create and initialize the store
       store = createS3AStore(clientManager, rateLimitCapacity);
       // the s3 client is created through the store, rather than
       // directly through the client manager.
       // this is to aid mocking.
-      s3Client = store.getOrCreateS3Client();
+      s3Client = getStore().getOrCreateS3Client();
+
+      // get the input stream factory requirements.
+      final StreamFactoryRequirements factoryRequirements =
+          getStore().factoryRequirements();
+
+      // If the input stream can issue get requests outside spans,
+      // the auditor is forced to disable rejection of unaudited requests.
+      final EnumSet<AuditorFlags> flags = EnumSet.noneOf(AuditorFlags.class);
+      if (factoryRequirements.requires(ExpectUnauditedGetRequests)) {
+        flags.add(AuditorFlags.PermitOutOfBandOperations);
+      }
+      getAuditManager().setAuditFlags(flags);
+      // get the vector IO context from the factory.
+      vectoredIOContext = factoryRequirements.vectoredIOContext();
+
+      // thread pool init requires store to be created and
+      // the stream factory requirements to include its own requirements.
+      initThreadPools();
+
       // The filesystem is now ready to perform operations against
       // S3
       // This initiates a probe against S3 for the bucket existing.
@@ -836,7 +850,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
 
 
   /**
-   * Create the S3AStore instance.
+   * Create and start the S3AStore instance.
    * This is protected so that tests can override it.
    * @param clientManager client manager
    * @param rateLimitCapacity rate limit
@@ -845,7 +859,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   @VisibleForTesting
   protected S3AStore createS3AStore(final ClientManager clientManager,
       final int rateLimitCapacity) {
-    return new S3AStoreBuilder()
+    final S3AStore st = new S3AStoreBuilder()
         .withAuditSpanSource(getAuditManager())
         .withClientManager(clientManager)
         .withDurationTrackerFactory(getDurationTrackerFactory())
@@ -857,23 +871,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         .withReadRateLimiter(unlimitedRate())
         .withWriteRateLimiter(RateLimitingFactory.create(rateLimitCapacity))
         .build();
-  }
-
-  /**
-   * Populates the configurations related to vectored IO operation
-   * in the context which has to passed down to input streams.
-   * @param conf configuration object.
-   * @return VectoredIOContext.
-   */
-  private VectoredIOContext populateVectoredIOContext(Configuration conf) {
-    final int minSeekVectored = (int) longBytesOption(conf, AWS_S3_VECTOR_READS_MIN_SEEK_SIZE,
-            DEFAULT_AWS_S3_VECTOR_READS_MIN_SEEK_SIZE, 0);
-    final int maxReadSizeVectored = (int) longBytesOption(conf, AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE,
-            DEFAULT_AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE, 0);
-    return new VectoredIOContext()
-            .setMinSeekForVectoredReads(minSeekVectored)
-            .setMaxReadSizeForVectoredReads(maxReadSizeVectored)
-            .build();
+    st.init(getConf());
+    st.start();
+    return st;
   }
 
   /**
@@ -939,12 +939,16 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   }
 
   /**
-   * Initialize the thread pool.
+   * Initialize the thread pools.
+   * <p>
    * This must be re-invoked after replacing the S3Client during test
    * runs.
+   * <p>
+   * It requires the S3Store to have been instantiated.
    * @param conf configuration.
    */
-  private void initThreadPools(Configuration conf) {
+  private void initThreadPools() {
+    Configuration conf = getConf();
     final String name = "s3a-transfer-" + getBucket();
     int maxThreads = conf.getInt(MAX_THREADS, DEFAULT_MAX_THREADS);
     if (maxThreads < 2) {
@@ -960,7 +964,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
             TimeUnit.SECONDS,
             Duration.ZERO).getSeconds();
 
-    int numPrefetchThreads = this.prefetchEnabled ? this.prefetchBlockCount : 0;
+    final StreamFactoryRequirements factoryRequirements =
+        getStore().factoryRequirements();
+    int numPrefetchThreads = factoryRequirements.sharedThreads();
 
     int activeTasksForBoundedThreadPool = maxThreads;
     int waitingTasksForBoundedThreadPool = maxThreads + totalTasks + numPrefetchThreads;
@@ -978,7 +984,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     unboundedThreadPool.allowCoreThreadTimeOut(true);
     executorCapacity = intOption(conf,
         EXECUTOR_CAPACITY, DEFAULT_EXECUTOR_CAPACITY, 1);
-    if (prefetchEnabled) {
+    if (factoryRequirements.requiresFuturePool()) {
+      // create a future pool.
       final S3AInputStreamStatistics s3AInputStreamStatistics =
           statisticsContext.newInputStreamStatistics();
       futurePool = new ExecutorServiceFuturePool(
@@ -1329,6 +1336,14 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     return performanceFlags;
   }
 
+  /**
+   * Get the store for low-level operations.
+   * @return the store the S3A FS is working through.
+   */
+  private S3AStore getStore() {
+    return store;
+  }
+
   /**
    * Implementation of all operations used by delegation tokens.
    */
@@ -1534,7 +1549,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
 
     @Override
     public S3AStore getStore() {
-      return store;
+      return S3AFileSystem.this.getStore();
     }
 
     /**
@@ -1663,28 +1678,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    */
   File createTmpFileForWrite(String pathStr, long size,
       Configuration conf) throws IOException {
-    initLocalDirAllocatorIfNotInitialized(conf);
-    Path path = directoryAllocator.getLocalPathForWrite(pathStr,
-        size, conf);
-    File dir = new File(path.getParent().toUri().getPath());
-    String prefix = path.getName();
-    // create a temp file on this directory
-    return File.createTempFile(prefix, null, dir);
-  }
 
-  /**
-   * Initialize dir allocator if not already initialized.
-   *
-   * @param conf The Configuration object.
-   */
-  private void initLocalDirAllocatorIfNotInitialized(Configuration conf) {
-    if (directoryAllocator == null) {
-      synchronized (this) {
-        String bufferDir = conf.get(BUFFER_DIR) != null
-            ? BUFFER_DIR : HADOOP_TMP_DIR;
-        directoryAllocator = new LocalDirAllocator(bufferDir);
-      }
-    }
+    return getS3AInternals().getStore().createTemporaryFileForWriting(pathStr, size, conf);
   }
 
   /**
@@ -1877,100 +1872,41 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     fileInformation.applyOptions(readContext);
     LOG.debug("Opening '{}'", readContext);
 
-    if (this.prefetchEnabled) {
-      Configuration configuration = getConf();
-      initLocalDirAllocatorIfNotInitialized(configuration);
-      return new FSDataInputStream(
-          new S3APrefetchingInputStream(
-              readContext.build(),
-              createObjectAttributes(path, fileStatus),
-              createInputStreamCallbacks(auditSpan),
-              inputStreamStats,
-              configuration,
-              directoryAllocator));
-    } else {
-      return new FSDataInputStream(
-          new S3AInputStream(
-              readContext.build(),
-              createObjectAttributes(path, fileStatus),
-              createInputStreamCallbacks(auditSpan),
-                  inputStreamStats,
-                  new SemaphoredDelegatingExecutor(
-                          boundedThreadPool,
-                          vectoredActiveRangeReads,
-                          true,
-                          inputStreamStats)));
-    }
-  }
-
-  /**
-   * Override point: create the callbacks for S3AInputStream.
-   * @return an implementation of the InputStreamCallbacks,
-   */
-  private S3AInputStream.InputStreamCallbacks createInputStreamCallbacks(
+    // what does the stream need
+    final StreamFactoryRequirements requirements =
+        getStore().factoryRequirements();
+
+    // calculate the permit count.
+    final int permitCount = requirements.streamThreads()
+        + requirements.vectoredIOContext().getVectoredActiveRangeReads();
+    // create an executor which is a subset of the
+    // bounded thread pool.
+    final SemaphoredDelegatingExecutor pool = new SemaphoredDelegatingExecutor(
+        boundedThreadPool,
+        permitCount,
+        true,
+        inputStreamStats);
+
+    // do not validate() the parameters as the store
+    // completes this.
+    ObjectReadParameters parameters = new ObjectReadParameters()
+        .withBoundedThreadPool(pool)
+        .withCallbacks(createInputStreamCallbacks(auditSpan))
+        .withContext(readContext.build())
+        .withObjectAttributes(createObjectAttributes(path, fileStatus))
+        .withStreamStatistics(inputStreamStats);
+    return new FSDataInputStream(getStore().readObject(parameters));
+  }
+
+  /**
+   * Override point: create the callbacks for ObjectInputStream.
+   * @return an implementation of ObjectInputStreamCallbacks.
+   */
+  private ObjectInputStreamCallbacks createInputStreamCallbacks(
       final AuditSpan auditSpan) {
-    return new InputStreamCallbacksImpl(auditSpan);
+    return new InputStreamCallbacksImpl(auditSpan, getStore(), fsHandler, unboundedThreadPool);
   }
 
-  /**
-   * Operations needed by S3AInputStream to read data.
-   */
-  private final class InputStreamCallbacksImpl implements
-      S3AInputStream.InputStreamCallbacks {
-
-    /**
-     * Audit span to activate before each call.
-     */
-    private final AuditSpan auditSpan;
-
-    /**
-     * Create.
-     * @param auditSpan Audit span to activate before each call.
-     */
-    private InputStreamCallbacksImpl(final AuditSpan auditSpan) {
-      this.auditSpan = requireNonNull(auditSpan);
-    }
-
-    /**
-     * Closes the audit span.
-     */
-    @Override
-    public void close()  {
-      auditSpan.close();
-    }
-
-    @Override
-    public GetObjectRequest.Builder newGetRequestBuilder(final String key) {
-      // active the audit span used for the operation
-      try (AuditSpan span = auditSpan.activate()) {
-        return getRequestFactory().newGetObjectRequestBuilder(key);
-      }
-    }
-
-    @Override
-    public ResponseInputStream<GetObjectResponse> getObject(GetObjectRequest request) throws
-        IOException {
-      // active the audit span used for the operation
-      try (AuditSpan span = auditSpan.activate()) {
-        return fsHandler.getObject(store, request, getRequestFactory());
-      }
-    }
-
-    @Override
-    public <T> CompletableFuture<T> submit(final CallableRaisingIOE<T> operation) {
-      CompletableFuture<T> result = new CompletableFuture<>();
-      unboundedThreadPool.submit(() ->
-          LambdaUtils.eval(result, () -> {
-            LOG.debug("Starting submitted operation in {}", auditSpan.getSpanId());
-            try (AuditSpan span = auditSpan.activate()) {
-              return operation.apply();
-            } finally {
-              LOG.debug("Completed submitted operation in {}", auditSpan.getSpanId());
-            }
-          }));
-      return result;
-    }
-  }
 
   /**
    * Callbacks for WriteOperationHelper.
@@ -1982,7 +1918,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     @Retries.OnceRaw
     public CompleteMultipartUploadResponse completeMultipartUpload(
         CompleteMultipartUploadRequest request) {
-      return store.completeMultipartUpload(request);
+      return getStore().completeMultipartUpload(request);
     }
 
     @Override
@@ -1992,7 +1928,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         final RequestBody body,
         final DurationTrackerFactory durationTrackerFactory)
         throws AwsServiceException, UncheckedIOException {
-      return store.uploadPart(request, body, durationTrackerFactory);
+      return getStore().uploadPart(request, body, durationTrackerFactory);
     }
 
   }
@@ -2016,9 +1952,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         fileStatus,
         vectoredIOContext,
         IOStatisticsContext.getCurrentIOStatisticsContext().getAggregator(),
-        futurePool,
-        prefetchBlockSize,
-        prefetchBlockCount)
+        futurePool)
         .withAuditSpan(auditSpan);
     openFileHelper.applyDefaultOptions(roc);
     return roc.build();
@@ -2756,7 +2690,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
      */
     @Override
     public long getObjectSize(S3Object s3Object) throws IOException {
-      return fsHandler.getS3ObjectSize(s3Object.key(), s3Object.size(), store, null);
+      return fsHandler.getS3ObjectSize(s3Object.key(), s3Object.size(), getStore(), null);
     }
 
     @Override
@@ -2987,7 +2921,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    */
   protected DurationTrackerFactory nonNullDurationTrackerFactory(
       DurationTrackerFactory factory) {
-    return store.nonNullDurationTrackerFactory(factory);
+    return getStore().nonNullDurationTrackerFactory(factory);
   }
 
   /**
@@ -3025,7 +2959,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       ChangeTracker changeTracker,
       Invoker changeInvoker,
       String operation) throws IOException {
-    return store.headObject(key, changeTracker, changeInvoker, fsHandler, operation);
+    return getStore().headObject(key, changeTracker, changeInvoker, fsHandler, operation);
   }
 
   /**
@@ -3173,7 +3107,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   protected void deleteObject(String key)
       throws SdkException, IOException {
     incrementWriteOperations();
-    store.deleteObject(getRequestFactory()
+    getStore().deleteObject(getRequestFactory()
         .newDeleteObjectRequestBuilder(key)
         .build());
   }
@@ -3227,7 +3161,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   private DeleteObjectsResponse deleteObjects(DeleteObjectsRequest deleteRequest)
       throws MultiObjectDeleteException, SdkException, IOException {
     incrementWriteOperations();
-    DeleteObjectsResponse response = store.deleteObjects(deleteRequest).getValue();
+    DeleteObjectsResponse response = getStore().deleteObjects(deleteRequest).getValue();
     if (!response.errors().isEmpty()) {
       throw new MultiObjectDeleteException(response.errors());
     }
@@ -3270,7 +3204,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   @Retries.OnceRaw
   public UploadInfo putObject(PutObjectRequest putObjectRequest, File file,
       ProgressableProgressListener listener) throws IOException {
-    return store.putObject(putObjectRequest, file, listener);
+    return getStore().putObject(putObjectRequest, file, listener);
   }
 
   /**
@@ -3369,7 +3303,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * @param bytes bytes in the request.
    */
   protected void incrementPutStartStatistics(long bytes) {
-    store.incrementPutStartStatistics(bytes);
+    getStore().incrementPutStartStatistics(bytes);
   }
 
   /**
@@ -3380,7 +3314,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * @param bytes bytes in the request.
    */
   protected void incrementPutCompletedStatistics(boolean success, long bytes) {
-    store.incrementPutCompletedStatistics(success, bytes);
+    getStore().incrementPutCompletedStatistics(success, bytes);
   }
 
   /**
@@ -3391,7 +3325,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * @param bytes bytes successfully uploaded.
    */
   protected void incrementPutProgressStatistics(String key, long bytes) {
-    store.incrementPutProgressStatistics(key, bytes);
+    getStore().incrementPutProgressStatistics(key, bytes);
   }
 
   /**
@@ -4259,7 +4193,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     ProgressableProgressListener listener =
         new ProgressableProgressListener(store, key, progress);
     UploadInfo info = putObject(putObjectRequest, file, listener);
-    PutObjectResponse result = store.waitForUploadCompletion(key, info).response();
+    PutObjectResponse result = getStore().waitForUploadCompletion(key, info).response();
     listener.uploadCompleted(info.getFileUpload());
     return result;
   }
@@ -4354,22 +4288,24 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   protected synchronized void stopAllServices() {
     try {
       trackDuration(getDurationTrackerFactory(), FILESYSTEM_CLOSE.getSymbol(), () -> {
-        closeAutocloseables(LOG, store);
+        closeAutocloseables(LOG, getStore());
         store = null;
         s3Client = null;
 
         // At this point the S3A client is shut down,
         // now the executor pools are closed
+
+        // shut future pool first as it wraps the bounded thread pool
+        if (futurePool != null) {
+          futurePool.shutdown(LOG, THREAD_POOL_SHUTDOWN_DELAY_SECONDS, TimeUnit.SECONDS);
+          futurePool = null;
+        }
         HadoopExecutors.shutdown(boundedThreadPool, LOG,
             THREAD_POOL_SHUTDOWN_DELAY_SECONDS, TimeUnit.SECONDS);
         boundedThreadPool = null;
         HadoopExecutors.shutdown(unboundedThreadPool, LOG,
             THREAD_POOL_SHUTDOWN_DELAY_SECONDS, TimeUnit.SECONDS);
         unboundedThreadPool = null;
-        if (futurePool != null) {
-          futurePool.shutdown(LOG, THREAD_POOL_SHUTDOWN_DELAY_SECONDS, TimeUnit.SECONDS);
-          futurePool = null;
-        }
         // other services are shutdown.
         cleanupWithLogger(LOG,
             delegationTokens.orElse(null),
@@ -4575,7 +4511,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
           () -> {
             incrementStatistic(OBJECT_COPY_REQUESTS);
 
-            Copy copy = store.getOrCreateTransferManager().copy(
+            Copy copy = getStore().getOrCreateTransferManager().copy(
                 CopyRequest.builder()
                     .copyObjectRequest(copyRequest)
                     .build());
@@ -5434,15 +5370,18 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     case AWS_S3_ACCESS_GRANTS_ENABLED:
       return s3AccessGrantsEnabled;
 
-      // stream leak detection.
-    case StreamStatisticNames.STREAM_LEAKS:
-      return !prefetchEnabled;
-
     default:
       // is it a performance flag?
       if (performanceFlags.hasCapability(capability)) {
         return true;
       }
+
+      // ask the store for what capabilities it offers
+      // this may include input and output capabilites -and more
+      if (getStore() != null && getStore().hasPathCapability(path, capability)) {
+        return true;
+      }
+
       // fall through
     }
 
@@ -5703,7 +5642,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    */
   protected BulkDeleteOperation.BulkDeleteOperationCallbacks createBulkDeleteCallbacks(
       Path path, int pageSize, AuditSpanS3A span) {
-    return new BulkDeleteOperationCallbacksImpl(store, pathToKey(path), pageSize, span);
+    return new BulkDeleteOperationCallbacksImpl(getStore(), pathToKey(path), pageSize, span);
   }
 
 }

+ 99 - 289
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java

@@ -19,7 +19,6 @@
 package org.apache.hadoop.fs.s3a;
 
 import javax.annotation.Nullable;
-import java.io.Closeable;
 import java.io.EOFException;
 import java.io.IOException;
 import java.io.InputStream;
@@ -30,7 +29,6 @@ import java.util.List;
 import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.function.IntFunction;
 
@@ -41,7 +39,9 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.fs.impl.LeakReporter;
-import org.apache.hadoop.fs.statistics.StreamStatisticNames;
+import org.apache.hadoop.fs.s3a.impl.streams.InputStreamType;
+import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStream;
+import org.apache.hadoop.fs.s3a.impl.streams.ObjectReadParameters;
 import org.apache.hadoop.util.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -49,7 +49,6 @@ import org.apache.hadoop.classification.VisibleForTesting;
 import org.apache.hadoop.fs.CanSetReadahead;
 import org.apache.hadoop.fs.CanUnbuffer;
 import org.apache.hadoop.fs.FSExceptionMessages;
-import org.apache.hadoop.fs.FSInputStream;
 import org.apache.hadoop.fs.FileRange;
 import org.apache.hadoop.fs.StreamCapabilities;
 import org.apache.hadoop.fs.impl.CombinedFileRange;
@@ -57,17 +56,11 @@ import org.apache.hadoop.fs.VectoredReadUtils;
 import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
 import org.apache.hadoop.fs.s3a.impl.InternalConstants;
 import org.apache.hadoop.fs.s3a.impl.SDKStreamDrainer;
-import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
 import org.apache.hadoop.fs.statistics.DurationTracker;
-import org.apache.hadoop.fs.statistics.IOStatistics;
-import org.apache.hadoop.fs.statistics.IOStatisticsAggregator;
 import org.apache.hadoop.fs.statistics.IOStatisticsSource;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.util.functional.CallableRaisingIOE;
 
 
-import static java.util.Objects.requireNonNull;
-import static org.apache.commons.lang3.StringUtils.isNotEmpty;
 import static org.apache.hadoop.fs.VectoredReadUtils.isOrderedDisjoint;
 import static org.apache.hadoop.fs.VectoredReadUtils.mergeSortedRanges;
 import static org.apache.hadoop.fs.VectoredReadUtils.validateAndSortRanges;
@@ -94,7 +87,7 @@ import static org.apache.hadoop.util.functional.FutureIO.awaitFuture;
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
+public class S3AInputStream extends ObjectInputStream implements CanSetReadahead,
         CanUnbuffer, StreamCapabilities, IOStatisticsSource {
 
   public static final String E_NEGATIVE_READAHEAD_VALUE
@@ -134,6 +127,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
    * and returned in {@link #getPos()}.
    */
   private long pos;
+
   /**
    * Closed bit. Volatile so reads are non-blocking.
    * Updates must be in a synchronized block to guarantee an atomic check and
@@ -144,35 +138,14 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
    * Input stream returned by a getObject call.
    */
   private ResponseInputStream<GetObjectResponse> wrappedStream;
-  private final S3AReadOpContext context;
-  private final InputStreamCallbacks client;
-
-  /**
-   * Thread pool used for vectored IO operation.
-   */
-  private final ExecutorService boundedThreadPool;
-  private final String bucket;
-  private final String key;
-  private final String pathStr;
-
-  /**
-   * Content length from HEAD or openFile option.
-   */
-  private final long contentLength;
   /**
    * Content length in format for vector IO.
    */
   private final Optional<Long> fileLength;
 
-  private final String uri;
 
-  private final S3AInputStreamStatistics streamStatistics;
-  private S3AInputPolicy inputPolicy;
   private long readahead = Constants.DEFAULT_READAHEAD_RANGE;
 
-  /** Vectored IO context. */
-  private final VectoredIOContext vectoredIOContext;
-
   /**
    * This is the actual position within the object, used by
    * lazy seek to decide whether to seek on the next read or not.
@@ -193,96 +166,32 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
   /** change tracker. */
   private final ChangeTracker changeTracker;
 
-  /**
-   * IOStatistics report.
-   */
-  private final IOStatistics ioStatistics;
-
   /**
    * Threshold for stream reads to switch to
    * asynchronous draining.
    */
-  private long asyncDrainThreshold;
-
-  /** Aggregator used to aggregate per thread IOStatistics. */
-  private final IOStatisticsAggregator threadIOStatistics;
-
-  /**
-   * Report of leaks.
-   * with report and abort unclosed streams in finalize().
-   */
-  private final LeakReporter leakReporter;
+  private final long asyncDrainThreshold;
 
   /**
    * Create the stream.
    * This does not attempt to open it; that is only done on the first
    * actual read() operation.
-   * @param ctx operation context
-   * @param s3Attributes object attributes
-   * @param client S3 client to use
-   * @param streamStatistics stream io stats.
-   * @param boundedThreadPool thread pool to use.
-   */
-  public S3AInputStream(S3AReadOpContext ctx,
-                        S3ObjectAttributes s3Attributes,
-                        InputStreamCallbacks client,
-                        S3AInputStreamStatistics streamStatistics,
-                        ExecutorService boundedThreadPool) {
-    Preconditions.checkArgument(isNotEmpty(s3Attributes.getBucket()),
-        "No Bucket");
-    Preconditions.checkArgument(isNotEmpty(s3Attributes.getKey()), "No Key");
-    long l = s3Attributes.getLen();
-    Preconditions.checkArgument(l >= 0, "Negative content length");
-    this.context = ctx;
-    this.bucket = s3Attributes.getBucket();
-    this.key = s3Attributes.getKey();
-    this.pathStr = s3Attributes.getPath().toString();
-    this.contentLength = l;
-    this.fileLength = Optional.of(contentLength);
-    this.client = client;
-    this.uri = "s3a://" + this.bucket + "/" + this.key;
-    this.streamStatistics = streamStatistics;
-    this.ioStatistics = streamStatistics.getIOStatistics();
-    this.changeTracker = new ChangeTracker(uri,
-        ctx.getChangeDetectionPolicy(),
-        streamStatistics.getChangeTrackerStatistics(),
-        s3Attributes);
-    setInputPolicy(ctx.getInputPolicy());
-    setReadahead(ctx.getReadahead());
-    this.asyncDrainThreshold = ctx.getAsyncDrainThreshold();
-    this.boundedThreadPool = boundedThreadPool;
-    this.vectoredIOContext = context.getVectoredIOContext();
-    this.threadIOStatistics = requireNonNull(ctx.getIOStatisticsAggregator());
-    // build the leak reporter
-    this.leakReporter = new LeakReporter(
-        "Stream not closed while reading " + uri,
-        this::isStreamOpen,
-        () -> abortInFinalizer());
-  }
-
-  /**
-   * Finalizer.
-   * <p>
-   * Verify that the inner stream is closed.
-   * <p>
-   * If it is not, it means streams are being leaked in application code.
-   * Log a warning, including the stack trace of the caller,
-   * then abort the stream.
-   * <p>
-   * This does not attempt to invoke {@link #close()} as that is
-   * a more complex operation, and this method is being executed
-   * during a GC finalization phase.
-   * <p>
-   * Applications MUST close their streams; this is a defensive
-   * operation to return http connections and warn the end users
-   * that their applications are at risk of running out of connections.
    *
-   * {@inheritDoc}
+   * @param parameters creation parameters.
    */
-  @Override
-  protected void finalize() throws Throwable {
-    leakReporter.close();
-    super.finalize();
+  public S3AInputStream(ObjectReadParameters parameters) {
+
+    super(InputStreamType.Classic, parameters);
+
+
+    this.fileLength = Optional.of(getContentLength());
+    S3AReadOpContext context = getContext();
+    this.changeTracker = new ChangeTracker(getUri(),
+        context.getChangeDetectionPolicy(),
+        getS3AStreamStatistics().getChangeTrackerStatistics(),
+        getObjectAttributes());
+    setReadahead(context.getReadahead());
+    this.asyncDrainThreshold = context.getAsyncDrainThreshold();
   }
 
   /**
@@ -290,7 +199,8 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
    * Not synchronized; the flag is volatile.
    * @return true if the stream is still open.
    */
-  private boolean isStreamOpen() {
+  @Override
+  protected boolean isStreamOpen() {
     return !closed;
   }
 
@@ -298,10 +208,11 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
    * Brute force stream close; invoked by {@link LeakReporter}.
    * All exceptions raised are ignored.
    */
-  private void abortInFinalizer() {
+  @Override
+  protected void abortInFinalizer() {
     try {
       // stream was leaked: update statistic
-      streamStatistics.streamLeaked();
+      getS3AStreamStatistics().streamLeaked();
       // abort the stream. This merges statistics into the filesystem.
       closeStream("finalize()", true, true).get();
     } catch (InterruptedException | ExecutionException ignroed) {
@@ -309,32 +220,12 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
     }
   }
 
-  /**
-   * Set/update the input policy of the stream.
-   * This updates the stream statistics.
-   * @param inputPolicy new input policy.
-   */
-  private void setInputPolicy(S3AInputPolicy inputPolicy) {
-    LOG.debug("Switching to input policy {}", inputPolicy);
-    this.inputPolicy = inputPolicy;
-    streamStatistics.inputPolicySet(inputPolicy.ordinal());
-  }
-
-  /**
-   * Get the current input policy.
-   * @return input policy.
-   */
-  @VisibleForTesting
-  public S3AInputPolicy getInputPolicy() {
-    return inputPolicy;
-  }
-
   /**
    * If the stream is in Adaptive mode, switch to random IO at this
    * point. Unsynchronized.
    */
   private void maybeSwitchToRandomIO() {
-    if (inputPolicy.isAdaptive()) {
+    if (getInputPolicy().isAdaptive()) {
       setInputPolicy(S3AInputPolicy.Random);
     }
   }
@@ -355,24 +246,24 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
       closeStream("reopen(" + reason + ")", forceAbort, false);
     }
 
-    contentRangeFinish = calculateRequestLimit(inputPolicy, targetPos,
-        length, contentLength, readahead);
+    contentRangeFinish = calculateRequestLimit(getInputPolicy(), targetPos,
+        length, getContentLength(), readahead);
     LOG.debug("reopen({}) for {} range[{}-{}], length={}," +
         " streamPosition={}, nextReadPosition={}, policy={}",
-        uri, reason, targetPos, contentRangeFinish, length,  pos, nextReadPos,
-        inputPolicy);
+        getUri(), reason, targetPos, contentRangeFinish, length,  pos, nextReadPos,
+        getInputPolicy());
 
-    GetObjectRequest request = client.newGetRequestBuilder(key)
+    GetObjectRequest request = getCallbacks().newGetRequestBuilder(getKey())
         .range(S3AUtils.formatRange(targetPos, contentRangeFinish - 1))
         .applyMutation(changeTracker::maybeApplyConstraint)
         .build();
-    long opencount = streamStatistics.streamOpened();
+    long opencount = getS3AStreamStatistics().streamOpened();
     String operation = opencount == 0 ? OPERATION_OPEN : OPERATION_REOPEN;
     String text = String.format("%s %s at %d",
-        operation, uri, targetPos);
-    wrappedStream = onceTrackingDuration(text, uri,
-        streamStatistics.initiateGetRequest(), () ->
-            client.getObject(request));
+        operation, getUri(), targetPos);
+    wrappedStream = onceTrackingDuration(text, getUri(),
+        getS3AStreamStatistics().initiateGetRequest(), () ->
+            getCallbacks().getObject(request));
 
     changeTracker.processResponse(wrappedStream.response(), operation,
         targetPos);
@@ -396,7 +287,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
           + " " + targetPos);
     }
 
-    if (this.contentLength <= 0) {
+    if (this.getContentLength() <= 0) {
       return;
     }
 
@@ -414,7 +305,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
       seek(positiveTargetPos);
     } catch (IOException ioe) {
       LOG.debug("Ignoring IOE on seek of {} to {}",
-          uri, positiveTargetPos, ioe);
+          getUri(), positiveTargetPos, ioe);
     }
   }
 
@@ -449,12 +340,12 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
           && diff < forwardSeekLimit;
       if (skipForward) {
         // the forward seek range is within the limits
-        LOG.debug("Forward seek on {}, of {} bytes", uri, diff);
+        LOG.debug("Forward seek on {}, of {} bytes", getUri(), diff);
         long skipped = wrappedStream.skip(diff);
         if (skipped > 0) {
           pos += skipped;
         }
-        streamStatistics.seekForwards(diff, skipped);
+        getS3AStreamStatistics().seekForwards(diff, skipped);
 
         if (pos == targetPos) {
           // all is well
@@ -464,15 +355,15 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
         } else {
           // log a warning; continue to attempt to re-open
           LOG.warn("Failed to seek on {} to {}. Current position {}",
-              uri, targetPos,  pos);
+              getUri(), targetPos,  pos);
         }
       } else {
         // not attempting to read any bytes from the stream
-        streamStatistics.seekForwards(diff, 0);
+        getS3AStreamStatistics().seekForwards(diff, 0);
       }
     } else if (diff < 0) {
       // backwards seek
-      streamStatistics.seekBackwards(diff);
+      getS3AStreamStatistics().seekBackwards(diff);
       // if the stream is in "Normal" mode, switch to random IO at this
       // point, as it is indicative of columnar format IO
       maybeSwitchToRandomIO();
@@ -513,8 +404,8 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
   @Retries.RetryTranslated
   private void lazySeek(long targetPos, long len) throws IOException {
 
-    Invoker invoker = context.getReadInvoker();
-    invoker.retry("lazySeek to " + targetPos, pathStr, true,
+    Invoker invoker = getContext().getReadInvoker();
+    invoker.retry("lazySeek to " + targetPos, getPathStr(), true,
         () -> {
           //For lazy seek
           seekInStream(targetPos, len);
@@ -532,9 +423,9 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
    * @param bytesRead number of bytes read
    */
   private void incrementBytesRead(long bytesRead) {
-    streamStatistics.bytesRead(bytesRead);
-    if (context.stats != null && bytesRead > 0) {
-      context.stats.incrementBytesRead(bytesRead);
+    getS3AStreamStatistics().bytesRead(bytesRead);
+    if (getContext().stats != null && bytesRead > 0) {
+      getContext().stats.incrementBytesRead(bytesRead);
     }
   }
 
@@ -542,7 +433,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
   @Retries.RetryTranslated
   public synchronized int read() throws IOException {
     checkNotClosed();
-    if (this.contentLength == 0 || (nextReadPos >= contentLength)) {
+    if (this.getContentLength() == 0 || (nextReadPos >= getContentLength())) {
       return -1;
     }
 
@@ -554,8 +445,8 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
       return -1;
     }
 
-    Invoker invoker = context.getReadInvoker();
-    int byteRead = invoker.retry("read", pathStr, true,
+    Invoker invoker = getContext().getReadInvoker();
+    int byteRead = invoker.retry("read", getPathStr(), true,
         () -> {
           int b;
           // When exception happens before re-setting wrappedStream in "reopen" called
@@ -597,13 +488,13 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
     if (LOG.isDebugEnabled()) {
       LOG.debug("Got exception while trying to read from stream {}, " +
           "client: {} object: {}, trying to recover: ",
-          uri, client, objectResponse, ioe);
+          getUri(), getCallbacks(), objectResponse, ioe);
     } else {
       LOG.info("Got exception while trying to read from stream {}, " +
           "client: {} object: {}, trying to recover: " + ioe,
-          uri, client, objectResponse);
+          getUri(), getCallbacks(), objectResponse);
     }
-    streamStatistics.readException();
+    getS3AStreamStatistics().readException();
     closeStream("failure recovery", forceAbort, false);
   }
 
@@ -638,7 +529,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
       return 0;
     }
 
-    if (this.contentLength == 0 || (nextReadPos >= contentLength)) {
+    if (this.getContentLength() == 0 || (nextReadPos >= getContentLength())) {
       return -1;
     }
 
@@ -649,10 +540,10 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
       return -1;
     }
 
-    Invoker invoker = context.getReadInvoker();
+    Invoker invoker = getContext().getReadInvoker();
 
-    streamStatistics.readOperationStarted(nextReadPos, len);
-    int bytesRead = invoker.retry("read", pathStr, true,
+    getS3AStreamStatistics().readOperationStarted(nextReadPos, len);
+    int bytesRead = invoker.retry("read", getPathStr(), true,
         () -> {
           int bytes;
           // When exception happens before re-setting wrappedStream in "reopen" called
@@ -685,7 +576,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
     } else {
       streamReadResultNegative();
     }
-    streamStatistics.readOperationCompleted(len, bytesRead);
+    getS3AStreamStatistics().readOperationCompleted(len, bytesRead);
     return bytesRead;
   }
 
@@ -696,7 +587,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
    */
   private void checkNotClosed() throws IOException {
     if (closed) {
-      throw new IOException(uri + ": " + FSExceptionMessages.STREAM_IS_CLOSED);
+      throw new IOException(getUri() + ": " + FSExceptionMessages.STREAM_IS_CLOSED);
     }
   }
 
@@ -717,28 +608,14 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
         // close or abort the stream; blocking
         closeStream("close() operation", false, true);
         // end the client+audit span.
-        client.close();
-        // this is actually a no-op
-        super.close();
+        getCallbacks().close();
+
       } finally {
-        // merge the statistics back into the FS statistics.
-        streamStatistics.close();
-        // Collect ThreadLevel IOStats
-        mergeThreadIOStatistics(streamStatistics.getIOStatistics());
+        super.close();
       }
     }
   }
 
-  /**
-   * Merging the current thread's IOStatistics with the current IOStatistics
-   * context.
-   *
-   * @param streamIOStats Stream statistics to be merged into thread
-   *                      statistics aggregator.
-   */
-  private void mergeThreadIOStatistics(IOStatistics streamIOStats) {
-    threadIOStatistics.aggregate(streamIOStats);
-  }
 
   /**
    * Close a stream: decide whether to abort or close, based on
@@ -776,11 +653,11 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
     boolean shouldAbort = forceAbort || remaining > readahead;
     CompletableFuture<Boolean> operation;
     SDKStreamDrainer<ResponseInputStream<GetObjectResponse>> drainer = new SDKStreamDrainer<>(
-        uri,
+        getUri(),
         wrappedStream,
         shouldAbort,
         (int) remaining,
-        streamStatistics,
+        getS3AStreamStatistics(),
         reason);
 
     if (blocking || shouldAbort || remaining <= asyncDrainThreshold) {
@@ -792,7 +669,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
     } else {
       LOG.debug("initiating asynchronous drain of {} bytes", remaining);
       // schedule an async drain/abort
-      operation = client.submit(drainer);
+      operation = getCallbacks().submit(drainer);
     }
 
     // either the stream is closed in the blocking call or the async call is
@@ -817,7 +694,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
   @InterfaceStability.Unstable
   public synchronized boolean resetConnection() throws IOException {
     checkNotClosed();
-    LOG.info("Forcing reset of connection to {}", uri);
+    LOG.info("Forcing reset of connection to {}", getUri());
     return awaitFuture(closeStream("reset()", true, true));
   }
 
@@ -839,7 +716,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
   @InterfaceAudience.Private
   @InterfaceStability.Unstable
   public synchronized long remainingInFile() {
-    return this.contentLength - this.pos;
+    return this.getContentLength() - this.pos;
   }
 
   /**
@@ -879,23 +756,24 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
   @Override
   @InterfaceStability.Unstable
   public String toString() {
-    String s = streamStatistics.toString();
+    String s = getS3AStreamStatistics().toString();
     synchronized (this) {
       final StringBuilder sb = new StringBuilder(
           "S3AInputStream{");
-      sb.append(uri);
+      sb.append(super.toString()).append(" ");
+      sb.append(getUri());
       sb.append(" wrappedStream=")
           .append(isObjectStreamOpen() ? "open" : "closed");
-      sb.append(" read policy=").append(inputPolicy);
+      sb.append(" read policy=").append(getInputPolicy());
       sb.append(" pos=").append(pos);
       sb.append(" nextReadPos=").append(nextReadPos);
-      sb.append(" contentLength=").append(contentLength);
+      sb.append(" contentLength=").append(getContentLength());
       sb.append(" contentRangeStart=").append(contentRangeStart);
       sb.append(" contentRangeFinish=").append(contentRangeFinish);
       sb.append(" remainingInCurrentRequest=")
           .append(remainingInCurrentRequest());
       sb.append(" ").append(changeTracker);
-      sb.append(" ").append(vectoredIOContext);
+      sb.append(" ").append(getVectoredIOContext());
       sb.append('\n').append(s);
       sb.append('}');
       return sb.toString();
@@ -920,7 +798,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
       throws IOException {
     checkNotClosed();
     validatePositionedReadArgs(position, buffer, offset, length);
-    streamStatistics.readFullyOperationStarted(position, length);
+    getS3AStreamStatistics().readFullyOperationStarted(position, length);
     if (length == 0) {
       return;
     }
@@ -945,22 +823,6 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
     }
   }
 
-  /**
-   * {@inheritDoc}.
-   */
-  @Override
-  public int minSeekForVectorReads() {
-    return vectoredIOContext.getMinSeekForVectorReads();
-  }
-
-  /**
-   * {@inheritDoc}.
-   */
-  @Override
-  public int maxReadSizeForVectorReads() {
-    return vectoredIOContext.getMaxReadSizeForVectorReads();
-  }
-
   /**
    * {@inheritDoc}
    * Vectored read implementation for S3AInputStream.
@@ -971,10 +833,10 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
   @Override
   public synchronized void readVectored(List<? extends FileRange> ranges,
                            IntFunction<ByteBuffer> allocate) throws IOException {
-    LOG.debug("Starting vectored read on path {} for ranges {} ", pathStr, ranges);
+    LOG.debug("Starting vectored read on path {} for ranges {} ", getPathStr(), ranges);
     checkNotClosed();
     if (stopVectoredIOOperations.getAndSet(false)) {
-      LOG.debug("Reinstating vectored read operation for path {} ", pathStr);
+      LOG.debug("Reinstating vectored read operation for path {} ", getPathStr());
     }
 
     // prepare to read
@@ -992,26 +854,28 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
 
     if (isOrderedDisjoint(sortedRanges, 1, minSeekForVectorReads())) {
       LOG.debug("Not merging the ranges as they are disjoint");
-      streamStatistics.readVectoredOperationStarted(sortedRanges.size(), sortedRanges.size());
+      getS3AStreamStatistics().readVectoredOperationStarted(sortedRanges.size(),
+          sortedRanges.size());
       for (FileRange range: sortedRanges) {
         ByteBuffer buffer = allocate.apply(range.getLength());
-        boundedThreadPool.submit(() -> readSingleRange(range, buffer));
+        getBoundedThreadPool().submit(() -> readSingleRange(range, buffer));
       }
     } else {
       LOG.debug("Trying to merge the ranges as they are not disjoint");
       List<CombinedFileRange> combinedFileRanges = mergeSortedRanges(sortedRanges,
               1, minSeekForVectorReads(),
               maxReadSizeForVectorReads());
-      streamStatistics.readVectoredOperationStarted(sortedRanges.size(), combinedFileRanges.size());
+      getS3AStreamStatistics().readVectoredOperationStarted(sortedRanges.size(),
+          combinedFileRanges.size());
       LOG.debug("Number of original ranges size {} , Number of combined ranges {} ",
               ranges.size(), combinedFileRanges.size());
       for (CombinedFileRange combinedFileRange: combinedFileRanges) {
-        boundedThreadPool.submit(
+        getBoundedThreadPool().submit(
             () -> readCombinedRangeAndUpdateChildren(combinedFileRange, allocate));
       }
     }
     LOG.debug("Finished submitting vectored read to threadpool" +
-            " on path {} for ranges {} ", pathStr, ranges);
+            " on path {} for ranges {} ", getPathStr(), ranges);
   }
 
   /**
@@ -1022,7 +886,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
    */
   private void readCombinedRangeAndUpdateChildren(CombinedFileRange combinedFileRange,
                                                   IntFunction<ByteBuffer> allocate) {
-    LOG.debug("Start reading {} from path {} ", combinedFileRange, pathStr);
+    LOG.debug("Start reading {} from path {} ", combinedFileRange, getPathStr());
     ResponseInputStream<GetObjectResponse> rangeContent = null;
     try {
       rangeContent = getS3ObjectInputStream("readCombinedFileRange",
@@ -1030,7 +894,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
               combinedFileRange.getLength());
       populateChildBuffers(combinedFileRange, rangeContent, allocate);
     } catch (Exception ex) {
-      LOG.debug("Exception while reading {} from path {} ", combinedFileRange, pathStr, ex);
+      LOG.debug("Exception while reading {} from path {} ", combinedFileRange, getPathStr(), ex);
       // complete exception all the underlying ranges which have not already
       // finished.
       for(FileRange child : combinedFileRange.getUnderlying()) {
@@ -1041,7 +905,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
     } finally {
       IOUtils.cleanupWithLogger(LOG, rangeContent);
     }
-    LOG.debug("Finished reading {} from path {} ", combinedFileRange, pathStr);
+    LOG.debug("Finished reading {} from path {} ", combinedFileRange, getPathStr());
   }
 
   /**
@@ -1129,7 +993,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
         remaining -= readCount;
       }
     } finally {
-      streamStatistics.readVectoredBytesDiscarded(drainBytes);
+      getS3AStreamStatistics().readVectoredBytesDiscarded(drainBytes);
       LOG.debug("{} bytes drained from stream ", drainBytes);
     }
   }
@@ -1140,7 +1004,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
    * @param buffer buffer to fill.
    */
   private void readSingleRange(FileRange range, ByteBuffer buffer) {
-    LOG.debug("Start reading {} from {} ", range, pathStr);
+    LOG.debug("Start reading {} from {} ", range, getPathStr());
     if (range.getLength() == 0) {
       // a zero byte read.
       buffer.flip();
@@ -1155,12 +1019,12 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
       populateBuffer(range, buffer, objectRange);
       range.getData().complete(buffer);
     } catch (Exception ex) {
-      LOG.warn("Exception while reading a range {} from path {} ", range, pathStr, ex);
+      LOG.warn("Exception while reading a range {} from path {} ", range, getPathStr(), ex);
       range.getData().completeExceptionally(ex);
     } finally {
       IOUtils.cleanupWithLogger(LOG, objectRange);
     }
-    LOG.debug("Finished reading range {} from path {} ", range, pathStr);
+    LOG.debug("Finished reading range {} from path {} ", range, getPathStr());
   }
 
   /**
@@ -1274,18 +1138,18 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
                                                              long position,
                                                              int length)
       throws IOException {
-    final GetObjectRequest request = client.newGetRequestBuilder(key)
+    final GetObjectRequest request = getCallbacks().newGetRequestBuilder(getKey())
         .range(S3AUtils.formatRange(position, position + length - 1))
         .applyMutation(changeTracker::maybeApplyConstraint)
         .build();
-    DurationTracker tracker = streamStatistics.initiateGetRequest();
+    DurationTracker tracker = getS3AStreamStatistics().initiateGetRequest();
     ResponseInputStream<GetObjectResponse> objectRange;
-    Invoker invoker = context.getReadInvoker();
+    Invoker invoker = getContext().getReadInvoker();
     try {
-      objectRange = invoker.retry(operationName, pathStr, true,
+      objectRange = invoker.retry(operationName, getPathStr(), true,
         () -> {
           checkIfVectoredIOStopped();
-          return client.getObject(request);
+          return getCallbacks().getObject(request);
         });
 
     } catch (IOException ex) {
@@ -1312,18 +1176,6 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
     }
   }
 
-  /**
-   * Access the input stream statistics.
-   * This is for internal testing and may be removed without warning.
-   * @return the statistics for this input stream
-   */
-  @InterfaceAudience.Private
-  @InterfaceStability.Unstable
-  @VisibleForTesting
-  public S3AInputStreamStatistics getS3AStreamStatistics() {
-    return streamStatistics;
-  }
-
   @Override
   public synchronized void setReadahead(Long readahead) {
     this.readahead = validateReadahead(readahead);
@@ -1409,8 +1261,8 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
       stopVectoredIOOperations.set(true);
       closeStream("unbuffer()", false, false);
     } finally {
-      streamStatistics.unbuffered();
-      if (inputPolicy.isAdaptive()) {
+      getS3AStreamStatistics().unbuffered();
+      if (getInputPolicy().isAdaptive()) {
         S3AInputPolicy policy = S3AInputPolicy.Random;
         setInputPolicy(policy);
       }
@@ -1420,15 +1272,12 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
   @Override
   public boolean hasCapability(String capability) {
     switch (toLowerCase(capability)) {
-    case StreamCapabilities.IOSTATISTICS:
     case StreamCapabilities.IOSTATISTICS_CONTEXT:
-    case StreamStatisticNames.STREAM_LEAKS:
     case StreamCapabilities.READAHEAD:
     case StreamCapabilities.UNBUFFER:
-    case StreamCapabilities.VECTOREDIO:
       return true;
     default:
-      return false;
+      return super.hasCapability(capability);
     }
   }
 
@@ -1441,11 +1290,6 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
     return wrappedStream != null;
   }
 
-  @Override
-  public IOStatistics getIOStatistics() {
-    return ioStatistics;
-  }
-
   /**
    * Get the wrapped stream.
    * This is for testing only.
@@ -1457,38 +1301,4 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
     return wrappedStream;
   }
 
-  /**
-   * Callbacks for input stream IO.
-   */
-  public interface InputStreamCallbacks extends Closeable {
-
-    /**
-     * Create a GET request builder.
-     * @param key object key
-     * @return the request builder
-     */
-    GetObjectRequest.Builder newGetRequestBuilder(String key);
-
-    /**
-     * Execute the request.
-     * When CSE is enabled with reading of unencrypted data, The object is checked if it is
-     * encrypted and if so, the request is made with encrypted S3 client. If the object is
-     * not encrypted, the request is made with unencrypted s3 client.
-     * @param request the request
-     * @return the response
-     * @throws IOException on any failure.
-     */
-    @Retries.OnceRaw
-    ResponseInputStream<GetObjectResponse> getObject(GetObjectRequest request) throws IOException;
-
-    /**
-     * Submit some asynchronous work, for example, draining a stream.
-     * @param operation operation to invoke
-     * @param <T> return type
-     * @return a future.
-     */
-    <T> CompletableFuture<T> submit(CallableRaisingIOE<T> operation);
-
-  }
-
 }

+ 7 - 37
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java

@@ -34,7 +34,7 @@ import org.apache.hadoop.util.Preconditions;
 import static java.util.Objects.requireNonNull;
 
 /**
- * Read-specific operation context struct.
+ * Read-specific operation context structure.
  */
 public class S3AReadOpContext extends S3AOpContext {
 
@@ -75,15 +75,11 @@ public class S3AReadOpContext extends S3AOpContext {
   /** Thread-level IOStatistics aggregator. **/
   private final IOStatisticsAggregator ioStatisticsAggregator;
 
-  // S3 reads are prefetched asynchronously using this future pool.
+  /**
+   * Pool for any future IO.
+   */
   private ExecutorServiceFuturePool futurePool;
 
-  // Size in bytes of a single prefetch block.
-  private final int prefetchBlockSize;
-
-  // Size of prefetch queue (in number of blocks).
-  private final int prefetchBlockCount;
-
   /**
    * Instantiate.
    * @param path path of read
@@ -93,9 +89,7 @@ public class S3AReadOpContext extends S3AOpContext {
    * @param dstFileStatus target file status
    * @param vectoredIOContext context for vectored read operation.
    * @param ioStatisticsAggregator IOStatistics aggregator for each thread.
-   * @param futurePool the ExecutorServiceFuturePool instance used by async prefetches.
-   * @param prefetchBlockSize the size (in number of bytes) of each prefetched block.
-   * @param prefetchBlockCount maximum number of prefetched blocks.
+   * @param futurePool Pool for any future IO
    */
   public S3AReadOpContext(
       final Path path,
@@ -105,9 +99,7 @@ public class S3AReadOpContext extends S3AOpContext {
       FileStatus dstFileStatus,
       VectoredIOContext vectoredIOContext,
       IOStatisticsAggregator ioStatisticsAggregator,
-      ExecutorServiceFuturePool futurePool,
-      int prefetchBlockSize,
-      int prefetchBlockCount) {
+      ExecutorServiceFuturePool futurePool) {
 
     super(invoker, stats, instrumentation,
         dstFileStatus);
@@ -115,12 +107,7 @@ public class S3AReadOpContext extends S3AOpContext {
     this.vectoredIOContext = requireNonNull(vectoredIOContext, "vectoredIOContext");
     this.ioStatisticsAggregator = ioStatisticsAggregator;
     this.futurePool = futurePool;
-    Preconditions.checkArgument(
-        prefetchBlockSize > 0, "invalid prefetchBlockSize %d", prefetchBlockSize);
-    this.prefetchBlockSize = prefetchBlockSize;
-    Preconditions.checkArgument(
-        prefetchBlockCount > 0, "invalid prefetchBlockCount %d", prefetchBlockCount);
-    this.prefetchBlockCount = prefetchBlockCount;
+
   }
 
   /**
@@ -265,23 +252,6 @@ public class S3AReadOpContext extends S3AOpContext {
     return this.futurePool;
   }
 
-  /**
-   * Gets the size in bytes of a single prefetch block.
-   *
-   * @return the size in bytes of a single prefetch block.
-   */
-  public int getPrefetchBlockSize() {
-    return this.prefetchBlockSize;
-  }
-
-  /**
-   * Gets the size of prefetch queue (in number of blocks).
-   *
-   * @return the size of prefetch queue (in number of blocks).
-   */
-  public int getPrefetchBlockCount() {
-    return this.prefetchBlockCount;
-  }
 
   @Override
   public String toString() {

+ 65 - 1
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java

@@ -45,15 +45,20 @@ import software.amazon.awssdk.transfer.s3.model.CompletedFileUpload;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.PathCapabilities;
 import org.apache.hadoop.fs.s3a.api.RequestFactory;
 import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
 import org.apache.hadoop.fs.s3a.impl.ClientManager;
 import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteException;
 import org.apache.hadoop.fs.s3a.impl.S3AFileSystemOperations;
 import org.apache.hadoop.fs.s3a.impl.StoreContext;
+import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamFactory;
 import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
 import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
 import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import org.apache.hadoop.service.Service;
 
 /**
  * Interface for the S3A Store;
@@ -63,10 +68,19 @@ import org.apache.hadoop.fs.statistics.IOStatisticsSource;
  * The {@link ClientManager} interface is used to create the AWS clients;
  * the base implementation forwards to the implementation of this interface
  * passed in at construction time.
+ * <p>
+ * The interface extends the Hadoop {@link Service} interface
+ * and follows its lifecycle: it MUST NOT be used until
+ * {@link Service#init(Configuration)} has been invoked.
  */
 @InterfaceAudience.LimitedPrivate("Extensions")
 @InterfaceStability.Unstable
-public interface S3AStore extends IOStatisticsSource, ClientManager {
+public interface S3AStore extends
+    ClientManager,
+    IOStatisticsSource,
+    ObjectInputStreamFactory,
+    PathCapabilities,
+    Service {
 
   /**
    * Acquire write capacity for operations.
@@ -302,4 +316,54 @@ public interface S3AStore extends IOStatisticsSource, ClientManager {
   @Retries.OnceRaw
   CompleteMultipartUploadResponse completeMultipartUpload(
       CompleteMultipartUploadRequest request);
+
+  /**
+   * Get the directory allocator.
+   * @return the directory allocator
+   */
+  LocalDirAllocator getDirectoryAllocator();
+
+  /**
+   * Demand create the directory allocator, then create a temporary file.
+   * This does not mark the file for deletion when a process exits.
+   * Pass in a file size of {@link LocalDirAllocator#SIZE_UNKNOWN} if the
+   * size is unknown.
+   * {@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
+   */
+  File createTemporaryFileForWriting(String pathStr,
+      long size,
+      Configuration conf) throws IOException;
+
+
+  /*
+   =============== BEGIN ObjectInputStreamFactory ===============
+   */
+
+  /**
+   * Return the capabilities of input streams created
+   * through the store.
+   * @param capability string to query the stream support for.
+   * @return capabilities declared supported in streams.
+   */
+  boolean inputStreamHasCapability(String capability);
+
+  /**
+   * The StreamCapabilities is part of ObjectInputStreamFactory.
+   * To avoid confusion with any other streams which may
+   * be added here: always return false.
+   * @param capability string to query the stream support for.
+   * @return false, always.
+   */
+  default boolean hasCapability(String capability) {
+    return false;
+  }
+
+  /*
+   =============== END ObjectInputStreamFactory ===============
+   */
 }

+ 89 - 15
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/VectoredIOContext.java

@@ -18,14 +18,12 @@
 
 package org.apache.hadoop.fs.s3a;
 
-import java.util.List;
-import java.util.function.IntFunction;
+import static org.apache.hadoop.util.Preconditions.checkState;
 
 /**
- * Context related to vectored IO operation.
- * See {@link S3AInputStream#readVectored(List, IntFunction)}.
+ * Configuration information for vectored IO.
  */
-public class VectoredIOContext {
+public final class VectoredIOContext {
 
   /**
    * What is the smallest reasonable seek that we should group
@@ -36,43 +34,119 @@ public class VectoredIOContext {
   /**
    * What is the largest size that we should group ranges
    * together during vectored read operation.
-   * Setting this value 0 will disable merging of ranges.
+   * Setting this value to 0 will disable merging of ranges.
    */
   private int maxReadSizeForVectorReads;
 
+  /**
+   * Maximum number of active range read operation a single
+   * input stream can have.
+   */
+  private int vectoredActiveRangeReads;
+
+  /**
+   * Can this instance be updated?
+   */
+  private boolean immutable = false;
+
   /**
    * Default no arg constructor.
    */
   public VectoredIOContext() {
   }
 
-  public VectoredIOContext setMinSeekForVectoredReads(int minSeek) {
-    this.minSeekForVectorReads = minSeek;
+  /**
+   * Make immutable.
+   * @return this instance.
+   */
+  public VectoredIOContext build() {
+    immutable = true;
     return this;
   }
 
-  public VectoredIOContext setMaxReadSizeForVectoredReads(int maxSize) {
-    this.maxReadSizeForVectorReads = maxSize;
-    return this;
+  /**
+   * Verify this object is still mutable.
+   * @throws IllegalStateException if not.
+   */
+  private void checkMutable() {
+    checkState(!immutable, "Instance is immutable");
   }
 
-  public VectoredIOContext build() {
+  /**
+   * What is the threshold at which a seek() to a new location
+   * is initiated, rather than merging ranges?
+   * Set to zero to disable range merging entirely.
+   * @param minSeek minimum amount of data to skip.
+   * @return this instance.
+   */
+  public VectoredIOContext setMinSeekForVectoredReads(int minSeek) {
+    checkMutable();
+    checkState(minSeek >= 0);
+    this.minSeekForVectorReads = minSeek;
     return this;
   }
 
+  /**
+   * What is the threshold at which a seek() to a new location
+   * is initiated, rather than merging ranges?
+   * @return a number greater than or equal to zero.
+   */
   public int getMinSeekForVectorReads() {
     return minSeekForVectorReads;
   }
 
+  /**
+   * What is the largest size that we should group ranges
+   * together during vectored read operation?
+   * @param maxSize maximum size
+   * @return this instance.
+   */
+  public VectoredIOContext setMaxReadSizeForVectoredReads(int maxSize) {
+    checkMutable();
+    checkState(maxSize >= 0);
+    this.maxReadSizeForVectorReads = maxSize;
+    return this;
+  }
+
+  /**
+   * The largest size that we should group ranges.
+   * together during vectored read operation
+   * @return a number greater than or equal to zero.
+   */
   public int getMaxReadSizeForVectorReads() {
     return maxReadSizeForVectorReads;
   }
 
+  /**
+   * Maximum number of active range read operation a single
+   * input stream can have.
+   * @return number of extra threads for reading, or zero.
+   */
+  public int getVectoredActiveRangeReads() {
+    return vectoredActiveRangeReads;
+  }
+
+  /**
+   * Maximum number of active range read operation a single
+   * input stream can have.
+   * @param activeReads number of extra threads for reading, or zero.
+   * @return this instance.
+   * number of extra threads for reading, or zero.
+   */
+  public VectoredIOContext setVectoredActiveRangeReads(
+      final int activeReads) {
+    checkMutable();
+    checkState(activeReads >= 0);
+    this.vectoredActiveRangeReads = activeReads;
+    return this;
+  }
+
   @Override
   public String toString() {
     return "VectoredIOContext{" +
-            "minSeekForVectorReads=" + minSeekForVectorReads +
-            ", maxReadSizeForVectorReads=" + maxReadSizeForVectorReads +
-            '}';
+        "minSeekForVectorReads=" + minSeekForVectorReads +
+        ", maxReadSizeForVectorReads=" + maxReadSizeForVectorReads +
+        ", vectoredActiveRangeReads=" + vectoredActiveRangeReads +
+        '}';
   }
 }

+ 11 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditIntegration.java

@@ -38,6 +38,7 @@ import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
 import static java.util.Objects.requireNonNull;
 import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_ENABLED;
 import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_ENABLED_DEFAULT;
+import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.REJECT_OUT_OF_SPAN_OPERATIONS;
 import static org.apache.hadoop.fs.s3a.audit.impl.S3AInternalAuditConstants.AUDIT_SPAN_EXECUTION_ATTRIBUTE;
 
 /**
@@ -186,4 +187,14 @@ public final class AuditIntegration {
         || exception.getCause() instanceof AuditFailureException;
   }
 
+  /**
+   * Check if the configuration is set to reject operations that are
+   * performed outside of an audit span.
+   *
+   * @param conf the configuration to check
+   * @return true if operations outside of an audit span should be rejected, false otherwise
+   */
+  public static boolean isRejectOutOfSpan(final Configuration conf) {
+    return conf.getBoolean(REJECT_OUT_OF_SPAN_OPERATIONS, false);
+  }
 }

+ 7 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditManagerS3A.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.fs.s3a.audit;
 
 import java.io.IOException;
+import java.util.EnumSet;
 import java.util.List;
 
 import software.amazon.awssdk.core.interceptor.ExecutionInterceptor;
@@ -90,4 +91,10 @@ public interface AuditManagerS3A extends Service,
    */
   boolean checkAccess(Path path, S3AFileStatus status, FsAction mode)
       throws IOException;
+
+  /**
+   * Update audit flags, especially the out of span rejection option.
+   * @param flags audit flags.
+   */
+  void setAuditFlags(EnumSet<AuditorFlags> flags);
 }

+ 29 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditorFlags.java

@@ -0,0 +1,29 @@
+/*
+ * 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.audit;
+
+/**
+ * Flags which can be passed down during initialization, or after it.
+ */
+public enum AuditorFlags {
+  /**
+   * Are out of band operations allowed?
+   */
+  PermitOutOfBandOperations
+}

+ 7 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/OperationAuditor.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.fs.s3a.audit;
 
 import java.io.IOException;
+import java.util.EnumSet;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -41,6 +42,12 @@ public interface OperationAuditor extends Service,
    */
   void init(OperationAuditorOptions options);
 
+  /**
+   * Update audit flags, especially the out of span rejection option.
+   * @param flags audit flags.
+   */
+  void setAuditFlags(EnumSet<AuditorFlags> flags);
+
   /**
    * Get the unbonded span to use after deactivating an active
    * span.

+ 79 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractOperationAuditor.java

@@ -18,9 +18,17 @@
 
 package org.apache.hadoop.fs.s3a.audit.impl;
 
+import java.util.EnumSet;
 import java.util.UUID;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.audit.AuditIntegration;
+import org.apache.hadoop.fs.s3a.audit.AuditorFlags;
 import org.apache.hadoop.fs.s3a.audit.OperationAuditor;
 import org.apache.hadoop.fs.s3a.audit.OperationAuditorOptions;
 import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
@@ -38,6 +46,9 @@ import static java.util.Objects.requireNonNull;
 public abstract class AbstractOperationAuditor extends AbstractService
     implements OperationAuditor {
 
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AbstractOperationAuditor.class);
+
   /**
    * Base of IDs is a UUID.
    */
@@ -60,6 +71,11 @@ public abstract class AbstractOperationAuditor extends AbstractService
    */
   private OperationAuditorOptions options;
 
+  /**
+   * Should out of span requests be rejected?
+   */
+  private AtomicBoolean rejectOutOfSpan = new AtomicBoolean(false);
+
   /**
    * Auditor ID as a UUID.
    */
@@ -71,6 +87,11 @@ public abstract class AbstractOperationAuditor extends AbstractService
    */
   private final String auditorID = auditorUUID.toString();
 
+  /**
+   * Audit flags which can be passed down to subclasses.
+   */
+  private EnumSet<AuditorFlags> auditorFlags;
+
   /**
    * Construct.
    * @param name name
@@ -91,6 +112,15 @@ public abstract class AbstractOperationAuditor extends AbstractService
     init(opts.getConfiguration());
   }
 
+  @Override
+  protected void serviceInit(final Configuration conf) throws Exception {
+    super.serviceInit(conf);
+    setRejectOutOfSpan(AuditIntegration.isRejectOutOfSpan(conf));
+    LOG.debug("{}: Out of span operations will be {}",
+        getName(),
+        isRejectOutOfSpan() ? "rejected" : "ignored");
+  }
+
   @Override
   public String getAuditorId() {
     return auditorID;
@@ -120,4 +150,53 @@ public abstract class AbstractOperationAuditor extends AbstractService
     return String.format("%s-%08d",
         auditorID, SPAN_ID_COUNTER.incrementAndGet());
   }
+
+  /**
+   * Should out of scope ops be rejected?
+   * @return true if out of span calls should be rejected.
+   */
+  protected boolean isRejectOutOfSpan() {
+    return rejectOutOfSpan.get();
+  }
+
+  /**
+   * Enable/disable out of span rejection.
+   * @param rejectOutOfSpan new value.
+   */
+  protected void setRejectOutOfSpan(boolean rejectOutOfSpan) {
+    this.rejectOutOfSpan.set(rejectOutOfSpan);
+  }
+
+  /**
+   * Update Auditor flags.
+   * Calls {@link #auditorFlagsChanged(EnumSet)} after the update.
+   * @param flags audit flags.
+   */
+  @Override
+  public void setAuditFlags(final EnumSet<AuditorFlags> flags) {
+    auditorFlags = flags;
+    auditorFlagsChanged(flags);
+  }
+
+  /**
+   * Get the current set of auditor flags.
+   *
+   * @return the current set of auditor flags.
+   */
+  public EnumSet<AuditorFlags> getAuditorFlags() {
+    return auditorFlags;
+  }
+
+  /**
+   * Notification that the auditor flags have been updated.
+   * @param flags audit flags.
+   */
+  protected void auditorFlagsChanged(EnumSet<AuditorFlags> flags) {
+    // if out of band operations are allowed, configuration settings are overridden
+    if (flags.contains(AuditorFlags.PermitOutOfBandOperations)) {
+      LOG.debug("Out of span operations are required by the stream factory");
+      setRejectOutOfSpan(false);
+    }
+  }
+
 }

+ 7 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/ActiveAuditManagerS3A.java

@@ -22,6 +22,7 @@ import javax.annotation.Nullable;
 import java.io.IOException;
 import java.lang.reflect.Constructor;
 import java.util.ArrayList;
+import java.util.EnumSet;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -35,6 +36,7 @@ import software.amazon.awssdk.http.SdkHttpResponse;
 import software.amazon.awssdk.transfer.s3.progress.TransferListener;
 
 import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.fs.s3a.audit.AuditorFlags;
 import org.apache.hadoop.util.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -392,6 +394,11 @@ public final class ActiveAuditManagerS3A
         operation, path1, path2));
   }
 
+  @Override
+  public void setAuditFlags(final EnumSet<AuditorFlags> flags) {
+    auditor.setAuditFlags(flags);
+  }
+
   /**
    * Return a list of execution interceptors for the AWS SDK which
    * relays to this class.

+ 4 - 10
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java

@@ -65,7 +65,6 @@ import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.OUTSIDE_SPAN;
 import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.REFERRER_HEADER_ENABLED;
 import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.REFERRER_HEADER_ENABLED_DEFAULT;
 import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.REFERRER_HEADER_FILTER;
-import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.REJECT_OUT_OF_SPAN_OPERATIONS;
 import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.UNAUDITED_OPERATION;
 import static org.apache.hadoop.fs.s3a.commit.CommitUtils.extractJobID;
 import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.HEADER_REFERRER;
@@ -97,11 +96,6 @@ public class LoggingAuditor
    */
   private AuditSpanS3A warningSpan;
 
-  /**
-   * Should out of scope ops be rejected?
-   */
-  private boolean rejectOutOfSpan;
-
   /**
    * Map of attributes which will be added to all operations.
    */
@@ -171,8 +165,6 @@ public class LoggingAuditor
   @Override
   protected void serviceInit(final Configuration conf) throws Exception {
     super.serviceInit(conf);
-    rejectOutOfSpan = conf.getBoolean(
-        REJECT_OUT_OF_SPAN_OPERATIONS, false);
     // attach the job ID if there is one in the configuration used
     // to create this file.
     String jobID = extractJobID(conf);
@@ -187,6 +179,7 @@ public class LoggingAuditor
         currentContext, createSpanID(), null, null);
     isMultipartUploadEnabled = conf.getBoolean(MULTIPART_UPLOADS_ENABLED,
               DEFAULT_MULTIPART_UPLOAD_ENABLED);
+    LOG.debug("Initialized {}", this);
   }
 
   @Override
@@ -195,7 +188,7 @@ public class LoggingAuditor
         "LoggingAuditor{");
     sb.append("ID='").append(getAuditorId()).append('\'');
     sb.append(", headerEnabled=").append(headerEnabled);
-    sb.append(", rejectOutOfSpan=").append(rejectOutOfSpan);
+    sb.append(", rejectOutOfSpan=").append(isRejectOutOfSpan());
     sb.append(", isMultipartUploadEnabled=").append(isMultipartUploadEnabled);
     sb.append('}');
     return sb.toString();
@@ -264,6 +257,7 @@ public class LoggingAuditor
   HttpReferrerAuditHeader getReferrer(AuditSpanS3A span) {
     return ((LoggingAuditSpan) span).getReferrer();
   }
+
   /**
    * Span which logs at debug and sets the HTTP referrer on
    * invocations.
@@ -545,7 +539,7 @@ public class LoggingAuditor
       } else {
         final RuntimeException ex = new AuditFailureException(unaudited);
         LOG.debug(unaudited, ex);
-        if (rejectOutOfSpan) {
+        if (isRejectOutOfSpan()) {
           throw ex;
         }
       }

+ 8 - 1
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditManagerS3A.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.fs.s3a.audit.impl;
 import javax.annotation.Nullable;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.EnumSet;
 import java.util.List;
 import java.util.UUID;
 
@@ -34,6 +35,7 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.s3a.S3AFileStatus;
 import org.apache.hadoop.fs.s3a.audit.AuditManagerS3A;
 import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A;
+import org.apache.hadoop.fs.s3a.audit.AuditorFlags;
 import org.apache.hadoop.fs.s3a.audit.OperationAuditor;
 import org.apache.hadoop.fs.s3a.audit.OperationAuditorOptions;
 import org.apache.hadoop.service.CompositeService;
@@ -77,7 +79,7 @@ public class NoopAuditManagerS3A extends CompositeService
   @Override
   protected void serviceInit(final Configuration conf) throws Exception {
     super.serviceInit(conf);
-    NoopAuditor audit = new NoopAuditor(this);
+    NoopAuditor audit = new NoopAuditor("NoopAuditor", this);
     final OperationAuditorOptions options =
         OperationAuditorOptions.builder()
             .withConfiguration(conf)
@@ -169,4 +171,9 @@ public class NoopAuditManagerS3A extends CompositeService
       final String path2) {
     return NoopSpan.INSTANCE;
   }
+
+  @Override
+  public void setAuditFlags(final EnumSet<AuditorFlags> flags) {
+    /* no-op */
+  }
 }

+ 11 - 5
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditor.java

@@ -47,23 +47,29 @@ public class NoopAuditor extends AbstractOperationAuditor {
    * Constructor.
    * This will be used when the auditor is created through
    * configuration and classloading.
+   * @param name  auditor name
    */
-  public NoopAuditor() {
-    this(null);
+  public NoopAuditor(String name) {
+    this(name, null);
   }
 
   /**
    * Constructor when explicitly created within
    * the {@link NoopAuditManagerS3A}.
+   * @param name  auditor name
    * @param activationCallbacks Activation callbacks.
    */
   public NoopAuditor(
-      NoopSpan.SpanActivationCallbacks activationCallbacks) {
-    super("NoopAuditor");
+      String name, NoopSpan.SpanActivationCallbacks activationCallbacks) {
+    super(name);
     this.unbondedSpan = createSpan("unbonded", null, null);
     this.activationCallbacks = activationCallbacks;
   }
 
+  public NoopAuditor() {
+    this("NoopAuditor");
+  }
+
   @Override
   public AuditSpanS3A createSpan(
       final String operation,
@@ -86,7 +92,7 @@ public class NoopAuditor extends AbstractOperationAuditor {
    */
   public static NoopAuditor createAndStartNoopAuditor(Configuration conf,
       NoopSpan.SpanActivationCallbacks activationCallbacks) {
-    NoopAuditor noop = new NoopAuditor(activationCallbacks);
+    NoopAuditor noop = new NoopAuditor("NoopAuditor", activationCallbacks);
     final OperationAuditorOptions options =
         OperationAuditorOptions.builder()
             .withConfiguration(conf)

+ 3 - 6
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClientManager.java

@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.fs.s3a.impl;
 
-import java.io.Closeable;
 import java.io.IOException;
 import java.io.UncheckedIOException;
 
@@ -26,11 +25,13 @@ import software.amazon.awssdk.services.s3.S3AsyncClient;
 import software.amazon.awssdk.services.s3.S3Client;
 import software.amazon.awssdk.transfer.s3.S3TransferManager;
 
+import org.apache.hadoop.service.Service;
+
 /**
  * Interface for on-demand/async creation of AWS clients
  * and extension services.
  */
-public interface ClientManager extends Closeable {
+public interface ClientManager extends Service {
 
   /**
    * Get the transfer manager, creating it and any dependencies if needed.
@@ -76,8 +77,4 @@ public interface ClientManager extends Closeable {
    */
   S3Client getOrCreateAsyncS3ClientUnchecked() throws UncheckedIOException;
 
-  /**
-   * Close operation is required to not raise exceptions.
-   */
-  void close();
 }

+ 22 - 38
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClientManagerImpl.java

@@ -25,7 +25,6 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Future;
-import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -35,6 +34,7 @@ import software.amazon.awssdk.transfer.s3.S3TransferManager;
 
 import org.apache.hadoop.fs.s3a.S3ClientFactory;
 import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
+import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.util.functional.CallableRaisingIOE;
 import org.apache.hadoop.util.functional.LazyAutoCloseableReference;
 
@@ -49,11 +49,13 @@ import static org.apache.hadoop.util.functional.FutureIO.awaitAllFutures;
 
 /**
  * Client manager for on-demand creation of S3 clients,
- * with parallelized close of them in {@link #close()}.
+ * with parallelized close of them in {@link #serviceStop()}.
  * Updates {@link org.apache.hadoop.fs.s3a.Statistic#STORE_CLIENT_CREATION}
  * to track count and duration of client creation.
  */
-public class ClientManagerImpl implements ClientManager {
+public class ClientManagerImpl
+    extends AbstractService
+    implements ClientManager {
 
   public static final Logger LOG = LoggerFactory.getLogger(ClientManagerImpl.class);
 
@@ -67,11 +69,6 @@ public class ClientManagerImpl implements ClientManager {
    */
   private final S3ClientFactory unencryptedClientFactory;
 
-  /**
-   * Closed flag.
-   */
-  private final AtomicBoolean closed = new AtomicBoolean(false);
-
   /**
    * Parameters to create sync/async clients.
    */
@@ -115,12 +112,13 @@ public class ClientManagerImpl implements ClientManager {
       final S3ClientFactory unencryptedClientFactory,
       final S3ClientFactory.S3ClientCreationParameters clientCreationParameters,
       final DurationTrackerFactory durationTrackerFactory) {
+    super("ClientManager");
     this.clientFactory = requireNonNull(clientFactory);
     this.unencryptedClientFactory = unencryptedClientFactory;
     this.clientCreationParameters = requireNonNull(clientCreationParameters);
     this.durationTrackerFactory = requireNonNull(durationTrackerFactory);
     this.s3Client = new LazyAutoCloseableReference<>(createS3Client());
-    this.s3AsyncClient = new LazyAutoCloseableReference<>(createAyncClient());
+    this.s3AsyncClient = new LazyAutoCloseableReference<>(createAsyncClient());
     this.unencryptedS3Client = new LazyAutoCloseableReference<>(createUnencryptedS3Client());
     this.transferManager = new LazyAutoCloseableReference<>(createTransferManager());
 
@@ -143,7 +141,7 @@ public class ClientManagerImpl implements ClientManager {
    * Create the function to create the S3 Async client.
    * @return a callable which will create the client.
    */
-  private CallableRaisingIOE<S3AsyncClient> createAyncClient() {
+  private CallableRaisingIOE<S3AsyncClient> createAsyncClient() {
     return trackDurationOfOperation(
         durationTrackerFactory,
         STORE_CLIENT_CREATION.getSymbol(),
@@ -226,26 +224,8 @@ public class ClientManagerImpl implements ClientManager {
     return transferManager.eval();
   }
 
-  /**
-   * Check that the client manager is not closed.
-   * @throws IllegalStateException if it is closed.
-   */
-  private void checkNotClosed() {
-    checkState(!closed.get(), "Client manager is closed");
-  }
-
-  /**
-   * Close() is synchronized to avoid race conditions between
-   * slow client creation and this close operation.
-   * <p>
-   * The objects are all deleted in parallel
-   */
   @Override
-  public synchronized void close() {
-    if (closed.getAndSet(true)) {
-      // re-entrant close.
-      return;
-    }
+  protected void serviceStop() throws Exception {
     // queue the closures.
     List<Future<Object>> l = new ArrayList<>();
     l.add(closeAsync(transferManager));
@@ -253,14 +233,18 @@ public class ClientManagerImpl implements ClientManager {
     l.add(closeAsync(s3Client));
     l.add(closeAsync(unencryptedS3Client));
 
-    // once all are queued, await their completion
-    // and swallow any exception.
-    try {
-      awaitAllFutures(l);
-    } catch (Exception e) {
-      // should never happen.
-      LOG.warn("Exception in close", e);
-    }
+    // once all are queued, await their completion;
+    // exceptions will be swallowed.
+    awaitAllFutures(l);
+    super.serviceStop();
+  }
+
+  /**
+   * Check that the client manager is not closed.
+   * @throws IllegalStateException if it is closed.
+   */
+  private void checkNotClosed() {
+    checkState(!isInState(STATE.STOPPED), "Client manager is closed");
   }
 
   /**
@@ -297,7 +281,7 @@ public class ClientManagerImpl implements ClientManager {
   @Override
   public String toString() {
     return "ClientManagerImpl{" +
-        "closed=" + closed.get() +
+        "state=" + getServiceState() +
         ", s3Client=" + s3Client +
         ", s3AsyncClient=" + s3AsyncClient +
         ", unencryptedS3Client=" + unencryptedS3Client +

+ 123 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InputStreamCallbacksImpl.java

@@ -0,0 +1,123 @@
+/*
+ * 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.impl;
+
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadPoolExecutor;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.core.ResponseInputStream;
+import software.amazon.awssdk.services.s3.model.GetObjectRequest;
+import software.amazon.awssdk.services.s3.model.GetObjectResponse;
+
+import org.apache.hadoop.fs.s3a.S3AStore;
+import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamCallbacks;
+import org.apache.hadoop.fs.store.audit.AuditSpan;
+import org.apache.hadoop.util.functional.CallableRaisingIOE;
+
+import static java.util.Objects.requireNonNull;
+import static org.apache.hadoop.util.LambdaUtils.eval;
+
+/**
+ * Callbacks for object stream operations.
+ */
+public class InputStreamCallbacksImpl implements ObjectInputStreamCallbacks {
+
+  private static final Logger LOG = LoggerFactory.getLogger(InputStreamCallbacksImpl.class);
+
+  /**
+   * Audit span to activate before each call.
+   */
+  private final AuditSpan auditSpan;
+
+  /**
+   * store operations.
+   */
+  private final S3AStore store;
+
+  /**
+   * crypto FS operations.
+   */
+  private final S3AFileSystemOperations fsOperations;
+
+  /**
+   * A (restricted) thread pool for asynchronous operations.
+   */
+  private final ThreadPoolExecutor threadPool;
+
+  /**
+   * Create.
+   * @param auditSpan Audit span to activate before each call.
+   * @param store store operations
+   * @param fsOperations crypto FS operations.
+   * @param threadPool thread pool for async operations.
+   */
+  public InputStreamCallbacksImpl(
+      final AuditSpan auditSpan,
+      final S3AStore store,
+      final S3AFileSystemOperations fsOperations,
+      final ThreadPoolExecutor threadPool) {
+    this.auditSpan = requireNonNull(auditSpan);
+    this.store = requireNonNull(store);
+    this.fsOperations = requireNonNull(fsOperations);
+    this.threadPool = requireNonNull(threadPool);
+  }
+
+  /**
+   * Closes the audit span.
+   */
+  @Override
+  public void close()  {
+    auditSpan.close();
+  }
+
+  @Override
+  public GetObjectRequest.Builder newGetRequestBuilder(final String key) {
+    // active the audit span used for the operation
+    try (AuditSpan span = auditSpan.activate()) {
+      return store.getRequestFactory().newGetObjectRequestBuilder(key);
+    }
+  }
+
+  @Override
+  public ResponseInputStream<GetObjectResponse> getObject(GetObjectRequest request) throws
+      IOException {
+    // active the audit span used for the operation
+    try (AuditSpan span = auditSpan.activate()) {
+      return fsOperations.getObject(store, request, store.getRequestFactory());
+    }
+  }
+
+  @Override
+  public <T> CompletableFuture<T> submit(final CallableRaisingIOE<T> operation) {
+    CompletableFuture<T> result = new CompletableFuture<>();
+    threadPool.submit(() ->
+        eval(result, () -> {
+          LOG.debug("Starting submitted operation in {}", auditSpan.getSpanId());
+          try (AuditSpan span = auditSpan.activate()) {
+            return operation.apply();
+          } finally {
+            LOG.debug("Completed submitted operation in {}", auditSpan.getSpanId());
+          }
+        }));
+    return result;
+  }
+}

+ 203 - 10
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java

@@ -57,7 +57,11 @@ import software.amazon.awssdk.transfer.s3.model.CompletedFileUpload;
 import software.amazon.awssdk.transfer.s3.model.FileUpload;
 import software.amazon.awssdk.transfer.s3.model.UploadFileRequest;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StreamCapabilities;
 import org.apache.hadoop.fs.s3a.Invoker;
 import org.apache.hadoop.fs.s3a.ProgressableProgressListener;
 import org.apache.hadoop.fs.s3a.Retries;
@@ -69,16 +73,26 @@ import org.apache.hadoop.fs.s3a.Statistic;
 import org.apache.hadoop.fs.s3a.UploadInfo;
 import org.apache.hadoop.fs.s3a.api.RequestFactory;
 import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A;
+import org.apache.hadoop.fs.s3a.impl.streams.FactoryBindingParameters;
+import org.apache.hadoop.fs.s3a.impl.streams.InputStreamType;
+import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStream;
+import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamFactory;
+import org.apache.hadoop.fs.s3a.impl.streams.ObjectReadParameters;
+import org.apache.hadoop.fs.s3a.impl.streams.StreamFactoryRequirements;
 import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
 import org.apache.hadoop.fs.statistics.DurationTracker;
 import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
 import org.apache.hadoop.fs.statistics.IOStatistics;
 import org.apache.hadoop.fs.store.audit.AuditSpanSource;
+import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.util.DurationInfo;
+import org.apache.hadoop.util.Preconditions;
 import org.apache.hadoop.util.RateLimiting;
 import org.apache.hadoop.util.functional.Tuples;
 
 import static java.util.Objects.requireNonNull;
+import static org.apache.hadoop.fs.s3a.Constants.BUFFER_DIR;
+import static org.apache.hadoop.fs.s3a.Constants.HADOOP_TMP_DIR;
 import static org.apache.hadoop.fs.s3a.S3AUtils.extractException;
 import static org.apache.hadoop.fs.s3a.S3AUtils.getPutRequestLength;
 import static org.apache.hadoop.fs.s3a.S3AUtils.isThrottleException;
@@ -99,17 +113,21 @@ import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_THROTTLED;
 import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_THROTTLE_RATE;
 import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isObjectNotFound;
 import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DELETE_CONSIDERED_IDEMPOTENT;
+import static org.apache.hadoop.fs.s3a.impl.streams.StreamIntegration.factoryFromConfig;
 import static org.apache.hadoop.fs.statistics.StoreStatisticNames.ACTION_HTTP_GET_REQUEST;
 import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfOperation;
 import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier;
 import static org.apache.hadoop.util.Preconditions.checkArgument;
+import static org.apache.hadoop.util.StringUtils.toLowerCase;
 
 /**
  * Store Layer.
  * This is where lower level storage operations are intended
  * to move.
  */
-public class S3AStoreImpl implements S3AStore {
+public class S3AStoreImpl
+    extends CompositeService
+    implements S3AStore, ObjectInputStreamFactory {
 
   private static final Logger LOG = LoggerFactory.getLogger(S3AStoreImpl.class);
 
@@ -165,7 +183,20 @@ public class S3AStoreImpl implements S3AStore {
    */
   private final FileSystem.Statistics fsStatistics;
 
-  /** Constructor to create S3A store. */
+  /**
+   * Allocator of local FS storage.
+   */
+  private LocalDirAllocator directoryAllocator;
+
+  /**
+   * Factory for input streams.
+   */
+  private ObjectInputStreamFactory objectInputStreamFactory;
+
+  /**
+   * Constructor to create S3A store.
+   * Package private, as {@link S3AStoreBuilder} creates them.
+   * */
   S3AStoreImpl(StoreContextFactory storeContextFactory,
       ClientManager clientManager,
       DurationTrackerFactory durationTrackerFactory,
@@ -176,25 +207,90 @@ public class S3AStoreImpl implements S3AStore {
       RateLimiting writeRateLimiter,
       AuditSpanSource<AuditSpanS3A> auditSpanSource,
       @Nullable FileSystem.Statistics fsStatistics) {
-    this.storeContextFactory = requireNonNull(storeContextFactory);
+    super("S3AStore");
+    this.auditSpanSource = requireNonNull(auditSpanSource);
     this.clientManager = requireNonNull(clientManager);
     this.durationTrackerFactory = requireNonNull(durationTrackerFactory);
+    this.fsStatistics = fsStatistics;
     this.instrumentation = requireNonNull(instrumentation);
     this.statisticsContext = requireNonNull(statisticsContext);
+    this.storeContextFactory = requireNonNull(storeContextFactory);
     this.storageStatistics = requireNonNull(storageStatistics);
     this.readRateLimiter = requireNonNull(readRateLimiter);
     this.writeRateLimiter = requireNonNull(writeRateLimiter);
-    this.auditSpanSource = requireNonNull(auditSpanSource);
     this.storeContext = requireNonNull(storeContextFactory.createStoreContext());
-    this.fsStatistics = fsStatistics;
-    this.invoker = storeContext.getInvoker();
-    this.bucket = storeContext.getBucket();
-    this.requestFactory = storeContext.getRequestFactory();
+
+    this.invoker = requireNonNull(storeContext.getInvoker());
+    this.bucket = requireNonNull(storeContext.getBucket());
+    this.requestFactory = requireNonNull(storeContext.getRequestFactory());
+    addService(clientManager);
+  }
+
+  /**
+   * Create and initialize any subsidiary services, including the input stream factory.
+   * @param conf configuration
+   */
+  @Override
+  protected void serviceInit(final Configuration conf) throws Exception {
+
+    // create and register the stream factory, which will
+    // then follow the service lifecycle
+    objectInputStreamFactory = factoryFromConfig(conf);
+    addService(objectInputStreamFactory);
+
+    // init all child services, including the stream factory
+    super.serviceInit(conf);
+
+    // pass down extra information to the stream factory.
+    finishStreamFactoryInit();
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    super.serviceStart();
+    initLocalDirAllocator();
+  }
+
+  /**
+   * Return the store path capabilities.
+   * If the object stream factory is non-null, hands off the
+   * query to that factory if not handled here.
+   * @param path path to query the capability of.
+   * @param capability non-null, non-empty string to query the path for support.
+   * @return known capabilities
+   */
+  @Override
+  public boolean hasPathCapability(final Path path, final String capability) {
+    switch (toLowerCase(capability)) {
+    case StreamCapabilities.IOSTATISTICS:
+      return true;
+    default:
+      return inputStreamHasCapability(capability);
+    }
   }
 
+  /**
+   * Return the capabilities of input streams created
+   * through the store.
+   * @param capability string to query the stream support for.
+   * @return capabilities declared supported in streams.
+   */
   @Override
-  public void close() {
-    clientManager.close();
+  public boolean inputStreamHasCapability(final String capability) {
+    if (objectInputStreamFactory != null) {
+      return objectInputStreamFactory.hasCapability(capability);
+    }
+    return false;
+  }
+
+  /**
+   * Initialize dir allocator if not already initialized.
+   */
+  private void initLocalDirAllocator() {
+    String bufferDir = getConfig().get(BUFFER_DIR) != null
+        ? BUFFER_DIR
+        : HADOOP_TMP_DIR;
+    directoryAllocator = new LocalDirAllocator(bufferDir);
   }
 
   /** Acquire write capacity for rate limiting {@inheritDoc}. */
@@ -808,4 +904,101 @@ public class S3AStoreImpl implements S3AStore {
     return getS3Client().completeMultipartUpload(request);
   }
 
+  /**
+   * Get the directory allocator.
+   * @return the directory allocator
+   */
+  @Override
+  public LocalDirAllocator getDirectoryAllocator() {
+    return directoryAllocator;
+  }
+
+  /**
+   * Demand create the directory allocator, then create a temporary file.
+   * This does not mark the file for deletion when a process exits.
+   * Pass in a file size of {@link LocalDirAllocator#SIZE_UNKNOWN} if the
+   * size is unknown.
+   * {@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
+   */
+  @Override
+  public File createTemporaryFileForWriting(String pathStr,
+      long size,
+      Configuration conf) throws IOException {
+    requireNonNull(directoryAllocator, "directory allocator not initialized");
+    Path path = directoryAllocator.getLocalPathForWrite(pathStr,
+        size, conf);
+    File dir = new File(path.getParent().toUri().getPath());
+    String prefix = path.getName();
+    // create a temp file on this directory
+    return File.createTempFile(prefix, null, dir);
+  }
+
+  /*
+   =============== BEGIN ObjectInputStreamFactory ===============
+   */
+
+  /**
+   * All stream factory initialization required after {@code Service.init()},
+   * after all other services have themselves been initialized.
+   */
+  private void finishStreamFactoryInit() throws IOException {
+    // must be on be invoked during service initialization
+    Preconditions.checkState(isInState(STATE.INITED),
+        "Store is in wrong state: %s", getServiceState());
+    Preconditions.checkState(clientManager.isInState(STATE.INITED),
+        "Client Manager is in wrong state: %s", clientManager.getServiceState());
+
+    // finish initialization and pass down callbacks to self
+    objectInputStreamFactory.bind(new FactoryBindingParameters(new FactoryCallbacks()));
+  }
+
+  @Override /* ObjectInputStreamFactory */
+  public ObjectInputStream readObject(ObjectReadParameters parameters)
+      throws IOException {
+    parameters.withDirectoryAllocator(getDirectoryAllocator());
+    return objectInputStreamFactory.readObject(parameters.validate());
+  }
+
+  @Override /* ObjectInputStreamFactory */
+  public StreamFactoryRequirements factoryRequirements() {
+    return objectInputStreamFactory.factoryRequirements();
+  }
+
+  /**
+   * This operation is not implemented, as
+   * is this class which invokes it on the actual factory.
+   * @param factoryBindingParameters ignored
+   * @throws UnsupportedOperationException always
+   */
+  @Override /* ObjectInputStreamFactory */
+  public void bind(final FactoryBindingParameters factoryBindingParameters) {
+    throw new UnsupportedOperationException("Not supported");
+  }
+
+  @Override /* ObjectInputStreamFactory */
+  public InputStreamType streamType() {
+    return objectInputStreamFactory.streamType();
+  }
+
+  /**
+   * Callbacks from {@link ObjectInputStreamFactory} instances.
+   */
+  private class FactoryCallbacks implements StreamFactoryCallbacks {
+
+    @Override
+    public S3AsyncClient getOrCreateAsyncClient(final boolean requireCRT) throws IOException {
+      // Needs support of the CRT before the requireCRT can be used
+      LOG.debug("Stream factory requested async client");
+      return clientManager().getOrCreateAsyncClient();
+    }
+  }
+
+  /*
+   =============== END ObjectInputStreamFactory ===============
+   */
 }

+ 96 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/AbstractObjectInputStreamFactory.java

@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.impl.streams;
+
+import org.apache.hadoop.fs.StreamCapabilities;
+import org.apache.hadoop.fs.statistics.StreamStatisticNames;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.util.Preconditions;
+
+import static org.apache.hadoop.util.StringUtils.toLowerCase;
+
+/**
+ * Base implementation of {@link ObjectInputStreamFactory}.
+ */
+public abstract class AbstractObjectInputStreamFactory extends AbstractService
+    implements ObjectInputStreamFactory {
+
+  /**
+   * Parameters passed down in
+   * {@link #bind(FactoryBindingParameters)}.
+   */
+  private FactoryBindingParameters bindingParameters;
+
+  /**
+   * Callbacks.
+   */
+  private StreamFactoryCallbacks callbacks;
+
+  /**
+   * Constructor.
+   * @param name service name.
+   */
+  protected AbstractObjectInputStreamFactory(final String name) {
+    super(name);
+  }
+
+  /**
+   * Bind to the callbacks.
+   * <p>
+   * The base class checks service state then stores
+   * the callback interface.
+   * @param factoryBindingParameters parameters for the factory binding
+   */
+  @Override
+  public void bind(final FactoryBindingParameters factoryBindingParameters) {
+    // must be on be invoked during service initialization
+    Preconditions.checkState(isInState(STATE.INITED),
+        "Input Stream factory %s is in wrong state: %s",
+        this, getServiceState());
+    bindingParameters = factoryBindingParameters;
+    callbacks = bindingParameters.callbacks();
+  }
+
+  /**
+   * Return base capabilities of all stream factories,
+   * defining what the base ObjectInputStream class does.
+   * This also includes the probe for stream type capability.
+   * @param capability string to query the stream support for.
+   * @return true if implemented
+   */
+  @Override
+  public boolean hasCapability(final String capability) {
+    switch (toLowerCase(capability)) {
+    case StreamCapabilities.IOSTATISTICS:
+    case StreamStatisticNames.STREAM_LEAKS:
+      return true;
+    default:
+      // dynamic probe for the name of this stream
+      return streamType().capability().equals(capability);
+    }
+  }
+
+  /**
+   * Get the factory callbacks.
+   * @return callbacks.
+   */
+  protected StreamFactoryCallbacks callbacks() {
+    return callbacks;
+  }
+}

+ 73 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ClassicObjectInputStreamFactory.java

@@ -0,0 +1,73 @@
+/*
+ * 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.impl.streams;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.StreamCapabilities;
+import org.apache.hadoop.fs.s3a.S3AInputStream;
+
+import static org.apache.hadoop.fs.s3a.impl.streams.StreamIntegration.populateVectoredIOContext;
+import static org.apache.hadoop.util.StringUtils.toLowerCase;
+
+/**
+ * Factory of classic {@link S3AInputStream} instances.
+ */
+public class ClassicObjectInputStreamFactory extends AbstractObjectInputStreamFactory {
+
+  public ClassicObjectInputStreamFactory() {
+    super("ClassicObjectInputStreamFactory");
+  }
+
+  @Override
+  public ObjectInputStream readObject(final ObjectReadParameters parameters)
+      throws IOException {
+    return new S3AInputStream(parameters);
+  }
+
+  @Override
+  public boolean hasCapability(final String capability) {
+
+    switch (toLowerCase(capability)) {
+    case StreamCapabilities.IOSTATISTICS_CONTEXT:
+    case StreamCapabilities.READAHEAD:
+    case StreamCapabilities.UNBUFFER:
+    case StreamCapabilities.VECTOREDIO:
+      return true;
+    default:
+      return super.hasCapability(capability);
+    }
+  }
+
+  @Override
+  public InputStreamType streamType() {
+    return InputStreamType.Classic;
+  }
+
+  /**
+   * Get the number of background threads required for this factory.
+   * @return the count of background threads.
+   */
+  @Override
+  public StreamFactoryRequirements factoryRequirements() {
+    return new StreamFactoryRequirements(0, 0,
+        populateVectoredIOContext(getConfig()));
+  }
+
+}

+ 46 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/FactoryBindingParameters.java

@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.impl.streams;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * Parameters passed down to {@link ObjectInputStreamFactory#bind}.
+ */
+public class FactoryBindingParameters {
+
+  /**
+   * Callbacks which may be invoked by a stream factory directly.
+   */
+  private final ObjectInputStreamFactory.StreamFactoryCallbacks callbacks;
+
+  /**
+   * @param callbacks callback implementation.
+   */
+  public FactoryBindingParameters(final ObjectInputStreamFactory.StreamFactoryCallbacks callbacks) {
+    this.callbacks = requireNonNull(callbacks);
+  }
+
+  /**
+   * Callbacks which may be invoked by a stream factory directly.
+   */
+  ObjectInputStreamFactory.StreamFactoryCallbacks callbacks() {
+    return callbacks;
+  }
+}

+ 125 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/InputStreamType.java

@@ -0,0 +1,125 @@
+/*
+ * 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.impl.streams;
+
+import java.util.function.Function;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.prefetch.PrefetchingInputStreamFactory;
+
+import static org.apache.hadoop.fs.s3a.Constants.INPUT_STREAM_TYPE;
+import static org.apache.hadoop.fs.s3a.impl.streams.StreamIntegration.loadCustomFactory;
+
+/**
+ * Enum of input stream types.
+ * <p>
+ * Each enum value contains the factory function actually used to create
+ * the factory.
+ */
+public enum InputStreamType {
+
+  /**
+   * The classic input stream.
+   */
+  Classic(StreamIntegration.CLASSIC, 1, c ->
+      new ClassicObjectInputStreamFactory()),
+
+  /**
+   * The prefetching input stream.
+   */
+  Prefetch(StreamIntegration.PREFETCH, 2, c ->
+      new PrefetchingInputStreamFactory()),
+
+  /**
+   * The analytics input stream.
+   */
+  Analytics(StreamIntegration.ANALYTICS, 3, c -> {
+    throw new IllegalArgumentException("not yet supported");
+  }),
+
+  /**
+   * The a custom input stream.
+   */
+  Custom(StreamIntegration.CUSTOM, 4, c -> {
+    return loadCustomFactory(c);
+  });
+
+  /**
+   * Name.
+   */
+  private final String name;
+
+  /**
+   * Stream ID.
+   */
+  private final int streamID;
+
+  /**
+   * Factory lambda-expression.
+   */
+  private final Function<Configuration, ObjectInputStreamFactory> factory;
+
+  /**
+   * String name.
+   * @return the name
+   */
+  public String getName() {
+    return name;
+  }
+
+  /**
+   * Constructor.
+   * @param name name, used in configuration binding and capability.
+   * @param id ID
+   * @param factory factory factory function. "metafactory", really.
+   */
+  InputStreamType(final String name,
+      final int id,
+      final Function<Configuration, ObjectInputStreamFactory> factory) {
+    this.name = name;
+    this.streamID = id;
+    this.factory = factory;
+  }
+
+  /**
+   * Get the ID of this stream.
+   * Isolated from the enum ID in case it ever needs to be tuned.
+   * @return the numeric ID of the stream.
+   */
+  public int streamID() {
+    return streamID;
+  }
+
+  /**
+   * Get the capability string for this stream type.
+   * @return the name of a string to probe for.
+   */
+  public String capability() {
+    return INPUT_STREAM_TYPE + "." + getName();
+  }
+
+  /**
+   * Factory constructor.
+   * @return the factory function associated with this stream type.
+   */
+  public Function<Configuration, ObjectInputStreamFactory> factory() {
+    return factory;
+  }
+
+}

+ 421 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStream.java

@@ -0,0 +1,421 @@
+/*
+ * 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.impl.streams;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutorService;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.StreamCapabilities;
+import org.apache.hadoop.fs.impl.LeakReporter;
+import org.apache.hadoop.fs.s3a.S3AInputPolicy;
+import org.apache.hadoop.fs.s3a.S3AReadOpContext;
+import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
+import org.apache.hadoop.fs.s3a.VectoredIOContext;
+import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsAggregator;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import org.apache.hadoop.fs.statistics.StreamStatisticNames;
+
+import static java.util.Objects.requireNonNull;
+import static org.apache.commons.lang3.StringUtils.isNotEmpty;
+import static org.apache.hadoop.util.Preconditions.checkArgument;
+import static org.apache.hadoop.util.StringUtils.toLowerCase;
+
+/**
+ * A stream of data from an S3 object.
+ * <p>
+ * The base class includes common methods, stores
+ * common data and incorporates leak tracking.
+ */
+public abstract class ObjectInputStream extends FSInputStream
+    implements StreamCapabilities, IOStatisticsSource {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ObjectInputStream.class);
+
+  /**
+   * IOStatistics report.
+   */
+  private final IOStatistics ioStatistics;
+
+  /**
+   * Read-specific operation context structure.
+   */
+  private final S3AReadOpContext context;
+
+  /**
+   * Callbacks for reading input stream data from the S3 Store.
+   */
+  private final ObjectInputStreamCallbacks callbacks;
+
+  /**
+   * Thread pool used for bounded IO operations.
+   */
+  private final ExecutorService boundedThreadPool;
+
+  /**
+   * URI of path.
+   */
+  private final String uri;
+
+  /**
+   * Store bucket.
+   */
+  private final String bucket;
+
+  /**
+   * Store key.
+   */
+  private final String key;
+
+  /**
+   * Path URI as a string.
+   */
+  private final String pathStr;
+
+  /**
+   * Content length from HEAD or openFile option.
+   */
+  private final long contentLength;
+
+  /**
+   * Attributes of the remote object.
+   */
+  private final S3ObjectAttributes objectAttributes;
+
+  /**
+   * Stream statistics.
+   */
+  private final S3AInputStreamStatistics streamStatistics;
+
+  /** Aggregator used to aggregate per thread IOStatistics. */
+  private final IOStatisticsAggregator threadIOStatistics;
+
+  /**
+   * Report of leaks.
+   * with report and abort unclosed streams in finalize().
+   */
+  private final LeakReporter leakReporter;
+
+  /**
+   * Stream type.
+   */
+  private final InputStreamType streamType;
+
+  /**
+   * Requested input policy.
+   */
+  private S3AInputPolicy inputPolicy;
+
+
+  /** Vectored IO context. */
+  private final VectoredIOContext vectoredIOContext;
+
+  /**
+   * Constructor.
+   * @param streamType stream type enum.
+   * @param parameters extensible parameter list.
+   */
+  protected ObjectInputStream(
+      final InputStreamType streamType,
+      final ObjectReadParameters parameters) {
+
+    this.streamType = requireNonNull(streamType);
+    this.objectAttributes = parameters.getObjectAttributes();
+    checkArgument(isNotEmpty(objectAttributes.getBucket()),
+        "No Bucket");
+    checkArgument(isNotEmpty(objectAttributes.getKey()), "No Key");
+    long l = objectAttributes.getLen();
+    checkArgument(l >= 0, "Negative content length");
+    this.context = parameters.getContext();
+    this.contentLength = l;
+
+    this.bucket = objectAttributes.getBucket();
+    this.key = objectAttributes.getKey();
+    this.pathStr = objectAttributes.getPath().toString();
+    this.callbacks = parameters.getCallbacks();
+    this.uri = "s3a://" + bucket + "/" + key;
+    this.streamStatistics = parameters.getStreamStatistics();
+    this.ioStatistics = streamStatistics.getIOStatistics();
+    this.inputPolicy = context.getInputPolicy();
+    streamStatistics.inputPolicySet(inputPolicy.ordinal());
+    this.boundedThreadPool = parameters.getBoundedThreadPool();
+    this.threadIOStatistics = requireNonNull(context.getIOStatisticsAggregator());
+    // build the leak reporter
+    this.leakReporter = new LeakReporter(
+        "Stream not closed while reading " + uri,
+        this::isStreamOpen,
+        this::abortInFinalizer);
+    this.vectoredIOContext = getContext().getVectoredIOContext();
+  }
+
+  /**
+   * Probe for stream being open.
+   * Not synchronized; the flag is volatile.
+   * @return true if the stream is still open.
+   */
+  protected abstract boolean isStreamOpen();
+
+  /**
+   * Brute force stream close; invoked by {@link LeakReporter}.
+   * All exceptions raised are ignored.
+   */
+  protected abstract void abortInFinalizer();
+
+  /**
+   * Close the stream.
+   * This triggers publishing of the stream statistics back to the filesystem
+   * statistics.
+   * This operation is synchronized, so that only one thread can attempt to
+   * @throws IOException on any problem
+   */
+  @Override
+  public synchronized void close() throws IOException {
+    // end the client+audit span.
+    callbacks.close();
+    // merge the statistics back into the FS statistics.
+    streamStatistics.close();
+    // Collect ThreadLevel IOStats
+    mergeThreadIOStatistics(streamStatistics.getIOStatistics());
+  }
+
+  /**
+   * Merging the current thread's IOStatistics with the current IOStatistics
+   * context.
+   * @param streamIOStats Stream statistics to be merged into thread
+   * statistics aggregator.
+   */
+  protected void mergeThreadIOStatistics(IOStatistics streamIOStats) {
+    threadIOStatistics.aggregate(streamIOStats);
+  }
+
+  /**
+   * Finalizer.
+   * <p>
+   * Verify that the inner stream is closed.
+   * <p>
+   * If it is not, it means streams are being leaked in application code.
+   * Log a warning, including the stack trace of the caller,
+   * then abort the stream.
+   * <p>
+   * This does not attempt to invoke {@link #close()} as that is
+   * a more complex operation, and this method is being executed
+   * during a GC finalization phase.
+   * <p>
+   * Applications MUST close their streams; this is a defensive
+   * operation to return http connections and warn the end users
+   * that their applications are at risk of running out of connections.
+   *
+   * {@inheritDoc}
+   */
+  @Override
+  protected void finalize() throws Throwable {
+    leakReporter.close();
+    super.finalize();
+  }
+
+  /**
+   * Get the current input policy.
+   * @return input policy.
+   */
+  @VisibleForTesting
+  public S3AInputPolicy getInputPolicy() {
+    return inputPolicy;
+  }
+
+  /**
+   * Set/update the input policy of the stream.
+   * This updates the stream statistics.
+   * @param inputPolicy new input policy.
+   */
+  protected void setInputPolicy(S3AInputPolicy inputPolicy) {
+    LOG.debug("Switching to input policy {}", inputPolicy);
+    this.inputPolicy = inputPolicy;
+    streamStatistics.inputPolicySet(inputPolicy.ordinal());
+  }
+
+  /**
+   * Access the input stream statistics.
+   * This is for internal testing and may be removed without warning.
+   * @return the statistics for this input stream
+   */
+  @InterfaceAudience.Private
+  @InterfaceStability.Unstable
+  @VisibleForTesting
+  public S3AInputStreamStatistics getS3AStreamStatistics() {
+    return streamStatistics;
+  }
+
+  @Override
+  public IOStatistics getIOStatistics() {
+    return ioStatistics;
+  }
+
+  /**
+   * Declare the base capabilities implemented by this class and so by
+   * all subclasses.
+   * <p>
+   * Subclasses MUST override this if they add more capabilities,
+   * or actually remove any of these.
+   * @param capability string to query the stream support for.
+   * @return true if all implementations are known to have the specific
+   * capability.
+   */
+  @Override
+  public boolean hasCapability(String capability) {
+    switch (toLowerCase(capability)) {
+    case StreamCapabilities.IOSTATISTICS:
+    case StreamStatisticNames.STREAM_LEAKS:
+      return true;
+    default:
+      // dynamic probe for the name of this stream
+      if (streamType.capability().equals(capability)) {
+        return true;
+      }
+      return false;
+    }
+  }
+
+  /**
+   * Read-specific operation context structure.
+   * @return Read-specific operation context structure.
+   */
+  protected final S3AReadOpContext getContext() {
+    return context;
+  }
+
+  /**
+   * Callbacks for reading input stream data from the S3 Store.
+   * @return Callbacks for reading input stream data from the S3 Store.
+   */
+  protected final ObjectInputStreamCallbacks getCallbacks() {
+    return callbacks;
+  }
+
+  /**
+   * Thread pool used for bounded IO operations.
+   * @return Thread pool used for bounded IO operations.
+   */
+  protected final ExecutorService getBoundedThreadPool() {
+    return boundedThreadPool;
+  }
+
+  /**
+   * URI of path.
+   * @return URI of path.
+   */
+  protected final String getUri() {
+    return uri;
+  }
+
+  /**
+   * Store bucket.
+   * @return Store bucket.
+   */
+  protected final String getBucket() {
+    return bucket;
+  }
+
+  /**
+   * Store key.
+   * @return Store key.
+   */
+  protected final String getKey() {
+    return key;
+  }
+
+  /**
+   * Path URI as a string.
+   * @return Path URI as a string.
+   */
+  protected final String getPathStr() {
+    return pathStr;
+  }
+
+  /**
+   * Content length from HEAD or openFile option.
+   * @return Content length from HEAD or openFile option.
+   */
+  protected final long getContentLength() {
+    return contentLength;
+  }
+
+  /**
+   * Aggregator used to aggregate per thread IOStatistics.
+   * @return Aggregator used to aggregate per thread IOStatistics.
+   */
+  protected final IOStatisticsAggregator getThreadIOStatistics() {
+    return threadIOStatistics;
+  }
+
+  /**
+   * Attributes of the remote object.
+   * @return Attributes of the remote object.
+   */
+  protected final S3ObjectAttributes getObjectAttributes() {
+    return objectAttributes;
+  }
+
+  /**
+   * Get Vectored IO context.
+   * @return Vectored IO context.
+   */
+  protected VectoredIOContext getVectoredIOContext() {
+    return vectoredIOContext;
+  }
+
+  /**
+   * {@inheritDoc}.
+   */
+  @Override
+  public int minSeekForVectorReads() {
+    return vectoredIOContext.getMinSeekForVectorReads();
+  }
+
+  /**
+   * {@inheritDoc}.
+   */
+  @Override
+  public int maxReadSizeForVectorReads() {
+    return vectoredIOContext.getMaxReadSizeForVectorReads();
+  }
+
+  public InputStreamType streamType() {
+    return streamType;
+  }
+
+  @Override
+  public String toString() {
+    return "ObjectInputStream{" +
+        "streamType=" + streamType +
+        ", uri='" + uri + '\'' +
+        ", contentLength=" + contentLength +
+        ", inputPolicy=" + inputPolicy +
+        ", vectoredIOContext=" + vectoredIOContext +
+        "} " + super.toString();
+  }
+}

+ 64 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStreamCallbacks.java

@@ -0,0 +1,64 @@
+/*
+ * 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.impl.streams;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+
+import software.amazon.awssdk.core.ResponseInputStream;
+import software.amazon.awssdk.services.s3.model.GetObjectRequest;
+import software.amazon.awssdk.services.s3.model.GetObjectResponse;
+
+import org.apache.hadoop.fs.s3a.Retries;
+import org.apache.hadoop.util.functional.CallableRaisingIOE;
+
+/**
+ * Callbacks for reading object data from the S3 Store.
+ */
+public interface ObjectInputStreamCallbacks extends Closeable {
+
+  /**
+   * Create a GET request builder.
+   * @param key object key
+   * @return the request builder
+   */
+  GetObjectRequest.Builder newGetRequestBuilder(String key);
+
+  /**
+   * Execute the request.
+   * When CSE is enabled with reading of unencrypted data, The object is checked if it is
+   * encrypted and if so, the request is made with encrypted S3 client. If the object is
+   * not encrypted, the request is made with unencrypted s3 client.
+   * @param request the request
+   * @return the response
+   * @throws IOException on any failure.
+   */
+  @Retries.OnceRaw
+  ResponseInputStream<GetObjectResponse> getObject(GetObjectRequest request) throws IOException;
+
+  /**
+   * Submit some asynchronous work, for example, draining a stream.
+   * @param operation operation to invoke
+   * @param <T> return type
+   * @return a future.
+   */
+  <T> CompletableFuture<T> submit(CallableRaisingIOE<T> operation);
+
+}

+ 90 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStreamFactory.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.impl.streams;
+
+import java.io.IOException;
+
+import software.amazon.awssdk.services.s3.S3AsyncClient;
+
+import org.apache.hadoop.fs.StreamCapabilities;
+import org.apache.hadoop.service.Service;
+
+/**
+ * A Factory for {@link ObjectInputStream} streams.
+ * <p>
+ * This class is instantiated during initialization of
+ * {@code S3AStore}, it then follows the same service
+ * lifecycle.
+ * <p>
+ * Note for maintainers: do try and keep this mostly stable.
+ * If new parameters need to be added, expand the
+ * {@link ObjectReadParameters} class, rather than change the
+ * interface signature.
+ */
+public interface ObjectInputStreamFactory
+    extends Service, StreamCapabilities {
+
+  /**
+   * Set extra initialization parameters.
+   * This MUST ONLY be invoked between {@code init()}
+   * and {@code start()}.
+   * @param factoryBindingParameters parameters for the factory binding
+   * @throws IOException if IO problems.
+   */
+  void bind(FactoryBindingParameters factoryBindingParameters) throws IOException;
+
+  /**
+   * Create a new input stream.
+   * There is no requirement to actually contact the store; this is generally done
+   * lazily.
+   * @param parameters parameters.
+   * @return the input stream
+   * @throws IOException problem creating the stream.
+   */
+  ObjectInputStream readObject(ObjectReadParameters parameters)
+      throws IOException;
+
+  /**
+   * Get requirements from the factory which then tune behavior
+   * elsewhere in the system.
+   * @return the count of background threads.
+   */
+  StreamFactoryRequirements factoryRequirements();
+
+  /**
+   * Get the input stream type.
+   * @return the specific stream type this factory produces.
+   */
+  InputStreamType streamType();
+
+  /**
+   * Callbacks for stream factories.
+   */
+  interface StreamFactoryCallbacks {
+
+    /**
+     * Get the Async S3Client, raising a failure to create as an IOException.
+     * @param requireCRT is the CRT required.
+     * @return the Async S3 client
+     * @throws IOException failure to create the client.
+     */
+    S3AsyncClient getOrCreateAsyncClient(boolean requireCRT) throws IOException;
+  }
+}
+

+ 190 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectReadParameters.java

@@ -0,0 +1,190 @@
+/*
+ * 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.impl.streams;
+
+import java.util.concurrent.ExecutorService;
+
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.s3a.S3AReadOpContext;
+import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
+import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * Parameters for object input streams created through
+ * {@link ObjectInputStreamFactory}.
+ * It is designed to support extra parameters added
+ * in future.
+ * <p>Note that the {@link #validate()}
+ * operation does not freeze the parameters -instead it simply
+ * verifies that all required values are set.
+ */
+public final class ObjectReadParameters {
+
+  /**
+   *  Read operation context.
+   */
+  private S3AReadOpContext context;
+
+  /**
+   * Attributes of the object.
+   */
+  private S3ObjectAttributes objectAttributes;
+
+  /**
+   * Callbacks to the store.
+   */
+  private ObjectInputStreamCallbacks callbacks;
+
+  /**
+   * Stream statistics callback.
+   */
+  private S3AInputStreamStatistics streamStatistics;
+
+  /**
+   * Bounded thread pool for submitting asynchronous
+   * work.
+   */
+  private ExecutorService boundedThreadPool;
+
+  /**
+   * Allocator of local FS storage.
+   */
+  private LocalDirAllocator directoryAllocator;
+
+  /**
+   * @return Read operation context.
+   */
+  public S3AReadOpContext getContext() {
+    return context;
+  }
+
+  /**
+   * Set read operation context.
+   * @param value new value
+   * @return the builder
+   */
+  public ObjectReadParameters withContext(S3AReadOpContext value) {
+    context = value;
+    return this;
+  }
+
+  /**
+   * @return Attributes of the object.
+   */
+  public S3ObjectAttributes getObjectAttributes() {
+    return objectAttributes;
+  }
+
+  /**
+   * Set object attributes.
+   * @param value new value
+   * @return the builder
+   */
+  public ObjectReadParameters withObjectAttributes(S3ObjectAttributes value) {
+    objectAttributes = value;
+    return this;
+  }
+
+  /**
+   * @return callbacks to store read operations.
+   */
+  public ObjectInputStreamCallbacks getCallbacks() {
+    return callbacks;
+  }
+
+  /**
+   * Set callbacks to store read operation.
+   * @param value new value
+   * @return the builder
+   */
+  public ObjectReadParameters withCallbacks(ObjectInputStreamCallbacks value) {
+    callbacks = value;
+    return this;
+  }
+
+  /**
+   * @return Stream statistics.
+   */
+  public S3AInputStreamStatistics getStreamStatistics() {
+    return streamStatistics;
+  }
+
+  /**
+   * Set SetStream statistics callback.
+   * @param value new value
+   * @return the builder
+   */
+  public ObjectReadParameters withStreamStatistics(S3AInputStreamStatistics value) {
+    streamStatistics = value;
+    return this;
+  }
+
+  /**
+   * @return Bounded thread pool for submitting asynchronous work.
+   */
+  public ExecutorService getBoundedThreadPool() {
+    return boundedThreadPool;
+  }
+
+  /**
+   * Set bounded thread pool.
+   * @param value new value
+   * @return the builder
+   */
+  public ObjectReadParameters withBoundedThreadPool(ExecutorService value) {
+    boundedThreadPool = value;
+    return this;
+  }
+
+  /**
+   * Getter.
+   * @return Allocator of local FS storage.
+   */
+  public LocalDirAllocator getDirectoryAllocator() {
+    return directoryAllocator;
+  }
+
+  /**
+   * Set allocator of local FS storage.
+   * @param value new value
+   * @return the builder
+   */
+  public ObjectReadParameters withDirectoryAllocator(final LocalDirAllocator value) {
+    directoryAllocator = value;
+    return this;
+  }
+
+  /**
+   * Validate that all attributes are as expected.
+   * Mock tests can skip this if required.
+   * @return the object.
+   */
+  public ObjectReadParameters validate() {
+    // please keep in alphabetical order.
+    requireNonNull(boundedThreadPool, "boundedThreadPool");
+    requireNonNull(callbacks, "callbacks");
+    requireNonNull(context, "context");
+    requireNonNull(directoryAllocator, "directoryAllocator");
+    requireNonNull(objectAttributes, "objectAttributes");
+    requireNonNull(streamStatistics, "streamStatistics");
+    return this;
+  }
+}

+ 148 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/StreamFactoryRequirements.java

@@ -0,0 +1,148 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.impl.streams;
+
+import java.util.Arrays;
+import java.util.EnumSet;
+
+import org.apache.hadoop.fs.s3a.VectoredIOContext;
+
+/**
+ * Requirements for requirements for streams from this factory,
+ * including threading and vector IO, and of
+ * the Filesystem instance itself via
+ * {@link Requirements}.
+ * The FS is expected to adapt its internal configuration based on
+ * the requirements passed back by the stream factory after its
+ * creation.
+ */
+public class StreamFactoryRequirements {
+
+  /**
+   * Number of shared threads to included in the bounded pool.
+   */
+  private final int sharedThreads;
+
+  /**
+   * How many threads per stream, ignoring vector IO requirements?
+   */
+  private final int streamThreads;
+
+  /**
+   * VectoredIO behaviour.
+   */
+  private final VectoredIOContext vectoredIOContext;
+
+  /**
+   * Set of requirement flags.
+   */
+  private final EnumSet<Requirements> requirementFlags;
+
+  /**
+   * Create the thread options.
+   * @param sharedThreads Number of shared threads to included in the bounded pool.
+   * @param streamThreads How many threads per stream, ignoring vector IO requirements.
+   * @param vectoredIOContext vector IO settings -made immutable if not already done.
+   * @param requirements requirement flags of the factory and stream.
+   */
+  public StreamFactoryRequirements(
+      final int sharedThreads,
+      final int streamThreads,
+      final VectoredIOContext vectoredIOContext,
+      final Requirements...requirements) {
+    this.sharedThreads = sharedThreads;
+    this.streamThreads = streamThreads;
+    this.vectoredIOContext = vectoredIOContext.build();
+    if (requirements.length == 0) {
+      this.requirementFlags = EnumSet.noneOf(Requirements.class);
+    } else {
+      this.requirementFlags = EnumSet.copyOf((Arrays.asList(requirements)));
+    }
+  }
+
+  /**
+   * Number of shared threads to included in the bounded pool.
+   * @return extra threads to be created in the FS thread pool.
+   */
+  public int sharedThreads() {
+    return sharedThreads;
+  }
+
+  /**
+   * The maximum number of threads which can be used should by a single input stream.
+   * @return thread pool requirements.
+   */
+  public int streamThreads() {
+    return streamThreads;
+  }
+
+  /**
+   * Should the future pool be created?
+   * @return true if the future pool is required.
+   */
+  public boolean requiresFuturePool() {
+    return requires(Requirements.RequiresFuturePool);
+  }
+
+  /**
+   * The VectorIO requirements of streams.
+   * @return vector IO context.
+   */
+  public VectoredIOContext vectoredIOContext() {
+    return vectoredIOContext;
+  }
+
+  /**
+   * Does this factory have this requirement?
+   * @param r requirement to probe for.
+   * @return true if this is a requirement.
+   */
+  public boolean requires(Requirements r) {
+    return requirementFlags.contains(r);
+  }
+
+  @Override
+  public String toString() {
+    return "StreamFactoryRequirements{" +
+        "sharedThreads=" + sharedThreads +
+        ", streamThreads=" + streamThreads +
+        ", requirementFlags=" + requirementFlags +
+        ", vectoredIOContext=" + vectoredIOContext +
+        '}';
+  }
+
+  /**
+   * Requirements a factory may have.
+   */
+  public enum Requirements {
+
+    /**
+     * Expect Unaudited GETs.
+     * Disables auditor warning/errors about GET requests being
+     * issued outside an audit span.
+     */
+    ExpectUnauditedGetRequests,
+
+    /**
+     * Requires a future pool bound to the thread pool.
+     */
+    RequiresFuturePool
+
+  }
+}

+ 213 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/StreamIntegration.java

@@ -0,0 +1,213 @@
+/*
+ * 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.impl.streams;
+
+import java.lang.reflect.Constructor;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.ConfigurationHelper;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.apache.hadoop.fs.s3a.VectoredIOContext;
+import org.apache.hadoop.fs.store.LogExactlyOnce;
+
+import static org.apache.commons.lang3.StringUtils.isEmpty;
+import static org.apache.hadoop.fs.s3a.Constants.AWS_S3_VECTOR_ACTIVE_RANGE_READS;
+import static org.apache.hadoop.fs.s3a.Constants.AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE;
+import static org.apache.hadoop.fs.s3a.Constants.AWS_S3_VECTOR_READS_MIN_SEEK_SIZE;
+import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_AWS_S3_VECTOR_ACTIVE_RANGE_READS;
+import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE;
+import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_AWS_S3_VECTOR_READS_MIN_SEEK_SIZE;
+import static org.apache.hadoop.fs.s3a.Constants.INPUT_STREAM_CUSTOM_FACTORY;
+import static org.apache.hadoop.fs.s3a.Constants.INPUT_STREAM_TYPE;
+import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY;
+import static org.apache.hadoop.fs.s3a.S3AUtils.intOption;
+import static org.apache.hadoop.fs.s3a.S3AUtils.longBytesOption;
+import static org.apache.hadoop.util.Preconditions.checkArgument;
+
+/**
+ * Stream integration, including factory construction.
+ */
+public final class StreamIntegration {
+
+  /**
+   * Enum/config name of a classic S3AInputStream: {@value}.
+   */
+  public static final String CLASSIC = "classic";
+
+  /**
+   * Enum/config name of of the Pinterest S3APrefetchingInputStream: {@value}.
+   */
+  public static final String PREFETCH = "prefetch";
+
+  /**
+   * Enum/config name of the analytics input stream: {@value}.
+   */
+  public static final String ANALYTICS = "analytics";
+
+  /**
+   * Reads in a classname : {@value}.
+   */
+  public static final String CUSTOM = "custom";
+
+  /**
+   * Special string for configuration only; is
+   * mapped to the default stream type: {@value}.
+   */
+  public static final String DEFAULT = "default";
+
+  /**
+   * What is the default type?
+   */
+  public static final InputStreamType DEFAULT_STREAM_TYPE = InputStreamType.Classic;
+
+  /**
+   * Configuration deprecation log for warning about use of the
+   * now deprecated {@code "fs.s3a.prefetch.enabled"} option..
+   */
+  private static final Logger LOG_DEPRECATION =
+      LoggerFactory.getLogger(
+          "org.apache.hadoop.conf.Configuration.deprecation");
+
+  /**
+   * Warn once on use of prefetch configuration option.
+   */
+  private static final LogExactlyOnce WARN_PREFETCH_KEY = new LogExactlyOnce(LOG_DEPRECATION);
+
+  public static final String E_EMPTY_CUSTOM_CLASSNAME =
+      "Configuration option " + INPUT_STREAM_CUSTOM_FACTORY
+          + " is required when the input stream type is \"custom\"";
+
+  public static final String E_INVALID_STREAM_TYPE = "Invalid stream type:";
+
+  private StreamIntegration() {
+  }
+
+  /**
+   * Create the input stream factory the configuration asks for.
+   * <p>
+   * This does not initialize the factory.
+   * <p>
+   * See {@link #determineInputStreamType(Configuration)} for the
+   * resolution algorithm.
+   * @param conf configuration
+   * @return a stream factory.
+   * @throws RuntimeException any binding/loading/instantiation problem
+   */
+  public static ObjectInputStreamFactory factoryFromConfig(final Configuration conf) {
+
+    // Construct the factory.
+    return determineInputStreamType(conf)
+        .factory()
+        .apply(conf);
+  }
+
+  /**
+   * Determine the input stream type for the supplied configuration.
+   * <p>
+   * This does not perform any instantiation.
+   * <p>
+   * If the option {@code "fs.s3a.prefetch.enabled"} is set, the
+   * prefetch stream is selected, after printing a
+   * warning the first time this happens.
+   * <p>
+   * If the input stream type is declared as "default", then whatever
+   * the current default stream type is returned, as defined by
+   * {@link #DEFAULT_STREAM_TYPE}.
+   * @param conf configuration
+   * @return a stream factory.
+   */
+  static InputStreamType determineInputStreamType(final Configuration conf) {
+    // work out the default stream; this includes looking at the
+    // deprecated prefetch enabled key to see if it is set.
+    if (conf.getBoolean(PREFETCH_ENABLED_KEY, false)) {
+      // prefetch enabled, warn (once) then change it to be the default.
+      WARN_PREFETCH_KEY.info("Using {} is deprecated: choose the appropriate stream in {}",
+          PREFETCH_ENABLED_KEY, INPUT_STREAM_TYPE);
+      return InputStreamType.Prefetch;
+    }
+
+    // retrieve the enum value, returning the configured value or
+    // the (calculated) default
+    return ConfigurationHelper.resolveEnum(conf,
+        INPUT_STREAM_TYPE,
+        InputStreamType.class,
+        s -> {
+          if (isEmpty(s) || DEFAULT.equalsIgnoreCase(s)) {
+            // return default type.
+            return DEFAULT_STREAM_TYPE;
+          } else {
+            // any other value
+            throw new IllegalArgumentException(E_INVALID_STREAM_TYPE
+                + " \"" + s + "\"");
+          }
+        });
+  }
+
+  /**
+   * Load the input stream factory defined in the option
+   * {@link Constants#INPUT_STREAM_CUSTOM_FACTORY}.
+   * @param conf configuration to use
+   * @return the custom factory
+   * @throws RuntimeException any binding/loading/instantiation problem
+   */
+  static ObjectInputStreamFactory loadCustomFactory(Configuration conf) {
+
+    // make sure the classname option is actually set
+    final String name = conf.getTrimmed(INPUT_STREAM_CUSTOM_FACTORY, "");
+    checkArgument(!isEmpty(name), E_EMPTY_CUSTOM_CLASSNAME);
+
+    final Class<? extends ObjectInputStreamFactory> factoryClass =
+        conf.getClass(INPUT_STREAM_CUSTOM_FACTORY,
+            null,
+            ObjectInputStreamFactory.class);
+
+    try {
+      final Constructor<? extends ObjectInputStreamFactory> ctor =
+          factoryClass.getConstructor();
+      return ctor.newInstance();
+    } catch (Exception e) {
+      throw new RuntimeException("Failed to instantiate custom class "
+          + name + " " + e, e);
+    }
+  }
+
+  /**
+   * Populates the configurations related to vectored IO operations.
+   * The context is still mutable at this point.
+   * @param conf configuration object.
+   * @return VectoredIOContext.
+   */
+  public static VectoredIOContext populateVectoredIOContext(Configuration conf) {
+    final int minSeekVectored = (int) longBytesOption(conf, AWS_S3_VECTOR_READS_MIN_SEEK_SIZE,
+        DEFAULT_AWS_S3_VECTOR_READS_MIN_SEEK_SIZE, 0);
+    final int maxReadSizeVectored =
+        (int) longBytesOption(conf, AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE,
+            DEFAULT_AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE, 0);
+    final int vectoredActiveRangeReads = intOption(conf,
+        AWS_S3_VECTOR_ACTIVE_RANGE_READS, DEFAULT_AWS_S3_VECTOR_ACTIVE_RANGE_READS, 1);
+    return new VectoredIOContext()
+        .setMinSeekForVectoredReads(minSeekVectored)
+        .setMaxReadSizeForVectoredReads(maxReadSizeVectored)
+        .setVectoredActiveRangeReads(vectoredActiveRangeReads);
+  }
+
+}

+ 29 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/package-info.java

@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/**
+ * Input and Output stream support.
+ * <p>
+ * A lot of the existing stream work is elsewhere,
+ * this module is where ongoing work should take place.
+ */
+
+@InterfaceAudience.Private
+package org.apache.hadoop.fs.s3a.impl.streams;
+
+import org.apache.hadoop.classification.InterfaceAudience;

+ 67 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/PrefetchOptions.java

@@ -0,0 +1,67 @@
+/*
+ * 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.prefetch;
+
+import static org.apache.hadoop.util.Preconditions.checkArgument;
+
+/**
+ * Options for the prefetch stream which are built up in {@link PrefetchingInputStreamFactory}
+ * and passed down.
+ */
+public class PrefetchOptions {
+
+  /** Size in bytes of a single prefetch block. */
+  private final int prefetchBlockSize;
+
+  /** Size of prefetch queue (in number of blocks). */
+  private final int prefetchBlockCount;
+
+  /**
+   * Constructor.
+   * @param prefetchBlockSize the size (in number of bytes) of each prefetched block.
+   * @param prefetchBlockCount maximum number of prefetched blocks.
+   */
+  public PrefetchOptions(final int prefetchBlockSize, final int prefetchBlockCount) {
+
+    checkArgument(
+        prefetchBlockSize > 0, "invalid prefetchBlockSize %d", prefetchBlockSize);
+    this.prefetchBlockSize = prefetchBlockSize;
+    checkArgument(
+        prefetchBlockCount > 0, "invalid prefetchBlockCount %d", prefetchBlockCount);
+    this.prefetchBlockCount = prefetchBlockCount;
+  }
+
+  /**
+   * Gets the size in bytes of a single prefetch block.
+   *
+   * @return the size in bytes of a single prefetch block.
+   */
+  public int getPrefetchBlockSize() {
+    return prefetchBlockSize;
+  }
+
+  /**
+   * Gets the size of prefetch queue (in number of blocks).
+   *
+   * @return the size of prefetch queue (in number of blocks).
+   */
+  public int getPrefetchBlockCount() {
+    return prefetchBlockCount;
+  }
+}

+ 109 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/PrefetchingInputStreamFactory.java

@@ -0,0 +1,109 @@
+/*
+ * 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.prefetch;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.VectoredIOContext;
+import org.apache.hadoop.fs.s3a.impl.streams.AbstractObjectInputStreamFactory;
+import org.apache.hadoop.fs.s3a.impl.streams.InputStreamType;
+import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStream;
+import org.apache.hadoop.fs.s3a.impl.streams.ObjectReadParameters;
+import org.apache.hadoop.fs.s3a.impl.streams.StreamFactoryRequirements;
+
+import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_COUNT_KEY;
+import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_DEFAULT_COUNT;
+import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_DEFAULT_SIZE;
+import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_SIZE_KEY;
+import static org.apache.hadoop.fs.s3a.S3AUtils.intOption;
+import static org.apache.hadoop.fs.s3a.S3AUtils.longBytesOption;
+import static org.apache.hadoop.fs.s3a.impl.streams.StreamIntegration.populateVectoredIOContext;
+import static org.apache.hadoop.util.Preconditions.checkState;
+
+/**
+ * Factory for prefetching streams.
+ * <p>
+ * Reads and validates prefetch configuration options during service init.
+ */
+public class PrefetchingInputStreamFactory extends AbstractObjectInputStreamFactory {
+
+  /** Size in bytes of a single prefetch block. */
+  private int prefetchBlockSize;
+
+  /** Size of prefetch queue (in number of blocks). */
+  private int prefetchBlockCount;
+
+  /**
+   * Shared prefetch options.
+   */
+  private PrefetchOptions prefetchOptions;
+
+  public PrefetchingInputStreamFactory() {
+    super("PrefetchingInputStreamFactory");
+  }
+
+  @Override
+  public InputStreamType streamType() {
+    return InputStreamType.Prefetch;
+  }
+
+  @Override
+  protected void serviceInit(final Configuration conf) throws Exception {
+    super.serviceInit(conf);
+    long prefetchBlockSizeLong =
+        longBytesOption(conf, PREFETCH_BLOCK_SIZE_KEY, PREFETCH_BLOCK_DEFAULT_SIZE, 1);
+    checkState(prefetchBlockSizeLong < Integer.MAX_VALUE,
+        "S3A prefetch block size exceeds int limit");
+    prefetchBlockSize = (int) prefetchBlockSizeLong;
+    prefetchBlockCount =
+        intOption(conf, PREFETCH_BLOCK_COUNT_KEY, PREFETCH_BLOCK_DEFAULT_COUNT, 1);
+
+    prefetchOptions = new PrefetchOptions(
+        prefetchBlockSize,
+        prefetchBlockCount);
+  }
+
+  @Override
+  public ObjectInputStream readObject(final ObjectReadParameters parameters) throws IOException {
+    return new S3APrefetchingInputStream(parameters,
+        getConfig(),
+        prefetchOptions);
+  }
+
+  /**
+   * Calculate Return StreamFactoryRequirements.
+   * @return thread count a vector minimum seek of 0.
+   */
+  @Override
+  public StreamFactoryRequirements factoryRequirements() {
+    // fill in the vector context
+    // and then disable range merging.
+    // this ensures that no reads are made for data which is then discarded...
+    // so the prefetch and block read code doesn't ever do wasteful fetches.
+    final VectoredIOContext vectorContext = populateVectoredIOContext(getConfig())
+        .setMinSeekForVectoredReads(0);
+
+    return new StreamFactoryRequirements(prefetchBlockCount,
+        0,
+        vectorContext,
+        StreamFactoryRequirements.Requirements.RequiresFuturePool);
+  }
+
+}

+ 8 - 8
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ACachingInputStream.java

@@ -32,10 +32,10 @@ import org.apache.hadoop.fs.impl.prefetch.BlockManager;
 import org.apache.hadoop.fs.impl.prefetch.BlockManagerParameters;
 import org.apache.hadoop.fs.impl.prefetch.BufferData;
 import org.apache.hadoop.fs.impl.prefetch.FilePosition;
-import org.apache.hadoop.fs.s3a.S3AInputStream;
 import org.apache.hadoop.fs.s3a.S3AReadOpContext;
 import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
 import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
+import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamCallbacks;
 
 import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_PREFETCH_MAX_BLOCKS_COUNT;
 import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_MAX_BLOCKS_COUNT;
@@ -63,25 +63,25 @@ public class S3ACachingInputStream extends S3ARemoteInputStream {
    * Initializes a new instance of the {@code S3ACachingInputStream} class.
    *
    * @param context read-specific operation context.
-   * @param s3Attributes attributes of the S3 object being read.
+   * @param prefetchOptions prefetch stream specific options
+   * @param s3Attributes attributes of the S3a object being read.
    * @param client callbacks used for interacting with the underlying S3 client.
    * @param streamStatistics statistics for this stream.
    * @param conf the configuration.
    * @param localDirAllocator the local dir allocator instance.
-   * @throws IllegalArgumentException if context is null.
-   * @throws IllegalArgumentException if s3Attributes is null.
-   * @throws IllegalArgumentException if client is null.
+   * @throws NullPointerException if a required parameter is null.
    */
   public S3ACachingInputStream(
       S3AReadOpContext context,
+      PrefetchOptions prefetchOptions,
       S3ObjectAttributes s3Attributes,
-      S3AInputStream.InputStreamCallbacks client,
+      ObjectInputStreamCallbacks client,
       S3AInputStreamStatistics streamStatistics,
       Configuration conf,
       LocalDirAllocator localDirAllocator) {
-    super(context, s3Attributes, client, streamStatistics);
+    super(context, prefetchOptions, s3Attributes, client, streamStatistics);
 
-    this.numBlocksToPrefetch = this.getContext().getPrefetchBlockCount();
+    this.numBlocksToPrefetch = prefetchOptions.getPrefetchBlockCount();
     int bufferPoolSize = this.numBlocksToPrefetch + 1;
     BlockManagerParameters blockManagerParamsBuilder =
         new BlockManagerParameters()

+ 5 - 3
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3AInMemoryInputStream.java

@@ -27,10 +27,10 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.fs.impl.prefetch.BufferData;
 import org.apache.hadoop.fs.impl.prefetch.FilePosition;
-import org.apache.hadoop.fs.s3a.S3AInputStream;
 import org.apache.hadoop.fs.s3a.S3AReadOpContext;
 import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
 import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
+import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamCallbacks;
 
 /**
  * Provides an {@code InputStream} that allows reading from an S3 file.
@@ -50,6 +50,7 @@ public class S3AInMemoryInputStream extends S3ARemoteInputStream {
    * Initializes a new instance of the {@code S3AInMemoryInputStream} class.
    *
    * @param context read-specific operation context.
+   * @param prefetchOptions prefetching options.
    * @param s3Attributes attributes of the S3 object being read.
    * @param client callbacks used for interacting with the underlying S3 client.
    * @param streamStatistics statistics for this stream.
@@ -60,10 +61,11 @@ public class S3AInMemoryInputStream extends S3ARemoteInputStream {
    */
   public S3AInMemoryInputStream(
       S3AReadOpContext context,
+      PrefetchOptions prefetchOptions,
       S3ObjectAttributes s3Attributes,
-      S3AInputStream.InputStreamCallbacks client,
+      ObjectInputStreamCallbacks client,
       S3AInputStreamStatistics streamStatistics) {
-    super(context, s3Attributes, client, streamStatistics);
+    super(context, prefetchOptions, s3Attributes, client, streamStatistics);
     int fileSize = (int) s3Attributes.getLen();
     this.buffer = ByteBuffer.allocate(fileSize);
     LOG.debug("Created in-memory input stream for {} (size = {})",

+ 44 - 23
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchingInputStream.java

@@ -30,17 +30,21 @@ import org.apache.hadoop.classification.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CanSetReadahead;
 import org.apache.hadoop.fs.FSExceptionMessages;
-import org.apache.hadoop.fs.FSInputStream;
 import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.StreamCapabilities;
 import org.apache.hadoop.fs.impl.prefetch.Validate;
-import org.apache.hadoop.fs.s3a.S3AInputStream;
 import org.apache.hadoop.fs.s3a.S3AReadOpContext;
 import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
+import org.apache.hadoop.fs.s3a.impl.streams.InputStreamType;
+import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStream;
+import org.apache.hadoop.fs.s3a.impl.streams.ObjectReadParameters;
 import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
+import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamCallbacks;
 import org.apache.hadoop.fs.statistics.IOStatistics;
 import org.apache.hadoop.fs.statistics.IOStatisticsSource;
 
+import static org.apache.hadoop.util.StringUtils.toLowerCase;
+
 /**
  * Enhanced {@code InputStream} for reading from S3.
  *
@@ -48,7 +52,7 @@ import org.apache.hadoop.fs.statistics.IOStatisticsSource;
  * blocks of configurable size from the underlying S3 file.
  */
 public class S3APrefetchingInputStream
-    extends FSInputStream
+    extends ObjectInputStream
     implements CanSetReadahead, StreamCapabilities, IOStatisticsSource {
 
   private static final Logger LOG = LoggerFactory.getLogger(
@@ -74,26 +78,24 @@ public class S3APrefetchingInputStream
    */
   private S3AInputStreamStatistics inputStreamStatistics = null;
 
+
   /**
    * Initializes a new instance of the {@code S3APrefetchingInputStream} class.
-   *
-   * @param context read-specific operation context.
-   * @param s3Attributes attributes of the S3 object being read.
-   * @param client callbacks used for interacting with the underlying S3 client.
-   * @param streamStatistics statistics for this stream.
+   * @param parameters creation parameters.
    * @param conf the configuration.
-   * @param localDirAllocator the local dir allocator instance retrieved from S3A FS.
-   * @throws IllegalArgumentException if context is null.
-   * @throws IllegalArgumentException if s3Attributes is null.
-   * @throws IllegalArgumentException if client is null.
+   * @param prefetchOptions prefetch stream specific options
+   * @throws NullPointerException if a required parameter is null.
    */
   public S3APrefetchingInputStream(
-      S3AReadOpContext context,
-      S3ObjectAttributes s3Attributes,
-      S3AInputStream.InputStreamCallbacks client,
-      S3AInputStreamStatistics streamStatistics,
-      Configuration conf,
-      LocalDirAllocator localDirAllocator) {
+      final ObjectReadParameters parameters,
+      final Configuration conf,
+      final PrefetchOptions prefetchOptions) {
+    super(InputStreamType.Prefetch, parameters);
+    S3ObjectAttributes s3Attributes = parameters.getObjectAttributes();
+    ObjectInputStreamCallbacks client = parameters.getCallbacks();
+    S3AInputStreamStatistics streamStatistics = parameters.getStreamStatistics();
+    final S3AReadOpContext context = parameters.getContext();
+    LocalDirAllocator localDirAllocator = parameters.getDirectoryAllocator();
 
     Validate.checkNotNull(context, "context");
     Validate.checkNotNull(s3Attributes, "s3Attributes");
@@ -106,10 +108,11 @@ public class S3APrefetchingInputStream
     Validate.checkNotNull(streamStatistics, "streamStatistics");
 
     long fileSize = s3Attributes.getLen();
-    if (fileSize <= context.getPrefetchBlockSize()) {
+    if (fileSize <= prefetchOptions.getPrefetchBlockSize()) {
       LOG.debug("Creating in memory input stream for {}", context.getPath());
       this.inputStream = new S3AInMemoryInputStream(
           context,
+          prefetchOptions,
           s3Attributes,
           client,
           streamStatistics);
@@ -117,6 +120,7 @@ public class S3APrefetchingInputStream
       LOG.debug("Creating in caching input stream for {}", context.getPath());
       this.inputStream = new S3ACachingInputStream(
           context,
+          prefetchOptions,
           s3Attributes,
           client,
           streamStatistics,
@@ -198,6 +202,22 @@ public class S3APrefetchingInputStream
     }
   }
 
+
+  @Override
+  protected boolean isStreamOpen() {
+    return !isClosed();
+  }
+
+  @Override
+  protected void abortInFinalizer() {
+    getS3AStreamStatistics().streamLeaked();
+    try {
+      close();
+    } catch (IOException ignored) {
+
+    }
+  }
+
   /**
    * Updates internal data such that the next read will take place at the given {@code pos}.
    *
@@ -230,11 +250,12 @@ public class S3APrefetchingInputStream
    */
   @Override
   public boolean hasCapability(String capability) {
-    if (!isClosed()) {
-      return inputStream.hasCapability(capability);
+    switch (toLowerCase(capability)) {
+    case StreamCapabilities.READAHEAD:
+      return true;
+    default:
+      return super.hasCapability(capability);
     }
-
-    return false;
   }
 
   /**

+ 7 - 7
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteInputStream.java

@@ -36,11 +36,11 @@ import org.apache.hadoop.fs.impl.prefetch.BlockData;
 import org.apache.hadoop.fs.impl.prefetch.FilePosition;
 import org.apache.hadoop.fs.impl.prefetch.Validate;
 import org.apache.hadoop.fs.s3a.S3AInputPolicy;
-import org.apache.hadoop.fs.s3a.S3AInputStream;
 import org.apache.hadoop.fs.s3a.S3AReadOpContext;
 import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
 import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
 import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
+import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamCallbacks;
 import org.apache.hadoop.fs.statistics.IOStatistics;
 import org.apache.hadoop.fs.statistics.IOStatisticsSource;
 
@@ -98,7 +98,7 @@ public abstract class S3ARemoteInputStream
   private S3ObjectAttributes s3Attributes;
 
   /** Callbacks used for interacting with the underlying S3 client. */
-  private S3AInputStream.InputStreamCallbacks client;
+  private ObjectInputStreamCallbacks client;
 
   /** Used for reporting input stream access statistics. */
   private final S3AInputStreamStatistics streamStatistics;
@@ -113,18 +113,18 @@ public abstract class S3ARemoteInputStream
    * Initializes a new instance of the {@code S3ARemoteInputStream} class.
    *
    * @param context read-specific operation context.
+   * @param prefetchOptions prefetch stream specific options
    * @param s3Attributes attributes of the S3 object being read.
    * @param client callbacks used for interacting with the underlying S3 client.
    * @param streamStatistics statistics for this stream.
    *
-   * @throws IllegalArgumentException if context is null.
-   * @throws IllegalArgumentException if s3Attributes is null.
-   * @throws IllegalArgumentException if client is null.
+   * @throws NullPointerException if a required parameter is null.
    */
   public S3ARemoteInputStream(
       S3AReadOpContext context,
+      PrefetchOptions prefetchOptions,
       S3ObjectAttributes s3Attributes,
-      S3AInputStream.InputStreamCallbacks client,
+      ObjectInputStreamCallbacks client,
       S3AInputStreamStatistics streamStatistics) {
 
     this.context = requireNonNull(context);
@@ -143,7 +143,7 @@ public abstract class S3ARemoteInputStream
     setReadahead(context.getReadahead());
 
     long fileSize = s3Attributes.getLen();
-    int bufferSize = context.getPrefetchBlockSize();
+    int bufferSize = prefetchOptions.getPrefetchBlockSize();
 
     this.blockData = new BlockData(fileSize, bufferSize);
     this.fpos = new FilePosition(fileSize, bufferSize);

+ 3 - 2
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObject.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
 import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
 import org.apache.hadoop.fs.s3a.impl.SDKStreamDrainer;
 import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
+import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamCallbacks;
 import org.apache.hadoop.fs.statistics.DurationTracker;
 
 
@@ -60,7 +61,7 @@ public class S3ARemoteObject {
   /**
    * Callbacks used for interacting with the underlying S3 client.
    */
-  private final S3AInputStream.InputStreamCallbacks client;
+  private final ObjectInputStreamCallbacks client;
 
   /**
    * Used for reporting input stream access statistics.
@@ -100,7 +101,7 @@ public class S3ARemoteObject {
   public S3ARemoteObject(
       S3AReadOpContext context,
       S3ObjectAttributes s3Attributes,
-      S3AInputStream.InputStreamCallbacks client,
+      ObjectInputStreamCallbacks client,
       S3AInputStreamStatistics streamStatistics,
       ChangeTracker changeTracker) {
 

+ 7 - 1
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java

@@ -72,6 +72,7 @@ import static org.apache.hadoop.fs.s3a.Invoker.LOG_EVENT;
 import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
 import static org.apache.hadoop.fs.s3a.commit.staging.StagingCommitterConstants.FILESYSTEM_TEMP_PATH;
 import static org.apache.hadoop.fs.s3a.impl.InternalConstants.S3A_DYNAMIC_CAPABILITIES;
+import static org.apache.hadoop.fs.s3a.impl.streams.StreamIntegration.DEFAULT_STREAM_TYPE;
 import static org.apache.hadoop.fs.s3a.select.SelectConstants.SELECT_UNSUPPORTED;
 import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString;
 import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics;
@@ -454,12 +455,17 @@ public abstract class S3GuardTool extends Configured implements Tool,
       String encryption =
           printOption(out, "\tEncryption", Constants.S3_ENCRYPTION_ALGORITHM,
               "none");
-      printOption(out, "\tInput seek policy", INPUT_FADVISE,
+
+      // stream input
+      printOption(out, "\tInput stream type", INPUT_STREAM_TYPE,
+          DEFAULT_STREAM_TYPE.getName());
+      printOption(out, "\tInput seek policy", INPUT_STREAM_TYPE,
           Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_DEFAULT);
       printOption(out, "\tChange Detection Source", CHANGE_DETECT_SOURCE,
           CHANGE_DETECT_SOURCE_DEFAULT);
       printOption(out, "\tChange Detection Mode", CHANGE_DETECT_MODE,
           CHANGE_DETECT_MODE_DEFAULT);
+
       // committers
       println(out, "%nS3A Committers");
       boolean magic = fs.hasPathCapability(

+ 12 - 2
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/prefetching.md

@@ -39,7 +39,8 @@ Multiple blocks may be read in parallel.
 
 |Property    |Meaning    |Default    |
 |---|---|---|
-|`fs.s3a.prefetch.enabled`    |Enable the prefetch input stream    |`false` |
+| `fs.s3a.input.stream.type` |Uses the prefetch input stream when set to `prefetch`   |`classic` |
+|(deprecated) `fs.s3a.prefetch.enabled`    |Enable the prefetch input stream    |`false` |
 |`fs.s3a.prefetch.block.size`    |Size of a block    |`8M`    |
 |`fs.s3a.prefetch.block.count`    |Number of blocks to prefetch    |`8`    |
 
@@ -47,9 +48,18 @@ The default size of a block is 8MB, and the minimum allowed block size is 1 byte
 Decreasing block size will increase the number of blocks to be read for a file.
 A smaller block size may negatively impact performance as the number of prefetches required will increase.
 
+The original option to enable prefetching was the boolean option `fs.s3a.prefetch.enabled`.
+
+This has been superseded by the option `fs.s3a.input.stream.type` which now takes an enumeration of values; `prefetch` selects the prefetching stream.
+
+1. The original option is deprecated.
+2. It is supported *provided the option `fs.s3a.input.stream.type` is unset.
+3. The first time a stream created through the `fs.s3a.input.stream.type` option,
+   a warning message is printed.
+
 ### Key Components
 
-`S3PrefetchingInputStream` - When prefetching is enabled, S3AFileSystem will return an instance of
+`S3PrefetchingInputStream` - When the prefetch stream is used, S3AFileSystem will return an instance of
 this class as the input stream.
 Depending on the remote file size, it will either use
 the `S3InMemoryInputStream` or the `S3CachingInputStream` as the underlying input stream.

+ 228 - 0
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/reading.md

@@ -0,0 +1,228 @@
+t<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+# Reading Data From S3 Storage.
+
+One of the most important --and performance sensitive-- parts
+of the S3A connector is reading data from storage.
+This is always evolving, based on experience, and benchmarking,
+and in collaboration with other projects.
+
+## Key concepts
+
+* Data is read from S3 through an instance of an `ObjectInputStream`.
+* There are different implementations of this in the codebase:
+  `classic`, `analytics` and `prefetch`; these are called _stream types_
+* The choice of which stream type to use is made in the hadoop configuration.
+
+Configuration Options
+
+
+| Property                   | Permitted Values                                        | Default   | Meaning                    |
+|----------------------------|---------------------------------------------------------|-----------|----------------------------|
+| `fs.s3a.input.stream.type` | `default`, `classic`, `analytics`, `prefetch`, `custom` | `classic` | Name of stream type to use |
+
+### Stream type `default`
+
+The default implementation for this release of Hadoop.
+
+```xml
+<property>
+  <name>fs.s3a.input.stream.type</name>
+  <value>default</value>
+</property>
+```
+
+The choice of which stream type to use by default may change in future releases.
+
+It is currently `classic`.
+
+### Stream type `classic`
+
+This is the classic S3A input stream, present since the original addition of the S3A connector
+to the Hadoop codebase.
+
+```xml
+<property>
+  <name>fs.s3a.input.stream.type</name>
+  <value>classic</value>
+</property>
+```
+
+Strengths
+* Stable
+* Petabytes of data are read through the connector a day,  so well tested in production.
+* Resilience to network and service failures acquired "a stack trace at at time"
+* Implements Vector IO through parallel HTTP requests.
+
+Weaknesses
+* Takes effort to tune for different file formats/read strategies (sequential, random etc),
+  and suboptimal if not correctly tuned for the workloads.
+* Non-vectored reads are blocking, with the sole buffering being that from
+  the http client library and layers beneath.
+* Can keep HTTP connections open too long/not returned to the connection pool.
+  This can consume both network resources and can consume all connections
+  in the pool if streams are not correctly closed by applications.
+
+### Stream type `analytics`
+
+An input stream aware-of and adapted-to the columnar storage
+formats used in production, currently with specific support for
+Apache Parquet.
+
+```xml
+<property>
+  <name>fs.s3a.input.stream.type</name>
+  <value>analytics</value>
+</property>
+```
+
+Strengths
+* Significant speedup measured when reading Parquet files through Spark.
+* Prefetching can also help other read patterns/file formats.
+* Parquet V1 Footer caching reduces HEAD/GET requests when opening
+  and reading files.
+
+Weaknesses
+* Requires an extra library.
+* Currently considered in "stabilization".
+* Likely to need more tuning on failure handling, either in the S3A code or
+  (better) the underlying library.
+* Not yet benchmarked with other applications (Apache Hive or ORC).
+* Vector IO API falls back to a series of sequential read calls.
+  For Parquet the format-aware prefetching will satisfy the requests,
+  but for ORC this may be be very inefficient.
+
+It delivers tangible speedup for reading Parquet files where the reader
+is deployed within AWS infrastructure, it will just take time to encounter
+all the failure conditions which the classic connectors have encountered
+and had to address.
+
+This library is where all future feature development is focused,
+including benchmark-based tuning for other file formats.
+
+
+### Stream type `prefetch`
+
+This input stream prefetches data in multi-MB blocks and caches these
+on the local disk's buffer directory.
+
+```xml
+<property>
+  <name>fs.s3a.input.stream.type</name>
+  <value>prefetch</value>
+</property>
+```
+
+Strengths
+* Format agnostic.
+* Asynchronous, parallel pre-fetching of blocks.
+* Blocking on-demand reads of any blocks which are required and not cached.
+  This may be done in multiple parallel reads, as required.
+* Blocks are cached, so that backwards and random IO is very
+  efficient if using data already read/prefetched.
+
+Weaknesses
+* Caching of blocks is for the duration of the filesystem instance,
+  this works for transient worker processes, but not for
+  long-lived processes such as Spark or HBase workers.
+* No prediction of which blocks are to be prefetched next,
+  so can be wasteful of prefetch reads, while still blocking on application
+  read operations.
+
+
+
+## Vector IO and Stream Types
+
+All streams support VectorIO to some degree.
+
+| Stream      | Support                                                     |
+|-------------|-------------------------------------------------------------|
+| `classic`   | Parallel issuing of GET request with range coalescing       |
+| `prefetch`  | Sequential reads, using prefetched blocks as appropriate    |
+| `analytics` | Sequential reads, using prefetched blocks as where possible |
+
+Because the analytics streams is doing parquet-aware RowGroup prefetch, its
+prefetched blocks should align with Parquet read sequences through vectored
+reads, as well the unvectored reads.
+
+This does not hold for ORC.
+When reading ORC files with a version of the ORC library which is
+configured to use the vector IO API, it is likely to be significantly
+faster to use the classic stream and its parallel reads.
+
+
+## Developer Topics
+
+### Stream IOStatistics
+
+Some of the streams support detailed IOStatistics, which will get aggregated into
+the filesystem IOStatistics when the stream is closed(), or possibly after `unbuffer()`.
+
+The filesystem aggregation can be displayed when the instance is closed, which happens
+in process termination, if not earlier:
+```xml
+  <property>
+    <name>fs.thread.level.iostatistics.enabled</name>
+    <value>true</value>
+  </property>
+```
+
+### Capabilities Probe for stream type and features.
+
+`StreamCapabilities.hasCapability()` can be used to probe for the active
+stream type and its capabilities.
+
+### Unbuffer() support
+
+The `unbuffer()` operation requires the stream to release all client-side
+resources: buffer, connections to remote servers, cached files etc.
+This is used in some query engines, including Apache Impala, to keep
+streams open for rapid re-use, avoiding the overhead of re-opening files.
+
+Only the classic stream supports `CanUnbuffer.unbuffer()`;
+the other streams must be closed rather than kept open for an extended
+period of time.
+
+### Stream Leak alerts
+
+All input streams MUST be closed via a `close()` call once no-longer needed
+-this is the only way to guarantee a timely release of HTTP connections
+and local resources.
+
+Some applications/libraries neglect to close the stram
+
+### Custom Stream Types
+
+There is a special stream type `custom`.
+This is primarily used internally for testing, however it may also be used by
+anyone who wishes to experiment with alternative input stream implementations.
+
+If it is requested, then the name of the _factory_ for streams must be set in the
+property `fs.s3a.input.stream.custom.factory`.
+
+This must be a classname to an implementation of the factory service,
+`org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamFactory`.
+Consult the source and javadocs of the package `org.apache.hadoop.fs.s3a.impl.streams` for
+details.
+
+*Note* this is very much internal code and unstable: any use of this should be considered
+experimental, unstable -and is not recommended for production use.
+
+
+
+| Property                             | Permitted Values                       | Meaning                     |
+|--------------------------------------|----------------------------------------|-----------------------------|
+| `fs.s3a.input.stream.custom.factory` | name of factory class on the classpath | classname of custom factory |
+

+ 1 - 1
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java

@@ -84,7 +84,7 @@ public class ITestS3AContractSeek extends AbstractContractSeekTest {
    * which S3A Supports.
    * @return a list of seek policies to test.
    */
-  @Parameterized.Parameters
+  @Parameterized.Parameters(name="policy={0}")
   public static Collection<Object[]> params() {
     return Arrays.asList(new Object[][]{
         {FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL, Default_JSSE},

+ 14 - 4
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java

@@ -58,6 +58,7 @@ import org.apache.hadoop.security.ProviderUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.alias.CredentialProvider;
 import org.apache.hadoop.security.alias.CredentialProviderFactory;
+import org.apache.hadoop.test.AbstractHadoopTestBase;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.http.HttpStatus;
@@ -75,7 +76,7 @@ import static org.junit.Assert.*;
 /**
  * S3A tests for configuration, especially credentials.
  */
-public class ITestS3AConfiguration {
+public class ITestS3AConfiguration extends AbstractHadoopTestBase {
   private static final String EXAMPLE_ID = "AKASOMEACCESSKEY";
   private static final String EXAMPLE_KEY =
       "RGV0cm9pdCBSZ/WQgY2xl/YW5lZCB1cAEXAMPLE";
@@ -487,11 +488,20 @@ public class ITestS3AConfiguration {
     conf = new Configuration();
     conf.unset(Constants.BUFFER_DIR);
     fs = S3ATestUtils.createTestFileSystem(conf);
-    File tmp = fs.createTmpFileForWrite("out-", 1024, conf);
+    File tmp = createTemporaryFileForWriting();
     assertTrue("not found: " + tmp, tmp.exists());
     tmp.delete();
   }
 
+  /**
+   * Create a temporary file for writing; requires the FS to have been created/initialized.
+   * @return a temporary file
+   * @throws IOException creation issues.
+   */
+  private File createTemporaryFileForWriting() throws IOException {
+    return fs.getS3AInternals().getStore().createTemporaryFileForWriting("out-", 1024, conf);
+  }
+
   @Test
   public void testDirectoryAllocatorRR() throws Throwable {
     File dir1 = GenericTestUtils.getRandomizedTestDir();
@@ -501,9 +511,9 @@ public class ITestS3AConfiguration {
     conf = new Configuration();
     conf.set(Constants.BUFFER_DIR, dir1 + ", " + dir2);
     fs = S3ATestUtils.createTestFileSystem(conf);
-    File tmp1 = fs.createTmpFileForWrite("out-", 1024, conf);
+    File tmp1 = createTemporaryFileForWriting();
     tmp1.delete();
-    File tmp2 = fs.createTmpFileForWrite("out-", 1024, conf);
+    File tmp2 = createTemporaryFileForWriting();
     tmp2.delete();
     assertNotEquals("round robin not working",
         tmp1.getParent(), tmp2.getParent());

+ 27 - 17
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AInputStreamLeakage.java

@@ -29,6 +29,8 @@ import org.slf4j.LoggerFactory;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStream;
+import org.apache.hadoop.fs.statistics.IOStatistics;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 
@@ -58,7 +60,7 @@ public class ITestS3AInputStreamLeakage extends AbstractS3ATestBase {
   @Override
   public void setup() throws Exception {
     super.setup();
-    assume("Stream leak detection not avaialable",
+    assume("Stream leak detection not available",
         getFileSystem().hasCapability(STREAM_LEAKS));
   }
 
@@ -81,10 +83,6 @@ public class ITestS3AInputStreamLeakage extends AbstractS3ATestBase {
    * <p>
    * The stream leak counter of the FileSystem is also updated; this
    * is verified.
-   * <p>
-   * Note: if the stream under test is not an S3AInputStream (i.e. is a prefetching one,
-   * this test is skipped. If/when the prefetching stream adds the same code,
-   * this check can be removed.
    */
   @Test
   public void testFinalizer() throws Throwable {
@@ -100,7 +98,7 @@ public class ITestS3AInputStreamLeakage extends AbstractS3ATestBase {
 
     try {
       Assertions.assertThat(in.hasCapability(STREAM_LEAKS))
-          .describedAs("Stream leak detection not supported in: " + in.getClass())
+          .describedAs("Stream leak detection not supported in: %s", in.getWrappedStream())
           .isTrue();
 
       Assertions.assertThat(in.read())
@@ -108,12 +106,12 @@ public class ITestS3AInputStreamLeakage extends AbstractS3ATestBase {
           .isEqualTo(DATASET[0]);
 
       // get a weak ref so that after a GC we can look for it and verify it is gone
-      Assertions.assertThat(((S3AInputStream) in.getWrappedStream()).isObjectStreamOpen())
-          .describedAs("stream http connection status")
-          .isTrue();
-      // weak reference to track GC progress
-      WeakReference<S3AInputStream> wrs =
-          new WeakReference<>((S3AInputStream) in.getWrappedStream());
+      WeakReference<ObjectInputStream> wrs =
+          new WeakReference<>((ObjectInputStream) in.getWrappedStream());
+
+      boolean isClassicStream = wrs.get() instanceof S3AInputStream;
+      final IOStatistics fsStats = fs.getIOStatistics();
+      final long leaks = fsStats.counters().getOrDefault(STREAM_LEAKS, 0L);
 
       // Capture the logs
       GenericTestUtils.LogCapturer logs =
@@ -125,7 +123,7 @@ public class ITestS3AInputStreamLeakage extends AbstractS3ATestBase {
       in = null;
       // force the gc.
       System.gc();
-      // make sure the GC removed the S3AInputStream.
+      // make sure the GC removed the Stream.
       Assertions.assertThat(wrs.get())
           .describedAs("weak stream reference wasn't GC'd")
           .isNull();
@@ -144,14 +142,26 @@ public class ITestS3AInputStreamLeakage extends AbstractS3ATestBase {
       LOG.info("output of leak log is {}", output);
       Assertions.assertThat(output)
           .describedAs("output from the logs during GC")
-          .contains("drain or abort reason finalize()")  // stream release
+          .contains("Stream not closed")  // stream release
           .contains(path.toUri().toString())             // path
           .contains(Thread.currentThread().getName())    // thread
           .contains("testFinalizer");                    // stack
-
       // verify that leakages are added to the FS statistics
-      assertThatStatisticCounter(fs.getIOStatistics(), STREAM_LEAKS)
-          .isEqualTo(1);
+
+      // for classic stream the counter is 1, but for prefetching
+      // the count is greater -the inner streams can also
+      // get finalized while open so increment the leak counter
+      // multiple times.
+      assertThatStatisticCounter(fsStats, STREAM_LEAKS)
+          .isGreaterThanOrEqualTo(leaks + 1);
+      if (isClassicStream) {
+        Assertions.assertThat(output)
+            .describedAs("output from the logs during GC")
+            .contains("drain or abort reason finalize()");  // stream release
+        assertThatStatisticCounter(fsStats, STREAM_LEAKS)
+            .isEqualTo(leaks + 1);
+      }
+
     } finally {
       if (in != null) {
         IOUtils.cleanupWithLogger(LOG, in);

+ 2 - 3
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingCacheFiles.java

@@ -40,7 +40,7 @@ import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest;
 import static org.apache.hadoop.fs.s3a.Constants.BUFFER_DIR;
 import static org.apache.hadoop.fs.s3a.Constants.ENDPOINT;
 import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_SIZE_KEY;
-import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.enablePrefetching;
 import static org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils.getExternalData;
 import static org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils.isUsingDefaultExternalDataFile;
 import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
@@ -84,10 +84,9 @@ public class ITestS3APrefetchingCacheFiles extends AbstractS3ACostTest {
     Configuration configuration = super.createConfiguration();
     if (isUsingDefaultExternalDataFile(configuration)) {
       S3ATestUtils.removeBaseAndBucketOverrides(configuration,
-          PREFETCH_ENABLED_KEY,
           ENDPOINT);
     }
-    configuration.setBoolean(PREFETCH_ENABLED_KEY, true);
+    enablePrefetching(configuration);
     // use a small block size unless explicitly set in the test config.
     configuration.setInt(PREFETCH_BLOCK_SIZE_KEY, BLOCK_SIZE);
     // patch buffer dir with a unique path for test isolation.

+ 2 - 4
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingInputStream.java

@@ -35,7 +35,7 @@ import org.apache.hadoop.fs.statistics.IOStatistics;
 import org.apache.hadoop.test.LambdaTestUtils;
 
 import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_SIZE_KEY;
-import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.enablePrefetching;
 import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticMaximum;
 import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue;
 import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticGaugeValue;
@@ -73,10 +73,8 @@ public class ITestS3APrefetchingInputStream extends AbstractS3ACostTest {
 
   @Override
   public Configuration createConfiguration() {
-    Configuration conf = super.createConfiguration();
-    S3ATestUtils.removeBaseAndBucketOverrides(conf, PREFETCH_ENABLED_KEY);
+    Configuration conf = enablePrefetching(super.createConfiguration());
     S3ATestUtils.removeBaseAndBucketOverrides(conf, PREFETCH_BLOCK_SIZE_KEY);
-    conf.setBoolean(PREFETCH_ENABLED_KEY, true);
     conf.setInt(PREFETCH_BLOCK_SIZE_KEY, BLOCK_SIZE);
     return conf;
   }

+ 5 - 6
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingLruEviction.java

@@ -43,8 +43,8 @@ import org.apache.hadoop.fs.statistics.IOStatistics;
 import org.apache.hadoop.test.LambdaTestUtils;
 
 import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_SIZE_KEY;
-import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY;
 import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_MAX_BLOCKS_COUNT;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.enablePrefetching;
 import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter;
 import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValues;
 import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticGaugeValue;
@@ -85,11 +85,10 @@ public class ITestS3APrefetchingLruEviction extends AbstractS3ACostTest {
 
   @Override
   public Configuration createConfiguration() {
-    Configuration conf = super.createConfiguration();
-    S3ATestUtils.removeBaseAndBucketOverrides(conf, PREFETCH_ENABLED_KEY);
-    S3ATestUtils.removeBaseAndBucketOverrides(conf, PREFETCH_MAX_BLOCKS_COUNT);
-    S3ATestUtils.removeBaseAndBucketOverrides(conf, PREFETCH_BLOCK_SIZE_KEY);
-    conf.setBoolean(PREFETCH_ENABLED_KEY, true);
+    Configuration conf = enablePrefetching(super.createConfiguration());
+    S3ATestUtils.removeBaseAndBucketOverrides(conf,
+        PREFETCH_MAX_BLOCKS_COUNT,
+        PREFETCH_BLOCK_SIZE_KEY);
     conf.setInt(PREFETCH_MAX_BLOCKS_COUNT, Integer.parseInt(maxBlocks));
     conf.setInt(PREFETCH_BLOCK_SIZE_KEY, BLOCK_SIZE);
     return conf;

+ 11 - 7
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java

@@ -31,9 +31,8 @@ import org.apache.hadoop.fs.statistics.IOStatisticAssertions;
 import org.apache.hadoop.fs.statistics.StreamStatisticNames;
 
 import static org.apache.hadoop.fs.s3a.Constants.ALLOW_REQUESTER_PAYS;
-import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_DEFAULT;
-import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY;
 import static org.apache.hadoop.fs.s3a.Constants.S3A_BUCKET_PROBE;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.streamType;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 /**
@@ -80,15 +79,20 @@ public class ITestS3ARequesterPays extends AbstractS3ATestBase {
       inputStream.seek(0);
       inputStream.readByte();
 
-      if (conf.getBoolean(PREFETCH_ENABLED_KEY, PREFETCH_ENABLED_DEFAULT)) {
-        // For S3APrefetchingInputStream, verify a call was made
-        IOStatisticAssertions.assertThatStatisticCounter(inputStream.getIOStatistics(),
-            StreamStatisticNames.STREAM_READ_OPENED).isEqualTo(1);
-      } else {
+      switch (streamType(getFileSystem())) {
+      case Classic:
         // For S3AInputStream, verify > 1 call was made,
         // so we're sure it is correctly configured for each request
         IOStatisticAssertions.assertThatStatisticCounter(inputStream.getIOStatistics(),
             StreamStatisticNames.STREAM_READ_OPENED).isGreaterThan(1);
+        break;
+      case Prefetch:
+        // For S3APrefetchingInputStream, verify a call was made
+        IOStatisticAssertions.assertThatStatisticCounter(inputStream.getIOStatistics(),
+            StreamStatisticNames.STREAM_READ_OPENED).isEqualTo(1);
+        break;
+      default:
+        break;
       }
 
       // Check list calls work without error

+ 46 - 6
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java

@@ -45,6 +45,7 @@ import org.apache.hadoop.fs.s3a.impl.S3ExpressStorage;
 import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum;
 import org.apache.hadoop.fs.s3a.impl.StoreContext;
 import org.apache.hadoop.fs.s3a.impl.StoreContextBuilder;
+import org.apache.hadoop.fs.s3a.impl.streams.InputStreamType;
 import org.apache.hadoop.fs.s3a.prefetch.S3APrefetchingInputStream;
 import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics;
 import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
@@ -104,6 +105,7 @@ import java.util.stream.Collectors;
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
 import static org.apache.hadoop.fs.impl.FlagSet.createFlagSet;
+import static org.apache.hadoop.fs.s3a.impl.streams.InputStreamType.Prefetch;
 import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit;
 import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletion;
 import static org.apache.hadoop.fs.s3a.impl.S3ExpressStorage.STORE_CAPABILITY_S3_EXPRESS_STORAGE;
@@ -118,6 +120,7 @@ import static org.apache.hadoop.fs.s3a.S3ATestConstants.*;
 import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.S3AUtils.buildEncryptionSecrets;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+import static org.apache.hadoop.util.functional.FunctionalIO.uncheckIOExceptions;
 import static org.apache.hadoop.util.functional.RemoteIterators.mappingRemoteIterator;
 import static org.apache.hadoop.util.functional.RemoteIterators.toList;
 import static org.junit.Assert.*;
@@ -722,10 +725,8 @@ public final class S3ATestUtils {
     }
     conf.set(BUFFER_DIR, tmpDir);
 
-    boolean prefetchEnabled =
-        getTestPropertyBool(conf, PREFETCH_ENABLED_KEY, PREFETCH_ENABLED_DEFAULT);
-    conf.setBoolean(PREFETCH_ENABLED_KEY, prefetchEnabled);
-
+    conf.set(INPUT_STREAM_TYPE,
+        getTestProperty(conf, INPUT_STREAM_TYPE, INPUT_STREAM_TYPE_DEFAULT));
     return conf;
   }
 
@@ -1780,11 +1781,50 @@ public final class S3ATestUtils {
   /**
    * Disable Prefetching streams from S3AFileSystem in tests.
    * @param conf Configuration to remove the prefetch property from.
+   * @return patched config
+   */
+  public static Configuration disablePrefetching(Configuration conf) {
+    removeBaseAndBucketOverrides(conf,
+        PREFETCH_ENABLED_KEY,
+        INPUT_STREAM_TYPE);
+    return conf;
+  }
+
+
+  /**
+   *Enable Prefetching streams from S3AFileSystem in tests.
+   * @param conf Configuration to update
+   * @return patched config
+   */
+  public static Configuration enablePrefetching(Configuration conf) {
+    removeBaseAndBucketOverrides(conf,
+        PREFETCH_ENABLED_KEY,
+        INPUT_STREAM_TYPE);
+    conf.setEnum(INPUT_STREAM_TYPE, Prefetch);
+    return conf;
+  }
+
+  /**
+   * Probe for a filesystem having a specific stream type;
+   * this is done through filesystem capabilities.
+   * @param fs filesystem
+   * @param type stream type
+   * @return true if the fs has the specific type.
    */
-  public static void disablePrefetching(Configuration conf) {
-    removeBaseAndBucketOverrides(conf, PREFETCH_ENABLED_KEY);
+  public static boolean hasInputStreamType(FileSystem fs, InputStreamType type) {
+    return uncheckIOExceptions(() ->
+        fs.hasPathCapability(new Path("/"),
+            type.capability()));
   }
 
+  /**
+   * What is the stream type of this filesystem?
+   * @param fs filesystem to probe
+   * @return the stream type
+   */
+  public static InputStreamType streamType(S3AFileSystem fs) {
+    return fs.getS3AInternals().getStore().streamType();
+  }
   /**
    * Skip root tests if the system properties/config says so.
    * @param conf configuration to check

+ 16 - 11
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java

@@ -40,6 +40,8 @@ import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.s3a.audit.impl.NoopSpan;
 import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets;
+import org.apache.hadoop.fs.s3a.impl.streams.ObjectReadParameters;
+import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamCallbacks;
 import org.apache.hadoop.util.functional.CallableRaisingIOE;
 import org.apache.http.NoHttpResponseException;
 
@@ -164,7 +166,7 @@ public class TestS3AInputStreamRetry extends AbstractS3AMockTest {
    * @return a stream.
    */
   private S3AInputStream getMockedS3AInputStream(
-      S3AInputStream.InputStreamCallbacks streamCallback) {
+      ObjectInputStreamCallbacks streamCallback) {
     Path path = new Path("test-path");
     String eTag = "test-etag";
     String versionId = "test-version-id";
@@ -187,12 +189,15 @@ public class TestS3AInputStreamRetry extends AbstractS3AMockTest {
         s3AFileStatus,
         NoopSpan.INSTANCE);
 
-    return new S3AInputStream(
-        s3AReadOpContext,
-        s3ObjectAttributes,
-        streamCallback,
-        s3AReadOpContext.getS3AStatisticsContext().newInputStreamStatistics(),
-            null);
+    ObjectReadParameters parameters = new ObjectReadParameters()
+        .withCallbacks(streamCallback)
+        .withObjectAttributes(s3ObjectAttributes)
+        .withContext(s3AReadOpContext)
+        .withStreamStatistics(
+            s3AReadOpContext.getS3AStatisticsContext().newInputStreamStatistics())
+        .withBoundedThreadPool(null);
+
+    return new S3AInputStream(parameters);
   }
 
   /**
@@ -203,7 +208,7 @@ public class TestS3AInputStreamRetry extends AbstractS3AMockTest {
    * @param ex exception to raise on failure
    * @return mocked object.
    */
-  private S3AInputStream.InputStreamCallbacks failingInputStreamCallbacks(
+  private ObjectInputStreamCallbacks failingInputStreamCallbacks(
       final RuntimeException ex) {
 
     GetObjectResponse objectResponse = GetObjectResponse.builder()
@@ -238,7 +243,7 @@ public class TestS3AInputStreamRetry extends AbstractS3AMockTest {
    * @param ex exception to raise on failure
    * @return mocked object.
    */
-  private S3AInputStream.InputStreamCallbacks maybeFailInGetCallback(
+  private ObjectInputStreamCallbacks maybeFailInGetCallback(
       final RuntimeException ex,
       final Function<Integer, Boolean> failurePredicate) {
     GetObjectResponse objectResponse = GetObjectResponse.builder()
@@ -259,13 +264,13 @@ public class TestS3AInputStreamRetry extends AbstractS3AMockTest {
   * @param streamFactory factory for the stream to return on the given attempt.
   * @return mocked object.
   */
-  private S3AInputStream.InputStreamCallbacks mockInputStreamCallback(
+  private ObjectInputStreamCallbacks mockInputStreamCallback(
       final RuntimeException ex,
       final Function<Integer, Boolean> failurePredicate,
       final Function<Integer, ResponseInputStream<GetObjectResponse>> streamFactory) {
 
 
-    return new S3AInputStream.InputStreamCallbacks() {
+    return new ObjectInputStreamCallbacks() {
       private int attempt = 0;
 
       @Override

+ 10 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AccessCheckingAuditor.java

@@ -20,6 +20,9 @@ package org.apache.hadoop.fs.s3a.audit;
 
 import java.io.IOException;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.s3a.S3AFileStatus;
@@ -33,10 +36,14 @@ public class AccessCheckingAuditor  extends NoopAuditor {
   public static final String CLASS =
       "org.apache.hadoop.fs.s3a.audit.AccessCheckingAuditor";
 
+  private static final Logger LOG =
+       LoggerFactory.getLogger(AccessCheckingAuditor.class);
+
   /** Flag to enable/disable access. */
   private boolean accessAllowed = true;
 
   public AccessCheckingAuditor() {
+    super("AccessCheckingAuditor");
   }
 
   public void setAccessAllowed(final boolean accessAllowed) {
@@ -48,6 +55,9 @@ public class AccessCheckingAuditor  extends NoopAuditor {
       final S3AFileStatus status,
       final FsAction mode)
       throws IOException {
+
+    LOG.debug("Check access to {}; allowed={}",
+        path, accessAllowed);
     return accessAllowed;
   }
 }

+ 25 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AuditTestSupport.java

@@ -18,7 +18,10 @@
 
 package org.apache.hadoop.fs.s3a.audit;
 
+import org.assertj.core.api.Assumptions;
+
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
 import org.apache.hadoop.fs.s3a.S3ATestUtils;
 import org.apache.hadoop.fs.s3a.audit.impl.NoopAuditManagerS3A;
 import org.apache.hadoop.fs.s3a.audit.impl.NoopAuditor;
@@ -29,6 +32,7 @@ import static org.apache.hadoop.fs.s3a.Statistic.AUDIT_ACCESS_CHECK_FAILURE;
 import static org.apache.hadoop.fs.s3a.Statistic.AUDIT_FAILURE;
 import static org.apache.hadoop.fs.s3a.Statistic.AUDIT_REQUEST_EXECUTION;
 import static org.apache.hadoop.fs.s3a.Statistic.AUDIT_SPAN_CREATION;
+import static org.apache.hadoop.fs.s3a.audit.AuditIntegration.isRejectOutOfSpan;
 import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_ENABLED;
 import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_EXECUTION_INTERCEPTORS;
 import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_SERVICE_CLASSNAME;
@@ -122,6 +126,8 @@ public final class AuditTestSupport {
   /**
    * Remove all overridden values for
    * the test bucket/global in the given config.
+   * Note that the rejection flag may be overridden by the
+   * requirements returned by the output stream factory.
    * @param conf configuration to patch
    * @return the configuration.
    */
@@ -134,4 +140,23 @@ public final class AuditTestSupport {
         AUDIT_ENABLED);
     return conf;
   }
+
+  /**
+   * Skip a test if the filesystem's audit manager has had them disabled.
+   * @param fs filesystem
+   */
+  public static void requireOutOfSpanOperationsRejected(final S3AFileSystem fs) {
+    Assumptions.assumeThat(outOfSpanOperationAreRejected(fs))
+        .describedAs("Out of span operations rejected")
+        .isTrue();
+  }
+
+  /**
+   * Are Out of Span operations rejected by the filesystem's audit manager?
+   * @param fs filesystem
+   * @return true if out of span calls raise exceptions
+   */
+  private static boolean outOfSpanOperationAreRejected(final S3AFileSystem fs) {
+    return isRejectOutOfSpan(fs.getAuditManager().getConfig());
+  }
 }

+ 9 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditManager.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.fs.s3a.audit;
 
 import java.nio.file.AccessDeniedException;
+import java.util.EnumSet;
 
 import org.assertj.core.api.Assertions;
 import org.junit.Test;
@@ -32,6 +33,7 @@ import org.apache.hadoop.fs.statistics.IOStatistics;
 import static org.apache.hadoop.fs.s3a.Statistic.AUDIT_FAILURE;
 import static org.apache.hadoop.fs.s3a.Statistic.AUDIT_REQUEST_EXECUTION;
 import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.enableLoggingAuditor;
+import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.requireOutOfSpanOperationsRejected;
 import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.resetAuditOptions;
 import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_EXECUTION_INTERCEPTORS;
 import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_REQUEST_HANDLERS;
@@ -78,6 +80,9 @@ public class ITestAuditManager extends AbstractS3ACostTest {
   public void testInvokeOutOfSpanRejected() throws Throwable {
     describe("Operations against S3 will be rejected outside of a span");
     final S3AFileSystem fs = getFileSystem();
+
+    requireOutOfSpanOperationsRejected(fs);
+
     final long failures0 = lookupCounterStatistic(iostats(),
         AUDIT_FAILURE.getSymbol());
     final long exec0 = lookupCounterStatistic(iostats(),
@@ -109,6 +114,10 @@ public class ITestAuditManager extends AbstractS3ACostTest {
         .isGreaterThan(exec0);
     assertThatStatisticCounter(iostats(), AUDIT_FAILURE.getSymbol())
         .isGreaterThan(failures0);
+
+    // stop rejecting out of span requests
+    fs.getAuditManager().setAuditFlags(EnumSet.of(AuditorFlags.PermitOutOfBandOperations));
+    writer.listMultipartUploads("/");
   }
 
   @Test

+ 3 - 4
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestConnectionTimeouts.java

@@ -55,10 +55,10 @@ import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_PERFORMANCE_FLAGS;
 import static org.apache.hadoop.fs.s3a.Constants.MAXIMUM_CONNECTIONS;
 import static org.apache.hadoop.fs.s3a.Constants.MAX_ERROR_RETRIES;
 import static org.apache.hadoop.fs.s3a.Constants.PART_UPLOAD_TIMEOUT;
-import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY;
 import static org.apache.hadoop.fs.s3a.Constants.REQUEST_TIMEOUT;
 import static org.apache.hadoop.fs.s3a.Constants.RETRY_LIMIT;
 import static org.apache.hadoop.fs.s3a.Constants.SOCKET_TIMEOUT;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.disablePrefetching;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
 import static org.apache.hadoop.fs.s3a.commit.CommitConstants.MAGIC_PATH_PREFIX;
 import static org.apache.hadoop.fs.s3a.impl.ConfigurationHelper.setDurationAsMillis;
@@ -104,7 +104,7 @@ public class ITestConnectionTimeouts extends AbstractS3ATestBase {
    * @return a configuration to use for the brittle FS.
    */
   private Configuration timingOutConfiguration() {
-    Configuration conf = new Configuration(getConfiguration());
+    Configuration conf = disablePrefetching(new Configuration(getConfiguration()));
     removeBaseAndBucketOverrides(conf,
         CONNECTION_TTL,
         CONNECTION_ACQUISITION_TIMEOUT,
@@ -113,7 +113,6 @@ public class ITestConnectionTimeouts extends AbstractS3ATestBase {
         MAX_ERROR_RETRIES,
         MAXIMUM_CONNECTIONS,
         PART_UPLOAD_TIMEOUT,
-        PREFETCH_ENABLED_KEY,
         REQUEST_TIMEOUT,
         SOCKET_TIMEOUT,
         FS_S3A_CREATE_PERFORMANCE,
@@ -125,7 +124,7 @@ public class ITestConnectionTimeouts extends AbstractS3ATestBase {
     conf.setInt(MAX_ERROR_RETRIES, 0);
     // needed to ensure that streams are kept open.
     // without this the tests is unreliable in batch runs.
-    conf.setBoolean(PREFETCH_ENABLED_KEY, false);
+    disablePrefetching(conf);
     conf.setInt(RETRY_LIMIT, 0);
     conf.setBoolean(FS_S3A_CREATE_PERFORMANCE, true);
     final Duration ms10 = Duration.ofMillis(10);

+ 339 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/streams/TestStreamFactories.java

@@ -0,0 +1,339 @@
+/*
+ * 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.impl.streams;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import software.amazon.awssdk.services.s3.S3AsyncClient;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.VectoredIOContext;
+import org.apache.hadoop.fs.s3a.prefetch.PrefetchingInputStreamFactory;
+import org.apache.hadoop.test.AbstractHadoopTestBase;
+
+import static org.apache.hadoop.fs.s3a.Constants.INPUT_STREAM_CUSTOM_FACTORY;
+import static org.apache.hadoop.fs.s3a.Constants.INPUT_STREAM_TYPE;
+import static org.apache.hadoop.fs.s3a.Constants.INPUT_STREAM_TYPE_CLASSIC;
+import static org.apache.hadoop.fs.s3a.Constants.INPUT_STREAM_TYPE_CUSTOM;
+import static org.apache.hadoop.fs.s3a.Constants.INPUT_STREAM_TYPE_DEFAULT;
+import static org.apache.hadoop.fs.s3a.Constants.INPUT_STREAM_TYPE_PREFETCH;
+import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY;
+import static org.apache.hadoop.fs.s3a.impl.streams.StreamFactoryRequirements.Requirements.ExpectUnauditedGetRequests;
+import static org.apache.hadoop.fs.s3a.impl.streams.StreamFactoryRequirements.Requirements.RequiresFuturePool;
+import static org.apache.hadoop.fs.s3a.impl.streams.StreamIntegration.DEFAULT_STREAM_TYPE;
+import static org.apache.hadoop.fs.s3a.impl.streams.StreamIntegration.E_EMPTY_CUSTOM_CLASSNAME;
+import static org.apache.hadoop.fs.s3a.impl.streams.StreamIntegration.E_INVALID_STREAM_TYPE;
+import static org.apache.hadoop.fs.s3a.impl.streams.StreamIntegration.factoryFromConfig;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Unit test for stream factory creation.
+ * Verifies mapping of name to type, default handling,
+ * legacy prefetch switch and failure handling.
+ */
+public class TestStreamFactories extends AbstractHadoopTestBase {
+
+  /**
+   * The empty string and "default" both map to the classic stream.
+   */
+  @Test
+  public void testDefaultFactoryCreation() throws Throwable {
+    load("", DEFAULT_STREAM_TYPE,
+        ClassicObjectInputStreamFactory.class);
+    load(INPUT_STREAM_TYPE_DEFAULT, DEFAULT_STREAM_TYPE,
+        ClassicObjectInputStreamFactory.class);
+  }
+
+  /**
+   * Classic factory.
+   */
+  @Test
+  public void testClassicFactoryCreation() throws Throwable {
+    final ClassicObjectInputStreamFactory f =
+        load(INPUT_STREAM_TYPE_CLASSIC, DEFAULT_STREAM_TYPE,
+            ClassicObjectInputStreamFactory.class);
+    final StreamFactoryRequirements requirements = f.factoryRequirements();
+    Assertions.assertThat(requirements.requiresFuturePool())
+        .describedAs("requires future pool of %s", requirements)
+        .isFalse();
+    assertRequirement(requirements,
+        ExpectUnauditedGetRequests,
+        false);
+  }
+
+  /**
+   * Asset taht the requirements matches the specified need.
+   * @param requirements requirements instance
+   * @param probe requirement to probe for.
+   * @param shouldMatch is the requirement to be met to to fail?
+   */
+  private static void assertRequirement(
+      final StreamFactoryRequirements requirements,
+      final StreamFactoryRequirements.Requirements probe,
+      final boolean shouldMatch) {
+    Assertions.assertThat(requirements.requires(probe))
+        .describedAs("%s of %s", probe, requirements)
+        .isEqualTo(shouldMatch);
+  }
+
+  /**
+   * Prefetch factory.
+   */
+  @Test
+  public void testPrefetchFactoryCreation() throws Throwable {
+    // load from config option
+    final PrefetchingInputStreamFactory f = load(INPUT_STREAM_TYPE_PREFETCH,
+        InputStreamType.Prefetch,
+        PrefetchingInputStreamFactory.class);
+    final StreamFactoryRequirements requirements = f.factoryRequirements();
+    Assertions.assertThat(requirements.requiresFuturePool())
+        .describedAs("requires future pool of %s", requirements)
+        .isTrue();
+    assertRequirement(requirements,
+        ExpectUnauditedGetRequests,
+        false);
+    assertRequirement(requirements,
+        RequiresFuturePool,
+        true);
+  }
+
+  /**
+   * Prefetch factory via the prefect enabled flag.
+   * This is returned before any attempt is made to instantiate
+   * the stream type option.
+   */
+
+  @Test
+  public void testPrefetchEnabledFlag() throws Throwable {
+
+    // request an analytics stream
+    final Configuration conf = configWithStream("undefined");
+    // but then set the prefetch key
+    conf.setBoolean(PREFETCH_ENABLED_KEY, true);
+    assertFactorySatisfies(factoryFromConfig(conf),
+        INPUT_STREAM_TYPE_PREFETCH,
+        InputStreamType.Prefetch,
+        PrefetchingInputStreamFactory.class);
+  }
+
+  @Test
+  public void testRequirementFlagsNoElements() throws Throwable {
+    VectoredIOContext vertex = new VectoredIOContext();
+
+    // no elements
+    final StreamFactoryRequirements r1 =
+        new StreamFactoryRequirements(1, 2, vertex);
+    assertRequirement(r1, ExpectUnauditedGetRequests, false);
+    assertRequirement(r1, RequiresFuturePool, false);
+    Assertions.assertThat(r1.requiresFuturePool())
+        .describedAs("requiresFuturePool() %s", r1)
+        .isFalse();
+    Assertions.assertThat(r1)
+        .describedAs("%s", r1)
+        .matches(r -> !r.requiresFuturePool(), "requiresFuturePool")
+        .satisfies(r ->
+            Assertions.assertThat(r.sharedThreads()).isEqualTo(1))
+        .satisfies(r ->
+            Assertions.assertThat(r.streamThreads()).isEqualTo(2));
+  }
+
+  @Test
+  public void testRequirementFlagsFutures() throws Throwable {
+    VectoredIOContext vertex = new VectoredIOContext();
+
+    final StreamFactoryRequirements r1 =
+        new StreamFactoryRequirements(1, 2, vertex, RequiresFuturePool);
+    assertRequirement(r1, ExpectUnauditedGetRequests, false);
+    assertRequirement(r1, RequiresFuturePool, true);
+    Assertions.assertThat(r1.requiresFuturePool())
+        .describedAs("requiresFuturePool() %s", r1)
+        .isTrue();
+  }
+
+  @Test
+  public void testRequirementFlagsUnaudited() throws Throwable {
+    VectoredIOContext vertex = new VectoredIOContext();
+
+    final StreamFactoryRequirements r1 =
+        new StreamFactoryRequirements(1, 2, vertex, ExpectUnauditedGetRequests);
+    assertRequirement(r1, ExpectUnauditedGetRequests, true);
+    assertRequirement(r1, RequiresFuturePool, false);
+  }
+
+
+  /**
+   * Create a factory, assert that it satisfies the requirements.
+   * @param name name: only used for assertion messages.
+   * @param type expected stream type.
+   * @param clazz expected class.
+   * @param <T> class to expect
+   */
+  private static <T extends ObjectInputStreamFactory> T load(
+      String name,
+      InputStreamType type,
+      Class<T> clazz) throws IOException {
+
+    final ObjectInputStreamFactory factory = factory(name);
+    assertFactorySatisfies(factory, name, type, clazz);
+    factory.init(new Configuration(false));
+    factory.bind(new FactoryBindingParameters(new Callbacks()));
+    return (T)factory;
+  }
+
+  /**
+   * Assert that a factory satisfies the requirements.
+   * @param factory factory
+   * @param name name: only used for assertion messages.
+   * @param type expected stream type.
+   * @param clazz expected class.
+   * @param <T> class to expect
+   */
+  private static <T extends ObjectInputStreamFactory> void assertFactorySatisfies(
+      final ObjectInputStreamFactory factory,
+      final String name,
+      final InputStreamType type,
+      final Class<T> clazz) {
+    assertThat(factory)
+        .describedAs("Factory for stream %s", name)
+        .isInstanceOf(clazz)
+        .satisfies(f ->
+            assertThat(factory.streamType()).isEqualTo(type));
+  }
+
+  /**
+   * When an unknown stream type is passed in, it is rejected.
+   */
+  @Test
+  public void testUnknownStreamType() throws Throwable {
+    final String name = "unknown";
+    intercept(IllegalArgumentException.class, E_INVALID_STREAM_TYPE,
+        () -> factory(name));
+  }
+
+  /**
+   * Create a factory, using the given name as the configuration option.
+   * @param name stream name.
+   * @return the factory
+   */
+  private static ObjectInputStreamFactory factory(final String name) {
+    return factoryFromConfig(configWithStream(name));
+  }
+
+  /**
+   * Create a configuration with the given name declared as the input
+   * stream.
+   * @param name stream name.
+   * @return the prepared configuration.
+   */
+  private static Configuration configWithStream(final String name) {
+    final Configuration conf = new Configuration(false);
+    conf.set(INPUT_STREAM_TYPE, name);
+    return conf;
+  }
+
+  /**
+   * Custom factory loading: the good path.
+   */
+  @Test
+  public void testCustomFactoryLoad() throws Throwable {
+    final Configuration conf = configWithStream(INPUT_STREAM_TYPE_CUSTOM);
+    conf.set(INPUT_STREAM_CUSTOM_FACTORY, CustomFactory.class.getName());
+    final ObjectInputStreamFactory factory = factoryFromConfig(conf);
+    assertThat(factory.streamType())
+        .isEqualTo(InputStreamType.Custom);
+    assertThat(factory)
+        .isInstanceOf(CustomFactory.class);
+  }
+
+  /**
+   * A custom factory must have a classname.
+   */
+  @Test
+  public void testCustomFactoryUndefined() throws Throwable {
+    intercept(IllegalArgumentException.class, E_EMPTY_CUSTOM_CLASSNAME,
+        () -> factory(INPUT_STREAM_TYPE_CUSTOM));
+  }
+
+  /**
+   * Constructor failures are passed in, deeply wrapped though.
+   */
+  @Test
+  public void testCustomConstructorFailure() throws Throwable {
+    final Configuration conf = configWithStream(INPUT_STREAM_TYPE_CUSTOM);
+    conf.set(INPUT_STREAM_CUSTOM_FACTORY, FactoryFailsToInstantiate.class.getName());
+    final RuntimeException ex =
+        intercept(RuntimeException.class, "InvocationTargetException",
+            () -> factoryFromConfig(conf));
+    assertThat(ex.getCause().getCause())
+        .describedAs("innermost exception")
+        .isInstanceOf(UncheckedIOException.class);
+  }
+
+  /**
+   * Simple factory.
+   */
+  public static class CustomFactory extends AbstractObjectInputStreamFactory {
+
+    public CustomFactory() {
+      super("custom");
+    }
+
+    @Override
+    public InputStreamType streamType() {
+      return InputStreamType.Custom;
+    }
+
+    @Override
+    public ObjectInputStream readObject(final ObjectReadParameters parameters) throws IOException {
+      return null;
+    }
+
+    @Override
+    public StreamFactoryRequirements factoryRequirements() {
+      return null;
+    }
+  }
+
+  /**
+   * Factory which raises an exception during construction.
+   */
+  public static final class FactoryFailsToInstantiate extends CustomFactory {
+
+    public FactoryFailsToInstantiate() {
+      throw new UncheckedIOException("failed to instantiate", new IOException());
+    }
+
+  }
+
+  /**
+   * Callbacks from {@link ObjectInputStreamFactory} instances.
+   */
+  private static final class Callbacks implements ObjectInputStreamFactory.StreamFactoryCallbacks {
+
+    @Override
+    public S3AsyncClient getOrCreateAsyncClient(final boolean requireCRT) throws IOException {
+      throw new UnsupportedOperationException("not implemented");
+    }
+  }
+
+}

+ 3 - 4
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java

@@ -40,6 +40,7 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem;
 import org.apache.hadoop.fs.s3a.S3AInputStream;
 import org.apache.hadoop.fs.s3a.S3ATestUtils;
 import org.apache.hadoop.fs.s3a.Statistic;
+import org.apache.hadoop.fs.s3a.impl.streams.InputStreamType;
 import org.apache.hadoop.fs.statistics.IOStatistics;
 
 import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FOOTER_CACHE;
@@ -52,12 +53,11 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.readStream;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.skip;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.writeTextFile;
 import static org.apache.hadoop.fs.s3a.Constants.CHECKSUM_VALIDATION;
-import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_DEFAULT;
-import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.assertStreamIsNotChecksummed;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.getS3AInputStream;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.streamType;
 import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_BYTES_READ_CLOSE;
 import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_OPENED;
 import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_SEEK_BYTES_SKIPPED;
@@ -452,8 +452,7 @@ public class ITestS3AOpenCost extends AbstractS3ACostTest {
    * @return true if the fs has prefetching enabled.
    */
   private boolean prefetching()  {
-    return getFileSystem().getConf().getBoolean(
-        PREFETCH_ENABLED_KEY, PREFETCH_ENABLED_DEFAULT);
+    return InputStreamType.Prefetch == streamType(getFileSystem());
   }
 
   /**

+ 2 - 3
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestUnbufferDraining.java

@@ -48,11 +48,11 @@ import static org.apache.hadoop.fs.s3a.Constants.ESTABLISH_TIMEOUT;
 import static org.apache.hadoop.fs.s3a.Constants.INPUT_FADVISE;
 import static org.apache.hadoop.fs.s3a.Constants.MAXIMUM_CONNECTIONS;
 import static org.apache.hadoop.fs.s3a.Constants.MAX_ERROR_RETRIES;
-import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY;
 import static org.apache.hadoop.fs.s3a.Constants.READAHEAD_RANGE;
 import static org.apache.hadoop.fs.s3a.Constants.REQUEST_TIMEOUT;
 import static org.apache.hadoop.fs.s3a.Constants.RETRY_LIMIT;
 import static org.apache.hadoop.fs.s3a.Constants.SOCKET_TIMEOUT;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.disablePrefetching;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
 import static org.apache.hadoop.fs.s3a.impl.ConfigurationHelper.setDurationAsSeconds;
 import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue;
@@ -98,7 +98,7 @@ public class ITestUnbufferDraining extends AbstractS3ACostTest {
 
   @Override
   public Configuration createConfiguration() {
-    Configuration conf = super.createConfiguration();
+    Configuration conf = disablePrefetching(super.createConfiguration());
     removeBaseAndBucketOverrides(conf,
         ASYNC_DRAIN_THRESHOLD,
         CHECKSUM_VALIDATION,
@@ -106,7 +106,6 @@ public class ITestUnbufferDraining extends AbstractS3ACostTest {
         INPUT_FADVISE,
         MAX_ERROR_RETRIES,
         MAXIMUM_CONNECTIONS,
-        PREFETCH_ENABLED_KEY,
         READAHEAD_RANGE,
         REQUEST_TIMEOUT,
         RETRY_LIMIT,

+ 4 - 4
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/MockS3ARemoteObject.java

@@ -29,8 +29,8 @@ import software.amazon.awssdk.services.s3.model.GetObjectRequest;
 import software.amazon.awssdk.services.s3.model.GetObjectResponse;
 
 import org.apache.hadoop.fs.impl.prefetch.Validate;
-import org.apache.hadoop.fs.s3a.S3AInputStream;
 import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext;
+import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamCallbacks;
 import org.apache.hadoop.util.functional.CallableRaisingIOE;
 
 /**
@@ -54,7 +54,7 @@ class MockS3ARemoteObject extends S3ARemoteObject {
 
   MockS3ARemoteObject(int size, boolean throwExceptionOnOpen) {
     super(
-        S3APrefetchFakes.createReadContext(null, KEY, size, 1, 1),
+        S3APrefetchFakes.createReadContext(null, KEY, size),
         S3APrefetchFakes.createObjectAttributes(BUCKET, KEY, size),
         S3APrefetchFakes.createInputStreamCallbacks(BUCKET),
         EmptyS3AStatisticsContext.EMPTY_INPUT_STREAM_STATISTICS,
@@ -95,8 +95,8 @@ class MockS3ARemoteObject extends S3ARemoteObject {
     return (byte) (offset % 128);
   }
 
-  public static S3AInputStream.InputStreamCallbacks createClient(String bucketName) {
-    return new S3AInputStream.InputStreamCallbacks() {
+  public static ObjectInputStreamCallbacks createClient(String bucketName) {
+    return new ObjectInputStreamCallbacks() {
       @Override
       public ResponseInputStream<GetObjectResponse> getObject(
           GetObjectRequest request) {

+ 23 - 20
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchFakes.java

@@ -52,7 +52,6 @@ import org.apache.hadoop.fs.s3a.Invoker;
 import org.apache.hadoop.fs.s3a.S3AEncryptionMethods;
 import org.apache.hadoop.fs.s3a.S3AFileStatus;
 import org.apache.hadoop.fs.s3a.S3AInputPolicy;
-import org.apache.hadoop.fs.s3a.S3AInputStream;
 import org.apache.hadoop.fs.s3a.S3AReadOpContext;
 import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
 import org.apache.hadoop.fs.s3a.VectoredIOContext;
@@ -62,6 +61,7 @@ import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
 import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
 import org.apache.hadoop.fs.s3a.statistics.impl.CountingChangeTracker;
 import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext;
+import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamCallbacks;
 import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
 import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.io.retry.RetryPolicy;
@@ -133,9 +133,7 @@ public final class S3APrefetchFakes {
   public static S3AReadOpContext createReadContext(
       ExecutorServiceFuturePool futurePool,
       String key,
-      int fileSize,
-      int prefetchBlockSize,
-      int prefetchBlockCount) {
+      int fileSize) {
 
     S3AFileStatus fileStatus = createFileStatus(key, fileSize);
     org.apache.hadoop.fs.Path path = new org.apache.hadoop.fs.Path(key);
@@ -156,9 +154,8 @@ public final class S3APrefetchFakes {
             .setMaxReadSizeForVectoredReads(1)
             .build(),
         emptyStatisticsStore(),
-        futurePool,
-        prefetchBlockSize,
-        prefetchBlockCount)
+        futurePool
+    )
         .withChangeDetectionPolicy(
             ChangeDetectionPolicy.createPolicy(ChangeDetectionPolicy.Mode.None,
                 ChangeDetectionPolicy.Source.ETag, false))
@@ -187,7 +184,7 @@ public final class S3APrefetchFakes {
         AbortableInputStream.create(new ByteArrayInputStream(buffer), () -> {}));
   }
 
-  public static S3AInputStream.InputStreamCallbacks createInputStreamCallbacks(
+  public static ObjectInputStreamCallbacks createInputStreamCallbacks(
       String bucket) {
 
     GetObjectResponse objectResponse = GetObjectResponse.builder()
@@ -197,7 +194,7 @@ public final class S3APrefetchFakes {
     ResponseInputStream<GetObjectResponse> responseInputStream =
         createS3ObjectInputStream(objectResponse, new byte[8]);
 
-    return new S3AInputStream.InputStreamCallbacks() {
+    return new ObjectInputStreamCallbacks() {
       @Override
       public ResponseInputStream<GetObjectResponse> getObject(GetObjectRequest request) {
         return responseInputStream;
@@ -234,21 +231,25 @@ public final class S3APrefetchFakes {
     S3AReadOpContext s3AReadOpContext = createReadContext(
         futurePool,
         key,
-        fileSize,
-        prefetchBlockSize,
-        prefetchBlockCount);
+        fileSize
+    );
 
-    S3AInputStream.InputStreamCallbacks callbacks =
+    ObjectInputStreamCallbacks callbacks =
         createInputStreamCallbacks(bucket);
     S3AInputStreamStatistics stats =
         s3AReadOpContext.getS3AStatisticsContext().newInputStreamStatistics();
 
+    final PrefetchOptions options =
+        new PrefetchOptions(prefetchBlockSize, prefetchBlockCount);
     if (clazz == FakeS3AInMemoryInputStream.class) {
-      return new FakeS3AInMemoryInputStream(s3AReadOpContext,
+      return new FakeS3AInMemoryInputStream(s3AReadOpContext, options,
           s3ObjectAttributes, callbacks, stats);
     } else if (clazz == FakeS3ACachingInputStream.class) {
-      return new FakeS3ACachingInputStream(s3AReadOpContext, s3ObjectAttributes,
-          callbacks, stats);
+      return new FakeS3ACachingInputStream(s3AReadOpContext,
+          options,
+          s3ObjectAttributes,
+          callbacks,
+          stats);
     }
 
     throw new RuntimeException("Unsupported class: " + clazz);
@@ -288,10 +289,11 @@ public final class S3APrefetchFakes {
 
     public FakeS3AInMemoryInputStream(
         S3AReadOpContext context,
+        PrefetchOptions prefetchOptions,
         S3ObjectAttributes s3Attributes,
-        S3AInputStream.InputStreamCallbacks client,
+        ObjectInputStreamCallbacks client,
         S3AInputStreamStatistics streamStatistics) {
-      super(context, s3Attributes, client, streamStatistics);
+      super(context, prefetchOptions, s3Attributes, client, streamStatistics);
     }
 
     @Override
@@ -390,10 +392,11 @@ public final class S3APrefetchFakes {
 
     public FakeS3ACachingInputStream(
         S3AReadOpContext context,
+        PrefetchOptions prefetchOptions,
         S3ObjectAttributes s3Attributes,
-        S3AInputStream.InputStreamCallbacks client,
+        ObjectInputStreamCallbacks client,
         S3AInputStreamStatistics streamStatistics) {
-      super(context, s3Attributes, client, streamStatistics, CONF,
+      super(context, prefetchOptions, s3Attributes, client, streamStatistics, CONF,
           new LocalDirAllocator(
               CONF.get(BUFFER_DIR) != null ? BUFFER_DIR : HADOOP_TMP_DIR));
     }

+ 11 - 8
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteInputStream.java

@@ -31,11 +31,11 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSExceptionMessages;
 import org.apache.hadoop.fs.impl.prefetch.ExceptionAsserts;
 import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool;
-import org.apache.hadoop.fs.s3a.S3AInputStream;
 import org.apache.hadoop.fs.s3a.S3AReadOpContext;
 import org.apache.hadoop.fs.s3a.S3ATestUtils;
 import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
 import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
+import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamCallbacks;
 import org.apache.hadoop.test.AbstractHadoopTestBase;
 
 import static org.assertj.core.api.Assertions.assertThat;
@@ -53,13 +53,14 @@ public class TestS3ARemoteInputStream extends AbstractHadoopTestBase {
   private final ExecutorServiceFuturePool futurePool =
       new ExecutorServiceFuturePool(threadPool);
 
-  private final S3AInputStream.InputStreamCallbacks client =
+  private final ObjectInputStreamCallbacks client =
       MockS3ARemoteObject.createClient("bucket");
 
   @Test
   public void testArgChecks() throws Exception {
     S3AReadOpContext readContext =
-        S3APrefetchFakes.createReadContext(futurePool, "key", 10, 10, 1);
+        S3APrefetchFakes.createReadContext(futurePool, "key", 10);
+    PrefetchOptions prefetchOptions = new PrefetchOptions(10, 1);
     S3ObjectAttributes attrs =
         S3APrefetchFakes.createObjectAttributes("bucket", "key", 10);
     S3AInputStreamStatistics stats =
@@ -67,23 +68,25 @@ public class TestS3ARemoteInputStream extends AbstractHadoopTestBase {
 
     Configuration conf = S3ATestUtils.prepareTestConfiguration(new Configuration());
     // Should not throw.
-    new S3ACachingInputStream(readContext, attrs, client, stats, conf, null);
+    new S3ACachingInputStream(readContext, prefetchOptions, attrs, client, stats, conf, null);
 
     ExceptionAsserts.assertThrows(
         NullPointerException.class,
-        () -> new S3ACachingInputStream(null, attrs, client, stats, conf, null));
+        () -> new S3ACachingInputStream(null, null, attrs, client, stats, conf, null));
 
     ExceptionAsserts.assertThrows(
         NullPointerException.class,
-        () -> new S3ACachingInputStream(readContext, null, client, stats, conf, null));
+        () -> new S3ACachingInputStream(readContext, null, null, client, stats, conf, null));
 
     ExceptionAsserts.assertThrows(
         NullPointerException.class,
-        () -> new S3ACachingInputStream(readContext, attrs, null, stats, conf, null));
+        () -> new S3ACachingInputStream(readContext, prefetchOptions, attrs, null, stats, conf,
+            null));
 
     ExceptionAsserts.assertThrows(
         NullPointerException.class,
-        () -> new S3ACachingInputStream(readContext, attrs, client, null, conf, null));
+        () -> new S3ACachingInputStream(readContext, prefetchOptions, attrs, client, null, conf,
+            null));
   }
 
   @Test

+ 3 - 3
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteObject.java

@@ -26,11 +26,11 @@ import org.junit.Test;
 
 import org.apache.hadoop.fs.impl.prefetch.ExceptionAsserts;
 import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool;
-import org.apache.hadoop.fs.s3a.S3AInputStream;
 import org.apache.hadoop.fs.s3a.S3AReadOpContext;
 import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
 import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
 import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
+import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamCallbacks;
 import org.apache.hadoop.test.AbstractHadoopTestBase;
 
 public class TestS3ARemoteObject extends AbstractHadoopTestBase {
@@ -40,13 +40,13 @@ public class TestS3ARemoteObject extends AbstractHadoopTestBase {
   private final ExecutorServiceFuturePool futurePool =
       new ExecutorServiceFuturePool(threadPool);
 
-  private final S3AInputStream.InputStreamCallbacks client =
+  private final ObjectInputStreamCallbacks client =
       MockS3ARemoteObject.createClient("bucket");
 
   @Test
   public void testArgChecks() throws Exception {
     S3AReadOpContext readContext =
-        S3APrefetchFakes.createReadContext(futurePool, "key", 10, 10, 1);
+        S3APrefetchFakes.createReadContext(futurePool, "key", 10);
     S3ObjectAttributes attrs =
         S3APrefetchFakes.createObjectAttributes("bucket", "key", 10);
     S3AInputStreamStatistics stats =

+ 17 - 30
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java

@@ -58,6 +58,7 @@ import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_RE
 import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
 import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.disablePrefetching;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.getInputStreamStatistics;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.getS3AInputStream;
 import static org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils.isUsingDefaultExternalDataFile;
@@ -72,6 +73,7 @@ import static org.apache.hadoop.fs.statistics.StoreStatisticNames.ACTION_HTTP_GE
 import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MAX;
 import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MEAN;
 import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MIN;
+import static org.apache.hadoop.io.Sizes.*;
 import static org.apache.hadoop.util.functional.FutureIO.awaitFuture;
 
 /**
@@ -80,15 +82,15 @@ import static org.apache.hadoop.util.functional.FutureIO.awaitFuture;
 public class ITestS3AInputStreamPerformance extends S3AScaleTestBase {
   private static final Logger LOG = LoggerFactory.getLogger(
       ITestS3AInputStreamPerformance.class);
-  private static final int READAHEAD_128K = 128 * _1KB;
+  private static final int READAHEAD_128K = S_128K;
 
   private S3AFileSystem s3aFS;
   private Path testData;
   private FileStatus testDataStatus;
   private FSDataInputStream in;
   private S3AInputStreamStatistics streamStatistics;
-  public static final int BLOCK_SIZE = 32 * 1024;
-  public static final int BIG_BLOCK_SIZE = 256 * 1024;
+  public static final int BLOCK_SIZE = S_32K;
+  public static final int BIG_BLOCK_SIZE = S_256K;
 
   private static final IOStatisticsSnapshot IOSTATS = snapshotIOStatistics();
 
@@ -99,15 +101,12 @@ public class ITestS3AInputStreamPerformance extends S3AScaleTestBase {
 
   @Override
   protected Configuration createScaleConfiguration() {
-    Configuration conf = super.createScaleConfiguration();
-    S3ATestUtils.removeBaseAndBucketOverrides(conf,
-        PREFETCH_ENABLED_KEY);
+    Configuration conf = disablePrefetching(super.createScaleConfiguration());
     if (isUsingDefaultExternalDataFile(conf)) {
       S3ATestUtils.removeBaseAndBucketOverrides(
           conf,
           ENDPOINT);
     }
-    conf.setBoolean(PREFETCH_ENABLED_KEY, false);
     return conf;
   }
 
@@ -118,8 +117,8 @@ public class ITestS3AInputStreamPerformance extends S3AScaleTestBase {
   @Before
   public void openFS() throws IOException {
     Configuration conf = getConf();
-    conf.setInt(SOCKET_SEND_BUFFER, 16 * 1024);
-    conf.setInt(SOCKET_RECV_BUFFER, 16 * 1024);
+    conf.setInt(SOCKET_SEND_BUFFER, S_16K);
+    conf.setInt(SOCKET_RECV_BUFFER, S_16K);
     // look up the test file, no requirement to be set.
     String testFile =  conf.getTrimmed(KEY_CSVTEST_FILE,
         PublicDatasetTestUtils.DEFAULT_EXTERNAL_FILE);
@@ -283,7 +282,7 @@ public class ITestS3AInputStreamPerformance extends S3AScaleTestBase {
     // implicitly rounding down here
     long blockCount = len / blockSize;
     long totalToRead = blockCount * blockSize;
-    long minimumBandwidth = 128 * 1024;
+    long minimumBandwidth = S_128K;
     int maxResetCount = 4;
     int resetCount = 0;
     for (long i = 0; i < blockCount; i++) {
@@ -473,22 +472,10 @@ public class ITestS3AInputStreamPerformance extends S3AScaleTestBase {
     logStreamStatistics();
   }
 
-  public static final int _4K = 4 * 1024;
-  public static final int _8K = 8 * 1024;
-  public static final int _16K = 16 * 1024;
-  public static final int _32K = 32 * 1024;
-  public static final int _64K = 64 * 1024;
-  public static final int _128K = 128 * 1024;
-  public static final int _256K = 256 * 1024;
-  public static final int _1MB = 1024 * 1024;
-  public static final int _2MB = 2 * _1MB;
-  public static final int _10MB = _1MB * 10;
-  public static final int _5MB = _1MB * 5;
-
   private static final int[][] RANDOM_IO_SEQUENCE = {
-      {_2MB, _128K},
-      {_128K, _128K},
-      {_5MB, _64K},
+      {S_2M, S_128K},
+      {S_128K, S_128K},
+      {S_5M, S_64K},
       {_1MB, _1MB},
   };
 
@@ -536,7 +523,7 @@ public class ITestS3AInputStreamPerformance extends S3AScaleTestBase {
       long expectedOpenCount)
       throws IOException {
     describe("Random IO with policy \"%s\"", policy);
-    byte[] buffer = new byte[_1MB];
+    byte[] buffer = new byte[S_1M];
     long totalBytesRead = 0;
     final long len = testDataStatus.getLen();
     in = openTestFile(policy, 0);
@@ -588,15 +575,15 @@ public class ITestS3AInputStreamPerformance extends S3AScaleTestBase {
   public void testRandomReadOverBuffer() throws Throwable {
     describe("read over a buffer, making sure that the requests" +
         " spans readahead ranges");
-    int datasetLen = _32K;
+    int datasetLen = S_32K;
     S3AFileSystem fs = getFileSystem();
     Path dataFile = path("testReadOverBuffer.bin");
     byte[] sourceData = dataset(datasetLen, 0, 64);
     // relies on the field 'fs' referring to the R/W FS
-    writeDataset(fs, dataFile, sourceData, datasetLen, _16K, true);
+    writeDataset(fs, dataFile, sourceData, datasetLen, S_16K, true);
     byte[] buffer = new byte[datasetLen];
-    int readahead = _8K;
-    int halfReadahead = _4K;
+    int readahead = S_8K;
+    int halfReadahead = S_4K;
     in = openDataFile(fs, dataFile, S3AInputPolicy.Random, readahead, datasetLen);
 
     LOG.info("Starting initial reads");

+ 4 - 0
hadoop-tools/hadoop-aws/src/test/resources/log4j.properties

@@ -98,3 +98,7 @@ log4j.logger.org.apache.hadoop.fs.s3a.S3AStorageStatistics=INFO
 # uncomment this to get S3 Delete requests to return the list of deleted objects
 # log4.logger.org.apache.hadoop.fs.s3a.impl.RequestFactoryImpl=TRACE
 
+# debug service lifecycle of components such as S3AStore and
+# services it launches itself.
+# log4.logger.org.apache.hadoop.service=DEBUG
+