Commit graph

98 commits

Author SHA1 Message Date
HyukjinKwon 00d06cad56 [SPARK-31915][SQL][PYTHON] Resolve the grouping column properly per the case sensitivity in grouped and cogrouped pandas UDFs
### What changes were proposed in this pull request?

This is another approach to fix the issue. See the previous try https://github.com/apache/spark/pull/28745. It was too invasive so I took more conservative approach.

This PR proposes to resolve grouping attributes separately first so it can be properly referred when `FlatMapGroupsInPandas` and `FlatMapCoGroupsInPandas` are resolved without ambiguity.

Previously,

```python
from pyspark.sql.functions import *
df = spark.createDataFrame([[1, 1]], ["column", "Score"])
pandas_udf("column integer, Score float", PandasUDFType.GROUPED_MAP)
def my_pandas_udf(pdf):
    return pdf.assign(Score=0.5)

df.groupby('COLUMN').apply(my_pandas_udf).show()
```

was failed as below:

```
pyspark.sql.utils.AnalysisException: "Reference 'COLUMN' is ambiguous, could be: COLUMN, COLUMN.;"
```
because the unresolved `COLUMN` in `FlatMapGroupsInPandas` doesn't know which reference to take from the child projection.

After this fix, it resolves the child projection first with grouping keys and pass, to `FlatMapGroupsInPandas`, the attribute as a grouping key from the child projection that is positionally selected.

### Why are the changes needed?

To resolve grouping keys correctly.

### Does this PR introduce _any_ user-facing change?

Yes,

```python
from pyspark.sql.functions import *
df = spark.createDataFrame([[1, 1]], ["column", "Score"])
pandas_udf("column integer, Score float", PandasUDFType.GROUPED_MAP)
def my_pandas_udf(pdf):
    return pdf.assign(Score=0.5)

df.groupby('COLUMN').apply(my_pandas_udf).show()
```

```python
df1 = spark.createDataFrame([(1, 1)], ("column", "value"))
df2 = spark.createDataFrame([(1, 1)], ("column", "value"))

df1.groupby("COLUMN").cogroup(
    df2.groupby("COLUMN")
).applyInPandas(lambda r, l: r + l, df1.schema).show()
```

Before:

```
pyspark.sql.utils.AnalysisException: Reference 'COLUMN' is ambiguous, could be: COLUMN, COLUMN.;
```

```
pyspark.sql.utils.AnalysisException: cannot resolve '`COLUMN`' given input columns: [COLUMN, COLUMN, value, value];;
'FlatMapCoGroupsInPandas ['COLUMN], ['COLUMN], <lambda>(column#9L, value#10L, column#13L, value#14L), [column#22L, value#23L]
:- Project [COLUMN#9L, column#9L, value#10L]
:  +- LogicalRDD [column#9L, value#10L], false
+- Project [COLUMN#13L, column#13L, value#14L]
   +- LogicalRDD [column#13L, value#14L], false
```

After:

```
+------+-----+
|column|Score|
+------+-----+
|     1|  0.5|
+------+-----+
```

```
+------+-----+
|column|value|
+------+-----+
|     2|    2|
+------+-----+
```

### How was this patch tested?

Unittests were added and manually tested.

Closes #28777 from HyukjinKwon/SPARK-31915-another.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2020-06-10 15:54:07 -07:00
Takuya UESHIN 032d17933b [SPARK-31945][SQL][PYSPARK] Enable cache for the same Python function
### What changes were proposed in this pull request?

This PR proposes to make `PythonFunction` holds `Seq[Byte]` instead of `Array[Byte]` to be able to compare if the byte array has the same values for the cache manager.

### Why are the changes needed?

Currently the cache manager doesn't use the cache for `udf` if the `udf` is created again even if the functions is the same.

```py
>>> func = lambda x: x

>>> df = spark.range(1)
>>> df.select(udf(func)("id")).cache()
```
```py
>>> df.select(udf(func)("id")).explain()
== Physical Plan ==
*(2) Project [pythonUDF0#14 AS <lambda>(id)#12]
+- BatchEvalPython [<lambda>(id#0L)], [pythonUDF0#14]
 +- *(1) Range (0, 1, step=1, splits=12)
```

This is because `PythonFunction` holds `Array[Byte]`, and `equals` method of array equals only when the both array is the same instance.

### Does this PR introduce _any_ user-facing change?

Yes, if the user reuse the Python function for the UDF, the cache manager will detect the same function and use the cache for it.

### How was this patch tested?

I added a test case and manually.

```py
>>> df.select(udf(func)("id")).explain()
== Physical Plan ==
InMemoryTableScan [<lambda>(id)#12]
   +- InMemoryRelation [<lambda>(id)#12], StorageLevel(disk, memory, deserialized, 1 replicas)
         +- *(2) Project [pythonUDF0#5 AS <lambda>(id)#3]
            +- BatchEvalPython [<lambda>(id#0L)], [pythonUDF0#5]
               +- *(1) Range (0, 1, step=1, splits=12)
```

Closes #28774 from ueshin/issues/SPARK-31945/udf_cache.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-10 16:38:59 +09:00
HyukjinKwon e69466056f [SPARK-31849][PYTHON][SQL] Make PySpark SQL exceptions more Pythonic
### What changes were proposed in this pull request?

This PR proposes to make PySpark exception more Pythonic by hiding JVM stacktrace by default. It can be enabled by turning on `spark.sql.pyspark.jvmStacktrace.enabled` configuration.

```
Traceback (most recent call last):
  ...
pyspark.sql.utils.PythonException:
  An exception was thrown from Python worker in the executor. The below is the Python worker stacktrace.
Traceback (most recent call last):
  ...
```

If this `spark.sql.pyspark.jvmStacktrace.enabled` is enabled, it appends:

```
JVM stacktrace:
org.apache.spark.Exception: ...
  ...
```

For example, the codes below:

```python
from pyspark.sql.functions import udf
udf
def divide_by_zero(v):
    raise v / 0

spark.range(1).select(divide_by_zero("id")).show()
```

will show an error messages that looks like Python exception thrown from the local.

<details>
<summary>Python exception message when <code>spark.sql.pyspark.jvmStacktrace.enabled</code> is off (default)</summary>

```
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/.../spark/python/pyspark/sql/dataframe.py", line 427, in show
    print(self._jdf.showString(n, 20, vertical))
  File "/.../spark/python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 1305, in __call__
  File "/.../spark/python/pyspark/sql/utils.py", line 131, in deco
    raise_from(converted)
  File "<string>", line 3, in raise_from
pyspark.sql.utils.PythonException:
  An exception was thrown from Python worker in the executor. The below is the Python worker stacktrace.
Traceback (most recent call last):
  File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 605, in main
    process()
  File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 597, in process
    serializer.dump_stream(out_iter, outfile)
  File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 223, in dump_stream
    self.serializer.dump_stream(self._batched(iterator), stream)
  File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 141, in dump_stream
    for obj in iterator:
  File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 212, in _batched
    for item in iterator:
  File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 450, in mapper
    result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
  File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 450, in <genexpr>
    result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
  File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 90, in <lambda>
    return lambda *a: f(*a)
  File "/.../spark/python/lib/pyspark.zip/pyspark/util.py", line 107, in wrapper
    return f(*args, **kwargs)
  File "<stdin>", line 3, in divide_by_zero
ZeroDivisionError: division by zero
```

</details>

<details>
<summary>Python exception message when <code>spark.sql.pyspark.jvmStacktrace.enabled</code> is on</summary>

```
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/.../spark/python/pyspark/sql/dataframe.py", line 427, in show
    print(self._jdf.showString(n, 20, vertical))
  File "/.../spark/python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 1305, in __call__
  File "/.../spark/python/pyspark/sql/utils.py", line 137, in deco
    raise_from(converted)
  File "<string>", line 3, in raise_from
pyspark.sql.utils.PythonException:
  An exception was thrown from Python worker in the executor. The below is the Python worker stacktrace.
Traceback (most recent call last):
  File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 605, in main
    process()
  File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 597, in process
    serializer.dump_stream(out_iter, outfile)
  File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 223, in dump_stream
    self.serializer.dump_stream(self._batched(iterator), stream)
  File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 141, in dump_stream
    for obj in iterator:
  File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 212, in _batched
    for item in iterator:
  File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 450, in mapper
    result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
  File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 450, in <genexpr>
    result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
  File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 90, in <lambda>
    return lambda *a: f(*a)
  File "/.../spark/python/lib/pyspark.zip/pyspark/util.py", line 107, in wrapper
    return f(*args, **kwargs)
  File "<stdin>", line 3, in divide_by_zero
ZeroDivisionError: division by zero

JVM stacktrace:
org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1.0 failed 4 times, most recent failure: Lost task 0.3 in stage 1.0 (TID 4, 192.168.35.193, executor 0): org.apache.spark.api.python.PythonException: Traceback (most recent call last):
  File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 605, in main
    process()
  File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 597, in process
    serializer.dump_stream(out_iter, outfile)
  File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 223, in dump_stream
    self.serializer.dump_stream(self._batched(iterator), stream)
  File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 141, in dump_stream
    for obj in iterator:
  File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 212, in _batched
    for item in iterator:
  File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 450, in mapper
    result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
  File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 450, in <genexpr>
    result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
  File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 90, in <lambda>
    return lambda *a: f(*a)
  File "/.../spark/python/lib/pyspark.zip/pyspark/util.py", line 107, in wrapper
    return f(*args, **kwargs)
  File "<stdin>", line 3, in divide_by_zero
ZeroDivisionError: division by zero

	at org.apache.spark.api.python.BasePythonRunner$ReaderIterator.handlePythonException(PythonRunner.scala:516)
	at org.apache.spark.sql.execution.python.PythonUDFRunner$$anon$2.read(PythonUDFRunner.scala:81)
	at org.apache.spark.sql.execution.python.PythonUDFRunner$$anon$2.read(PythonUDFRunner.scala:64)
	at org.apache.spark.api.python.BasePythonRunner$ReaderIterator.hasNext(PythonRunner.scala:469)
	at org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:37)
	at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:489)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage2.processNext(Unknown Source)
	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:753)
	at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:340)
	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:898)
	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:898)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
	at org.apache.spark.scheduler.Task.run(Task.scala:127)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:469)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:472)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)

Driver stacktrace:
	at org.apache.spark.scheduler.DAGScheduler.failJobAndIndependentStages(DAGScheduler.scala:2117)
	at org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2(DAGScheduler.scala:2066)
	at org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2$adapted(DAGScheduler.scala:2065)
	at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
	at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
	at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
	at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:2065)
	at org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1(DAGScheduler.scala:1021)
	at org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1$adapted(DAGScheduler.scala:1021)
	at scala.Option.foreach(Option.scala:407)
	at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:1021)
	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:2297)
	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2246)
	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2235)
	at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:49)
	at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:823)
	at org.apache.spark.SparkContext.runJob(SparkContext.scala:2108)
	at org.apache.spark.SparkContext.runJob(SparkContext.scala:2129)
	at org.apache.spark.SparkContext.runJob(SparkContext.scala:2148)
	at org.apache.spark.sql.execution.SparkPlan.executeTake(SparkPlan.scala:467)
	at org.apache.spark.sql.execution.SparkPlan.executeTake(SparkPlan.scala:420)
	at org.apache.spark.sql.execution.CollectLimitExec.executeCollect(limit.scala:47)
	at org.apache.spark.sql.Dataset.collectFromPlan(Dataset.scala:3653)
	at org.apache.spark.sql.Dataset.$anonfun$head$1(Dataset.scala:2695)
	at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3644)
	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:103)
	at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163)
	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:763)
	at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
	at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3642)
	at org.apache.spark.sql.Dataset.head(Dataset.scala:2695)
	at org.apache.spark.sql.Dataset.take(Dataset.scala:2902)
	at org.apache.spark.sql.Dataset.getRows(Dataset.scala:300)
	at org.apache.spark.sql.Dataset.showString(Dataset.scala:337)
	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:282)
	at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
	at py4j.commands.CallCommand.execute(CallCommand.java:79)
	at py4j.GatewayConnection.run(GatewayConnection.java:238)
	at java.lang.Thread.run(Thread.java:748)
Caused by: org.apache.spark.api.python.PythonException: Traceback (most recent call last):
  File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 605, in main
    process()
  File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 597, in process
    serializer.dump_stream(out_iter, outfile)
  File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 223, in dump_stream
    self.serializer.dump_stream(self._batched(iterator), stream)
  File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 141, in dump_stream
    for obj in iterator:
  File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 212, in _batched
    for item in iterator:
  File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 450, in mapper
    result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
  File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 450, in <genexpr>
    result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
  File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 90, in <lambda>
    return lambda *a: f(*a)
  File "/.../spark/python/lib/pyspark.zip/pyspark/util.py", line 107, in wrapper
    return f(*args, **kwargs)
  File "<stdin>", line 3, in divide_by_zero
ZeroDivisionError: division by zero

	at org.apache.spark.api.python.BasePythonRunner$ReaderIterator.handlePythonException(PythonRunner.scala:516)
	at org.apache.spark.sql.execution.python.PythonUDFRunner$$anon$2.read(PythonUDFRunner.scala:81)
	at org.apache.spark.sql.execution.python.PythonUDFRunner$$anon$2.read(PythonUDFRunner.scala:64)
	at org.apache.spark.api.python.BasePythonRunner$ReaderIterator.hasNext(PythonRunner.scala:469)
	at org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:37)
	at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:489)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage2.processNext(Unknown Source)
	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:753)
	at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:340)
	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:898)
	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:898)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
	at org.apache.spark.scheduler.Task.run(Task.scala:127)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:469)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:472)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	... 1 more
```

