Skip to content

HADOOP-16202. Enhance openFile() for better read performance against object stores #2584

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Conversation

steveloughran
Copy link
Contributor

@steveloughran steveloughran commented Jan 4, 2021

This PR

  • declares a standard set of openFile() parameters for setting length, seek/read policy
    and split start/end.
  • supports all of these in s3a
  • as well as the threshold to trigger an async drain of s3a stream in a seek,
    fs.s3a.input.async.drain.threshold
  • documents all of this
  • fixes up all places in the code which reads a whole file (fs shell, distcp etc)
    to set its read policy to be whole-file.

as a result of this, on the s3a client you can open files without needing
a head request, or even a filestatus, just the length.
and if, in a cluster, you set the default read policy to be random,
shell and distcp read performance doesn't collapse.

@steveloughran
Copy link
Contributor Author

MR client not compiling; not seeing useful information from yetus.

@steveloughran steveloughran force-pushed the s3/HADOOP-16202-enhance-openfile branch from 3b3a4f8 to 41aa610 Compare January 8, 2021 16:30
@steveloughran
Copy link
Contributor Author

style

./hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FutureIO.java:210:    FSBuilder<T, U> propagateOptions(: 'FSBuilder' has incorrect indentation level 4, expected level should be 6. [Indentation]
./hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java:182:import static org.apache.hadoop.fs.impl.AbstractFSBuilderImpl.rejectUnknownMandatoryKeys;:15: Unused import - org.apache.hadoop.fs.impl.AbstractFSBuilderImpl.rejectUnknownMandatoryKeys. [UnusedImports]
./hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileOperation.java:220:        fileStatus = createStatus(path, fileLength, blockSize);: 'if' child has incorrect indentation level 8, expected level should be 6. [Indentation]
./hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileOperation.java:221:      }: 'if rcurly' has incorrect indentation level 6, expected level should be 4. [Indentation]
./hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileOperation.java:323:    private OpenFileInformation(:13: More than 7 parameters (found 10). [ParameterNumber]
./hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestS3AOpenFileOperation.java:119:    assertOpenFile(: 'assertOpenFile' has incorrect indentation level 4, expected level should be 6. [Indentation]

javadocs

[ERROR]                      ^
[ERROR] /home/jenkins/jenkins-home/workspace/hadoop-multibranch_PR-2584/src/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java:126: error: bad use of '>'
[ERROR]    *   fs.example.s3a.option => s3a:option
[ERROR]                               ^
[ERROR] /home/jenkins/jenkins-home/workspace/hadoop-multibranch_PR-2584/src/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java:127: error: bad use of '>'
[ERROR]    *   fs.example.fs.io.policy => s3a.io.policy
[ERROR]                                 ^
[ERROR] /home/jenkins/jenkins-home/workspace/hadoop-multibranch_PR-2584/src/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java:128: error: bad use of '>'
[ERROR]    *   fs.example.something => something
[ERROR]                              ^
[ERROR] /home/jenkins/jenkins-home/workspace/hadoop-multibranch_PR-2584/src/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java:154: error: bad use of '>'
[ERROR]    *   fs.example.s3a.option => s3a:option
[ERROR]                               ^
[ERROR] /home/jenkins/jenkins-home/workspace/hadoop-multibranch_PR-2584/src/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java:155: error: bad use of '>'
[ERROR]    *   fs.example.fs.io.policy => s3a.io.policy
[ERROR]                                 ^
[ERROR] /home/jenkins/jenkins-home/workspace/hadoop-multibranch_PR-2584/src/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java:156: error: bad use of '>'
[ERROR]    *   fs.example.something => something
[ERROR]                              ^
[ERROR] /home/jenkins/jenkins-home/workspace/hadoop-multibranch_PR-2584/src/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FutureIO.java:197: error: bad use of '>'
[ERROR]    *   fs.example.s3a.option => s3a:option
[ERROR]                               ^
[ERROR] /home/jenkins/jenkins-home/workspace/hadoop-multibranch_PR-2584/src/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FutureIO.java:198: error: bad use of '>'
[ERROR]    *   fs.example.fs.io.policy => s3a.io.policy
[ERROR]                                 ^
[ERROR] /home/jenkins/jenkins-home/workspace/hadoop-multibranch_PR-2584/src/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FutureIO.java:199: error: bad use of '>'
[ERROR]    *   fs.example.something => something
[ERROR]                              ^
[ERROR] /home/jenkins/jenkins-home/workspace/hadoop-multibranch_PR-2584/src/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FutureIO.java:227: error: bad use of '>'
[ERROR]    *   fs.example.s3a.option => s3a:option
[ERROR]                               ^
[ERROR] /home/jenkins/jenkins-home/workspace/hadoop-multibranch_PR-2584/src/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FutureIO.java:228: error: bad use of '>'
[ERROR]    *   fs.example.fs.io.policy => s3a.io.policy
[ERROR]                                 ^
[ERROR] /home/jenkins/jenkins-home/workspace/hadoop-multibranch_PR-2584/src/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FutureIO.java:229: error: bad use of '>'
[ERROR]    *   fs.example.something => something

@steveloughran
Copy link
Contributor Author

steveloughran commented Jan 13, 2021

I'm thinking we should be more ambitious in read policy than just "fadvise", because we can then use it as a declaration for the input streams to tune all their params, eg. buffer sizing, whether to do async prefetch.

Then we could allow stores to support not-just seek policies, but declare what you were planning to read, e.g. "parquet-bytebuffer", to mean "I'm reading parquet files through the bytebuffer positioned read API"

openFile("s3a://datasets/set1/input.parquet).
  opt("fs.openfile.policy, "parquet-vectored, parquet,random")
 .build().get()

example opt(fs.openfile.read.policy, "parquet-vectored, parquet, random") to mean "optimise for parquet for vectored IO, then generic vectored IO, then generic random IO". Store implementors would get to make their own decisions as to what to set based on profiling &c. We'd need the applications to set policy on openFile() -so would need to know what names to use. That we can discuss with them, maybe by predefining some options which may be supported

@steveloughran steveloughran force-pushed the s3/HADOOP-16202-enhance-openfile branch from 41aa610 to 0214072 Compare January 13, 2021 18:51
@hadoop-yetus

This comment has been minimized.

@apache apache deleted a comment from hadoop-yetus Jan 15, 2021
@steveloughran
Copy link
Contributor Author

@ThomasMarquardt could you take a look @ this ?

  • I've updated the docs as suggested
  • proposed making the policy broader than just seek policy, so allowing stores to turn on whatever other tuning options they have, especially for file types they've profiled
    The goal there is rather than set cluster wide options which work well for some datatypes but are suboptimal for others, the app provides more information down.

weakness there is that with multiple libraries working with Parquet data (spark, parquet.jar, iceberg, impala) it's not enough to declare the format. You'd really need to declare your app and version

Copy link
Contributor

@ThomasMarquardt ThomasMarquardt left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

My biggest concern, or point of confusion, is the must() vs opt() thing. It feels too complex to me,. Instead, I think we should stick with opt() and allow filesystems to ignore options they don't understand. I guess I'm thinking along the lines that the app passes options as hints to the filesystem in hope that the hints might help, but things will still work if the filesystem does not understand the app's hint.

Comment on lines 887 to +892
parameters.getMandatoryKeys(),
Collections.emptySet(),
FS_OPTION_OPENFILE_STANDARD_OPTIONS,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

With this change, ChecksumFileSystem.openFileWithOptions has the same implementation as the base class, so you can remove this override.

* @see #opt(String, String)
*/
B opt(@Nonnull String key, long value);

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We could use properties (getter and setter) for the standard options. We'd have one for buffer size, one for file length, etc.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I agree, but as well as it already being out there, I want to let applications compile against any version of hadoop with the API, even if a specific FS option isn't available, alongside allowing for custom FS opts. As an example, I have a PoC of parquet lib which uses this which is designed to compile against 3.3.x. (that isn't something I've stuck up as I use it to see how this stuff could be added to a library...highlights what is broken right now, specifically; S3A openFile.withFileStatus fails if used via Hive because hive wraps the FileStatus to a different type from S3AFileStatus.

@@ -4655,7 +4656,7 @@ public FutureDataInputStreamBuilder openFile(PathHandle pathHandle)
final OpenFileParameters parameters) throws IOException {
AbstractFSBuilderImpl.rejectUnknownMandatoryKeys(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This rejectUnknownMandatoryKeys function would not be necessary if the mandatory keys were strongly typed fields.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I agree, but we need flexibility of linking, ability of clients to work with any FS implementation, etc etc

* @param deleteSource delete the source?
* @param overwrite overwrite files at destination?
* @param conf configuration to use when opening files
* @return true if the operation succeeded.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What is the benefit of returning true on success and throwing on a failure, as opposed to returning void and throwing on failure?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

none. its just always been that way. and yes, it is wrong

/**
* Prefix for all standard filesystem options: {@value}.
*/
public static final String FILESYSTEM_OPTION = "fs.option.";
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think you could remove this and define the constants below with a string literal, or at least make it private.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

made it private

try {
// Always do sequential reads.
try (FSDataInputStream in = item.openFile(
FS_OPTION_OPENFILE_READ_POLICY_NORMAL)) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The comment above says do sequential reads, but then the sequential option isn't passed here, but seems like it should be.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fixed

.withFileStatus(statusFromListing)
.build()
.get();
```

Here the seek policy of `random` has been specified,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Now the terminology is "read policy".

relevant for object stores where the probes for existence are expensive, and,
even with an asynchronous open, may be considered needless.

### <a name="openfile(pathhandle)"></a> `FutureDataInputStreamBuilder openFile(PathHandle)`
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Consider combining the docs for openFile(Path) and openFile(PathHandle) to avoid repetition.

Comment on lines 150 to 154
If set as an `opt()` parameter, unsupported "standard" options MUST be ignored,
as MUST unrecognized standard options.

If set as an `must()` parameter, unsupported "standard" options MUST be ignored.
unrecognized standard options MUST be rejected.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I've read this a few times and it is not clear. I think you mean to say:

An option set via opt() that is not supported by the filesystem must be ignored. On the other hand, an option set via must() that is not supported by the filesystem must result in an exception.

Comment on lines 162 to 163
This means that it is not a requirement for the stores to actually read the
the read policy or file length values and use them when opening files.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Now I'm very confused. Sounds like options set via must() can also be ignored, so why do we have opt() and must()?

I think we should keep this simple, and only have opt(). Get rid of must(). If a file system does not support the option, it should be ignored. Nice and simple.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

i'm trying to say that the read and length policies must be understood, but may not be used, if the implementation chooses not to

@steveloughran
Copy link
Contributor Author

  1. we've already shipped that
  2. it's in createFile too

imagine in future you want to do something more than a hint. Case in point: the SQL API in AWS S3. there's no way an app could issue SQL commands which include transforming the response without knowing whether or not they are processed. hence must().

how about i clarify this in the docs and recommend use of opt() except in such special cases?

@ThomasMarquardt
Copy link
Contributor

OK, so must() has already shipped, in that case sure, it should stay. I still have a question. If I pass an option via must(), can it be ignored by the implementation? I read your documentation, specifically, "This means that it is not a requirement for the stores to actually read the the read policy or file length values and use them when opening files." To me, this says that options passed via must() can be ignored. If this is true, then what is the difference between must() and opt()? Is it that must() can result in an unsupported exception, but opt() never results in an unsupported exception?

@steveloughran
Copy link
Contributor Author

correct.
must() == raise an error if not recognised.
opt() entirely optional

note that createFile() has the same api/rules/ common builder codebase. i hope to add some options there for the stores to say "skip all probes for paths existing, being a dir, having a parent..." goal is max performance writes on paths where the writer knows it is building up a dir tree from scratch, or using unique names. in particular: flink/spark checkpoints

@steveloughran
Copy link
Contributor Author

Thomas, to clarify a bit more

yes, an FS can choose to ignore an option, -but it must recognise the option and so make a conscious decision "this doesn't matter"

  • so with the standard length/read policy options, an FS can concude "I see that and don't support it", and just ignore it -a decision made on an option-by-option basis.
  • if something unrecognised comes in as .must() the FS must raise an exception "I don't recognise that".
  • if something it does recognise as a .must() which it knows it doesn't support and which it MUST be able to, then it MUST raise an exception.

Example: vectored IO will only work if the FS supports seek, which ftp doesn't. Assuming we add an option to say "I Want vectored IO" then all filesystems will get this out the box as a well known-and-supported feature, but ftp will need some explicit code to say "fail if vectored IO is requested"

@steveloughran steveloughran added fs fs/s3 changes related to hadoop-aws; submitter must declare test endpoint labels Jan 22, 2021
@steveloughran steveloughran force-pushed the s3/HADOOP-16202-enhance-openfile branch from 26dacf1 to cac2661 Compare February 1, 2021 18:27
@steveloughran
Copy link
Contributor Author

rebased to fix compile problems; the final patch is the one with changes since Thomas's last review

  • move all the text on options into the fsdatainputstreambuilder.md file
  • try to define the validation of must/opt in pseudo-python
  • make sure the doc is consistent in saying FNFE and permission issues MUST be delayed until future.get() and MAY be delayed until the first actual read. This is to give object stores the maximum time for async probes. Example: you could initiate a HEAD request in build() but not block for its completion until read()

S3A added some extra tuning of the input stream/openFile logic, so the stream no longer gets access to the S3 client, simply callbacks

Testing: in progress

@steveloughran steveloughran force-pushed the s3/HADOOP-16202-enhance-openfile branch from cac2661 to 7e105dc Compare February 15, 2021 10:33
@steveloughran steveloughran force-pushed the s3/HADOOP-16202-enhance-openfile branch from 7e105dc to f5013f9 Compare March 17, 2021 21:08
@steveloughran steveloughran force-pushed the s3/HADOOP-16202-enhance-openfile branch 3 times, most recently from 340be22 to 700f998 Compare April 26, 2021 09:58
@apache apache deleted a comment from hadoop-yetus Jul 12, 2021
@apache apache deleted a comment from hadoop-yetus Jul 12, 2021
@steveloughran steveloughran force-pushed the s3/HADOOP-16202-enhance-openfile branch from 700f998 to d28fa9d Compare August 26, 2021 19:42
@apache apache deleted a comment from hadoop-yetus Aug 27, 2021
@steveloughran steveloughran force-pushed the s3/HADOOP-16202-enhance-openfile branch from 103472c to c26e133 Compare September 6, 2021 15:20
@steveloughran steveloughran force-pushed the s3/HADOOP-16202-enhance-openfile branch from c26e133 to fe4fa5c Compare November 24, 2021 17:02
Copy link
Contributor

@mukund-thakur mukund-thakur left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM +1
Ran aws tests as well didn't saw any new failure.

asfgit pushed a commit that referenced this pull request Apr 24, 2022
This defines standard option and values for the
openFile() builder API for opening a file:

fs.option.openfile.read.policy
 A list of the desired read policy, in preferred order.
 standard values are
 adaptive, default, random, sequential, vector, whole-file

fs.option.openfile.length
 How long the file is.

fs.option.openfile.split.start
 start of a task's split

fs.option.openfile.split.end
 end of a task's split

These can be used by filesystem connectors to optimize their
reading of the source file, including but not limited to
* skipping existence/length probes when opening a file
* choosing a policy for prefetching/caching data

The hadoop shell commands which read files all declare "whole-file"
and "sequential", as appropriate.

Contributed by Steve Loughran.

Change-Id: Ia290f79ea7973ce8713d4f90f1315b24d7a23da1
asfgit pushed a commit that referenced this pull request Apr 24, 2022
These changes ensure that sequential files are opened with the
right read policy, and split start/end is passed in.

As well as offering opportunities for filesystem clients to
choose fetch/cache/seek policies, the settings ensure that
processing text files on an s3 bucket where the default policy
is "random" will still be processed efficiently.

This commit depends on the associated hadoop-common patch,
which must be committed first.

Contributed by Steve Loughran.

Change-Id: Ic6713fd752441cf42ebe8739d05c2293a5db9f94
asfgit pushed a commit that referenced this pull request Apr 24, 2022
S3A input stream support for the few fs.option.openfile settings.
As well as supporting the read policy option and values,
if the file length is declared in fs.option.openfile.length
then no HEAD request will be issued when opening a file.
This can cut a few tens of milliseconds off the operation.

The patch adds a new openfile parameter/FS configuration option
fs.s3a.input.async.drain.threshold (default: 16000).
It declares the number of bytes remaining in the http input stream
above which any operation to read and discard the rest of the stream,
"draining", is executed asynchronously.
This asynchronous draining offers some performance benefit on seek-heavy
file IO.

Contributed by Steve Loughran.

Change-Id: I9b0626bbe635e9fd97ac0f463f5e7167e0111e39
asfgit pushed a commit that referenced this pull request Apr 24, 2022
Stops the abfs connector warning if openFile().withFileStatus()
is invoked with a FileStatus is not an abfs VersionedFileStatus.

Contributed by Steve Loughran.

Change-Id: I85076b365eb30aaef2ed35139fa8714efd4d048e
steveloughran added a commit to steveloughran/hadoop that referenced this pull request Apr 27, 2022
…1)

This defines standard option and values for the
openFile() builder API for opening a file:

fs.option.openfile.read.policy
 A list of the desired read policy, in preferred order.
 standard values are
 adaptive, default, random, sequential, vector, whole-file

fs.option.openfile.length
 How long the file is.

fs.option.openfile.split.start
 start of a task's split

fs.option.openfile.split.end
 end of a task's split

These can be used by filesystem connectors to optimize their
reading of the source file, including but not limited to
* skipping existence/length probes when opening a file
* choosing a policy for prefetching/caching data

The hadoop shell commands which read files all declare "whole-file"
and "sequential", as appropriate.

Contributed by Steve Loughran.

Change-Id: Ia290f79ea7973ce8713d4f90f1315b24d7a23da1
steveloughran added a commit to steveloughran/hadoop that referenced this pull request Apr 27, 2022
…e#2584/2)

These changes ensure that sequential files are opened with the
right read policy, and split start/end is passed in.

As well as offering opportunities for filesystem clients to
choose fetch/cache/seek policies, the settings ensure that
processing text files on an s3 bucket where the default policy
is "random" will still be processed efficiently.

This commit depends on the associated hadoop-common patch,
which must be committed first.

Contributed by Steve Loughran.

Change-Id: Ic6713fd752441cf42ebe8739d05c2293a5db9f94
steveloughran added a commit to steveloughran/hadoop that referenced this pull request Apr 27, 2022
S3A input stream support for the few fs.option.openfile settings.
As well as supporting the read policy option and values,
if the file length is declared in fs.option.openfile.length
then no HEAD request will be issued when opening a file.
This can cut a few tens of milliseconds off the operation.

The patch adds a new openfile parameter/FS configuration option
fs.s3a.input.async.drain.threshold (default: 16000).
It declares the number of bytes remaining in the http input stream
above which any operation to read and discard the rest of the stream,
"draining", is executed asynchronously.
This asynchronous draining offers some performance benefit on seek-heavy
file IO.

Contributed by Steve Loughran.

Change-Id: I9b0626bbe635e9fd97ac0f463f5e7167e0111e39
steveloughran added a commit to steveloughran/hadoop that referenced this pull request Apr 27, 2022
Stops the abfs connector warning if openFile().withFileStatus()
is invoked with a FileStatus is not an abfs VersionedFileStatus.

Contributed by Steve Loughran.

Change-Id: I85076b365eb30aaef2ed35139fa8714efd4d048e
asfgit pushed a commit that referenced this pull request Apr 27, 2022
This defines standard option and values for the
openFile() builder API for opening a file:

fs.option.openfile.read.policy
 A list of the desired read policy, in preferred order.
 standard values are
 adaptive, default, random, sequential, vector, whole-file

fs.option.openfile.length
 How long the file is.

fs.option.openfile.split.start
 start of a task's split

fs.option.openfile.split.end
 end of a task's split

These can be used by filesystem connectors to optimize their
reading of the source file, including but not limited to
* skipping existence/length probes when opening a file
* choosing a policy for prefetching/caching data

The hadoop shell commands which read files all declare "whole-file"
and "sequential", as appropriate.

Contributed by Steve Loughran.

Change-Id: Ia290f79ea7973ce8713d4f90f1315b24d7a23da1
asfgit pushed a commit that referenced this pull request Apr 27, 2022
These changes ensure that sequential files are opened with the
right read policy, and split start/end is passed in.

As well as offering opportunities for filesystem clients to
choose fetch/cache/seek policies, the settings ensure that
processing text files on an s3 bucket where the default policy
is "random" will still be processed efficiently.

This commit depends on the associated hadoop-common patch,
which must be committed first.

Contributed by Steve Loughran.

Change-Id: Ic6713fd752441cf42ebe8739d05c2293a5db9f94
asfgit pushed a commit that referenced this pull request Apr 27, 2022
S3A input stream support for the few fs.option.openfile settings.
As well as supporting the read policy option and values,
if the file length is declared in fs.option.openfile.length
then no HEAD request will be issued when opening a file.
This can cut a few tens of milliseconds off the operation.

The patch adds a new openfile parameter/FS configuration option
fs.s3a.input.async.drain.threshold (default: 16000).
It declares the number of bytes remaining in the http input stream
above which any operation to read and discard the rest of the stream,
"draining", is executed asynchronously.
This asynchronous draining offers some performance benefit on seek-heavy
file IO.

Contributed by Steve Loughran.

Change-Id: I9b0626bbe635e9fd97ac0f463f5e7167e0111e39
asfgit pushed a commit that referenced this pull request Apr 27, 2022
Stops the abfs connector warning if openFile().withFileStatus()
is invoked with a FileStatus is not an abfs VersionedFileStatus.

Contributed by Steve Loughran.

Change-Id: I85076b365eb30aaef2ed35139fa8714efd4d048e
@apache apache deleted a comment from hadoop-yetus Apr 29, 2022
@steveloughran
Copy link
Contributor Author

merged

HarshitGupta11 pushed a commit to HarshitGupta11/hadoop that referenced this pull request Nov 28, 2022
…1)

This defines standard option and values for the
openFile() builder API for opening a file:

fs.option.openfile.read.policy
 A list of the desired read policy, in preferred order.
 standard values are
 adaptive, default, random, sequential, vector, whole-file

fs.option.openfile.length
 How long the file is.

fs.option.openfile.split.start
 start of a task's split

fs.option.openfile.split.end
 end of a task's split

These can be used by filesystem connectors to optimize their
reading of the source file, including but not limited to
* skipping existence/length probes when opening a file
* choosing a policy for prefetching/caching data

The hadoop shell commands which read files all declare "whole-file"
and "sequential", as appropriate.

Contributed by Steve Loughran.

Change-Id: Ia290f79ea7973ce8713d4f90f1315b24d7a23da1
HarshitGupta11 pushed a commit to HarshitGupta11/hadoop that referenced this pull request Nov 28, 2022
…e#2584/2)

These changes ensure that sequential files are opened with the
right read policy, and split start/end is passed in.

As well as offering opportunities for filesystem clients to
choose fetch/cache/seek policies, the settings ensure that
processing text files on an s3 bucket where the default policy
is "random" will still be processed efficiently.

This commit depends on the associated hadoop-common patch,
which must be committed first.

Contributed by Steve Loughran.

Change-Id: Ic6713fd752441cf42ebe8739d05c2293a5db9f94
HarshitGupta11 pushed a commit to HarshitGupta11/hadoop that referenced this pull request Nov 28, 2022
S3A input stream support for the few fs.option.openfile settings.
As well as supporting the read policy option and values,
if the file length is declared in fs.option.openfile.length
then no HEAD request will be issued when opening a file.
This can cut a few tens of milliseconds off the operation.

The patch adds a new openfile parameter/FS configuration option
fs.s3a.input.async.drain.threshold (default: 16000).
It declares the number of bytes remaining in the http input stream
above which any operation to read and discard the rest of the stream,
"draining", is executed asynchronously.
This asynchronous draining offers some performance benefit on seek-heavy
file IO.

Contributed by Steve Loughran.

Change-Id: I9b0626bbe635e9fd97ac0f463f5e7167e0111e39
HarshitGupta11 pushed a commit to HarshitGupta11/hadoop that referenced this pull request Nov 28, 2022
Stops the abfs connector warning if openFile().withFileStatus()
is invoked with a FileStatus is not an abfs VersionedFileStatus.

Contributed by Steve Loughran.

Change-Id: I85076b365eb30aaef2ed35139fa8714efd4d048e
@steveloughran steveloughran deleted the s3/HADOOP-16202-enhance-openfile branch December 15, 2022 14:18
deepakdamri pushed a commit to acceldata-io/hadoop that referenced this pull request Jan 21, 2025
…1)

This defines standard option and values for the
openFile() builder API for opening a file:

fs.option.openfile.read.policy
 A list of the desired read policy, in preferred order.
 standard values are
 adaptive, default, random, sequential, vector, whole-file

fs.option.openfile.length
 How long the file is.

fs.option.openfile.split.start
 start of a task's split

fs.option.openfile.split.end
 end of a task's split

These can be used by filesystem connectors to optimize their
reading of the source file, including but not limited to
* skipping existence/length probes when opening a file
* choosing a policy for prefetching/caching data

The hadoop shell commands which read files all declare "whole-file"
and "sequential", as appropriate.

Contributed by Steve Loughran.

Change-Id: Ia290f79ea7973ce8713d4f90f1315b24d7a23da1
deepakdamri pushed a commit to acceldata-io/hadoop that referenced this pull request Jan 21, 2025
…1)

This defines standard option and values for the
openFile() builder API for opening a file:

fs.option.openfile.read.policy
 A list of the desired read policy, in preferred order.
 standard values are
 adaptive, default, random, sequential, vector, whole-file

fs.option.openfile.length
 How long the file is.

fs.option.openfile.split.start
 start of a task's split

fs.option.openfile.split.end
 end of a task's split

These can be used by filesystem connectors to optimize their
reading of the source file, including but not limited to
* skipping existence/length probes when opening a file
* choosing a policy for prefetching/caching data

The hadoop shell commands which read files all declare "whole-file"
and "sequential", as appropriate.

Contributed by Steve Loughran.

Change-Id: Ia290f79ea7973ce8713d4f90f1315b24d7a23da1
deepakdamri pushed a commit to acceldata-io/hadoop that referenced this pull request Jan 21, 2025
…1)

This defines standard option and values for the
openFile() builder API for opening a file:

fs.option.openfile.read.policy
 A list of the desired read policy, in preferred order.
 standard values are
 adaptive, default, random, sequential, vector, whole-file

fs.option.openfile.length
 How long the file is.

fs.option.openfile.split.start
 start of a task's split

fs.option.openfile.split.end
 end of a task's split

These can be used by filesystem connectors to optimize their
reading of the source file, including but not limited to
* skipping existence/length probes when opening a file
* choosing a policy for prefetching/caching data

The hadoop shell commands which read files all declare "whole-file"
and "sequential", as appropriate.

Contributed by Steve Loughran.

Change-Id: Ia290f79ea7973ce8713d4f90f1315b24d7a23da1
deepakdamri pushed a commit to acceldata-io/hadoop that referenced this pull request Jan 21, 2025
…1)

This defines standard option and values for the
openFile() builder API for opening a file:

fs.option.openfile.read.policy
 A list of the desired read policy, in preferred order.
 standard values are
 adaptive, default, random, sequential, vector, whole-file

fs.option.openfile.length
 How long the file is.

fs.option.openfile.split.start
 start of a task's split

fs.option.openfile.split.end
 end of a task's split

These can be used by filesystem connectors to optimize their
reading of the source file, including but not limited to
* skipping existence/length probes when opening a file
* choosing a policy for prefetching/caching data

The hadoop shell commands which read files all declare "whole-file"
and "sequential", as appropriate.

Contributed by Steve Loughran.

Change-Id: Ia290f79ea7973ce8713d4f90f1315b24d7a23da1
deepakdamri pushed a commit to acceldata-io/hadoop that referenced this pull request Jan 21, 2025
…1)

This defines standard option and values for the
openFile() builder API for opening a file:

fs.option.openfile.read.policy
 A list of the desired read policy, in preferred order.
 standard values are
 adaptive, default, random, sequential, vector, whole-file

fs.option.openfile.length
 How long the file is.

fs.option.openfile.split.start
 start of a task's split

fs.option.openfile.split.end
 end of a task's split

These can be used by filesystem connectors to optimize their
reading of the source file, including but not limited to
* skipping existence/length probes when opening a file
* choosing a policy for prefetching/caching data

The hadoop shell commands which read files all declare "whole-file"
and "sequential", as appropriate.

Contributed by Steve Loughran.

Change-Id: Ia290f79ea7973ce8713d4f90f1315b24d7a23da1
deepakdamri pushed a commit to acceldata-io/hadoop that referenced this pull request Jan 21, 2025
…1)

This defines standard option and values for the
openFile() builder API for opening a file:

fs.option.openfile.read.policy
 A list of the desired read policy, in preferred order.
 standard values are
 adaptive, default, random, sequential, vector, whole-file

fs.option.openfile.length
 How long the file is.

fs.option.openfile.split.start
 start of a task's split

fs.option.openfile.split.end
 end of a task's split

These can be used by filesystem connectors to optimize their
reading of the source file, including but not limited to
* skipping existence/length probes when opening a file
* choosing a policy for prefetching/caching data

The hadoop shell commands which read files all declare "whole-file"
and "sequential", as appropriate.

Contributed by Steve Loughran.

Change-Id: Ia290f79ea7973ce8713d4f90f1315b24d7a23da1
deepakdamri pushed a commit to acceldata-io/hadoop that referenced this pull request Jan 21, 2025
…1)

This defines standard option and values for the
openFile() builder API for opening a file:

fs.option.openfile.read.policy
 A list of the desired read policy, in preferred order.
 standard values are
 adaptive, default, random, sequential, vector, whole-file

fs.option.openfile.length
 How long the file is.

fs.option.openfile.split.start
 start of a task's split

fs.option.openfile.split.end
 end of a task's split

These can be used by filesystem connectors to optimize their
reading of the source file, including but not limited to
* skipping existence/length probes when opening a file
* choosing a policy for prefetching/caching data

The hadoop shell commands which read files all declare "whole-file"
and "sequential", as appropriate.

Contributed by Steve Loughran.

Change-Id: Ia290f79ea7973ce8713d4f90f1315b24d7a23da1
deepakdamri pushed a commit to acceldata-io/hadoop that referenced this pull request Jan 21, 2025
…1)

This defines standard option and values for the
openFile() builder API for opening a file:

fs.option.openfile.read.policy
 A list of the desired read policy, in preferred order.
 standard values are
 adaptive, default, random, sequential, vector, whole-file

fs.option.openfile.length
 How long the file is.

fs.option.openfile.split.start
 start of a task's split

fs.option.openfile.split.end
 end of a task's split

These can be used by filesystem connectors to optimize their
reading of the source file, including but not limited to
* skipping existence/length probes when opening a file
* choosing a policy for prefetching/caching data

The hadoop shell commands which read files all declare "whole-file"
and "sequential", as appropriate.

Contributed by Steve Loughran.

Change-Id: Ia290f79ea7973ce8713d4f90f1315b24d7a23da1
deepakdamri pushed a commit to acceldata-io/hadoop that referenced this pull request Jan 21, 2025
S3A input stream support for the few fs.option.openfile settings.
As well as supporting the read policy option and values,
if the file length is declared in fs.option.openfile.length
then no HEAD request will be issued when opening a file.
This can cut a few tens of milliseconds off the operation.

The patch adds a new openfile parameter/FS configuration option
fs.s3a.input.async.drain.threshold (default: 16000).
It declares the number of bytes remaining in the http input stream
above which any operation to read and discard the rest of the stream,
"draining", is executed asynchronously.
This asynchronous draining offers some performance benefit on seek-heavy
file IO.

Contributed by Steve Loughran.

Change-Id: I9b0626bbe635e9fd97ac0f463f5e7167e0111e39
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
fs/s3 changes related to hadoop-aws; submitter must declare test endpoint fs
Projects
None yet
Development

Successfully merging this pull request may close these issues.

6 participants