[ https://issues.apache.org/jira/browse/SPARK-13932?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15230767#comment-15230767 ]
Davies Liu commented on SPARK-13932: ------------------------------------ This will be fixed in https://github.com/apache/spark/pull/12235 BTW, GROUPING__ID will be deprecated in 2.0, you need to change to use grouping_id(), because GROUPING__ID have different result than documented. > CUBE Query with filter (HAVING) and condition (IF) raises an AnalysisException > ------------------------------------------------------------------------------ > > Key: SPARK-13932 > URL: https://issues.apache.org/jira/browse/SPARK-13932 > Project: Spark > Issue Type: Bug > Components: SQL > Affects Versions: 1.6.0, 1.6.1, 2.0.0 > Reporter: Tien-Dung LE > > A complex aggregate query using condition in the aggregate function and GROUP > BY HAVING clause raises an exception. This issue only happens in Spark > version 1.6.+ but not in Spark 1.5.+. > Here is a typical error message {code} > org.apache.spark.sql.AnalysisException: Reference 'b' is ambiguous, could be: > b#55, b#124.; line 1 pos 178 > at > org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:287) > {code} > Here is a code snippet to re-produce the error in a spark-shell session: > {code} > import sqlContext.implicits._ > case class Toto( a: String = f"${(math.random*1e6).toLong}%06.0f", > b: Int = (math.random*1e3).toInt, > n: Int = (math.random*1e3).toInt, > m: Double = (math.random*1e3)) > val data = sc.parallelize(1 to 1e6.toInt).map(i => Toto()) > val df: org.apache.spark.sql.DataFrame = sqlContext.createDataFrame( data ) > df.registerTempTable( "toto" ) > val sqlSelect1 = "SELECT a, b, COUNT(1) AS k1, COUNT(1) AS k2, SUM(m) AS > k3, GROUPING__ID" > val sqlSelect2 = "SELECT a, b, COUNT(1) AS k1, COUNT(IF(n > 500,1,0)) AS > k2, SUM(m) AS k3, GROUPING__ID" > val sqlGroupBy = "FROM toto GROUP BY a, b GROUPING SETS ((a,b),(a),(b))" > val sqlHaving = "HAVING ((GROUPING__ID & 1) == 1) AND (b > 500)" > sqlContext.sql( s"$sqlSelect1 $sqlGroupBy $sqlHaving" ) // OK > sqlContext.sql( s"$sqlSelect2 $sqlGroupBy" ) // OK > sqlContext.sql( s"$sqlSelect2 $sqlGroupBy $sqlHaving" ) // ERROR > {code} > And here is the full log > {code} > scala> sqlContext.sql( s"$sqlSelect1 $sqlGroupBy $sqlHaving" ) > res12: org.apache.spark.sql.DataFrame = [a: string, b: int, k1: bigint, k2: > bigint, k3: double, GROUPING__ID: int] > scala> sqlContext.sql( s"$sqlSelect2 $sqlGroupBy" ) > res13: org.apache.spark.sql.DataFrame = [a: string, b: int, k1: bigint, k2: > bigint, k3: double, GROUPING__ID: int] > scala> sqlContext.sql( s"$sqlSelect2 $sqlGroupBy $sqlHaving" ) // ERROR > org.apache.spark.sql.AnalysisException: Reference 'b' is ambiguous, could be: > b#55, b#124.; line 1 pos 178 > 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.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:316) > at > org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:316) > at > org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:265) > 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.trees.TreeNode.transformChildren(TreeNode.scala:305) > at > org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:316) > at > org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:316) > at > org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:316) > at > org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:265) > 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.trees.TreeNode.transformChildren(TreeNode.scala:305) > at > org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:316) > at > org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:316) > at > org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:316) > at > org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:265) > 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.trees.TreeNode.transformChildren(TreeNode.scala:305) > at > org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:316) > at > org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:316) > at > org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:316) > at > org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:265) > 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.trees.TreeNode.transformChildren(TreeNode.scala:305) > at > org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:316) > 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.scala:121) > 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.catalyst.analysis.Analyzer$ResolveAggregateFunctions$$anonfun$apply$14.applyOrElse(Analyzer.scala:622) > at > org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAggregateFunctions$$anonfun$apply$14.applyOrElse(Analyzer.scala:614) > 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$ResolveAggregateFunctions$.apply(Analyzer.scala:614) > at > org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAggregateFunctions$.apply(Analyzer.scala:613) > 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.SQLContext.sql(SQLContext.scala:817) > at > $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:39) > at > $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:44) > at > $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:46) > at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:48) > at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:50) > at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:52) > at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:54) > at $iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:56) > at $iwC$$iwC$$iwC$$iwC.<init>(<console>:58) > at $iwC$$iwC$$iwC.<init>(<console>:60) > at $iwC$$iwC.<init>(<console>:62) > at $iwC.<init>(<console>:64) > at <init>(<console>:66) > at .<init>(<console>:70) > at .<clinit>(<console>) > at .<init>(<console>:7) > at .<clinit>(<console>) > at $print(<console>) > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > at > sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:483) > at > org.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065) > at > org.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1346) > at > org.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:840) > at org.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:871) > at org.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:819) > at > org.apache.spark.repl.SparkILoop.reallyInterpret$1(SparkILoop.scala:857) > at > org.apache.spark.repl.SparkILoop.interpretStartingWith(SparkILoop.scala:902) > at org.apache.spark.repl.SparkILoop.command(SparkILoop.scala:814) > at org.apache.spark.repl.SparkILoop.processLine$1(SparkILoop.scala:657) > at org.apache.spark.repl.SparkILoop.innerLoop$1(SparkILoop.scala:665) > at > org.apache.spark.repl.SparkILoop.org$apache$spark$repl$SparkILoop$$loop(SparkILoop.scala:670) > at > org.apache.spark.repl.SparkILoop$$anonfun$org$apache$spark$repl$SparkILoop$$process$1.apply$mcZ$sp(SparkILoop.scala:997) > at > org.apache.spark.repl.SparkILoop$$anonfun$org$apache$spark$repl$SparkILoop$$process$1.apply(SparkILoop.scala:945) > at > org.apache.spark.repl.SparkILoop$$anonfun$org$apache$spark$repl$SparkILoop$$process$1.apply(SparkILoop.scala:945) > at > scala.tools.nsc.util.ScalaClassLoader$.savingContextLoader(ScalaClassLoader.scala:135) > at > org.apache.spark.repl.SparkILoop.org$apache$spark$repl$SparkILoop$$process(SparkILoop.scala:945) > at org.apache.spark.repl.SparkILoop.process(SparkILoop.scala:1059) > at org.apache.spark.repl.Main$.main(Main.scala:31) > at org.apache.spark.repl.Main.main(Main.scala) > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > at > sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:483) > at > org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:731) > at > org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:181) > at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:206) > at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:121) > at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org