Skip to content

HADOOP-18175. fix test failures with prefetching s3a input stream #4212

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

monthonk
Copy link
Contributor

Description of PR

HADOOP-18175 Fix many test failures on prefetching branch.

ITestS3AContractUnbuffer -> changed getPos() on a closed file to return 0

ITestS3AUnbuffer -> skipping unsupported input stream for now, HADOOP-18184 should address this test later

ITestS3ARequesterPays -> wired up streamStatistics.streamOpened() and changed assertion based on stream type

ITestS3AFileContextStatistics -> moved seek past EOF validation to read(), which is what S3AInputStream does

How was this patch tested?

Tested in eu-west-1 with both fs.s3a.prefetch.enabled=true and fs.s3a.prefetch.enabled=false

For code changes:

  • Does the title or this PR starts with the corresponding JIRA issue id (e.g. 'HADOOP-17799. Your PR title ...')?
  • Object storage: have the integration tests been executed and the endpoint declared according to the connector-specific documentation?
  • If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under ASF 2.0?
  • If applicable, have you updated the LICENSE, LICENSE-binary, NOTICE-binary files?

@@ -192,6 +197,12 @@ private boolean isObjectStreamOpen(FSDataInputStream inputStream) {
return ((S3AInputStream) inputStream.getWrappedStream()).isObjectStreamOpen();
}

