aws-sdk-java-v2 icon indicating copy to clipboard operation
aws-sdk-java-v2 copied to clipboard

XMLStreamException should be configured as a retryable exception

Open drewschleit opened this issue 1 year ago • 4 comments

Describe the bug

My Apache Iceberg workload failed due to an error from a single S3 call, which wasn't retried by the SDK client.

It looks like, while the client was reading an error XML body, it encountered a Connection reset on the socket. This error condition should be considered retryable and automatically retried according to the default retry policy.

Although IOException is configured as a retryable error, it was wrapped in an XMLStreamException which is not configured as retryable.

Expected Behavior

A Connection reset error during an SDK call should be retried according to the default retry policy.

Current Behavior

A Connection reset error which occurs during parsing of error XML (such as if a 503 was returned) is not retried.

Reproduction Steps

I don't have a reproduction available. See the following stack trace.

reason=ExceptionFailure(software.amazon.awssdk.core.exception.SdkClientException,Could not parse XML response.,[Ljava.lang.StackTraceElement;@5f41a94,software.amazon.awssdk.core.exception.SdkClientException: Could not parse XML response.
	at software.amazon.awssdk.core.exception.SdkClientException$BuilderImpl.build(SdkClientException.java:111)
	at software.amazon.awssdk.core.exception.SdkClientException.create(SdkClientException.java:47)
	at software.amazon.awssdk.protocols.query.unmarshall.XmlDomParser.parse(XmlDomParser.java:59)
	at software.amazon.awssdk.protocols.xml.internal.unmarshall.XmlResponseParserUtils.parse(XmlResponseParserUtils.java:64)
	at software.amazon.awssdk.protocols.xml.internal.unmarshall.AwsXmlPredicatedResponseHandler.parseResponse(AwsXmlPredicatedResponseHandler.java:116)
	at software.amazon.awssdk.protocols.xml.internal.unmarshall.AwsXmlPredicatedResponseHandler.handleResponse(AwsXmlPredicatedResponseHandler.java:96)
	at software.amazon.awssdk.protocols.xml.internal.unmarshall.AwsXmlPredicatedResponseHandler.handle(AwsXmlPredicatedResponseHandler.java:85)
	at software.amazon.awssdk.protocols.xml.internal.unmarshall.AwsXmlPredicatedResponseHandler.handle(AwsXmlPredicatedResponseHandler.java:43)
	at software.amazon.awssdk.awscore.client.handler.AwsSyncClientHandler$Crc32ValidationResponseHandler.handle(AwsSyncClientHandler.java:93)
	at software.amazon.awssdk.core.internal.handler.BaseClientHandler.lambda$successTransformationResponseHandler$7(BaseClientHandler.java:279)
	at software.amazon.awssdk.core.internal.http.pipeline.stages.HandleResponseStage.execute(HandleResponseStage.java:50)
	at software.amazon.awssdk.core.internal.http.pipeline.stages.HandleResponseStage.execute(HandleResponseStage.java:38)
	at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
	at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:72)
	at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:42)
	at software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:78)
	at software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:40)
	at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:55)
	at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:39)
	at software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:81)
	at software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:36)
	at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
	at software.amazon.awssdk.core.internal.http.StreamManagingStage.execute(StreamManagingStage.java:56)
	at software.amazon.awssdk.core.internal.http.StreamManagingStage.execute(StreamManagingStage.java:36)
	at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.executeWithTimer(ApiCallTimeoutTrackingStage.java:80)
	at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.execute(ApiCallTimeoutTrackingStage.java:60)
	at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.execute(ApiCallTimeoutTrackingStage.java:42)
	at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallMetricCollectionStage.execute(ApiCallMetricCollectionStage.java:50)
	at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallMetricCollectionStage.execute(ApiCallMetricCollectionStage.java:32)
	at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
	at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
	at software.amazon.awssdk.core.internal.http.pipeline.stages.ExecutionFailureExceptionReportingStage.execute(ExecutionFailureExceptionReportingStage.java:37)
	at software.amazon.awssdk.core.internal.http.pipeline.stages.ExecutionFailureExceptionReportingStage.execute(ExecutionFailureExceptionReportingStage.java:26)
	at software.amazon.awssdk.core.internal.http.AmazonSyncHttpClient$RequestExecutionBuilderImpl.execute(AmazonSyncHttpClient.java:224)
	at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.invoke(BaseSyncClientHandler.java:103)
	at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.doExecute(BaseSyncClientHandler.java:173)
	at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.lambda$execute$1(BaseSyncClientHandler.java:80)
	at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.measureApiCallSuccess(BaseSyncClientHandler.java:182)
	at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.execute(BaseSyncClientHandler.java:74)
	at software.amazon.awssdk.core.client.handler.SdkSyncClientHandler.execute(SdkSyncClientHandler.java:45)
	at software.amazon.awssdk.awscore.client.handler.AwsSyncClientHandler.execute(AwsSyncClientHandler.java:53)
	at software.amazon.awssdk.services.s3.DefaultS3Client.completeMultipartUpload(DefaultS3Client.java:727)
	at org.apache.iceberg.util.Tasks$Builder.runTaskWithRetry(Tasks.java:413)
	at org.apache.iceberg.util.Tasks$Builder.runSingleThreaded(Tasks.java:219)
	at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:203)
	at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:196)
	at org.apache.iceberg.aws.s3.S3OutputStream.completeMultiPartUpload(S3OutputStream.java:382)
	at org.apache.iceberg.aws.s3.S3OutputStream.completeUploads(S3OutputStream.java:444)
	at org.apache.iceberg.aws.s3.S3OutputStream.close(S3OutputStream.java:265)
	at org.apache.iceberg.shaded.org.apache.parquet.io.DelegatingPositionOutputStream.close(DelegatingPositionOutputStream.java:38)
	at org.apache.iceberg.shaded.org.apache.parquet.hadoop.ParquetFileWriter.end(ParquetFileWriter.java:1204)
	at org.apache.iceberg.parquet.ParquetWriter.close(ParquetWriter.java:257)
	at org.apache.iceberg.io.DataWriter.close(DataWriter.java:82)
	at org.apache.iceberg.io.RollingFileWriter.closeCurrentWriter(RollingFileWriter.java:122)
	at org.apache.iceberg.io.RollingFileWriter.close(RollingFileWriter.java:147)
	at org.apache.iceberg.io.RollingDataWriter.close(RollingDataWriter.java:32)
	at org.apache.iceberg.io.FanoutWriter.closeWriters(FanoutWriter.java:82)
	at org.apache.iceberg.io.FanoutWriter.close(FanoutWriter.java:74)
	at org.apache.iceberg.io.FanoutDataWriter.close(FanoutDataWriter.java:31)
	at org.apache.iceberg.spark.source.SparkWrite$PartitionedDataWriter.close(SparkWrite.java:794)
	at org.apache.iceberg.spark.source.SparkWrite$PartitionedDataWriter.commit(SparkWrite.java:776)
	at org.apache.spark.sql.execution.datasources.v2.WritingSparkTask.$anonfun$run$1(WriteToDataSourceV2Exec.scala:459)
	at org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1409)
	at org.apache.spark.sql.execution.datasources.v2.WritingSparkTask.run(WriteToDataSourceV2Exec.scala:486)
	at org.apache.spark.sql.execution.datasources.v2.WritingSparkTask.run$(WriteToDataSourceV2Exec.scala:425)
	at org.apache.spark.sql.execution.datasources.v2.DataWritingSparkTask$.run(WriteToDataSourceV2Exec.scala:491)
	at org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec.$anonfun$writeWithV2$2(WriteToDataSourceV2Exec.scala:388)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:93)
	at org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:161)
	at org.apache.spark.scheduler.Task.run(Task.scala:143)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Executor.scala:629)
	at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(SparkErrorUtils.scala:64)
	at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(SparkErrorUtils.scala:61)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:95)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:632)
	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
	at java.base/java.lang.Thread.run(Thread.java:840)
