Navigation Menu

Skip to content
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-16830. Add public IOStatistics API + S3A implementation #1820

Closed

Conversation

steveloughran
Copy link
Contributor

@steveloughran steveloughran commented Jan 27, 2020

Statistics APIs and s3a stream implementation.

https://issues.apache.org/jira/browse/HADOOP-16830

Any FSInputStream subclass which is implements IOStatisticsSource will
have its statistics logged in a toString(), which makes for easy logging
of stats in downstream code.

Change-Id: If9a9b2396229088886f1b41cd5417ee78206aea8

@steveloughran steveloughran added enhancement fs/s3 changes related to hadoop-aws; submitter must declare test endpoint work in progress PRs still Work in Progress; reviews not expected but still welcome labels Jan 27, 2020
@steveloughran steveloughran changed the title HADOOP-16830. Add public IOStatistics API. HADOOP-16830. Add public IOStatistics API + S3A implementation Jan 28, 2020
@steveloughran
Copy link
Contributor Author

HADOOP-16830. Evolution of S3AStatisticsContext

This is still at a -is this the right design?- stage.

New class ActiveOperationContext. This is meant to be superclass of all operation context's, supplanting the S3AOpContext.

I've also made WriteOperationHelper a subclass of it -which is one thing I'm not so sure of. Better as a constructor param.

Almost all production references to the S3AInstrumentation class now refer to a much more minimal S3AStatisticsContext.

The goal there is to have one which increments op-specific counters as well as the FS counters/metrics; for now there's a relay to
S3AInstrumentation, and an empty one for testing/no-ops.

Sticking this up while I do other things briefly, its here to show my direction of change.

Regarding the IOStats API, I want to move off always returning something to making it Optional<>; that way callers can see its not there, but don't
need to play is !null calls before acting. As usual, I have mixed feelings there.

Next Steps

  • IOStats again
  • Make WriteOperationHelper something we can construct with the stats, so all ops which go through it pass the stats back to the caller.
  • Pass in to the AbstractStoreOperation constructor
  • and have S3AFileSystem.OperationCallbacksImpl do the same. These will both be instantiated for each operation, so can keep the stats context around without having to force the *Operation classes to add it everywhere

We inevitably will need to pass it around as an argument to every single innerXYX and other internal operation in S3AFS.

Oops.

It is critical, therefore, that ActiveOperationContext is something we don't find we need to replace in future; something we can just extend with things like

  • trace span info
  • custom UA header
  • custom AWS auth chain

A WriteOperationContext will be a subclass with

  • enum about parent dir state to enable us to skip empty dir marking when we knew it wasn't needed
  • any actions to do in finishedWrite
    ...

Testing.

All the intermittently flaky tests failed for me. I'm going to do an aggregate "fix the flaky's" patch

@steveloughran
Copy link
Contributor Author

Checkstyle is all about unread stats names; I need to make sure they are in use in S3A then change checkstyle to not worry about that class

@apache apache deleted a comment from hadoop-yetus Jan 31, 2020
@apache apache deleted a comment from hadoop-yetus Jan 31, 2020
@apache apache deleted a comment from hadoop-yetus Feb 1, 2020
@steveloughran steveloughran force-pushed the s3/HADOOP-16830-iostatistics branch 2 times, most recently from 738d334 to f035eae Compare February 19, 2020 19:19
@steveloughran
Copy link
Contributor Author