</details>

<details>
<summary>Python exception message without this change</summary>

```
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/.../spark/python/pyspark/sql/dataframe.py", line 427, in show
    print(self._jdf.showString(n, 20, vertical))
  File "/.../spark/python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 1305, in __call__
  File "/.../spark/python/pyspark/sql/utils.py", line 98, in deco
    return f(*a, **kw)
  File "/.../spark/python/lib/py4j-0.10.9-src.zip/py4j/protocol.py", line 328, in get_return_value
py4j.protocol.Py4JJavaError: An error occurred while calling o160.showString.
: org.apache.spark.SparkException: Job aborted due to stage failure: Task 10 in stage 5.0 failed 4 times, most recent failure: Lost task 10.3 in stage 5.0 (TID 37, 192.168.35.193, executor 3): org.apache.spark.api.python.PythonException: Traceback (most recent call last):
  File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 605, in main
    process()
  File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 597, in process
    serializer.dump_stream(out_iter, outfile)
  File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 223, in dump_stream
    self.serializer.dump_stream(self._batched(iterator), stream)
  File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 141, in dump_stream
    for obj in iterator:
  File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 212, in _batched
    for item in iterator:
  File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 450, in mapper
    result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
  File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 450, in <genexpr>
    result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
  File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 90, in <lambda>
    return lambda *a: f(*a)
  File "/.../spark/python/lib/pyspark.zip/pyspark/util.py", line 107, in wrapper
    return f(*args, **kwargs)
  File "<stdin>", line 3, in divide_by_zero
ZeroDivisionError: division by zero

	at org.apache.spark.api.python.BasePythonRunner$ReaderIterator.handlePythonException(PythonRunner.scala:516)
	at org.apache.spark.sql.execution.python.PythonUDFRunner$$anon$2.read(PythonUDFRunner.scala:81)
	at org.apache.spark.sql.execution.python.PythonUDFRunner$$anon$2.read(PythonUDFRunner.scala:64)
	at org.apache.spark.api.python.BasePythonRunner$ReaderIterator.hasNext(PythonRunner.scala:469)
	at org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:37)
	at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:489)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage2.processNext(Unknown Source)
	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:753)
	at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:340)
	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:898)
	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:898)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
	at org.apache.spark.scheduler.Task.run(Task.scala:127)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:469)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:472)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)

Driver stacktrace:
	at org.apache.spark.scheduler.DAGScheduler.failJobAndIndependentStages(DAGScheduler.scala:2117)
	at org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2(DAGScheduler.scala:2066)
	at org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2$adapted(DAGScheduler.scala:2065)
	at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
	at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
	at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
	at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:2065)
	at org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1(DAGScheduler.scala:1021)
	at org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1$adapted(DAGScheduler.scala:1021)
	at scala.Option.foreach(Option.scala:407)
	at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:1021)
	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:2297)
	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2246)
	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2235)
	at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:49)
	at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:823)
	at org.apache.spark.SparkContext.runJob(SparkContext.scala:2108)
	at org.apache.spark.SparkContext.runJob(SparkContext.scala:2129)
	at org.apache.spark.SparkContext.runJob(SparkContext.scala:2148)
	at org.apache.spark.sql.execution.SparkPlan.executeTake(SparkPlan.scala:467)
	at org.apache.spark.sql.execution.SparkPlan.executeTake(SparkPlan.scala:420)
	at org.apache.spark.sql.execution.CollectLimitExec.executeCollect(limit.scala:47)
	at org.apache.spark.sql.Dataset.collectFromPlan(Dataset.scala:3653)
	at org.apache.spark.sql.Dataset.$anonfun$head$1(Dataset.scala:2695)
	at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3644)
	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:103)
	at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163)
	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:763)
	at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
	at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3642)
	at org.apache.spark.sql.Dataset.head(Dataset.scala:2695)
	at org.apache.spark.sql.Dataset.take(Dataset.scala:2902)
	at org.apache.spark.sql.Dataset.getRows(Dataset.scala:300)
	at org.apache.spark.sql.Dataset.showString(Dataset.scala:337)
	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:282)
	at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
	at py4j.commands.CallCommand.execute(CallCommand.java:79)
	at py4j.GatewayConnection.run(GatewayConnection.java:238)
	at java.lang.Thread.run(Thread.java:748)
Caused by: org.apache.spark.api.python.PythonException: Traceback (most recent call last):
  File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 605, in main
    process()
  File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 597, in process
    serializer.dump_stream(out_iter, outfile)
  File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 223, in dump_stream
    self.serializer.dump_stream(self._batched(iterator), stream)
  File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 141, in dump_stream
    for obj in iterator:
  File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 212, in _batched
    for item in iterator:
  File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 450, in mapper
    result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
  File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 450, in <genexpr>
    result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
  File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 90, in <lambda>
    return lambda *a: f(*a)
  File "/.../spark/python/lib/pyspark.zip/pyspark/util.py", line 107, in wrapper
    return f(*args, **kwargs)
  File "<stdin>", line 3, in divide_by_zero
ZeroDivisionError: division by zero

	at org.apache.spark.api.python.BasePythonRunner$ReaderIterator.handlePythonException(PythonRunner.scala:516)
	at org.apache.spark.sql.execution.python.PythonUDFRunner$$anon$2.read(PythonUDFRunner.scala:81)
	at org.apache.spark.sql.execution.python.PythonUDFRunner$$anon$2.read(PythonUDFRunner.scala:64)
	at org.apache.spark.api.python.BasePythonRunner$ReaderIterator.hasNext(PythonRunner.scala:469)
	at org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:37)
	at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:489)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage2.processNext(Unknown Source)
	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:753)
	at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:340)
	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:898)
	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:898)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
	at org.apache.spark.scheduler.Task.run(Task.scala:127)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:469)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:472)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	... 1 more
```

</details>

<br/>

Another example with Python 3.7:

```python
sql("a")
```

<details>
<summary>Python exception message when <code>spark.sql.pyspark.jvmStacktrace.enabled</code> is off (default)</summary>

```
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/.../spark/python/pyspark/sql/session.py", line 646, in sql
    return DataFrame(self._jsparkSession.sql(sqlQuery), self._wrapped)
  File "/.../spark/python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 1305, in __call__
  File "/.../spark/python/pyspark/sql/utils.py", line 131, in deco
    raise_from(converted)
  File "<string>", line 3, in raise_from
pyspark.sql.utils.ParseException:
mismatched input 'a' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0)

== SQL ==
a
^^^
```

</details>

<details>
<summary>Python exception message when <code>spark.sql.pyspark.jvmStacktrace.enabled</code> is on</summary>

```
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/.../spark/python/pyspark/sql/session.py", line 646, in sql
    return DataFrame(self._jsparkSession.sql(sqlQuery), self._wrapped)
  File "/.../spark/python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 1305, in __call__
  File "/.../spark/python/pyspark/sql/utils.py", line 131, in deco
    raise_from(converted)
  File "<string>", line 3, in raise_from
pyspark.sql.utils.ParseException:
mismatched input 'a' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0)

== SQL ==
a
^^^

JVM stacktrace:
org.apache.spark.sql.catalyst.parser.ParseException:
mismatched input 'a' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0)

== SQL ==
a
^^^

	at org.apache.spark.sql.catalyst.parser.ParseException.withCommand(ParseDriver.scala:266)
	at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parse(ParseDriver.scala:133)
	at org.apache.spark.sql.execution.SparkSqlParser.parse(SparkSqlParser.scala:49)
	at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parsePlan(ParseDriver.scala:81)
	at org.apache.spark.sql.SparkSession.$anonfun$sql$2(SparkSession.scala:604)
	at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
	at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:604)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:763)
	at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:601)
	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:282)
	at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
	at py4j.commands.CallCommand.execute(CallCommand.java:79)
	at py4j.GatewayConnection.run(GatewayConnection.java:238)
	at java.lang.Thread.run(Thread.java:748)
```

</details>

<details>
<summary>Python exception message without this change</summary>

```
Traceback (most recent call last):
  File "/.../spark/python/pyspark/sql/utils.py", line 98, in deco
    return f(*a, **kw)
  File "/.../spark/python/lib/py4j-0.10.9-src.zip/py4j/protocol.py", line 328, in get_return_value
py4j.protocol.Py4JJavaError: An error occurred while calling o26.sql.
: org.apache.spark.sql.catalyst.parser.ParseException:
mismatched input 'a' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0)

== SQL ==
a
^^^

	at org.apache.spark.sql.catalyst.parser.ParseException.withCommand(ParseDriver.scala:266)
	at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parse(ParseDriver.scala:133)
	at org.apache.spark.sql.execution.SparkSqlParser.parse(SparkSqlParser.scala:49)
	at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parsePlan(ParseDriver.scala:81)
	at org.apache.spark.sql.SparkSession.$anonfun$sql$2(SparkSession.scala:604)
	at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
	at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:604)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:763)
	at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:601)
	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:282)
	at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
	at py4j.commands.CallCommand.execute(CallCommand.java:79)
	at py4j.GatewayConnection.run(GatewayConnection.java:238)
	at java.lang.Thread.run(Thread.java:748)

During handling of the above exception, another exception occurred:

Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/.../spark/python/pyspark/sql/session.py", line 646, in sql
    return DataFrame(self._jsparkSession.sql(sqlQuery), self._wrapped)
  File "/.../spark/python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 1305, in __call__
  File "/.../spark/python/pyspark/sql/utils.py", line 102, in deco
    raise converted
pyspark.sql.utils.ParseException:
mismatched input 'a' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0)

== SQL ==
a
^^^
```

</details>

### Why are the changes needed?

Currently, PySpark exceptions are very unfriendly to Python users with causing a bunch of JVM stacktrace. See "Python exception message without this change" above.

### Does this PR introduce _any_ user-facing change?

Yes, it will change the exception message. See the examples above.

### How was this patch tested?

Manually tested by

```bash
./bin/pyspark --conf spark.sql.pyspark.jvmStacktrace.enabled=true
```

and running the examples above.

Closes #28661 from HyukjinKwon/python-debug.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-01 09:45:21 +09:00
Bryan Cutler 8bbb666622 [SPARK-25351][PYTHON][TEST][FOLLOWUP] Fix test assertions to be consistent
### What changes were proposed in this pull request?
Followup to make assertions from recent test consistent with the rest of the module

### Why are the changes needed?

Better to use assertions from `unittest` and be consistent

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Existing tests

Closes #28659 from BryanCutler/arrow-category-test-fix-SPARK-25351.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-28 10:27:15 +09:00
iRakson 2f92ea0df4 [SPARK-31763][PYSPARK] Add inputFiles method in PySpark DataFrame Class
### What changes were proposed in this pull request?
Adds `inputFiles()` method to PySpark `DataFrame`. Using this, PySpark users can list all files constituting a `DataFrame`.