Caused by: org.apache.hadoop.shaded.com.ctc.wstx.exc.WstxIOException: Connection reset
	at org.apache.hadoop.shaded.com.ctc.wstx.sr.StreamScanner.constructFromIOE(StreamScanner.java:653)
	at org.apache.hadoop.shaded.com.ctc.wstx.sr.StreamScanner.loadMore(StreamScanner.java:1017)
	at org.apache.hadoop.shaded.com.ctc.wstx.sr.StreamScanner.getNextAfterWS(StreamScanner.java:848)
	at org.apache.hadoop.shaded.com.ctc.wstx.sr.BasicStreamReader.nextFromProlog(BasicStreamReader.java:2107)
	at org.apache.hadoop.shaded.com.ctc.wstx.sr.BasicStreamReader.next(BasicStreamReader.java:1179)
	at org.apache.hadoop.shaded.com.ctc.wstx.evt.WstxEventReader.nextEvent(WstxEventReader.java:283)
	at software.amazon.awssdk.protocols.query.unmarshall.XmlDomParser.parse(XmlDomParser.java:55)
	... 75 more
Caused by: java.net.SocketException: Connection reset
	at java.base/sun.nio.ch.NioSocketImpl.implRead(NioSocketImpl.java:328)
	at java.base/sun.nio.ch.NioSocketImpl.read(NioSocketImpl.java:355)
	at java.base/sun.nio.ch.NioSocketImpl$1.read(NioSocketImpl.java:808)
	at java.base/java.net.Socket$SocketInputStream.read(Socket.java:966)
	at java.base/sun.security.ssl.SSLSocketInputRecord.read(SSLSocketInputRecord.java:484)
	at java.base/sun.security.ssl.SSLSocketInputRecord.readHeader(SSLSocketInputRecord.java:478)
	at java.base/sun.security.ssl.SSLSocketInputRecord.bytesInCompletePacket(SSLSocketInputRecord.java:70)
	at java.base/sun.security.ssl.SSLSocketImpl.readApplicationRecord(SSLSocketImpl.java:1465)
	at java.base/sun.security.ssl.SSLSocketImpl$AppInputStream.read(SSLSocketImpl.java:1069)
	at org.apache.http.impl.io.SessionInputBufferImpl.streamRead(SessionInputBufferImpl.java:137)
	at org.apache.http.impl.io.SessionInputBufferImpl.fillBuffer(SessionInputBufferImpl.java:153)
	at org.apache.http.impl.io.SessionInputBufferImpl.readLine(SessionInputBufferImpl.java:280)
	at org.apache.http.impl.io.ChunkedInputStream.getChunkSize(ChunkedInputStream.java:261)
	at org.apache.http.impl.io.ChunkedInputStream.nextChunk(ChunkedInputStream.java:222)
	at org.apache.http.impl.io.ChunkedInputStream.read(ChunkedInputStream.java:183)
	at org.apache.http.conn.EofSensorInputStream.read(EofSensorInputStream.java:135)
	at java.base/java.io.FilterInputStream.read(FilterInputStream.java:132)
	at java.base/java.io.FilterInputStream.read(FilterInputStream.java:132)
	at software.amazon.awssdk.core.io.SdkFilterInputStream.read(SdkFilterInputStream.java:66)
	at software.amazon.awssdk.core.internal.metrics.BytesReadTrackingInputStream.read(BytesReadTrackingInputStream.java:49)
	at java.base/java.io.FilterInputStream.read(FilterInputStream.java:132)
	at java.base/java.io.FilterInputStream.read(FilterInputStream.java:132)
	at software.amazon.awssdk.utils.LookaheadInputStream.read(LookaheadInputStream.java:59)
	at java.base/java.io.FilterInputStream.read(FilterInputStream.java:132)
	at software.amazon.awssdk.utils.LookaheadInputStream.read(LookaheadInputStream.java:59)
	at org.apache.hadoop.shaded.com.ctc.wstx.io.BaseReader.readBytes(BaseReader.java:155)
	at org.apache.hadoop.shaded.com.ctc.wstx.io.UTF8Reader.loadMore(UTF8Reader.java:369)
	at org.apache.hadoop.shaded.com.ctc.wstx.io.UTF8Reader.read(UTF8Reader.java:112)
	at org.apache.hadoop.shaded.com.ctc.wstx.io.ReaderSource.readInto(ReaderSource.java:89)
	at org.apache.hadoop.shaded.com.ctc.wstx.io.BranchingReaderSource.readInto(BranchingReaderSource.java:57)
	at org.apache.hadoop.shaded.com.ctc.wstx.sr.StreamScanner.loadMore(StreamScanner.java:1011)
	... 80 more

