[ 
https://issues.apache.org/jira/browse/SPARK-25816?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16665637#comment-16665637
 ] 

Brian Zhang commented on SPARK-25816:
-------------------------------------

Here is another reproduce that should be related to this same issue:

val v0 = sqlContext.read.avro("final_allDatatypes_Spark.avro");
val v00 = v0.toDF(v0.schema.fields.indices.view.map("" + _):_*)
val v001 = v00.select($"0".as("0"), 
$"1".as("1"),$"2".as("2"),$"3".as("3"),$"4".as("4"),$"5".as("5"),$"6".as("6"),$"7".as("7"),$"8".as("8"))
val v013 = $"8"
val v010 = map(v013, v013)
 
v001.where(map(v013, v010)(v013)(v013)==="dummy")

 

org.apache.spark.sql.AnalysisException: Reference '8' is ambiguous, could be: 
8, 8.;
 at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:213)
 at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveChildren(LogicalPlan.scala:97)
 at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences$$anonfun$36.apply(Analyzer.scala:822)
 at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences$$anonfun$36.apply(Analyzer.scala:824)
 at 
org.apache.spark.sql.catalyst.analysis.package$.withPosition(package.scala:53)
 at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences$.org$apache$spark$sql$catalyst$analysis$Analyzer$ResolveReferences$$resolve(Analyzer.scala:821)
 at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences$$anonfun$org$apache$spark$sql$catalyst$analysis$Analyzer$ResolveReferences$$resolve$2.apply(Analyzer.scala:830)
 at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences$$anonfun$org$apache$spark$sql$catalyst$analysis$Analyzer$ResolveReferences$$resolve$2.apply(Analyzer.scala:830)
 at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:306)
 at 
org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187)
 at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304)
 at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences$.org$apache$spark$sql$catalyst$analysis$Analyzer$ResolveReferences$$resolve(Analyzer.scala:830)
 at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences$$anonfun$org$apache$spark$sql$catalyst$analysis$Analyzer$ResolveReferences$$resolve$2.apply(Analyzer.scala:830)
 at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences$$anonfun$org$apache$spark$sql$catalyst$analysis$Analyzer$ResolveReferences$$resolve$2.apply(Analyzer.scala:830)
 at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:306)
 at 
org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187)
 at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304)
 at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences$.org$apache$spark$sql$catalyst$analysis$Analyzer$ResolveReferences$$resolve(Analyzer.scala:830)
 at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences$$anonfun$apply$9$$anonfun$applyOrElse$36.apply(Analyzer.scala:891)
 at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences$$anonfun$apply$9$$anonfun$applyOrElse$36.apply(Analyzer.scala:891)
 at 
org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$1.apply(QueryPlan.scala:107)
 at 
org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$1.apply(QueryPlan.scala:107)
 at 
org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:70)
 at 
org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpression$1(QueryPlan.scala:106)
 at 
org.apache.spark.sql.catalyst.plans.QueryPlan.org$apache$spark$sql$catalyst$plans$QueryPlan$$recursiveTransform$1(QueryPlan.scala:118)
 at 
org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$2.apply(QueryPlan.scala:127)
 at 
org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187)
 at 
org.apache.spark.sql.catalyst.plans.QueryPlan.mapExpressions(QueryPlan.scala:127)
 at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences$$anonfun$apply$9.applyOrElse(Analyzer.scala:891)
 at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences$$anonfun$apply$9.applyOrElse(Analyzer.scala:833)
 at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:289)
 at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:289)
 at 
org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:70)
 at org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:288)
 at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:286)
 at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:286)
 at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:306)
 at 
org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187)
 at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304)
 at org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:286)
 at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences$.apply(Analyzer.scala:833)
 at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences$.apply(Analyzer.scala:690)
 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.LinearSeqOptimized$class.foldLeft(LinearSeqOptimized.scala:124)
 at scala.collection.immutable.List.foldLeft(List.scala:84)
 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:381)
 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:124)
 at org.apache.spark.sql.catalyst.analysis.Analyzer.execute(Analyzer.scala:118)
 at 