**Before changes:**

```
>>> spark.read.load("examples/src/main/resources/people.json", format="json").inputFiles()
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/***/***/spark/python/pyspark/sql/dataframe.py", line 1388, in __getattr__
    "'%s' object has no attribute '%s'" % (self.__class__.__name__, name))
AttributeError: 'DataFrame' object has no attribute 'inputFiles'
```

**After changes:**

```
>>> spark.read.load("examples/src/main/resources/people.json", format="json").inputFiles()
[u'file:///***/***/spark/examples/src/main/resources/people.json']
```

### Why are the changes needed?
This method is already supported for spark with scala and java.

### Does this PR introduce _any_ user-facing change?
Yes, Now users can list all files of a DataFrame using `inputFiles()`

### How was this patch tested?
UT added.

Closes #28652 from iRakson/SPARK-31763.

Authored-by: iRakson <raksonrakesh@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-28 09:52:08 +09:00
Jalpan Randeri 339b0ecadb [SPARK-25351][SQL][PYTHON] Handle Pandas category type when converting from Python with Arrow
Handle Pandas category type while converting from python with Arrow enabled. The category column will be converted to whatever type the category elements are as is the case with Arrow disabled.

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
New unit tests were added for `createDataFrame` and scalar `pandas_udf`

Closes #26585 from jalpan-randeri/feature-pyarrow-dictionary-type.

Authored-by: Jalpan Randeri <randerij@amazon.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2020-05-27 17:27:29 -07:00
Kent Yao b31ae7bb0b [SPARK-31615][SQL] Pretty string output for sql method of RuntimeReplaceable expressions
### What changes were proposed in this pull request?

The RuntimeReplaceable ones are runtime replaceable, thus, their original parameters are not going to be resolved to PrettyAttribute and remain debug style string if we directly implement their `sql` methods with their parameters' `sql` methods.

This PR is raised with suggestions by maropu and cloud-fan https://github.com/apache/spark/pull/28402/files#r417656589. In this PR, we re-implement the `sql` methods of  the RuntimeReplaceable ones with toPettySQL

### Why are the changes needed?

Consistency of schema output between RuntimeReplaceable expressions and normal ones.

For example, `date_format` vs `to_timestamp`, before this PR, they output differently

#### Before
```sql
select date_format(timestamp '2019-10-06', 'yyyy-MM-dd uuuu')
struct<date_format(TIMESTAMP '2019-10-06 00:00:00', yyyy-MM-dd uuuu):string>

select to_timestamp("2019-10-06S10:11:12.12345", "yyyy-MM-dd'S'HH:mm:ss.SSSSSS")
struct<to_timestamp('2019-10-06S10:11:12.12345', 'yyyy-MM-dd\'S\'HH:mm:ss.SSSSSS'):timestamp>
```
#### After

```sql
select date_format(timestamp '2019-10-06', 'yyyy-MM-dd uuuu')
struct<date_format(TIMESTAMP '2019-10-06 00:00:00', yyyy-MM-dd uuuu):string>

select to_timestamp("2019-10-06T10:11:12'12", "yyyy-MM-dd'T'HH:mm:ss''SSSS")

struct<to_timestamp(2019-10-06T10:11:12'12, yyyy-MM-dd'T'HH:mm:ss''SSSS):timestamp>

````

### Does this PR introduce _any_ user-facing change?

Yes, the schema output style changed for the runtime replaceable expressions as shown in the above example

### How was this patch tested?
regenerate all related tests

Closes #28420 from yaooqinn/SPARK-31615.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-05-07 14:40:26 +09:00
HyukjinKwon 5dd581c88a [SPARK-29664][PYTHON][SQL][FOLLOW-UP] Add deprecation warnings for getItem instead
### What changes were proposed in this pull request?

This PR proposes to use a different approach instead of breaking it per Micheal's rubric added at https://spark.apache.org/versioning-policy.html. It deprecates the behaviour for now. It will be gradually removed in the future releases.

After this change,

```python
import warnings
warnings.simplefilter("always")
from pyspark.sql.functions import *
df = spark.range(2)
map_col = create_map(lit(0), lit(100), lit(1), lit(200))
df.withColumn("mapped", map_col.getItem(col('id'))).show()
```

```
/.../python/pyspark/sql/column.py:311: DeprecationWarning: A column as 'key' in getItem is
deprecated as of Spark 3.0, and will not be supported in the future release. Use `column[key]`
or `column.key` syntax instead.
  DeprecationWarning)
...
```

```python
import warnings
warnings.simplefilter("always")
from pyspark.sql.functions import *
df = spark.range(2)
struct_col = struct(lit(0), lit(100), lit(1), lit(200))
df.withColumn("struct", struct_col.getField(lit("col1"))).show()
```

```
/.../spark/python/pyspark/sql/column.py:336: DeprecationWarning: A column as 'name'
in getField is deprecated as of Spark 3.0, and will not be supported in the future release. Use
`column[name]` or `column.name` syntax instead.
  DeprecationWarning)
```

### Why are the changes needed?

To prevent the radical behaviour change after the amended versioning policy.

### Does this PR introduce any user-facing change?

Yes, it will show the deprecated warning message.

### How was this patch tested?

Manually tested.

Closes #28327 from HyukjinKwon/SPARK-29664.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-04-27 14:49:22 +09:00
Takuya UESHIN 87be3641eb [SPARK-31441] Support duplicated column names for toPandas with arrow execution
### What changes were proposed in this pull request?

This PR is adding support duplicated column names for `toPandas` with Arrow execution.

### Why are the changes needed?

When we execute `toPandas()` with Arrow execution, it fails if the column names have duplicates.

```py
>>> spark.sql("select 1 v, 1 v").toPandas()
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/path/to/lib/python3.7/site-packages/pyspark/sql/dataframe.py", line 2132, in toPandas
    pdf = table.to_pandas()
  File "pyarrow/array.pxi", line 441, in pyarrow.lib._PandasConvertible.to_pandas
  File "pyarrow/table.pxi", line 1367, in pyarrow.lib.Table._to_pandas
  File "/path/to/lib/python3.7/site-packages/pyarrow/pandas_compat.py", line 653, in table_to_blockmanager
    columns = _deserialize_column_index(table, all_columns, column_indexes)
  File "/path/to/lib/python3.7/site-packages/pyarrow/pandas_compat.py", line 704, in _deserialize_column_index
    columns = _flatten_single_level_multiindex(columns)
  File "/path/to/lib/python3.7/site-packages/pyarrow/pandas_compat.py", line 937, in _flatten_single_level_multiindex
    raise ValueError('Found non-unique column index')
ValueError: Found non-unique column index
```

### Does this PR introduce any user-facing change?

Yes, previously we will face an error above, but after this PR, we will see the result:

```py
>>> spark.sql("select 1 v, 1 v").toPandas()
   v  v
0  1  1
```

### How was this patch tested?

Added and modified related tests.

Closes #28210 from ueshin/issues/SPARK-31441/to_pandas.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-04-14 14:08:56 +09:00
HyukjinKwon 4fafdcd63b [SPARK-26412][PYTHON][FOLLOW-UP] Improve error messages in Scala iterator pandas UDF
### What changes were proposed in this pull request?

This PR proposes to improve the error message from Scalar iterator pandas UDF.

### Why are the changes needed?

To show the correct error messages.

### Does this PR introduce any user-facing change?

Yes, but only in unreleased branches.

```python
import pandas as pd
from pyspark.sql.functions import pandas_udf, PandasUDFType

pandas_udf('long', PandasUDFType.SCALAR_ITER)
def pandas_plus_one(iterator):
      for _ in iterator:
            yield pd.Series(1)

spark.range(10).repartition(1).select(pandas_plus_one("id")).show()
```
```python
import pandas as pd
from pyspark.sql.functions import pandas_udf, PandasUDFType

pandas_udf('long', PandasUDFType.SCALAR_ITER)
def pandas_plus_one(iterator):
      for _ in iterator:
            yield pd.Series(list(range(20)))

spark.range(10).repartition(1).select(pandas_plus_one("id")).show()
```

**Before:**

```
RuntimeError: The number of output rows of pandas iterator UDF should
be the same with input rows. The input rows number is 10 but the output
rows number is 1.
```
```
AssertionError: Pandas MAP_ITER UDF outputted more rows than input rows.
```

**After:**

```
RuntimeError: The length of output in Scalar iterator pandas UDF should be
the same with the input's; however, the length of output was 1 and the length
of input was 10.
```
```
AssertionError: Pandas SCALAR_ITER UDF outputted more rows than input rows.
```

### How was this patch tested?

Unittests were fixed accordingly.

Closes #28135 from HyukjinKwon/SPARK-26412-followup.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-04-09 13:14:41 +09:00
Liang-Chi Hsieh 1f02871489 [SPARK-30921][PYSPARK] Predicates on python udf should not be pushdown through Aggregate
### What changes were proposed in this pull request?

This patch proposed to skip predicates on PythonUDFs to be pushdown through Aggregate.

### Why are the changes needed?

The predicates on PythonUDFs cannot be pushdown through Aggregate. Pushed down predicates cannot be evaluate because PythonUDFs cannot be evaluated on Filter and cause error like:

```
Caused by: java.lang.UnsupportedOperationException: Cannot generate code for expression: mean(input[1, struct<bar:bigint>, true].bar)
        at org.apache.spark.sql.catalyst.expressions.Unevaluable.doGenCode(Expression.scala:304)
        at org.apache.spark.sql.catalyst.expressions.Unevaluable.doGenCode$(Expression.scala:303)
        at org.apache.spark.sql.catalyst.expressions.PythonUDF.doGenCode(PythonUDF.scala:52)
        at org.apache.spark.sql.catalyst.expressions.Expression.$anonfun$genCode$3(Expression.scala:146)
        at scala.Option.getOrElse(Option.scala:189)
        at org.apache.spark.sql.catalyst.expressions.Expression.genCode(Expression.scala:141)
        at org.apache.spark.sql.catalyst.expressions.CastBase.doGenCode(Cast.scala:821)
        at org.apache.spark.sql.catalyst.expressions.Expression.$anonfun$genCode$3(Expression.scala:146)
        at scala.Option.getOrElse(Option.scala:189)
```

### Does this PR introduce any user-facing change?

Yes. Previously the predicates on PythonUDFs will be pushdown through Aggregate can cause error. After this change, the query can work.

### How was this patch tested?

Unit test.

Closes #28089 from viirya/SPARK-30921.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-04-06 09:36:20 +09:00
HyukjinKwon 3165a95a04 [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas
### What changes were proposed in this pull request?

This PR proposes to make pandas function APIs (`groupby.(cogroup.)applyInPandas` and `mapInPandas`) to ignore Python type hints.

### Why are the changes needed?

Python type hints are optional. It shouldn't affect where pandas UDFs are not used.
This is also a future work for them to support other type hints. We shouldn't at least throw an exception at this moment.

### Does this PR introduce any user-facing change?

No, it's master-only change.

```python
import pandas as pd

def pandas_plus_one(pdf: pd.DataFrame) -> pd.DataFrame:
    return pdf + 1

spark.range(10).groupby('id').applyInPandas(pandas_plus_one, schema="id long").show()
```
```python
import pandas as pd

def pandas_plus_one(left: pd.DataFrame, right: pd.DataFrame) -> pd.DataFrame:
    return left + 1

spark.range(10).groupby('id').cogroup(spark.range(10).groupby("id")).applyInPandas(pandas_plus_one, schema="id long").show()
```

```python
from typing import Iterator
import pandas as pd

def pandas_plus_one(iter: Iterator[pd.DataFrame]) -> Iterator[pd.DataFrame]:
    return map(lambda v: v + 1, iter)

