Skip to content

Conversation

@steveloughran
Copy link
Contributor

Description of PR

HADOOP-19485. S3A: Upgrade AWS V2 SDK to 2.29.52
HADOOP-19485. S3A: Test failures after SDK upgrade
HADOOP-19455. S3A: SDK client to log metrics at TRACE
HADOOP-19492. S3A: Some tests failing on third-party stores
HADOOP-19516. S3A: SDK reads content twice during PUT to S3 Express store.

How was this patch tested?

s3 london

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?

This is the latest release of the AWS SDK which does not make
changes that are incompatible with third-party stores.

Any followup will require addressing HADOOP-19490.

Change-Id: I51bccc4376681b6b3058060b5aa8af9730886e12
Code changes related to HADOOP-19485.

AwsSdkWorkarounds no longer needs to cut back on transfer manager logging
(HADOOP-19272).
- Remove log downgrade and change assertion to expect nothing to be logged.
- remove false positives from log

ITestS3AEndpointRegion failure:

Change in state of AwsExecutionAttribute.ENDPOINT_OVERRIDDEN
attribute requires test tuning to match.

Change-Id: Iee88e13c113daf852c82c4a11c5029a62efabb71
Switch is in client; commented out in test log properties;
covered in troubleshooting doc

Change-Id: If70447d8eb3d3d0e03db5c169cd1aabf844931bd
* Includes fix for the assumeStoreAwsHosted() logic.
* Documents how to turn off multipart uploads with third-party stores

Change-Id: Iae344b372dceaca981426035e062b542af25f0cd
…tore.

Downgrade warning message from INFO to DEBUG.

Change-Id: Ibb7be344e44e2d66d779c40bf086edba5ce7d7d8
@steveloughran
Copy link
Contributor Author

one failure I'm getting on the tests (other than the known v4 signing regression) is that the vector IO tests are failing 412 "not satisfied".

going to rebuild without the library update to see if the issue exists there too.

[ERROR] testVectoredReadAfterNormalRead[Buffer type : direct](org.apache.hadoop.fs.contract.s3a.ITestS3AContractAnalyticsStreamVectoredRead)  Time elapsed: 3.967 s  <<< ERROR!
java.io.IOException: Cannot read block file
        at software.amazon.s3.analyticsaccelerator.io.physical.data.Block.getDataWithRetries(Block.java:239)
        at software.amazon.s3.analyticsaccelerator.io.physical.data.Block.read(Block.java:187)
        at software.amazon.s3.analyticsaccelerator.io.physical.data.Blob.read(Blob.java:112)
        at software.amazon.s3.analyticsaccelerator.io.physical.impl.PhysicalIOImpl.lambda$read$3(PhysicalIOImpl.java:162)
        at software.amazon.s3.analyticsaccelerator.common.telemetry.DefaultTelemetry.measure(DefaultTelemetry.java:102)
        at software.amazon.s3.analyticsaccelerator.common.telemetry.Telemetry.measureVerbose(Telemetry.java:243)
        at software.amazon.s3.analyticsaccelerator.io.physical.impl.PhysicalIOImpl.read(PhysicalIOImpl.java:151)
        at software.amazon.s3.analyticsaccelerator.io.logical.impl.DefaultLogicalIOImpl.lambda$read$1(DefaultLogicalIOImpl.java:92)
        at software.amazon.s3.analyticsaccelerator.common.telemetry.DefaultTelemetry.measureConditionally(DefaultTelemetry.java:141)
        at software.amazon.s3.analyticsaccelerator.io.logical.impl.DefaultLogicalIOImpl.read(DefaultLogicalIOImpl.java:81)
        at software.amazon.s3.analyticsaccelerator.S3SeekableInputStream.lambda$read$3(S3SeekableInputStream.java:155)
        at software.amazon.s3.analyticsaccelerator.common.telemetry.DefaultTelemetry.measure(DefaultTelemetry.java:102)
        at software.amazon.s3.analyticsaccelerator.common.telemetry.Telemetry.measureVerbose(Telemetry.java:243)
        at software.amazon.s3.analyticsaccelerator.S3SeekableInputStream.read(S3SeekableInputStream.java:145)
        at org.apache.hadoop.fs.s3a.impl.streams.AnalyticsStream.read(AnalyticsStream.java:123)
        at java.io.DataInputStream.readFully(DataInputStream.java:195)
        at org.apache.hadoop.fs.contract.AbstractContractVectoredReadTest.testVectoredReadAfterNormalRead(AbstractContractVectoredReadTest.java:490)
        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:59)
        at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
        at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
        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:61)
        at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:299)
        at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:293)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at java.lang.Thread.run(Thread.java:750)
