[ https://issues.apache.org/jira/browse/SPARK-27233?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16798972#comment-16798972 ]
Kritsada Limpawatkul commented on SPARK-27233: ---------------------------------------------- I found another problem probably related to this. My code is trying to save into Hive table with Parquet format with a partition column. {code:java} val testTable = "testseq" val df = Seq((1L, Some(Seq(1L,2L,3L))), (1L, None)).toDF("a", "seq") sqlContext.sql(s"DROP TABLE IF EXISTS $testTable") sqlContext.sql("SET hive.exec.dynamic.partition = true") sqlContext.sql("SET hive.exec.dynamic.partition.mode = nonstrict") df.write.format("hive").partitionBy("a").option("fileFormat", "parquet").saveAsTable(testTable){code} An error message is assertion failed. {code:java} assertion failed java.lang.AssertionError: assertion failed at scala.Predef$.assert(Predef.scala:156) at org.apache.spark.sql.hive.HiveMetastoreCatalog.convertToLogicalRelation(HiveMetastoreCatalog.scala:214) at org.apache.spark.sql.hive.RelationConversions.org$apache$spark$sql$hive$RelationConversions$$convert(HiveStrategies.scala:207) at org.apache.spark.sql.hive.RelationConversions$$anonfun$apply$4.applyOrElse(HiveStrategies.scala:239) at org.apache.spark.sql.hive.RelationConversions$$anonfun$apply$4.applyOrElse(HiveStrategies.scala:228) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$$anonfun$resolveOperatorsDown$1$$anonfun$2.apply(AnalysisHelper.scala:108) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$$anonfun$resolveOperatorsDown$1$$anonfun$2.apply(AnalysisHelper.scala:108) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:70) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$$anonfun$resolveOperatorsDown$1.apply(AnalysisHelper.scala:107) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$$anonfun$resolveOperatorsDown$1.apply(AnalysisHelper.scala:106) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.allowInvokingTransformsInAnalyzer(AnalysisHelper.scala:194) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$class.resolveOperatorsDown(AnalysisHelper.scala:106) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperatorsDown(LogicalPlan.scala:29) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$$anonfun$resolveOperatorsDown$1$$anonfun$apply$6.apply(AnalysisHelper.scala:113) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$$anonfun$resolveOperatorsDown$1$$anonfun$apply$6.apply(AnalysisHelper.scala:113) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:326) at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187) at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:324) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$$anonfun$resolveOperatorsDown$1.apply(AnalysisHelper.scala:113) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$$anonfun$resolveOperatorsDown$1.apply(AnalysisHelper.scala:106) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.allowInvokingTransformsInAnalyzer(AnalysisHelper.scala:194) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$class.resolveOperatorsDown(AnalysisHelper.scala:106) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperatorsDown(LogicalPlan.scala:29) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$class.resolveOperators(AnalysisHelper.scala:73) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperators(LogicalPlan.scala:29) at org.apache.spark.sql.hive.RelationConversions.apply(HiveStrategies.scala:228) at org.apache.spark.sql.hive.RelationConversions.apply(HiveStrategies.scala:180) at org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1$$anonfun$apply$1.apply(RuleExecutor.scala:87) at org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1$$anonfun$apply$1.apply(RuleExecutor.scala:84) at scala.collection.IndexedSeqOptimized$class.foldl(IndexedSeqOptimized.scala:57) at scala.collection.IndexedSeqOptimized$class.foldLeft(IndexedSeqOptimized.scala:66) at scala.collection.mutable.ArrayBuffer.foldLeft(ArrayBuffer.scala:48) at org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1.apply(RuleExecutor.scala:84) at org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1.apply(RuleExecutor.scala:76) at scala.collection.immutable.List.foreach(List.scala:392) at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:76) at org.apache.spark.sql.catalyst.analysis.Analyzer.org$apache$spark$sql$catalyst$analysis$Analyzer$$executeSameContext(Analyzer.scala:127) at org.apache.spark.sql.catalyst.analysis.Analyzer.execute(Analyzer.scala:121) at org.apache.spark.sql.catalyst.analysis.Analyzer$$anonfun$executeAndCheck$1.apply(Analyzer.scala:106) at org.apache.spark.sql.catalyst.analysis.Analyzer$$anonfun$executeAndCheck$1.apply(Analyzer.scala:105) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.markInAnalyzer(AnalysisHelper.scala:201) at org.apache.spark.sql.catalyst.analysis.Analyzer.executeAndCheck(Analyzer.scala:105) at org.apache.spark.sql.execution.QueryExecution.analyzed$lzycompute(QueryExecution.scala:57) at org.apache.spark.sql.execution.QueryExecution.analyzed(QueryExecution.scala:55) at org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:47) at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:79) at org.apache.spark.sql.SparkSession.table(SparkSession.scala:628) at org.apache.spark.sql.SparkSession.table(SparkSession.scala:624) at org.apache.spark.sql.internal.CatalogImpl.uncacheTable(CatalogImpl.scala:445) at org.apache.spark.sql.hive.execution.InsertIntoHiveTable.run(InsertIntoHiveTable.scala:107) at org.apache.spark.sql.hive.execution.CreateHiveTableAsSelectCommand.run(CreateHiveTableAsSelectCommand.scala:86) at org.apache.spark.sql.execution.command.DataWritingCommandExec.sideEffectResult$lzycompute(commands.scala:104) at org.apache.spark.sql.execution.command.DataWritingCommandExec.sideEffectResult(commands.scala:102) at org.apache.spark.sql.execution.command.DataWritingCommandExec.doExecute(commands.scala:122) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127) at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152) at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127) at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:80) at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:80) at org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:668) at org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:668) at org.apache.spark.sql.execution.SQLExecution$$anonfun$withNewExecutionId$1.apply(SQLExecution.scala:78) at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:125) at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:73) at org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:668) at org.apache.spark.sql.DataFrameWriter.createTable(DataFrameWriter.scala:465) at org.apache.spark.sql.DataFrameWriter.saveAsTable(DataFrameWriter.scala:444) at org.apache.spark.sql.DataFrameWriter.saveAsTable(DataFrameWriter.scala:400){code} After I looks at HiveMetastoreCatalog, the assertion is trying to check the schema. {code:java} // The inferred schema may have different field names as the table schema, we should respect // it, but also respect the exprId in table relation output. assert(result.output.length == relation.output.length && result.output.zip(relation.output).forall { case (a1, a2) => a1.dataType == a2.dataType }){code} And the change of containsNull in ArrayType makes the assertion failed. PS. It also happens when I change 'fileFormat' from 'parquet' to 'orc'. But 'sequencefile', 'rcfile', 'textfile' and 'avro' work correctly. > Schema of ArrayType change after saveAsTable and read > ------------------------------------------------------ > > Key: SPARK-27233 > URL: https://issues.apache.org/jira/browse/SPARK-27233 > Project: Spark > Issue Type: Bug > Components: SQL > Affects Versions: 2.4.0 > Environment: Spark 2.4 > Scala 2.11 > Reporter: Kritsada Limpawatkul > Priority: Major > > This is code for reproducing. > {code:java} > val testTable = "testseq" > val df = Seq(Some(Seq(1L,2L,3L)), None).toDF("seq") > sqlContext.sql(s"DROP TABLE IF EXISTS $testTable") > df.write.format("parquet").saveAsTable(testTable) > val res = sqlContext.table(testTable) > assert(df.schema === res.schema){code} > My code is trying to save dataframe with array type and read from that table. > After checking schema, I found that schema has change. > This is the assertion message. > {code:java} > Expected :StructType(StructField(seq,ArrayType(LongType,true),true)) > Actual :StructType(StructField(seq,ArrayType(LongType,false),true)){code} > containsNull in ArrayType changes from false to true after reading from the > table. -- This message was sent by Atlassian JIRA (v7.6.3#76005) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org