Federico Ponzi created SPARK-13966: -------------------------------------- Summary: Regression using .withColumn() on a parquet Key: SPARK-13966 URL: https://issues.apache.org/jira/browse/SPARK-13966 Project: Spark Issue Type: Bug Affects Versions: 2.0.0 Environment: Linux 3.16.0-4-amd64 #1 SMP Debian 3.16.7-ckt20-1+deb8u3 (2016-01-17) x86_64 GNU/Linux
Reporter: Federico Ponzi If we load a parquet, add a column with {{withcolumn()}} with a date type and try to select a join of himself, we get a {{java.util.NoSuchElementException: key not found: key#6}} Here is a simple program to reproduce it: {code} from pyspark.sql import SQLContext, Row from pyspark import SparkContext from pyspark.sql.functions import from_unixtime, lit sc = SparkContext() sqlContext = SQLContext(sc) df = sqlContext.createDataFrame(sc.parallelize([Row(x=123)])) df.write.parquet("/tmp/testcase", mode="overwrite") df = sqlContext.read.parquet("/tmp/testcase") # df = df.unionAll(df.limit(0)) # WORKAROUND df = df.withColumn("key", from_unixtime(lit(1457650800))) # also happens with a .cast("timestamp") df.registerTempTable("test") res = sqlContext.sql("SELECT COUNT(1) from test t1, test t2 where t1.key = t2.key") res.show() {code} This only occurs when the added columns is of type timestamp, and dosen't happen in Spark 1.6.x {noformat} Traceback (most recent call last): File "/tmp/bug.py", line 17, in <module> res.show() File "/usr/local/spark/python/lib/pyspark.zip/pyspark/sql/dataframe.py", line 217, in show File "/usr/local/spark/python/lib/py4j-0.9.2-src.zip/py4j/java_gateway.py", line 836, in __call__ File "/usr/local/spark/python/lib/pyspark.zip/pyspark/sql/utils.py", line 45, in deco File "/usr/local/spark/python/lib/py4j-0.9.2-src.zip/py4j/protocol.py", line 310, in get_return_value py4j.protocol.Py4JJavaError: An error occurred while calling o67.showString. : java.util.NoSuchElementException: key not found: key#6 at scala.collection.MapLike$class.default(MapLike.scala:228) at org.apache.spark.sql.catalyst.expressions.AttributeMap.default(AttributeMap.scala:38) at scala.collection.MapLike$class.apply(MapLike.scala:141) at org.apache.spark.sql.catalyst.expressions.AttributeMap.apply(AttributeMap.scala:38) at org.apache.spark.sql.execution.datasources.DataSourceStrategy$$anonfun$35$$anonfun$apply$2.applyOrElse(DataSourceStrategy.scala:566) at org.apache.spark.sql.execution.datasources.DataSourceStrategy$$anonfun$35$$anonfun$apply$2.applyOrElse(DataSourceStrategy.scala:565) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:259) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:259) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:67) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:258) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:264) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:264) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:301) at scala.collection.Iterator$$anon$11.next(Iterator.scala:370) at scala.collection.Iterator$class.foreach(Iterator.scala:742) at scala.collection.AbstractIterator.foreach(Iterator.scala:1194) at scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:59) at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:104) at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:48) at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:308) at scala.collection.AbstractIterator.to(Iterator.scala:1194) at scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:300) at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1194) at scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:287) at scala.collection.AbstractIterator.toArray(Iterator.scala:1194) at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:350) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:264) at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:248) at org.apache.spark.sql.execution.datasources.DataSourceStrategy$$anonfun$35.apply(DataSourceStrategy.scala:565) at org.apache.spark.sql.execution.datasources.DataSourceStrategy$$anonfun$35.apply(DataSourceStrategy.scala:565) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:245) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:245) at scala.collection.immutable.List.foreach(List.scala:381) at scala.collection.TraversableLike$class.map(TraversableLike.scala:245) at scala.collection.immutable.List.map(List.scala:285) at org.apache.spark.sql.execution.datasources.DataSourceStrategy$.pruneFilterProjectRaw(DataSourceStrategy.scala:565) at org.apache.spark.sql.execution.datasources.DataSourceStrategy$.pruneFilterProject(DataSourceStrategy.scala:533) at org.apache.spark.sql.execution.datasources.DataSourceStrategy$.apply(DataSourceStrategy.scala:225) at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58) at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58) at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:396) at org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:59) at org.apache.spark.sql.catalyst.planning.QueryPlanner.planLater(QueryPlanner.scala:54) at org.apache.spark.sql.execution.SparkStrategies$EquiJoinSelection$.apply(SparkStrategies.scala:114) at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58) at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58) at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:396) at org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:59) at org.apache.spark.sql.catalyst.planning.QueryPlanner.planLater(QueryPlanner.scala:54) at org.apache.spark.sql.execution.SparkStrategies$BasicOperators$.apply(SparkStrategies.scala:342) at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58) at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58) at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:396) at org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:59) at org.apache.spark.sql.catalyst.planning.QueryPlanner.planLater(QueryPlanner.scala:54) at org.apache.spark.sql.execution.SparkStrategies$Aggregation$.apply(SparkStrategies.scala:234) at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58) at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58) at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:396) at org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:59) at org.apache.spark.sql.catalyst.planning.QueryPlanner.planLater(QueryPlanner.scala:54) at org.apache.spark.sql.execution.SparkStrategies$SpecialLimits$.apply(SparkStrategies.scala:52) at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58) at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58) at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:396) at org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:59) at org.apache.spark.sql.execution.QueryExecution.sparkPlan$lzycompute(QueryExecution.scala:52) at org.apache.spark.sql.execution.QueryExecution.sparkPlan(QueryExecution.scala:50) at org.apache.spark.sql.execution.QueryExecution.executedPlan$lzycompute(QueryExecution.scala:57) at org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:57) at org.apache.spark.sql.Dataset.withTypedCallback(Dataset.scala:2085) at org.apache.spark.sql.Dataset.head(Dataset.scala:1621) at org.apache.spark.sql.Dataset.take(Dataset.scala:1765) at org.apache.spark.sql.Dataset.showString(Dataset.scala:189) 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:498) at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:231) at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:381) at py4j.Gateway.invoke(Gateway.java:290) at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:133) at py4j.commands.CallCommand.execute(CallCommand.java:79) at py4j.GatewayConnection.run(GatewayConnection.java:209) at java.lang.Thread.run(Thread.java:745) {noformat} -- 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