WARNING] Tests run: 14, Failures: 0, Errors: 0, Skipped: 14, Time elapsed: 17.413 s - in org.apache.hadoop.fs.s3a.commit.terasort.ITestTerasortOnS3A
[INFO] Running org.apache.hadoop.fs.s3a.fileContext.ITestS3AFileContextStatistics
[ERROR] Tests run: 3, Failures: 1, Errors: 0, Skipped: 0, Time elapsed: 5.958 s <<< FAILURE! - in org.apache.hadoop.fs.s3a.fileContext.ITestS3AFileContextStatistics
[ERROR] testStatistics(org.apache.hadoop.fs.s3a.fileContext.ITestS3AFileContextStatistics)  Time elapsed: 2.603 s  <<< FAILURE!
java.lang.AssertionError: expected:<512> but was:<0>
	at org.junit.Assert.fail(Assert.java:88)
	at org.junit.Assert.failNotEquals(Assert.java:834)
	at org.junit.Assert.assertEquals(Assert.java:645)
	at org.junit.Assert.assertEquals(Assert.java:631)
	at org.apache.hadoop.fs.s3a.fileContext.ITestS3AFileContextStatistics.verifyWrittenBytes(ITestS3AFileContextStatistics.java:68)
	at org.apache.hadoop.fs.FCStatisticsBaseTest.testStatistics(FCStatisticsBaseTest.java:103)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
	at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325)
	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78)
	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57)
	at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
	at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
	at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
	at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
	at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
	at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
	at org.apache.maven.surefire.junit4.JUnit4Provider.execute(JUnit4Provider.java:365)
	at org.apache.maven.surefire.junit4.JUnit4Provider.executeWithRerun(JUnit4Provider.java:273)
	at org.apache.maven.surefire.junit4.JUnit4Provider.executeTestSet(JUnit4Provider.java:238)
	at org.apache.maven.surefire.junit4.JUnit4Provider.invoke(JUnit4Provider.java:159)
	at org.apache.maven.surefire.booter.ForkedBooter.invokeProviderInSameClassLoader(ForkedBooter.java:384)
	at org.apache.maven.surefire.booter.ForkedBooter.runSuitesInProcess(ForkedBooter.java:345)
	at org.apache.maven.surefire.booter.ForkedBooter.execute(ForkedBooter.java:126)
	at org.apache.maven.surefire.booter.ForkedBooter.main(ForkedBooter.java:418)

@apache apache deleted a comment from hadoop-yetus Feb 20, 2020
@apache apache deleted a comment from hadoop-yetus Feb 24, 2020
@apache apache deleted a comment from hadoop-yetus Feb 24, 2020
@apache apache deleted a comment from hadoop-yetus Feb 27, 2020
@steveloughran
Copy link
Contributor Author

s3a test failures are all mock-related. sigh


[ERROR] testFakeDirectory(org.apache.hadoop.fs.s3a.TestS3AGetFileStatus)  Time elapsed: 0.016 s  <<< ERROR!
org.apache.hadoop.metrics2.MetricsException: Metric name stream_read_operations already exists!
	at org.apache.hadoop.metrics2.lib.MetricsRegistry.checkMetricName(MetricsRegistry.java:427)
	at org.apache.hadoop.metrics2.lib.MetricsRegistry.newCounter(MetricsRegistry.java:129)
	at org.apache.hadoop.metrics2.lib.MetricsRegistry.newCounter(MetricsRegistry.java:118)
	at org.apache.hadoop.fs.s3a.S3AInstrumentation.counter(S3AInstrumentation.java:307)
	at org.apache.hadoop.fs.s3a.S3AInstrumentation.counter(S3AInstrumentation.java:316)
	at org.apache.hadoop.fs.s3a.S3AInstrumentation.<init>(S3AInstrumentation.java:239)
	at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:354)
	at org.apache.hadoop.fs.s3a.AbstractS3AMockTest.setup(AbstractS3AMockTest.java:62)

@steveloughran steveloughran force-pushed the s3/HADOOP-16830-iostatistics branch 4 times, most recently from c531518 to 5e0f2b3 Compare March 27, 2020 14:52
@apache apache deleted a comment from hadoop-yetus Apr 14, 2020
@apache apache deleted a comment from hadoop-yetus Apr 14, 2020
@apache apache deleted a comment from hadoop-yetus Apr 14, 2020
@apache apache deleted a comment from hadoop-yetus Apr 14, 2020
@apache apache deleted a comment from hadoop-yetus Apr 14, 2020
@steveloughran
Copy link
Contributor Author

  • tweak of the EmptyIOStatistics.
    We do need tests of the dynamic stuff next
  • S3A stats -findbugs changed to turn off complaints about volatile

the change to the builder API needed to wire up AWS stats collection is broken regarding region signing calculation:

[ERROR] testSelectNothing(org.apache.hadoop.fs.s3a.select.ITestS3SelectCLI)  Time elapsed: 2.059 s  <<< ERROR!
org.apache.hadoop.fs.s3a.AWSBadRequestException: getFileStatus on s3a://landsat-pds/scene_list.gz: com.amazonaws.services.s3.model.AmazonS3Exception: Bad Request (Service: Amazon S3; Status Code: 400; Error Code: 400 Bad Request; Request ID: 9BDEA365F9EE00C9; S3 Extended Request ID: oeSi4G7orAk2b3ogGQp9ydQf6a0ktIuQPFKKQ+tox6OngBliI6NNPcZTIPLI37U21ffivz/HDw8=), S3 Extended Request ID: oeSi4G7orAk2b3ogGQp9ydQf6a0ktIuQPFKKQ+tox6OngBliI6NNPcZTIPLI37U21ffivz/HDw8=:400 Bad Request: Bad Request (Service: Amazon S3; Status Code: 400; Error Code: 400 Bad Request; Request ID: 9BDEA365F9EE00C9; S3 Extended Request ID: oeSi4G7orAk2b3ogGQp9ydQf6a0ktIuQPFKKQ+tox6OngBliI6NNPcZTIPLI37U21ffivz/HDw8=)
	at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:241)
	at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:168)
	at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:3014)
	at org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:2932)
	at org.apache.hadoop.fs.s3a.S3AFileSystem.extractOrFetchSimpleFileStatus(S3AFileSystem.java:4636)
	at org.apache.hadoop.fs.s3a.S3AFileSystem.select(S3AFileSystem.java:4562)
	at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$null$24(S3AFileSystem.java:4719)
	at org.apache.hadoop.util.LambdaUtils.eval(LambdaUtils.java:52)
	at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$openFileWithOptions$25(S3AFileSystem.java:4718)
	at java.util.concurrent.FutureTask.run(FutureTask.java:266)

need to understand more; may want to do an SDK updated too

@steveloughran
Copy link
Contributor Author

I've been convinced that we should just move to long off volatile for the stream statistics

  • it's only statistics and a bit of lag is acceptable
  • it's better to have non-blocking reads than blocking to log stats in different threads
  • and trying to be clever about volatile is doomed.

Conclusion: we go with long and document in the IOStatistics API That it is better to be fast than synchronized if given a choice.


/**
* Probe for an attribute of this statistics set.
* @return attributes.
Copy link
Contributor

Choose a reason for hiding this comment

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

Return is boolean.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

will fix

/**
* Treemaps sort their insertions so the iterator is ordered.
*/
private final Map<String, ToLongFunction<String>> evaluators
Copy link
Contributor

@mukund-thakur mukund-thakur Apr 21, 2020

Choose a reason for hiding this comment

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

What is the intent of ToLong function here? Why can't we use Long directly?

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 is the "Dynamic" stats source -you build it with functions and whenever that stat is retrieved, it's re-evaluated. In implementation's I'm providing getters to values, with the standard one being a get() on an atomic long.

every time you look up a statistic, you get the newest value