spark.range(10).mapInPandas(pandas_plus_one, schema="id long").show()
```

**Before:**

Exception

**After:**

```
+---+
| id|
+---+
|  1|
|  2|
|  3|
|  4|
|  5|
|  6|
|  7|
|  8|
|  9|
| 10|
+---+
```

### How was this patch tested?

Closes #28052 from HyukjinKwon/SPARK-31287.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-29 13:59:18 +09:00
Liang-Chi Hsieh 559d3e4051 [SPARK-31186][PYSPARK][SQL] toPandas should not fail on duplicate column names
### What changes were proposed in this pull request?

When `toPandas` API works on duplicate column names produced from operators like join, we see the error like:

```
ValueError: The truth value of a Series is ambiguous. Use a.empty, a.bool(), a.item(), a.any() or a.all().
```

This patch fixes the error in `toPandas` API.

### Why are the changes needed?

To make `toPandas` work on dataframe with duplicate column names.

### Does this PR introduce any user-facing change?

Yes. Previously calling `toPandas` API on a dataframe with duplicate column names will fail. After this patch, it will produce correct result.

### How was this patch tested?

Unit test.

Closes #28025 from viirya/SPARK-31186.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-27 12:10:30 +09:00
zero323 01f20394ac [SPARK-30569][SQL][PYSPARK][SPARKR] Add percentile_approx DSL functions
### What changes were proposed in this pull request?

- Adds following overloaded variants to Scala `o.a.s.sql.functions`:

  - `percentile_approx(e: Column, percentage: Array[Double], accuracy: Long): Column`
  - `percentile_approx(columnName: String, percentage: Array[Double], accuracy: Long): Column`
  - `percentile_approx(e: Column, percentage: Double, accuracy: Long): Column`
  - `percentile_approx(columnName: String, percentage: Double, accuracy: Long): Column`
  - `percentile_approx(e: Column, percentage: Seq[Double], accuracy: Long): Column` (primarily for
Python interop).
  - `percentile_approx(columnName: String, percentage: Seq[Double], accuracy: Long): Column`

- Adds `percentile_approx` to `pyspark.sql.functions`.

- Adds `percentile_approx` function to SparkR.

### Why are the changes needed?

Currently we support `percentile_approx` only in SQL expression. It is inconvenient and makes this function relatively unknown.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

New unit tests for SparkR an PySpark.

As for now there are no additional tests in Scala API ‒ `ApproximatePercentile` is well tested and Python (including docstrings) and R tests provide additional tests, so it seems unnecessary.

Closes #27278 from zero323/SPARK-30569.

Lead-authored-by: zero323 <mszymkiewicz@gmail.com>
Co-authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-17 10:44:21 +09:00
zero323 7de33f56e8 [SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?

This PR add Python API for invoking following higher functions:

- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`

to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types

- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`

Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:

1. Creating  required `UnresolvedNamedLambdaVariables`  exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.

### Why are the changes needed?

Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions

```python
df.selectExpr("transform(values, x -> x + 1)")
```

This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support.  Additionally DSL used, though  very simple, is not documented.

With changes propose here, above query could be rewritten as:

```python
df.select(transform("values", lambda x: x + 1))
```

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.

### Notes

If approved, the same approach can be used in SparkR.

Closes #27406 from zero323/SPARK-30681.

Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-28 12:59:39 +09:00
Alex Favaro 96c1a4401d [SPARK-30856][SQL][PYSPARK] Fix SQLContext.getOrCreate() when SparkContext is restarted
### What changes were proposed in this pull request?

As discussed on the Jira ticket, this change clears the SQLContext._instantiatedContext class attribute when the SparkSession is stopped. That way, the attribute will be reset with a new, usable SQLContext when a new SparkSession is started.

### Why are the changes needed?

When the underlying SQLContext is instantiated for a SparkSession, the instance is saved as a class attribute and returned from subsequent calls to SQLContext.getOrCreate(). If the SparkContext is stopped and a new one started, the SQLContext class attribute is never cleared so any code which calls SQLContext.getOrCreate() will get a SQLContext with a reference to the old, unusable SparkContext.

A similar issue was identified and fixed for SparkSession in [SPARK-19055](https://issues.apache.org/jira/browse/SPARK-19055), but the fix did not change SQLContext as well. I ran into this because mllib still [uses](https://github.com/apache/spark/blob/master/python/pyspark/mllib/common.py#L105) SQLContext.getOrCreate() under the hood.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

A new test was added. I verified that the test fails without the included change.

Closes #27610 from afavaro/restart-sqlcontext.

Authored-by: Alex Favaro <alex.favaro@affirm.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-20 12:21:24 +09:00
HyukjinKwon e065e22e5e [SPARK-30861][PYTHON][SQL] Deprecate constructor of SQLContext and getOrCreate in SQLContext at PySpark
### What changes were proposed in this pull request?

This PR proposes to deprecate the APIs at `SQLContext` removed in SPARK-25908. We should remove equivalent APIs; however, seems we missed to deprecate.

While I am here, I fix one more issue. After SPARK-25908, `sc._jvm.SQLContext.getOrCreate` dose not exist anymore. So,

```python
from pyspark.sql import SQLContext
from pyspark import SparkContext
sc = SparkContext.getOrCreate()
SQLContext.getOrCreate(sc).range(10).show()
```

throws an exception as below:

```
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/.../spark/python/pyspark/sql/context.py", line 110, in getOrCreate
    jsqlContext = sc._jvm.SQLContext.getOrCreate(sc._jsc.sc())
  File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1516, in __getattr__
py4j.protocol.Py4JError: org.apache.spark.sql.SQLContext.getOrCreate does not exist in the JVM
```

After this PR:

```
/.../spark/python/pyspark/sql/context.py:113: DeprecationWarning: Deprecated in 3.0.0. Use SparkSession.builder.getOrCreate() instead.
  DeprecationWarning)
+---+
| id|
+---+
|  0|
|  1|
|  2|
|  3|
|  4|
|  5|
|  6|
|  7|
|  8|
|  9|
+---+
```

In case of the constructor of `SQLContext`, after this PR:

```python
from pyspark.sql import SQLContext
sc = SparkContext.getOrCreate()
SQLContext(sc)
```

```
/.../spark/python/pyspark/sql/context.py:77: DeprecationWarning: Deprecated in 3.0.0. Use SparkSession.builder.getOrCreate() instead.
  DeprecationWarning)
```

### Why are the changes needed?

To promote to use SparkSession, and keep the API party consistent with Scala side.

### Does this PR introduce any user-facing change?

Yes, it will show deprecation warning to users.

### How was this patch tested?

Manually tested as described above. Unittests were also added.

Closes #27614 from HyukjinKwon/SPARK-30861.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-19 11:17:47 +09:00
yi.wu 68d7edf949 [SPARK-30812][SQL][CORE] Revise boolean config name to comply with new config naming policy
### What changes were proposed in this pull request?

Revise below config names to comply with [new config naming policy](http://apache-spark-developers-list.1001551.n3.nabble.com/DISCUSS-naming-policy-of-Spark-configs-td28875.html):

SQL:
* spark.sql.execution.subquery.reuse.enabled / [SPARK-27083](https://issues.apache.org/jira/browse/SPARK-27083)
* spark.sql.legacy.allowNegativeScaleOfDecimal.enabled / [SPARK-30252](https://issues.apache.org/jira/browse/SPARK-30252)
* spark.sql.adaptive.optimizeSkewedJoin.enabled / [SPARK-29544](https://issues.apache.org/jira/browse/SPARK-29544)
* spark.sql.legacy.property.nonReserved / [SPARK-30183](https://issues.apache.org/jira/browse/SPARK-30183)
* spark.sql.streaming.forceDeleteTempCheckpointLocation.enabled / [SPARK-26389](https://issues.apache.org/jira/browse/SPARK-26389)
* spark.sql.analyzer.failAmbiguousSelfJoin.enabled / [SPARK-28344](https://issues.apache.org/jira/browse/SPARK-28344)
* spark.sql.adaptive.shuffle.reducePostShufflePartitions.enabled / [SPARK-30074](https://issues.apache.org/jira/browse/SPARK-30074)
* spark.sql.execution.pandas.arrowSafeTypeConversion / [SPARK-25811](https://issues.apache.org/jira/browse/SPARK-25811)
* spark.sql.legacy.looseUpcast / [SPARK-24586](https://issues.apache.org/jira/browse/SPARK-24586)
* spark.sql.legacy.arrayExistsFollowsThreeValuedLogic / [SPARK-28052](https://issues.apache.org/jira/browse/SPARK-28052)
* spark.sql.sources.ignoreDataLocality.enabled / [SPARK-29189](https://issues.apache.org/jira/browse/SPARK-29189)
* spark.sql.adaptive.shuffle.fetchShuffleBlocksInBatch.enabled / [SPARK-9853](https://issues.apache.org/jira/browse/SPARK-9853)

CORE:
* spark.eventLog.erasureCoding.enabled / [SPARK-25855](https://issues.apache.org/jira/browse/SPARK-25855)
* spark.shuffle.readHostLocalDisk.enabled / [SPARK-30235](https://issues.apache.org/jira/browse/SPARK-30235)
* spark.scheduler.listenerbus.logSlowEvent.enabled / [SPARK-29001](https://issues.apache.org/jira/browse/SPARK-29001)
* spark.resources.coordinate.enable / [SPARK-27371](https://issues.apache.org/jira/browse/SPARK-27371)
* spark.eventLog.logStageExecutorMetrics.enabled / [SPARK-23429](https://issues.apache.org/jira/browse/SPARK-23429)

### Why are the changes needed?

To comply with the config naming policy.

### Does this PR introduce any user-facing change?

No. Configurations listed above are all newly added in Spark 3.0.

### How was this patch tested?

Pass Jenkins.

Closes #27563 from Ngone51/revise_boolean_conf_name.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-18 20:39:50 +08:00
Liang Zhang d8c0599e54 [SPARK-30791][SQL][PYTHON] Add 'sameSemantics' and 'sementicHash' methods in Dataset
### What changes were proposed in this pull request?
This PR added two DeveloperApis to the Dataset[T] class. Both methods are just exposing lower-level methods to the Dataset[T] class.

### Why are the changes needed?
They are useful for checking whether two dataframes are the same when implementing dataframe caching in python, and also get a unique ID. It's easier to use if we wrap the lower-level APIs.

### Does this PR introduce any user-facing change?
```
scala> val df1 = Seq((1,2),(4,5)).toDF("col1", "col2")
df1: org.apache.spark.sql.DataFrame = [col1: int, col2: int]

scala> val df2 = Seq((1,2),(4,5)).toDF("col1", "col2")
df2: org.apache.spark.sql.DataFrame = [col1: int, col2: int]

scala> val df3 = Seq((0,2),(4,5)).toDF("col1", "col2")
df3: org.apache.spark.sql.DataFrame = [col1: int, col2: int]

scala> val df4 = Seq((0,2),(4,5)).toDF("col0", "col2")
df4: org.apache.spark.sql.DataFrame = [col0: int, col2: int]

scala> df1.semanticHash
res0: Int = 594427822

scala> df2.semanticHash
res1: Int = 594427822

scala> df1.sameSemantics(df2)
res2: Boolean = true

scala> df1.sameSemantics(df3)
res3: Boolean = false

scala> df3.semanticHash
res4: Int = -1592702048

scala> df4.semanticHash
res5: Int = -1592702048

