You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by "Steve Loughran (JIRA)" <ji...@apache.org> on 2018/11/05 10:34:00 UTC

[jira] [Commented] (HADOOP-15229) Add FileSystem builder-based openFile() API to match createFile()

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

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

Patch 001. 

* pulls up the core of {{FSDataOutputStreamBuilder}} into a class {{AbstractCloseableBuilder}} which is a builder for anything implementing the {{Closeable}} interface. this can be the base class for any FS builder we add in future 

* it has the opt/must operations so guarantees consistent semantics everywhere.
* takes path and pathhandle as optional values, stores both and provides access to the optional and resolved getPath/getPathHandle values.

Adds a subclass {{InputStreamBuilder}} which can be used for anything creating an input stream; adds FS/FC binding.

Adds a protected subclass of that {{org.apache.hadoop.fs.FileSystem.FSDataInputStreamBuilder}}. which bonds to the openFile(path|pathHandle) methods of FileSystem.

Adds {{FileSystem.FSDataInputStreamBuilder openFile}} commands for path & path handle

This is the basic semantics I want

Two things to validate it beyond open
* add the ability to set filesystem seek options
* how well it works for a specific Select API call for S3a

I Think for FS seek it'd be good to have a new protected open(path, Set<String> mandatory, Configuration conf) method for FS & FC, called by {{FSDataInputStreamBuilder}} for passing in configurations. The base one would reject any mandatory option; subclasses can support fs:fadvise and anything else we agree on; object stores their own.








> 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
>            Priority: Major
>         Attachments: HADOOP-15229-001.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-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org