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

Steve Loughran commented on HADOOP-15229:
-----------------------------------------

Feedback from Miklos Gergely

bq. we don’t directly open a file with FileSystem.open(), but using 
InputFormat.getRecordReader(), which returns some RecordReader, and that is 
what is opening the file with FileSystem.open().

bq.I suggest the following modification:

bq. 1. create an interface which has a function for altering a 
FSDataInputStreamBuilder, e.g. FSDataInputStreamParametrizer { void 
parametrizeFSDataInputStreamBuilder(FSDataInputStreamBuilder builder); }, maybe 
come up with a better name :)

bq. 2 add an optional FSDataInputStreamParametrizer argument to 
InputFormat.getRecordReader, which is also passed to the RecordReader created 
by it. 

bq. 3. classes like LineRecordReader if the extra parametrized is not null then 
they should open the files with fs.openPath(), and before invoking build, send 
the builder to the parametrizer

bq. This way the new feature could be used even if the user doesn’t want to 
open the file directly, but use an InputFormat / RecordReader.

> Add FileSystem builder-based openFile() API to match createFile()
> -----------------------------------------------------------------
>
>                 Key: HADOOP-15229
>                 URL: https://issues.apache.org/jira/browse/HADOOP-15229
>             Project: Hadoop Common
>          Issue Type: New Feature
>          Components: fs, fs/azure, fs/s3
>    Affects Versions: 3.0.0
>            Reporter: Steve Loughran
>            Assignee: Steve Loughran
>            Priority: Major
>         Attachments: HADOOP-15229-001.patch, HADOOP-15229-002.patch, 
> HADOOP-15229-003.patch, HADOOP-15229-004.patch, HADOOP-15229-004.patch, 
> HADOOP-15229-005.patch, HADOOP-15229-006.patch
>
>
> Replicate HDFS-1170 and HADOOP-14365 with an API to open files.
> A key requirement of this is not HDFS, it's to put in the fadvise policy for 
> working with object stores, where getting the decision to do a full GET and 
> TCP abort on seek vs smaller GETs is fundamentally different: the wrong 
> option can cost you minutes. S3A and Azure both have adaptive policies now 
> (first backward seek), but they still don't do it that well.
> Columnar formats (ORC, Parquet) should be able to say "fs.input.fadvise" 
> "random" as an option when they open files; I can imagine other options too.
> The Builder model of [~eddyxu] is the one to mimic, method for method. 
> Ideally with as much code reuse as possible



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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

Reply via email to