scala> df4.sameSemantics(df3)
res6: Boolean = true
```

### How was this patch tested?
Unit test in scala and doctest in python.

Note: comments are copied from the corresponding lower-level APIs.
Note: There are some issues to be fixed that would improve the hash collision rate: https://github.com/apache/spark/pull/27565#discussion_r379881028

Closes #27565 from liangz1/df-same-result.

Authored-by: Liang Zhang <liang.zhang@databricks.com>
Signed-off-by: WeichenXu <weichen.xu@databricks.com>
2020-02-18 09:22:26 +08:00
Bryan Cutler 07a9885f27 [SPARK-30777][PYTHON][TESTS] Fix test failures for Pandas >= 1.0.0
### What changes were proposed in this pull request?

Fix PySpark test failures for using Pandas >= 1.0.0.

### Why are the changes needed?

Pandas 1.0.0 has recently been released and has API changes that result in PySpark test failures, this PR fixes the broken tests.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

Manually tested with Pandas 1.0.1 and PyArrow 0.16.0

Closes #27529 from BryanCutler/pandas-fix-tests-1.0-SPARK-30777.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-11 10:03:01 +09:00
HyukjinKwon 692e3ddb4e [SPARK-27870][PYTHON][FOLLOW-UP] Rename spark.sql.pandas.udf.buffer.size to spark.sql.execution.pandas.udf.buffer.size
### What changes were proposed in this pull request?

This PR renames `spark.sql.pandas.udf.buffer.size` to `spark.sql.execution.pandas.udf.buffer.size` to be more consistent with other pandas configuration prefixes, given:
-  `spark.sql.execution.pandas.arrowSafeTypeConversion`
- `spark.sql.execution.pandas.respectSessionTimeZone`
- `spark.sql.legacy.execution.pandas.groupedMap.assignColumnsByName`
- other configurations like `spark.sql.execution.arrow.*`.

### Why are the changes needed?

To make configuration names consistent.

### Does this PR introduce any user-facing change?

No because this configuration was not released yet.

### How was this patch tested?

Existing tests should cover.

Closes #27450 from HyukjinKwon/SPARK-27870-followup.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-05 11:38:33 +09:00
zero323 2330a5682d [SPARK-30607][SQL][PYSPARK][SPARKR] Add overlay wrappers for SparkR and PySpark
### What changes were proposed in this pull request?

This PR adds:

- `pyspark.sql.functions.overlay` function to PySpark
- `overlay` function to SparkR

### Why are the changes needed?

Feature parity. At the moment R and Python users can access this function only using SQL or `expr` / `selectExpr`.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

New unit tests.

Closes #27325 from zero323/SPARK-30607.

Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-23 16:16:47 +09:00
HyukjinKwon ab0890bdb1 [SPARK-28264][PYTHON][SQL] Support type hints in pandas UDF and rename/move inconsistent pandas UDF types
### What changes were proposed in this pull request?

This PR proposes to redesign pandas UDFs as described in [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing).

```python
from pyspark.sql.functions import pandas_udf
import pandas as pd

pandas_udf("long")
def plug_one(s: pd.Series) -> pd.Series:
    return s + 1

spark.range(10).select(plug_one("id")).show()
```

```
+------------+
|plug_one(id)|
+------------+
|           1|
|           2|
|           3|
|           4|
|           5|
|           6|
|           7|
|           8|
|           9|
|          10|
+------------+
```

Note that, this PR address one of the future improvements described [here](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit#heading=h.h3ncjpk6ujqu), "A couple of less-intuitive pandas UDF types" (by zero323) together.

In short,

- Adds new way with type hints as an alternative and experimental way.
    ```python
    pandas_udf(schema='...')
    def func(c1: Series, c2: Series) -> DataFrame:
        pass
    ```

- Replace and/or add an alias for three types below from UDF, and make them as separate standalone APIs. So, `pandas_udf` is now consistent with regular `udf`s and other expressions.

    `df.mapInPandas(udf)`  -replace-> `df.mapInPandas(f, schema)`
    `df.groupby.apply(udf)`  -alias-> `df.groupby.applyInPandas(f, schema)`
    `df.groupby.cogroup.apply(udf)`  -replace-> `df.groupby.cogroup.applyInPandas(f, schema)`

    *`df.groupby.apply` was added from 2.3 while the other were added in the master only.

- No deprecation for the existing ways for now.
    ```python
    pandas_udf(schema='...', functionType=PandasUDFType.SCALAR)
    def func(c1, c2):
        pass
    ```
If users are happy with this, I plan to deprecate the existing way and declare using type hints is not experimental anymore.

One design goal in this PR was that, avoid touching the internal (since we didn't deprecate the old ways for now), but supports type hints with a minimised changes only at the interface.

- Once we deprecate or remove the old ways, I think it requires another refactoring for the internal in the future. At the very least, we should rename internal pandas evaluation types.
- If users find this experimental type hints isn't quite helpful, we should simply revert the changes at the interface level.

### Why are the changes needed?

In order to address old design issues. Please see [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing).

### Does this PR introduce any user-facing change?

For behaviour changes, No.

It adds new ways to use pandas UDFs by using type hints. See below.

**SCALAR**:

```python
pandas_udf(schema='...')
def func(c1: Series, c2: DataFrame) -> Series:
    pass  # DataFrame represents a struct column
```

**SCALAR_ITER**:

```python
pandas_udf(schema='...')
def func(iter: Iterator[Tuple[Series, DataFrame, ...]]) -> Iterator[Series]:
    pass  # Same as SCALAR but wrapped by Iterator
```

**GROUPED_AGG**:

```python
pandas_udf(schema='...')
def func(c1: Series, c2: DataFrame) -> int:
    pass  # DataFrame represents a struct column
```

**GROUPED_MAP**:

This was added in Spark 2.3 as of SPARK-20396. As described above, it keeps the existing behaviour. Additionally, we now have a new alias `groupby.applyInPandas` for `groupby.apply`. See the example below:

```python
def func(pdf):
    return pdf

df.groupby("...").applyInPandas(func, schema=df.schema)
```

**MAP_ITER**: this is not a pandas UDF anymore

This was added in Spark 3.0 as of SPARK-28198; and this PR replaces the usages. See the example below:

```python
def func(iter):
    for df in iter:
        yield df

df.mapInPandas(func, df.schema)
```

**COGROUPED_MAP**: this is not a pandas UDF anymore

This was added in Spark 3.0 as of SPARK-27463; and this PR replaces the usages. See the example below:

```python
def asof_join(left, right):
    return pd.merge_asof(left, right, on="...", by="...")

 df1.groupby("...").cogroup(df2.groupby("...")).applyInPandas(asof_join, schema="...")
```

### How was this patch tested?

Unittests added and tested against Python 2.7, 3.6 and 3.7.

Closes #27165 from HyukjinKwon/revisit-pandas.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-22 15:32:58 +09:00
yi.wu ff39c9271c [SPARK-30252][SQL] Disallow negative scale of Decimal
### What changes were proposed in this pull request?

This PR propose to disallow negative `scale` of `Decimal` in Spark. And this PR brings two behavior changes:

1) for literals like `1.23E4BD` or `1.23E4`(with `spark.sql.legacy.exponentLiteralAsDecimal.enabled`=true, see [SPARK-29956](https://issues.apache.org/jira/browse/SPARK-29956)), we set its `(precision, scale)` to (5, 0) rather than (3, -2);
2) add negative `scale` check inside the decimal method if it exposes to set `scale` explicitly. If check fails, `AnalysisException` throws.

And user could still use `spark.sql.legacy.allowNegativeScaleOfDecimal.enabled` to restore the previous behavior.

### Why are the changes needed?

According to SQL standard,
> 4.4.2 Characteristics of numbers
An exact numeric type has a precision P and a scale S. P is a positive integer that determines the number of significant digits in a particular radix R, where R is either 2 or 10. S is a non-negative integer.

scale of Decimal should always be non-negative. And other mainstream databases, like Presto, PostgreSQL, also don't allow negative scale.

Presto:
```
presto:default> create table t (i decimal(2, -1));
Query 20191213_081238_00017_i448h failed: line 1:30: mismatched input '-'. Expecting: <integer>, <type>
create table t (i decimal(2, -1))
```

PostgrelSQL:
```
postgres=# create table t(i decimal(2, -1));
ERROR:  NUMERIC scale -1 must be between 0 and precision 2
LINE 1: create table t(i decimal(2, -1));
                         ^
```

And, actually, Spark itself already doesn't allow to create table with negative decimal types using SQL:
```
scala> spark.sql("create table t(i decimal(2, -1))");
org.apache.spark.sql.catalyst.parser.ParseException:
no viable alternative at input 'create table t(i decimal(2, -'(line 1, pos 28)

== SQL ==
create table t(i decimal(2, -1))
----------------------------^^^

  at org.apache.spark.sql.catalyst.parser.ParseException.withCommand(ParseDriver.scala:263)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parse(ParseDriver.scala:130)
  at org.apache.spark.sql.execution.SparkSqlParser.parse(SparkSqlParser.scala:48)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parsePlan(ParseDriver.scala:76)
  at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:605)
  at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
  at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:605)
  ... 35 elided
```

However, it is still possible to create such table or `DatFrame` using Spark SQL programming API:
```
scala> val tb =
 CatalogTable(
  TableIdentifier("test", None),
  CatalogTableType.MANAGED,
  CatalogStorageFormat.empty,
  StructType(StructField("i", DecimalType(2, -1) ) :: Nil))
```
```
scala> spark.sql("SELECT 1.23E4BD")
res2: org.apache.spark.sql.DataFrame = [1.23E+4: decimal(3,-2)]
```
while, these two different behavior could make user confused.

On the other side, even if user creates such table or `DataFrame` with negative scale decimal type, it can't write data out if using format, like `parquet` or `orc`. Because these formats have their own check for negative scale and fail on it.
```
scala> spark.sql("SELECT 1.23E4BD").write.saveAsTable("parquet")
19/12/13 17:37:04 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0)
java.lang.IllegalArgumentException: Invalid DECIMAL scale: -2
	at org.apache.parquet.Preconditions.checkArgument(Preconditions.java:53)
	at org.apache.parquet.schema.Types$BasePrimitiveBuilder.decimalMetadata(Types.java:495)
	at org.apache.parquet.schema.Types$BasePrimitiveBuilder.build(Types.java:403)
	at org.apache.parquet.schema.Types$BasePrimitiveBuilder.build(Types.java:309)
	at org.apache.parquet.schema.Types$Builder.named(Types.java:290)
	at org.apache.spark.sql.execution.datasources.parquet.SparkToParquetSchemaConverter.convertField(ParquetSchemaConverter.scala:428)
	at org.apache.spark.sql.execution.datasources.parquet.SparkToParquetSchemaConverter.convertField(ParquetSchemaConverter.scala:334)
	at org.apache.spark.sql.execution.datasources.parquet.SparkToParquetSchemaConverter.$anonfun$convert$2(ParquetSchemaConverter.scala:326)
	at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
	at scala.collection.Iterator.foreach(Iterator.scala:941)
	at scala.collection.Iterator.foreach$(Iterator.scala:941)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1429)
	at scala.collection.IterableLike.foreach(IterableLike.scala:74)
	at scala.collection.IterableLike.foreach$(IterableLike.scala:73)
	at org.apache.spark.sql.types.StructType.foreach(StructType.scala:99)
	at scala.collection.TraversableLike.map(TraversableLike.scala:238)
	at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
	at org.apache.spark.sql.types.StructType.map(StructType.scala:99)
	at org.apache.spark.sql.execution.datasources.parquet.SparkToParquetSchemaConverter.convert(ParquetSchemaConverter.scala:326)
	at org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport.init(ParquetWriteSupport.scala:97)
	at org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:388)
	at org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:349)
	at org.apache.spark.sql.execution.datasources.parquet.ParquetOutputWriter.<init>(ParquetOutputWriter.scala:37)
	at org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat$$anon$1.newInstance(ParquetFileFormat.scala:150)
	at org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.newOutputWriter(FileFormatDataWriter.scala:124)
	at org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.<init>(FileFormatDataWriter.scala:109)
	at org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:264)
	at org.apache.spark.sql.execution.datasources.FileFormatWriter$.$anonfun$write$15(FileFormatWriter.scala:205)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
	at org.apache.spark.scheduler.Task.run(Task.scala:127)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:441)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:444)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
```

So, I think it would be better to disallow negative scale totally and make behaviors above be consistent.

### Does this PR introduce any user-facing change?

Yes, if `spark.sql.legacy.allowNegativeScaleOfDecimal.enabled=false`, user couldn't create Decimal value with negative scale anymore.

### How was this patch tested?

Added new tests in `ExpressionParserSuite` and `DecimalSuite`;
Updated `SQLQueryTestSuite`.

Closes #26881 from Ngone51/nonnegative-scale.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-21 21:09:48 +08:00
HyukjinKwon 1881caa95e [SPARK-29188][PYTHON][FOLLOW-UP] Explicitly disable Arrow execution for all test of toPandas empty types
### What changes were proposed in this pull request?

Another followup of 4398dfa709

I missed two more tests added:

```
======================================================================
ERROR [0.133s]: test_to_pandas_from_mixed_dataframe (pyspark.sql.tests.test_dataframe.DataFrameTests)
----------------------------------------------------------------------
Traceback (most recent call last):
  File "/home/jenkins/python/pyspark/sql/tests/test_dataframe.py", line 617, in test_to_pandas_from_mixed_dataframe
    self.assertTrue(np.all(pdf_with_only_nulls.dtypes == pdf_with_some_nulls.dtypes))
