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 "Owen O'Malley (JIRA)" <ji...@apache.org> on 2018/12/03 17:45:00 UTC

[jira] [Comment Edited] (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=16707584#comment-16707584 ] 

Owen O'Malley edited comment on HADOOP-15229 at 12/3/18 5:44 PM:
-----------------------------------------------------------------

[~stevel@apache.org] with the problems were hitting, you're probably right that we need something more like HADOOP-11867.

So we could define an interface for the input streams.

{code:java}
package org.apache.hadoop.fs;
public interface InputStreamOptions {
  /**
   * Test whether the given FileSystem supports the given option.
   */
  boolean supportsOption(String name);

  // repeat optional(...) and require(...) for long, double, and boolean values

  /**
   * Set an optional option to the specified value.
   */
  ReaderBuilder optional(String name, String value);

  /**
   * Set a required option to the specified value.
   */
  ReaderBuilder require(String name, String value);
}
{code}

Of course, we'd need default implementations that ignore optional and fail on require.

Next, we make PositionedReadable extend InputStreamOptions, so that with an open stream you can change the options for that stream.

Finally, we extend RecordReader API to also extend InputStreamOptions and pass the options down to the underlying stream.

Thoughts?


was (Author: owen.omalley):
[~stevel@apache.org] with the problems were hitting, you're probably right that we need something more like HADOOP-11867.

So we could define an interface for the input streams.

{code:java}
package org.apache.hadoop.fs;
public interface InputStreamOptions {
  /**
   * Test whether the given FileSystem supports the given option.
   */
  boolean supportsOption(String name);

  // repeat optional(...) and require(...) for long, double, and boolean values

  /**
   * Set an optional option to the specified value.
   */
  ReaderBuilder optional(String name, String value);

  /**
   * Set a required option to the specified value.
   */
  ReaderBuilder require(String name, String value);
}
{code:java}

Of course, we'd need default implementations that ignore optional and fail on require.

Next, we make PositionedReadable extend InputStreamOptions, so that with an open stream you can change the options for that stream.

Finally, we extend RecordReader API to also extend InputStreamOptions and pass the options down to the underlying stream.

Thoughts?

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