Possible Solution

Add XMLStreamException to the default list of retryable exceptions in SdkDefaultRetrySetting.RETRYABLE_EXCEPTIONS.

Additional Information/Context

No response

AWS Java SDK version used

2.22.12

JDK version used

Java 8

Operating System and version

EMR Serverless

drewschleit avatar Jul 30 '24 16:07 drewschleit

just compared this to the s3a error code. this stack trace implies that our recursive scan for socket errors will pick this up, which is probably why we've never seen it.

https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java#L171

I'm going be ruthless and say that although the iceberg team consider that their S3 client is superior to the S3A code on account of it being lighter weight -it hasn't been deployed broadly enough to actually encounter many of them more common failures and it clearly contains the assumption that the SDK error handling is sufficient. It isn't

steveloughran avatar Aug 12 '24 14:08 steveloughran

Not all cases of XMLStreamException should be retried (malformed XMLs for example), so this shouldn't be added to the default retryable list.

As you mentioned, the IOException was wrapped in a XMLStreamException. The SDK is behaving as expected.

debora-ito avatar Aug 13 '24 21:08 debora-ito

I disagree that the current behavior is the desired behavior. Certainly if an IOException is encountered in this specific code path, it should be retried, just as IOExceptions which happen in other SDK code paths are retried (it's configured in SdkDefaultRetrySetting.RETRYABLE_EXCEPTIONS, after all). Based on the S3A code that Steve posted above, you can see that your customers are telling you that they want these exceptions to be retried by default.

I'd argue that any malformed XML is likely transient and should be retried as well -- we don't expect for AWS services to persistently serve malformed XML. Such as event would either be due to corruption, or some sort of server-side bug, both of which are expected to be transient.

Regardless, if we can identify XML stream exceptions which are definitively not retryable, then the solution here would be to introduce a separate exception for those cases. Or, alternatively, perform a recursive search through the exception stack to discover retryable exceptions like S3A is doing.

drewschleit avatar Aug 13 '24 21:08 drewschleit

@drewschleit I'd recommend going all the way down.

one problem with XML parser exceptions is that sometimes the network returns HTML and that really isn't recoverable.

Key examples

  • some proxy server in the way complaining about something
  • Azure OAuth server returning error messages in HTML. not relevant here. obviously

SDK should be looking at content type and failing fast there -does anyone know if does?

when the content is text/xml then the parser errors indicate recoverable network errors except that special case of V1 listing of a versioned bucket timing out with too many tombstones to skip. S3FileIO uses v2 listing everywhere though -doesn't it?

steveloughran avatar Aug 14 '24 17:08 steveloughran

I see XMLStreamException was added as a custom retry condition in iceberg, so I'll go ahead and close this.

debora-ito avatar Apr 30 '25 18:04 debora-ito

This issue is now closed. Comments on closed issues are hard for our team to see. If you need more assistance, please open a new issue that references this one.

github-actions[bot] avatar Apr 30 '25 18:04 github-actions[bot]