-
Notifications
You must be signed in to change notification settings - Fork 8.9k
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
HADOOP-18180. Replace use of twitter util-core with java futures #4115
HADOOP-18180. Replace use of twitter util-core with java futures #4115
Conversation
@steveloughran would you be able to review this when you have time? |
did you run the integration tests? we have to be ruthless here about the test process for anything which goes near the object stores, as yetus doesn't have any cloud credentials |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
looks good, i'm pleased to see how easy it is to get off the twitter libs, as having scala on the cp was going to be a blocker for the merge.
- please tell us the s3 endpoint you ran the integration tests against
- once we are happy with the fs.common package, i plan to move it to hadoop-common and into o.a.h.fs.impl. leaving it in hadoop-aws for now makes dev a bit more agile (no need to compile two jars for each change). something to bear in mind, as this stuff is more broadly useful
@@ -433,13 +434,13 @@ private void addToCacheAndRelease(BufferData data, Future<Void> blockFuture) { | |||
} | |||
|
|||
try { | |||
Await.result(blockFuture); | |||
blockFuture.get(); //TODO consider calling get(long timeout, TimeUnit unit) instead |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
if there's a todo item here, it'll need fixing or at least understanding of why a fix isn't needed
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
would it be possible to agree a timeout value - maybe 1 hour? - otherwise, this could block indefinitely
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
this is also the same problem that would occur with the previous Await.result call
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I updated this with a 1 hour timeout
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/CachingBlockManager.java
Outdated
Show resolved
Hide resolved
...op-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/ExecutorServiceFuturePool.java
Show resolved
Hide resolved
I must admit to only having tested this with unit tests. I'm not really an expert on running hadoop integration tests. Would https://github.com/findify/s3mock or something similar be an option? |
🎊 +1 overall
This message was automatically generated. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
minor comments, we are nearly done. for yetus;s complaints, a @SuppressWarnings("unchecked")
should do it
@@ -354,7 +355,7 @@ private void readBlock(BufferData data, boolean isPrefetch, BufferData.State... | |||
} | |||
|
|||
@Override | |||
public Void applyE() { | |||
public Void get() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
we need to think what we want to do here.
- we would want errors to be counted and included in stream stats
- but we've seen abfs apps flooded with stack traces when a transient network error breaks every prefetch
- its good to be aware of transient errors, but not worry too much until the final reaqd.
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3CachingBlockManager.java
Outdated
Show resolved
Hide resolved
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
Show resolved
Hide resolved
...ools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestExecutorServiceFuturePool.java
Outdated
Show resolved
Hide resolved
...ools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestExecutorServiceFuturePool.java
Outdated
Show resolved
Hide resolved
...ools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestExecutorServiceFuturePool.java
Outdated
Show resolved
Hide resolved
Future<Void> future = futurePool.executeFunction(() -> { | ||
throw new IllegalStateException("deliberate"); | ||
}); | ||
assertThrows(ExecutionException.class, () -> future.get(30, TimeUnit.SECONDS)); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
prefer LambdaTestUtils intercept, which includes the toString value of the operation on a failure.
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3InputStream.java
Outdated
Show resolved
Hide resolved
public class ExecutorServiceFuturePool { | ||
private ExecutorService executor; | ||
|
||
public ExecutorServiceFuturePool(ExecutorService executor) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
interesting q here about what service to support/integrate with here, especially for faireness in processs with many readers.
created https://issues.apache.org/jira/browse/HADOOP-18186 as a followup to this.
hi, the testing process is in hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md ; takes about 15 min from your laptop; cleans up the bucket afterwards so there's no long term billing. you can try against a local mino server; do note some outstanding quirks there (https://issues.apache.org/jira/browse/HADOOP-18019 ). |
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
@steveloughran @monthonk thanks for checking the PR. I'm not really a hadoop-aws user. I just saw the coding problem and being familiar with Scala and Java asynch programming, I thought I'd try my hand at a patch. |
🎊 +1 overall
This message was automatically generated. |
reopened due a request from @steveloughran |
🎊 +1 overall
This message was automatically generated. |
Looking at the test results shared by Monthon...
I think all but one of these are expected - ITestMarkerTool.testRunLimitedLandsatAudit:320 is a known issue in trunk (HADOOP-18168) and 4 of those are known to fail in this feature branch (HADOOP-18175). The interesting one is not mentioned in HADOOP-18175 is |
unbuffer happens on some networks, marker is known. let me try locally. this is a feature branch and I'm not too worried about patches introducing failures, especially this one, which must go in |
stack of the failure
it's happening in the seek(oldPos) code in the default implementation of positioned read, because the previous unpositioned read read the whole file, so pos() is at file length.
This is quite a good little failure; I'm impressed but who ever managed to think at this corner case. pos is at EOF, so returning to it is raising the EOF. It is unrelated to this PR; I will file a JIRA. We will probably want seek() to not do the checks, and have read() do the validation, which is mostly what s3a does , as it only rejects negative offsets. |
+1 merged. only thought after i'd hit the merge that i should include a reference to the HADOOP-18028 jira in the message; I will do that in the others, so a log grep will find them all. |
…3A prefetching stream (apache#4115) Contributed by PJ Fanning.
…3A prefetching stream (apache#4115) Contributed by PJ Fanning.
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
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
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
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
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
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
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
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
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
Description of PR
HADOOP-18180 replace use of twitter util-core - it uses scala 2.11 and will cause issue for Spark and other scala based tools that use different versions of scala
How was this patch tested?
For code changes:
LICENSE
,LICENSE-binary
,NOTICE-binary
files?