[
https://issues.apache.org/jira/browse/SPARK-13493?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16280909#comment-16280909
]
Ratnakar Choudhary edited comment on SPARK-13493 at 12/6/17 9:06 PM:
---------------------------------------------------------------------
Hi, I am on Spark 2.1 and still seeing this issue where I am providing the
schema while reading the Json files and getting the DataSet however when I try
to write or perform any operation on this DataSet I get error of the column
ambiguous due to the case sensitivity of 2 columns.
Is there any work around that I can use? I tried to set the conf with the
spark.sql.caseSensitiveAnalysis = true along with spark.sql.caseSensitive=true
but did not help.
sample code:
StructType schema = sampleRows.schema();
Dataset<Row> rows = new
DataFrameReader(sparkSession).schema(schema).json(s3AKeys.toArray(new
String[s3AKeys.size()])).repartition(numberOfPartitions);
rows.write().mode(SaveMode.Append).parquet(String.format(S3A_PREFIX + "%s/%s",
outputBucket, outputPrefix));
was (Author: [email protected]):
Hi, I am on Spark 2.1 and still seeing this issue where I am providing the
schema while reading the Json files and getting the DataSet however when I try
to write or perform any operation on this DataSet I get error of the column
ambiguous due to the case sensitivity of 2 columns.
Is there any work around that I can use? I tried to set the conf with the
spark.sql.caseSensitiveAnalysis = true along with spark.sql.caseSensitive=true
but did not help.
> json to DataFrame to parquet does not respect case sensitiveness
> ----------------------------------------------------------------
>
> Key: SPARK-13493
> URL: https://issues.apache.org/jira/browse/SPARK-13493
> Project: Spark
> Issue Type: Bug
> Components: SQL
> Affects Versions: 1.6.0
> Reporter: Michel Lemay
> Priority: Minor
>
> Not sure where the problem should be fixed exactly but here it is:
> {noformat}
> $ spark-shell --conf spark.sql.caseSensitive=false
> scala> sqlContext.getConf("spark.sql.caseSensitive")
> res2: String = false
> scala> val data = List("""{"field": 1}""","""{"field": 2}""","""{"field":
> 3}""","""{"field": 4}""","""{"FIELD": 5}""")
> scala> val jsonDF = sqlContext.read.json(sc.parallelize(data))
> scala> jsonDF.printSchema
> root
> |-- FIELD: long (nullable = true)
> |-- field: long (nullable = true)
> {noformat}
> And when persisting this as parquet:
> {noformat}
> scala> jsonDF.write.parquet("out")
> org.apache.spark.sql.AnalysisException: Reference 'FIELD' is ambiguous, could
> be: FIELD#0L, FIELD#1L.;
> at
> org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:287)
> at
> org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveChildren(LogicalPlan.scala:171)
> at
> org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences$$anonfun$apply$10$$anonfun$applyOrElse$4$$anonfun$26.apply(Analyzer.scala:471
> at
> org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences$$anonfun$apply$10$$anonfun$applyOrElse$4$$anonfun$26.apply(Analyzer.scala:471
> at
> org.apache.spark.sql.catalyst.analysis.package$.withPosition(package.scala:48)
> at
> org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences$$anonfun$apply$10$$anonfun$applyOrElse$4.applyOrElse(Analyzer.scala:471)
> at
> org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences$$anonfun$apply$10$$anonfun$applyOrElse$4.applyOrElse(Analyzer.scala:467)
> at
> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:319)
> at
> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:319)
> at
> org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:53)
> at
> org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:318)
> at
> org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpressionUp$1(QueryPlan.scala:107)
> at
> org.apache.spark.sql.catalyst.plans.QueryPlan.org$apache$spark$sql$catalyst$plans$QueryPlan$$recursiveTransform$2(QueryPlan.scala:117)
> at
> org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$org$apache$spark$sql$catalyst$plans$QueryPlan$$recursiveTransform$2$1.apply(QueryPlan.sc
> at
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
> at
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
> at scala.collection.immutable.List.foreach(List.scala:318)
> at
> scala.collection.TraversableLike$class.map(TraversableLike.scala:244)
> at scala.collection.AbstractTraversable.map(Traversable.scala:105)
> at
> org.apache.spark.sql.catalyst.plans.QueryPlan.org$apache$spark$sql$catalyst$plans$QueryPlan$$recursiveTransform$2(QueryPlan.scala:121)
> at
> org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$2.apply(QueryPlan.scala:125)
> at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
> at scala.collection.Iterator$class.foreach(Iterator.scala:727)
> at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
> at
> scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:48)
> at
> scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:103)
> at
> scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:47)
> at
> scala.collection.TraversableOnce$class.to(TraversableOnce.scala:273)
> at scala.collection.AbstractIterator.to(Iterator.scala:1157)
> at
> scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:265)
> at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1157)
> at
> scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:252)
> at scala.collection.AbstractIterator.toArray(Iterator.scala:1157)
> at
> org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpressionsUp(QueryPlan.scala:125)
> at
> org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences$$anonfun$apply$10.applyOrElse(Analyzer.scala:467)
> at
> org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences$$anonfun$apply$10.applyOrElse(Analyzer.scala:347)
> at
> org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolveOperators$1.apply(LogicalPlan.scala:57)
> at
> org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolveOperators$1.apply(LogicalPlan.scala:57)
> at
> org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:53)
> at
> org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperators(LogicalPlan.scala:56)
> at
> org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences$.apply(Analyzer.scala:347)
> at
> org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences$.apply(Analyzer.scala:328)
> at
> org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1$$anonfun$apply$1.apply(RuleExecutor.scala:83)
> at
> org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1$$anonfun$apply$1.apply(RuleExecutor.scala:80)
> at
> scala.collection.LinearSeqOptimized$class.foldLeft(LinearSeqOptimized.scala:111)
> at scala.collection.immutable.List.foldLeft(List.scala:84)
> at
> org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1.apply(RuleExecutor.scala:80)
> at
> org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1.apply(RuleExecutor.scala:72)
> at scala.collection.immutable.List.foreach(List.scala:318)
> at
> org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:72)
> at
> org.apache.spark.sql.execution.QueryExecution.analyzed$lzycompute(QueryExecution.scala:36)
> at
> org.apache.spark.sql.execution.QueryExecution.analyzed(QueryExecution.scala:36)
> at
> org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:34)
> at org.apache.spark.sql.DataFrame.<init>(DataFrame.scala:133)
> at org.apache.spark.sql.DataFrame$.apply(DataFrame.scala:52)
> at
> org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelation.run(InsertIntoHadoopFsRelation.scala:106)
> at
> org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult$lzycompute(commands.scala:58)
> at
> org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult(commands.scala:56)
> at
> org.apache.spark.sql.execution.ExecutedCommand.doExecute(commands.scala:70)
> at
> org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:132)
> at
> org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:130)
> at
> org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:150)
> at
> org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:130)
> at
> org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:55)
> at
> org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:55)
> at
> org.apache.spark.sql.execution.datasources.ResolvedDataSource$.apply(ResolvedDataSource.scala:256)
> at
> org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:148)
> at
> org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:139)
> at
> org.apache.spark.sql.DataFrameWriter.parquet(DataFrameWriter.scala:329)
> {noformat}
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]