Caused by: software.amazon.awssdk.services.s3.model.S3Exception: At least one of the pre-conditions you specified did not hold (Service: S3, Status Code: 412, Request ID: JPZ41WS486WJK5JB, Extended Request ID: M9cC1AT7UK5wKNwQzvq6hYMBvY/syM3cTF2L6sCsnkasjNBLCCBOcYLRdyeqKXdhZyab1rXsagHm7R8rUqi/+hU4FAGXHlxj)
        at software.amazon.awssdk.services.s3.model.S3Exception$BuilderImpl.build(S3Exception.java:104)
        at software.amazon.awssdk.services.s3.model.S3Exception$BuilderImpl.build(S3Exception.java:58)
        at software.amazon.awssdk.protocols.query.internal.unmarshall.AwsXmlErrorUnmarshaller.unmarshall(AwsXmlErrorUnmarshaller.java:98)
        at software.amazon.awssdk.protocols.query.unmarshall.AwsXmlErrorProtocolUnmarshaller.handle(AwsXmlErrorProtocolUnmarshaller.java:102)
        at software.amazon.awssdk.protocols.query.unmarshall.AwsXmlErrorProtocolUnmarshaller.handle(AwsXmlErrorProtocolUnmarshaller.java:82)
        at software.amazon.awssdk.core.http.MetricCollectingHttpResponseHandler.lambda$handle$0(MetricCollectingHttpResponseHandler.java:52)
        at software.amazon.awssdk.core.internal.util.MetricUtils.measureDurationUnsafe(MetricUtils.java:102)
        at software.amazon.awssdk.core.internal.util.MetricUtils.measureDurationUnsafe(MetricUtils.java:95)
        at software.amazon.awssdk.core.http.MetricCollectingHttpResponseHandler.handle(MetricCollectingHttpResponseHandler.java:52)
        at software.amazon.awssdk.core.internal.http.async.AsyncResponseHandler.lambda$prepare$0(AsyncResponseHandler.java:92)
        at java.util.concurrent.CompletableFuture.uniCompose(CompletableFuture.java:966)
        at java.util.concurrent.CompletableFuture$UniCompose.tryFire(CompletableFuture.java:940)
        at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
        at java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975)
        at software.amazon.awssdk.core.internal.http.async.AsyncResponseHandler$BaosSubscriber.onComplete(AsyncResponseHandler.java:135)
        at software.amazon.awssdk.core.internal.metrics.BytesReadTrackingPublisher$BytesReadTracker.onComplete(BytesReadTrackingPublisher.java:74)
        at software.amazon.awssdk.http.nio.netty.internal.ResponseHandler$DataCountingPublisher$1.onComplete(ResponseHandler.java:519)
        at software.amazon.awssdk.http.nio.netty.internal.ResponseHandler.runAndLogError(ResponseHandler.java:254)
        at software.amazon.awssdk.http.nio.netty.internal.ResponseHandler.access$600(ResponseHandler.java:77)
        at software.amazon.awssdk.http.nio.netty.internal.ResponseHandler$PublisherAdapter$1.onComplete(ResponseHandler.java:375)
        at software.amazon.awssdk.http.nio.netty.internal.nrs.HandlerPublisher.publishMessage(HandlerPublisher.java:402)
        at software.amazon.awssdk.http.nio.netty.internal.nrs.HandlerPublisher.flushBuffer(HandlerPublisher.java:338)
        at software.amazon.awssdk.http.nio.netty.internal.nrs.HandlerPublisher.receivedDemand(HandlerPublisher.java:291)
        at software.amazon.awssdk.http.nio.netty.internal.nrs.HandlerPublisher.access$200(HandlerPublisher.java:61)
        at software.amazon.awssdk.http.nio.netty.internal.nrs.HandlerPublisher$ChannelSubscription$1.run(HandlerPublisher.java:495)
        at software.amazon.awssdk.thirdparty.io.netty.util.concurrent.AbstractEventExecutor.runTask(AbstractEventExecutor.java:173)
        at software.amazon.awssdk.thirdparty.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:166)
        at software.amazon.awssdk.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:472)
        at software.amazon.awssdk.thirdparty.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:566)
        at software.amazon.awssdk.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)
        at software.amazon.awssdk.thirdparty.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
        ... 1 more

