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

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

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


##########
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md:
##########
@@ -0,0 +1,122 @@
+<!---
+  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.
+-->
+
+# `FileSystem.openFile()`/`FileContext.openFile()`
+
+This is a method provided by both FileSystem and FileContext for
+advanced file opening options and, where implemented,
+an asynchrounous/lazy opening of a file.
+
+Creates a builder to open a file, supporting options
+both standard and filesystem specific. The return
+value of the `build()` call is a `Future<FSDataInputStream>`,
+which must be waited on. The file opening may be
+asynchronous, and it may actually be postponed (including
+permission/existence checks) until reads are actually
+performed.
+
+This API call was added to `FileSystem` and `FileContext` in
+Hadoop 3.3.0; it was tuned in Hadoop 3.3.1 as follows.
+
+* Added `opt(key, long)` and `must(key, long)`.
+* Declared that `withFileStatus(null)` is allowed.
+* Declared that `withFileStatus(status)` only checks
+  the filename of the path, not the full path.
+  This is needed to support passthrough/mounted filesystems.
+* Added standard option keys.
+
+###  <a name="openfile_path_"></a> `FutureDataInputStreamBuilder openFile(Path 
path)`
+
+Creates a [`FutureDataInputStreamBuilder`](fsdatainputstreambuilder.html)
+to construct a operation to open the file at `path` for reading.
+
+When `build()` is invoked on the returned `FutureDataInputStreamBuilder` 
instance,
+the builder parameters are verified and
+`FileSystem.openFileWithOptions(Path, OpenFileParameters)` or
+`AbstractFileSystem.openFileWithOptions(Path, OpenFileParameters)` invoked.
+
+These protected methods returns a `CompletableFuture<FSDataInputStream>`
+which, when its `get()` method is called, either returns an input
+stream of the contents of opened file, or raises an exception.
+
+The base implementation of the `FileSystem.openFileWithOptions(PathHandle, 
OpenFileParameters)`
+ultimately invokes `FileSystem.open(Path, int)`.
+
+Thus the chain `FileSystem.openFile(path).build().get()` has the same 
preconditions
+and postconditions as `FileSystem.open(Path p, int bufferSize)`
+
+However, there is one difference which implementations are free to
+take advantage of:
+
+The returned stream MAY implement a lazy open where file non-existence or
+access permission failures may not surface until the first `read()` of the
+actual data.
+
+This saves network IO on object stores.
+
+The `openFile()` operation MAY check the state of the filesystem during its
+invocation, but as the state of the filesystem may change betwen this call and

Review Comment:
   fixed now





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

    Worklog Id:     (was: 758588)
    Time Spent: 20h 10m  (was: 20h)

> 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: 20h 10m
>  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.7#820007)

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

Reply via email to