Details
Description
I'm in the process of upgrading spark+hadoop versions for my workflows and observing a weird behavior regression. I'm setting
spark.hadoop.fs.s3a.aws.credentials.provider=org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider spark.hadoop.fs.s3.impl=org.apache.hadoop.fs.s3a.S3AFileSystem spark.sql.catalogImplementation=hive spark.hadoop.aws.region=us-west-2 ...many other things, I think these might be the relevant ones though...
in Spark config and I'm observing some non-fatal warnings/exceptions (see below for some examples). The warnings/exceptions randomly appear for some tasks, which causes them to fail, but then when Spark retries the task it will succeed. The initial tasks don't always fail either, just sometimes.
I also found that if I switch to a SimpleAWSCredentials and use static keys, then I don't see any issues.
My old setup was spark v3.0.2 with hadoop-aws v3.2.1 which also does not have these warnings/exceptions.
From reading some other tickets I thought perhaps adding
spark.sql.hive.metastore.sharedPrefixes=com.amazonaws
would help, but it did not.
Appreciate any suggestions for how to proceed or debug further
Example stack traces:
First one for an s3 read
WARN TaskSetManager: Lost task 27.0 in stage 4.0 (TID 29) (<ip> executor 13): java.nio.file.AccessDeniedException: s3a://bucket/path/to/part.snappy.parquet: org.apache.hadoop.fs.s3a.CredentialInitializationException: Provider TemporaryAWSCredentialsProvider has no credentials at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:206) at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:170) at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:3289) at org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:3185) at org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:3053) at org.apache.parquet.hadoop.util.HadoopInputFile.fromPath(HadoopInputFile.java:39) at org.apache.spark.sql.execution.datasources.parquet.ParquetFooterReader.readFooter(ParquetFooterReader.java:39) at org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat.footerFileMetaData$lzycompute$1(ParquetFileFormat.scala:268) at org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat.footerFileMetaData$1(ParquetFileFormat.scala:267) at org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat.$anonfun$buildReaderWithPartitionValues$2(ParquetFileFormat.scala:270) at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.org$apache$spark$sql$execution$datasources$FileScanRDD$$anon$$readCurrentFile(FileScanRDD.scala:116) at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:164) at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:93) at org.apache.spark.sql.execution.FileSourceScanExec$$anon$1.hasNext(DataSourceScanExec.scala:522) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage7.columnartorow_nextBatch_0$(Unknown Source) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage7.processNext(Unknown Source) at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43) at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:759) at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460) at org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:140) at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52) at org.apache.spark.scheduler.Task.run(Task.scala:131) at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:506) at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1462) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509) 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) Caused by: org.apache.hadoop.fs.s3a.CredentialInitializationException: Provider TemporaryAWSCredentialsProvider has no credentials at org.apache.hadoop.fs.s3a.auth.AbstractSessionCredentialsProvider.getCredentials(AbstractSessionCredentialsProvider.java:130) at org.apache.hadoop.fs.s3a.AWSCredentialProviderList.getCredentials(AWSCredentialProviderList.java:177) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.getCredentialsFromContext(AmazonHttpClient.java:1266) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.runBeforeRequestHandlers(AmazonHttpClient.java:842) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:792) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:779) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:753) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:713) at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:695) at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:559) at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:539) at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:5437) at com.amazonaws.services.s3.AmazonS3Client.getBucketRegionViaHeadRequest(AmazonS3Client.java:6408) at com.amazonaws.services.s3.AmazonS3Client.fetchRegionFromCache(AmazonS3Client.java:6381) at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:5422) at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:5384) at com.amazonaws.services.s3.AmazonS3Client.getObjectMetadata(AmazonS3Client.java:1367) at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$getObjectMetadata$6(S3AFileSystem.java:2066) at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:412) at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:375) at org.apache.hadoop.fs.s3a.S3AFileSystem.getObjectMetadata(S3AFileSystem.java:2056) at org.apache.hadoop.fs.s3a.S3AFileSystem.getObjectMetadata(S3AFileSystem.java:2032) at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:3273) ... 27 more
And here is one for an s3 write which is similar but slightly different
WARN TaskSetManager: Lost task 21.0 in stage 78.0 (TID 1358) (<ip> executor 11): org.apache.spark.SparkException: Task failed while writing rows. at org.apache.spark.sql.errors.QueryExecutionErrors$.taskFailedWhileWritingRowsError(QueryExecutionErrors.scala:500) at org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:321) at org.apache.spark.sql.execution.datasources.FileFormatWriter$.$anonfun$write$16(FileFormatWriter.scala:229) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90) at org.apache.spark.scheduler.Task.run(Task.scala:131) at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:506) at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1462) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509) 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) Caused by: java.nio.file.AccessDeniedException: s3a://bucket/path/to/_temporary/0/_temporary/attempt_0123456789: org.apache.hadoop.fs.s3a.CredentialInitializationException: Provider TemporaryAWSCredentialsProvider has no credentials at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:206) at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:170) at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:3289) at org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:3185) at org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:3053) at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:1760) at org.apache.hadoop.fs.s3a.S3AFileSystem.exists(S3AFileSystem.java:4263) at org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.needsTaskCommit(FileOutputCommitter.java:674) at org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.needsTaskCommit(FileOutputCommitter.java:663) at org.apache.spark.mapred.SparkHadoopMapRedUtil$.commitTask(SparkHadoopMapRedUtil.scala:61) at org.apache.spark.internal.io.HadoopMapReduceCommitProtocol.commitTask(HadoopMapReduceCommitProtocol.scala:269) at org.apache.spark.sql.execution.datasources.FileFormatDataWriter.$anonfun$commit$1(FileFormatDataWriter.scala:107) at org.apache.spark.util.Utils$.timeTakenMs(Utils.scala:605) at org.apache.spark.sql.execution.datasources.FileFormatDataWriter.commit(FileFormatDataWriter.scala:107) at org.apache.spark.sql.execution.datasources.FileFormatWriter$.$anonfun$executeTask$1(FileFormatWriter.scala:305) at org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1496) at org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:311) ... 9 more Caused by: org.apache.hadoop.fs.s3a.CredentialInitializationException: Provider TemporaryAWSCredentialsProvider has no credentials at org.apache.hadoop.fs.s3a.auth.AbstractSessionCredentialsProvider.getCredentials(AbstractSessionCredentialsProvider.java:130) at org.apache.hadoop.fs.s3a.AWSCredentialProviderList.getCredentials(AWSCredentialProviderList.java:177) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.getCredentialsFromContext(AmazonHttpClient.java:1266) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.runBeforeRequestHandlers(AmazonHttpClient.java:842) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:792) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:779) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:753) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:713) at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:695) at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:559) at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:539) at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:5437) at com.amazonaws.services.s3.AmazonS3Client.getBucketRegionViaHeadRequest(AmazonS3Client.java:6408) at com.amazonaws.services.s3.AmazonS3Client.fetchRegionFromCache(AmazonS3Client.java:6381) at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:5422) at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:5384) at com.amazonaws.services.s3.AmazonS3Client.getObjectMetadata(AmazonS3Client.java:1367) at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$getObjectMetadata$6(S3AFileSystem.java:2066) at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:412) at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:375) at org.apache.hadoop.fs.s3a.S3AFileSystem.getObjectMetadata(S3AFileSystem.java:2056) at org.apache.hadoop.fs.s3a.S3AFileSystem.getObjectMetadata(S3AFileSystem.java:2032) at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:3273) ... 23 more
Attachments
Issue Links
- is caused by
-
HADOOP-17454 [s3a] Disable bucket existence check - set fs.s3a.bucket.probe to 0
- Resolved
- relates to
-
SPARK-38934 Provider TemporaryAWSCredentialsProvider has no credentials
- Reopened
-
HADOOP-18078 TemporaryAWSCredentialsProvider has no credentials
- Resolved
- links to