[ 
https://issues.apache.org/jira/browse/HADOOP-16202?focusedWorklogId=755834&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-755834
 ]

ASF GitHub Bot logged work on HADOOP-16202:
-------------------------------------------

                Author: ASF GitHub Bot
            Created on: 12/Apr/22 15:27
            Start Date: 12/Apr/22 15:27
    Worklog Time Spent: 10m 
      Work Description: steveloughran commented on code in PR #2584:
URL: https://github.com/apache/hadoop/pull/2584#discussion_r848572530


##########
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstreambuilder.md:
##########
@@ -65,37 +65,97 @@ If a filesystem implementation extends the `FileStatus` 
returned in its
 implementation MAY use this information when opening the file.
 
 This is relevant with those stores which return version/etag information,
-including the S3A and ABFS connectors -they MAY use this to guarantee that
-the file they opened is exactly the one returned in the listing.
+-they MAY use this to guarantee that the file they opened
+is exactly the one returned in the listing.
+
+
+The final `status.getPath().getName()` element of the supplied status MUST 
equal
+the name value of the path supplied to the  `openFile(path)` call.
+
+Filesystems MUST NOT validate the rest of the path.
+This is needed to support viewfs and other mount-point wrapper filesystems
+where schemas and paths are different. These often create their own FileStatus 
results
+
+Preconditions
+
+```python
+status == null or status.getPath().getName() == path.getName()
+
+```
+
+Filesystems MUST NOT require the class of `status` to equal
+that of any specific subclass their implementation returns in filestatus/list
+operations. This is to support wrapper filesystems and 
serialization/deserialization
+of the status.
+
 
 ### Set optional or mandatory parameters
 
-    FSDataInputStreamBuilder opt(String key, ...)
-    FSDataInputStreamBuilder must(String key, ...)
+    FutureDataInputStreamBuilder opt(String key, ...)
+    FutureDataInputStreamBuilder must(String key, ...)
 
 Set optional or mandatory parameters to the builder. Using `opt()` or `must()`,
 client can specify FS-specific parameters without inspecting the concrete type
 of `FileSystem`.
 
+Example:
+
 ```java
 out = fs.openFile(path)
-    .opt("fs.s3a.experimental.input.fadvise", "random")
-    .must("fs.s3a.readahead.range", 256 * 1024)
+    .must("fs.option.openfile.read.policy", "random")
+    .opt("fs.http.connection.timeout", 30_000L)
     .withFileStatus(statusFromListing)
     .build()
     .get();
 ```
 
+Here the read policy of `random` has been specified,
+with the requirement that the filesystem implementation must understand the 
option.
+An http-specific option has been supplied which may be interpreted by any 
store;
+If the filesystem opening the file does not recognize the option, it can 
safely be
+ignored.
+
+### When to use `opt()` versus `must()`
+
+The difference between `opt()` versus `must()` is how the FileSystem opening
+the file must react to an option which it does not recognize.
+
+* `opt(name, value)`: if `name` is not reco

Review Comment:
   cut the line





Issue Time Tracking
-------------------

    Worklog Id:     (was: 755834)
    Time Spent: 17h 50m  (was: 17h 40m)

> Enhance openFile() for better read performance against object stores 
> ---------------------------------------------------------------------
>
>                 Key: HADOOP-16202
>                 URL: https://issues.apache.org/jira/browse/HADOOP-16202
>             Project: Hadoop Common
>          Issue Type: Bug
>          Components: fs, fs/s3, tools/distcp
>    Affects Versions: 3.3.0
>            Reporter: Steve Loughran
>            Assignee: Steve Loughran
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 17h 50m
>  Remaining Estimate: 0h
>
> The {{openFile()}} builder API lets us add new options when reading a file
> Add an option {{"fs.s3a.open.option.length"}} which takes a long and allows 
> the length of the file to be declared. If set, *no check for the existence of 
> the file is issued when opening the file*
> Also: withFileStatus() to take any FileStatus implementation, rather than 
> only S3AFileStatus -and not check that the path matches the path being 
> opened. Needed to support viewFS-style wrapping and mounting.
> and Adopt where appropriate to stop clusters with S3A reads switched to 
> random IO from killing download/localization
> * fs shell copyToLocal
> * distcp
> * IOUtils.copy



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

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

Reply via email to