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

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

Patch 003

This incorporates HADOOP-15364: S3 select, as a validation of this design 
beyond just "we can do this"

A key change since before is that the return of the openFile().build() call is 
actually a CompletableFuture; you have to call that to get the stream
* Base implementation actually evaluates in the build(), but hides the outcome 
in the future, to line people up for lazy failures.
* FS Specs says "lazy eval", as does the test of when FNFEs surface
* S3A impl submits classic open and new select into its worker pool: the 
GET/POST requests are all asynchronous.
* Tests in AbstractContractOpenTest for the openFile(path) sequence (but not 
PathHandle, yet)

If you can line things up for async code execution then this will hide the 
overhead of initiating an HTTP request, and the even longer delays of the 
select. Otherwise, well, IOExceptions get wrapped more than they need to be. 
The more you do with lambda expressions, the more checked exceptions suck.

In use the API Will look like
{code}
FileSystem.FSDataInputStreamBuilder builder =
    filesystem.openFile("s3a://bucket/path-to-file.csv")
        .must("s3a:fs.s3a.select.sql",
            "SELECT * FROM S3OBJECT s WHERE s.\"odd\" = `TRUE`")
        .must("s3a:fs.s3a.select.compression", "NONE")
        .must("s3a:fs.s3a.select.csv.header", "use");
CompletableFuture<FSDataInputStream> future = builder.build();
try (FSDataInputStream select = future.get()) {
  // process the output
  stream.read();
}
{code}

> 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
>
>
> 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: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org

Reply via email to