private void skipIfCannotUnbuffer(InputStream inputStream) {
Copy link
Contributor

Choose a reason for hiding this comment

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

You should just be able to use inputStream.hasCapability(StreamCapabilities.UNBUFFER) instead of this method

Copy link
Contributor Author

Choose a reason for hiding this comment

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

At this point we still don't know what kind of inputStream it is, the method hasCapability only visible after we cast it to S3AInputStream or S3PrefetchingInputStream

Copy link
Contributor Author

Choose a reason for hiding this comment

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

just noticed we can use FSDataInputStream.hasCapability(...), I will change to it

Copy link
Contributor

Choose a reason for hiding this comment

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

you can also use StoreImplementationUtils when needed -which it isn't here

IOStatisticAssertions.assertThatStatisticCounter(inputStream.getIOStatistics(),
StreamStatisticNames.STREAM_READ_OPENED).isEqualTo(1);
} else {
// For S3InputStream, verify > 1 call was made,
Copy link
Contributor

Choose a reason for hiding this comment

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

should this be For S3AInputStream ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

you're right, updated the comment

@hadoop-yetus
Copy link

🎊 +1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 0m 39s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 0s No case conflicting files found.
+0 🆗 codespell 0m 1s codespell was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
+1 💚 test4tests 0m 0s The patch appears to include 3 new or modified test files.
_ feature-HADOOP-18028-s3a-prefetch Compile Tests _
+1 💚 mvninstall 39m 52s feature-HADOOP-18028-s3a-prefetch passed
+1 💚 compile 0m 53s feature-HADOOP-18028-s3a-prefetch passed with JDK Ubuntu-11.0.14.1+1-Ubuntu-0ubuntu1.20.04
+1 💚 compile 0m 49s feature-HADOOP-18028-s3a-prefetch passed with JDK Private Build-1.8.0_312-8u312-b07-0ubuntu1~20.04-b07
+1 💚 checkstyle 0m 45s feature-HADOOP-18028-s3a-prefetch passed
+1 💚 mvnsite 0m 55s feature-HADOOP-18028-s3a-prefetch passed
+1 💚 javadoc 0m 42s feature-HADOOP-18028-s3a-prefetch passed with JDK Ubuntu-11.0.14.1+1-Ubuntu-0ubuntu1.20.04
+1 💚 javadoc 0m 44s feature-HADOOP-18028-s3a-prefetch passed with JDK Private Build-1.8.0_312-8u312-b07-0ubuntu1~20.04-b07
+1 💚 spotbugs 1m 33s feature-HADOOP-18028-s3a-prefetch passed
+1 💚 shadedclient 22m 24s branch has no errors when building and testing our client artifacts.
_ Patch Compile Tests _
+1 💚 mvninstall 0m 38s the patch passed
+1 💚 compile 0m 42s the patch passed with JDK Ubuntu-11.0.14.1+1-Ubuntu-0ubuntu1.20.04
+1 💚 javac 0m 42s the patch passed
+1 💚 compile 0m 36s the patch passed with JDK Private Build-1.8.0_312-8u312-b07-0ubuntu1~20.04-b07
+1 💚 javac 0m 36s the patch passed
+1 💚 blanks 0m 0s The patch has no blanks issues.
+1 💚 checkstyle 0m 23s the patch passed
+1 💚 mvnsite 0m 43s the patch passed
+1 💚 javadoc 0m 24s the patch passed with JDK Ubuntu-11.0.14.1+1-Ubuntu-0ubuntu1.20.04
+1 💚 javadoc 0m 31s the patch passed with JDK Private Build-1.8.0_312-8u312-b07-0ubuntu1~20.04-b07
+1 💚 spotbugs 1m 17s the patch passed
+1 💚 shadedclient 21m 40s patch has no errors when building and testing our client artifacts.
_ Other Tests _
+1 💚 unit 2m 54s hadoop-aws in the patch passed.
+1 💚 asflicense 0m 52s The patch does not generate ASF License warnings.
100m 40s
Subsystem Report/Notes
Docker ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4212/5/artifact/out/Dockerfile
GITHUB PR #4212
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell
uname Linux 5f5921be7f77 4.15.0-156-generic #163-Ubuntu SMP Thu Aug 19 23:31:58 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision feature-HADOOP-18028-s3a-prefetch / e1eb23e
Default Java Private Build-1.8.0_312-8u312-b07-0ubuntu1~20.04-b07
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.14.1+1-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_312-8u312-b07-0ubuntu1~20.04-b07
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4212/5/testReport/
Max. process+thread count 675 (vs. ulimit of 5500)
modules C: hadoop-tools/hadoop-aws U: hadoop-tools/hadoop-aws
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4212/5/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org

This message was automatically generated.

@@ -103,8 +103,7 @@ public synchronized int available() throws IOException {
*/
@Override
public synchronized long getPos() throws IOException {
Copy link
Contributor

Choose a reason for hiding this comment

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

this method no longer throws an IOException

Copy link
Contributor Author

Choose a reason for hiding this comment

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

this.inputStream.getPos() below could throw it, so I have to pass it up here

@monthonk
Copy link
Contributor Author

monthonk commented May 3, 2022

@mukund-thakur @steveloughran would you be able to have a look at this one? Thanks!

@apache apache deleted a comment from hadoop-yetus May 3, 2022
@apache apache deleted a comment from hadoop-yetus May 3, 2022
@apache apache deleted a comment from hadoop-yetus May 3, 2022
@apache apache deleted a comment from hadoop-yetus May 3, 2022
@steveloughran
Copy link
Contributor

LGTM.

+1, merging

@steveloughran steveloughran merged commit f38bbe2 into apache:feature-HADOOP-18028-s3a-prefetch May 4, 2022
@steveloughran
Copy link
Contributor

merged!

ahmarsuhail pushed a commit to ahmarsuhail/hadoop that referenced this pull request May 19, 2022
steveloughran pushed a commit to steveloughran/hadoop that referenced this pull request Jul 28, 2022
steveloughran added a commit to steveloughran/hadoop that referenced this pull request Aug 2, 2022
This is the the a rollup patch of the HADOOP-18028 S3A performance input stream
feature branch.

Contains

HADOOP-18028. High performance S3A input stream (apache#4109)

This is the the merge of the HADOOP-18028 S3A performance input stream.
This patch on its own is incomplete and must be accompanied by all other commits
with HADOOP-18028 in their git commit message. Consult the JIRA for that list

Contributed by Bhalchandra Pandit.

HADOOP-18180. Replace use of twitter util-core with java futures in S3A prefetching stream (apache#4115)

Contributed by PJ Fanning.

HADOOP-18177. Document prefetching architecture. (apache#4205)

Contributed by Ahmar Suhail

HADOOP-18175. fix test failures with prefetching s3a input stream (apache#4212)

Contributed by Monthon Klongklaew

HADOOP-18231.  S3A prefetching: fix failing tests & drain stream async.  (apache#4386)

* adds in new test for prefetching input stream
* creates streamStats before opening stream
* updates numBlocks calculation method
* fixes ITestS3AOpenCost.testOpenFileLongerLength
* drains stream async
* fixes failing unit test

Contributed by Ahmar Suhail

HADOOP-18254. Disable S3A prefetching by default. (apache#4469)

Contributed by Ahmar Suhail

HADOOP-18190. Collect IOStatistics during S3A prefetching (apache#4458)

This adds iOStatisticsConnection to the S3PrefetchingInputStream class, with
new statistic names in StreamStatistics.

This stream is not (yet) IOStatisticsContext aware.

Contributed by Ahmar Suhail.

Change-Id: I48f217086531c12d6e2f0f91e39f17054a74d20f
steveloughran added a commit to steveloughran/hadoop that referenced this pull request Aug 17, 2022
This is the the preview release of the HADOOP-18028 S3A performance input stream.
It is still stabilizing, but ready to test.

Contains

HADOOP-18028. High performance S3A input stream (apache#4109)
	Contributed by Bhalchandra Pandit.

HADOOP-18180. Replace use of twitter util-core with java futures (apache#4115)
	Contributed by PJ Fanning.

HADOOP-18177. Document prefetching architecture. (apache#4205)
	Contributed by Ahmar Suhail

HADOOP-18175. fix test failures with prefetching s3a input stream (apache#4212)
 Contributed by Monthon Klongklaew

HADOOP-18231.  S3A prefetching: fix failing tests & drain stream async.  (apache#4386)

	* adds in new test for prefetching input stream
	* creates streamStats before opening stream
	* updates numBlocks calculation method
	* fixes ITestS3AOpenCost.testOpenFileLongerLength
	* drains stream async
	* fixes failing unit test

	Contributed by Ahmar Suhail

HADOOP-18254. Disable S3A prefetching by default. (apache#4469)
	Contributed by Ahmar Suhail

HADOOP-18190. Collect IOStatistics during S3A prefetching (apache#4458)

	This adds iOStatisticsConnection to the S3PrefetchingInputStream class, with
	new statistic names in StreamStatistics.

	This stream is not (yet) IOStatisticsContext aware.

	Contributed by Ahmar Suhail

HADOOP-18379 rebase feature/HADOOP-18028-s3a-prefetch to trunk
HADOOP-18187. Convert s3a prefetching to use JavaDoc for fields and enums.
HADOOP-18318. Update class names to be clear they belong to S3A prefetching
	Contributed by Steve Loughran

Change-Id: I6511c51c3580c57eb72e8ea686c88e3917d12a06
steveloughran added a commit that referenced this pull request Aug 18, 2022
This is the the preview release of the HADOOP-18028 S3A performance input stream.
It is still stabilizing, but ready to test.

Contains

HADOOP-18028. High performance S3A input stream (#4109)
	Contributed by Bhalchandra Pandit.

HADOOP-18180. Replace use of twitter util-core with java futures (#4115)
	Contributed by PJ Fanning.

HADOOP-18177. Document prefetching architecture. (#4205)
	Contributed by Ahmar Suhail

HADOOP-18175. fix test failures with prefetching s3a input stream (#4212)
 Contributed by Monthon Klongklaew

HADOOP-18231.  S3A prefetching: fix failing tests & drain stream async.  (#4386)

	* adds in new test for prefetching input stream
	* creates streamStats before opening stream
	* updates numBlocks calculation method
	* fixes ITestS3AOpenCost.testOpenFileLongerLength
	* drains stream async
	* fixes failing unit test

	Contributed by Ahmar Suhail

HADOOP-18254. Disable S3A prefetching by default. (#4469)
	Contributed by Ahmar Suhail

HADOOP-18190. Collect IOStatistics during S3A prefetching (#4458)

	This adds iOStatisticsConnection to the S3PrefetchingInputStream class, with
	new statistic names in StreamStatistics.

	This stream is not (yet) IOStatisticsContext aware.

	Contributed by Ahmar Suhail

HADOOP-18379 rebase feature/HADOOP-18028-s3a-prefetch to trunk
HADOOP-18187. Convert s3a prefetching to use JavaDoc for fields and enums.
HADOOP-18318. Update class names to be clear they belong to S3A prefetching
	Contributed by Steve Loughran
HarshitGupta11 pushed a commit to HarshitGupta11/hadoop that referenced this pull request Nov 28, 2022
This is the the preview release of the HADOOP-18028 S3A performance input stream.
It is still stabilizing, but ready to test.

Contains

HADOOP-18028. High performance S3A input stream (apache#4109)
	Contributed by Bhalchandra Pandit.

HADOOP-18180. Replace use of twitter util-core with java futures (apache#4115)
	Contributed by PJ Fanning.

HADOOP-18177. Document prefetching architecture. (apache#4205)
	Contributed by Ahmar Suhail

HADOOP-18175. fix test failures with prefetching s3a input stream (apache#4212)
 Contributed by Monthon Klongklaew

HADOOP-18231.  S3A prefetching: fix failing tests & drain stream async.  (apache#4386)

	* adds in new test for prefetching input stream
	* creates streamStats before opening stream
	* updates numBlocks calculation method
	* fixes ITestS3AOpenCost.testOpenFileLongerLength
	* drains stream async
	* fixes failing unit test

	Contributed by Ahmar Suhail

HADOOP-18254. Disable S3A prefetching by default. (apache#4469)
	Contributed by Ahmar Suhail

HADOOP-18190. Collect IOStatistics during S3A prefetching (apache#4458)

	This adds iOStatisticsConnection to the S3PrefetchingInputStream class, with
	new statistic names in StreamStatistics.

	This stream is not (yet) IOStatisticsContext aware.

	Contributed by Ahmar Suhail

HADOOP-18379 rebase feature/HADOOP-18028-s3a-prefetch to trunk
HADOOP-18187. Convert s3a prefetching to use JavaDoc for fields and enums.
HADOOP-18318. Update class names to be clear they belong to S3A prefetching
	Contributed by Steve Loughran
steveloughran added a commit to steveloughran/hadoop that referenced this pull request Apr 14, 2023
This is the the preview release of the HADOOP-18028 S3A performance input stream.
It is still stabilizing, but ready to test.

Contains

HADOOP-18028. High performance S3A input stream (apache#4109)
	Contributed by Bhalchandra Pandit.

HADOOP-18180. Replace use of twitter util-core with java futures (apache#4115)
	Contributed by PJ Fanning.

HADOOP-18177. Document prefetching architecture. (apache#4205)
	Contributed by Ahmar Suhail

HADOOP-18175. fix test failures with prefetching s3a input stream (apache#4212)
 Contributed by Monthon Klongklaew

HADOOP-18231.  S3A prefetching: fix failing tests & drain stream async.  (apache#4386)

	* adds in new test for prefetching input stream
	* creates streamStats before opening stream
	* updates numBlocks calculation method
	* fixes ITestS3AOpenCost.testOpenFileLongerLength
	* drains stream async
	* fixes failing unit test

	Contributed by Ahmar Suhail

HADOOP-18254. Disable S3A prefetching by default. (apache#4469)
	Contributed by Ahmar Suhail

HADOOP-18190. Collect IOStatistics during S3A prefetching (apache#4458)

	This adds iOStatisticsConnection to the S3PrefetchingInputStream class, with
	new statistic names in StreamStatistics.

	This stream is not (yet) IOStatisticsContext aware.

	Contributed by Ahmar Suhail

HADOOP-18379 rebase feature/HADOOP-18028-s3a-prefetch to trunk
HADOOP-18187. Convert s3a prefetching to use JavaDoc for fields and enums.
HADOOP-18318. Update class names to be clear they belong to S3A prefetching
	Contributed by Steve Loughran

Change-Id: I3eca19564dc0c0cb83184f4a42605dbafd908937
asfgit pushed a commit that referenced this pull request Apr 24, 2023
This is the the preview release of the HADOOP-18028 S3A performance input stream.
It is still stabilizing, but ready to test.

Contains

HADOOP-18028. High performance S3A input stream (#4109)
	Contributed by Bhalchandra Pandit.

HADOOP-18180. Replace use of twitter util-core with java futures (#4115)
	Contributed by PJ Fanning.

HADOOP-18177. Document prefetching architecture. (#4205)
	Contributed by Ahmar Suhail

HADOOP-18175. fix test failures with prefetching s3a input stream (#4212)
 Contributed by Monthon Klongklaew

HADOOP-18231.  S3A prefetching: fix failing tests & drain stream async.  (#4386)

	* adds in new test for prefetching input stream
	* creates streamStats before opening stream
	* updates numBlocks calculation method
	* fixes ITestS3AOpenCost.testOpenFileLongerLength
	* drains stream async
	* fixes failing unit test

	Contributed by Ahmar Suhail

HADOOP-18254. Disable S3A prefetching by default. (#4469)
	Contributed by Ahmar Suhail

HADOOP-18190. Collect IOStatistics during S3A prefetching (#4458)

	This adds iOStatisticsConnection to the S3PrefetchingInputStream class, with
	new statistic names in StreamStatistics.

	This stream is not (yet) IOStatisticsContext aware.

	Contributed by Ahmar Suhail

HADOOP-18379 rebase feature/HADOOP-18028-s3a-prefetch to trunk
HADOOP-18187. Convert s3a prefetching to use JavaDoc for fields and enums.
HADOOP-18318. Update class names to be clear they belong to S3A prefetching
	Contributed by Steve Loughran
asfgit pushed a commit that referenced this pull request Apr 25, 2023
This is the the preview release of the HADOOP-18028 S3A performance input stream.
It is still stabilizing, but ready to test.

Contains

HADOOP-18028. High performance S3A input stream (#4109)
	Contributed by Bhalchandra Pandit.

HADOOP-18180. Replace use of twitter util-core with java futures (#4115)
	Contributed by PJ Fanning.

HADOOP-18177. Document prefetching architecture. (#4205)
	Contributed by Ahmar Suhail

HADOOP-18175. fix test failures with prefetching s3a input stream (#4212)
 Contributed by Monthon Klongklaew

HADOOP-18231.  S3A prefetching: fix failing tests & drain stream async.  (#4386)

	* adds in new test for prefetching input stream
	* creates streamStats before opening stream
	* updates numBlocks calculation method
	* fixes ITestS3AOpenCost.testOpenFileLongerLength
	* drains stream async
	* fixes failing unit test

	Contributed by Ahmar Suhail

HADOOP-18254. Disable S3A prefetching by default. (#4469)
	Contributed by Ahmar Suhail

HADOOP-18190. Collect IOStatistics during S3A prefetching (#4458)

	This adds iOStatisticsConnection to the S3PrefetchingInputStream class, with
	new statistic names in StreamStatistics.

	This stream is not (yet) IOStatisticsContext aware.

	Contributed by Ahmar Suhail

HADOOP-18379 rebase feature/HADOOP-18028-s3a-prefetch to trunk
HADOOP-18187. Convert s3a prefetching to use JavaDoc for fields and enums.
HADOOP-18318. Update class names to be clear they belong to S3A prefetching
	Contributed by Steve Loughran
asfgit pushed a commit that referenced this pull request Apr 26, 2023
This is the the preview release of the HADOOP-18028 S3A performance input stream.
It is still stabilizing, but ready to test.

Contains

HADOOP-18028. High performance S3A input stream (#4109)
	Contributed by Bhalchandra Pandit.

HADOOP-18180. Replace use of twitter util-core with java futures (#4115)
	Contributed by PJ Fanning.

HADOOP-18177. Document prefetching architecture. (#4205)
	Contributed by Ahmar Suhail

HADOOP-18175. fix test failures with prefetching s3a input stream (#4212)
 Contributed by Monthon Klongklaew

HADOOP-18231.  S3A prefetching: fix failing tests & drain stream async.  (#4386)

	* adds in new test for prefetching input stream
	* creates streamStats before opening stream
	* updates numBlocks calculation method
	* fixes ITestS3AOpenCost.testOpenFileLongerLength
	* drains stream async
	* fixes failing unit test

	Contributed by Ahmar Suhail

HADOOP-18254. Disable S3A prefetching by default. (#4469)
	Contributed by Ahmar Suhail

HADOOP-18190. Collect IOStatistics during S3A prefetching (#4458)

	This adds iOStatisticsConnection to the S3PrefetchingInputStream class, with
	new statistic names in StreamStatistics.

	This stream is not (yet) IOStatisticsContext aware.

	Contributed by Ahmar Suhail

HADOOP-18379 rebase feature/HADOOP-18028-s3a-prefetch to trunk
HADOOP-18187. Convert s3a prefetching to use JavaDoc for fields and enums.
HADOOP-18318. Update class names to be clear they belong to S3A prefetching
	Contributed by Steve Loughran
asfgit pushed a commit that referenced this pull request Apr 28, 2023
This is the the preview release of the HADOOP-18028 S3A performance input stream.
It is still stabilizing, but ready to test.

Contains

HADOOP-18028. High performance S3A input stream (#4109)
	Contributed by Bhalchandra Pandit.

HADOOP-18180. Replace use of twitter util-core with java futures (#4115)
	Contributed by PJ Fanning.

HADOOP-18177. Document prefetching architecture. (#4205)
	Contributed by Ahmar Suhail

HADOOP-18175. fix test failures with prefetching s3a input stream (#4212)
 Contributed by Monthon Klongklaew

HADOOP-18231.  S3A prefetching: fix failing tests & drain stream async.  (#4386)

	* adds in new test for prefetching input stream
	* creates streamStats before opening stream
	* updates numBlocks calculation method
	* fixes ITestS3AOpenCost.testOpenFileLongerLength
	* drains stream async
	* fixes failing unit test

	Contributed by Ahmar Suhail

HADOOP-18254. Disable S3A prefetching by default. (#4469)
	Contributed by Ahmar Suhail

HADOOP-18190. Collect IOStatistics during S3A prefetching (#4458)

	This adds iOStatisticsConnection to the S3PrefetchingInputStream class, with
	new statistic names in StreamStatistics.

	This stream is not (yet) IOStatisticsContext aware.

	Contributed by Ahmar Suhail

HADOOP-18379 rebase feature/HADOOP-18028-s3a-prefetch to trunk
HADOOP-18187. Convert s3a prefetching to use JavaDoc for fields and enums.
HADOOP-18318. Update class names to be clear they belong to S3A prefetching
	Contributed by Steve Loughran
deepakdamri pushed a commit to acceldata-io/hadoop that referenced this pull request Jan 21, 2025
This is the the preview release of the HADOOP-18028 S3A performance input stream.
It is still stabilizing, but ready to test.

Contains

HADOOP-18028. High performance S3A input stream (apache#4109)
	Contributed by Bhalchandra Pandit.

HADOOP-18180. Replace use of twitter util-core with java futures (apache#4115)
	Contributed by PJ Fanning.

HADOOP-18177. Document prefetching architecture. (apache#4205)
	Contributed by Ahmar Suhail

HADOOP-18175. fix test failures with prefetching s3a input stream (apache#4212)
 Contributed by Monthon Klongklaew

HADOOP-18231.  S3A prefetching: fix failing tests & drain stream async.  (apache#4386)

	* adds in new test for prefetching input stream
	* creates streamStats before opening stream
	* updates numBlocks calculation method
	* fixes ITestS3AOpenCost.testOpenFileLongerLength
	* drains stream async
	* fixes failing unit test

	Contributed by Ahmar Suhail

HADOOP-18254. Disable S3A prefetching by default. (apache#4469)
	Contributed by Ahmar Suhail

HADOOP-18190. Collect IOStatistics during S3A prefetching (apache#4458)

	This adds iOStatisticsConnection to the S3PrefetchingInputStream class, with
	new statistic names in StreamStatistics.

	This stream is not (yet) IOStatisticsContext aware.

	Contributed by Ahmar Suhail

HADOOP-18379 rebase feature/HADOOP-18028-s3a-prefetch to trunk
HADOOP-18187. Convert s3a prefetching to use JavaDoc for fields and enums.
HADOOP-18318. Update class names to be clear they belong to S3A prefetching
	Contributed by Steve Loughran
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants