Andreas Maier created SPARK-22249:
-------------------------------------

             Summary: UnsupportedOperationException: empty.reduceLeft when 
caching a dataframe
                 Key: SPARK-22249
                 URL: https://issues.apache.org/jira/browse/SPARK-22249
             Project: Spark
          Issue Type: Bug
          Components: PySpark
    Affects Versions: 2.2.0
         Environment: $ uname -a
Darwin MAC-UM-024.local 16.7.0 Darwin Kernel Version 16.7.0: Thu Jun 15 
17:36:27 PDT 2017; root:xnu-3789.70.16~2/RELEASE_X86_64 x86_64
$ pyspark --version
Welcome to
      ____              __
     / __/__  ___ _____/ /__
    _\ \/ _ \/ _ `/ __/  '_/
   /___/ .__/\_,_/_/ /_/\_\   version 2.2.0
      /_/
                        
Using Scala version 2.11.8, Java HotSpot(TM) 64-Bit Server VM, 1.8.0_92
Branch 
Compiled by user jenkins on 2017-06-30T22:58:04Z
Revision 
Url 
            Reporter: Andreas Maier


It seems that the {{isin()}} method with an empty list as argument only works, 
if the dataframe is not cached. If it is cached, it results in an exception. To 
reproduce

{code:java}
$ pyspark
>>> df = spark.createDataFrame([pyspark.Row(KEY="value")])
>>> df.where(df["KEY"].isin([])).show()
+---+
|KEY|
+---+
+---+

>>> df.cache()
DataFrame[KEY: string]
>>> df.where(df["KEY"].isin([])).show()
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File 
"/usr/local/anaconda3/envs/<myenv>/lib/python3.6/site-packages/pyspark/sql/dataframe.py",
 line 336, in show
    print(self._jdf.showString(n, 20))
  File 
"/usr/local/anaconda3/envs/<myenv>/lib/python3.6/site-packages/pyspark/python/lib/py4j-0.10.4-src.zip/py4j/java_gateway.py",
 line 1133, in __call__
  File 
"/usr/local/anaconda3/envs/<myenv>/lib/python3.6/site-packages/pyspark/sql/utils.py",
 line 63, in deco
    return f(*a, **kw)
  File 
"/usr/local/anaconda3/envs/<myenv>/lib/python3.6/site-packages/pyspark/python/lib/py4j-0.10.4-src.zip/py4j/protocol.py",
 line 319, in get_return_value
py4j.protocol.Py4JJavaError: An error occurred while calling o302.showString.
: java.lang.UnsupportedOperationException: empty.reduceLeft
        at 
scala.collection.TraversableOnce$class.reduceLeft(TraversableOnce.scala:180)
        at 
scala.collection.mutable.ArrayBuffer.scala$collection$IndexedSeqOptimized$$super$reduceLeft(ArrayBuffer.scala:48)
        at 
scala.collection.IndexedSeqOptimized$class.reduceLeft(IndexedSeqOptimized.scala:74)
        at scala.collection.mutable.ArrayBuffer.reduceLeft(ArrayBuffer.scala:48)
        at 
scala.collection.TraversableOnce$class.reduce(TraversableOnce.scala:208)
        at scala.collection.AbstractTraversable.reduce(Traversable.scala:104)
        at 
org.apache.spark.sql.execution.columnar.InMemoryTableScanExec$$anonfun$1.applyOrElse(InMemoryTableScanExec.scala:107)
        at 
org.apache.spark.sql.execution.columnar.InMemoryTableScanExec$$anonfun$1.applyOrElse(InMemoryTableScanExec.scala:71)
        at scala.PartialFunction$Lifted.apply(PartialFunction.scala:223)
        at scala.PartialFunction$Lifted.apply(PartialFunction.scala:219)
        at 
org.apache.spark.sql.execution.columnar.InMemoryTableScanExec$$anonfun$2.apply(InMemoryTableScanExec.scala:112)
        at 
org.apache.spark.sql.execution.columnar.InMemoryTableScanExec$$anonfun$2.apply(InMemoryTableScanExec.scala:111)
        at 
scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
        at 
scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
        at scala.collection.immutable.List.foreach(List.scala:381)
        at 
scala.collection.TraversableLike$class.flatMap(TraversableLike.scala:241)
        at scala.collection.immutable.List.flatMap(List.scala:344)
        at 
org.apache.spark.sql.execution.columnar.InMemoryTableScanExec.<init>(InMemoryTableScanExec.scala:111)
        at 
org.apache.spark.sql.execution.SparkStrategies$InMemoryScans$$anonfun$3.apply(SparkStrategies.scala:307)
        at 
org.apache.spark.sql.execution.SparkStrategies$InMemoryScans$$anonfun$3.apply(SparkStrategies.scala:307)
        at 
org.apache.spark.sql.execution.SparkPlanner.pruneFilterProject(SparkPlanner.scala:99)
        at 
org.apache.spark.sql.execution.SparkStrategies$InMemoryScans$.apply(SparkStrategies.scala:303)
        at 
org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:62)
        at 
org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:62)
        at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)
        at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440)
        at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:439)
        at 
org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:92)
        at 
org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$2$$anonfun$apply$2.apply(QueryPlanner.scala:77)
        at 
org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$2$$anonfun$apply$2.apply(QueryPlanner.scala:74)
        at 
scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:157)
        at 
scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:157)
        at scala.collection.Iterator$class.foreach(Iterator.scala:893)
        at scala.collection.AbstractIterator.foreach(Iterator.scala:1336)
        at 
scala.collection.TraversableOnce$class.foldLeft(TraversableOnce.scala:157)
        at scala.collection.AbstractIterator.foldLeft(Iterator.scala:1336)
        at 
org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$2.apply(QueryPlanner.scala:74)
        at 
org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$2.apply(QueryPlanner.scala:66)
        at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)
        at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440)
        at 
org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:92)
        at 
org.apache.spark.sql.execution.QueryExecution.sparkPlan$lzycompute(QueryExecution.scala:84)
        at 
org.apache.spark.sql.execution.QueryExecution.sparkPlan(QueryExecution.scala:80)
        at 
org.apache.spark.sql.execution.QueryExecution.executedPlan$lzycompute(QueryExecution.scala:89)
        at 
org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:89)
        at org.apache.spark.sql.Dataset.withAction(Dataset.scala:2832)
        at org.apache.spark.sql.Dataset.head(Dataset.scala:2153)
        at org.apache.spark.sql.Dataset.take(Dataset.scala:2366)
        at org.apache.spark.sql.Dataset.showString(Dataset.scala:245)
        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:244)
        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)
{code}




--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org

Reply via email to