You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by "ASF GitHub Bot (Jira)" <ji...@apache.org> on 2020/10/22 15:54:00 UTC
[jira] [Work logged] (HADOOP-17201) Spark job with s3acommitter
stuck at the last stage
[ https://issues.apache.org/jira/browse/HADOOP-17201?focusedWorklogId=503780&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-503780 ]
ASF GitHub Bot logged work on HADOOP-17201:
-------------------------------------------
Author: ASF GitHub Bot
Created on: 22/Oct/20 15:53
Start Date: 22/Oct/20 15:53
Worklog Time Spent: 10m
Work Description: steveloughran opened a new pull request #2402:
URL: https://github.com/apache/hadoop/pull/2402
make putObject & putObjectDirect retrying everywhere, update @RetryPolicy
annotation, and make sure callers are not attempting to retry the methods
test failure related to inconistent s3 client creating too many throttle events for the retrier to handle...need to look @ my settings to see if I've turned off the AWS SDK retries there
```
[ERROR] Tests run: 17, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 60.785 s <<< FAILURE! - in org.apache.hadoop.fs.s3a.commit.ITestCommitOperations
[ERROR] testCommitEmptyFile(org.apache.hadoop.fs.s3a.commit.ITestCommitOperations) Time elapsed: 3.289 s <<< ERROR!
com.amazonaws.AmazonServiceException: throttled count = 1 (Service: null; Status Code: 503; Error Code: null; Request ID: null)
at org.apache.hadoop.fs.s3a.InconsistentAmazonS3Client.maybeFail(InconsistentAmazonS3Client.java:571)
at org.apache.hadoop.fs.s3a.InconsistentAmazonS3Client.maybeFail(InconsistentAmazonS3Client.java:586)
at org.apache.hadoop.fs.s3a.InconsistentAmazonS3Client.putObject(InconsistentAmazonS3Client.java:226)
at com.amazonaws.services.s3.transfer.internal.UploadCallable.uploadInOneChunk(UploadCallable.java:131)
at com.amazonaws.services.s3.transfer.internal.UploadCallable.call(UploadCallable.java:123)
at com.amazonaws.services.s3.transfer.internal.UploadMonitor.call(UploadMonitor.java:143)
at com.amazonaws.services.s3.transfer.internal.UploadMonitor.call(UploadMonitor.java:48)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:748)
```
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
Issue Time Tracking
-------------------
Worklog Id: (was: 503780)
Remaining Estimate: 0h
Time Spent: 10m
> Spark job with s3acommitter stuck at the last stage
> ---------------------------------------------------
>
> Key: HADOOP-17201
> URL: https://issues.apache.org/jira/browse/HADOOP-17201
> Project: Hadoop Common
> Issue Type: Bug
> Components: fs/s3
> Affects Versions: 3.2.1
> Environment: we are on spark 2.4.5/hadoop 3.2.1 with s3a committer.
> spark.hadoop.fs.s3a.committer.magic.enabled: 'true'
> spark.hadoop.fs.s3a.committer.name: magic
> Reporter: Dyno
> Priority: Major
> Attachments: exec-120.log, exec-125.log, exec-25.log, exec-31.log, exec-36.log, exec-44.log, exec-5.log, exec-64.log, exec-7.log
>
> Time Spent: 10m
> Remaining Estimate: 0h
>
> usually our spark job took 1 hour or 2 to finish, occasionally it runs for more than 3 hour and then we know it's stuck and usually the executor has stack like this
> {{
> "Executor task launch worker for task 78620" #265 daemon prio=5 os_prio=0 tid=0x00007f73e0005000 nid=0x12d waiting on condition [0x00007f74cb291000]
> java.lang.Thread.State: TIMED_WAITING (sleeping)
> at java.lang.Thread.sleep(Native Method)
> at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:349)
> at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:285)
> at org.apache.hadoop.fs.s3a.S3AFileSystem.deleteObjects(S3AFileSystem.java:1457)
> at org.apache.hadoop.fs.s3a.S3AFileSystem.removeKeys(S3AFileSystem.java:1717)
> at org.apache.hadoop.fs.s3a.S3AFileSystem.deleteUnnecessaryFakeDirectories(S3AFileSystem.java:2785)
> at org.apache.hadoop.fs.s3a.S3AFileSystem.finishedWrite(S3AFileSystem.java:2751)
> at org.apache.hadoop.fs.s3a.WriteOperationHelper.lambda$finalizeMultipartUpload$1(WriteOperationHelper.java:238)
> at org.apache.hadoop.fs.s3a.WriteOperationHelper$$Lambda$210/1059071691.execute(Unknown Source)
> at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:109)
> at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$3(Invoker.java:265)
> at org.apache.hadoop.fs.s3a.Invoker$$Lambda$23/586859139.execute(Unknown Source)
> at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:322)
> at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:261)
> at org.apache.hadoop.fs.s3a.WriteOperationHelper.finalizeMultipartUpload(WriteOperationHelper.java:226)
> at org.apache.hadoop.fs.s3a.WriteOperationHelper.completeMPUwithRetries(WriteOperationHelper.java:271)
> at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.complete(S3ABlockOutputStream.java:660)
> at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.access$200(S3ABlockOutputStream.java:521)
> at org.apache.hadoop.fs.s3a.S3ABlockOutputStream.close(S3ABlockOutputStream.java:385)
> at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72)
> at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:101)
> at org.apache.parquet.hadoop.util.HadoopPositionOutputStream.close(HadoopPositionOutputStream.java:64)
> at org.apache.parquet.hadoop.ParquetFileWriter.end(ParquetFileWriter.java:685)
> at org.apache.parquet.hadoop.InternalParquetRecordWriter.close(InternalParquetRecordWriter.java:122)
> at org.apache.parquet.hadoop.ParquetRecordWriter.close(ParquetRecordWriter.java:165)
> at org.apache.spark.sql.execution.datasources.parquet.ParquetOutputWriter.close(ParquetOutputWriter.scala:42)
> at org.apache.spark.sql.execution.datasources.FileFormatDataWriter.releaseResources(FileFormatDataWriter.scala:57)
> at org.apache.spark.sql.execution.datasources.FileFormatDataWriter.commit(FileFormatDataWriter.scala:74)
> at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask$3.apply(FileFormatWriter.scala:247)
> at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask$3.apply(FileFormatWriter.scala:242)
> at org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1394)
> at org.apache.spark.sql.execution.datasources.FileFormatWriter$.org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask(FileFormatWriter.scala:248)
> at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply(FileFormatWriter.scala:170)
> at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply(FileFormatWriter.scala:169)
> at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
> at org.apache.spark.scheduler.Task.run(Task.scala:123)
> at org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:408)
> at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360)
> at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:414)
> at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:748)
> Locked ownable synchronizers:
> - <0x00000003a57332e0> (a java.util.concurrent.ThreadPoolExecutor$Worker)
> }}
> captured jstack on the stuck executors in case it's useful.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)
---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org