2013-07-16 20:21:33 -04:00
|
|
|
#
|
|
|
|
# Licensed to the Apache Software Foundation (ASF) under one or more
|
|
|
|
# contributor license agreements. See the NOTICE file distributed with
|
|
|
|
# this work for additional information regarding copyright ownership.
|
|
|
|
# The ASF licenses this file to You under the Apache License, Version 2.0
|
|
|
|
# (the "License"); you may not use this file except in compliance with
|
|
|
|
# the License. You may obtain a copy of the License at
|
|
|
|
#
|
|
|
|
# http://www.apache.org/licenses/LICENSE-2.0
|
|
|
|
#
|
|
|
|
# Unless required by applicable law or agreed to in writing, software
|
|
|
|
# distributed under the License is distributed on an "AS IS" BASIS,
|
|
|
|
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
|
|
# See the License for the specific language governing permissions and
|
|
|
|
# limitations under the License.
|
|
|
|
#
|
|
|
|
|
2012-08-10 04:10:02 -04:00
|
|
|
"""
|
|
|
|
Worker that receives input from Piped RDD.
|
|
|
|
"""
|
2015-04-16 19:20:57 -04:00
|
|
|
from __future__ import print_function
|
2013-02-01 03:25:19 -05:00
|
|
|
import os
|
2012-08-10 04:10:02 -04:00
|
|
|
import sys
|
2013-05-06 19:34:30 -04:00
|
|
|
import time
|
2018-12-02 04:41:08 -05:00
|
|
|
# 'resource' is a Unix specific module.
|
|
|
|
has_resource_module = True
|
|
|
|
try:
|
|
|
|
import resource
|
|
|
|
except ImportError:
|
|
|
|
has_resource_module = False
|
2013-01-31 21:02:28 -05:00
|
|
|
import traceback
|
2014-09-14 01:31:21 -04:00
|
|
|
|
2013-01-20 04:57:44 -05:00
|
|
|
from pyspark.accumulators import _accumulatorRegistry
|
2012-08-25 16:59:01 -04:00
|
|
|
from pyspark.broadcast import Broadcast, _broadcastRegistry
|
2018-08-28 21:47:38 -04:00
|
|
|
from pyspark.java_gateway import local_connect_and_auth
|
2018-08-21 18:54:30 -04:00
|
|
|
from pyspark.taskcontext import BarrierTaskContext, TaskContext
|
2013-01-21 19:42:24 -05:00
|
|
|
from pyspark.files import SparkFiles
|
2017-11-17 10:43:08 -05:00
|
|
|
from pyspark.rdd import PythonEvalType
|
2018-08-21 18:54:30 -04:00
|
|
|
from pyspark.serializers import write_with_length, write_int, read_long, read_bool, \
|
2017-11-17 10:43:08 -05:00
|
|
|
write_long, read_int, SpecialLengths, UTF8Deserializer, PickleSerializer, \
|
2019-03-21 04:44:51 -04:00
|
|
|
BatchedSerializer, ArrowStreamPandasUDFSerializer
|
2019-03-07 11:52:24 -05:00
|
|
|
from pyspark.sql.types import to_arrow_type, StructType
|
[SPARK-23754][PYTHON][FOLLOWUP] Move UDF stop iteration wrapping from driver to executor
## What changes were proposed in this pull request?
SPARK-23754 was fixed in #21383 by changing the UDF code to wrap the user function, but this required a hack to save its argspec. This PR reverts this change and fixes the `StopIteration` bug in the worker
## How does this work?
The root of the problem is that when an user-supplied function raises a `StopIteration`, pyspark might stop processing data, if this function is used in a for-loop. The solution is to catch `StopIteration`s exceptions and re-raise them as `RuntimeError`s, so that the execution fails and the error is reported to the user. This is done using the `fail_on_stopiteration` wrapper, in different ways depending on where the function is used:
- In RDDs, the user function is wrapped in the driver, because this function is also called in the driver itself.
- In SQL UDFs, the function is wrapped in the worker, since all processing happens there. Moreover, the worker needs the signature of the user function, which is lost when wrapping it, but passing this signature to the worker requires a not so nice hack.
## How was this patch tested?
Same tests, plus tests for pandas UDFs
Author: edorigatti <emilio.dorigatti@gmail.com>
Closes #21467 from e-dorigatti/fix_udf_hack.
2018-06-10 22:15:42 -04:00
|
|
|
from pyspark.util import _get_argspec, fail_on_stopiteration
|
2014-09-14 01:31:21 -04:00
|
|
|
from pyspark import shuffle
|
2013-11-05 20:52:39 -05:00
|
|
|
|
2018-06-23 21:28:46 -04:00
|
|
|
if sys.version >= '3':
|
|
|
|
basestring = str
|
2019-03-12 11:23:26 -04:00
|
|
|
else:
|
|
|
|
from itertools import imap as map # use iterator map by default
|
2018-06-23 21:28:46 -04:00
|
|
|
|
2013-11-05 20:52:39 -05:00
|
|
|
pickleSer = PickleSerializer()
|
2014-01-28 22:50:26 -05:00
|
|
|
utf8_deserializer = UTF8Deserializer()
|
2012-08-25 19:46:07 -04:00
|
|
|
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2013-05-06 19:34:30 -04:00
|
|
|
def report_times(outfile, boot, init, finish):
|
2013-11-03 00:13:18 -04:00
|
|
|
write_int(SpecialLengths.TIMING_DATA, outfile)
|
2015-04-16 19:20:57 -04:00
|
|
|
write_long(int(1000 * boot), outfile)
|
|
|
|
write_long(int(1000 * init), outfile)
|
|
|
|
write_long(int(1000 * finish), outfile)
|
2013-03-10 16:54:46 -04:00
|
|
|
|
|
|
|
|
2014-09-24 15:10:09 -04:00
|
|
|
def add_path(path):
|
|
|
|
# worker can be used, so donot add path multiple times
|
|
|
|
if path not in sys.path:
|
|
|
|
# overwrite system packages
|
|
|
|
sys.path.insert(1, path)
|
|
|
|
|
|
|
|
|
[SPARK-14215] [SQL] [PYSPARK] Support chained Python UDFs
## What changes were proposed in this pull request?
This PR brings the support for chained Python UDFs, for example
```sql
select udf1(udf2(a))
select udf1(udf2(a) + 3)
select udf1(udf2(a) + udf3(b))
```
Also directly chained unary Python UDFs are put in single batch of Python UDFs, others may require multiple batches.
For example,
```python
>>> sqlContext.sql("select double(double(1))").explain()
== Physical Plan ==
WholeStageCodegen
: +- Project [pythonUDF#10 AS double(double(1))#9]
: +- INPUT
+- !BatchPythonEvaluation double(double(1)), [pythonUDF#10]
+- Scan OneRowRelation[]
>>> sqlContext.sql("select double(double(1) + double(2))").explain()
== Physical Plan ==
WholeStageCodegen
: +- Project [pythonUDF#19 AS double((double(1) + double(2)))#16]
: +- INPUT
+- !BatchPythonEvaluation double((pythonUDF#17 + pythonUDF#18)), [pythonUDF#17,pythonUDF#18,pythonUDF#19]
+- !BatchPythonEvaluation double(2), [pythonUDF#17,pythonUDF#18]
+- !BatchPythonEvaluation double(1), [pythonUDF#17]
+- Scan OneRowRelation[]
```
TODO: will support multiple unrelated Python UDFs in one batch (another PR).
## How was this patch tested?
Added new unit tests for chained UDFs.
Author: Davies Liu <davies@databricks.com>
Closes #12014 from davies/py_udfs.
2016-03-29 18:06:29 -04:00
|
|
|
def read_command(serializer, file):
|
|
|
|
command = serializer._read_with_length(file)
|
|
|
|
if isinstance(command, Broadcast):
|
|
|
|
command = serializer.loads(command.value)
|
|
|
|
return command
|
|
|
|
|
|
|
|
|
2017-09-25 21:54:00 -04:00
|
|
|
def chain(f, g):
|
2017-09-22 04:17:41 -04:00
|
|
|
"""chain two functions together """
|
2017-09-25 21:54:00 -04:00
|
|
|
return lambda *a: g(f(*a))
|
[SPARK-14267] [SQL] [PYSPARK] execute multiple Python UDFs within single batch
## What changes were proposed in this pull request?
This PR support multiple Python UDFs within single batch, also improve the performance.
```python
>>> from pyspark.sql.types import IntegerType
>>> sqlContext.registerFunction("double", lambda x: x * 2, IntegerType())
>>> sqlContext.registerFunction("add", lambda x, y: x + y, IntegerType())
>>> sqlContext.sql("SELECT double(add(1, 2)), add(double(2), 1)").explain(True)
== Parsed Logical Plan ==
'Project [unresolvedalias('double('add(1, 2)), None),unresolvedalias('add('double(2), 1), None)]
+- OneRowRelation$
== Analyzed Logical Plan ==
double(add(1, 2)): int, add(double(2), 1): int
Project [double(add(1, 2))#14,add(double(2), 1)#15]
+- Project [double(add(1, 2))#14,add(double(2), 1)#15]
+- Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15]
+- EvaluatePython [add(pythonUDF1#17, 1)], [pythonUDF0#18]
+- EvaluatePython [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17]
+- OneRowRelation$
== Optimized Logical Plan ==
Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15]
+- EvaluatePython [add(pythonUDF1#17, 1)], [pythonUDF0#18]
+- EvaluatePython [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17]
+- OneRowRelation$
== Physical Plan ==
WholeStageCodegen
: +- Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15]
: +- INPUT
+- !BatchPythonEvaluation [add(pythonUDF1#17, 1)], [pythonUDF0#16,pythonUDF1#17,pythonUDF0#18]
+- !BatchPythonEvaluation [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17]
+- Scan OneRowRelation[]
```
## How was this patch tested?
Added new tests.
Using the following script to benchmark 1, 2 and 3 udfs,
```
df = sqlContext.range(1, 1 << 23, 1, 4)
double = F.udf(lambda x: x * 2, LongType())
print df.select(double(df.id)).count()
print df.select(double(df.id), double(df.id + 1)).count()
print df.select(double(df.id), double(df.id + 1), double(df.id + 2)).count()
```
Here is the results:
N | Before | After | speed up
---- |------------ | -------------|------
1 | 22 s | 7 s | 3.1X
2 | 38 s | 13 s | 2.9X
3 | 58 s | 16 s | 3.6X
This benchmark ran locally with 4 CPUs. For 3 UDFs, it launched 12 Python before before this patch, 4 process after this patch. After this patch, it will use less memory for multiple UDFs than before (less buffering).
Author: Davies Liu <davies@databricks.com>
Closes #12057 from davies/multi_udfs.
2016-03-31 19:40:20 -04:00
|
|
|
|
|
|
|
|
|
|
|
def wrap_udf(f, return_type):
|
|
|
|
if return_type.needConversion():
|
|
|
|
toInternal = return_type.toInternal
|
|
|
|
return lambda *a: toInternal(f(*a))
|
|
|
|
else:
|
|
|
|
return lambda *a: f(*a)
|
|
|
|
|
|
|
|
|
2018-01-30 07:55:55 -05:00
|
|
|
def wrap_scalar_pandas_udf(f, return_type):
|
2017-10-20 15:44:30 -04:00
|
|
|
arrow_return_type = to_arrow_type(return_type)
|
2017-10-10 18:32:01 -04:00
|
|
|
|
2017-10-20 15:44:30 -04:00
|
|
|
def verify_result_length(*a):
|
|
|
|
result = f(*a)
|
|
|
|
if not hasattr(result, "__len__"):
|
2019-03-07 11:52:24 -05:00
|
|
|
pd_type = "Pandas.DataFrame" if type(return_type) == StructType else "Pandas.Series"
|
2018-05-13 16:19:03 -04:00
|
|
|
raise TypeError("Return type of the user-defined function should be "
|
2019-03-07 11:52:24 -05:00
|
|
|
"{}, but is {}".format(pd_type, type(result)))
|
2017-10-20 15:44:30 -04:00
|
|
|
if len(result) != len(a[0]):
|
|
|
|
raise RuntimeError("Result vector from pandas_udf was not the required length: "
|
|
|
|
"expected %d, got %d" % (len(a[0]), len(result)))
|
|
|
|
return result
|
2017-10-10 18:32:01 -04:00
|
|
|
|
2017-10-20 15:44:30 -04:00
|
|
|
return lambda *a: (verify_result_length(*a), arrow_return_type)
|
2017-09-22 04:17:41 -04:00
|
|
|
|
|
|
|
|
2019-03-21 04:44:51 -04:00
|
|
|
def wrap_grouped_map_pandas_udf(f, return_type, argspec):
|
2018-06-23 21:28:46 -04:00
|
|
|
|
2018-03-08 06:29:07 -05:00
|
|
|
def wrapped(key_series, value_series):
|
2017-11-17 10:43:08 -05:00
|
|
|
import pandas as pd
|
2018-03-08 06:29:07 -05:00
|
|
|
|
|
|
|
if len(argspec.args) == 1:
|
|
|
|
result = f(pd.concat(value_series, axis=1))
|
|
|
|
elif len(argspec.args) == 2:
|
|
|
|
key = tuple(s[0] for s in key_series)
|
|
|
|
result = f(key, pd.concat(value_series, axis=1))
|
2017-11-17 10:43:08 -05:00
|
|
|
|
|
|
|
if not isinstance(result, pd.DataFrame):
|
|
|
|
raise TypeError("Return type of the user-defined function should be "
|
|
|
|
"pandas.DataFrame, but is {}".format(type(result)))
|
|
|
|
if not len(result.columns) == len(return_type):
|
|
|
|
raise RuntimeError(
|
|
|
|
"Number of columns of the returned pandas.DataFrame "
|
|
|
|
"doesn't match specified schema. "
|
|
|
|
"Expected: {} Actual: {}".format(len(return_type), len(result.columns)))
|
2019-03-21 04:44:51 -04:00
|
|
|
return result
|
2018-06-23 21:28:46 -04:00
|
|
|
|
2019-03-21 04:44:51 -04:00
|
|
|
return lambda k, v: [(wrapped(k, v), to_arrow_type(return_type))]
|
2017-11-17 10:43:08 -05:00
|
|
|
|
|
|
|
|
2018-01-30 07:55:55 -05:00
|
|
|
def wrap_grouped_agg_pandas_udf(f, return_type):
|
2018-01-23 00:11:30 -05:00
|
|
|
arrow_return_type = to_arrow_type(return_type)
|
|
|
|
|
|
|
|
def wrapped(*series):
|
|
|
|
import pandas as pd
|
|
|
|
result = f(*series)
|
[SPARK-23352][PYTHON] Explicitly specify supported types in Pandas UDFs
## What changes were proposed in this pull request?
This PR targets to explicitly specify supported types in Pandas UDFs.
The main change here is to add a deduplicated and explicit type checking in `returnType` ahead with documenting this; however, it happened to fix multiple things.
1. Currently, we don't support `BinaryType` in Pandas UDFs, for example, see:
```python
from pyspark.sql.functions import pandas_udf
pudf = pandas_udf(lambda x: x, "binary")
df = spark.createDataFrame([[bytearray(1)]])
df.select(pudf("_1")).show()
```
```
...
TypeError: Unsupported type in conversion to Arrow: BinaryType
```
We can document this behaviour for its guide.
2. Also, the grouped aggregate Pandas UDF fails fast on `ArrayType` but seems we can support this case.
```python
from pyspark.sql.functions import pandas_udf, PandasUDFType
foo = pandas_udf(lambda v: v.mean(), 'array<double>', PandasUDFType.GROUPED_AGG)
df = spark.range(100).selectExpr("id", "array(id) as value")
df.groupBy("id").agg(foo("value")).show()
```
```
...
NotImplementedError: ArrayType, StructType and MapType are not supported with PandasUDFType.GROUPED_AGG
```
3. Since we can check the return type ahead, we can fail fast before actual execution.
```python
# we can fail fast at this stage because we know the schema ahead
pandas_udf(lambda x: x, BinaryType())
```
## How was this patch tested?
Manually tested and unit tests for `BinaryType` and `ArrayType(...)` were added.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #20531 from HyukjinKwon/pudf-cleanup.
2018-02-12 06:49:36 -05:00
|
|
|
return pd.Series([result])
|
2018-01-23 00:11:30 -05:00
|
|
|
|
|
|
|
return lambda *a: (wrapped(*a), arrow_return_type)
|
|
|
|
|
|
|
|
|
[SPARK-24561][SQL][PYTHON] User-defined window aggregation functions with Pandas UDF (bounded window)
## What changes were proposed in this pull request?
This PR implements a new feature - window aggregation Pandas UDF for bounded window.
#### Doc:
https://docs.google.com/document/d/14EjeY5z4-NC27-SmIP9CsMPCANeTcvxN44a7SIJtZPc/edit#heading=h.c87w44wcj3wj
#### Example:
```
from pyspark.sql.functions import pandas_udf, PandasUDFType
from pyspark.sql.window import Window
df = spark.range(0, 10, 2).toDF('v')
w1 = Window.partitionBy().orderBy('v').rangeBetween(-2, 4)
w2 = Window.partitionBy().orderBy('v').rowsBetween(-2, 2)
pandas_udf('double', PandasUDFType.GROUPED_AGG)
def avg(v):
return v.mean()
df.withColumn('v_mean', avg(df['v']).over(w1)).show()
# +---+------+
# | v|v_mean|
# +---+------+
# | 0| 1.0|
# | 2| 2.0|
# | 4| 4.0|
# | 6| 6.0|
# | 8| 7.0|
# +---+------+
df.withColumn('v_mean', avg(df['v']).over(w2)).show()
# +---+------+
# | v|v_mean|
# +---+------+
# | 0| 2.0|
# | 2| 3.0|
# | 4| 4.0|
# | 6| 5.0|
# | 8| 6.0|
# +---+------+
```
#### High level changes:
This PR modifies the existing WindowInPandasExec physical node to deal with unbounded (growing, shrinking and sliding) windows.
* `WindowInPandasExec` now share the same base class as `WindowExec` and share utility functions. See `WindowExecBase`
* `WindowFunctionFrame` now has two new functions `currentLowerBound` and `currentUpperBound` - to return the lower and upper window bound for the current output row. It is also modified to allow `AggregateProcessor` == null. Null aggregator processor is used for `WindowInPandasExec` where we don't have an aggregator and only uses lower and upper bound functions from `WindowFunctionFrame`
* The biggest change is in `WindowInPandasExec`, where it is modified to take `currentLowerBound` and `currentUpperBound` and write those values together with the input data to the python process for rolling window aggregation. See `WindowInPandasExec` for more details.
#### Discussion
In benchmarking, I found numpy variant of the rolling window UDF is much faster than the pandas version:
Spark SQL window function: 20s
Pandas variant: ~80s
Numpy variant: 10s
Numpy variant with numba: 4s
Allowing numpy variant of the vectorized UDFs is something I want to discuss because of the performance improvement, but doesn't have to be in this PR.
## How was this patch tested?
New tests
Closes #22305 from icexelloss/SPARK-24561-bounded-window-udf.
Authored-by: Li Jin <ice.xelloss@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-17 20:15:21 -05:00
|
|
|
def wrap_window_agg_pandas_udf(f, return_type, runner_conf, udf_index):
|
|
|
|
window_bound_types_str = runner_conf.get('pandas_window_bound_types')
|
|
|
|
window_bound_type = [t.strip().lower() for t in window_bound_types_str.split(',')][udf_index]
|
|
|
|
if window_bound_type == 'bounded':
|
|
|
|
return wrap_bounded_window_agg_pandas_udf(f, return_type)
|
|
|
|
elif window_bound_type == 'unbounded':
|
|
|
|
return wrap_unbounded_window_agg_pandas_udf(f, return_type)
|
|
|
|
else:
|
|
|
|
raise RuntimeError("Invalid window bound type: {} ".format(window_bound_type))
|
|
|
|
|
|
|
|
|
|
|
|
def wrap_unbounded_window_agg_pandas_udf(f, return_type):
|
[SPARK-22239][SQL][PYTHON] Enable grouped aggregate pandas UDFs as window functions with unbounded window frames
## What changes were proposed in this pull request?
This PR enables using a grouped aggregate pandas UDFs as window functions. The semantics is the same as using SQL aggregation function as window functions.
```
>>> from pyspark.sql.functions import pandas_udf, PandasUDFType
>>> from pyspark.sql import Window
>>> df = spark.createDataFrame(
... [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)],
... ("id", "v"))
>>> pandas_udf("double", PandasUDFType.GROUPED_AGG)
... def mean_udf(v):
... return v.mean()
>>> w = Window.partitionBy('id')
>>> df.withColumn('mean_v', mean_udf(df['v']).over(w)).show()
+---+----+------+
| id| v|mean_v|
+---+----+------+
| 1| 1.0| 1.5|
| 1| 2.0| 1.5|
| 2| 3.0| 6.0|
| 2| 5.0| 6.0|
| 2|10.0| 6.0|
+---+----+------+
```
The scope of this PR is somewhat limited in terms of:
(1) Only supports unbounded window, which acts essentially as group by.
(2) Only supports aggregation functions, not "transform" like window functions (n -> n mapping)
Both of these are left as future work. Especially, (1) needs careful thinking w.r.t. how to pass rolling window data to python efficiently. (2) is a bit easier but does require more changes therefore I think it's better to leave it as a separate PR.
## How was this patch tested?
WindowPandasUDFTests
Author: Li Jin <ice.xelloss@gmail.com>
Closes #21082 from icexelloss/SPARK-22239-window-udf.
2018-06-12 21:10:52 -04:00
|
|
|
# This is similar to grouped_agg_pandas_udf, the only difference
|
|
|
|
# is that window_agg_pandas_udf needs to repeat the return value
|
|
|
|
# to match window length, where grouped_agg_pandas_udf just returns
|
|
|
|
# the scalar value.
|
|
|
|
arrow_return_type = to_arrow_type(return_type)
|
|
|
|
|
|
|
|
def wrapped(*series):
|
|
|
|
import pandas as pd
|
|
|
|
result = f(*series)
|
|
|
|
return pd.Series([result]).repeat(len(series[0]))
|
|
|
|
|
|
|
|
return lambda *a: (wrapped(*a), arrow_return_type)
|
|
|
|
|
|
|
|
|
[SPARK-24561][SQL][PYTHON] User-defined window aggregation functions with Pandas UDF (bounded window)
## What changes were proposed in this pull request?
This PR implements a new feature - window aggregation Pandas UDF for bounded window.
#### Doc:
https://docs.google.com/document/d/14EjeY5z4-NC27-SmIP9CsMPCANeTcvxN44a7SIJtZPc/edit#heading=h.c87w44wcj3wj
#### Example:
```
from pyspark.sql.functions import pandas_udf, PandasUDFType
from pyspark.sql.window import Window
df = spark.range(0, 10, 2).toDF('v')
w1 = Window.partitionBy().orderBy('v').rangeBetween(-2, 4)
w2 = Window.partitionBy().orderBy('v').rowsBetween(-2, 2)
pandas_udf('double', PandasUDFType.GROUPED_AGG)
def avg(v):
return v.mean()
df.withColumn('v_mean', avg(df['v']).over(w1)).show()
# +---+------+
# | v|v_mean|
# +---+------+
# | 0| 1.0|
# | 2| 2.0|
# | 4| 4.0|
# | 6| 6.0|
# | 8| 7.0|
# +---+------+
df.withColumn('v_mean', avg(df['v']).over(w2)).show()
# +---+------+
# | v|v_mean|
# +---+------+
# | 0| 2.0|
# | 2| 3.0|
# | 4| 4.0|
# | 6| 5.0|
# | 8| 6.0|
# +---+------+
```
#### High level changes:
This PR modifies the existing WindowInPandasExec physical node to deal with unbounded (growing, shrinking and sliding) windows.
* `WindowInPandasExec` now share the same base class as `WindowExec` and share utility functions. See `WindowExecBase`
* `WindowFunctionFrame` now has two new functions `currentLowerBound` and `currentUpperBound` - to return the lower and upper window bound for the current output row. It is also modified to allow `AggregateProcessor` == null. Null aggregator processor is used for `WindowInPandasExec` where we don't have an aggregator and only uses lower and upper bound functions from `WindowFunctionFrame`
* The biggest change is in `WindowInPandasExec`, where it is modified to take `currentLowerBound` and `currentUpperBound` and write those values together with the input data to the python process for rolling window aggregation. See `WindowInPandasExec` for more details.
#### Discussion
In benchmarking, I found numpy variant of the rolling window UDF is much faster than the pandas version:
Spark SQL window function: 20s
Pandas variant: ~80s
Numpy variant: 10s
Numpy variant with numba: 4s
Allowing numpy variant of the vectorized UDFs is something I want to discuss because of the performance improvement, but doesn't have to be in this PR.
## How was this patch tested?
New tests
Closes #22305 from icexelloss/SPARK-24561-bounded-window-udf.
Authored-by: Li Jin <ice.xelloss@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-17 20:15:21 -05:00
|
|
|
def wrap_bounded_window_agg_pandas_udf(f, return_type):
|
|
|
|
arrow_return_type = to_arrow_type(return_type)
|
|
|
|
|
|
|
|
def wrapped(begin_index, end_index, *series):
|
|
|
|
import pandas as pd
|
|
|
|
result = []
|
|
|
|
|
|
|
|
# Index operation is faster on np.ndarray,
|
|
|
|
# So we turn the index series into np array
|
|
|
|
# here for performance
|
|
|
|
begin_array = begin_index.values
|
|
|
|
end_array = end_index.values
|
|
|
|
|
|
|
|
for i in range(len(begin_array)):
|
|
|
|
# Note: Create a slice from a series for each window is
|
|
|
|
# actually pretty expensive. However, there
|
|
|
|
# is no easy way to reduce cost here.
|
|
|
|
# Note: s.iloc[i : j] is about 30% faster than s[i: j], with
|
|
|
|
# the caveat that the created slices shares the same
|
|
|
|
# memory with s. Therefore, user are not allowed to
|
|
|
|
# change the value of input series inside the window
|
|
|
|
# function. It is rare that user needs to modify the
|
|
|
|
# input series in the window function, and therefore,
|
|
|
|
# it is be a reasonable restriction.
|
|
|
|
# Note: Calling reset_index on the slices will increase the cost
|
|
|
|
# of creating slices by about 100%. Therefore, for performance
|
|
|
|
# reasons we don't do it here.
|
|
|
|
series_slices = [s.iloc[begin_array[i]: end_array[i]] for s in series]
|
|
|
|
result.append(f(*series_slices))
|
|
|
|
return pd.Series(result)
|
|
|
|
|
|
|
|
return lambda *a: (wrapped(*a), arrow_return_type)
|
|
|
|
|
|
|
|
|
|
|
|
def read_single_udf(pickleSer, infile, eval_type, runner_conf, udf_index):
|
[SPARK-14267] [SQL] [PYSPARK] execute multiple Python UDFs within single batch
## What changes were proposed in this pull request?
This PR support multiple Python UDFs within single batch, also improve the performance.
```python
>>> from pyspark.sql.types import IntegerType
>>> sqlContext.registerFunction("double", lambda x: x * 2, IntegerType())
>>> sqlContext.registerFunction("add", lambda x, y: x + y, IntegerType())
>>> sqlContext.sql("SELECT double(add(1, 2)), add(double(2), 1)").explain(True)
== Parsed Logical Plan ==
'Project [unresolvedalias('double('add(1, 2)), None),unresolvedalias('add('double(2), 1), None)]
+- OneRowRelation$
== Analyzed Logical Plan ==
double(add(1, 2)): int, add(double(2), 1): int
Project [double(add(1, 2))#14,add(double(2), 1)#15]
+- Project [double(add(1, 2))#14,add(double(2), 1)#15]
+- Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15]
+- EvaluatePython [add(pythonUDF1#17, 1)], [pythonUDF0#18]
+- EvaluatePython [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17]
+- OneRowRelation$
== Optimized Logical Plan ==
Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15]
+- EvaluatePython [add(pythonUDF1#17, 1)], [pythonUDF0#18]
+- EvaluatePython [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17]
+- OneRowRelation$
== Physical Plan ==
WholeStageCodegen
: +- Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15]
: +- INPUT
+- !BatchPythonEvaluation [add(pythonUDF1#17, 1)], [pythonUDF0#16,pythonUDF1#17,pythonUDF0#18]
+- !BatchPythonEvaluation [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17]
+- Scan OneRowRelation[]
```
## How was this patch tested?
Added new tests.
Using the following script to benchmark 1, 2 and 3 udfs,
```
df = sqlContext.range(1, 1 << 23, 1, 4)
double = F.udf(lambda x: x * 2, LongType())
print df.select(double(df.id)).count()
print df.select(double(df.id), double(df.id + 1)).count()
print df.select(double(df.id), double(df.id + 1), double(df.id + 2)).count()
```
Here is the results:
N | Before | After | speed up
---- |------------ | -------------|------
1 | 22 s | 7 s | 3.1X
2 | 38 s | 13 s | 2.9X
3 | 58 s | 16 s | 3.6X
This benchmark ran locally with 4 CPUs. For 3 UDFs, it launched 12 Python before before this patch, 4 process after this patch. After this patch, it will use less memory for multiple UDFs than before (less buffering).
Author: Davies Liu <davies@databricks.com>
Closes #12057 from davies/multi_udfs.
2016-03-31 19:40:20 -04:00
|
|
|
num_arg = read_int(infile)
|
|
|
|
arg_offsets = [read_int(infile) for i in range(num_arg)]
|
|
|
|
row_func = None
|
|
|
|
for i in range(read_int(infile)):
|
|
|
|
f, return_type = read_command(pickleSer, infile)
|
|
|
|
if row_func is None:
|
|
|
|
row_func = f
|
|
|
|
else:
|
2017-09-25 21:54:00 -04:00
|
|
|
row_func = chain(row_func, f)
|
2017-11-17 10:43:08 -05:00
|
|
|
|
[SPARK-23754][PYTHON][FOLLOWUP] Move UDF stop iteration wrapping from driver to executor
## What changes were proposed in this pull request?
SPARK-23754 was fixed in #21383 by changing the UDF code to wrap the user function, but this required a hack to save its argspec. This PR reverts this change and fixes the `StopIteration` bug in the worker
## How does this work?
The root of the problem is that when an user-supplied function raises a `StopIteration`, pyspark might stop processing data, if this function is used in a for-loop. The solution is to catch `StopIteration`s exceptions and re-raise them as `RuntimeError`s, so that the execution fails and the error is reported to the user. This is done using the `fail_on_stopiteration` wrapper, in different ways depending on where the function is used:
- In RDDs, the user function is wrapped in the driver, because this function is also called in the driver itself.
- In SQL UDFs, the function is wrapped in the worker, since all processing happens there. Moreover, the worker needs the signature of the user function, which is lost when wrapping it, but passing this signature to the worker requires a not so nice hack.
## How was this patch tested?
Same tests, plus tests for pandas UDFs
Author: edorigatti <emilio.dorigatti@gmail.com>
Closes #21467 from e-dorigatti/fix_udf_hack.
2018-06-10 22:15:42 -04:00
|
|
|
# make sure StopIteration's raised in the user code are not ignored
|
|
|
|
# when they are processed in a for loop, raise them as RuntimeError's instead
|
|
|
|
func = fail_on_stopiteration(row_func)
|
|
|
|
|
[SPARK-14267] [SQL] [PYSPARK] execute multiple Python UDFs within single batch
## What changes were proposed in this pull request?
This PR support multiple Python UDFs within single batch, also improve the performance.
```python
>>> from pyspark.sql.types import IntegerType
>>> sqlContext.registerFunction("double", lambda x: x * 2, IntegerType())
>>> sqlContext.registerFunction("add", lambda x, y: x + y, IntegerType())
>>> sqlContext.sql("SELECT double(add(1, 2)), add(double(2), 1)").explain(True)
== Parsed Logical Plan ==
'Project [unresolvedalias('double('add(1, 2)), None),unresolvedalias('add('double(2), 1), None)]
+- OneRowRelation$
== Analyzed Logical Plan ==
double(add(1, 2)): int, add(double(2), 1): int
Project [double(add(1, 2))#14,add(double(2), 1)#15]
+- Project [double(add(1, 2))#14,add(double(2), 1)#15]
+- Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15]
+- EvaluatePython [add(pythonUDF1#17, 1)], [pythonUDF0#18]
+- EvaluatePython [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17]
+- OneRowRelation$
== Optimized Logical Plan ==
Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15]
+- EvaluatePython [add(pythonUDF1#17, 1)], [pythonUDF0#18]
+- EvaluatePython [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17]
+- OneRowRelation$
== Physical Plan ==
WholeStageCodegen
: +- Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15]
: +- INPUT
+- !BatchPythonEvaluation [add(pythonUDF1#17, 1)], [pythonUDF0#16,pythonUDF1#17,pythonUDF0#18]
+- !BatchPythonEvaluation [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17]
+- Scan OneRowRelation[]
```
## How was this patch tested?
Added new tests.
Using the following script to benchmark 1, 2 and 3 udfs,
```
df = sqlContext.range(1, 1 << 23, 1, 4)
double = F.udf(lambda x: x * 2, LongType())
print df.select(double(df.id)).count()
print df.select(double(df.id), double(df.id + 1)).count()
print df.select(double(df.id), double(df.id + 1), double(df.id + 2)).count()
```
Here is the results:
N | Before | After | speed up
---- |------------ | -------------|------
1 | 22 s | 7 s | 3.1X
2 | 38 s | 13 s | 2.9X
3 | 58 s | 16 s | 3.6X
This benchmark ran locally with 4 CPUs. For 3 UDFs, it launched 12 Python before before this patch, 4 process after this patch. After this patch, it will use less memory for multiple UDFs than before (less buffering).
Author: Davies Liu <davies@databricks.com>
Closes #12057 from davies/multi_udfs.
2016-03-31 19:40:20 -04:00
|
|
|
# the last returnType will be the return type of UDF
|
2018-01-30 07:55:55 -05:00
|
|
|
if eval_type == PythonEvalType.SQL_SCALAR_PANDAS_UDF:
|
[SPARK-23754][PYTHON][FOLLOWUP] Move UDF stop iteration wrapping from driver to executor
## What changes were proposed in this pull request?
SPARK-23754 was fixed in #21383 by changing the UDF code to wrap the user function, but this required a hack to save its argspec. This PR reverts this change and fixes the `StopIteration` bug in the worker
## How does this work?
The root of the problem is that when an user-supplied function raises a `StopIteration`, pyspark might stop processing data, if this function is used in a for-loop. The solution is to catch `StopIteration`s exceptions and re-raise them as `RuntimeError`s, so that the execution fails and the error is reported to the user. This is done using the `fail_on_stopiteration` wrapper, in different ways depending on where the function is used:
- In RDDs, the user function is wrapped in the driver, because this function is also called in the driver itself.
- In SQL UDFs, the function is wrapped in the worker, since all processing happens there. Moreover, the worker needs the signature of the user function, which is lost when wrapping it, but passing this signature to the worker requires a not so nice hack.
## How was this patch tested?
Same tests, plus tests for pandas UDFs
Author: edorigatti <emilio.dorigatti@gmail.com>
Closes #21467 from e-dorigatti/fix_udf_hack.
2018-06-10 22:15:42 -04:00
|
|
|
return arg_offsets, wrap_scalar_pandas_udf(func, return_type)
|
2018-01-30 07:55:55 -05:00
|
|
|
elif eval_type == PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF:
|
[SPARK-23754][PYTHON][FOLLOWUP] Move UDF stop iteration wrapping from driver to executor
## What changes were proposed in this pull request?
SPARK-23754 was fixed in #21383 by changing the UDF code to wrap the user function, but this required a hack to save its argspec. This PR reverts this change and fixes the `StopIteration` bug in the worker
## How does this work?
The root of the problem is that when an user-supplied function raises a `StopIteration`, pyspark might stop processing data, if this function is used in a for-loop. The solution is to catch `StopIteration`s exceptions and re-raise them as `RuntimeError`s, so that the execution fails and the error is reported to the user. This is done using the `fail_on_stopiteration` wrapper, in different ways depending on where the function is used:
- In RDDs, the user function is wrapped in the driver, because this function is also called in the driver itself.
- In SQL UDFs, the function is wrapped in the worker, since all processing happens there. Moreover, the worker needs the signature of the user function, which is lost when wrapping it, but passing this signature to the worker requires a not so nice hack.
## How was this patch tested?
Same tests, plus tests for pandas UDFs
Author: edorigatti <emilio.dorigatti@gmail.com>
Closes #21467 from e-dorigatti/fix_udf_hack.
2018-06-10 22:15:42 -04:00
|
|
|
argspec = _get_argspec(row_func) # signature was lost when wrapping it
|
2019-03-21 04:44:51 -04:00
|
|
|
return arg_offsets, wrap_grouped_map_pandas_udf(func, return_type, argspec)
|
2018-01-30 07:55:55 -05:00
|
|
|
elif eval_type == PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF:
|
[SPARK-23754][PYTHON][FOLLOWUP] Move UDF stop iteration wrapping from driver to executor
## What changes were proposed in this pull request?
SPARK-23754 was fixed in #21383 by changing the UDF code to wrap the user function, but this required a hack to save its argspec. This PR reverts this change and fixes the `StopIteration` bug in the worker
## How does this work?
The root of the problem is that when an user-supplied function raises a `StopIteration`, pyspark might stop processing data, if this function is used in a for-loop. The solution is to catch `StopIteration`s exceptions and re-raise them as `RuntimeError`s, so that the execution fails and the error is reported to the user. This is done using the `fail_on_stopiteration` wrapper, in different ways depending on where the function is used:
- In RDDs, the user function is wrapped in the driver, because this function is also called in the driver itself.
- In SQL UDFs, the function is wrapped in the worker, since all processing happens there. Moreover, the worker needs the signature of the user function, which is lost when wrapping it, but passing this signature to the worker requires a not so nice hack.
## How was this patch tested?
Same tests, plus tests for pandas UDFs
Author: edorigatti <emilio.dorigatti@gmail.com>
Closes #21467 from e-dorigatti/fix_udf_hack.
2018-06-10 22:15:42 -04:00
|
|
|
return arg_offsets, wrap_grouped_agg_pandas_udf(func, return_type)
|
[SPARK-22239][SQL][PYTHON] Enable grouped aggregate pandas UDFs as window functions with unbounded window frames
## What changes were proposed in this pull request?
This PR enables using a grouped aggregate pandas UDFs as window functions. The semantics is the same as using SQL aggregation function as window functions.
```
>>> from pyspark.sql.functions import pandas_udf, PandasUDFType
>>> from pyspark.sql import Window
>>> df = spark.createDataFrame(
... [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)],
... ("id", "v"))
>>> pandas_udf("double", PandasUDFType.GROUPED_AGG)
... def mean_udf(v):
... return v.mean()
>>> w = Window.partitionBy('id')
>>> df.withColumn('mean_v', mean_udf(df['v']).over(w)).show()
+---+----+------+
| id| v|mean_v|
+---+----+------+
| 1| 1.0| 1.5|
| 1| 2.0| 1.5|
| 2| 3.0| 6.0|
| 2| 5.0| 6.0|
| 2|10.0| 6.0|
+---+----+------+
```
The scope of this PR is somewhat limited in terms of:
(1) Only supports unbounded window, which acts essentially as group by.
(2) Only supports aggregation functions, not "transform" like window functions (n -> n mapping)
Both of these are left as future work. Especially, (1) needs careful thinking w.r.t. how to pass rolling window data to python efficiently. (2) is a bit easier but does require more changes therefore I think it's better to leave it as a separate PR.
## How was this patch tested?
WindowPandasUDFTests
Author: Li Jin <ice.xelloss@gmail.com>
Closes #21082 from icexelloss/SPARK-22239-window-udf.
2018-06-12 21:10:52 -04:00
|
|
|
elif eval_type == PythonEvalType.SQL_WINDOW_AGG_PANDAS_UDF:
|
[SPARK-24561][SQL][PYTHON] User-defined window aggregation functions with Pandas UDF (bounded window)
## What changes were proposed in this pull request?
This PR implements a new feature - window aggregation Pandas UDF for bounded window.
#### Doc:
https://docs.google.com/document/d/14EjeY5z4-NC27-SmIP9CsMPCANeTcvxN44a7SIJtZPc/edit#heading=h.c87w44wcj3wj
#### Example:
```
from pyspark.sql.functions import pandas_udf, PandasUDFType
from pyspark.sql.window import Window
df = spark.range(0, 10, 2).toDF('v')
w1 = Window.partitionBy().orderBy('v').rangeBetween(-2, 4)
w2 = Window.partitionBy().orderBy('v').rowsBetween(-2, 2)
pandas_udf('double', PandasUDFType.GROUPED_AGG)
def avg(v):
return v.mean()
df.withColumn('v_mean', avg(df['v']).over(w1)).show()
# +---+------+
# | v|v_mean|
# +---+------+
# | 0| 1.0|
# | 2| 2.0|
# | 4| 4.0|
# | 6| 6.0|
# | 8| 7.0|
# +---+------+
df.withColumn('v_mean', avg(df['v']).over(w2)).show()
# +---+------+
# | v|v_mean|
# +---+------+
# | 0| 2.0|
# | 2| 3.0|
# | 4| 4.0|
# | 6| 5.0|
# | 8| 6.0|
# +---+------+
```
#### High level changes:
This PR modifies the existing WindowInPandasExec physical node to deal with unbounded (growing, shrinking and sliding) windows.
* `WindowInPandasExec` now share the same base class as `WindowExec` and share utility functions. See `WindowExecBase`
* `WindowFunctionFrame` now has two new functions `currentLowerBound` and `currentUpperBound` - to return the lower and upper window bound for the current output row. It is also modified to allow `AggregateProcessor` == null. Null aggregator processor is used for `WindowInPandasExec` where we don't have an aggregator and only uses lower and upper bound functions from `WindowFunctionFrame`
* The biggest change is in `WindowInPandasExec`, where it is modified to take `currentLowerBound` and `currentUpperBound` and write those values together with the input data to the python process for rolling window aggregation. See `WindowInPandasExec` for more details.
#### Discussion
In benchmarking, I found numpy variant of the rolling window UDF is much faster than the pandas version:
Spark SQL window function: 20s
Pandas variant: ~80s
Numpy variant: 10s
Numpy variant with numba: 4s
Allowing numpy variant of the vectorized UDFs is something I want to discuss because of the performance improvement, but doesn't have to be in this PR.
## How was this patch tested?
New tests
Closes #22305 from icexelloss/SPARK-24561-bounded-window-udf.
Authored-by: Li Jin <ice.xelloss@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-17 20:15:21 -05:00
|
|
|
return arg_offsets, wrap_window_agg_pandas_udf(func, return_type, runner_conf, udf_index)
|
2018-01-23 00:11:30 -05:00
|
|
|
elif eval_type == PythonEvalType.SQL_BATCHED_UDF:
|
[SPARK-23754][PYTHON][FOLLOWUP] Move UDF stop iteration wrapping from driver to executor
## What changes were proposed in this pull request?
SPARK-23754 was fixed in #21383 by changing the UDF code to wrap the user function, but this required a hack to save its argspec. This PR reverts this change and fixes the `StopIteration` bug in the worker
## How does this work?
The root of the problem is that when an user-supplied function raises a `StopIteration`, pyspark might stop processing data, if this function is used in a for-loop. The solution is to catch `StopIteration`s exceptions and re-raise them as `RuntimeError`s, so that the execution fails and the error is reported to the user. This is done using the `fail_on_stopiteration` wrapper, in different ways depending on where the function is used:
- In RDDs, the user function is wrapped in the driver, because this function is also called in the driver itself.
- In SQL UDFs, the function is wrapped in the worker, since all processing happens there. Moreover, the worker needs the signature of the user function, which is lost when wrapping it, but passing this signature to the worker requires a not so nice hack.
## How was this patch tested?
Same tests, plus tests for pandas UDFs
Author: edorigatti <emilio.dorigatti@gmail.com>
Closes #21467 from e-dorigatti/fix_udf_hack.
2018-06-10 22:15:42 -04:00
|
|
|
return arg_offsets, wrap_udf(func, return_type)
|
2018-01-23 00:11:30 -05:00
|
|
|
else:
|
|
|
|
raise ValueError("Unknown eval type: {}".format(eval_type))
|
[SPARK-14267] [SQL] [PYSPARK] execute multiple Python UDFs within single batch
## What changes were proposed in this pull request?
This PR support multiple Python UDFs within single batch, also improve the performance.
```python
>>> from pyspark.sql.types import IntegerType
>>> sqlContext.registerFunction("double", lambda x: x * 2, IntegerType())
>>> sqlContext.registerFunction("add", lambda x, y: x + y, IntegerType())
>>> sqlContext.sql("SELECT double(add(1, 2)), add(double(2), 1)").explain(True)
== Parsed Logical Plan ==
'Project [unresolvedalias('double('add(1, 2)), None),unresolvedalias('add('double(2), 1), None)]
+- OneRowRelation$
== Analyzed Logical Plan ==
double(add(1, 2)): int, add(double(2), 1): int
Project [double(add(1, 2))#14,add(double(2), 1)#15]
+- Project [double(add(1, 2))#14,add(double(2), 1)#15]
+- Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15]
+- EvaluatePython [add(pythonUDF1#17, 1)], [pythonUDF0#18]
+- EvaluatePython [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17]
+- OneRowRelation$
== Optimized Logical Plan ==
Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15]
+- EvaluatePython [add(pythonUDF1#17, 1)], [pythonUDF0#18]
+- EvaluatePython [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17]
+- OneRowRelation$
== Physical Plan ==
WholeStageCodegen
: +- Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15]
: +- INPUT
+- !BatchPythonEvaluation [add(pythonUDF1#17, 1)], [pythonUDF0#16,pythonUDF1#17,pythonUDF0#18]
+- !BatchPythonEvaluation [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17]
+- Scan OneRowRelation[]
```
## How was this patch tested?
Added new tests.
Using the following script to benchmark 1, 2 and 3 udfs,
```
df = sqlContext.range(1, 1 << 23, 1, 4)
double = F.udf(lambda x: x * 2, LongType())
print df.select(double(df.id)).count()
print df.select(double(df.id), double(df.id + 1)).count()
print df.select(double(df.id), double(df.id + 1), double(df.id + 2)).count()
```
Here is the results:
N | Before | After | speed up
---- |------------ | -------------|------
1 | 22 s | 7 s | 3.1X
2 | 38 s | 13 s | 2.9X
3 | 58 s | 16 s | 3.6X
This benchmark ran locally with 4 CPUs. For 3 UDFs, it launched 12 Python before before this patch, 4 process after this patch. After this patch, it will use less memory for multiple UDFs than before (less buffering).
Author: Davies Liu <davies@databricks.com>
Closes #12057 from davies/multi_udfs.
2016-03-31 19:40:20 -04:00
|
|
|
|
|
|
|
|
2017-09-22 04:17:41 -04:00
|
|
|
def read_udfs(pickleSer, infile, eval_type):
|
2018-06-23 21:28:46 -04:00
|
|
|
runner_conf = {}
|
|
|
|
|
|
|
|
if eval_type in (PythonEvalType.SQL_SCALAR_PANDAS_UDF,
|
|
|
|
PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF,
|
|
|
|
PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF,
|
|
|
|
PythonEvalType.SQL_WINDOW_AGG_PANDAS_UDF):
|
|
|
|
|
|
|
|
# Load conf used for pandas_udf evaluation
|
|
|
|
num_conf = read_int(infile)
|
|
|
|
for i in range(num_conf):
|
|
|
|
k = utf8_deserializer.loads(infile)
|
|
|
|
v = utf8_deserializer.loads(infile)
|
|
|
|
runner_conf[k] = v
|
|
|
|
|
|
|
|
# NOTE: if timezone is set here, that implies respectSessionTimeZone is True
|
|
|
|
timezone = runner_conf.get("spark.sql.session.timeZone", None)
|
2019-01-22 01:54:41 -05:00
|
|
|
safecheck = runner_conf.get("spark.sql.execution.pandas.arrowSafeTypeConversion",
|
|
|
|
"false").lower() == 'true'
|
2019-03-21 04:44:51 -04:00
|
|
|
# Used by SQL_GROUPED_MAP_PANDAS_UDF and SQL_SCALAR_PANDAS_UDF when returning StructType
|
2019-03-07 11:52:24 -05:00
|
|
|
assign_cols_by_name = runner_conf.get(
|
|
|
|
"spark.sql.legacy.execution.pandas.groupedMap.assignColumnsByName", "true")\
|
|
|
|
.lower() == "true"
|
|
|
|
|
2019-03-21 04:44:51 -04:00
|
|
|
ser = ArrowStreamPandasUDFSerializer(timezone, safecheck, assign_cols_by_name)
|
2018-06-23 21:28:46 -04:00
|
|
|
else:
|
|
|
|
ser = BatchedSerializer(PickleSerializer(), 100)
|
|
|
|
|
[SPARK-14267] [SQL] [PYSPARK] execute multiple Python UDFs within single batch
## What changes were proposed in this pull request?
This PR support multiple Python UDFs within single batch, also improve the performance.
```python
>>> from pyspark.sql.types import IntegerType
>>> sqlContext.registerFunction("double", lambda x: x * 2, IntegerType())
>>> sqlContext.registerFunction("add", lambda x, y: x + y, IntegerType())
>>> sqlContext.sql("SELECT double(add(1, 2)), add(double(2), 1)").explain(True)
== Parsed Logical Plan ==
'Project [unresolvedalias('double('add(1, 2)), None),unresolvedalias('add('double(2), 1), None)]
+- OneRowRelation$
== Analyzed Logical Plan ==
double(add(1, 2)): int, add(double(2), 1): int
Project [double(add(1, 2))#14,add(double(2), 1)#15]
+- Project [double(add(1, 2))#14,add(double(2), 1)#15]
+- Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15]
+- EvaluatePython [add(pythonUDF1#17, 1)], [pythonUDF0#18]
+- EvaluatePython [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17]
+- OneRowRelation$
== Optimized Logical Plan ==
Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15]
+- EvaluatePython [add(pythonUDF1#17, 1)], [pythonUDF0#18]
+- EvaluatePython [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17]
+- OneRowRelation$
== Physical Plan ==
WholeStageCodegen
: +- Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15]
: +- INPUT
+- !BatchPythonEvaluation [add(pythonUDF1#17, 1)], [pythonUDF0#16,pythonUDF1#17,pythonUDF0#18]
+- !BatchPythonEvaluation [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17]
+- Scan OneRowRelation[]
```
## How was this patch tested?
Added new tests.
Using the following script to benchmark 1, 2 and 3 udfs,
```
df = sqlContext.range(1, 1 << 23, 1, 4)
double = F.udf(lambda x: x * 2, LongType())
print df.select(double(df.id)).count()
print df.select(double(df.id), double(df.id + 1)).count()
print df.select(double(df.id), double(df.id + 1), double(df.id + 2)).count()
```
Here is the results:
N | Before | After | speed up
---- |------------ | -------------|------
1 | 22 s | 7 s | 3.1X
2 | 38 s | 13 s | 2.9X
3 | 58 s | 16 s | 3.6X
This benchmark ran locally with 4 CPUs. For 3 UDFs, it launched 12 Python before before this patch, 4 process after this patch. After this patch, it will use less memory for multiple UDFs than before (less buffering).
Author: Davies Liu <davies@databricks.com>
Closes #12057 from davies/multi_udfs.
2016-03-31 19:40:20 -04:00
|
|
|
num_udfs = read_int(infile)
|
2017-05-10 19:50:57 -04:00
|
|
|
udfs = {}
|
|
|
|
call_udf = []
|
2018-03-08 06:29:07 -05:00
|
|
|
mapper_str = ""
|
|
|
|
if eval_type == PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF:
|
|
|
|
# Create function like this:
|
|
|
|
# lambda a: f([a[0]], [a[0], a[1]])
|
|
|
|
|
|
|
|
# We assume there is only one UDF here because grouped map doesn't
|
|
|
|
# support combining multiple UDFs.
|
|
|
|
assert num_udfs == 1
|
|
|
|
|
|
|
|
# See FlatMapGroupsInPandasExec for how arg_offsets are used to
|
|
|
|
# distinguish between grouping attributes and data attributes
|
[SPARK-24561][SQL][PYTHON] User-defined window aggregation functions with Pandas UDF (bounded window)
## What changes were proposed in this pull request?
This PR implements a new feature - window aggregation Pandas UDF for bounded window.
#### Doc:
https://docs.google.com/document/d/14EjeY5z4-NC27-SmIP9CsMPCANeTcvxN44a7SIJtZPc/edit#heading=h.c87w44wcj3wj
#### Example:
```
from pyspark.sql.functions import pandas_udf, PandasUDFType
from pyspark.sql.window import Window
df = spark.range(0, 10, 2).toDF('v')
w1 = Window.partitionBy().orderBy('v').rangeBetween(-2, 4)
w2 = Window.partitionBy().orderBy('v').rowsBetween(-2, 2)
pandas_udf('double', PandasUDFType.GROUPED_AGG)
def avg(v):
return v.mean()
df.withColumn('v_mean', avg(df['v']).over(w1)).show()
# +---+------+
# | v|v_mean|
# +---+------+
# | 0| 1.0|
# | 2| 2.0|
# | 4| 4.0|
# | 6| 6.0|
# | 8| 7.0|
# +---+------+
df.withColumn('v_mean', avg(df['v']).over(w2)).show()
# +---+------+
# | v|v_mean|
# +---+------+
# | 0| 2.0|
# | 2| 3.0|
# | 4| 4.0|
# | 6| 5.0|
# | 8| 6.0|
# +---+------+
```
#### High level changes:
This PR modifies the existing WindowInPandasExec physical node to deal with unbounded (growing, shrinking and sliding) windows.
* `WindowInPandasExec` now share the same base class as `WindowExec` and share utility functions. See `WindowExecBase`
* `WindowFunctionFrame` now has two new functions `currentLowerBound` and `currentUpperBound` - to return the lower and upper window bound for the current output row. It is also modified to allow `AggregateProcessor` == null. Null aggregator processor is used for `WindowInPandasExec` where we don't have an aggregator and only uses lower and upper bound functions from `WindowFunctionFrame`
* The biggest change is in `WindowInPandasExec`, where it is modified to take `currentLowerBound` and `currentUpperBound` and write those values together with the input data to the python process for rolling window aggregation. See `WindowInPandasExec` for more details.
#### Discussion
In benchmarking, I found numpy variant of the rolling window UDF is much faster than the pandas version:
Spark SQL window function: 20s
Pandas variant: ~80s
Numpy variant: 10s
Numpy variant with numba: 4s
Allowing numpy variant of the vectorized UDFs is something I want to discuss because of the performance improvement, but doesn't have to be in this PR.
## How was this patch tested?
New tests
Closes #22305 from icexelloss/SPARK-24561-bounded-window-udf.
Authored-by: Li Jin <ice.xelloss@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-17 20:15:21 -05:00
|
|
|
arg_offsets, udf = read_single_udf(
|
|
|
|
pickleSer, infile, eval_type, runner_conf, udf_index=0)
|
2018-03-08 06:29:07 -05:00
|
|
|
udfs['f'] = udf
|
|
|
|
split_offset = arg_offsets[0] + 1
|
|
|
|
arg0 = ["a[%d]" % o for o in arg_offsets[1: split_offset]]
|
|
|
|
arg1 = ["a[%d]" % o for o in arg_offsets[split_offset:]]
|
|
|
|
mapper_str = "lambda a: f([%s], [%s])" % (", ".join(arg0), ", ".join(arg1))
|
|
|
|
else:
|
|
|
|
# Create function like this:
|
|
|
|
# lambda a: (f0(a[0]), f1(a[1], a[2]), f2(a[3]))
|
|
|
|
# In the special case of a single UDF this will return a single result rather
|
|
|
|
# than a tuple of results; this is the format that the JVM side expects.
|
|
|
|
for i in range(num_udfs):
|
[SPARK-24561][SQL][PYTHON] User-defined window aggregation functions with Pandas UDF (bounded window)
## What changes were proposed in this pull request?
This PR implements a new feature - window aggregation Pandas UDF for bounded window.
#### Doc:
https://docs.google.com/document/d/14EjeY5z4-NC27-SmIP9CsMPCANeTcvxN44a7SIJtZPc/edit#heading=h.c87w44wcj3wj
#### Example:
```
from pyspark.sql.functions import pandas_udf, PandasUDFType
from pyspark.sql.window import Window
df = spark.range(0, 10, 2).toDF('v')
w1 = Window.partitionBy().orderBy('v').rangeBetween(-2, 4)
w2 = Window.partitionBy().orderBy('v').rowsBetween(-2, 2)
pandas_udf('double', PandasUDFType.GROUPED_AGG)
def avg(v):
return v.mean()
df.withColumn('v_mean', avg(df['v']).over(w1)).show()
# +---+------+
# | v|v_mean|
# +---+------+
# | 0| 1.0|
# | 2| 2.0|
# | 4| 4.0|
# | 6| 6.0|
# | 8| 7.0|
# +---+------+
df.withColumn('v_mean', avg(df['v']).over(w2)).show()
# +---+------+
# | v|v_mean|
# +---+------+
# | 0| 2.0|
# | 2| 3.0|
# | 4| 4.0|
# | 6| 5.0|
# | 8| 6.0|
# +---+------+
```
#### High level changes:
This PR modifies the existing WindowInPandasExec physical node to deal with unbounded (growing, shrinking and sliding) windows.
* `WindowInPandasExec` now share the same base class as `WindowExec` and share utility functions. See `WindowExecBase`
* `WindowFunctionFrame` now has two new functions `currentLowerBound` and `currentUpperBound` - to return the lower and upper window bound for the current output row. It is also modified to allow `AggregateProcessor` == null. Null aggregator processor is used for `WindowInPandasExec` where we don't have an aggregator and only uses lower and upper bound functions from `WindowFunctionFrame`
* The biggest change is in `WindowInPandasExec`, where it is modified to take `currentLowerBound` and `currentUpperBound` and write those values together with the input data to the python process for rolling window aggregation. See `WindowInPandasExec` for more details.
#### Discussion
In benchmarking, I found numpy variant of the rolling window UDF is much faster than the pandas version:
Spark SQL window function: 20s
Pandas variant: ~80s
Numpy variant: 10s
Numpy variant with numba: 4s
Allowing numpy variant of the vectorized UDFs is something I want to discuss because of the performance improvement, but doesn't have to be in this PR.
## How was this patch tested?
New tests
Closes #22305 from icexelloss/SPARK-24561-bounded-window-udf.
Authored-by: Li Jin <ice.xelloss@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-17 20:15:21 -05:00
|
|
|
arg_offsets, udf = read_single_udf(
|
|
|
|
pickleSer, infile, eval_type, runner_conf, udf_index=i)
|
2018-03-08 06:29:07 -05:00
|
|
|
udfs['f%d' % i] = udf
|
|
|
|
args = ["a[%d]" % o for o in arg_offsets]
|
|
|
|
call_udf.append("f%d(%s)" % (i, ", ".join(args)))
|
|
|
|
mapper_str = "lambda a: (%s)" % (", ".join(call_udf))
|
[SPARK-14267] [SQL] [PYSPARK] execute multiple Python UDFs within single batch
## What changes were proposed in this pull request?
This PR support multiple Python UDFs within single batch, also improve the performance.
```python
>>> from pyspark.sql.types import IntegerType
>>> sqlContext.registerFunction("double", lambda x: x * 2, IntegerType())
>>> sqlContext.registerFunction("add", lambda x, y: x + y, IntegerType())
>>> sqlContext.sql("SELECT double(add(1, 2)), add(double(2), 1)").explain(True)
== Parsed Logical Plan ==
'Project [unresolvedalias('double('add(1, 2)), None),unresolvedalias('add('double(2), 1), None)]
+- OneRowRelation$
== Analyzed Logical Plan ==
double(add(1, 2)): int, add(double(2), 1): int
Project [double(add(1, 2))#14,add(double(2), 1)#15]
+- Project [double(add(1, 2))#14,add(double(2), 1)#15]
+- Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15]
+- EvaluatePython [add(pythonUDF1#17, 1)], [pythonUDF0#18]
+- EvaluatePython [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17]
+- OneRowRelation$
== Optimized Logical Plan ==
Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15]
+- EvaluatePython [add(pythonUDF1#17, 1)], [pythonUDF0#18]
+- EvaluatePython [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17]
+- OneRowRelation$
== Physical Plan ==
WholeStageCodegen
: +- Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15]
: +- INPUT
+- !BatchPythonEvaluation [add(pythonUDF1#17, 1)], [pythonUDF0#16,pythonUDF1#17,pythonUDF0#18]
+- !BatchPythonEvaluation [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17]
+- Scan OneRowRelation[]
```
## How was this patch tested?
Added new tests.
Using the following script to benchmark 1, 2 and 3 udfs,
```
df = sqlContext.range(1, 1 << 23, 1, 4)
double = F.udf(lambda x: x * 2, LongType())
print df.select(double(df.id)).count()
print df.select(double(df.id), double(df.id + 1)).count()
print df.select(double(df.id), double(df.id + 1), double(df.id + 2)).count()
```
Here is the results:
N | Before | After | speed up
---- |------------ | -------------|------
1 | 22 s | 7 s | 3.1X
2 | 38 s | 13 s | 2.9X
3 | 58 s | 16 s | 3.6X
This benchmark ran locally with 4 CPUs. For 3 UDFs, it launched 12 Python before before this patch, 4 process after this patch. After this patch, it will use less memory for multiple UDFs than before (less buffering).
Author: Davies Liu <davies@databricks.com>
Closes #12057 from davies/multi_udfs.
2016-03-31 19:40:20 -04:00
|
|
|
|
2018-03-08 06:29:07 -05:00
|
|
|
mapper = eval(mapper_str, udfs)
|
[SPARK-14267] [SQL] [PYSPARK] execute multiple Python UDFs within single batch
## What changes were proposed in this pull request?
This PR support multiple Python UDFs within single batch, also improve the performance.
```python
>>> from pyspark.sql.types import IntegerType
>>> sqlContext.registerFunction("double", lambda x: x * 2, IntegerType())
>>> sqlContext.registerFunction("add", lambda x, y: x + y, IntegerType())
>>> sqlContext.sql("SELECT double(add(1, 2)), add(double(2), 1)").explain(True)
== Parsed Logical Plan ==
'Project [unresolvedalias('double('add(1, 2)), None),unresolvedalias('add('double(2), 1), None)]
+- OneRowRelation$
== Analyzed Logical Plan ==
double(add(1, 2)): int, add(double(2), 1): int
Project [double(add(1, 2))#14,add(double(2), 1)#15]
+- Project [double(add(1, 2))#14,add(double(2), 1)#15]
+- Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15]
+- EvaluatePython [add(pythonUDF1#17, 1)], [pythonUDF0#18]
+- EvaluatePython [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17]
+- OneRowRelation$
== Optimized Logical Plan ==
Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15]
+- EvaluatePython [add(pythonUDF1#17, 1)], [pythonUDF0#18]
+- EvaluatePython [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17]
+- OneRowRelation$
== Physical Plan ==
WholeStageCodegen
: +- Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15]
: +- INPUT
+- !BatchPythonEvaluation [add(pythonUDF1#17, 1)], [pythonUDF0#16,pythonUDF1#17,pythonUDF0#18]
+- !BatchPythonEvaluation [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17]
+- Scan OneRowRelation[]
```
## How was this patch tested?
Added new tests.
Using the following script to benchmark 1, 2 and 3 udfs,
```
df = sqlContext.range(1, 1 << 23, 1, 4)
double = F.udf(lambda x: x * 2, LongType())
print df.select(double(df.id)).count()
print df.select(double(df.id), double(df.id + 1)).count()
print df.select(double(df.id), double(df.id + 1), double(df.id + 2)).count()
```
Here is the results:
N | Before | After | speed up
---- |------------ | -------------|------
1 | 22 s | 7 s | 3.1X
2 | 38 s | 13 s | 2.9X
3 | 58 s | 16 s | 3.6X
This benchmark ran locally with 4 CPUs. For 3 UDFs, it launched 12 Python before before this patch, 4 process after this patch. After this patch, it will use less memory for multiple UDFs than before (less buffering).
Author: Davies Liu <davies@databricks.com>
Closes #12057 from davies/multi_udfs.
2016-03-31 19:40:20 -04:00
|
|
|
func = lambda _, it: map(mapper, it)
|
2017-09-22 04:17:41 -04:00
|
|
|
|
[SPARK-14267] [SQL] [PYSPARK] execute multiple Python UDFs within single batch
## What changes were proposed in this pull request?
This PR support multiple Python UDFs within single batch, also improve the performance.
```python
>>> from pyspark.sql.types import IntegerType
>>> sqlContext.registerFunction("double", lambda x: x * 2, IntegerType())
>>> sqlContext.registerFunction("add", lambda x, y: x + y, IntegerType())
>>> sqlContext.sql("SELECT double(add(1, 2)), add(double(2), 1)").explain(True)
== Parsed Logical Plan ==
'Project [unresolvedalias('double('add(1, 2)), None),unresolvedalias('add('double(2), 1), None)]
+- OneRowRelation$
== Analyzed Logical Plan ==
double(add(1, 2)): int, add(double(2), 1): int
Project [double(add(1, 2))#14,add(double(2), 1)#15]
+- Project [double(add(1, 2))#14,add(double(2), 1)#15]
+- Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15]
+- EvaluatePython [add(pythonUDF1#17, 1)], [pythonUDF0#18]
+- EvaluatePython [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17]
+- OneRowRelation$
== Optimized Logical Plan ==
Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15]
+- EvaluatePython [add(pythonUDF1#17, 1)], [pythonUDF0#18]
+- EvaluatePython [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17]
+- OneRowRelation$
== Physical Plan ==
WholeStageCodegen
: +- Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15]
: +- INPUT
+- !BatchPythonEvaluation [add(pythonUDF1#17, 1)], [pythonUDF0#16,pythonUDF1#17,pythonUDF0#18]
+- !BatchPythonEvaluation [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17]
+- Scan OneRowRelation[]
```
## How was this patch tested?
Added new tests.
Using the following script to benchmark 1, 2 and 3 udfs,
```
df = sqlContext.range(1, 1 << 23, 1, 4)
double = F.udf(lambda x: x * 2, LongType())
print df.select(double(df.id)).count()
print df.select(double(df.id), double(df.id + 1)).count()
print df.select(double(df.id), double(df.id + 1), double(df.id + 2)).count()
```
Here is the results:
N | Before | After | speed up
---- |------------ | -------------|------
1 | 22 s | 7 s | 3.1X
2 | 38 s | 13 s | 2.9X
3 | 58 s | 16 s | 3.6X
This benchmark ran locally with 4 CPUs. For 3 UDFs, it launched 12 Python before before this patch, 4 process after this patch. After this patch, it will use less memory for multiple UDFs than before (less buffering).
Author: Davies Liu <davies@databricks.com>
Closes #12057 from davies/multi_udfs.
2016-03-31 19:40:20 -04:00
|
|
|
# profiling is not supported for UDF
|
|
|
|
return func, None, ser, ser
|
[SPARK-14215] [SQL] [PYSPARK] Support chained Python UDFs
## What changes were proposed in this pull request?
This PR brings the support for chained Python UDFs, for example
```sql
select udf1(udf2(a))
select udf1(udf2(a) + 3)
select udf1(udf2(a) + udf3(b))
```
Also directly chained unary Python UDFs are put in single batch of Python UDFs, others may require multiple batches.
For example,
```python
>>> sqlContext.sql("select double(double(1))").explain()
== Physical Plan ==
WholeStageCodegen
: +- Project [pythonUDF#10 AS double(double(1))#9]
: +- INPUT
+- !BatchPythonEvaluation double(double(1)), [pythonUDF#10]
+- Scan OneRowRelation[]
>>> sqlContext.sql("select double(double(1) + double(2))").explain()
== Physical Plan ==
WholeStageCodegen
: +- Project [pythonUDF#19 AS double((double(1) + double(2)))#16]
: +- INPUT
+- !BatchPythonEvaluation double((pythonUDF#17 + pythonUDF#18)), [pythonUDF#17,pythonUDF#18,pythonUDF#19]
+- !BatchPythonEvaluation double(2), [pythonUDF#17,pythonUDF#18]
+- !BatchPythonEvaluation double(1), [pythonUDF#17]
+- Scan OneRowRelation[]
```
TODO: will support multiple unrelated Python UDFs in one batch (another PR).
## How was this patch tested?
Added new unit tests for chained UDFs.
Author: Davies Liu <davies@databricks.com>
Closes #12014 from davies/py_udfs.
2016-03-29 18:06:29 -04:00
|
|
|
|
|
|
|
|
2013-05-06 19:34:30 -04:00
|
|
|
def main(infile, outfile):
|
2014-02-26 17:50:37 -05:00
|
|
|
try:
|
|
|
|
boot_time = time.time()
|
|
|
|
split_index = read_int(infile)
|
|
|
|
if split_index == -1: # for unit tests
|
2018-03-08 06:38:34 -05:00
|
|
|
sys.exit(-1)
|
2013-08-15 19:01:19 -04:00
|
|
|
|
2015-05-18 15:55:13 -04:00
|
|
|
version = utf8_deserializer.loads(infile)
|
|
|
|
if version != "%d.%d" % sys.version_info[:2]:
|
|
|
|
raise Exception(("Python in worker has different version %s than that in " +
|
2016-11-10 05:23:45 -05:00
|
|
|
"driver %s, PySpark cannot run with different minor versions." +
|
|
|
|
"Please check environment variables PYSPARK_PYTHON and " +
|
|
|
|
"PYSPARK_DRIVER_PYTHON are correctly set.") %
|
2015-05-18 15:55:13 -04:00
|
|
|
("%d.%d" % sys.version_info[:2], version))
|
|
|
|
|
2018-08-21 18:54:30 -04:00
|
|
|
# read inputs only for a barrier task
|
|
|
|
isBarrier = read_bool(infile)
|
|
|
|
boundPort = read_int(infile)
|
|
|
|
secret = UTF8Deserializer().loads(infile)
|
[SPARK-25004][CORE] Add spark.executor.pyspark.memory limit.
## What changes were proposed in this pull request?
This adds `spark.executor.pyspark.memory` to configure Python's address space limit, [`resource.RLIMIT_AS`](https://docs.python.org/3/library/resource.html#resource.RLIMIT_AS). Limiting Python's address space allows Python to participate in memory management. In practice, we see fewer cases of Python taking too much memory because it doesn't know to run garbage collection. This results in YARN killing fewer containers. This also improves error messages so users know that Python is consuming too much memory:
```
File "build/bdist.linux-x86_64/egg/package/library.py", line 265, in fe_engineer
fe_eval_rec.update(f(src_rec_prep, mat_rec_prep))
File "build/bdist.linux-x86_64/egg/package/library.py", line 163, in fe_comp
comparisons = EvaluationUtils.leven_list_compare(src_rec_prep.get(item, []), mat_rec_prep.get(item, []))
File "build/bdist.linux-x86_64/egg/package/evaluationutils.py", line 25, in leven_list_compare
permutations = sorted(permutations, reverse=True)
MemoryError
```
The new pyspark memory setting is used to increase requested YARN container memory, instead of sharing overhead memory between python and off-heap JVM activity.
## How was this patch tested?
Tested memory limits in our YARN cluster and verified that MemoryError is thrown.
Author: Ryan Blue <blue@apache.org>
Closes #21977 from rdblue/SPARK-25004-add-python-memory-limit.
2018-08-28 15:31:33 -04:00
|
|
|
|
|
|
|
# set up memory limits
|
|
|
|
memory_limit_mb = int(os.environ.get('PYSPARK_EXECUTOR_MEMORY_MB', "-1"))
|
2018-12-02 04:41:08 -05:00
|
|
|
if memory_limit_mb > 0 and has_resource_module:
|
|
|
|
total_memory = resource.RLIMIT_AS
|
|
|
|
try:
|
[SPARK-25004][CORE] Add spark.executor.pyspark.memory limit.
## What changes were proposed in this pull request?
This adds `spark.executor.pyspark.memory` to configure Python's address space limit, [`resource.RLIMIT_AS`](https://docs.python.org/3/library/resource.html#resource.RLIMIT_AS). Limiting Python's address space allows Python to participate in memory management. In practice, we see fewer cases of Python taking too much memory because it doesn't know to run garbage collection. This results in YARN killing fewer containers. This also improves error messages so users know that Python is consuming too much memory:
```
File "build/bdist.linux-x86_64/egg/package/library.py", line 265, in fe_engineer
fe_eval_rec.update(f(src_rec_prep, mat_rec_prep))
File "build/bdist.linux-x86_64/egg/package/library.py", line 163, in fe_comp
comparisons = EvaluationUtils.leven_list_compare(src_rec_prep.get(item, []), mat_rec_prep.get(item, []))
File "build/bdist.linux-x86_64/egg/package/evaluationutils.py", line 25, in leven_list_compare
permutations = sorted(permutations, reverse=True)
MemoryError
```
The new pyspark memory setting is used to increase requested YARN container memory, instead of sharing overhead memory between python and off-heap JVM activity.
## How was this patch tested?
Tested memory limits in our YARN cluster and verified that MemoryError is thrown.
Author: Ryan Blue <blue@apache.org>
Closes #21977 from rdblue/SPARK-25004-add-python-memory-limit.
2018-08-28 15:31:33 -04:00
|
|
|
(soft_limit, hard_limit) = resource.getrlimit(total_memory)
|
|
|
|
msg = "Current mem limits: {0} of max {1}\n".format(soft_limit, hard_limit)
|
|
|
|
print(msg, file=sys.stderr)
|
|
|
|
|
|
|
|
# convert to bytes
|
|
|
|
new_limit = memory_limit_mb * 1024 * 1024
|
|
|
|
|
|
|
|
if soft_limit == resource.RLIM_INFINITY or new_limit < soft_limit:
|
|
|
|
msg = "Setting mem limits to {0} of max {1}\n".format(new_limit, new_limit)
|
|
|
|
print(msg, file=sys.stderr)
|
|
|
|
resource.setrlimit(total_memory, (new_limit, new_limit))
|
|
|
|
|
2018-12-02 04:41:08 -05:00
|
|
|
except (resource.error, OSError, ValueError) as e:
|
|
|
|
# not all systems support resource limits, so warn instead of failing
|
|
|
|
print("WARN: Failed to set memory limit: {0}\n".format(e), file=sys.stderr)
|
[SPARK-25004][CORE] Add spark.executor.pyspark.memory limit.
## What changes were proposed in this pull request?
This adds `spark.executor.pyspark.memory` to configure Python's address space limit, [`resource.RLIMIT_AS`](https://docs.python.org/3/library/resource.html#resource.RLIMIT_AS). Limiting Python's address space allows Python to participate in memory management. In practice, we see fewer cases of Python taking too much memory because it doesn't know to run garbage collection. This results in YARN killing fewer containers. This also improves error messages so users know that Python is consuming too much memory:
```
File "build/bdist.linux-x86_64/egg/package/library.py", line 265, in fe_engineer
fe_eval_rec.update(f(src_rec_prep, mat_rec_prep))
File "build/bdist.linux-x86_64/egg/package/library.py", line 163, in fe_comp
comparisons = EvaluationUtils.leven_list_compare(src_rec_prep.get(item, []), mat_rec_prep.get(item, []))
File "build/bdist.linux-x86_64/egg/package/evaluationutils.py", line 25, in leven_list_compare
permutations = sorted(permutations, reverse=True)
MemoryError
```
The new pyspark memory setting is used to increase requested YARN container memory, instead of sharing overhead memory between python and off-heap JVM activity.
## How was this patch tested?
Tested memory limits in our YARN cluster and verified that MemoryError is thrown.
Author: Ryan Blue <blue@apache.org>
Closes #21977 from rdblue/SPARK-25004-add-python-memory-limit.
2018-08-28 15:31:33 -04:00
|
|
|
|
2014-09-14 01:31:21 -04:00
|
|
|
# initialize global state
|
2018-08-21 18:54:30 -04:00
|
|
|
taskContext = None
|
|
|
|
if isBarrier:
|
|
|
|
taskContext = BarrierTaskContext._getOrCreate()
|
|
|
|
BarrierTaskContext._initialize(boundPort, secret)
|
|
|
|
else:
|
|
|
|
taskContext = TaskContext._getOrCreate()
|
|
|
|
# read inputs for TaskContext info
|
2016-12-20 18:51:21 -05:00
|
|
|
taskContext._stageId = read_int(infile)
|
|
|
|
taskContext._partitionId = read_int(infile)
|
|
|
|
taskContext._attemptNumber = read_int(infile)
|
|
|
|
taskContext._taskAttemptId = read_long(infile)
|
2018-05-31 14:23:57 -04:00
|
|
|
taskContext._localProperties = dict()
|
|
|
|
for i in range(read_int(infile)):
|
|
|
|
k = utf8_deserializer.loads(infile)
|
|
|
|
v = utf8_deserializer.loads(infile)
|
|
|
|
taskContext._localProperties[k] = v
|
|
|
|
|
2014-09-14 01:31:21 -04:00
|
|
|
shuffle.MemoryBytesSpilled = 0
|
|
|
|
shuffle.DiskBytesSpilled = 0
|
|
|
|
_accumulatorRegistry.clear()
|
|
|
|
|
2014-02-26 17:50:37 -05:00
|
|
|
# fetch name of workdir
|
|
|
|
spark_files_dir = utf8_deserializer.loads(infile)
|
|
|
|
SparkFiles._root_directory = spark_files_dir
|
|
|
|
SparkFiles._is_running_on_worker = True
|
2013-08-15 19:01:19 -04:00
|
|
|
|
2014-02-26 17:50:37 -05:00
|
|
|
# fetch names of includes (*.zip and *.egg files) and construct PYTHONPATH
|
2014-09-24 15:10:09 -04:00
|
|
|
add_path(spark_files_dir) # *.py files that were added will be copied here
|
2014-07-22 01:30:53 -04:00
|
|
|
num_python_includes = read_int(infile)
|
2014-02-26 17:50:37 -05:00
|
|
|
for _ in range(num_python_includes):
|
|
|
|
filename = utf8_deserializer.loads(infile)
|
2014-09-24 15:10:09 -04:00
|
|
|
add_path(os.path.join(spark_files_dir, filename))
|
2015-04-16 19:20:57 -04:00
|
|
|
if sys.version > '3':
|
|
|
|
import importlib
|
|
|
|
importlib.invalidate_caches()
|
2013-08-15 19:01:19 -04:00
|
|
|
|
2014-05-10 16:02:13 -04:00
|
|
|
# fetch names and values of broadcast variables
|
2018-08-13 22:35:34 -04:00
|
|
|
needs_broadcast_decryption_server = read_bool(infile)
|
2014-05-10 16:02:13 -04:00
|
|
|
num_broadcast_variables = read_int(infile)
|
2018-08-13 22:35:34 -04:00
|
|
|
if needs_broadcast_decryption_server:
|
|
|
|
# read the decrypted data from a server in the jvm
|
|
|
|
port = read_int(infile)
|
|
|
|
auth_secret = utf8_deserializer.loads(infile)
|
|
|
|
(broadcast_sock_file, _) = local_connect_and_auth(port, auth_secret)
|
|
|
|
|
2014-05-10 16:02:13 -04:00
|
|
|
for _ in range(num_broadcast_variables):
|
|
|
|
bid = read_long(infile)
|
2014-09-13 19:22:04 -04:00
|
|
|
if bid >= 0:
|
2018-08-13 22:35:34 -04:00
|
|
|
if needs_broadcast_decryption_server:
|
|
|
|
read_bid = read_long(broadcast_sock_file)
|
|
|
|
assert(read_bid == bid)
|
|
|
|
_broadcastRegistry[bid] = \
|
|
|
|
Broadcast(sock_file=broadcast_sock_file)
|
|
|
|
else:
|
|
|
|
path = utf8_deserializer.loads(infile)
|
|
|
|
_broadcastRegistry[bid] = Broadcast(path=path)
|
|
|
|
|
2014-09-13 19:22:04 -04:00
|
|
|
else:
|
|
|
|
bid = - bid - 1
|
2014-09-18 21:11:48 -04:00
|
|
|
_broadcastRegistry.pop(bid)
|
2014-05-10 16:02:13 -04:00
|
|
|
|
2018-08-13 22:35:34 -04:00
|
|
|
if needs_broadcast_decryption_server:
|
|
|
|
broadcast_sock_file.write(b'1')
|
|
|
|
broadcast_sock_file.close()
|
|
|
|
|
2014-09-13 19:22:04 -04:00
|
|
|
_accumulatorRegistry.clear()
|
2017-09-22 04:17:41 -04:00
|
|
|
eval_type = read_int(infile)
|
|
|
|
if eval_type == PythonEvalType.NON_UDF:
|
[SPARK-14215] [SQL] [PYSPARK] Support chained Python UDFs
## What changes were proposed in this pull request?
This PR brings the support for chained Python UDFs, for example
```sql
select udf1(udf2(a))
select udf1(udf2(a) + 3)
select udf1(udf2(a) + udf3(b))
```
Also directly chained unary Python UDFs are put in single batch of Python UDFs, others may require multiple batches.
For example,
```python
>>> sqlContext.sql("select double(double(1))").explain()
== Physical Plan ==
WholeStageCodegen
: +- Project [pythonUDF#10 AS double(double(1))#9]
: +- INPUT
+- !BatchPythonEvaluation double(double(1)), [pythonUDF#10]
+- Scan OneRowRelation[]
>>> sqlContext.sql("select double(double(1) + double(2))").explain()
== Physical Plan ==
WholeStageCodegen
: +- Project [pythonUDF#19 AS double((double(1) + double(2)))#16]
: +- INPUT
+- !BatchPythonEvaluation double((pythonUDF#17 + pythonUDF#18)), [pythonUDF#17,pythonUDF#18,pythonUDF#19]
+- !BatchPythonEvaluation double(2), [pythonUDF#17,pythonUDF#18]
+- !BatchPythonEvaluation double(1), [pythonUDF#17]
+- Scan OneRowRelation[]
```
TODO: will support multiple unrelated Python UDFs in one batch (another PR).
## How was this patch tested?
Added new unit tests for chained UDFs.
Author: Davies Liu <davies@databricks.com>
Closes #12014 from davies/py_udfs.
2016-03-29 18:06:29 -04:00
|
|
|
func, profiler, deserializer, serializer = read_command(pickleSer, infile)
|
2017-09-22 04:17:41 -04:00
|
|
|
else:
|
|
|
|
func, profiler, deserializer, serializer = read_udfs(pickleSer, infile, eval_type)
|
[SPARK-14215] [SQL] [PYSPARK] Support chained Python UDFs
## What changes were proposed in this pull request?
This PR brings the support for chained Python UDFs, for example
```sql
select udf1(udf2(a))
select udf1(udf2(a) + 3)
select udf1(udf2(a) + udf3(b))
```
Also directly chained unary Python UDFs are put in single batch of Python UDFs, others may require multiple batches.
For example,
```python
>>> sqlContext.sql("select double(double(1))").explain()
== Physical Plan ==
WholeStageCodegen
: +- Project [pythonUDF#10 AS double(double(1))#9]
: +- INPUT
+- !BatchPythonEvaluation double(double(1)), [pythonUDF#10]
+- Scan OneRowRelation[]
>>> sqlContext.sql("select double(double(1) + double(2))").explain()
== Physical Plan ==
WholeStageCodegen
: +- Project [pythonUDF#19 AS double((double(1) + double(2)))#16]
: +- INPUT
+- !BatchPythonEvaluation double((pythonUDF#17 + pythonUDF#18)), [pythonUDF#17,pythonUDF#18,pythonUDF#19]
+- !BatchPythonEvaluation double(2), [pythonUDF#17,pythonUDF#18]
+- !BatchPythonEvaluation double(1), [pythonUDF#17]
+- Scan OneRowRelation[]
```
TODO: will support multiple unrelated Python UDFs in one batch (another PR).
## How was this patch tested?
Added new unit tests for chained UDFs.
Author: Davies Liu <davies@databricks.com>
Closes #12014 from davies/py_udfs.
2016-03-29 18:06:29 -04:00
|
|
|
|
2014-02-26 17:50:37 -05:00
|
|
|
init_time = time.time()
|
[SPARK-3478] [PySpark] Profile the Python tasks
This patch add profiling support for PySpark, it will show the profiling results
before the driver exits, here is one example:
```
============================================================
Profile of RDD<id=3>
============================================================
5146507 function calls (5146487 primitive calls) in 71.094 seconds
Ordered by: internal time, cumulative time
ncalls tottime percall cumtime percall filename:lineno(function)
5144576 68.331 0.000 68.331 0.000 statcounter.py:44(merge)
20 2.735 0.137 71.071 3.554 statcounter.py:33(__init__)
20 0.017 0.001 0.017 0.001 {cPickle.dumps}
1024 0.003 0.000 0.003 0.000 t.py:16(<lambda>)
20 0.001 0.000 0.001 0.000 {reduce}
21 0.001 0.000 0.001 0.000 {cPickle.loads}
20 0.001 0.000 0.001 0.000 copy_reg.py:95(_slotnames)
41 0.001 0.000 0.001 0.000 serializers.py:461(read_int)
40 0.001 0.000 0.002 0.000 serializers.py:179(_batched)
62 0.000 0.000 0.000 0.000 {method 'read' of 'file' objects}
20 0.000 0.000 71.072 3.554 rdd.py:863(<lambda>)
20 0.000 0.000 0.001 0.000 serializers.py:198(load_stream)
40/20 0.000 0.000 71.072 3.554 rdd.py:2093(pipeline_func)
41 0.000 0.000 0.002 0.000 serializers.py:130(load_stream)
40 0.000 0.000 71.072 1.777 rdd.py:304(func)
20 0.000 0.000 71.094 3.555 worker.py:82(process)
```
Also, use can show profile result manually by `sc.show_profiles()` or dump it into disk
by `sc.dump_profiles(path)`, such as
```python
>>> sc._conf.set("spark.python.profile", "true")
>>> rdd = sc.parallelize(range(100)).map(str)
>>> rdd.count()
100
>>> sc.show_profiles()
============================================================
Profile of RDD<id=1>
============================================================
284 function calls (276 primitive calls) in 0.001 seconds
Ordered by: internal time, cumulative time
ncalls tottime percall cumtime percall filename:lineno(function)
4 0.000 0.000 0.000 0.000 serializers.py:198(load_stream)
4 0.000 0.000 0.000 0.000 {reduce}
12/4 0.000 0.000 0.001 0.000 rdd.py:2092(pipeline_func)
4 0.000 0.000 0.000 0.000 {cPickle.loads}
4 0.000 0.000 0.000 0.000 {cPickle.dumps}
104 0.000 0.000 0.000 0.000 rdd.py:852(<genexpr>)
8 0.000 0.000 0.000 0.000 serializers.py:461(read_int)
12 0.000 0.000 0.000 0.000 rdd.py:303(func)
```
The profiling is disabled by default, can be enabled by "spark.python.profile=true".
Also, users can dump the results into disks automatically for future analysis, by "spark.python.profile.dump=path_to_dump"
This is bugfix of #2351 cc JoshRosen
Author: Davies Liu <davies.liu@gmail.com>
Closes #2556 from davies/profiler and squashes the following commits:
e68df5a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
858e74c [Davies Liu] compatitable with python 2.6
7ef2aa0 [Davies Liu] bugfix, add tests for show_profiles and dump_profiles()
2b0daf2 [Davies Liu] fix docs
7a56c24 [Davies Liu] bugfix
cba9463 [Davies Liu] move show_profiles and dump_profiles to SparkContext
fb9565b [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
116d52a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
09d02c3 [Davies Liu] Merge branch 'master' into profiler
c23865c [Davies Liu] Merge branch 'master' into profiler
15d6f18 [Davies Liu] add docs for two configs
dadee1a [Davies Liu] add docs string and clear profiles after show or dump
4f8309d [Davies Liu] address comment, add tests
0a5b6eb [Davies Liu] fix Python UDF
4b20494 [Davies Liu] add profile for python
2014-09-30 21:24:57 -04:00
|
|
|
|
|
|
|
def process():
|
|
|
|
iterator = deserializer.load_stream(infile)
|
|
|
|
serializer.dump_stream(func(split_index, iterator), outfile)
|
|
|
|
|
2015-01-28 16:48:06 -05:00
|
|
|
if profiler:
|
|
|
|
profiler.profile(process)
|
[SPARK-3478] [PySpark] Profile the Python tasks
This patch add profiling support for PySpark, it will show the profiling results
before the driver exits, here is one example:
```
============================================================
Profile of RDD<id=3>
============================================================
5146507 function calls (5146487 primitive calls) in 71.094 seconds
Ordered by: internal time, cumulative time
ncalls tottime percall cumtime percall filename:lineno(function)
5144576 68.331 0.000 68.331 0.000 statcounter.py:44(merge)
20 2.735 0.137 71.071 3.554 statcounter.py:33(__init__)
20 0.017 0.001 0.017 0.001 {cPickle.dumps}
1024 0.003 0.000 0.003 0.000 t.py:16(<lambda>)
20 0.001 0.000 0.001 0.000 {reduce}
21 0.001 0.000 0.001 0.000 {cPickle.loads}
20 0.001 0.000 0.001 0.000 copy_reg.py:95(_slotnames)
41 0.001 0.000 0.001 0.000 serializers.py:461(read_int)
40 0.001 0.000 0.002 0.000 serializers.py:179(_batched)
62 0.000 0.000 0.000 0.000 {method 'read' of 'file' objects}
20 0.000 0.000 71.072 3.554 rdd.py:863(<lambda>)
20 0.000 0.000 0.001 0.000 serializers.py:198(load_stream)
40/20 0.000 0.000 71.072 3.554 rdd.py:2093(pipeline_func)
41 0.000 0.000 0.002 0.000 serializers.py:130(load_stream)
40 0.000 0.000 71.072 1.777 rdd.py:304(func)
20 0.000 0.000 71.094 3.555 worker.py:82(process)
```
Also, use can show profile result manually by `sc.show_profiles()` or dump it into disk
by `sc.dump_profiles(path)`, such as
```python
>>> sc._conf.set("spark.python.profile", "true")
>>> rdd = sc.parallelize(range(100)).map(str)
>>> rdd.count()
100
>>> sc.show_profiles()
============================================================
Profile of RDD<id=1>
============================================================
284 function calls (276 primitive calls) in 0.001 seconds
Ordered by: internal time, cumulative time
ncalls tottime percall cumtime percall filename:lineno(function)
4 0.000 0.000 0.000 0.000 serializers.py:198(load_stream)
4 0.000 0.000 0.000 0.000 {reduce}
12/4 0.000 0.000 0.001 0.000 rdd.py:2092(pipeline_func)
4 0.000 0.000 0.000 0.000 {cPickle.loads}
4 0.000 0.000 0.000 0.000 {cPickle.dumps}
104 0.000 0.000 0.000 0.000 rdd.py:852(<genexpr>)
8 0.000 0.000 0.000 0.000 serializers.py:461(read_int)
12 0.000 0.000 0.000 0.000 rdd.py:303(func)
```
The profiling is disabled by default, can be enabled by "spark.python.profile=true".
Also, users can dump the results into disks automatically for future analysis, by "spark.python.profile.dump=path_to_dump"
This is bugfix of #2351 cc JoshRosen
Author: Davies Liu <davies.liu@gmail.com>
Closes #2556 from davies/profiler and squashes the following commits:
e68df5a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
858e74c [Davies Liu] compatitable with python 2.6
7ef2aa0 [Davies Liu] bugfix, add tests for show_profiles and dump_profiles()
2b0daf2 [Davies Liu] fix docs
7a56c24 [Davies Liu] bugfix
cba9463 [Davies Liu] move show_profiles and dump_profiles to SparkContext
fb9565b [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
116d52a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
09d02c3 [Davies Liu] Merge branch 'master' into profiler
c23865c [Davies Liu] Merge branch 'master' into profiler
15d6f18 [Davies Liu] add docs for two configs
dadee1a [Davies Liu] add docs string and clear profiles after show or dump
4f8309d [Davies Liu] address comment, add tests
0a5b6eb [Davies Liu] fix Python UDF
4b20494 [Davies Liu] add profile for python
2014-09-30 21:24:57 -04:00
|
|
|
else:
|
|
|
|
process()
|
2014-07-29 03:15:45 -04:00
|
|
|
except Exception:
|
|
|
|
try:
|
|
|
|
write_int(SpecialLengths.PYTHON_EXCEPTION_THROWN, outfile)
|
2015-04-16 19:20:57 -04:00
|
|
|
write_with_length(traceback.format_exc().encode("utf-8"), outfile)
|
2014-07-29 03:15:45 -04:00
|
|
|
except IOError:
|
|
|
|
# JVM close the socket
|
|
|
|
pass
|
|
|
|
except Exception:
|
|
|
|
# Write the error to stderr if it happened while serializing
|
2015-04-16 19:20:57 -04:00
|
|
|
print("PySpark worker failed with exception:", file=sys.stderr)
|
|
|
|
print(traceback.format_exc(), file=sys.stderr)
|
2018-03-08 06:38:34 -05:00
|
|
|
sys.exit(-1)
|
2013-03-10 16:54:46 -04:00
|
|
|
finish_time = time.time()
|
2013-05-06 19:34:30 -04:00
|
|
|
report_times(outfile, boot_time, init_time, finish_time)
|
2014-09-14 01:31:21 -04:00
|
|
|
write_long(shuffle.MemoryBytesSpilled, outfile)
|
|
|
|
write_long(shuffle.DiskBytesSpilled, outfile)
|
|
|
|
|
2013-01-20 04:57:44 -05:00
|
|
|
# Mark the beginning of the accumulators section of the output
|
2013-11-03 00:13:18 -04:00
|
|
|
write_int(SpecialLengths.END_OF_DATA_SECTION, outfile)
|
|
|
|
write_int(len(_accumulatorRegistry), outfile)
|
|
|
|
for (aid, accum) in _accumulatorRegistry.items():
|
2013-11-05 20:52:39 -05:00
|
|
|
pickleSer._write_with_length((aid, accum._value), outfile)
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2014-10-23 20:20:00 -04:00
|
|
|
# check end of stream
|
|
|
|
if read_int(infile) == SpecialLengths.END_OF_STREAM:
|
|
|
|
write_int(SpecialLengths.END_OF_STREAM, outfile)
|
|
|
|
else:
|
|
|
|
# write a different value to tell JVM to not reuse this worker
|
|
|
|
write_int(SpecialLengths.END_OF_DATA_SECTION, outfile)
|
2018-03-08 06:38:34 -05:00
|
|
|
sys.exit(-1)
|
2014-10-23 20:20:00 -04:00
|
|
|
|
2012-08-10 04:10:02 -04:00
|
|
|
|
|
|
|
if __name__ == '__main__':
|
2018-04-13 17:28:24 -04:00
|
|
|
# Read information about how to connect back to the JVM from the environment.
|
|
|
|
java_port = int(os.environ["PYTHON_WORKER_FACTORY_PORT"])
|
|
|
|
auth_secret = os.environ["PYTHON_WORKER_FACTORY_SECRET"]
|
2018-08-28 21:47:38 -04:00
|
|
|
(sock_file, _) = local_connect_and_auth(java_port, auth_secret)
|
2013-09-01 21:06:15 -04:00
|
|
|
main(sock_file, sock_file)
|