org.apache.spark.sql.catalyst.analysis.Analyzer.executeAndCheck(Analyzer.scala:103)
 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.<init>(Dataset.scala:172)
 at org.apache.spark.sql.Dataset.<init>(Dataset.scala:178)
 at org.apache.spark.sql.Dataset$.apply(Dataset.scala:65)
 at org.apache.spark.sql.Dataset.withTypedPlan(Dataset.scala:3301)
 at org.apache.spark.sql.Dataset.filter(Dataset.scala:1458)
 at org.apache.spark.sql.Dataset.where(Dataset.scala:1486)

 

> Functions does not resolve Columns correctly
> --------------------------------------------
>
>                 Key: SPARK-25816
>                 URL: https://issues.apache.org/jira/browse/SPARK-25816
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 2.3.0, 2.3.1
>            Reporter: Brian Zhang
>            Priority: Critical
>         Attachments: final_allDatatypes_Spark.avro, source.snappy.parquet
>
>
> When there is a duplicate column name in the current Dataframe and orginal 
> Dataframe where current df is selected from, Spark in 2.3.0 and 2.3.1 does 
> not resolve the column correctly when using it in the expression, hence 
> causing casting issue. The same code is working in Spark 2.2.1
> Please see below code to reproduce the issue
> import org.apache.spark._
> import org.apache.spark.rdd._
> import org.apache.spark.storage.StorageLevel._
> import org.apache.spark.sql._
> import org.apache.spark.sql.DataFrame
> import org.apache.spark.sql.types._
> import org.apache.spark.sql.functions._
> import org.apache.spark.sql.catalyst.expressions._
> import org.apache.spark.sql.Column
> val v0 = spark.read.parquet("/data/home/bzinfa/bz/source.snappy.parquet")
> val v00 = v0.toDF(v0.schema.fields.indices.view.map("" + _):_*)
> val v5 = v00.select($"13".as("0"),$"14".as("1"),$"15".as("2"))
> val v5_2 = $"2"
> v5.where(lit(500).<(v5_2(new Column(new MapKeys(v5_2.expr))(lit(0)))))
> //v00's 3rdcolumn is binary and 16th is map<string, double>
> Error:
> org.apache.spark.sql.AnalysisException: cannot resolve 'map_keys(`2`)' due to 
> data type mismatch: argument 1 requires map type, however, '`2`' is of binary 
> type.;
>  
>  'Project [0#1591, 1#1592, 2#1593] +- 'Filter (500 < 
> {color:#FF0000}2#1593{color}[map_keys({color:#FF0000}2#1561{color})[0]]) +- 
> Project [13#1572 AS 0#1591, 14#1573 AS 1#1592, 15#1574 AS 2#1593, 2#1561] +- 
> Project [c_bytes#1527 AS 0#1559, c_union#1528 AS 1#1560, c_fixed#1529 AS 
> 2#1561, c_boolean#1530 AS 3#1562, c_float#1531 AS 4#1563, c_double#1532 AS 
> 5#1564, c_int#1533 AS 6#1565, c_long#1534L AS 7#1566L, c_string#1535 AS 
> 8#1567, c_decimal_18_2#1536 AS 9#1568, c_decimal_28_2#1537 AS 10#1569, 
> c_decimal_38_2#1538 AS 11#1570, c_date#1539 AS 12#1571, simple_struct#1540 AS 
> 13#1572, simple_array#1541 AS 14#1573, simple_map#1542 AS 15#1574] +- 
> Relation[c_bytes#1527,c_union#1528,c_fixed#1529,c_boolean#1530,c_float#1531,c_double#1532,c_int#1533,c_long#1534L,c_string#1535,c_decimal_18_2#1536,c_decimal_28_2#1537,c_decimal_38_2#1538,c_date#1539,simple_struct#1540,simple_array#1541,simple_map#1542]
>  parquet



--
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

Reply via email to