[ERROR] testVectoredReadAfterNormalRead[Buffer type : array](org.apache.hadoop.fs.contract.s3a.ITestS3AContractAnalyticsStreamVectoredRead)  Time elapsed: 5.592 s  <<< ERROR!
java.io.IOException: Cannot read block file
        at software.amazon.s3.analyticsaccelerator.io.physical.data.Block.getDataWithRetries(Block.java:239)
        at software.amazon.s3.analyticsaccelerator.io.physical.data.Block.read(Block.java:187)
        at software.amazon.s3.analyticsaccelerator.io.physical.data.Blob.read(Blob.java:112)
        at software.amazon.s3.analyticsaccelerator.io.physical.impl.PhysicalIOImpl.lambda$read$3(PhysicalIOImpl.java:162)
        at software.amazon.s3.analyticsaccelerator.common.telemetry.DefaultTelemetry.measure(DefaultTelemetry.java:102)
        at software.amazon.s3.analyticsaccelerator.common.telemetry.Telemetry.measureVerbose(Telemetry.java:243)
        at software.amazon.s3.analyticsaccelerator.io.physical.impl.PhysicalIOImpl.read(PhysicalIOImpl.java:151)
        at software.amazon.s3.analyticsaccelerator.io.logical.impl.DefaultLogicalIOImpl.lambda$read$1(DefaultLogicalIOImpl.java:92)
        at software.amazon.s3.analyticsaccelerator.common.telemetry.DefaultTelemetry.measureConditionally(DefaultTelemetry.java:141)
        at software.amazon.s3.analyticsaccelerator.io.logical.impl.DefaultLogicalIOImpl.read(DefaultLogicalIOImpl.java:81)
        at software.amazon.s3.analyticsaccelerator.S3SeekableInputStream.lambda$read$3(S3SeekableInputStream.java:155)
        at software.amazon.s3.analyticsaccelerator.common.telemetry.DefaultTelemetry.measure(DefaultTelemetry.java:102)
        at software.amazon.s3.analyticsaccelerator.common.telemetry.Telemetry.measureVerbose(Telemetry.java:243)
        at software.amazon.s3.analyticsaccelerator.S3SeekableInputStream.read(S3SeekableInputStream.java:145)
        at org.apache.hadoop.fs.s3a.impl.streams.AnalyticsStream.read(AnalyticsStream.java:123)
        at java.io.DataInputStream.readFully(DataInputStream.java:195)
        at org.apache.hadoop.fs.contract.AbstractContractVectoredReadTest.testVectoredReadAfterNormalRead(AbstractContractVectoredReadTest.java:490)
        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:59)
        at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
        at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
        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:61)
        at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:299)
        at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:293)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at java.lang.Thread.run(Thread.java:750)