/**
* Support for working with statistics.
*/
public final class IOStatisticsSupport {
Copy link
Contributor

Choose a reason for hiding this comment

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

Duplicate class. Already present in impl.

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'll rename one "IOStatisticsImplementationSupport"...the other is for app use

/**
* Add a new evaluator to the statistics being built up.
* @param key key of this statistic
* @param source atomic counter
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: atomic long counter.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

will fix

/**
* Add a new evaluator to the statistics being built up.
* @param key key of this statistic
* @param source atomic counter
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: atomic integer counter. Since this is the only difference , I think we should mention.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

will fix

assertThat(iterator.hasNext())
.describedAs("iterator.hasNext()")
.isFalse();
assertThatThrownBy(iterator::next);
Copy link
Contributor

Choose a reason for hiding this comment

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

Should be asserting NoSuchElementException()

assertStatisticUntracked(stats, "anything");

// These actually test the assertion coverage
assertThatThrownBy(() ->
Copy link
Contributor

Choose a reason for hiding this comment

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

Do you mind explaining a bit more for this. Thanks

Copy link
Contributor Author

Choose a reason for hiding this comment

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

will add javadocs & pull into own test cases

The S3A rework is somewhat independent, and I think we could look at
to see what could be done to actually cut down the diff
(I have a cunning plan).

Change-Id: I5e72fb07af93506192f856d34c7591a1cf7d0ddb
Change-Id: I21fc73220de7ff3cf26bf3fde528c7b973878626
This adds stats collection from the AWS SDK. Failing as we
can't set an endpoint on a builder-created s3 client instance, yet
we need to use the builder to pass in the metrics. PITA

Change-Id: Ia217e1151720b25e924f32558f821e7cfc7121a9
Change-Id: I3883fffa076f84cc0c4733bb1aa772a535095688
* just go for the simpler null/non-null API
* writing unit tests and the helper assertions needed, initially
  just with an empty stats list

Change-Id: I937562cba2facb663bf3097ed904a4d9147c8fdb
TODO: support in local FS & tests
* tweak of the EmptyIOStatistics.
  We do need tests of the dynamic stuff next
* S3A stats -findbugs changed to turn off complaints about volatile
* AWS region binding code is all broken; landsat tests fail

Change-Id: I734847cfed8979780377817560b818947768efc5
Adding first functional test for S3A stats. Broken :(

Change-Id: I7d1a00f0fb8b7578dc9e1571b68af73c2f6320e7
Most of the work here is on getting the AWS Region binding code
working again. I Don't understand it yet, but this mimics
the old AWS SDK codepath.

Test-wise:

* block output stream counts bytes written
as they are buffered to blocks; this is the statistic which is
collected and examined.

* network binding test not updated w.r.t changes
in endpoint/region mapping

Change-Id: Ia7392dd113772b5917c6990b966c1b7a28f38f20
* IOStatistics has a keys() method to return all the keys
* moved S3A input stream the statistics to atomic longs (biggest change)
* S3AInputStream no longer counts bite skipped during a seat within a buffer
  as read()...that is the bytes read counter is purely of bytes read by
  the application
* renaming common statistic names so that they are more consistent

Change-Id: Ica60840cf8a3c256cbb17fdf2ef79415df7ca7d6
Getting in the AWS metrics wired up is a lot harder Danny should be because the switch to the Builder API for the AWS S3 client seems to break most of the endpoint binding logic -this surfaces in failed tests which explicitly look at the data in US West two and elsewhere. We could fix the tests -but as they document a regression, it is better to do what we can to make the problem go away.

Change-Id: I283aa6ad125df052f4b1dfca6308931ab7441a4c
@apache apache deleted a comment from hadoop-yetus Apr 27, 2020
@apache apache deleted a comment from hadoop-yetus Apr 27, 2020
@apache apache deleted a comment from hadoop-yetus Apr 27, 2020
@apache apache deleted a comment from hadoop-yetus Apr 27, 2020
Change-Id: I4ae3e39043ac5eddd4883fb1be846c43ae0d58b1
@steveloughran
Copy link
Contributor Author

@mukund-thakur thanks for the review -tried to address your comments. Look in the package-info for some docs on how I imagine these to be used.

Because the code doesn't compile, I'm going to rebase on trunk and submit that as a new PR

That PR will revert back to the current S3A code to create an S3 client, because the builder mechanism needed to wire up the AWS metrics collection in the SDK is failing some of the tests -it is complex enough that it's slowing down the rest of the patch. We can add that later.


[ERROR] testJobSubmissionCollectsTokens[2](org.apache.hadoop.fs.s3a.auth.delegation.ITestDelegatedMRJob)  Time elapsed: 8.485 s  <<< ERROR!
org.apache.hadoop.fs.s3a.AWSRedirectException: getFileStatus on s3a://osm-pds/planet/planet-latest.orc#_partition.lst: com.amazonaws.services.s3.model.AmazonS3Exception: The bucket is in this region: us-east-1. Please use this region to retry the request (Service: Amazon S3; Status Code: 301; Error Code: 301 Moved Permanently; Request ID: E6941931CE0008E4; S3 Extended Request ID: TkeYaR0Vf0nc/l6QgPLaKBO7E956CFqZYoSjTy4lLuqcQMQ0+U2gvnkewhAQan+0oYOT7UNeZF0=), S3 Extended Request ID: TkeYaR0Vf0nc/l6QgPLaKBO7E956CFqZYoSjTy4lLuqcQMQ0+U2gvnkewhAQan+0oYOT7UNeZF0=:301 Moved Permanently: The bucket is in this region: us-east-1. Please use this region to retry the request (Service: Amazon S3; Status Code: 301; Error Code: 301 Moved Permanently; Request ID: E6941931CE0008E4; S3 Extended Request ID: TkeYaR0Vf0nc/l6QgPLaKBO7E956CFqZYoSjTy4lLuqcQMQ0+U2gvnkewhAQan+0oYOT7UNeZF0=)
	at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:234)
	at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:168)
	at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:3019)
	at org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:2937)
	at org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:2821)
	at org.apache.hadoop.mapreduce.filecache.ClientDistributedCacheManager.getFileStatus(ClientDistributedCacheManager.java:325)
	at org.apache.hadoop.mapreduce.filecache.ClientDistributedCacheManager.getFileStatus(ClientDistributedCacheManager.java:236)
	at org.apache.hadoop.mapreduce.filecache.ClientDistributedCacheManager.determineTimestamps(ClientDistributedCacheManager.java:105)
	at org.apache.hadoop.mapreduce.filecache.ClientDistributedCacheManager.determineTimestampsAndCacheVisibilities(ClientDistributedCacheManager.java:69)
	at org.apache.hadoop.mapreduce.JobResourceUploader.uploadResourcesInternal(JobResourceUploader.java:222)
	at org.apache.hadoop.mapreduce.JobResourceUploader.uploadResources(JobResourceUploader.java:135)
	at org.apache.hadoop.mapreduce.JobSubmitter.copyAndConfigureFiles(JobSubmitter.java:99)
	at org.apache.hadoop.mapreduce.JobSubmitter.submitJobInternal(JobSubmitter.java:194)
	at org.apache.hadoop.mapreduce.Job$11.run(Job.java:1576)
	at org.apache.hadoop.mapreduce.Job$11.run(Job.java:1573)
	at java.security.AccessController.doPrivileged(Native Method)
	at javax.security.auth.Subject.doAs(Subject.java:422)
	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1845)
	at org.apache.hadoop.mapreduce.Job.submit(Job.java:1573)
	at org.apache.hadoop.fs.s3a.auth.delegation.ITestDelegatedMRJob.testJobSubmissionCollectsTokens(ITestDelegatedMRJob.java:286)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
	at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
	at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:298)
	at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:292)
	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
	at java.lang.Thread.run(Thread.java:748)
Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: The bucket is in this region: us-east-1. Please use this region to retry the request (Service: Amazon S3; Status Code: 301; Error Code: 301 Moved Permanently; Request ID: E6941931CE0008E4; S3 Extended Request ID: TkeYaR0Vf0nc/l6QgPLaKBO7E956CFqZYoSjTy4lLuqcQMQ0+U2gvnkewhAQan+0oYOT7UNeZF0=), S3 Extended Request ID: TkeYaR0Vf0nc/l6QgPLaKBO7E956CFqZYoSjTy4lLuqcQMQ0+U2gvnkewhAQan+0oYOT7UNeZF0=
	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1712)
	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1367)
	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1113)
	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:770)
	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:744)
	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:726)
	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:686)
	at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:668)
	at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:532)
	at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:512)
	at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4920)
	at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4866)
	at com.amazonaws.services.s3.AmazonS3Client.getObjectMetadata(AmazonS3Client.java:1320)
	at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$getObjectMetadata$6(S3AFileSystem.java:1899)
	at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:407)
	at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:370)
	at org.apache.hadoop.fs.s3a.S3AFileSystem.getObjectMetadata(S3AFileSystem.java:1892)
	at org.apache.hadoop.fs.s3a.S3AFileSystem.getObjectMetadata(S3AFileSystem.java:1868)
	at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:3006)
	... 32 more

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Comment
+0 🆗 reexec 1m 9s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 1s No case conflicting files found.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
+1 💚 test4tests 0m 0s The patch appears to include 21 new or modified test files.
_ trunk Compile Tests _
+0 🆗 mvndep 0m 46s Maven dependency ordering for branch
+1 💚 mvninstall 22m 11s trunk passed
+1 💚 compile 17m 59s trunk passed
+1 💚 checkstyle 2m 54s trunk passed
+1 💚 mvnsite 2m 10s trunk passed
+1 💚 shadedclient 21m 4s branch has no errors when building and testing our client artifacts.
+1 💚 javadoc 1m 29s trunk passed
+0 🆗 spotbugs 1m 7s Used deprecated FindBugs config; considering switching to SpotBugs.
+1 💚 findbugs 3m 12s trunk passed
-0 ⚠️ patch 1m 25s Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.
_ Patch Compile Tests _
+0 🆗 mvndep 0m 22s Maven dependency ordering for patch
+1 💚 mvninstall 1m 22s the patch passed
+1 💚 compile 17m 19s the patch passed
+1 💚 javac 17m 19s root generated 0 new + 1870 unchanged - 1 fixed = 1870 total (was 1871)
-0 ⚠️ checkstyle 2m 54s root: The patch generated 69 new + 100 unchanged - 19 fixed = 169 total (was 119)
+1 💚 mvnsite 2m 7s the patch passed
+1 💚 whitespace 0m 0s The patch has no whitespace issues.
+1 💚 xml 0m 1s The patch has no ill-formed XML file.
+1 💚 shadedclient 15m 8s patch has no errors when building and testing our client artifacts.
-1 ❌ javadoc 0m 54s hadoop-common-project_hadoop-common generated 1 new + 101 unchanged - 0 fixed = 102 total (was 101)
+1 💚 findbugs 3m 28s the patch passed
_ Other Tests _
-1 ❌ unit 9m 16s hadoop-common in the patch passed.
-1 ❌ unit 1m 38s hadoop-aws in the patch passed.
+1 💚 asflicense 0m 45s The patch does not generate ASF License warnings.
127m 17s
Reason Tests
Failed junit tests hadoop.io.compress.snappy.TestSnappyCompressorDecompressor
hadoop.io.compress.TestCompressorDecompressor
hadoop.fs.s3a.impl.TestNetworkBinding
Subsystem Report/Notes
Docker ClientAPI=1.40 ServerAPI=1.40 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1820/22/artifact/out/Dockerfile
GITHUB PR #1820
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle xml
uname Linux cac932c6d339 4.15.0-74-generic #84-Ubuntu SMP Thu Dec 19 08:06:28 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality personality/hadoop.sh
git revision trunk / 18d7dfb
Default Java Private Build-1.8.0_252-8u252-b09-1~18.04-b09
checkstyle https://builds.apache.org/job/hadoop-multibranch/job/PR-1820/22/artifact/out/diff-checkstyle-root.txt
javadoc https://builds.apache.org/job/hadoop-multibranch/job/PR-1820/22/artifact/out/diff-javadoc-javadoc-hadoop-common-project_hadoop-common.txt
unit https://builds.apache.org/job/hadoop-multibranch/job/PR-1820/22/artifact/out/patch-unit-hadoop-common-project_hadoop-common.txt
unit https://builds.apache.org/job/hadoop-multibranch/job/PR-1820/22/artifact/out/patch-unit-hadoop-tools_hadoop-aws.txt
Test Results https://builds.apache.org/job/hadoop-multibranch/job/PR-1820/22/testReport/
Max. process+thread count 1345 (vs. ulimit of 5500)
modules C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-aws U: .
Console output https://builds.apache.org/job/hadoop-multibranch/job/PR-1820/22/console
versions git=2.17.1 maven=3.6.0 findbugs=3.1.0-RC1
Powered by Apache Yetus 0.12.0 https://yetus.apache.org

This message was automatically generated.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
enhancement fs/s3 changes related to hadoop-aws; submitter must declare test endpoint work in progress PRs still Work in Progress; reviews not expected but still welcome
Projects
None yet
3 participants