AssertionError: False is not true
======================================================================
ERROR [0.061s]: test_to_pandas_from_null_dataframe (pyspark.sql.tests.test_dataframe.DataFrameTests)
----------------------------------------------------------------------
Traceback (most recent call last):
  File "/home/jenkins/python/pyspark/sql/tests/test_dataframe.py", line 588, in test_to_pandas_from_null_dataframe
    self.assertEqual(types[0], np.float64)
AssertionError: dtype('O') != <class 'numpy.float64'>
----------------------------------------------------------------------
```

### Why are the changes needed?

To make the test independent of default values of configuration.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Manually tested and Jenkins should test.

Closes #27250 from HyukjinKwon/SPARK-29188-followup2.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-17 15:00:18 +09:00
HyukjinKwon 4398dfa709 [SPARK-29188][PYTHON][FOLLOW-UP] Explicitly disable Arrow execution for the test of toPandas empty types
### What changes were proposed in this pull request?

This PR proposes to explicitly disable Arrow execution for the test of toPandas empty types. If `spark.sql.execution.arrow.pyspark.enabled` is enabled by default, this test alone fails as below:

```
======================================================================
ERROR [0.205s]: test_to_pandas_from_empty_dataframe (pyspark.sql.tests.test_dataframe.DataFrameTests)
----------------------------------------------------------------------
Traceback (most recent call last):
  File "/.../pyspark/sql/tests/test_dataframe.py", line 568, in test_to_pandas_from_empty_dataframe
    self.assertTrue(np.all(dtypes_when_empty_df == dtypes_when_nonempty_df))
AssertionError: False is not true
----------------------------------------------------------------------
```

it should be best to explicitly disable for the test that only works when it's disabled.

### Why are the changes needed?

To make the test independent of default values of configuration.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Manually tested and Jenkins should test.

Closes #27247 from HyukjinKwon/SPARK-29188-followup.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-16 19:27:30 -08:00
Maxim Gekk 1a9de8c31f [SPARK-30499][SQL] Remove SQL config spark.sql.execution.pandas.respectSessionTimeZone
### What changes were proposed in this pull request?
In the PR, I propose to remove the SQL config `spark.sql.execution.pandas.respectSessionTimeZone` which has been deprecated since Spark 2.3.

### Why are the changes needed?
To improve code maintainability.

### Does this PR introduce any user-facing change?
Yes.

### How was this patch tested?
by running python tests, https://spark.apache.org/docs/latest/building-spark.html#pyspark-tests-with-maven-or-sbt

Closes #27218 from MaxGekk/remove-respectSessionTimeZone.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-17 11:44:49 +09:00
jiake b389b8c5f0 [SPARK-30188][SQL] Resolve the failed unit tests when enable AQE
### What changes were proposed in this pull request?
Fix all the failed tests when enable AQE.

### Why are the changes needed?
Run more tests with AQE to catch bugs, and make it easier to enable AQE by default in the future.

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
Existing unit tests

Closes #26813 from JkSelf/enableAQEDefault.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-13 22:55:19 +08:00
Bryan Cutler f372d1cf4f [SPARK-29748][PYTHON][SQL] Remove Row field sorting in PySpark for version 3.6+
### What changes were proposed in this pull request?

Removing the sorting of PySpark SQL Row fields that were previously sorted by name alphabetically for Python versions 3.6 and above. Field order will now match that as entered. Rows will be used like tuples and are applied to schema by position. For Python versions < 3.6, the order of kwargs is not guaranteed and therefore will be sorted automatically as in previous versions of Spark.

### Why are the changes needed?

This caused inconsistent behavior in that local Rows could be applied to a schema by matching names, but once serialized the Row could only be used by position and the fields were possibly in a different order.

### Does this PR introduce any user-facing change?

Yes, Row fields are no longer sorted alphabetically but will be in the order entered. For Python < 3.6 `kwargs` can not guarantee the order as entered, so `Row`s will be automatically sorted.

An environment variable "PYSPARK_ROW_FIELD_SORTING_ENABLED" can be set that will override construction of `Row` to maintain compatibility with Spark 2.x.

### How was this patch tested?

Existing tests are run with PYSPARK_ROW_FIELD_SORTING_ENABLED=true and added new test with unsorted fields for Python 3.6+

Closes #26496 from BryanCutler/pyspark-remove-Row-sorting-SPARK-29748.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2020-01-10 14:37:59 -08:00
HyukjinKwon ee8d661058 [SPARK-30434][PYTHON][SQL] Move pandas related functionalities into 'pandas' sub-package
### What changes were proposed in this pull request?

This PR proposes to move pandas related functionalities into pandas package. Namely:

```bash
pyspark/sql/pandas
├── __init__.py
├── conversion.py  # Conversion between pandas <> PySpark DataFrames
├── functions.py   # pandas_udf
├── group_ops.py   # Grouped UDF / Cogrouped UDF + groupby.apply, groupby.cogroup.apply
├── map_ops.py     # Map Iter UDF + mapInPandas
├── serializers.py # pandas <> PyArrow serializers
├── types.py       # Type utils between pandas <> PyArrow
└── utils.py       # Version requirement checks
```

In order to separately locate `groupby.apply`, `groupby.cogroup.apply`, `mapInPandas`, `toPandas`, and `createDataFrame(pdf)` under `pandas` sub-package, I had to use a mix-in approach which Scala side uses often by `trait`, and also pandas itself uses this approach (see `IndexOpsMixin` as an example) to group related functionalities. Currently, you can think it's like Scala's self typed trait. See the structure below:

```python
class PandasMapOpsMixin(object):
    def mapInPandas(self, ...):
        ...
        return ...

    # other Pandas <> PySpark APIs
```

```python
class DataFrame(PandasMapOpsMixin):

    # other DataFrame APIs equivalent to Scala side.

```

Yes, This is a big PR but they are mostly just moving around except one case `createDataFrame` which I had to split the methods.

### Why are the changes needed?

There are pandas functionalities here and there and I myself gets lost where it was. Also, when you have to make a change commonly for all of pandas related features, it's almost impossible now.

Also, after this change, `DataFrame` and `SparkSession` become more consistent with Scala side since pandas is specific to Python, and this change separates pandas-specific APIs away from `DataFrame` or `SparkSession`.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Existing tests should cover. Also, I manually built the PySpark API documentation and checked.

Closes #27109 from HyukjinKwon/pandas-refactoring.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-09 10:22:50 +09:00
David 8e9bfea107 [SPARK-29188][PYTHON] toPandas (without Arrow) gets wrong dtypes when applied on empty DF
### What changes were proposed in this pull request?

An empty Spark DataFrame converted to a Pandas DataFrame wouldn't have the right column types. Several type mappings were missing.

### Why are the changes needed?

Empty Spark DataFrames can be used to write unit tests, and verified by converting them to Pandas first. But this can fail when the column types are wrong.

### Does this PR introduce any user-facing change?

Yes; the error reported in the JIRA issue should not happen anymore.

### How was this patch tested?

Through unit tests in `pyspark.sql.tests.test_dataframe.DataFrameTests#test_to_pandas_from_empty_dataframe`

Closes #26747 from dlindelof/SPARK-29188.

Authored-by: David <dlindelof@expediagroup.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-12 20:49:10 +09:00
Bago Amirbekian 8152a87235 [SPARK-28978][ ] Support > 256 args to python udf
### What changes were proposed in this pull request?

On the worker we express lambda functions as strings and then eval them to create a "mapper" function. This make the code hard to read & limits the # of arguments a udf can support to 256 for python <= 3.6.

This PR rewrites the mapper functions as nested functions instead of "lambda strings" and allows passing in more than 255 args.

### Why are the changes needed?
The jira ticket associated with this issue describes how MLflow uses udfs to consume columns as features. This pattern isn't unique and a limit of 255 features is quite low.

### Does this PR introduce any user-facing change?
Users can now pass more than 255 cols to a udf function.

### How was this patch tested?
Added a unit test for passing in > 255 args to udf.

Closes #26442 from MrBago/replace-lambdas-on-worker.

Authored-by: Bago Amirbekian <bago@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-11-08 19:19:14 -08:00
HyukjinKwon 7fc9db0853 [SPARK-29798][PYTHON][SQL] Infers bytes as binary type in createDataFrame in Python 3 at PySpark
### What changes were proposed in this pull request?

This PR proposes to infer bytes as binary types in Python 3. See https://github.com/apache/spark/pull/25749 for discussions. I have also checked that Arrow considers `bytes` as binary type, and PySpark UDF can also accepts `bytes` as a binary type.

Since `bytes` is not a `str` anymore in Python 3, it's clear to call it `BinaryType` in Python 3.

### Why are the changes needed?

To respect Python 3's `bytes` type and support Python's primitive types.

### Does this PR introduce any user-facing change?

Yes.

**Before:**

```python
>>> spark.createDataFrame([[b"abc"]])
Traceback (most recent call last):
  File "/.../spark/python/pyspark/sql/types.py", line 1036, in _infer_type
    return _infer_schema(obj)
  File "/.../spark/python/pyspark/sql/types.py", line 1062, in _infer_schema
    raise TypeError("Can not infer schema for type: %s" % type(row))
TypeError: Can not infer schema for type: <class 'bytes'>

During handling of the above exception, another exception occurred:

Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/.../spark/python/pyspark/sql/session.py", line 787, in createDataFrame
    rdd, schema = self._createFromLocal(map(prepare, data), schema)
  File "/.../spark/python/pyspark/sql/session.py", line 445, in _createFromLocal
    struct = self._inferSchemaFromList(data, names=schema)
  File "/.../spark/python/pyspark/sql/session.py", line 377, in _inferSchemaFromList
    schema = reduce(_merge_type, (_infer_schema(row, names) for row in data))
  File "/.../spark/python/pyspark/sql/session.py", line 377, in <genexpr>
    schema = reduce(_merge_type, (_infer_schema(row, names) for row in data))
  File "/.../spark/python/pyspark/sql/types.py", line 1064, in _infer_schema
    fields = [StructField(k, _infer_type(v), True) for k, v in items]
  File "/.../spark/python/pyspark/sql/types.py", line 1064, in <listcomp>
    fields = [StructField(k, _infer_type(v), True) for k, v in items]
  File "/.../spark/python/pyspark/sql/types.py", line 1038, in _infer_type
    raise TypeError("not supported type: %s" % type(obj))
TypeError: not supported type: <class 'bytes'>
```

**After:**

```python
>>> spark.createDataFrame([[b"abc"]])
DataFrame[_1: binary]
```

### How was this patch tested?
Unittest was added and manually tested.

Closes #26432 from HyukjinKwon/SPARK-29798.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-11-08 12:10:39 -08:00
Terry Kim 3175f4bf1b [SPARK-29664][PYTHON][SQL] Column.getItem behavior is not consistent with Scala
### What changes were proposed in this pull request?

This PR changes the behavior of `Column.getItem` to call `Column.getItem` on Scala side instead of `Column.apply`.

### Why are the changes needed?

The current behavior is not consistent with that of Scala.

In PySpark:
```Python
df = spark.range(2)
map_col = create_map(lit(0), lit(100), lit(1), lit(200))
df.withColumn("mapped", map_col.getItem(col('id'))).show()
# +---+------+
# | id|mapped|
# +---+------+
# |  0|   100|
# |  1|   200|
# +---+------+
```
In Scala:
```Scala
val df = spark.range(2)
val map_col = map(lit(0), lit(100), lit(1), lit(200))
// The following getItem results in the following exception, which is the right behavior:
// java.lang.RuntimeException: Unsupported literal type class org.apache.spark.sql.Column id
//  at org.apache.spark.sql.catalyst.expressions.Literal$.apply(literals.scala:78)
//  at org.apache.spark.sql.Column.getItem(Column.scala:856)
//  ... 49 elided
df.withColumn("mapped", map_col.getItem(col("id"))).show
```

