You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Emlyn Corrin (JIRA)" <ji...@apache.org> on 2016/10/27 23:31:58 UTC
[jira] [Commented] (SPARK-16648) LAST_VALUE(FALSE) OVER () throws
IndexOutOfBoundsException
[ https://issues.apache.org/jira/browse/SPARK-16648?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15613656#comment-15613656 ]
Emlyn Corrin commented on SPARK-16648:
--------------------------------------
Since Spark 2.0.1, the following snippet fails (I believe it worked under 2.0.0, so this issue seems like the most likely cause of change in behaviour):
{code}
from pyspark.sql import functions as F
ds = spark.createDataFrame(sc.parallelize([[1, 1, 2], [1, 2, 3], [1, 3, 4]]))
ds.groupBy(ds._1).agg(F.first(ds._2), F.countDistinct(ds._2), F.countDistinct(ds._2, ds._3)).show()
{code}
It works if any of the three arguments to {{.agg}} is removed.
The stack trace is:
{code}
Py4JJavaError Traceback (most recent call last)
<ipython-input-3-73596fd1f689> in <module>()
----> 1 ds.groupBy(ds._1).agg(F.first(ds._2),F.countDistinct(ds._2),F.countDistinct(ds._2, ds._3)).show()
/usr/local/Cellar/apache-spark/2.0.1/libexec/python/pyspark/sql/dataframe.py in show(self, n, truncate)
285 +---+-----+
286 """
--> 287 print(self._jdf.showString(n, truncate))
288
289 def __repr__(self):
/usr/local/Cellar/apache-spark/2.0.1/libexec/python/lib/py4j-0.10.3-src.zip/py4j/java_gateway.py in __call__(self, *args)
1131 answer = self.gateway_client.send_command(command)
1132 return_value = get_return_value(
-> 1133 answer, self.gateway_client, self.target_id, self.name)
1134
1135 for temp_arg in temp_args:
/usr/local/Cellar/apache-spark/2.0.1/libexec/python/pyspark/sql/utils.py in deco(*a, **kw)
61 def deco(*a, **kw):
62 try:
---> 63 return f(*a, **kw)
64 except py4j.protocol.Py4JJavaError as e:
65 s = e.java_exception.toString()
/usr/local/Cellar/apache-spark/2.0.1/libexec/python/lib/py4j-0.10.3-src.zip/py4j/protocol.py in get_return_value(answer, gateway_client, target_id, name)
317 raise Py4JJavaError(
318 "An error occurred while calling {0}{1}{2}.\n".
--> 319 format(target_id, ".", name), value)
320 else:
321 raise Py4JError(
Py4JJavaError: An error occurred while calling o76.showString.
: org.apache.spark.sql.catalyst.errors.package$TreeNodeException: makeCopy, tree: first(_2#1L)()
at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:56)
at org.apache.spark.sql.catalyst.trees.TreeNode.makeCopy(TreeNode.scala:387)
at org.apache.spark.sql.catalyst.trees.TreeNode.withNewChildren(TreeNode.scala:256)
at org.apache.spark.sql.catalyst.optimizer.RewriteDistinctAggregates$.org$apache$spark$sql$catalyst$optimizer$RewriteDistinctAggregates$$patchAggregateFunctionChildren$1(RewriteDistinctAggregates.scala:140)
at org.apache.spark.sql.catalyst.optimizer.RewriteDistinctAggregates$$anonfun$16.apply(RewriteDistinctAggregates.scala:182)
at org.apache.spark.sql.catalyst.optimizer.RewriteDistinctAggregates$$anonfun$16.apply(RewriteDistinctAggregates.scala:180)
at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
at scala.collection.AbstractTraversable.map(Traversable.scala:104)
at org.apache.spark.sql.catalyst.optimizer.RewriteDistinctAggregates$.rewrite(RewriteDistinctAggregates.scala:180)
at org.apache.spark.sql.catalyst.optimizer.RewriteDistinctAggregates$$anonfun$apply$1.applyOrElse(RewriteDistinctAggregates.scala:105)
at org.apache.spark.sql.catalyst.optimizer.RewriteDistinctAggregates$$anonfun$apply$1.applyOrElse(RewriteDistinctAggregates.scala:104)
at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:301)
at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:301)
at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:69)
at org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:300)
at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:298)
at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:298)
at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:321)
at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:179)
at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:319)
at org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:298)
at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:298)
at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:298)
at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:321)
at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:179)
at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:319)
at org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:298)
at org.apache.spark.sql.catalyst.optimizer.RewriteDistinctAggregates$.apply(RewriteDistinctAggregates.scala:104)
at org.apache.spark.sql.catalyst.optimizer.RewriteDistinctAggregates$.apply(RewriteDistinctAggregates.scala:102)
at org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1$$anonfun$apply$1.apply(RuleExecutor.scala:85)
at org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1$$anonfun$apply$1.apply(RuleExecutor.scala:82)
at scala.collection.IndexedSeqOptimized$class.foldl(IndexedSeqOptimized.scala:57)
at scala.collection.IndexedSeqOptimized$class.foldLeft(IndexedSeqOptimized.scala:66)
at scala.collection.mutable.WrappedArray.foldLeft(WrappedArray.scala:35)
at org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1.apply(RuleExecutor.scala:82)
at org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1.apply(RuleExecutor.scala:74)
at scala.collection.immutable.List.foreach(List.scala:381)
at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:74)
at org.apache.spark.sql.execution.QueryExecution.optimizedPlan$lzycompute(QueryExecution.scala:74)
at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:74)
at org.apache.spark.sql.execution.QueryExecution.sparkPlan$lzycompute(QueryExecution.scala:78)
at org.apache.spark.sql.execution.QueryExecution.sparkPlan(QueryExecution.scala:76)
at org.apache.spark.sql.execution.QueryExecution.executedPlan$lzycompute(QueryExecution.scala:83)
at org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:83)
at org.apache.spark.sql.Dataset.withTypedCallback(Dataset.scala:2572)
at org.apache.spark.sql.Dataset.head(Dataset.scala:1934)
at org.apache.spark.sql.Dataset.take(Dataset.scala:2149)
at org.apache.spark.sql.Dataset.showString(Dataset.scala:239)
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 py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:237)
at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)
at py4j.Gateway.invoke(Gateway.java:280)
at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
at py4j.commands.CallCommand.execute(CallCommand.java:79)
at py4j.GatewayConnection.run(GatewayConnection.java:214)
at java.lang.Thread.run(Thread.java:745)
Caused by: java.lang.reflect.InvocationTargetException
at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
at java.lang.reflect.Constructor.newInstance(Constructor.java:408)
at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$makeCopy$1$$anonfun$apply$13.apply(TreeNode.scala:413)
at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$makeCopy$1$$anonfun$apply$13.apply(TreeNode.scala:413)
at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:69)
at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$makeCopy$1.apply(TreeNode.scala:412)
at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$makeCopy$1.apply(TreeNode.scala:387)
at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:52)
... 62 more
Caused by: org.apache.spark.sql.AnalysisException: The second argument of First should be a boolean literal.;
at org.apache.spark.sql.catalyst.expressions.aggregate.First.<init>(First.scala:43)
... 72 more
{code}
> LAST_VALUE(FALSE) OVER () throws IndexOutOfBoundsException
> ----------------------------------------------------------
>
> Key: SPARK-16648
> URL: https://issues.apache.org/jira/browse/SPARK-16648
> Project: Spark
> Issue Type: Bug
> Components: SQL
> Affects Versions: 2.0.0
> Reporter: Cheng Lian
> Assignee: Cheng Lian
> Fix For: 2.0.1, 2.1.0
>
>
> The following simple SQL query reproduces this issue:
> {code:sql}
> SELECT LAST_VALUE(FALSE) OVER ();
> {code}
> Exception thrown:
> {noformat}
> java.lang.IndexOutOfBoundsException: 0
> at scala.collection.mutable.ResizableArray$class.apply(ResizableArray.scala:43)
> at scala.collection.mutable.ArrayBuffer.apply(ArrayBuffer.scala:48)
> at scala.collection.mutable.ArrayBuffer.remove(ArrayBuffer.scala:169)
> at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$2.apply(TreeNode.scala:244)
> at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:179)
> at org.apache.spark.sql.catalyst.trees.TreeNode.withNewChildren(TreeNode.scala:214)
> at org.apache.spark.sql.catalyst.analysis.TypeCoercion$ImplicitTypeCasts$$anonfun$apply$12.applyOrElse(TypeCoercion.scala:637)
> at org.apache.spark.sql.catalyst.analysis.TypeCoercion$ImplicitTypeCasts$$anonfun$apply$12.applyOrElse(TypeCoercion.scala:615)
> at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:279)
> at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:279)
> at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:69)
> at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:278)
> at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:284)
> at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:284)
> at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:321)
> at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:179)
> at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:319)
> at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:284)
> at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:284)
> at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:284)
> at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:321)
> at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:179)
> at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:319)
> at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:284)
> at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:284)
> at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:284)
> at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:321)
> at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:179)
> at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:319)
> at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:284)
> at org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpressionDown$1(QueryPlan.scala:156)
> at org.apache.spark.sql.catalyst.plans.QueryPlan.org$apache$spark$sql$catalyst$plans$QueryPlan$$recursiveTransform$1(QueryPlan.scala:166)
> at org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$org$apache$spark$sql$catalyst$plans$QueryPlan$$recursiveTransform$1$1.apply(QueryPlan.scala:170)
> at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
> at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
> at scala.collection.immutable.List.foreach(List.scala:381)
> at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
> at scala.collection.immutable.List.map(List.scala:285)
> at org.apache.spark.sql.catalyst.plans.QueryPlan.org$apache$spark$sql$catalyst$plans$QueryPlan$$recursiveTransform$1(QueryPlan.scala:170)
> at org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$4.apply(QueryPlan.scala:175)
> at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:179)
> at org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpressionsDown(QueryPlan.scala:175)
> at org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpressions(QueryPlan.scala:144)
> at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolveExpressions$1.applyOrElse(LogicalPlan.scala:79)
> at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolveExpressions$1.applyOrElse(LogicalPlan.scala:78)
> at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolveOperators$1.apply(LogicalPlan.scala:61)
> at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolveOperators$1.apply(LogicalPlan.scala:61)
> at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:69)
> at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperators(LogicalPlan.scala:60)
> at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveExpressions(LogicalPlan.scala:78)
> at org.apache.spark.sql.catalyst.analysis.TypeCoercion$ImplicitTypeCasts$.apply(TypeCoercion.scala:615)
> at org.apache.spark.sql.catalyst.analysis.TypeCoercion$ImplicitTypeCasts$.apply(TypeCoercion.scala:614)
> at org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1$$anonfun$apply$1.apply(RuleExecutor.scala:85)
> at org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1$$anonfun$apply$1.apply(RuleExecutor.scala:82)
> 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:82)
> at org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1.apply(RuleExecutor.scala:74)
> at scala.collection.immutable.List.foreach(List.scala:381)
> at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:74)
> at org.apache.spark.sql.execution.QueryExecution.analyzed$lzycompute(QueryExecution.scala:65)
> at org.apache.spark.sql.execution.QueryExecution.analyzed(QueryExecution.scala:63)
> at org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:49)
> at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:64)
> at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:582)
> ... 48 elided
> {noformat}
> This bug is a regression. Spark 1.6 doesn't have this issue.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)
---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org