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

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_r848573295


##########
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstreambuilder.md:
##########
@@ -128,26 +193,499 @@ completed, returns an input stream which can read data 
from the filesystem.
 
 The `build()` operation MAY perform the validation of the file's existence,
 its kind, so rejecting attempts to read from a directory or non-existent
-file. **Alternatively**, the `build()` operation may delay all checks
-until an asynchronous operation whose outcome is provided by the `Future`
+file. Alternatively
+* file existence/status checks MAY be performed asynchronously within the 
returned
+    `CompletableFuture<>`.
+* file existence/status checks MAY be postponed until the first byte is read in
+  any of the read such as `read()` or `PositionedRead`.
 
 That is, the precondition  `exists(FS, path)` and `isFile(FS, path)` are
-only guaranteed to have been met after the `get()` on the returned future is 
successful.
+only guaranteed to have been met after the `get()` called on returned future
+and an attempt has been made to read the stream.
 
-Thus, if even a file does not exist, the following call will still succeed, 
returning
-a future to be evaluated.
+Thus, if even when file does not exist, or is a directory rather than a file,
+the following call MUST succeed, returning a `CompletableFuture` to be 
evaluated.
 
 ```java
 Path p = new Path("file://tmp/file-which-does-not-exist");
 
 CompletableFuture<FSDataInputStream> future = p.getFileSystem(conf)
       .openFile(p)
-      .build;
+      .build();
+```
+
+The inability to access/read a file MUST raise an `IOException`or subclass
+in either the future's `get()` call, or, for late binding operations,
+when an operation to read data is invoked.
+
+Therefore the following sequence SHALL fail when invoked on the
+`future` returned by the previous example.
+
+```java
+  future.get().read();
+```
+
+Access permission checks have the same visibility requirements: permission 
failures
+MUST be delayed until the `get()` call and MAY be delayed into subsequent 
operations.
+
+Note: some operations on the input stream, such as `seek()` may not attempt 
any IO
+at all. Such operations MAY NOT raise exceotions when interacting with
+nonexistent/unreadable files.
+
+## <a name="options"></a> Standard `openFile()` options since Hadoop 3.3.3
+
+These are options which `FileSystem` and `FileContext` implementation
+MUST expected to recognise and MAY support by changing the behavior of

Review Comment:
   API is not new; in 3.3.0, and the createFile one has been around for longer. 
all we do here is declare some well known keys





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

    Worklog Id:     (was: 755837)
    Time Spent: 18h  (was: 17h 50m)

> 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: 18h
>  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