Caused by: software.amazon.awssdk.services.s3.model.S3Exception: At least one of the pre-conditions you specified did not hold (Service: S3, Status Code: 412, Request ID: 9YKDJ797E6XNRPY1, Extended Request ID: XatMA7lK+Ww1G1oyMoYRZOf2kSta5/+wkzHW7HXbAJu227n4vPx6RaJmFltbGg6p96hKSy5WnX/e2XgX9x0L/5YE8XGHAHyA)
        at software.amazon.awssdk.services.s3.model.S3Exception$BuilderImpl.build(S3Exception.java:104)
        at software.amazon.awssdk.services.s3.model.S3Exception$BuilderImpl.build(S3Exception.java:58)
        at software.amazon.awssdk.protocols.query.internal.unmarshall.AwsXmlErrorUnmarshaller.unmarshall(AwsXmlErrorUnmarshaller.java:98)
        at software.amazon.awssdk.protocols.query.unmarshall.AwsXmlErrorProtocolUnmarshaller.handle(AwsXmlErrorProtocolUnmarshaller.java:102)
        at software.amazon.awssdk.protocols.query.unmarshall.AwsXmlErrorProtocolUnmarshaller.handle(AwsXmlErrorProtocolUnmarshaller.java:82)
        at software.amazon.awssdk.core.http.MetricCollectingHttpResponseHandler.lambda$handle$0(MetricCollectingHttpResponseHandler.java:52)
        at software.amazon.awssdk.core.internal.util.MetricUtils.measureDurationUnsafe(MetricUtils.java:102)
        at software.amazon.awssdk.core.internal.util.MetricUtils.measureDurationUnsafe(MetricUtils.java:95)
        at software.amazon.awssdk.core.http.MetricCollectingHttpResponseHandler.handle(MetricCollectingHttpResponseHandler.java:52)
        at software.amazon.awssdk.core.internal.http.async.AsyncResponseHandler.lambda$prepare$0(AsyncResponseHandler.java:92)
        at java.util.concurrent.CompletableFuture.uniCompose(CompletableFuture.java:966)
        at java.util.concurrent.CompletableFuture$UniCompose.tryFire(CompletableFuture.java:940)
        at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
        at java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975)
        at software.amazon.awssdk.core.internal.http.async.AsyncResponseHandler$BaosSubscriber.onComplete(AsyncResponseHandler.java:135)
        at software.amazon.awssdk.core.internal.metrics.BytesReadTrackingPublisher$BytesReadTracker.onComplete(BytesReadTrackingPublisher.java:74)
        at software.amazon.awssdk.http.nio.netty.internal.ResponseHandler$DataCountingPublisher$1.onComplete(ResponseHandler.java:519)
        at software.amazon.awssdk.http.nio.netty.internal.ResponseHandler.runAndLogError(ResponseHandler.java:254)
        at software.amazon.awssdk.http.nio.netty.internal.ResponseHandler.access$600(ResponseHandler.java:77)
        at software.amazon.awssdk.http.nio.netty.internal.ResponseHandler$PublisherAdapter$1.onComplete(ResponseHandler.java:375)
        at software.amazon.awssdk.http.nio.netty.internal.nrs.HandlerPublisher.complete(HandlerPublisher.java:447)
        at software.amazon.awssdk.http.nio.netty.internal.nrs.HandlerPublisher.handlerRemoved(HandlerPublisher.java:435)
        at software.amazon.awssdk.thirdparty.io.netty.channel.AbstractChannelHandlerContext.callHandlerRemoved(AbstractChannelHandlerContext.java:1138)
        at software.amazon.awssdk.thirdparty.io.netty.channel.DefaultChannelPipeline.callHandlerRemoved0(DefaultChannelPipeline.java:586)
        at software.amazon.awssdk.thirdparty.io.netty.channel.DefaultChannelPipeline.remove(DefaultChannelPipeline.java:426)
        at software.amazon.awssdk.thirdparty.io.netty.channel.DefaultChannelPipeline.remove(DefaultChannelPipeline.java:372)
        at software.amazon.awssdk.http.nio.netty.internal.nrs.HttpStreamsHandler.removeHandlerIfActive(HttpStreamsHandler.java:370)
        at software.amazon.awssdk.http.nio.netty.internal.nrs.HttpStreamsHandler.handleReadHttpContent(HttpStreamsHandler.java:232)
        at software.amazon.awssdk.http.nio.netty.internal.nrs.HttpStreamsHandler.channelRead(HttpStreamsHandler.java:203)
        at software.amazon.awssdk.http.nio.netty.internal.nrs.HttpStreamsClientHandler.channelRead(HttpStreamsClientHandler.java:173)
        at software.amazon.awssdk.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:442)
        at software.amazon.awssdk.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
        at software.amazon.awssdk.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
        at software.amazon.awssdk.thirdparty.io.netty.handler.logging.LoggingHandler.channelRead(LoggingHandler.java:280)
        at software.amazon.awssdk.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:442)
        at software.amazon.awssdk.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
        at software.amazon.awssdk.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
        at software.amazon.awssdk.thirdparty.io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:289)
        at software.amazon.awssdk.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:442)
        at software.amazon.awssdk.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
        at software.amazon.awssdk.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
        at software.amazon.awssdk.thirdparty.io.netty.channel.CombinedChannelDuplexHandler$DelegatingChannelHandlerContext.fireChannelRead(CombinedChannelDuplexHandler.java:436)
        at software.amazon.awssdk.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:346)
        at software.amazon.awssdk.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:318)
        at software.amazon.awssdk.thirdparty.io.netty.channel.CombinedChannelDuplexHandler.channelRead(CombinedChannelDuplexHandler.java:251)
        at software.amazon.awssdk.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:442)
        at software.amazon.awssdk.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
        at software.amazon.awssdk.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
        at software.amazon.awssdk.thirdparty.io.netty.handler.ssl.SslHandler.unwrap(SslHandler.java:1503)
        at software.amazon.awssdk.thirdparty.io.netty.handler.ssl.SslHandler.decodeJdkCompatible(SslHandler.java:1366)
        at software.amazon.awssdk.thirdparty.io.netty.handler.ssl.SslHandler.decode(SslHandler.java:1415)
        at software.amazon.awssdk.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.decodeRemovalReentryProtection(ByteToMessageDecoder.java:530)
        at software.amazon.awssdk.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:469)
        at software.amazon.awssdk.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:290)
        at software.amazon.awssdk.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:444)
        at software.amazon.awssdk.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
        at software.amazon.awssdk.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
        at software.amazon.awssdk.thirdparty.io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:289)
        at software.amazon.awssdk.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:442)
        at software.amazon.awssdk.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
        at software.amazon.awssdk.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
        at software.amazon.awssdk.thirdparty.io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1357)
        at software.amazon.awssdk.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:440)
        at software.amazon.awssdk.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
        at software.amazon.awssdk.thirdparty.io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:868)
        at software.amazon.awssdk.thirdparty.io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:166)
        at software.amazon.awssdk.thirdparty.io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:788)
        at software.amazon.awssdk.thirdparty.io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:724)
        at software.amazon.awssdk.thirdparty.io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:650)
        at software.amazon.awssdk.thirdparty.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:562)
        at software.amazon.awssdk.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)
        at software.amazon.awssdk.thirdparty.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
        ... 1 more

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 11m 42s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 0s No case conflicting files found.
+0 🆗 codespell 0m 0s codespell was not available.
+0 🆗 detsecrets 0m 0s detect-secrets was not available.
+0 🆗 xmllint 0m 0s xmllint was not available.
+0 🆗 markdownlint 0m 0s markdownlint was not available.
+0 🆗 shelldocs 0m 0s Shelldocs was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
+1 💚 test4tests 0m 0s The patch appears to include 8 new or modified test files.
_ branch-3.4 Compile Tests _
+0 🆗 mvndep 1m 59s Maven dependency ordering for branch
+1 💚 mvninstall 33m 22s branch-3.4 passed
+1 💚 compile 16m 33s branch-3.4 passed with JDK Ubuntu-11.0.26+4-post-Ubuntu-1ubuntu120.04
+1 💚 compile 14m 55s branch-3.4 passed with JDK Private Build-1.8.0_442-8u442-b06us1-0ubuntu120.04-b06
+1 💚 checkstyle 4m 6s branch-3.4 passed
+1 💚 mvnsite 17m 15s branch-3.4 passed
+1 💚 javadoc 8m 18s branch-3.4 passed with JDK Ubuntu-11.0.26+4-post-Ubuntu-1ubuntu120.04
+1 💚 javadoc 7m 25s branch-3.4 passed with JDK Private Build-1.8.0_442-8u442-b06us1-0ubuntu120.04-b06
+0 🆗 spotbugs 0m 20s branch/hadoop-project no spotbugs output file (spotbugsXml.xml)
-1 ❌ spotbugs 29m 44s /branch-spotbugs-root-warnings.html root in branch-3.4 has 1 extant spotbugs warnings.
+1 💚 shadedclient 34m 49s branch has no errors when building and testing our client artifacts.
_ Patch Compile Tests _
+0 🆗 mvndep 0m 57s Maven dependency ordering for patch
+1 💚 mvninstall 30m 38s the patch passed
+1 💚 compile 15m 52s the patch passed with JDK Ubuntu-11.0.26+4-post-Ubuntu-1ubuntu120.04
+1 💚 javac 15m 52s the patch passed
+1 💚 compile 15m 10s the patch passed with JDK Private Build-1.8.0_442-8u442-b06us1-0ubuntu120.04-b06
+1 💚 javac 15m 10s the patch passed
+1 💚 blanks 0m 0s The patch has no blanks issues.
+1 💚 checkstyle 4m 7s root: The patch generated 0 new + 11 unchanged - 1 fixed = 11 total (was 12)
+1 💚 mvnsite 17m 47s the patch passed
+1 💚 shellcheck 0m 0s No new issues.
+1 💚 javadoc 8m 8s the patch passed with JDK Ubuntu-11.0.26+4-post-Ubuntu-1ubuntu120.04
+1 💚 javadoc 7m 23s the patch passed with JDK Private Build-1.8.0_442-8u442-b06us1-0ubuntu120.04-b06
+0 🆗 spotbugs 0m 20s hadoop-project has no data from spotbugs
+1 💚 shadedclient 61m 5s patch has no errors when building and testing our client artifacts.
_ Other Tests _
+1 💚 unit 730m 25s root in the patch passed.
+1 💚 asflicense 1m 51s The patch does not generate ASF License warnings.
1081m 7s
Subsystem Report/Notes
Docker ClientAPI=1.48 ServerAPI=1.48 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7573/1/artifact/out/Dockerfile
GITHUB PR #7573
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient codespell detsecrets xmllint spotbugs checkstyle markdownlint shellcheck shelldocs
uname Linux 40ab38e987b1 5.15.0-134-generic #145-Ubuntu SMP Wed Feb 12 20:08:39 UTC 2025 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision branch-3.4 / a13da89
Default Java Private Build-1.8.0_442-8u442-b06us1-0ubuntu120.04-b06
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.26+4-post-Ubuntu-1ubuntu120.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_442-8u442-b06us1-0ubuntu120.04-b06
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7573/1/testReport/
Max. process+thread count 4261 (vs. ulimit of 5500)
modules C: hadoop-project hadoop-tools/hadoop-aws . U: .
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7573/1/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2 shellcheck=0.7.0
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

@ahmarsuhail
Copy link
Contributor

looking at this failure now

@steveloughran
Copy link
Contributor Author

merge is done now; closing

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.

3 participants