### Does this PR introduce any user-facing change?

Yes. If the use wants to pass `Column` object to `getItem`, he/she now needs to use the indexing operator to achieve the previous behavior.

```Python
df = spark.range(2)
map_col = create_map(lit(0), lit(100), lit(1), lit(200))
df.withColumn("mapped", map_col[col('id'))].show()
# +---+------+
# | id|mapped|
# +---+------+
# |  0|   100|
# |  1|   200|
# +---+------+
```

### How was this patch tested?

Existing tests.

Closes #26351 from imback82/spark-29664.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-11-01 12:25:48 +09:00
Bryan Cutler beb8d2f8ad [SPARK-29402][PYTHON][TESTS] Added tests for grouped map pandas_udf with window
### What changes were proposed in this pull request?

Added tests for grouped map pandas_udf using a window.

### Why are the changes needed?

Current tests for grouped map do not use a window and this had previously caused an error due the window range being a struct column, which was not yet supported.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

New tests added.

Closes #26063 from BryanCutler/pyspark-pandas_udf-group-with-window-tests-SPARK-29402.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-10-11 16:19:13 -07:00
Chris Martin 76791b89f5 [SPARK-27463][PYTHON][FOLLOW-UP] Miscellaneous documentation and code cleanup of cogroup pandas UDF
Follow up from https://github.com/apache/spark/pull/24981 incorporating some comments from HyukjinKwon.

Specifically:

- Adding `CoGroupedData` to `pyspark/sql/__init__.py __all__` so that documentation is generated.
- Added pydoc, including example, for the use case whereby the user supplies a cogrouping function including a key.
- Added the boilerplate for doctests to cogroup.py.  Note that cogroup.py only contains the apply() function which has doctests disabled as per the  other Pandas Udfs.
- Restricted the newly exposed RelationalGroupedDataset constructor parameters to access only by the sql package.
- Some minor  formatting tweaks.

This was tested by running the appropriate unit tests.  I'm unsure as to how to check that my change will cause the documentation to be generated correctly, but it someone can describe how I can do this I'd be happy to check.

Closes #25939 from d80tb7/SPARK-27463-fixes.

Authored-by: Chris Martin <chris@cmartinit.co.uk>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-09-30 22:25:35 +09:00
Holden Karau 42050c3f4f [SPARK-27659][PYTHON] Allow PySpark to prefetch during toLocalIterator
### What changes were proposed in this pull request?

This PR allows Python toLocalIterator to prefetch the next partition while the first partition is being collected. The PR also adds a demo micro bench mark in the examples directory, we may wish to keep this or not.

### Why are the changes needed?

In https://issues.apache.org/jira/browse/SPARK-23961 / 5e79ae3b40 we changed PySpark to only pull one partition at a time. This is memory efficient, but if partitions take time to compute this can mean we're spending more time blocking.

### Does this PR introduce any user-facing change?

A new param is added to toLocalIterator

### How was this patch tested?

New unit test inside of `test_rdd.py` checks the time that the elements are evaluated at. Another test that the results remain the same are added to `test_dataframe.py`.

I also ran a micro benchmark in the examples directory `prefetch.py` which shows an improvement of ~40% in this specific use case.

>
> 19/08/16 17:11:36 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
> Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties
> Setting default log level to "WARN".
> To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
> Running timers:
>
> [Stage 32:>                                                         (0 + 1) / 1]
> Results:
>
> Prefetch time:
>
> 100.228110831
>
>
> Regular time:
>
> 188.341721614
>
>
>

Closes #25515 from holdenk/SPARK-27659-allow-pyspark-tolocalitr-to-prefetch.

Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
2019-09-20 09:59:31 -07:00
Xianjin YE 203bf9e569 [SPARK-19926][PYSPARK] make captured exception from JVM side user friendly
### What changes were proposed in this pull request?
The str of `CapaturedException` is now returned by str(self.desc) rather than repr(self.desc), which is more user-friendly. It also handles unicode under python2 specially.

### Why are the changes needed?
This is an improvement, and makes exception more human readable in python side.

### Does this PR introduce any user-facing change?
Before this pr,  select `中文字段` throws exception something likes below:

```
Traceback (most recent call last):
  File "/Users/advancedxy/code_workspace/github/spark/python/pyspark/sql/tests/test_utils.py", line 34, in test_capture_user_friendly_exception
    raise e
AnalysisException: u"cannot resolve '`\u4e2d\u6587\u5b57\u6bb5`' given input columns: []; line 1 pos 7;\n'Project ['\u4e2d\u6587\u5b57\u6bb5]\n+- OneRowRelation\n"
```

after this pr:
```
Traceback (most recent call last):
  File "/Users/advancedxy/code_workspace/github/spark/python/pyspark/sql/tests/test_utils.py", line 34, in test_capture_user_friendly_exception
    raise e
AnalysisException: cannot resolve '`中文字段`' given input columns: []; line 1 pos 7;
'Project ['中文字段]
+- OneRowRelation

```
### How was this patch
Add a new test to verify unicode are correctly converted and manual checks for thrown exceptions.

This pr's credits should go to uncleGen and is based on https://github.com/apache/spark/pull/17267

Closes #25814 from advancedxy/python_exception_19926_and_21045.

Authored-by: Xianjin YE <advancedxy@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-09-18 23:32:10 +09:00
Chris Martin 05988b256e [SPARK-27463][PYTHON] Support Dataframe Cogroup via Pandas UDFs
### What changes were proposed in this pull request?

Adds a new cogroup Pandas UDF.  This allows two grouped dataframes to be cogrouped together and apply a (pandas.DataFrame, pandas.DataFrame) -> pandas.DataFrame UDF to each cogroup.

**Example usage**

```
from pyspark.sql.functions import pandas_udf, PandasUDFType
df1 = spark.createDataFrame(
   [(20000101, 1, 1.0), (20000101, 2, 2.0), (20000102, 1, 3.0), (20000102, 2, 4.0)],
   ("time", "id", "v1"))

df2 = spark.createDataFrame(
   [(20000101, 1, "x"), (20000101, 2, "y")],
    ("time", "id", "v2"))

pandas_udf("time int, id int, v1 double, v2 string", PandasUDFType.COGROUPED_MAP)
   def asof_join(l, r):
      return pd.merge_asof(l, r, on="time", by="id")

df1.groupby("id").cogroup(df2.groupby("id")).apply(asof_join).show()

```

        +--------+---+---+---+
        |    time| id| v1| v2|
        +--------+---+---+---+
        |20000101|  1|1.0|  x|
        |20000102|  1|3.0|  x|
        |20000101|  2|2.0|  y|
        |20000102|  2|4.0|  y|
        +--------+---+---+---+

### How was this patch tested?

Added unit test test_pandas_udf_cogrouped_map

Closes #24981 from d80tb7/SPARK-27463-poc-arrow-stream.

Authored-by: Chris Martin <chris@cmartinit.co.uk>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-09-17 17:13:50 -07:00
Wenchen Fan 053dd858d3 [SPARK-28998][SQL] reorganize the packages of DS v2 interfaces/classes
### What changes were proposed in this pull request?

reorganize the packages of DS v2 interfaces/classes:
1. `org.spark.sql.connector.catalog`: put `TableCatalog`, `Table` and other related interfaces/classes
2. `org.spark.sql.connector.expression`: put `Expression`, `Transform` and other related interfaces/classes
3. `org.spark.sql.connector.read`: put `ScanBuilder`, `Scan` and other related interfaces/classes
4. `org.spark.sql.connector.write`: put `WriteBuilder`, `BatchWrite` and other related interfaces/classes

### Why are the changes needed?

Data Source V2 has evolved a lot. It's a bit weird that `Expression` is in `org.spark.sql.catalog.v2` and `Table` is in `org.spark.sql.sources.v2`.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

existing tests

Closes #25700 from cloud-fan/package.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-09-12 19:59:34 +08:00
HyukjinKwon 7ce0f2b499 [SPARK-29041][PYTHON] Allows createDataFrame to accept bytes as binary type
### What changes were proposed in this pull request?

This PR proposes to allow `bytes` as an acceptable type for binary type for `createDataFrame`.

### Why are the changes needed?

`bytes` is a standard type for binary in Python. This should be respected in PySpark side.

### Does this PR introduce any user-facing change?

Yes, _when specified type is binary_, we will allow `bytes` as a binary type. Previously this was not allowed in both Python 2 and Python 3 as below:

```python
spark.createDataFrame([[b"abcd"]], "col binary")
```

in Python 3

```
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/.../spark/python/pyspark/sql/session.py", line 787, in createDataFrame
    rdd, schema = self._createFromLocal(map(prepare, data), schema)
  File "/.../spark/python/pyspark/sql/session.py", line 442, in _createFromLocal
    data = list(data)
  File "/.../spark/python/pyspark/sql/session.py", line 769, in prepare
    verify_func(obj)
  File "/.../forked/spark/python/pyspark/sql/types.py", line 1403, in verify
    verify_value(obj)
  File "/.../spark/python/pyspark/sql/types.py", line 1384, in verify_struct
    verifier(v)
  File "/.../spark/python/pyspark/sql/types.py", line 1403, in verify
    verify_value(obj)
  File "/.../spark/python/pyspark/sql/types.py", line 1397, in verify_default
    verify_acceptable_types(obj)
  File "/.../spark/python/pyspark/sql/types.py", line 1282, in verify_acceptable_types
    % (dataType, obj, type(obj))))
TypeError: field col: BinaryType can not accept object b'abcd' in type <class 'bytes'>
```

in Python 2:

```
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/.../spark/python/pyspark/sql/session.py", line 787, in createDataFrame
    rdd, schema = self._createFromLocal(map(prepare, data), schema)
  File "/.../spark/python/pyspark/sql/session.py", line 442, in _createFromLocal
    data = list(data)
  File "/.../spark/python/pyspark/sql/session.py", line 769, in prepare
    verify_func(obj)
  File "/.../spark/python/pyspark/sql/types.py", line 1403, in verify
    verify_value(obj)
  File "/.../spark/python/pyspark/sql/types.py", line 1384, in verify_struct
    verifier(v)
  File "/.../spark/python/pyspark/sql/types.py", line 1403, in verify
    verify_value(obj)
  File "/.../spark/python/pyspark/sql/types.py", line 1397, in verify_default
    verify_acceptable_types(obj)
  File "/.../spark/python/pyspark/sql/types.py", line 1282, in verify_acceptable_types
    % (dataType, obj, type(obj))))
TypeError: field col: BinaryType can not accept object 'abcd' in type <type 'str'>
```

So, it won't break anything.

### How was this patch tested?

Unittests were added and also manually tested as below.

```bash
./run-tests --python-executables=python2,python3 --testnames "pyspark.sql.tests.test_serde"
```

Closes #25749 from HyukjinKwon/SPARK-29041.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-09-12 08:52:25 +09:00
Sean Owen 6378d4bc06 [SPARK-28980][CORE][SQL][STREAMING][MLLIB] Remove most items deprecated in Spark 2.2.0 or earlier, for Spark 3
### What changes were proposed in this pull request?

- Remove SQLContext.createExternalTable and Catalog.createExternalTable, deprecated in favor of createTable since 2.2.0, plus tests of deprecated methods
- Remove HiveContext, deprecated in 2.0.0, in favor of `SparkSession.builder.enableHiveSupport`
- Remove deprecated KinesisUtils.createStream methods, plus tests of deprecated methods, deprecate in 2.2.0
- Remove deprecated MLlib (not Spark ML) linear method support, mostly utility constructors and 'train' methods, and associated docs. This includes methods in LinearRegression, LogisticRegression, Lasso, RidgeRegression. These have been deprecated since 2.0.0
- Remove deprecated Pyspark MLlib linear method support, including LogisticRegressionWithSGD, LinearRegressionWithSGD, LassoWithSGD
- Remove 'runs' argument in KMeans.train() method, which has been a no-op since 2.0.0
- Remove deprecated ChiSqSelector isSorted protected method
- Remove deprecated 'yarn-cluster' and 'yarn-client' master argument in favor of 'yarn' and deploy mode 'cluster', etc

