Uploaded image for project: 'Hadoop Common'
  1. Hadoop Common
  2. HADOOP-14204

S3A multipart commit failing, "UnsupportedOperationException at java.util.Collections$UnmodifiableList.sort"

    Details

    • Type: Bug
    • Status: Resolved
    • Priority: Critical
    • Resolution: Fixed
    • Affects Version/s: 2.8.0
    • Fix Version/s: 2.9.0, 3.0.0-alpha4, 2.8.2
    • Component/s: fs/s3
    • Labels:
      None
    • Target Version/s:

      Description

      Stack trace seen trying to commit a multipart upload, as the EMR code (which takes a List<String> etags is trying to sort that list directly, which it can't do if the list doesn't want to be sorted.

      later versions of the SDK clone the list before sorting.

      We need to make sure that the list passed in can be sorted.

        Issue Links

          Activity

          Hide
          vinodkv Vinod Kumar Vavilapalli added a comment -

          2.8.1 became a security release. Moving fix-version to 2.8.2 after the fact.

          Show
          vinodkv Vinod Kumar Vavilapalli added a comment - 2.8.1 became a security release. Moving fix-version to 2.8.2 after the fact.
          Hide
          andrew.wang Andrew Wang added a comment -

          Please set the appropriate 3.x fix version when committing to trunk, thanks!

          Show
          andrew.wang Andrew Wang added a comment - Please set the appropriate 3.x fix version when committing to trunk, thanks!
          Hide
          hudson Hudson added a comment -

          FAILURE: Integrated in Jenkins build Hadoop-trunk-Commit #11435 (See https://builds.apache.org/job/Hadoop-trunk-Commit/11435/)
          HADOOP-14204 S3A multipart commit failing, (stevel: rev 2841666f1f2dec96761a0aa34a69cbb20297aa14)

          • (edit) hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
          Show
          hudson Hudson added a comment - FAILURE: Integrated in Jenkins build Hadoop-trunk-Commit #11435 (See https://builds.apache.org/job/Hadoop-trunk-Commit/11435/ ) HADOOP-14204 S3A multipart commit failing, (stevel: rev 2841666f1f2dec96761a0aa34a69cbb20297aa14) (edit) hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
          Hide
          liuml07 Mingliang Liu added a comment -

          +1 on this, Thanks Steve.

          Show
          liuml07 Mingliang Liu added a comment - +1 on this, Thanks Steve.
          Hide
          hadoopqa Hadoop QA added a comment -
          -1 overall



          Vote Subsystem Runtime Comment
          0 reexec 0m 19s Docker mode activated.
          +1 @author 0m 0s The patch does not contain any @author tags.
          -1 test4tests 0m 0s The patch doesn't appear to include any new or modified tests. Please justify why no new tests are needed for this patch. Also please list what manual steps were performed to verify this patch.
          +1 mvninstall 6m 55s branch-2.8 passed
          +1 compile 0m 16s branch-2.8 passed with JDK v1.8.0_121
          +1 compile 0m 19s branch-2.8 passed with JDK v1.7.0_121
          +1 checkstyle 0m 14s branch-2.8 passed
          +1 mvnsite 0m 24s branch-2.8 passed
          +1 mvneclipse 0m 15s branch-2.8 passed
          +1 findbugs 0m 34s branch-2.8 passed
          +1 javadoc 0m 13s branch-2.8 passed with JDK v1.8.0_121
          +1 javadoc 0m 16s branch-2.8 passed with JDK v1.7.0_121
          +1 mvninstall 0m 17s the patch passed
          +1 compile 0m 14s the patch passed with JDK v1.8.0_121
          +1 javac 0m 14s the patch passed
          +1 compile 0m 17s the patch passed with JDK v1.7.0_121
          +1 javac 0m 17s the patch passed
          -0 checkstyle 0m 11s hadoop-tools/hadoop-aws: The patch generated 1 new + 5 unchanged - 1 fixed = 6 total (was 6)
          +1 mvnsite 0m 22s the patch passed
          +1 mvneclipse 0m 11s the patch passed
          +1 whitespace 0m 0s The patch has no whitespace issues.
          +1 findbugs 0m 43s the patch passed
          +1 javadoc 0m 11s the patch passed with JDK v1.8.0_121
          +1 javadoc 0m 15s the patch passed with JDK v1.7.0_121
          +1 unit 0m 23s hadoop-aws in the patch passed with JDK v1.7.0_121.
          +1 asflicense 0m 18s The patch does not generate ASF License warnings.
          15m 19s



          Subsystem Report/Notes
          Docker Image:yetus/hadoop:5af2af1
          JIRA Issue HADOOP-14204
          JIRA Patch URL https://issues.apache.org/jira/secure/attachment/12859617/HADOOP-14204-branch-2.8-001.patch
          Optional Tests asflicense compile javac javadoc mvninstall mvnsite unit findbugs checkstyle
          uname Linux 3b61ea52345a 3.13.0-103-generic #150-Ubuntu SMP Thu Nov 24 10:34:17 UTC 2016 x86_64 x86_64 x86_64 GNU/Linux
          Build tool maven
          Personality /testptch/hadoop/patchprocess/precommit/personality/provided.sh
          git revision branch-2.8 / c7b6e0d
          Default Java 1.7.0_121
          Multi-JDK versions /usr/lib/jvm/java-8-oracle:1.8.0_121 /usr/lib/jvm/java-7-openjdk-amd64:1.7.0_121
          findbugs v3.0.0
          checkstyle https://builds.apache.org/job/PreCommit-HADOOP-Build/11857/artifact/patchprocess/diff-checkstyle-hadoop-tools_hadoop-aws.txt
          JDK v1.7.0_121 Test Results https://builds.apache.org/job/PreCommit-HADOOP-Build/11857/testReport/
          modules C: hadoop-tools/hadoop-aws U: hadoop-tools/hadoop-aws
          Console output https://builds.apache.org/job/PreCommit-HADOOP-Build/11857/console
          Powered by Apache Yetus 0.5.0-SNAPSHOT http://yetus.apache.org

          This message was automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - -1 overall Vote Subsystem Runtime Comment 0 reexec 0m 19s Docker mode activated. +1 @author 0m 0s The patch does not contain any @author tags. -1 test4tests 0m 0s The patch doesn't appear to include any new or modified tests. Please justify why no new tests are needed for this patch. Also please list what manual steps were performed to verify this patch. +1 mvninstall 6m 55s branch-2.8 passed +1 compile 0m 16s branch-2.8 passed with JDK v1.8.0_121 +1 compile 0m 19s branch-2.8 passed with JDK v1.7.0_121 +1 checkstyle 0m 14s branch-2.8 passed +1 mvnsite 0m 24s branch-2.8 passed +1 mvneclipse 0m 15s branch-2.8 passed +1 findbugs 0m 34s branch-2.8 passed +1 javadoc 0m 13s branch-2.8 passed with JDK v1.8.0_121 +1 javadoc 0m 16s branch-2.8 passed with JDK v1.7.0_121 +1 mvninstall 0m 17s the patch passed +1 compile 0m 14s the patch passed with JDK v1.8.0_121 +1 javac 0m 14s the patch passed +1 compile 0m 17s the patch passed with JDK v1.7.0_121 +1 javac 0m 17s the patch passed -0 checkstyle 0m 11s hadoop-tools/hadoop-aws: The patch generated 1 new + 5 unchanged - 1 fixed = 6 total (was 6) +1 mvnsite 0m 22s the patch passed +1 mvneclipse 0m 11s the patch passed +1 whitespace 0m 0s The patch has no whitespace issues. +1 findbugs 0m 43s the patch passed +1 javadoc 0m 11s the patch passed with JDK v1.8.0_121 +1 javadoc 0m 15s the patch passed with JDK v1.7.0_121 +1 unit 0m 23s hadoop-aws in the patch passed with JDK v1.7.0_121. +1 asflicense 0m 18s The patch does not generate ASF License warnings. 15m 19s Subsystem Report/Notes Docker Image:yetus/hadoop:5af2af1 JIRA Issue HADOOP-14204 JIRA Patch URL https://issues.apache.org/jira/secure/attachment/12859617/HADOOP-14204-branch-2.8-001.patch Optional Tests asflicense compile javac javadoc mvninstall mvnsite unit findbugs checkstyle uname Linux 3b61ea52345a 3.13.0-103-generic #150-Ubuntu SMP Thu Nov 24 10:34:17 UTC 2016 x86_64 x86_64 x86_64 GNU/Linux Build tool maven Personality /testptch/hadoop/patchprocess/precommit/personality/provided.sh git revision branch-2.8 / c7b6e0d Default Java 1.7.0_121 Multi-JDK versions /usr/lib/jvm/java-8-oracle:1.8.0_121 /usr/lib/jvm/java-7-openjdk-amd64:1.7.0_121 findbugs v3.0.0 checkstyle https://builds.apache.org/job/PreCommit-HADOOP-Build/11857/artifact/patchprocess/diff-checkstyle-hadoop-tools_hadoop-aws.txt JDK v1.7.0_121 Test Results https://builds.apache.org/job/PreCommit-HADOOP-Build/11857/testReport/ modules C: hadoop-tools/hadoop-aws U: hadoop-tools/hadoop-aws Console output https://builds.apache.org/job/PreCommit-HADOOP-Build/11857/console Powered by Apache Yetus 0.5.0-SNAPSHOT http://yetus.apache.org This message was automatically generated.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Patch 001; create a new, sortable list. This is what the later AWS sdk does internally, it is mostly harmless on those SDKs, and should prevent the problem on the version in Hadoop 2.7-2.8.

          Testing: s3a frankfurt, also rebuilt spark & ran the tests downstream, as that was where I saw it. No occurrences in repeated test runs.

          Show
          stevel@apache.org Steve Loughran added a comment - Patch 001; create a new, sortable list. This is what the later AWS sdk does internally, it is mostly harmless on those SDKs, and should prevent the problem on the version in Hadoop 2.7-2.8. Testing: s3a frankfurt, also rebuilt spark & ran the tests downstream, as that was where I saw it. No occurrences in repeated test runs.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          This is one of those Schoedingbugs: it doesn't exist until it surfaces, but now you see it, it's obvious that the code never worked. Except it does, doesn't it?

          Show
          stevel@apache.org Steve Loughran added a comment - This is one of those Schoedingbugs: it doesn't exist until it surfaces, but now you see it, it's obvious that the code never worked. Except it does, doesn't it?
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Issue is

          1. AWS SDK assumes passed in List<PartETag> can be sorted.
          2. We are generating it with Futures.allAsList(partETagsFutures).get();, which inside goes return new ListFuture<V>(ImmutableList.copyOf(futures), true, MoreExecutors.sameThreadExecutor());. That is: returns an immutable list.

          Fix is what the later SDKs do internally: copy the list elements into a new ArrayList.

          Show
          stevel@apache.org Steve Loughran added a comment - Issue is AWS SDK assumes passed in List<PartETag> can be sorted. We are generating it with Futures.allAsList(partETagsFutures).get(); , which inside goes return new ListFuture<V>(ImmutableList.copyOf(futures), true, MoreExecutors.sameThreadExecutor()); . That is: returns an immutable list. Fix is what the later SDKs do internally: copy the list elements into a new ArrayList.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Stack. This is my github cloud examples running with Spark master built against hadoop-2.8.0 RC3

             org.apache.spark.SparkException: Job aborted.
                at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply$mcV$sp(FileFormatWriter.scala:196)
                at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply(FileFormatWriter.scala:161)
                at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply(FileFormatWriter.scala:161)
                at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:65)
                at org.apache.spark.sql.execution.datasources.FileFormatWriter$.write(FileFormatWriter.scala:161)
                at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.run(InsertIntoHadoopFsRelationCommand.scala:137)
                at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:58)
                at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:56)
                at org.apache.spark.sql.execution.command.ExecutedCommandExec.doExecute(commands.scala:74)
                at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114)
                at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114)
                at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135)
                at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
                at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132)
                at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:113)
                at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:93)
                at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:93)
                at org.apache.spark.sql.execution.datasources.DataSource.writeInFileFormat(DataSource.scala:442)
                at org.apache.spark.sql.execution.datasources.DataSource.write(DataSource.scala:478)
                at org.apache.spark.sql.execution.datasources.SaveIntoDataSourceCommand.run(SaveIntoDataSourceCommand.scala:48)
                at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:58)
                at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:56)
                at org.apache.spark.sql.execution.command.ExecutedCommandExec.doExecute(commands.scala:74)
                at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114)
                at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114)
                at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135)
                at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
                at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132)
                at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:113)
                at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:93)
                at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:93)
                at org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:606)
                at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:233)
                at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:217)
                at org.apache.spark.sql.DataFrameWriter.parquet(DataFrameWriter.scala:509)
                at com.hortonworks.spark.cloud.examples.S3DataFrameExample.action(S3DataFrameExample.scala:160)
                at com.hortonworks.spark.cloud.ObjectStoreExample$class.action(ObjectStoreExample.scala:67)
                at com.hortonworks.spark.cloud.examples.S3DataFrameExample.action(S3DataFrameExample.scala:56)
                at com.hortonworks.spark.cloud.examples.S3DataFrameExampleSuite$$anonfun$2.apply$mcV$sp(S3DataFrameExampleSuite.scala:47)
                at com.hortonworks.spark.cloud.CloudSuite$$anonfun$ctest$1.apply$mcV$sp(CloudSuite.scala:133)
                at com.hortonworks.spark.cloud.CloudSuite$$anonfun$ctest$1.apply(CloudSuite.scala:131)
                at com.hortonworks.spark.cloud.CloudSuite$$anonfun$ctest$1.apply(CloudSuite.scala:131)
                at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22)
                at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85)
                at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
                at org.scalatest.Transformer.apply(Transformer.scala:22)
                at org.scalatest.Transformer.apply(Transformer.scala:20)
                at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166)
                at org.scalatest.Suite$class.withFixture(Suite.scala:1122)
                at org.scalatest.FunSuite.withFixture(FunSuite.scala:1555)
                at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163)
                at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
                at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
                at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
                at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:175)
                at com.hortonworks.spark.cloud.CloudSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(CloudSuite.scala:40)
                at org.scalatest.BeforeAndAfterEach$class.runTest(BeforeAndAfterEach.scala:255)
                at com.hortonworks.spark.cloud.CloudSuite.org$scalatest$BeforeAndAfter$$super$runTest(CloudSuite.scala:40)
                at org.scalatest.BeforeAndAfter$class.runTest(BeforeAndAfter.scala:200)
                at com.hortonworks.spark.cloud.CloudSuite.runTest(CloudSuite.scala:40)
                at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208)
                at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208)
                at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:413)
                at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:401)
                at scala.collection.immutable.List.foreach(List.scala:381)
                at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401)
                at org.scalatest.SuperEngine.org$scalatest$SuperEngine$$runTestsInBranch(Engine.scala:396)
                at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:483)
                at org.scalatest.FunSuiteLike$class.runTests(FunSuiteLike.scala:208)
                at org.scalatest.FunSuite.runTests(FunSuite.scala:1555)
                at org.scalatest.Suite$class.run(Suite.scala:1424)
                at org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1555)
                at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212)
                at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212)
                at org.scalatest.SuperEngine.runImpl(Engine.scala:545)
                at org.scalatest.FunSuiteLike$class.run(FunSuiteLike.scala:212)
                at com.hortonworks.spark.cloud.CloudSuite.org$scalatest$BeforeAndAfterAll$$super$run(CloudSuite.scala:40)
                at org.scalatest.BeforeAndAfterAll$class.liftedTree1$1(BeforeAndAfterAll.scala:257)
                at org.scalatest.BeforeAndAfterAll$class.run(BeforeAndAfterAll.scala:256)
                at com.hortonworks.spark.cloud.CloudSuite.org$scalatest$BeforeAndAfter$$super$run(CloudSuite.scala:40)
                at org.scalatest.BeforeAndAfter$class.run(BeforeAndAfter.scala:241)
                at com.hortonworks.spark.cloud.CloudSuite.run(CloudSuite.scala:40)
                at org.scalatest.Suite$class.callExecuteOnSuite$1(Suite.scala:1492)
                at org.scalatest.Suite$$anonfun$runNestedSuites$1.apply(Suite.scala:1528)
                at org.scalatest.Suite$$anonfun$runNestedSuites$1.apply(Suite.scala:1526)
                at scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
                at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186)
                at org.scalatest.Suite$class.runNestedSuites(Suite.scala:1526)
                at org.scalatest.tools.DiscoverySuite.runNestedSuites(DiscoverySuite.scala:29)
                at org.scalatest.Suite$class.run(Suite.scala:1421)
                at org.scalatest.tools.DiscoverySuite.run(DiscoverySuite.scala:29)
                at org.scalatest.tools.SuiteRunner.run(SuiteRunner.scala:55)
                at org.scalatest.tools.Runner$$anonfun$doRunRunRunDaDoRunRun$3.apply(Runner.scala:2563)
                at org.scalatest.tools.Runner$$anonfun$doRunRunRunDaDoRunRun$3.apply(Runner.scala:2557)
                at scala.collection.immutable.List.foreach(List.scala:381)
                at org.scalatest.tools.Runner$.doRunRunRunDaDoRunRun(Runner.scala:2557)
                at org.scalatest.tools.Runner$$anonfun$runOptionallyWithPassFailReporter$2.apply(Runner.scala:1044)
                at org.scalatest.tools.Runner$$anonfun$runOptionallyWithPassFailReporter$2.apply(Runner.scala:1043)
                at org.scalatest.tools.Runner$.withClassLoaderAndDispatchReporter(Runner.scala:2722)
                at org.scalatest.tools.Runner$.runOptionallyWithPassFailReporter(Runner.scala:1043)
                at org.scalatest.tools.Runner$.main(Runner.scala:860)
                at org.scalatest.tools.Runner.main(Runner.scala)
                Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1.0 (TID 1, localhost, executor driver): org.apache.spark.SparkException: Task failed while writing rows
          	at org.apache.spark.sql.execution.datasources.FileFormatWriter$.org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask(FileFormatWriter.scala:253)
          	at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1$$anonfun$12.apply(FileFormatWriter.scala:178)
          	at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1$$anonfun$12.apply(FileFormatWriter.scala:177)
          	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
          	at org.apache.spark.scheduler.Task.run(Task.scala:108)
          	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:317)
          	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
          	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
          	at java.lang.Thread.run(Thread.java:745)
          Caused by: java.lang.UnsupportedOperationException
          	at java.util.Collections$UnmodifiableList.sort(Collections.java:1331)
          	at java.util.Collections.sort(Collections.java:175)
          	at com.amazonaws.services.s3.model.transform.RequestXmlFactory.convertToXmlByteArray(RequestXmlFactory.java:42)
          	at com.amazonaws.services.s3.AmazonS3Client.completeMultipartUpload(AmazonS3Client.java:2692)
          	at org.apache.hadoop.fs.s3a.S3AFileSystem$WriteOperationHelper.completeMultipartUpload(S3AFileSystem.java:2298)
          	at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.complete(S3ABlockOutputStream.java:561)
          	at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.access$200(S3ABlockOutputStream.java:454)
          	at org.apache.hadoop.fs.s3a.S3ABlockOutputStream.close(S3ABlockOutputStream.java:352)
          	at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72)
          	at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106)
          	at org.apache.parquet.hadoop.ParquetFileWriter.end(ParquetFileWriter.java:639)
          	at org.apache.parquet.hadoop.InternalParquetRecordWriter.close(InternalParquetRecordWriter.java:117)
          	at org.apache.parquet.hadoop.ParquetRecordWriter.close(ParquetRecordWriter.java:163)
          	at org.apache.spark.sql.execution.datasources.parquet.ParquetOutputWriter.close(ParquetOutputWriter.scala:42)
          	at org.apache.spark.sql.execution.datasources.FileFormatWriter$SingleDirectoryWriteTask.releaseResources(FileFormatWriter.scala:317)
          	at org.apache.spark.sql.execution.datasources.FileFormatWriter$SingleDirectoryWriteTask.execute(FileFormatWriter.scala:311)
          	at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask$3.apply(FileFormatWriter.scala:239)
          	at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask$3.apply(FileFormatWriter.scala:237)
          	at org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1360)
          	at org.apache.spark.sql.execution.datasources.FileFormatWriter$.org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask(FileFormatWriter.scala:242)
          	... 8 more
          	Suppressed: java.lang.NullPointerException
          		at org.apache.parquet.hadoop.InternalParquetRecordWriter.flushRowGroupToStore(InternalParquetRecordWriter.java:160)
          		at org.apache.parquet.hadoop.InternalParquetRecordWriter.close(InternalParquetRecordWriter.java:109)
          		at org.apache.parquet.hadoop.ParquetRecordWriter.close(ParquetRecordWriter.java:163)
          		at org.apache.spark.sql.execution.datasources.parquet.ParquetOutputWriter.close(ParquetOutputWriter.scala:42)
          		at org.apache.spark.sql.execution.datasources.FileFormatWriter$SingleDirectoryWriteTask.releaseResources(FileFormatWriter.scala:317)
          		at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask$1.apply$mcV$sp(FileFormatWriter.scala:245)
          		at org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1369)
          		... 9 more
          
          Driver stacktrace:
                at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1495)
                at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1483)
                at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1482)
                at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
                at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
                at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1482)
                at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:810)
                at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:810)
                at scala.Option.foreach(Option.scala:257)
                at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:810)
                at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1710)
                at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1665)
                at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1654)
                at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48)
                at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:635)
                at org.apache.spark.SparkContext.runJob(SparkContext.scala:2019)
                at org.apache.spark.SparkContext.runJob(SparkContext.scala:2040)
                at org.apache.spark.SparkContext.runJob(SparkContext.scala:2072)
                at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply$mcV$sp(FileFormatWriter.scala:176)
                at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply(FileFormatWriter.scala:161)
                at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply(FileFormatWriter.scala:161)
                at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:65)
                at org.apache.spark.sql.execution.datasources.FileFormatWriter$.write(FileFormatWriter.scala:161)
                at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.run(InsertIntoHadoopFsRelationCommand.scala:137)
                at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:58)
                at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:56)
                at org.apache.spark.sql.execution.command.ExecutedCommandExec.doExecute(commands.scala:74)
                at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114)
                at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114)
                at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135)
                at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
                at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132)
                at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:113)
                at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:93)
                at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:93)
                at org.apache.spark.sql.execution.datasources.DataSource.writeInFileFormat(DataSource.scala:442)
                at org.apache.spark.sql.execution.datasources.DataSource.write(DataSource.scala:478)
                at org.apache.spark.sql.execution.datasources.SaveIntoDataSourceCommand.run(SaveIntoDataSourceCommand.scala:48)
                at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:58)
                at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:56)
                at org.apache.spark.sql.execution.command.ExecutedCommandExec.doExecute(commands.scala:74)
                at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114)
                at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114)
                at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135)
                at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
                at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132)
                at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:113)
                at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:93)
                at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:93)
                at org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:606)
                at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:233)
                at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:217)
                at org.apache.spark.sql.DataFrameWriter.parquet(DataFrameWriter.scala:509)
                at com.hortonworks.spark.cloud.examples.S3DataFrameExample.action(S3DataFrameExample.scala:160)
                at com.hortonworks.spark.cloud.ObjectStoreExample$class.action(ObjectStoreExample.scala:67)
                at com.hortonworks.spark.cloud.examples.S3DataFrameExample.action(S3DataFrameExample.scala:56)
                at com.hortonworks.spark.cloud.examples.S3DataFrameExampleSuite$$anonfun$2.apply$mcV$sp(S3DataFrameExampleSuite.scala:47)
                at com.hortonworks.spark.cloud.CloudSuite$$anonfun$ctest$1.apply$mcV$sp(CloudSuite.scala:133)
                at com.hortonworks.spark.cloud.CloudSuite$$anonfun$ctest$1.apply(CloudSuite.scala:131)
                at com.hortonworks.spark.cloud.CloudSuite$$anonfun$ctest$1.apply(CloudSuite.scala:131)
                at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22)
                at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85)
                at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
                at org.scalatest.Transformer.apply(Transformer.scala:22)
                at org.scalatest.Transformer.apply(Transformer.scala:20)
                at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166)
                at org.scalatest.Suite$class.withFixture(Suite.scala:1122)
                at org.scalatest.FunSuite.withFixture(FunSuite.scala:1555)
                at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163)
                at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
                at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
                at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
                at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:175)
                at com.hortonworks.spark.cloud.CloudSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(CloudSuite.scala:40)
                at org.scalatest.BeforeAndAfterEach$class.runTest(BeforeAndAfterEach.scala:255)
                at com.hortonworks.spark.cloud.CloudSuite.org$scalatest$BeforeAndAfter$$super$runTest(CloudSuite.scala:40)
                at org.scalatest.BeforeAndAfter$class.runTest(BeforeAndAfter.scala:200)
                at com.hortonworks.spark.cloud.CloudSuite.runTest(CloudSuite.scala:40)
                at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208)
                at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208)
                at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:413)
                at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:401)
                at scala.collection.immutable.List.foreach(List.scala:381)
                at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401)
                at org.scalatest.SuperEngine.org$scalatest$SuperEngine$$runTestsInBranch(Engine.scala:396)
                at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:483)
                at org.scalatest.FunSuiteLike$class.runTests(FunSuiteLike.scala:208)
                at org.scalatest.FunSuite.runTests(FunSuite.scala:1555)
                at org.scalatest.Suite$class.run(Suite.scala:1424)
                at org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1555)
                at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212)
                at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212)
                at org.scalatest.SuperEngine.runImpl(Engine.scala:545)
                at org.scalatest.FunSuiteLike$class.run(FunSuiteLike.scala:212)
                at com.hortonworks.spark.cloud.CloudSuite.org$scalatest$BeforeAndAfterAll$$super$run(CloudSuite.scala:40)
                at org.scalatest.BeforeAndAfterAll$class.liftedTree1$1(BeforeAndAfterAll.scala:257)
                at org.scalatest.BeforeAndAfterAll$class.run(BeforeAndAfterAll.scala:256)
                at com.hortonworks.spark.cloud.CloudSuite.org$scalatest$BeforeAndAfter$$super$run(CloudSuite.scala:40)
                at org.scalatest.BeforeAndAfter$class.run(BeforeAndAfter.scala:241)
                at com.hortonworks.spark.cloud.CloudSuite.run(CloudSuite.scala:40)
                at org.scalatest.Suite$class.callExecuteOnSuite$1(Suite.scala:1492)
                at org.scalatest.Suite$$anonfun$runNestedSuites$1.apply(Suite.scala:1528)
                at org.scalatest.Suite$$anonfun$runNestedSuites$1.apply(Suite.scala:1526)
                at scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
                at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186)
                at org.scalatest.Suite$class.runNestedSuites(Suite.scala:1526)
                at org.scalatest.tools.DiscoverySuite.runNestedSuites(DiscoverySuite.scala:29)
                at org.scalatest.Suite$class.run(Suite.scala:1421)
                at org.scalatest.tools.DiscoverySuite.run(DiscoverySuite.scala:29)
                at org.scalatest.tools.SuiteRunner.run(SuiteRunner.scala:55)
                at org.scalatest.tools.Runner$$anonfun$doRunRunRunDaDoRunRun$3.apply(Runner.scala:2563)
                at org.scalatest.tools.Runner$$anonfun$doRunRunRunDaDoRunRun$3.apply(Runner.scala:2557)
                at scala.collection.immutable.List.foreach(List.scala:381)
                at org.scalatest.tools.Runner$.doRunRunRunDaDoRunRun(Runner.scala:2557)
                at org.scalatest.tools.Runner$$anonfun$runOptionallyWithPassFailReporter$2.apply(Runner.scala:1044)
                at org.scalatest.tools.Runner$$anonfun$runOptionallyWithPassFailReporter$2.apply(Runner.scala:1043)
                at org.scalatest.tools.Runner$.withClassLoaderAndDispatchReporter(Runner.scala:2722)
                at org.scalatest.tools.Runner$.runOptionallyWithPassFailReporter(Runner.scala:1043)
                at org.scalatest.tools.Runner$.main(Runner.scala:860)
                at org.scalatest.tools.Runner.main(Runner.scala)
                Cause: org.apache.spark.SparkException: Task failed while writing rows
                at org.apache.spark.sql.execution.datasources.FileFormatWriter$.org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask(FileFormatWriter.scala:253)
                at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1$$anonfun$12.apply(FileFormatWriter.scala:178)
                at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1$$anonfun$12.apply(FileFormatWriter.scala:177)
                at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
                at org.apache.spark.scheduler.Task.run(Task.scala:108)
                at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:317)
                at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
                at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
                at java.lang.Thread.run(Thread.java:745)
                Cause: java.lang.UnsupportedOperationException
                at java.util.Collections$UnmodifiableList.sort(Collections.java:1331)
                at java.util.Collections.sort(Collections.java:175)
                at com.amazonaws.services.s3.model.transform.RequestXmlFactory.convertToXmlByteArray(RequestXmlFactory.java:42)
                at com.amazonaws.services.s3.AmazonS3Client.completeMultipartUpload(AmazonS3Client.java:2692)
                at org.apache.hadoop.fs.s3a.S3AFileSystem$WriteOperationHelper.completeMultipartUpload(S3AFileSystem.java:2298)
                at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.complete(S3ABlockOutputStream.java:561)
                at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.access$200(S3ABlockOutputStream.java:454)
                at org.apache.hadoop.fs.s3a.S3ABlockOutputStream.close(S3ABlockOutputStream.java:352)
                at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72)
                at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106)
                at org.apache.parquet.hadoop.ParquetFileWriter.end(ParquetFileWriter.java:639)
                at org.apache.parquet.hadoop.InternalParquetRecordWriter.close(InternalParquetRecordWriter.java:117)
                at org.apache.parquet.hadoop.ParquetRecordWriter.close(ParquetRecordWriter.java:163)
                at org.apache.spark.sql.execution.datasources.parquet.ParquetOutputWriter.close(ParquetOutputWriter.scala:42)
                at org.apache.spark.sql.execution.datasources.FileFormatWriter$SingleDirectoryWriteTask.releaseResources(FileFormatWriter.scala:317)
                at org.apache.spark.sql.execution.datasources.FileFormatWriter$SingleDirectoryWriteTask.execute(FileFormatWriter.scala:311)
                at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask$3.apply(FileFormatWriter.scala:239)
                at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask$3.apply(FileFormatWriter.scala:237)
                at org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1360)
                at org.apache.spark.sql.execution.datasources.FileFormatWriter$.org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask(FileFormatWriter.scala:242)
                at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1$$anonfun$12.apply(FileFormatWriter.scala:178)
                at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1$$anonfun$12.apply(FileFormatWriter.scala:177)
                at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
                at org.apache.spark.scheduler.Task.run(Task.scala:108)
                at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:317)
                at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
                at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
                at java.lang.Thread.run(Thread.java:745)
          
          Show
          stevel@apache.org Steve Loughran added a comment - Stack. This is my github cloud examples running with Spark master built against hadoop-2.8.0 RC3 org.apache.spark.SparkException: Job aborted. at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply$mcV$sp(FileFormatWriter.scala:196) at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply(FileFormatWriter.scala:161) at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply(FileFormatWriter.scala:161) at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:65) at org.apache.spark.sql.execution.datasources.FileFormatWriter$.write(FileFormatWriter.scala:161) at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.run(InsertIntoHadoopFsRelationCommand.scala:137) at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:58) at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:56) at org.apache.spark.sql.execution.command.ExecutedCommandExec.doExecute(commands.scala:74) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114) at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:113) at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:93) at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:93) at org.apache.spark.sql.execution.datasources.DataSource.writeInFileFormat(DataSource.scala:442) at org.apache.spark.sql.execution.datasources.DataSource.write(DataSource.scala:478) at org.apache.spark.sql.execution.datasources.SaveIntoDataSourceCommand.run(SaveIntoDataSourceCommand.scala:48) at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:58) at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:56) at org.apache.spark.sql.execution.command.ExecutedCommandExec.doExecute(commands.scala:74) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114) at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:113) at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:93) at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:93) at org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:606) at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:233) at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:217) at org.apache.spark.sql.DataFrameWriter.parquet(DataFrameWriter.scala:509) at com.hortonworks.spark.cloud.examples.S3DataFrameExample.action(S3DataFrameExample.scala:160) at com.hortonworks.spark.cloud.ObjectStoreExample$class.action(ObjectStoreExample.scala:67) at com.hortonworks.spark.cloud.examples.S3DataFrameExample.action(S3DataFrameExample.scala:56) at com.hortonworks.spark.cloud.examples.S3DataFrameExampleSuite$$anonfun$2.apply$mcV$sp(S3DataFrameExampleSuite.scala:47) at com.hortonworks.spark.cloud.CloudSuite$$anonfun$ctest$1.apply$mcV$sp(CloudSuite.scala:133) at com.hortonworks.spark.cloud.CloudSuite$$anonfun$ctest$1.apply(CloudSuite.scala:131) at com.hortonworks.spark.cloud.CloudSuite$$anonfun$ctest$1.apply(CloudSuite.scala:131) at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22) at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85) at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104) at org.scalatest.Transformer.apply(Transformer.scala:22) at org.scalatest.Transformer.apply(Transformer.scala:20) at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166) at org.scalatest.Suite$class.withFixture(Suite.scala:1122) at org.scalatest.FunSuite.withFixture(FunSuite.scala:1555) at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163) at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175) at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175) at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306) at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:175) at com.hortonworks.spark.cloud.CloudSuite.org$scalatest$BeforeAndAfterEach$$ super $runTest(CloudSuite.scala:40) at org.scalatest.BeforeAndAfterEach$class.runTest(BeforeAndAfterEach.scala:255) at com.hortonworks.spark.cloud.CloudSuite.org$scalatest$BeforeAndAfter$$ super $runTest(CloudSuite.scala:40) at org.scalatest.BeforeAndAfter$class.runTest(BeforeAndAfter.scala:200) at com.hortonworks.spark.cloud.CloudSuite.runTest(CloudSuite.scala:40) at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208) at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208) at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:413) at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:401) at scala.collection.immutable.List.foreach(List.scala:381) at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401) at org.scalatest.SuperEngine.org$scalatest$SuperEngine$$runTestsInBranch(Engine.scala:396) at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:483) at org.scalatest.FunSuiteLike$class.runTests(FunSuiteLike.scala:208) at org.scalatest.FunSuite.runTests(FunSuite.scala:1555) at org.scalatest.Suite$class.run(Suite.scala:1424) at org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$ super $run(FunSuite.scala:1555) at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212) at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212) at org.scalatest.SuperEngine.runImpl(Engine.scala:545) at org.scalatest.FunSuiteLike$class.run(FunSuiteLike.scala:212) at com.hortonworks.spark.cloud.CloudSuite.org$scalatest$BeforeAndAfterAll$$ super $run(CloudSuite.scala:40) at org.scalatest.BeforeAndAfterAll$class.liftedTree1$1(BeforeAndAfterAll.scala:257) at org.scalatest.BeforeAndAfterAll$class.run(BeforeAndAfterAll.scala:256) at com.hortonworks.spark.cloud.CloudSuite.org$scalatest$BeforeAndAfter$$ super $run(CloudSuite.scala:40) at org.scalatest.BeforeAndAfter$class.run(BeforeAndAfter.scala:241) at com.hortonworks.spark.cloud.CloudSuite.run(CloudSuite.scala:40) at org.scalatest.Suite$class.callExecuteOnSuite$1(Suite.scala:1492) at org.scalatest.Suite$$anonfun$runNestedSuites$1.apply(Suite.scala:1528) at org.scalatest.Suite$$anonfun$runNestedSuites$1.apply(Suite.scala:1526) at scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33) at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186) at org.scalatest.Suite$class.runNestedSuites(Suite.scala:1526) at org.scalatest.tools.DiscoverySuite.runNestedSuites(DiscoverySuite.scala:29) at org.scalatest.Suite$class.run(Suite.scala:1421) at org.scalatest.tools.DiscoverySuite.run(DiscoverySuite.scala:29) at org.scalatest.tools.SuiteRunner.run(SuiteRunner.scala:55) at org.scalatest.tools.Runner$$anonfun$doRunRunRunDaDoRunRun$3.apply(Runner.scala:2563) at org.scalatest.tools.Runner$$anonfun$doRunRunRunDaDoRunRun$3.apply(Runner.scala:2557) at scala.collection.immutable.List.foreach(List.scala:381) at org.scalatest.tools.Runner$.doRunRunRunDaDoRunRun(Runner.scala:2557) at org.scalatest.tools.Runner$$anonfun$runOptionallyWithPassFailReporter$2.apply(Runner.scala:1044) at org.scalatest.tools.Runner$$anonfun$runOptionallyWithPassFailReporter$2.apply(Runner.scala:1043) at org.scalatest.tools.Runner$.withClassLoaderAndDispatchReporter(Runner.scala:2722) at org.scalatest.tools.Runner$.runOptionallyWithPassFailReporter(Runner.scala:1043) at org.scalatest.tools.Runner$.main(Runner.scala:860) at org.scalatest.tools.Runner.main(Runner.scala) Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1.0 (TID 1, localhost, executor driver): org.apache.spark.SparkException: Task failed while writing rows at org.apache.spark.sql.execution.datasources.FileFormatWriter$.org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask(FileFormatWriter.scala:253) at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1$$anonfun$12.apply(FileFormatWriter.scala:178) at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1$$anonfun$12.apply(FileFormatWriter.scala:177) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87) at org.apache.spark.scheduler.Task.run(Task.scala:108) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:317) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang. Thread .run( Thread .java:745) Caused by: java.lang.UnsupportedOperationException at java.util.Collections$UnmodifiableList.sort(Collections.java:1331) at java.util.Collections.sort(Collections.java:175) at com.amazonaws.services.s3.model.transform.RequestXmlFactory.convertToXmlByteArray(RequestXmlFactory.java:42) at com.amazonaws.services.s3.AmazonS3Client.completeMultipartUpload(AmazonS3Client.java:2692) at org.apache.hadoop.fs.s3a.S3AFileSystem$WriteOperationHelper.completeMultipartUpload(S3AFileSystem.java:2298) at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.complete(S3ABlockOutputStream.java:561) at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.access$200(S3ABlockOutputStream.java:454) at org.apache.hadoop.fs.s3a.S3ABlockOutputStream.close(S3ABlockOutputStream.java:352) at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72) at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106) at org.apache.parquet.hadoop.ParquetFileWriter.end(ParquetFileWriter.java:639) at org.apache.parquet.hadoop.InternalParquetRecordWriter.close(InternalParquetRecordWriter.java:117) at org.apache.parquet.hadoop.ParquetRecordWriter.close(ParquetRecordWriter.java:163) at org.apache.spark.sql.execution.datasources.parquet.ParquetOutputWriter.close(ParquetOutputWriter.scala:42) at org.apache.spark.sql.execution.datasources.FileFormatWriter$SingleDirectoryWriteTask.releaseResources(FileFormatWriter.scala:317) at org.apache.spark.sql.execution.datasources.FileFormatWriter$SingleDirectoryWriteTask.execute(FileFormatWriter.scala:311) at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask$3.apply(FileFormatWriter.scala:239) at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask$3.apply(FileFormatWriter.scala:237) at org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1360) at org.apache.spark.sql.execution.datasources.FileFormatWriter$.org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask(FileFormatWriter.scala:242) ... 8 more Suppressed: java.lang.NullPointerException at org.apache.parquet.hadoop.InternalParquetRecordWriter.flushRowGroupToStore(InternalParquetRecordWriter.java:160) at org.apache.parquet.hadoop.InternalParquetRecordWriter.close(InternalParquetRecordWriter.java:109) at org.apache.parquet.hadoop.ParquetRecordWriter.close(ParquetRecordWriter.java:163) at org.apache.spark.sql.execution.datasources.parquet.ParquetOutputWriter.close(ParquetOutputWriter.scala:42) at org.apache.spark.sql.execution.datasources.FileFormatWriter$SingleDirectoryWriteTask.releaseResources(FileFormatWriter.scala:317) at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask$1.apply$mcV$sp(FileFormatWriter.scala:245) at org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1369) ... 9 more Driver stacktrace: at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1495) at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1483) at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1482) at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48) at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1482) at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:810) at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:810) at scala.Option.foreach(Option.scala:257) at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:810) at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1710) at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1665) at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1654) at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48) at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:635) at org.apache.spark.SparkContext.runJob(SparkContext.scala:2019) at org.apache.spark.SparkContext.runJob(SparkContext.scala:2040) at org.apache.spark.SparkContext.runJob(SparkContext.scala:2072) at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply$mcV$sp(FileFormatWriter.scala:176) at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply(FileFormatWriter.scala:161) at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply(FileFormatWriter.scala:161) at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:65) at org.apache.spark.sql.execution.datasources.FileFormatWriter$.write(FileFormatWriter.scala:161) at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.run(InsertIntoHadoopFsRelationCommand.scala:137) at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:58) at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:56) at org.apache.spark.sql.execution.command.ExecutedCommandExec.doExecute(commands.scala:74) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114) at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:113) at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:93) at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:93) at org.apache.spark.sql.execution.datasources.DataSource.writeInFileFormat(DataSource.scala:442) at org.apache.spark.sql.execution.datasources.DataSource.write(DataSource.scala:478) at org.apache.spark.sql.execution.datasources.SaveIntoDataSourceCommand.run(SaveIntoDataSourceCommand.scala:48) at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:58) at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:56) at org.apache.spark.sql.execution.command.ExecutedCommandExec.doExecute(commands.scala:74) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114) at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:113) at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:93) at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:93) at org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:606) at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:233) at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:217) at org.apache.spark.sql.DataFrameWriter.parquet(DataFrameWriter.scala:509) at com.hortonworks.spark.cloud.examples.S3DataFrameExample.action(S3DataFrameExample.scala:160) at com.hortonworks.spark.cloud.ObjectStoreExample$class.action(ObjectStoreExample.scala:67) at com.hortonworks.spark.cloud.examples.S3DataFrameExample.action(S3DataFrameExample.scala:56) at com.hortonworks.spark.cloud.examples.S3DataFrameExampleSuite$$anonfun$2.apply$mcV$sp(S3DataFrameExampleSuite.scala:47) at com.hortonworks.spark.cloud.CloudSuite$$anonfun$ctest$1.apply$mcV$sp(CloudSuite.scala:133) at com.hortonworks.spark.cloud.CloudSuite$$anonfun$ctest$1.apply(CloudSuite.scala:131) at com.hortonworks.spark.cloud.CloudSuite$$anonfun$ctest$1.apply(CloudSuite.scala:131) at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22) at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85) at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104) at org.scalatest.Transformer.apply(Transformer.scala:22) at org.scalatest.Transformer.apply(Transformer.scala:20) at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166) at org.scalatest.Suite$class.withFixture(Suite.scala:1122) at org.scalatest.FunSuite.withFixture(FunSuite.scala:1555) at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163) at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175) at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175) at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306) at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:175) at com.hortonworks.spark.cloud.CloudSuite.org$scalatest$BeforeAndAfterEach$$ super $runTest(CloudSuite.scala:40) at org.scalatest.BeforeAndAfterEach$class.runTest(BeforeAndAfterEach.scala:255) at com.hortonworks.spark.cloud.CloudSuite.org$scalatest$BeforeAndAfter$$ super $runTest(CloudSuite.scala:40) at org.scalatest.BeforeAndAfter$class.runTest(BeforeAndAfter.scala:200) at com.hortonworks.spark.cloud.CloudSuite.runTest(CloudSuite.scala:40) at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208) at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208) at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:413) at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:401) at scala.collection.immutable.List.foreach(List.scala:381) at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401) at org.scalatest.SuperEngine.org$scalatest$SuperEngine$$runTestsInBranch(Engine.scala:396) at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:483) at org.scalatest.FunSuiteLike$class.runTests(FunSuiteLike.scala:208) at org.scalatest.FunSuite.runTests(FunSuite.scala:1555) at org.scalatest.Suite$class.run(Suite.scala:1424) at org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$ super $run(FunSuite.scala:1555) at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212) at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212) at org.scalatest.SuperEngine.runImpl(Engine.scala:545) at org.scalatest.FunSuiteLike$class.run(FunSuiteLike.scala:212) at com.hortonworks.spark.cloud.CloudSuite.org$scalatest$BeforeAndAfterAll$$ super $run(CloudSuite.scala:40) at org.scalatest.BeforeAndAfterAll$class.liftedTree1$1(BeforeAndAfterAll.scala:257) at org.scalatest.BeforeAndAfterAll$class.run(BeforeAndAfterAll.scala:256) at com.hortonworks.spark.cloud.CloudSuite.org$scalatest$BeforeAndAfter$$ super $run(CloudSuite.scala:40) at org.scalatest.BeforeAndAfter$class.run(BeforeAndAfter.scala:241) at com.hortonworks.spark.cloud.CloudSuite.run(CloudSuite.scala:40) at org.scalatest.Suite$class.callExecuteOnSuite$1(Suite.scala:1492) at org.scalatest.Suite$$anonfun$runNestedSuites$1.apply(Suite.scala:1528) at org.scalatest.Suite$$anonfun$runNestedSuites$1.apply(Suite.scala:1526) at scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33) at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186) at org.scalatest.Suite$class.runNestedSuites(Suite.scala:1526) at org.scalatest.tools.DiscoverySuite.runNestedSuites(DiscoverySuite.scala:29) at org.scalatest.Suite$class.run(Suite.scala:1421) at org.scalatest.tools.DiscoverySuite.run(DiscoverySuite.scala:29) at org.scalatest.tools.SuiteRunner.run(SuiteRunner.scala:55) at org.scalatest.tools.Runner$$anonfun$doRunRunRunDaDoRunRun$3.apply(Runner.scala:2563) at org.scalatest.tools.Runner$$anonfun$doRunRunRunDaDoRunRun$3.apply(Runner.scala:2557) at scala.collection.immutable.List.foreach(List.scala:381) at org.scalatest.tools.Runner$.doRunRunRunDaDoRunRun(Runner.scala:2557) at org.scalatest.tools.Runner$$anonfun$runOptionallyWithPassFailReporter$2.apply(Runner.scala:1044) at org.scalatest.tools.Runner$$anonfun$runOptionallyWithPassFailReporter$2.apply(Runner.scala:1043) at org.scalatest.tools.Runner$.withClassLoaderAndDispatchReporter(Runner.scala:2722) at org.scalatest.tools.Runner$.runOptionallyWithPassFailReporter(Runner.scala:1043) at org.scalatest.tools.Runner$.main(Runner.scala:860) at org.scalatest.tools.Runner.main(Runner.scala) Cause: org.apache.spark.SparkException: Task failed while writing rows at org.apache.spark.sql.execution.datasources.FileFormatWriter$.org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask(FileFormatWriter.scala:253) at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1$$anonfun$12.apply(FileFormatWriter.scala:178) at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1$$anonfun$12.apply(FileFormatWriter.scala:177) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87) at org.apache.spark.scheduler.Task.run(Task.scala:108) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:317) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang. Thread .run( Thread .java:745) Cause: java.lang.UnsupportedOperationException at java.util.Collections$UnmodifiableList.sort(Collections.java:1331) at java.util.Collections.sort(Collections.java:175) at com.amazonaws.services.s3.model.transform.RequestXmlFactory.convertToXmlByteArray(RequestXmlFactory.java:42) at com.amazonaws.services.s3.AmazonS3Client.completeMultipartUpload(AmazonS3Client.java:2692) at org.apache.hadoop.fs.s3a.S3AFileSystem$WriteOperationHelper.completeMultipartUpload(S3AFileSystem.java:2298) at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.complete(S3ABlockOutputStream.java:561) at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.access$200(S3ABlockOutputStream.java:454) at org.apache.hadoop.fs.s3a.S3ABlockOutputStream.close(S3ABlockOutputStream.java:352) at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72) at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106) at org.apache.parquet.hadoop.ParquetFileWriter.end(ParquetFileWriter.java:639) at org.apache.parquet.hadoop.InternalParquetRecordWriter.close(InternalParquetRecordWriter.java:117) at org.apache.parquet.hadoop.ParquetRecordWriter.close(ParquetRecordWriter.java:163) at org.apache.spark.sql.execution.datasources.parquet.ParquetOutputWriter.close(ParquetOutputWriter.scala:42) at org.apache.spark.sql.execution.datasources.FileFormatWriter$SingleDirectoryWriteTask.releaseResources(FileFormatWriter.scala:317) at org.apache.spark.sql.execution.datasources.FileFormatWriter$SingleDirectoryWriteTask.execute(FileFormatWriter.scala:311) at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask$3.apply(FileFormatWriter.scala:239) at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask$3.apply(FileFormatWriter.scala:237) at org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1360) at org.apache.spark.sql.execution.datasources.FileFormatWriter$.org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask(FileFormatWriter.scala:242) at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1$$anonfun$12.apply(FileFormatWriter.scala:178) at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1$$anonfun$12.apply(FileFormatWriter.scala:177) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87) at org.apache.spark.scheduler.Task.run(Task.scala:108) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:317) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang. Thread .run( Thread .java:745)

            People

            • Assignee:
              stevel@apache.org Steve Loughran
              Reporter:
              stevel@apache.org Steve Loughran
            • Votes:
              0 Vote for this issue
              Watchers:
              8 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development