[ 
https://issues.apache.org/jira/browse/SPARK-45289?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tanawat Panmongkol updated SPARK-45289:
---------------------------------------
    Environment: 
Spark version: 3.5.0

Deployment mode: client (Docker)

OS: Ubuntu (Docker image)

Java/JVM version: OpenJDK 11

Packages: 

  was:
Spark version: 3.5.0

Deployment mode: client (Docker)

OS: Ubuntu (Docker image)

Java/JVM version: OpenJDK 11


> ClassCastException when reading Delta table on AWS S3
> -----------------------------------------------------
>
>                 Key: SPARK-45289
>                 URL: https://issues.apache.org/jira/browse/SPARK-45289
>             Project: Spark
>          Issue Type: Bug
>          Components: Spark Core, SQL
>    Affects Versions: 3.5.0
>         Environment: Spark version: 3.5.0
> Deployment mode: client (Docker)
> OS: Ubuntu (Docker image)
> Java/JVM version: OpenJDK 11
> Packages: 
>            Reporter: Tanawat Panmongkol
>            Priority: Major
>
> When attempting to read a Delta table from S3 using version 3.5.0, a 
> _*{{ClassCastException}}*_ occurs involving 
> {{_*org.apache.hadoop.fs.s3a.S3AFileStatus*_}} and 
> {_}*{{org.apache.spark.sql.execution.datasources.FileStatusWithMetadata}}*{_}.
>  The error appears to be related to the new feature SPARK-43039.
> _*Steps to Reproduce:*_
> {code:java}
> export AWS_ACCESS_KEY_ID='<ACCESS_KEY>'
> export AWS_SECRET_ACCESS_KEY='<SECRET_KEY>'
> export AWS_REGION='<REGION>'
> docker run --rm -it apache/spark:3.5.0-scala2.12-java11-ubuntu 
> /opt/spark/bin/spark-shell \
> --packages 
> 'org.apache.hadoop:hadoop-aws:3.3.4,io.delta:delta-core_2.12:2.4.0' \
> --conf 
> "spark.sql.catalog.spark_catalog=org.apache.spark.sql.delta.catalog.DeltaCatalog"
>  \
> --conf "spark.sql.extensions=io.delta.sql.DeltaSparkSessionExtension" \
> --conf "spark.hadoop.aws.region=$AWS_REGION" \
> --conf "spark.hadoop.fs.s3a.access.key=$AWS_ACCESS_KEY_ID" \
> --conf "spark.hadoop.fs.s3a.secret.key=$AWS_SECRET_ACCESS_KEY" \
> --conf "spark.hadoop.fs.s3.impl=org.apache.hadoop.fs.s3a.S3AFileSystem" \
> --conf "spark.hadoop.fs.s3a.impl=org.apache.hadoop.fs.s3a.S3AFileSystem" \
> --conf "spark.hadoop.fs.s3a.path.style.access=true" \
> --conf "spark.hadoop.fs.s3a.connection.ssl.enabled=true" \
> --conf "spark.jars.ivy=/tmp/ivy/cache"{code}
> {code:java}
> scala> 
> spark.read.format("delta").load("s3://<bucket_name>/<delta_table_name>/").show()
>  {code}
> *Logs:*
> {code:java}
> java.lang.ClassCastException: class org.apache.hadoop.fs.s3a.S3AFileStatus 
> cannot be cast to class 
> org.apache.spark.sql.execution.datasources.FileStatusWithMetadata 
> (org.apache.hadoop.fs.s3a.S3AFileStatus is in unnamed module of loader 
> scala.reflect.internal.util.ScalaClassLoader$URLClassLoader @1d5bb5c; 
> org.apache.spark.sql.execution.datasources.FileStatusWithMetadata is in 
> unnamed module of loader 'app')
>   at 
> scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:286)
>   at 
> scala.collection.IndexedSeqOptimized.foreach(IndexedSeqOptimized.scala:36)
>   at 
> scala.collection.IndexedSeqOptimized.foreach$(IndexedSeqOptimized.scala:33)
>   at scala.collection.mutable.WrappedArray.foreach(WrappedArray.scala:38)
>   at scala.collection.TraversableLike.map(TraversableLike.scala:286)
>   at scala.collection.TraversableLike.map$(TraversableLike.scala:279)
>   at scala.collection.AbstractTraversable.map(Traversable.scala:108)
>   at 
> org.apache.spark.sql.execution.FileSourceScanLike.$anonfun$setFilesNumAndSizeMetric$2(DataSourceScanExec.scala:466)
>   at 
> org.apache.spark.sql.execution.FileSourceScanLike.$anonfun$setFilesNumAndSizeMetric$2$adapted(DataSourceScanExec.scala:466)
>   at scala.collection.immutable.List.map(List.scala:293)
>   at 
> org.apache.spark.sql.execution.FileSourceScanLike.setFilesNumAndSizeMetric(DataSourceScanExec.scala:466)
>   at 
> org.apache.spark.sql.execution.FileSourceScanLike.selectedPartitions(DataSourceScanExec.scala:257)
>   at 
> org.apache.spark.sql.execution.FileSourceScanLike.selectedPartitions$(DataSourceScanExec.scala:251)
>   at 
> org.apache.spark.sql.execution.FileSourceScanExec.selectedPartitions$lzycompute(DataSourceScanExec.scala:506)
>   at 
> org.apache.spark.sql.execution.FileSourceScanExec.selectedPartitions(DataSourceScanExec.scala:506)
>   at 
> org.apache.spark.sql.execution.FileSourceScanLike.dynamicallySelectedPartitions(DataSourceScanExec.scala:286)
>   at 
> org.apache.spark.sql.execution.FileSourceScanLike.dynamicallySelectedPartitions$(DataSourceScanExec.scala:267)
>   at 
> org.apache.spark.sql.execution.FileSourceScanExec.dynamicallySelectedPartitions$lzycompute(DataSourceScanExec.scala:506)
>   at 
> org.apache.spark.sql.execution.FileSourceScanExec.dynamicallySelectedPartitions(DataSourceScanExec.scala:506)
>   at 
> org.apache.spark.sql.execution.FileSourceScanExec.inputRDD$lzycompute(DataSourceScanExec.scala:553)
>   at 
> org.apache.spark.sql.execution.FileSourceScanExec.inputRDD(DataSourceScanExec.scala:537)
>   at 
> org.apache.spark.sql.execution.FileSourceScanExec.doExecute(DataSourceScanExec.scala:575)
>   at 
> org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:195)
>   at 
> org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:246)
>   at 
> org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
>   at 
> org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:243)
>   at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:191)
>   at 
> org.apache.spark.sql.execution.InputAdapter.inputRDD(WholeStageCodegenExec.scala:527)
>   at 
> org.apache.spark.sql.execution.InputRDDCodegen.inputRDDs(WholeStageCodegenExec.scala:455)
>   at 
> org.apache.spark.sql.execution.InputRDDCodegen.inputRDDs$(WholeStageCodegenExec.scala:454)
>   at 
> org.apache.spark.sql.execution.InputAdapter.inputRDDs(WholeStageCodegenExec.scala:498)
>   at 
> org.apache.spark.sql.execution.ProjectExec.inputRDDs(basicPhysicalOperators.scala:51)
>   at 
> org.apache.spark.sql.execution.FilterExec.inputRDDs(basicPhysicalOperators.scala:242)
>   at 
> org.apache.spark.sql.execution.ProjectExec.inputRDDs(basicPhysicalOperators.scala:51)
>   at 
> org.apache.spark.sql.execution.WholeStageCodegenExec.doExecute(WholeStageCodegenExec.scala:751)
>   at 
> org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:195)
>   at 
> org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:246)
>   at 
> org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
>   at 
> org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:243)
>   at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:191)
>   at 
> org.apache.spark.sql.execution.SparkPlan.getByteArrayRdd(SparkPlan.scala:364)
>   at 
> org.apache.spark.sql.execution.SparkPlan.executeCollect(SparkPlan.scala:445)
>   at org.apache.spark.sql.Dataset.collectFromPlan(Dataset.scala:4344)
>   at org.apache.spark.sql.Dataset.$anonfun$collect$1(Dataset.scala:3585)
>   at org.apache.spark.sql.Dataset.$anonfun$withAction$2(Dataset.scala:4334)
>   at 
> org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:546)
>   at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:4332)
>   at 
> org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:125)
>   at 
> org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:201)
>   at 
> org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:108)
>   at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:900)
>   at 
> org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:66)
>   at org.apache.spark.sql.Dataset.withAction(Dataset.scala:4332)
>   at org.apache.spark.sql.Dataset.collect(Dataset.scala:3585){code}
> The issue does not occur with the following configurations:
>  # _Spark 3.3.3, hadoop-aws:3.3.2, delta-core_2.12:1.2.1_
>  # _Spark 3.4.1, hadoop-aws:3.3.4, delta-core_2.12:2.4.0_



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org

Reply via email to