Notes:

- I was not able to remove deprecated DataFrameReader.json(RDD) in favor of DataFrameReader.json(Dataset); the former was deprecated in 2.2.0, but, it is still needed to support Pyspark's .json() method, which can't use a Dataset.
- Looks like SQLContext.createExternalTable was not actually deprecated in Pyspark, but, almost certainly was meant to be? Catalog.createExternalTable was.
- I afterwards noted that the toDegrees, toRadians functions were almost removed fully in SPARK-25908, but Felix suggested keeping just the R version as they hadn't been technically deprecated. I'd like to revisit that. Do we really want the inconsistency? I'm not against reverting it again, but then that implies leaving SQLContext.createExternalTable just in Pyspark too, which seems weird.
- I *kept* LogisticRegressionWithSGD, LinearRegressionWithSGD, LassoWithSGD, RidgeRegressionWithSGD in Pyspark, though deprecated, as it is hard to remove them (still used by StreamingLogisticRegressionWithSGD?) and they are not fully removed in Scala. Maybe should not have been deprecated.

### Why are the changes needed?

Deprecated items are easiest to remove in a major release, so we should do so as much as possible for Spark 3. This does not target items deprecated 'recently' as of Spark 2.3, which is still 18 months old.

### Does this PR introduce any user-facing change?

Yes, in that deprecated items are removed from some public APIs.

### How was this patch tested?

Existing tests.

Closes #25684 from srowen/SPARK-28980.

Lead-authored-by: Sean Owen <sean.owen@databricks.com>
Co-authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-09-09 10:19:40 -05:00
HyukjinKwon 8848af2635 [SPARK-28881][PYTHON][TESTS][FOLLOW-UP] Use SparkSession(SparkContext(...)) to prevent for Spark conf to affect other tests
### What changes were proposed in this pull request?

This PR proposes to match the test with branch-2.4. See https://github.com/apache/spark/pull/25593#discussion_r318109047

Seems using `SparkSession.builder` with Spark conf possibly affects other tests.

### Why are the changes needed?
To match with branch-2.4 and to make easier to backport.

### Does this PR introduce any user-facing change?
No.

### How was this patch tested?
Test was fixed.

Closes #25603 from HyukjinKwon/SPARK-28881-followup.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-28 10:39:21 +09:00
WeichenXu 7f605f5559 [SPARK-28621][SQL] Make spark.sql.crossJoin.enabled default value true
### What changes were proposed in this pull request?

Make `spark.sql.crossJoin.enabled` default value true

### Why are the changes needed?

For implicit cross join, we can set up a watchdog to cancel it if running for a long time.
When "spark.sql.crossJoin.enabled" is false, because `CheckCartesianProducts` is implemented in logical plan stage, it may generate some mismatching error which may confuse end user:
* it's done in logical phase, so we may fail queries that can be executed via broadcast join, which is very fast.
* if we move the check to the physical phase, then a query may success at the beginning, and begin to fail when the table size gets larger (other people insert data to the table). This can be quite confusing.
* the CROSS JOIN syntax doesn't work well if join reorder happens.
* some non-equi-join will generate plan using cartesian product, but `CheckCartesianProducts` do not detect it and raise error.

So that in order to address this in simpler way, we can turn off showing this cross-join error by default.

For reference, I list some cases raising mismatching error here:
Providing:
```
spark.range(2).createOrReplaceTempView("sm1") // can be broadcast
spark.range(50000000).createOrReplaceTempView("bg1") // cannot be broadcast
spark.range(60000000).createOrReplaceTempView("bg2") // cannot be broadcast
```
1) Some join could be convert to broadcast nested loop join, but CheckCartesianProducts raise error. e.g.
```
select sm1.id, bg1.id from bg1 join sm1 where sm1.id < bg1.id
```
2) Some join will run by CartesianJoin but CheckCartesianProducts DO NOT raise error. e.g.
```
select bg1.id, bg2.id from bg1 join bg2 where bg1.id < bg2.id
```

### Does this PR introduce any user-facing change?

### How was this patch tested?

Closes #25520 from WeichenXu123/SPARK-28621.

Authored-by: WeichenXu <weichen.xu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-27 21:53:37 +08:00
HyukjinKwon 00cb2f99cc [SPARK-28881][PYTHON][TESTS] Add a test to make sure toPandas with Arrow optimization throws an exception per maxResultSize
### What changes were proposed in this pull request?
This PR proposes to add a test case for:

```bash
./bin/pyspark --conf spark.driver.maxResultSize=1m
spark.conf.set("spark.sql.execution.arrow.enabled",True)
```

```python
spark.range(10000000).toPandas()
```

```
Empty DataFrame
Columns: [id]
Index: []
```

which can result in partial results (see https://github.com/apache/spark/pull/25593#issuecomment-525153808). This regression was found between Spark 2.3 and Spark 2.4, and accidentally fixed.

### Why are the changes needed?
To prevent the same regression in the future.

### Does this PR introduce any user-facing change?
No.

### How was this patch tested?
Test was added.

Closes #25594 from HyukjinKwon/SPARK-28881.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-27 17:30:06 +09:00
Liang-Chi Hsieh e6a0385289 [SPARK-28422][SQL][PYTHON] GROUPED_AGG pandas_udf should work without group by clause
## What changes were proposed in this pull request?

A GROUPED_AGG pandas python udf can't work, if without group by clause, like `select udf(id) from table`.

This doesn't match with aggregate function like sum, count..., and also dataset API like `df.agg(udf(df['id']))`.

When we parse a udf (or an aggregate function) like that from SQL syntax, it is known as a function in a project. `GlobalAggregates` rule in analysis makes such project as aggregate, by looking for aggregate expressions. At the moment, we should also look for GROUPED_AGG pandas python udf.

## How was this patch tested?

Added tests.

Closes #25352 from viirya/SPARK-28422.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-14 00:32:33 +09:00
Anton Yanchenko bda5b51576 [SPARK-28454][PYTHON] Validate LongType in createDataFrame(verifySchema=True)
## What changes were proposed in this pull request?

Add missing validation for `LongType` in `pyspark.sql.types._make_type_verifier`.

## How was this patch tested?

Doctests / unittests / manual tests.

Unpatched version:
```
In [23]: s.createDataFrame([{'x': 1 << 64}], StructType([StructField('x', LongType())])).collect()
Out[23]: [Row(x=None)]
```

Patched:
```
In [5]: s.createDataFrame([{'x': 1 << 64}], StructType([StructField('x', LongType())])).collect()
---------------------------------------------------------------------------
ValueError                                Traceback (most recent call last)
<ipython-input-5-c1740fcadbf9> in <module>
----> 1 s.createDataFrame([{'x': 1 << 64}], StructType([StructField('x', LongType())])).collect()

/usr/local/lib/python3.5/site-packages/pyspark/sql/session.py in createDataFrame(self, data, schema, samplingRatio, verifySchema)
    689             rdd, schema = self._createFromRDD(data.map(prepare), schema, samplingRatio)
    690         else:
--> 691             rdd, schema = self._createFromLocal(map(prepare, data), schema)
    692         jrdd = self._jvm.SerDeUtil.toJavaArray(rdd._to_java_object_rdd())
    693         jdf = self._jsparkSession.applySchemaToPythonRDD(jrdd.rdd(), schema.json())

/usr/local/lib/python3.5/site-packages/pyspark/sql/session.py in _createFromLocal(self, data, schema)
    405         # make sure data could consumed multiple times
    406         if not isinstance(data, list):
--> 407             data = list(data)
    408
    409         if schema is None or isinstance(schema, (list, tuple)):

/usr/local/lib/python3.5/site-packages/pyspark/sql/session.py in prepare(obj)
    671
    672             def prepare(obj):
--> 673                 verify_func(obj)
    674                 return obj
    675         elif isinstance(schema, DataType):

/usr/local/lib/python3.5/site-packages/pyspark/sql/types.py in verify(obj)
   1427     def verify(obj):
   1428         if not verify_nullability(obj):
-> 1429             verify_value(obj)
   1430
   1431     return verify

/usr/local/lib/python3.5/site-packages/pyspark/sql/types.py in verify_struct(obj)
   1397             if isinstance(obj, dict):
   1398                 for f, verifier in verifiers:
-> 1399                     verifier(obj.get(f))
   1400             elif isinstance(obj, Row) and getattr(obj, "__from_dict__", False):
   1401                 # the order in obj could be different than dataType.fields

/usr/local/lib/python3.5/site-packages/pyspark/sql/types.py in verify(obj)
   1427     def verify(obj):
   1428         if not verify_nullability(obj):
-> 1429             verify_value(obj)
   1430
   1431     return verify

/usr/local/lib/python3.5/site-packages/pyspark/sql/types.py in verify_long(obj)
   1356             if obj < -9223372036854775808 or obj > 9223372036854775807:
   1357                 raise ValueError(
-> 1358                     new_msg("object of LongType out of range, got: %s" % obj))
   1359
   1360         verify_value = verify_long

ValueError: field x: object of LongType out of range, got: 18446744073709551616
```

Closes #25117 from simplylizz/master.

Authored-by: Anton Yanchenko <simplylizz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-08 11:47:25 +09:00
HyukjinKwon b8e13b0aea [SPARK-28153][PYTHON] Use AtomicReference at InputFileBlockHolder (to support input_file_name with Python UDF)
## What changes were proposed in this pull request?

This PR proposes to use `AtomicReference` so that parent and child threads can access to the same file block holder.

Python UDF expressions are turned to a plan and then it launches a separate thread to consume the input iterator. In the separate child thread, the iterator sets `InputFileBlockHolder.set` before the parent does which the parent thread is unable to read later.

1. In this separate child thread, if it happens to call `InputFileBlockHolder.set` first without initialization of the parent's thread local (which is done when the `ThreadLocal.get()` is first called), the child thread seems calling its own `initialValue` to initialize.

2. After that, the parent calls its own `initialValue` to initializes at the first call of `ThreadLocal.get()`.

3. Both now have two different references. Updating at child isn't reflected to parent.

This PR fixes it via initializing parent's thread local with `AtomicReference` for file status so that they can be used in each task, and children thread's update is reflected.

I also tried to explain this a bit more at https://github.com/apache/spark/pull/24958#discussion_r297203041.

## How was this patch tested?

Manually tested and unittest was added.

Closes #24958 from HyukjinKwon/SPARK-28153.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-07-31 22:40:01 +08:00
WeichenXu 3b14088541 [SPARK-26175][PYTHON] Redirect the standard input of the forked child to devnull in daemon
## What changes were proposed in this pull request?

PySpark worker daemon reads from stdin the worker PIDs to kill. 1bb60ab839/python/pyspark/daemon.py (L127)

However, the worker process is a forked process from the worker daemon process and we didn't close stdin on the child after fork. This means the child and user program can read stdin as well, which blocks daemon from receiving the PID to kill. This can cause issues because the task reaper might detect the task was not terminated and eventually kill the JVM.

This PR fix this by redirecting the standard input of the forked child to devnull.

## How was this patch tested?

Manually test.

In `pyspark`, run:
```
import subprocess
def task(_):
  subprocess.check_output(["cat"])

sc.parallelize(range(1), 1).mapPartitions(task).count()
```

Before:
The job will get stuck and press Ctrl+C to exit the job but the python worker process do not exit.
After:
The job finish correctly. The "cat" print nothing (because the dummay stdin is "/dev/null").
The python worker process exit normally.

Please review https://spark.apache.org/contributing.html before opening a pull request.

Closes #25138 from WeichenXu123/SPARK-26175.

Authored-by: WeichenXu <weichen.xu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-31 09:10:24 +09:00
zero323 a0c2fa63ab [SPARK-28439][PYTHON][SQL] Add support for count: Column in array_repeat
## What changes were proposed in this pull request?

This adds simple check for `count` argument:

- If it is a `Column` we apply `_to_java_column` before invoking JVM counterpart
- Otherwise we proceed as before.

## How was this patch tested?

Manual testing.

Closes #25193 from zero323/SPARK-28278.

Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-18 12:58:48 -07:00