[ 
https://issues.apache.org/jira/browse/HADOOP-19354?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17916016#comment-17916016
 ] 

ASF GitHub Bot commented on HADOOP-19354:
-----------------------------------------

ahmarsuhail commented on code in PR #7214:
URL: https://github.com/apache/hadoop/pull/7214#discussion_r1923466665


##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/StreamIntegration.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.impl.streams;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.store.LogExactlyOnce;
+
+import static org.apache.hadoop.fs.s3a.Constants.INPUT_STREAM_TYPE;
+import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY;
+
+/**
+ * Stream integration, including factory construction.
+ */
+public final class StreamIntegration {
+
+  private static final Logger LOG_DEPRECATION =
+      LoggerFactory.getLogger(
+          "org.apache.hadoop.conf.Configuration.deprecation");
+
+  /**
+   * Warn once on use of prefetch boolean flag rather than enum.
+   */
+  private static final LogExactlyOnce WARN_PREFETCH_KEY = new 
LogExactlyOnce(LOG_DEPRECATION);
+
+  /**
+   * Create the input stream factory the configuration asks for.
+   * This does not initialize the factory.
+   * @param conf configuration
+   * @return a stream factory.
+   */
+  public static ObjectInputStreamFactory createStreamFactory(final 
Configuration conf) {
+    // choose the default input stream type
+    InputStreamType defaultStream = InputStreamType.DEFAULT_STREAM_TYPE;
+    if (conf.getBoolean(PREFETCH_ENABLED_KEY, false)) {

Review Comment:
   We're saying `PREFETCH_ENABLED_KEY` deprecated, but still setting the stream 
type to prefetch. Is this something we want? If yes, we should make the message 
clearer to say "we're going to deprecate this in the future, but it works for 
now"



##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java:
##########
@@ -1877,100 +1859,43 @@ private FSDataInputStream executeOpen(
     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 StreamThreadOptions requirements =
+        getStore().threadRequirements();
+
+    // calculate the permit count.
+    final int permitCount = requirements.streamThreads() +
+        (requirements.vectorSupported()
+            ? vectoredActiveRangeReads
+            : 0);
+    // create an executor which is a subset of the
+    // bounded thread pool.
+    final SemaphoredDelegatingExecutor pool = new SemaphoredDelegatingExecutor(

Review Comment:
   Just a clarifying question, what is the benefit of creating a new 
`SemaphoredDelegatingExecutor` per stream vs just creating this once? 



##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/InputStreamType.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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;
+
+/**
+ * Enum of input stream types.
+ * Each enum value contains the factory function actually used to create
+ * the factory.
+ */
+public enum InputStreamType {

Review Comment:
   As discussed in https://github.com/apache/hadoop/pull/7295/, the 
S3SeekableInputStreamFactory requires a client to be passed in. For this, we 
need a way to pass in the ClientManager here.





> S3A: InputStreams to be created by factory under S3AStore
> ---------------------------------------------------------
>
>                 Key: HADOOP-19354
>                 URL: https://issues.apache.org/jira/browse/HADOOP-19354
>             Project: Hadoop Common
>          Issue Type: Sub-task
>          Components: fs/s3
>    Affects Versions: 3.4.2
>            Reporter: Steve Loughran
>            Assignee: Steve Loughran
>            Priority: Major
>              Labels: pull-request-available
>
> Migrate S3AInputStream creation into a factory pattern, push down into 
> S3AStore.
> Proposed factories
> * default: whatever this release has as default
> * classic: current S3AInputStream
> * prefetch: prefetching
> * analytics: new analytics stream
> * other: reads a classname from another prop, instantiates.
> Also proposed
> * stream to implement some stream capability to declare what they are 
> (classic, prefetch, analytics, other). 
> h2. Implementation
> All callbacks used by the stream also to call directly onto S3AStore.
> S3AFileSystem must not be invoked at all (if it is needed: PR is still not 
> ready).
> Some interface from Instrumentation will be passed to factory; this shall 
> include a way to create new per-stream 
> The factory shall implement org.apache.hadoop.service.Service; S3AStore shall 
> do same and become a subclass of CompositeService. It shall attach the 
> factory as a child, so they can follow the same lifecycle. We shall do the 
> same for anything else that gets pushed down.
> Everything related to stream creation must go from s3afs; and creation of the 
> factory itself. This must be done in S3AStore.initialize(). 
> As usual, this will complicate mocking. But the streams themselves should not 
> require changes, at least significant ones.
> Testing.
> * The huge file tests should be tuned so each of the different ones uses a 
> different stream, always.
> * use a -Dstream="factory name" to choose factory, rather than the -Dprefetch
> * if not set, whatever is in auth-keys gets picked up.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to