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-12-27 22:59:04 -05:00
|
|
|
import copy
|
2013-08-23 14:16:44 -04:00
|
|
|
import sys
|
2015-04-16 19:20:57 -04:00
|
|
|
import os
|
|
|
|
import operator
|
2012-08-25 21:00:25 -04:00
|
|
|
import shlex
|
2014-01-23 23:01:36 -05:00
|
|
|
import warnings
|
2014-04-03 18:42:17 -04:00
|
|
|
import heapq
|
2014-08-13 17:57:12 -04:00
|
|
|
import bisect
|
[SPARK-4327] [PySpark] Python API for RDD.randomSplit()
```
pyspark.RDD.randomSplit(self, weights, seed=None)
Randomly splits this RDD with the provided weights.
:param weights: weights for splits, will be normalized if they don't sum to 1
:param seed: random seed
:return: split RDDs in an list
>>> rdd = sc.parallelize(range(10), 1)
>>> rdd1, rdd2, rdd3 = rdd.randomSplit([0.4, 0.6, 1.0], 11)
>>> rdd1.collect()
[3, 6]
>>> rdd2.collect()
[0, 5, 7]
>>> rdd3.collect()
[1, 2, 4, 8, 9]
```
Author: Davies Liu <davies@databricks.com>
Closes #3193 from davies/randomSplit and squashes the following commits:
78bf997 [Davies Liu] fix tests, do not use numpy in randomSplit, no performance gain
f5fdf63 [Davies Liu] fix bug with int in weights
4dfa2cd [Davies Liu] refactor
f866bcf [Davies Liu] remove unneeded change
c7a2007 [Davies Liu] switch to python implementation
95a48ac [Davies Liu] Merge branch 'master' of github.com:apache/spark into randomSplit
0d9b256 [Davies Liu] refactor
1715ee3 [Davies Liu] address comments
41fce54 [Davies Liu] randomSplit()
2014-11-18 19:37:35 -05:00
|
|
|
import random
|
2015-04-16 19:20:57 -04:00
|
|
|
from subprocess import Popen, PIPE
|
|
|
|
from threading import Thread
|
|
|
|
from collections import defaultdict
|
|
|
|
from itertools import chain
|
|
|
|
from functools import reduce
|
2015-01-28 20:26:03 -05:00
|
|
|
from math import sqrt, log, isinf, isnan, pow, ceil
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2018-08-28 21:47:38 -04:00
|
|
|
from pyspark.java_gateway import local_connect_and_auth
|
[SPARK-23961][SPARK-27548][PYTHON] Fix error when toLocalIterator goes out of scope and properly raise errors from worker
## What changes were proposed in this pull request?
This fixes an error when a PySpark local iterator, for both RDD and DataFrames, goes out of scope and the connection is closed before fully consuming the iterator. The error occurs on the JVM in the serving thread, when Python closes the local socket while the JVM is writing to it. This usually happens when there is enough data to fill the socket read buffer, causing the write call to block.
Additionally, this fixes a problem when an error occurs in the Python worker and the collect job is cancelled with an exception. Previously, the Python driver was never notified of the error so the user could get a partial result (iteration until the error) and the application will continue. With this change, an error in the worker is sent to the Python iterator and is then raised.
The change here introduces a protocol for PySpark local iterators that work as follows:
1) The local socket connection is made when the iterator is created
2) When iterating, Python first sends a request for partition data as a non-zero integer
3) While the JVM local iterator over partitions has next, it triggers a job to collect the next partition
4) The JVM sends a nonzero response to indicate it has the next partition to send
5) The next partition is sent to Python and read by the PySpark deserializer
6) After sending the entire partition, an `END_OF_DATA_SECTION` is sent to Python which stops the deserializer and allows to make another request
7) When the JVM gets a request from Python but has already consumed it's local iterator, it will send a zero response to Python and both will close the socket cleanly
8) If an error occurs in the worker, a negative response is sent to Python followed by the error message. Python will then raise a RuntimeError with the message, stopping iteration.
9) When the PySpark local iterator is garbage-collected, it will read any remaining data from the current partition (this is data that has already been collected) and send a request of zero to tell the JVM to stop collection jobs and close the connection.
Steps 1, 3, 5, 6 are the same as before. Step 8 was completely missing before because errors in the worker were never communicated back to Python. The other steps add synchronization to allow for a clean closing of the socket, with a small trade-off in performance for each partition. This is mainly because the JVM does not start collecting partition data until it receives a request to do so, where before it would eagerly write all data until the socket receive buffer is full.
## How was this patch tested?
Added new unit tests for DataFrame and RDD `toLocalIterator` and tested not fully consuming the iterator. Manual tests with Python 2.7 and 3.6.
Closes #24070 from BryanCutler/pyspark-toLocalIterator-clean-stop-SPARK-23961.
Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-05-07 17:47:39 -04:00
|
|
|
from pyspark.serializers import AutoBatchedSerializer, BatchedSerializer, NoOpSerializer, \
|
|
|
|
CartesianDeserializer, CloudPickleSerializer, PairDeserializer, PickleSerializer, \
|
2020-07-13 22:22:44 -04:00
|
|
|
pack_long, read_int, write_int
|
2012-08-10 04:10:02 -04:00
|
|
|
from pyspark.join import python_join, python_left_outer_join, \
|
2014-09-24 23:39:09 -04:00
|
|
|
python_right_outer_join, python_full_outer_join, python_cogroup
|
2013-08-20 16:22:06 -04:00
|
|
|
from pyspark.statcounter import StatCounter
|
[SPARK-4327] [PySpark] Python API for RDD.randomSplit()
```
pyspark.RDD.randomSplit(self, weights, seed=None)
Randomly splits this RDD with the provided weights.
:param weights: weights for splits, will be normalized if they don't sum to 1
:param seed: random seed
:return: split RDDs in an list
>>> rdd = sc.parallelize(range(10), 1)
>>> rdd1, rdd2, rdd3 = rdd.randomSplit([0.4, 0.6, 1.0], 11)
>>> rdd1.collect()
[3, 6]
>>> rdd2.collect()
[0, 5, 7]
>>> rdd3.collect()
[1, 2, 4, 8, 9]
```
Author: Davies Liu <davies@databricks.com>
Closes #3193 from davies/randomSplit and squashes the following commits:
78bf997 [Davies Liu] fix tests, do not use numpy in randomSplit, no performance gain
f5fdf63 [Davies Liu] fix bug with int in weights
4dfa2cd [Davies Liu] refactor
f866bcf [Davies Liu] remove unneeded change
c7a2007 [Davies Liu] switch to python implementation
95a48ac [Davies Liu] Merge branch 'master' of github.com:apache/spark into randomSplit
0d9b256 [Davies Liu] refactor
1715ee3 [Davies Liu] address comments
41fce54 [Davies Liu] randomSplit()
2014-11-18 19:37:35 -05:00
|
|
|
from pyspark.rddsampler import RDDSampler, RDDRangeSampler, RDDStratifiedSampler
|
2014-03-12 02:57:05 -04:00
|
|
|
from pyspark.storagelevel import StorageLevel
|
2020-05-19 20:09:37 -04:00
|
|
|
from pyspark.resource.requests import ExecutorResourceRequests, TaskResourceRequests
|
|
|
|
from pyspark.resource.profile import ResourceProfile
|
2014-04-08 21:15:52 -04:00
|
|
|
from pyspark.resultiterable import ResultIterable
|
2015-09-20 00:40:21 -04:00
|
|
|
from pyspark.shuffle import Aggregator, ExternalMerger, \
|
[SPARK-3074] [PySpark] support groupByKey() with single huge key
This patch change groupByKey() to use external sort based approach, so it can support single huge key.
For example, it can group by a dataset including one hot key with 40 millions values (strings), using 500M memory for Python worker, finished in about 2 minutes. (it will need 6G memory in hash based approach).
During groupByKey(), it will do in-memory groupBy first. If the dataset can not fit in memory, then data will be partitioned by hash. If one partition still can not fit in memory, it will switch to sort based groupBy().
Author: Davies Liu <davies.liu@gmail.com>
Author: Davies Liu <davies@databricks.com>
Closes #1977 from davies/groupby and squashes the following commits:
af3713a [Davies Liu] make sure it's iterator
67772dd [Davies Liu] fix tests
e78c15c [Davies Liu] address comments
0b0fde8 [Davies Liu] address comments
0dcf320 [Davies Liu] address comments, rollback changes in ResultIterable
e3b8eab [Davies Liu] fix narrow dependency
2a1857a [Davies Liu] typo
d2f053b [Davies Liu] add repr for FlattedValuesSerializer
c6a2f8d [Davies Liu] address comments
9e2df24 [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
2b9c261 [Davies Liu] fix typo in comments
70aadcd [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
a14b4bd [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
ab5515b [Davies Liu] Merge branch 'master' into groupby
651f891 [Davies Liu] simplify GroupByKey
1578f2e [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
1f69f93 [Davies Liu] fix tests
0d3395f [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
341f1e0 [Davies Liu] add comments, refactor
47918b8 [Davies Liu] remove unused code
6540948 [Davies Liu] address comments:
17f4ec6 [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
4d4bc86 [Davies Liu] bugfix
8ef965e [Davies Liu] Merge branch 'master' into groupby
fbc504a [Davies Liu] Merge branch 'master' into groupby
779ed03 [Davies Liu] fix merge conflict
2c1d05b [Davies Liu] refactor, minor turning
b48cda5 [Davies Liu] Merge branch 'master' into groupby
85138e6 [Davies Liu] Merge branch 'master' into groupby
acd8e1b [Davies Liu] fix memory when groupByKey().count()
905b233 [Davies Liu] Merge branch 'sort' into groupby
1f075ed [Davies Liu] Merge branch 'master' into sort
4b07d39 [Davies Liu] compress the data while spilling
0a081c6 [Davies Liu] Merge branch 'master' into groupby
f157fe7 [Davies Liu] Merge branch 'sort' into groupby
eb53ca6 [Davies Liu] Merge branch 'master' into sort
b2dc3bf [Davies Liu] Merge branch 'sort' into groupby
644abaf [Davies Liu] add license in LICENSE
19f7873 [Davies Liu] improve tests
11ba318 [Davies Liu] typo
085aef8 [Davies Liu] Merge branch 'master' into groupby
3ee58e5 [Davies Liu] switch to sort based groupBy, based on size of data
1ea0669 [Davies Liu] choose sort based groupByKey() automatically
b40bae7 [Davies Liu] bugfix
efa23df [Davies Liu] refactor, add spark.shuffle.sort=False
250be4e [Davies Liu] flatten the combined values when dumping into disks
d05060d [Davies Liu] group the same key before shuffle, reduce the comparison during sorting
083d842 [Davies Liu] sorted based groupByKey()
55602ee [Davies Liu] use external sort in sortBy() and sortByKey()
2015-04-09 20:07:23 -04:00
|
|
|
get_used_memory, ExternalSorter, ExternalGroupBy
|
2014-09-15 22:28:17 -04:00
|
|
|
from pyspark.traceback_utils import SCCallSiteSync
|
2020-04-22 21:20:39 -04:00
|
|
|
from pyspark.util import fail_on_stopiteration, _parse_memory
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2014-09-03 14:49:45 -04:00
|
|
|
|
2012-12-27 20:55:33 -05:00
|
|
|
__all__ = ["RDD"]
|
|
|
|
|
2014-04-03 18:42:17 -04:00
|
|
|
|
2017-11-17 10:43:08 -05:00
|
|
|
class PythonEvalType(object):
|
|
|
|
"""
|
|
|
|
Evaluation type of python rdd.
|
|
|
|
|
|
|
|
These values are internal to PySpark.
|
|
|
|
|
|
|
|
These values should match values in org.apache.spark.api.python.PythonEvalType.
|
|
|
|
"""
|
|
|
|
NON_UDF = 0
|
|
|
|
|
|
|
|
SQL_BATCHED_UDF = 100
|
|
|
|
|
2018-01-30 07:55:55 -05:00
|
|
|
SQL_SCALAR_PANDAS_UDF = 200
|
|
|
|
SQL_GROUPED_MAP_PANDAS_UDF = 201
|
|
|
|
SQL_GROUPED_AGG_PANDAS_UDF = 202
|
[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
|
|
|
SQL_WINDOW_AGG_PANDAS_UDF = 203
|
[SPARK-26412][PYSPARK][SQL] Allow Pandas UDF to take an iterator of pd.Series or an iterator of tuple of pd.Series
## What changes were proposed in this pull request?
Allow Pandas UDF to take an iterator of pd.Series or an iterator of tuple of pd.Series.
Note the UDF input args will be always one iterator:
* if the udf take only column as input, the iterator's element will be pd.Series (corresponding to the column values batch)
* if the udf take multiple columns as inputs, the iterator's element will be a tuple composed of multiple `pd.Series`s, each one corresponding to the multiple columns as inputs (keep the same order). For example:
```
pandas_udf("int", PandasUDFType.SCALAR_ITER)
def the_udf(iterator):
for col1_batch, col2_batch in iterator:
yield col1_batch + col2_batch
df.select(the_udf("col1", "col2"))
```
The udf above will add col1 and col2.
I haven't add unit tests, but manually tests show it works fine. So it is ready for first pass review.
We can test several typical cases:
```
from pyspark.sql import SparkSession
from pyspark.sql.functions import pandas_udf, PandasUDFType
from pyspark.sql.functions import udf
from pyspark.taskcontext import TaskContext
df = spark.createDataFrame([(1, 20), (3, 40)], ["a", "b"])
pandas_udf("int", PandasUDFType.SCALAR_ITER)
def fi1(it):
pid = TaskContext.get().partitionId()
print("DBG: fi1: do init stuff, partitionId=" + str(pid))
for batch in it:
yield batch + 100
print("DBG: fi1: do close stuff, partitionId=" + str(pid))
pandas_udf("int", PandasUDFType.SCALAR_ITER)
def fi2(it):
pid = TaskContext.get().partitionId()
print("DBG: fi2: do init stuff, partitionId=" + str(pid))
for batch in it:
yield batch + 10000
print("DBG: fi2: do close stuff, partitionId=" + str(pid))
pandas_udf("int", PandasUDFType.SCALAR_ITER)
def fi3(it):
pid = TaskContext.get().partitionId()
print("DBG: fi3: do init stuff, partitionId=" + str(pid))
for x, y in it:
yield x + y * 10 + 100000
print("DBG: fi3: do close stuff, partitionId=" + str(pid))
pandas_udf("int", PandasUDFType.SCALAR)
def fp1(x):
return x + 1000
udf("int")
def fu1(x):
return x + 10
# test select "pandas iter udf/pandas udf/sql udf" expressions at the same time.
# Note this case the `fi1("a"), fi2("b"), fi3("a", "b")` will generate only one plan,
# and `fu1("a")`, `fp1("a")` will generate another two separate plans.
df.select(fi1("a"), fi2("b"), fi3("a", "b"), fu1("a"), fp1("a")).show()
# test chain two pandas iter udf together
# Note this case `fi2(fi1("a"))` will generate only one plan
# Also note the init stuff/close stuff call order will be like:
# (debug output following)
# DBG: fi2: do init stuff, partitionId=0
# DBG: fi1: do init stuff, partitionId=0
# DBG: fi1: do close stuff, partitionId=0
# DBG: fi2: do close stuff, partitionId=0
df.select(fi2(fi1("a"))).show()
# test more complex chain
# Note this case `fi1("a"), fi2("a")` will generate one plan,
# and `fi3(fi1_output, fi2_output)` will generate another plan
df.select(fi3(fi1("a"), fi2("a"))).show()
```
## How was this patch tested?
To be added.
Please review http://spark.apache.org/contributing.html before opening a pull request.
Closes #24643 from WeichenXu123/pandas_udf_iter.
Lead-authored-by: WeichenXu <weichen.xu@databricks.com>
Co-authored-by: Xiangrui Meng <meng@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-06-15 11:29:20 -04:00
|
|
|
SQL_SCALAR_PANDAS_ITER_UDF = 204
|
[SPARK-28198][PYTHON] Add mapPartitionsInPandas to allow an iterator of DataFrames
## What changes were proposed in this pull request?
This PR proposes to add `mapPartitionsInPandas` API to DataFrame by using existing `SCALAR_ITER` as below:
1. Filtering via setting the column
```python
from pyspark.sql.functions import pandas_udf, PandasUDFType
df = spark.createDataFrame([(1, 21), (2, 30)], ("id", "age"))
pandas_udf(df.schema, PandasUDFType.SCALAR_ITER)
def filter_func(iterator):
for pdf in iterator:
yield pdf[pdf.id == 1]
df.mapPartitionsInPandas(filter_func).show()
```
```
+---+---+
| id|age|
+---+---+
| 1| 21|
+---+---+
```
2. `DataFrame.loc`
```python
from pyspark.sql.functions import pandas_udf, PandasUDFType
import pandas as pd
df = spark.createDataFrame([['aa'], ['bb'], ['cc'], ['aa'], ['aa'], ['aa']], ["value"])
pandas_udf(df.schema, PandasUDFType.SCALAR_ITER)
def filter_func(iterator):
for pdf in iterator:
yield pdf.loc[pdf.value.str.contains('^a'), :]
df.mapPartitionsInPandas(filter_func).show()
```
```
+-----+
|value|
+-----+
| aa|
| aa|
| aa|
| aa|
+-----+
```
3. `pandas.melt`
```python
from pyspark.sql.functions import pandas_udf, PandasUDFType
import pandas as pd
df = spark.createDataFrame(
pd.DataFrame({'A': {0: 'a', 1: 'b', 2: 'c'},
'B': {0: 1, 1: 3, 2: 5},
'C': {0: 2, 1: 4, 2: 6}}))
pandas_udf("A string, variable string, value long", PandasUDFType.SCALAR_ITER)
def filter_func(iterator):
for pdf in iterator:
import pandas as pd
yield pd.melt(pdf, id_vars=['A'], value_vars=['B', 'C'])
df.mapPartitionsInPandas(filter_func).show()
```
```
+---+--------+-----+
| A|variable|value|
+---+--------+-----+
| a| B| 1|
| a| C| 2|
| b| B| 3|
| b| C| 4|
| c| B| 5|
| c| C| 6|
+---+--------+-----+
```
The current limitation of `SCALAR_ITER` is that it doesn't allow different length of result, which is pretty critical in practice - for instance, we cannot simply filter by using Pandas APIs but we merely just map N to N. This PR allows map N to M like flatMap.
This API mimics the way of `mapPartitions` but keeps API shape of `SCALAR_ITER` by allowing different results.
### How does this PR implement?
This PR adds mimics both `dapply` with Arrow optimization and Grouped Map Pandas UDF. At Python execution side, it reuses existing `SCALAR_ITER` code path.
Therefore, externally, we don't introduce any new type of Pandas UDF but internally we use another evaluation type code `205` (`SQL_MAP_PANDAS_ITER_UDF`).
This approach is similar with Pandas' Windows function implementation with Grouped Aggregation Pandas UDF functions - internally we have `203` (`SQL_WINDOW_AGG_PANDAS_UDF`) but externally we just share the same `GROUPED_AGG`.
## How was this patch tested?
Manually tested and unittests were added.
Closes #24997 from HyukjinKwon/scalar-udf-iter.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-01 21:54:16 -04:00
|
|
|
SQL_MAP_PANDAS_ITER_UDF = 205
|
[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 20:13:50 -04:00
|
|
|
SQL_COGROUPED_MAP_PANDAS_UDF = 206
|
2017-11-17 10:43:08 -05:00
|
|
|
|
|
|
|
|
2014-07-21 14:59:54 -04:00
|
|
|
def portable_hash(x):
|
|
|
|
"""
|
2015-04-16 19:20:57 -04:00
|
|
|
This function returns consistent hash code for builtin types, especially
|
2014-07-21 14:59:54 -04:00
|
|
|
for None and tuple with None.
|
|
|
|
|
2015-04-16 19:20:57 -04:00
|
|
|
The algorithm is similar to that one used by CPython 2.7
|
2014-07-21 14:59:54 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2014-07-21 14:59:54 -04:00
|
|
|
>>> portable_hash(None)
|
|
|
|
0
|
2014-09-02 18:47:47 -04:00
|
|
|
>>> portable_hash((None, 1)) & 0xffffffff
|
2014-07-21 14:59:54 -04:00
|
|
|
219750521
|
|
|
|
"""
|
2017-02-24 18:04:42 -05:00
|
|
|
|
2020-07-13 22:22:44 -04:00
|
|
|
if 'PYTHONHASHSEED' not in os.environ:
|
2015-04-16 19:20:57 -04:00
|
|
|
raise Exception("Randomness of hash of string should be disabled via PYTHONHASHSEED")
|
|
|
|
|
2014-07-21 14:59:54 -04:00
|
|
|
if x is None:
|
|
|
|
return 0
|
|
|
|
if isinstance(x, tuple):
|
|
|
|
h = 0x345678
|
|
|
|
for i in x:
|
|
|
|
h ^= portable_hash(i)
|
|
|
|
h *= 1000003
|
2015-04-16 19:20:57 -04:00
|
|
|
h &= sys.maxsize
|
2014-07-21 14:59:54 -04:00
|
|
|
h ^= len(x)
|
|
|
|
if h == -1:
|
|
|
|
h = -2
|
2015-09-17 13:02:15 -04:00
|
|
|
return int(h)
|
2014-07-21 14:59:54 -04:00
|
|
|
return hash(x)
|
|
|
|
|
|
|
|
|
[SPARK-2871] [PySpark] add approx API for RDD
RDD.countApprox(self, timeout, confidence=0.95)
:: Experimental ::
Approximate version of count() that returns a potentially incomplete
result within a timeout, even if not all tasks have finished.
>>> rdd = sc.parallelize(range(1000), 10)
>>> rdd.countApprox(1000, 1.0)
1000
RDD.sumApprox(self, timeout, confidence=0.95)
Approximate operation to return the sum within a timeout
or meet the confidence.
>>> rdd = sc.parallelize(range(1000), 10)
>>> r = sum(xrange(1000))
>>> (rdd.sumApprox(1000) - r) / r < 0.05
RDD.meanApprox(self, timeout, confidence=0.95)
:: Experimental ::
Approximate operation to return the mean within a timeout
or meet the confidence.
>>> rdd = sc.parallelize(range(1000), 10)
>>> r = sum(xrange(1000)) / 1000.0
>>> (rdd.meanApprox(1000) - r) / r < 0.05
True
Author: Davies Liu <davies.liu@gmail.com>
Closes #2095 from davies/approx and squashes the following commits:
e8c252b [Davies Liu] add approx API for RDD
2014-08-23 22:33:34 -04:00
|
|
|
class BoundedFloat(float):
|
|
|
|
"""
|
|
|
|
Bounded value is generated by approximate job, with confidence and low
|
|
|
|
bound and high bound.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
[SPARK-2871] [PySpark] add approx API for RDD
RDD.countApprox(self, timeout, confidence=0.95)
:: Experimental ::
Approximate version of count() that returns a potentially incomplete
result within a timeout, even if not all tasks have finished.
>>> rdd = sc.parallelize(range(1000), 10)
>>> rdd.countApprox(1000, 1.0)
1000
RDD.sumApprox(self, timeout, confidence=0.95)
Approximate operation to return the sum within a timeout
or meet the confidence.
>>> rdd = sc.parallelize(range(1000), 10)
>>> r = sum(xrange(1000))
>>> (rdd.sumApprox(1000) - r) / r < 0.05
RDD.meanApprox(self, timeout, confidence=0.95)
:: Experimental ::
Approximate operation to return the mean within a timeout
or meet the confidence.
>>> rdd = sc.parallelize(range(1000), 10)
>>> r = sum(xrange(1000)) / 1000.0
>>> (rdd.meanApprox(1000) - r) / r < 0.05
True
Author: Davies Liu <davies.liu@gmail.com>
Closes #2095 from davies/approx and squashes the following commits:
e8c252b [Davies Liu] add approx API for RDD
2014-08-23 22:33:34 -04:00
|
|
|
>>> BoundedFloat(100.0, 0.95, 95.0, 105.0)
|
|
|
|
100.0
|
|
|
|
"""
|
|
|
|
def __new__(cls, mean, confidence, low, high):
|
|
|
|
obj = float.__new__(cls, mean)
|
|
|
|
obj.confidence = confidence
|
|
|
|
obj.low = low
|
|
|
|
obj.high = high
|
|
|
|
return obj
|
|
|
|
|
|
|
|
|
[SPARK-23961][SPARK-27548][PYTHON] Fix error when toLocalIterator goes out of scope and properly raise errors from worker
## What changes were proposed in this pull request?
This fixes an error when a PySpark local iterator, for both RDD and DataFrames, goes out of scope and the connection is closed before fully consuming the iterator. The error occurs on the JVM in the serving thread, when Python closes the local socket while the JVM is writing to it. This usually happens when there is enough data to fill the socket read buffer, causing the write call to block.
Additionally, this fixes a problem when an error occurs in the Python worker and the collect job is cancelled with an exception. Previously, the Python driver was never notified of the error so the user could get a partial result (iteration until the error) and the application will continue. With this change, an error in the worker is sent to the Python iterator and is then raised.
The change here introduces a protocol for PySpark local iterators that work as follows:
1) The local socket connection is made when the iterator is created
2) When iterating, Python first sends a request for partition data as a non-zero integer
3) While the JVM local iterator over partitions has next, it triggers a job to collect the next partition
4) The JVM sends a nonzero response to indicate it has the next partition to send
5) The next partition is sent to Python and read by the PySpark deserializer
6) After sending the entire partition, an `END_OF_DATA_SECTION` is sent to Python which stops the deserializer and allows to make another request
7) When the JVM gets a request from Python but has already consumed it's local iterator, it will send a zero response to Python and both will close the socket cleanly
8) If an error occurs in the worker, a negative response is sent to Python followed by the error message. Python will then raise a RuntimeError with the message, stopping iteration.
9) When the PySpark local iterator is garbage-collected, it will read any remaining data from the current partition (this is data that has already been collected) and send a request of zero to tell the JVM to stop collection jobs and close the connection.
Steps 1, 3, 5, 6 are the same as before. Step 8 was completely missing before because errors in the worker were never communicated back to Python. The other steps add synchronization to allow for a clean closing of the socket, with a small trade-off in performance for each partition. This is mainly because the JVM does not start collecting partition data until it receives a request to do so, where before it would eagerly write all data until the socket receive buffer is full.
## How was this patch tested?
Added new unit tests for DataFrame and RDD `toLocalIterator` and tested not fully consuming the iterator. Manual tests with Python 2.7 and 3.6.
Closes #24070 from BryanCutler/pyspark-toLocalIterator-clean-stop-SPARK-23961.
Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-05-07 17:47:39 -04:00
|
|
|
def _create_local_socket(sock_info):
|
2019-06-26 16:05:41 -04:00
|
|
|
"""
|
|
|
|
Create a local socket that can be used to load deserialized data from the JVM
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
sock_info : tuple
|
|
|
|
Tuple containing port number and authentication secret for a local socket.
|
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
|
|
|
sockfile file descriptor of the local socket
|
2019-06-26 16:05:41 -04:00
|
|
|
"""
|
|
|
|
port = sock_info[0]
|
|
|
|
auth_secret = sock_info[1]
|
|
|
|
sockfile, sock = local_connect_and_auth(port, auth_secret)
|
[SPARK-23961][SPARK-27548][PYTHON] Fix error when toLocalIterator goes out of scope and properly raise errors from worker
## What changes were proposed in this pull request?
This fixes an error when a PySpark local iterator, for both RDD and DataFrames, goes out of scope and the connection is closed before fully consuming the iterator. The error occurs on the JVM in the serving thread, when Python closes the local socket while the JVM is writing to it. This usually happens when there is enough data to fill the socket read buffer, causing the write call to block.
Additionally, this fixes a problem when an error occurs in the Python worker and the collect job is cancelled with an exception. Previously, the Python driver was never notified of the error so the user could get a partial result (iteration until the error) and the application will continue. With this change, an error in the worker is sent to the Python iterator and is then raised.
The change here introduces a protocol for PySpark local iterators that work as follows:
1) The local socket connection is made when the iterator is created
2) When iterating, Python first sends a request for partition data as a non-zero integer
3) While the JVM local iterator over partitions has next, it triggers a job to collect the next partition
4) The JVM sends a nonzero response to indicate it has the next partition to send
5) The next partition is sent to Python and read by the PySpark deserializer
6) After sending the entire partition, an `END_OF_DATA_SECTION` is sent to Python which stops the deserializer and allows to make another request
7) When the JVM gets a request from Python but has already consumed it's local iterator, it will send a zero response to Python and both will close the socket cleanly
8) If an error occurs in the worker, a negative response is sent to Python followed by the error message. Python will then raise a RuntimeError with the message, stopping iteration.
9) When the PySpark local iterator is garbage-collected, it will read any remaining data from the current partition (this is data that has already been collected) and send a request of zero to tell the JVM to stop collection jobs and close the connection.
Steps 1, 3, 5, 6 are the same as before. Step 8 was completely missing before because errors in the worker were never communicated back to Python. The other steps add synchronization to allow for a clean closing of the socket, with a small trade-off in performance for each partition. This is mainly because the JVM does not start collecting partition data until it receives a request to do so, where before it would eagerly write all data until the socket receive buffer is full.
## How was this patch tested?
Added new unit tests for DataFrame and RDD `toLocalIterator` and tested not fully consuming the iterator. Manual tests with Python 2.7 and 3.6.
Closes #24070 from BryanCutler/pyspark-toLocalIterator-clean-stop-SPARK-23961.
Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-05-07 17:47:39 -04:00
|
|
|
# The RDD materialization time is unpredictable, if we set a timeout for socket reading
|
2016-12-20 16:12:16 -05:00
|
|
|
# operation, it will very possibly fail. See SPARK-18281.
|
|
|
|
sock.settimeout(None)
|
[SPARK-23961][SPARK-27548][PYTHON] Fix error when toLocalIterator goes out of scope and properly raise errors from worker
## What changes were proposed in this pull request?
This fixes an error when a PySpark local iterator, for both RDD and DataFrames, goes out of scope and the connection is closed before fully consuming the iterator. The error occurs on the JVM in the serving thread, when Python closes the local socket while the JVM is writing to it. This usually happens when there is enough data to fill the socket read buffer, causing the write call to block.
Additionally, this fixes a problem when an error occurs in the Python worker and the collect job is cancelled with an exception. Previously, the Python driver was never notified of the error so the user could get a partial result (iteration until the error) and the application will continue. With this change, an error in the worker is sent to the Python iterator and is then raised.
The change here introduces a protocol for PySpark local iterators that work as follows:
1) The local socket connection is made when the iterator is created
2) When iterating, Python first sends a request for partition data as a non-zero integer
3) While the JVM local iterator over partitions has next, it triggers a job to collect the next partition
4) The JVM sends a nonzero response to indicate it has the next partition to send
5) The next partition is sent to Python and read by the PySpark deserializer
6) After sending the entire partition, an `END_OF_DATA_SECTION` is sent to Python which stops the deserializer and allows to make another request
7) When the JVM gets a request from Python but has already consumed it's local iterator, it will send a zero response to Python and both will close the socket cleanly
8) If an error occurs in the worker, a negative response is sent to Python followed by the error message. Python will then raise a RuntimeError with the message, stopping iteration.
9) When the PySpark local iterator is garbage-collected, it will read any remaining data from the current partition (this is data that has already been collected) and send a request of zero to tell the JVM to stop collection jobs and close the connection.
Steps 1, 3, 5, 6 are the same as before. Step 8 was completely missing before because errors in the worker were never communicated back to Python. The other steps add synchronization to allow for a clean closing of the socket, with a small trade-off in performance for each partition. This is mainly because the JVM does not start collecting partition data until it receives a request to do so, where before it would eagerly write all data until the socket receive buffer is full.
## How was this patch tested?
Added new unit tests for DataFrame and RDD `toLocalIterator` and tested not fully consuming the iterator. Manual tests with Python 2.7 and 3.6.
Closes #24070 from BryanCutler/pyspark-toLocalIterator-clean-stop-SPARK-23961.
Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-05-07 17:47:39 -04:00
|
|
|
return sockfile
|
|
|
|
|
|
|
|
|
|
|
|
def _load_from_socket(sock_info, serializer):
|
2019-06-26 16:05:41 -04:00
|
|
|
"""
|
|
|
|
Connect to a local socket described by sock_info and use the given serializer to yield data
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
sock_info : tuple
|
|
|
|
Tuple containing port number and authentication secret for a local socket.
|
|
|
|
serializer : :py:class:`Serializer`
|
|
|
|
The PySpark serializer to use
|
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
|
|
|
result of :py:meth:`Serializer.load_stream`,
|
|
|
|
usually a generator that yields deserialized data
|
2019-06-26 16:05:41 -04:00
|
|
|
"""
|
[SPARK-23961][SPARK-27548][PYTHON] Fix error when toLocalIterator goes out of scope and properly raise errors from worker
## What changes were proposed in this pull request?
This fixes an error when a PySpark local iterator, for both RDD and DataFrames, goes out of scope and the connection is closed before fully consuming the iterator. The error occurs on the JVM in the serving thread, when Python closes the local socket while the JVM is writing to it. This usually happens when there is enough data to fill the socket read buffer, causing the write call to block.
Additionally, this fixes a problem when an error occurs in the Python worker and the collect job is cancelled with an exception. Previously, the Python driver was never notified of the error so the user could get a partial result (iteration until the error) and the application will continue. With this change, an error in the worker is sent to the Python iterator and is then raised.
The change here introduces a protocol for PySpark local iterators that work as follows:
1) The local socket connection is made when the iterator is created
2) When iterating, Python first sends a request for partition data as a non-zero integer
3) While the JVM local iterator over partitions has next, it triggers a job to collect the next partition
4) The JVM sends a nonzero response to indicate it has the next partition to send
5) The next partition is sent to Python and read by the PySpark deserializer
6) After sending the entire partition, an `END_OF_DATA_SECTION` is sent to Python which stops the deserializer and allows to make another request
7) When the JVM gets a request from Python but has already consumed it's local iterator, it will send a zero response to Python and both will close the socket cleanly
8) If an error occurs in the worker, a negative response is sent to Python followed by the error message. Python will then raise a RuntimeError with the message, stopping iteration.
9) When the PySpark local iterator is garbage-collected, it will read any remaining data from the current partition (this is data that has already been collected) and send a request of zero to tell the JVM to stop collection jobs and close the connection.
Steps 1, 3, 5, 6 are the same as before. Step 8 was completely missing before because errors in the worker were never communicated back to Python. The other steps add synchronization to allow for a clean closing of the socket, with a small trade-off in performance for each partition. This is mainly because the JVM does not start collecting partition data until it receives a request to do so, where before it would eagerly write all data until the socket receive buffer is full.
## How was this patch tested?
Added new unit tests for DataFrame and RDD `toLocalIterator` and tested not fully consuming the iterator. Manual tests with Python 2.7 and 3.6.
Closes #24070 from BryanCutler/pyspark-toLocalIterator-clean-stop-SPARK-23961.
Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-05-07 17:47:39 -04:00
|
|
|
sockfile = _create_local_socket(sock_info)
|
2016-12-20 16:12:16 -05:00
|
|
|
# The socket will be automatically closed when garbage-collected.
|
2018-04-13 17:28:24 -04:00
|
|
|
return serializer.load_stream(sockfile)
|
2015-03-09 19:24:06 -04:00
|
|
|
|
|
|
|
|
[SPARK-23961][SPARK-27548][PYTHON] Fix error when toLocalIterator goes out of scope and properly raise errors from worker
## What changes were proposed in this pull request?
This fixes an error when a PySpark local iterator, for both RDD and DataFrames, goes out of scope and the connection is closed before fully consuming the iterator. The error occurs on the JVM in the serving thread, when Python closes the local socket while the JVM is writing to it. This usually happens when there is enough data to fill the socket read buffer, causing the write call to block.
Additionally, this fixes a problem when an error occurs in the Python worker and the collect job is cancelled with an exception. Previously, the Python driver was never notified of the error so the user could get a partial result (iteration until the error) and the application will continue. With this change, an error in the worker is sent to the Python iterator and is then raised.
The change here introduces a protocol for PySpark local iterators that work as follows:
1) The local socket connection is made when the iterator is created
2) When iterating, Python first sends a request for partition data as a non-zero integer
3) While the JVM local iterator over partitions has next, it triggers a job to collect the next partition
4) The JVM sends a nonzero response to indicate it has the next partition to send
5) The next partition is sent to Python and read by the PySpark deserializer
6) After sending the entire partition, an `END_OF_DATA_SECTION` is sent to Python which stops the deserializer and allows to make another request
7) When the JVM gets a request from Python but has already consumed it's local iterator, it will send a zero response to Python and both will close the socket cleanly
8) If an error occurs in the worker, a negative response is sent to Python followed by the error message. Python will then raise a RuntimeError with the message, stopping iteration.
9) When the PySpark local iterator is garbage-collected, it will read any remaining data from the current partition (this is data that has already been collected) and send a request of zero to tell the JVM to stop collection jobs and close the connection.
Steps 1, 3, 5, 6 are the same as before. Step 8 was completely missing before because errors in the worker were never communicated back to Python. The other steps add synchronization to allow for a clean closing of the socket, with a small trade-off in performance for each partition. This is mainly because the JVM does not start collecting partition data until it receives a request to do so, where before it would eagerly write all data until the socket receive buffer is full.
## How was this patch tested?
Added new unit tests for DataFrame and RDD `toLocalIterator` and tested not fully consuming the iterator. Manual tests with Python 2.7 and 3.6.
Closes #24070 from BryanCutler/pyspark-toLocalIterator-clean-stop-SPARK-23961.
Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-05-07 17:47:39 -04:00
|
|
|
def _local_iterator_from_socket(sock_info, serializer):
|
|
|
|
|
|
|
|
class PyLocalIterable(object):
|
|
|
|
""" Create a synchronous local iterable over a socket """
|
|
|
|
|
|
|
|
def __init__(self, _sock_info, _serializer):
|
2019-06-26 16:05:41 -04:00
|
|
|
port, auth_secret, self.jsocket_auth_server = _sock_info
|
|
|
|
self._sockfile = _create_local_socket((port, auth_secret))
|
[SPARK-23961][SPARK-27548][PYTHON] Fix error when toLocalIterator goes out of scope and properly raise errors from worker
## What changes were proposed in this pull request?
This fixes an error when a PySpark local iterator, for both RDD and DataFrames, goes out of scope and the connection is closed before fully consuming the iterator. The error occurs on the JVM in the serving thread, when Python closes the local socket while the JVM is writing to it. This usually happens when there is enough data to fill the socket read buffer, causing the write call to block.
Additionally, this fixes a problem when an error occurs in the Python worker and the collect job is cancelled with an exception. Previously, the Python driver was never notified of the error so the user could get a partial result (iteration until the error) and the application will continue. With this change, an error in the worker is sent to the Python iterator and is then raised.
The change here introduces a protocol for PySpark local iterators that work as follows:
1) The local socket connection is made when the iterator is created
2) When iterating, Python first sends a request for partition data as a non-zero integer
3) While the JVM local iterator over partitions has next, it triggers a job to collect the next partition
4) The JVM sends a nonzero response to indicate it has the next partition to send
5) The next partition is sent to Python and read by the PySpark deserializer
6) After sending the entire partition, an `END_OF_DATA_SECTION` is sent to Python which stops the deserializer and allows to make another request
7) When the JVM gets a request from Python but has already consumed it's local iterator, it will send a zero response to Python and both will close the socket cleanly
8) If an error occurs in the worker, a negative response is sent to Python followed by the error message. Python will then raise a RuntimeError with the message, stopping iteration.
9) When the PySpark local iterator is garbage-collected, it will read any remaining data from the current partition (this is data that has already been collected) and send a request of zero to tell the JVM to stop collection jobs and close the connection.
Steps 1, 3, 5, 6 are the same as before. Step 8 was completely missing before because errors in the worker were never communicated back to Python. The other steps add synchronization to allow for a clean closing of the socket, with a small trade-off in performance for each partition. This is mainly because the JVM does not start collecting partition data until it receives a request to do so, where before it would eagerly write all data until the socket receive buffer is full.
## How was this patch tested?
Added new unit tests for DataFrame and RDD `toLocalIterator` and tested not fully consuming the iterator. Manual tests with Python 2.7 and 3.6.
Closes #24070 from BryanCutler/pyspark-toLocalIterator-clean-stop-SPARK-23961.
Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-05-07 17:47:39 -04:00
|
|
|
self._serializer = _serializer
|
|
|
|
self._read_iter = iter([]) # Initialize as empty iterator
|
|
|
|
self._read_status = 1
|
|
|
|
|
|
|
|
def __iter__(self):
|
|
|
|
while self._read_status == 1:
|
|
|
|
# Request next partition data from Java
|
|
|
|
write_int(1, self._sockfile)
|
|
|
|
self._sockfile.flush()
|
|
|
|
|
|
|
|
# If response is 1 then there is a partition to read, if 0 then fully consumed
|
|
|
|
self._read_status = read_int(self._sockfile)
|
|
|
|
if self._read_status == 1:
|
|
|
|
|
|
|
|
# Load the partition data as a stream and read each item
|
|
|
|
self._read_iter = self._serializer.load_stream(self._sockfile)
|
|
|
|
for item in self._read_iter:
|
|
|
|
yield item
|
|
|
|
|
2019-06-26 16:05:41 -04:00
|
|
|
# An error occurred, join serving thread and raise any exceptions from the JVM
|
[SPARK-23961][SPARK-27548][PYTHON] Fix error when toLocalIterator goes out of scope and properly raise errors from worker
## What changes were proposed in this pull request?
This fixes an error when a PySpark local iterator, for both RDD and DataFrames, goes out of scope and the connection is closed before fully consuming the iterator. The error occurs on the JVM in the serving thread, when Python closes the local socket while the JVM is writing to it. This usually happens when there is enough data to fill the socket read buffer, causing the write call to block.
Additionally, this fixes a problem when an error occurs in the Python worker and the collect job is cancelled with an exception. Previously, the Python driver was never notified of the error so the user could get a partial result (iteration until the error) and the application will continue. With this change, an error in the worker is sent to the Python iterator and is then raised.
The change here introduces a protocol for PySpark local iterators that work as follows:
1) The local socket connection is made when the iterator is created
2) When iterating, Python first sends a request for partition data as a non-zero integer
3) While the JVM local iterator over partitions has next, it triggers a job to collect the next partition
4) The JVM sends a nonzero response to indicate it has the next partition to send
5) The next partition is sent to Python and read by the PySpark deserializer
6) After sending the entire partition, an `END_OF_DATA_SECTION` is sent to Python which stops the deserializer and allows to make another request
7) When the JVM gets a request from Python but has already consumed it's local iterator, it will send a zero response to Python and both will close the socket cleanly
8) If an error occurs in the worker, a negative response is sent to Python followed by the error message. Python will then raise a RuntimeError with the message, stopping iteration.
9) When the PySpark local iterator is garbage-collected, it will read any remaining data from the current partition (this is data that has already been collected) and send a request of zero to tell the JVM to stop collection jobs and close the connection.
Steps 1, 3, 5, 6 are the same as before. Step 8 was completely missing before because errors in the worker were never communicated back to Python. The other steps add synchronization to allow for a clean closing of the socket, with a small trade-off in performance for each partition. This is mainly because the JVM does not start collecting partition data until it receives a request to do so, where before it would eagerly write all data until the socket receive buffer is full.
## How was this patch tested?
Added new unit tests for DataFrame and RDD `toLocalIterator` and tested not fully consuming the iterator. Manual tests with Python 2.7 and 3.6.
Closes #24070 from BryanCutler/pyspark-toLocalIterator-clean-stop-SPARK-23961.
Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-05-07 17:47:39 -04:00
|
|
|
elif self._read_status == -1:
|
2019-06-26 16:05:41 -04:00
|
|
|
self.jsocket_auth_server.getResult()
|
[SPARK-23961][SPARK-27548][PYTHON] Fix error when toLocalIterator goes out of scope and properly raise errors from worker
## What changes were proposed in this pull request?
This fixes an error when a PySpark local iterator, for both RDD and DataFrames, goes out of scope and the connection is closed before fully consuming the iterator. The error occurs on the JVM in the serving thread, when Python closes the local socket while the JVM is writing to it. This usually happens when there is enough data to fill the socket read buffer, causing the write call to block.
Additionally, this fixes a problem when an error occurs in the Python worker and the collect job is cancelled with an exception. Previously, the Python driver was never notified of the error so the user could get a partial result (iteration until the error) and the application will continue. With this change, an error in the worker is sent to the Python iterator and is then raised.
The change here introduces a protocol for PySpark local iterators that work as follows:
1) The local socket connection is made when the iterator is created
2) When iterating, Python first sends a request for partition data as a non-zero integer
3) While the JVM local iterator over partitions has next, it triggers a job to collect the next partition
4) The JVM sends a nonzero response to indicate it has the next partition to send
5) The next partition is sent to Python and read by the PySpark deserializer
6) After sending the entire partition, an `END_OF_DATA_SECTION` is sent to Python which stops the deserializer and allows to make another request
7) When the JVM gets a request from Python but has already consumed it's local iterator, it will send a zero response to Python and both will close the socket cleanly
8) If an error occurs in the worker, a negative response is sent to Python followed by the error message. Python will then raise a RuntimeError with the message, stopping iteration.
9) When the PySpark local iterator is garbage-collected, it will read any remaining data from the current partition (this is data that has already been collected) and send a request of zero to tell the JVM to stop collection jobs and close the connection.
Steps 1, 3, 5, 6 are the same as before. Step 8 was completely missing before because errors in the worker were never communicated back to Python. The other steps add synchronization to allow for a clean closing of the socket, with a small trade-off in performance for each partition. This is mainly because the JVM does not start collecting partition data until it receives a request to do so, where before it would eagerly write all data until the socket receive buffer is full.
## How was this patch tested?
Added new unit tests for DataFrame and RDD `toLocalIterator` and tested not fully consuming the iterator. Manual tests with Python 2.7 and 3.6.
Closes #24070 from BryanCutler/pyspark-toLocalIterator-clean-stop-SPARK-23961.
Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-05-07 17:47:39 -04:00
|
|
|
|
|
|
|
def __del__(self):
|
|
|
|
# If local iterator is not fully consumed,
|
|
|
|
if self._read_status == 1:
|
|
|
|
try:
|
|
|
|
# Finish consuming partition data stream
|
|
|
|
for _ in self._read_iter:
|
|
|
|
pass
|
|
|
|
# Tell Java to stop sending data and close connection
|
|
|
|
write_int(0, self._sockfile)
|
|
|
|
self._sockfile.flush()
|
|
|
|
except Exception:
|
|
|
|
# Ignore any errors, socket is automatically closed when garbage-collected
|
|
|
|
pass
|
|
|
|
|
|
|
|
return iter(PyLocalIterable(sock_info, serializer))
|
|
|
|
|
|
|
|
|
2015-02-17 19:54:57 -05:00
|
|
|
class Partitioner(object):
|
|
|
|
def __init__(self, numPartitions, partitionFunc):
|
|
|
|
self.numPartitions = numPartitions
|
|
|
|
self.partitionFunc = partitionFunc
|
|
|
|
|
|
|
|
def __eq__(self, other):
|
|
|
|
return (isinstance(other, Partitioner) and self.numPartitions == other.numPartitions
|
|
|
|
and self.partitionFunc == other.partitionFunc)
|
|
|
|
|
|
|
|
def __call__(self, k):
|
|
|
|
return self.partitionFunc(k) % self.numPartitions
|
|
|
|
|
|
|
|
|
2012-08-10 04:10:02 -04:00
|
|
|
class RDD(object):
|
2014-07-14 03:42:59 -04:00
|
|
|
|
2012-12-27 20:55:33 -05:00
|
|
|
"""
|
|
|
|
A Resilient Distributed Dataset (RDD), the basic abstraction in Spark.
|
|
|
|
Represents an immutable, partitioned collection of elements that can be
|
|
|
|
operated on in parallel.
|
|
|
|
"""
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2014-11-04 02:56:14 -05:00
|
|
|
def __init__(self, jrdd, ctx, jrdd_deserializer=AutoBatchedSerializer(PickleSerializer())):
|
2012-08-10 04:10:02 -04:00
|
|
|
self._jrdd = jrdd
|
|
|
|
self.is_cached = False
|
2013-01-20 16:59:45 -05:00
|
|
|
self.is_checkpointed = False
|
2020-04-22 21:20:39 -04:00
|
|
|
self.has_resource_profile = False
|
2012-08-10 04:10:02 -04:00
|
|
|
self.ctx = ctx
|
2013-11-05 20:52:39 -05:00
|
|
|
self._jrdd_deserializer = jrdd_deserializer
|
2014-03-06 15:45:27 -05:00
|
|
|
self._id = jrdd.id()
|
2015-02-17 19:54:57 -05:00
|
|
|
self.partitioner = None
|
2014-03-06 15:45:27 -05:00
|
|
|
|
2014-11-04 02:56:14 -05:00
|
|
|
def _pickled(self):
|
|
|
|
return self._reserialize(AutoBatchedSerializer(PickleSerializer()))
|
2014-07-30 16:19:05 -04:00
|
|
|
|
2014-03-06 15:45:27 -05:00
|
|
|
def id(self):
|
|
|
|
"""
|
|
|
|
A unique ID for this RDD (within its SparkContext).
|
|
|
|
"""
|
|
|
|
return self._id
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2013-12-19 07:38:20 -05:00
|
|
|
def __repr__(self):
|
|
|
|
return self._jrdd.toString()
|
|
|
|
|
2015-01-23 20:53:15 -05:00
|
|
|
def __getnewargs__(self):
|
|
|
|
# This method is called when attempting to pickle an RDD, which is always an error:
|
|
|
|
raise Exception(
|
|
|
|
"It appears that you are attempting to broadcast an RDD or reference an RDD from an "
|
|
|
|
"action or transformation. RDD transformations and actions can only be invoked by the "
|
|
|
|
"driver, not inside of other transformations; for example, "
|
|
|
|
"rdd1.map(lambda x: rdd2.values.count() * x) is invalid because the values "
|
|
|
|
"transformation and count action cannot be performed inside of the rdd1.map "
|
|
|
|
"transformation. For more information, see SPARK-5063."
|
|
|
|
)
|
|
|
|
|
2012-12-27 20:55:33 -05:00
|
|
|
@property
|
|
|
|
def context(self):
|
|
|
|
"""
|
2019-07-05 13:08:22 -04:00
|
|
|
The :class:`SparkContext` that this RDD was created on.
|
2012-12-27 20:55:33 -05:00
|
|
|
"""
|
|
|
|
return self.ctx
|
|
|
|
|
2012-08-10 04:10:02 -04:00
|
|
|
def cache(self):
|
2012-12-27 20:55:33 -05:00
|
|
|
"""
|
2019-07-05 13:08:22 -04:00
|
|
|
Persist this RDD with the default storage level (`MEMORY_ONLY`).
|
2012-12-27 20:55:33 -05:00
|
|
|
"""
|
2012-08-10 04:10:02 -04:00
|
|
|
self.is_cached = True
|
2015-12-18 23:06:05 -05:00
|
|
|
self.persist(StorageLevel.MEMORY_ONLY)
|
2012-08-10 04:10:02 -04:00
|
|
|
return self
|
|
|
|
|
2015-12-18 23:06:05 -05:00
|
|
|
def persist(self, storageLevel=StorageLevel.MEMORY_ONLY):
|
2013-09-06 02:36:27 -04:00
|
|
|
"""
|
2014-06-12 22:44:27 -04:00
|
|
|
Set this RDD's storage level to persist its values across operations
|
|
|
|
after the first time it is computed. This can only be used to assign
|
|
|
|
a new storage level if the RDD does not have a storage level set yet.
|
2019-07-05 13:08:22 -04:00
|
|
|
If no storage level is specified defaults to (`MEMORY_ONLY`).
|
2014-09-06 17:49:25 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2014-09-06 17:49:25 -04:00
|
|
|
>>> rdd = sc.parallelize(["b", "a", "c"])
|
|
|
|
>>> rdd.persist().is_cached
|
|
|
|
True
|
2013-09-06 02:36:27 -04:00
|
|
|
"""
|
|
|
|
self.is_cached = True
|
2013-09-07 17:41:31 -04:00
|
|
|
javaStorageLevel = self.ctx._getJavaStorageLevel(storageLevel)
|
|
|
|
self._jrdd.persist(javaStorageLevel)
|
2013-09-06 02:36:27 -04:00
|
|
|
return self
|
|
|
|
|
2019-02-01 19:29:55 -05:00
|
|
|
def unpersist(self, blocking=False):
|
2013-09-06 02:36:27 -04:00
|
|
|
"""
|
2014-06-12 22:44:27 -04:00
|
|
|
Mark the RDD as non-persistent, and remove all blocks for it from
|
|
|
|
memory and disk.
|
2019-02-01 19:29:55 -05:00
|
|
|
|
|
|
|
.. versionchanged:: 3.0.0
|
|
|
|
Added optional argument `blocking` to specify whether to block until all
|
|
|
|
blocks are deleted.
|
2013-09-06 02:36:27 -04:00
|
|
|
"""
|
|
|
|
self.is_cached = False
|
2019-02-01 19:29:55 -05:00
|
|
|
self._jrdd.unpersist(blocking)
|
2013-09-06 02:36:27 -04:00
|
|
|
return self
|
|
|
|
|
2013-01-16 22:15:14 -05:00
|
|
|
def checkpoint(self):
|
|
|
|
"""
|
2013-01-20 18:31:41 -05:00
|
|
|
Mark this RDD for checkpointing. It will be saved to a file inside the
|
2019-07-05 13:08:22 -04:00
|
|
|
checkpoint directory set with :meth:`SparkContext.setCheckpointDir` and
|
2013-01-20 18:31:41 -05:00
|
|
|
all references to its parent RDDs will be removed. This function must
|
|
|
|
be called before any job has been executed on this RDD. It is strongly
|
|
|
|
recommended that this RDD is persisted in memory, otherwise saving it
|
2013-01-16 22:15:14 -05:00
|
|
|
on a file will require recomputation.
|
|
|
|
"""
|
2013-01-20 16:59:45 -05:00
|
|
|
self.is_checkpointed = True
|
2013-01-16 22:15:14 -05:00
|
|
|
self._jrdd.rdd().checkpoint()
|
|
|
|
|
|
|
|
def isCheckpointed(self):
|
|
|
|
"""
|
2016-11-21 16:08:34 -05:00
|
|
|
Return whether this RDD is checkpointed and materialized, either reliably or locally.
|
2013-01-16 22:15:14 -05:00
|
|
|
"""
|
|
|
|
return self._jrdd.rdd().isCheckpointed()
|
|
|
|
|
2016-11-21 16:08:34 -05:00
|
|
|
def localCheckpoint(self):
|
|
|
|
"""
|
|
|
|
Mark this RDD for local checkpointing using Spark's existing caching layer.
|
|
|
|
|
|
|
|
This method is for users who wish to truncate RDD lineages while skipping the expensive
|
|
|
|
step of replicating the materialized data in a reliable distributed file system. This is
|
|
|
|
useful for RDDs with long lineages that need to be truncated periodically (e.g. GraphX).
|
|
|
|
|
|
|
|
Local checkpointing sacrifices fault-tolerance for performance. In particular, checkpointed
|
|
|
|
data is written to ephemeral local storage in the executors instead of to a reliable,
|
|
|
|
fault-tolerant storage. The effect is that if an executor fails during the computation,
|
|
|
|
the checkpointed data may no longer be accessible, causing an irrecoverable job failure.
|
|
|
|
|
|
|
|
This is NOT safe to use with dynamic allocation, which removes executors along
|
|
|
|
with their cached blocks. If you must use both features, you are advised to set
|
2019-07-05 13:08:22 -04:00
|
|
|
`spark.dynamicAllocation.cachedExecutorIdleTimeout` to a high value.
|
2016-11-21 16:08:34 -05:00
|
|
|
|
2019-07-05 13:08:22 -04:00
|
|
|
The checkpoint directory set through :meth:`SparkContext.setCheckpointDir` is not used.
|
2016-11-21 16:08:34 -05:00
|
|
|
"""
|
|
|
|
self._jrdd.rdd().localCheckpoint()
|
|
|
|
|
|
|
|
def isLocallyCheckpointed(self):
|
|
|
|
"""
|
|
|
|
Return whether this RDD is marked for local checkpointing.
|
|
|
|
|
|
|
|
Exposed for testing.
|
|
|
|
"""
|
|
|
|
return self._jrdd.rdd().isLocallyCheckpointed()
|
|
|
|
|
2013-01-16 22:15:14 -05:00
|
|
|
def getCheckpointFile(self):
|
|
|
|
"""
|
|
|
|
Gets the name of the file to which this RDD was checkpointed
|
2016-11-21 16:08:34 -05:00
|
|
|
|
|
|
|
Not defined if RDD is checkpointed locally.
|
2013-01-16 22:15:14 -05:00
|
|
|
"""
|
|
|
|
checkpointFile = self._jrdd.rdd().getCheckpointFile()
|
|
|
|
if checkpointFile.isDefined():
|
|
|
|
return checkpointFile.get()
|
|
|
|
|
2012-08-18 19:07:10 -04:00
|
|
|
def map(self, f, preservesPartitioning=False):
|
2012-12-27 20:55:33 -05:00
|
|
|
"""
|
2014-03-04 18:32:43 -05:00
|
|
|
Return a new RDD by applying a function to each element of this RDD.
|
2014-06-09 03:08:40 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2014-03-19 17:04:45 -04:00
|
|
|
>>> rdd = sc.parallelize(["b", "a", "c"])
|
|
|
|
>>> sorted(rdd.map(lambda x: (x, 1)).collect())
|
|
|
|
[('a', 1), ('b', 1), ('c', 1)]
|
2012-12-27 20:55:33 -05:00
|
|
|
"""
|
[SPARK-2010] [PySpark] [SQL] support nested structure in SchemaRDD
Convert Row in JavaSchemaRDD into Array[Any] and unpickle them as tuple in Python, then convert them into namedtuple, so use can access fields just like attributes.
This will let nested structure can be accessed as object, also it will reduce the size of serialized data and better performance.
root
|-- field1: integer (nullable = true)
|-- field2: string (nullable = true)
|-- field3: struct (nullable = true)
| |-- field4: integer (nullable = true)
| |-- field5: array (nullable = true)
| | |-- element: integer (containsNull = false)
|-- field6: array (nullable = true)
| |-- element: struct (containsNull = false)
| | |-- field7: string (nullable = true)
Then we can access them by row.field3.field5[0] or row.field6[5].field7
It also will infer the schema in Python, convert Row/dict/namedtuple/objects into tuple before serialization, then call applySchema in JVM. During inferSchema(), the top level of dict in row will be StructType, but any nested dictionary will be MapType.
You can use pyspark.sql.Row to convert unnamed structure into Row object, make the RDD can be inferable. Such as:
ctx.inferSchema(rdd.map(lambda x: Row(a=x[0], b=x[1]))
Or you could use Row to create a class just like namedtuple, for example:
Person = Row("name", "age")
ctx.inferSchema(rdd.map(lambda x: Person(*x)))
Also, you can call applySchema to apply an schema to a RDD of tuple/list and turn it into a SchemaRDD. The `schema` should be StructType, see the API docs for details.
schema = StructType([StructField("name, StringType, True),
StructType("age", IntegerType, True)])
ctx.applySchema(rdd, schema)
PS: In order to use namedtuple to inferSchema, you should make namedtuple picklable.
Author: Davies Liu <davies.liu@gmail.com>
Closes #1598 from davies/nested and squashes the following commits:
f1d15b6 [Davies Liu] verify schema with the first few rows
8852aaf [Davies Liu] check type of schema
abe9e6e [Davies Liu] address comments
61b2292 [Davies Liu] add @deprecated to pythonToJavaMap
1e5b801 [Davies Liu] improve cache of classes
51aa135 [Davies Liu] use Row to infer schema
e9c0d5c [Davies Liu] remove string typed schema
353a3f2 [Davies Liu] fix code style
63de8f8 [Davies Liu] fix typo
c79ca67 [Davies Liu] fix serialization of nested data
6b258b5 [Davies Liu] fix pep8
9d8447c [Davies Liu] apply schema provided by string of names
f5df97f [Davies Liu] refactor, address comments
9d9af55 [Davies Liu] use arrry to applySchema and infer schema in Python
84679b3 [Davies Liu] Merge branch 'master' of github.com:apache/spark into nested
0eaaf56 [Davies Liu] fix doc tests
b3559b4 [Davies Liu] use generated Row instead of namedtuple
c4ddc30 [Davies Liu] fix conflict between name of fields and variables
7f6f251 [Davies Liu] address all comments
d69d397 [Davies Liu] refactor
2cc2d45 [Davies Liu] refactor
182fb46 [Davies Liu] refactor
bc6e9e1 [Davies Liu] switch to new Schema API
547bf3e [Davies Liu] Merge branch 'master' into nested
a435b5a [Davies Liu] add docs and code refactor
2c8debc [Davies Liu] Merge branch 'master' into nested
644665a [Davies Liu] use tuple and namedtuple for schemardd
2014-08-01 21:47:41 -04:00
|
|
|
def func(_, iterator):
|
2018-05-30 06:11:33 -04:00
|
|
|
return map(fail_on_stopiteration(f), iterator)
|
[SPARK-2010] [PySpark] [SQL] support nested structure in SchemaRDD
Convert Row in JavaSchemaRDD into Array[Any] and unpickle them as tuple in Python, then convert them into namedtuple, so use can access fields just like attributes.
This will let nested structure can be accessed as object, also it will reduce the size of serialized data and better performance.
root
|-- field1: integer (nullable = true)
|-- field2: string (nullable = true)
|-- field3: struct (nullable = true)
| |-- field4: integer (nullable = true)
| |-- field5: array (nullable = true)
| | |-- element: integer (containsNull = false)
|-- field6: array (nullable = true)
| |-- element: struct (containsNull = false)
| | |-- field7: string (nullable = true)
Then we can access them by row.field3.field5[0] or row.field6[5].field7
It also will infer the schema in Python, convert Row/dict/namedtuple/objects into tuple before serialization, then call applySchema in JVM. During inferSchema(), the top level of dict in row will be StructType, but any nested dictionary will be MapType.
You can use pyspark.sql.Row to convert unnamed structure into Row object, make the RDD can be inferable. Such as:
ctx.inferSchema(rdd.map(lambda x: Row(a=x[0], b=x[1]))
Or you could use Row to create a class just like namedtuple, for example:
Person = Row("name", "age")
ctx.inferSchema(rdd.map(lambda x: Person(*x)))
Also, you can call applySchema to apply an schema to a RDD of tuple/list and turn it into a SchemaRDD. The `schema` should be StructType, see the API docs for details.
schema = StructType([StructField("name, StringType, True),
StructType("age", IntegerType, True)])
ctx.applySchema(rdd, schema)
PS: In order to use namedtuple to inferSchema, you should make namedtuple picklable.
Author: Davies Liu <davies.liu@gmail.com>
Closes #1598 from davies/nested and squashes the following commits:
f1d15b6 [Davies Liu] verify schema with the first few rows
8852aaf [Davies Liu] check type of schema
abe9e6e [Davies Liu] address comments
61b2292 [Davies Liu] add @deprecated to pythonToJavaMap
1e5b801 [Davies Liu] improve cache of classes
51aa135 [Davies Liu] use Row to infer schema
e9c0d5c [Davies Liu] remove string typed schema
353a3f2 [Davies Liu] fix code style
63de8f8 [Davies Liu] fix typo
c79ca67 [Davies Liu] fix serialization of nested data
6b258b5 [Davies Liu] fix pep8
9d8447c [Davies Liu] apply schema provided by string of names
f5df97f [Davies Liu] refactor, address comments
9d9af55 [Davies Liu] use arrry to applySchema and infer schema in Python
84679b3 [Davies Liu] Merge branch 'master' of github.com:apache/spark into nested
0eaaf56 [Davies Liu] fix doc tests
b3559b4 [Davies Liu] use generated Row instead of namedtuple
c4ddc30 [Davies Liu] fix conflict between name of fields and variables
7f6f251 [Davies Liu] address all comments
d69d397 [Davies Liu] refactor
2cc2d45 [Davies Liu] refactor
182fb46 [Davies Liu] refactor
bc6e9e1 [Davies Liu] switch to new Schema API
547bf3e [Davies Liu] Merge branch 'master' into nested
a435b5a [Davies Liu] add docs and code refactor
2c8debc [Davies Liu] Merge branch 'master' into nested
644665a [Davies Liu] use tuple and namedtuple for schemardd
2014-08-01 21:47:41 -04:00
|
|
|
return self.mapPartitionsWithIndex(func, preservesPartitioning)
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2012-12-27 20:55:33 -05:00
|
|
|
def flatMap(self, f, preservesPartitioning=False):
|
2012-08-10 04:10:02 -04:00
|
|
|
"""
|
2012-12-27 20:55:33 -05:00
|
|
|
Return a new RDD by first applying a function to all elements of this
|
|
|
|
RDD, and then flattening the results.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2012-08-10 04:10:02 -04:00
|
|
|
>>> rdd = sc.parallelize([2, 3, 4])
|
|
|
|
>>> sorted(rdd.flatMap(lambda x: range(1, x)).collect())
|
|
|
|
[1, 1, 1, 2, 2, 3]
|
2012-08-18 19:07:10 -04:00
|
|
|
>>> sorted(rdd.flatMap(lambda x: [(x, x), (x, x)]).collect())
|
2012-08-10 04:10:02 -04:00
|
|
|
[(2, 2), (2, 2), (3, 3), (3, 3), (4, 4), (4, 4)]
|
|
|
|
"""
|
2014-07-14 03:42:59 -04:00
|
|
|
def func(s, iterator):
|
2018-05-30 06:11:33 -04:00
|
|
|
return chain.from_iterable(map(fail_on_stopiteration(f), iterator))
|
2014-01-23 23:01:36 -05:00
|
|
|
return self.mapPartitionsWithIndex(func, preservesPartitioning)
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2012-12-27 20:55:33 -05:00
|
|
|
def mapPartitions(self, f, preservesPartitioning=False):
|
2012-08-25 01:51:45 -04:00
|
|
|
"""
|
2012-12-27 20:55:33 -05:00
|
|
|
Return a new RDD by applying a function to each partition of this RDD.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2012-08-25 01:51:45 -04:00
|
|
|
>>> rdd = sc.parallelize([1, 2, 3, 4], 2)
|
|
|
|
>>> def f(iterator): yield sum(iterator)
|
|
|
|
>>> rdd.mapPartitions(f).collect()
|
|
|
|
[3, 7]
|
|
|
|
"""
|
2014-07-14 03:42:59 -04:00
|
|
|
def func(s, iterator):
|
|
|
|
return f(iterator)
|
2014-08-27 16:18:33 -04:00
|
|
|
return self.mapPartitionsWithIndex(func, preservesPartitioning)
|
2014-01-23 23:01:36 -05:00
|
|
|
|
|
|
|
def mapPartitionsWithIndex(self, f, preservesPartitioning=False):
|
|
|
|
"""
|
|
|
|
Return a new RDD by applying a function to each partition of this RDD,
|
|
|
|
while tracking the index of the original partition.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2014-01-23 23:01:36 -05:00
|
|
|
>>> rdd = sc.parallelize([1, 2, 3, 4], 4)
|
|
|
|
>>> def f(splitIndex, iterator): yield splitIndex
|
|
|
|
>>> rdd.mapPartitionsWithIndex(f).sum()
|
|
|
|
6
|
|
|
|
"""
|
|
|
|
return PipelinedRDD(self, f, preservesPartitioning)
|
2013-01-08 19:04:41 -05:00
|
|
|
|
|
|
|
def mapPartitionsWithSplit(self, f, preservesPartitioning=False):
|
|
|
|
"""
|
2014-01-23 23:01:36 -05:00
|
|
|
|
2013-01-08 19:04:41 -05:00
|
|
|
Return a new RDD by applying a function to each partition of this RDD,
|
|
|
|
while tracking the index of the original partition.
|
2012-12-27 20:55:33 -05:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
.. deprecated:: 0.9.0
|
|
|
|
use :py:meth:`RDD.mapPartitionsWithIndex` instead.
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2013-01-08 19:04:41 -05:00
|
|
|
>>> rdd = sc.parallelize([1, 2, 3, 4], 4)
|
|
|
|
>>> def f(splitIndex, iterator): yield splitIndex
|
|
|
|
>>> rdd.mapPartitionsWithSplit(f).sum()
|
|
|
|
6
|
|
|
|
"""
|
2014-01-23 23:01:36 -05:00
|
|
|
warnings.warn("mapPartitionsWithSplit is deprecated; "
|
2014-07-14 03:42:59 -04:00
|
|
|
"use mapPartitionsWithIndex instead", DeprecationWarning, stacklevel=2)
|
2014-01-23 23:01:36 -05:00
|
|
|
return self.mapPartitionsWithIndex(f, preservesPartitioning)
|
2012-08-25 01:51:45 -04:00
|
|
|
|
2014-06-09 03:08:40 -04:00
|
|
|
def getNumPartitions(self):
|
2014-07-14 03:42:59 -04:00
|
|
|
"""
|
|
|
|
Returns the number of partitions in RDD
|
2014-08-06 17:12:21 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2014-07-14 03:42:59 -04:00
|
|
|
>>> rdd = sc.parallelize([1, 2, 3, 4], 2)
|
|
|
|
>>> rdd.getNumPartitions()
|
|
|
|
2
|
|
|
|
"""
|
|
|
|
return self._jrdd.partitions().size()
|
2014-06-09 03:08:40 -04:00
|
|
|
|
2012-08-10 04:10:02 -04:00
|
|
|
def filter(self, f):
|
|
|
|
"""
|
2012-12-27 20:55:33 -05:00
|
|
|
Return a new RDD containing only the elements that satisfy a predicate.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2012-08-10 04:10:02 -04:00
|
|
|
>>> rdd = sc.parallelize([1, 2, 3, 4, 5])
|
|
|
|
>>> rdd.filter(lambda x: x % 2 == 0).collect()
|
|
|
|
[2, 4]
|
|
|
|
"""
|
2014-07-14 03:42:59 -04:00
|
|
|
def func(iterator):
|
2018-05-30 06:11:33 -04:00
|
|
|
return filter(fail_on_stopiteration(f), iterator)
|
2014-08-27 16:18:33 -04:00
|
|
|
return self.mapPartitions(func, True)
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2014-09-16 14:39:57 -04:00
|
|
|
def distinct(self, numPartitions=None):
|
2012-08-10 04:10:02 -04:00
|
|
|
"""
|
2012-12-27 20:55:33 -05:00
|
|
|
Return a new RDD containing the distinct elements in this RDD.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2012-08-10 04:10:02 -04:00
|
|
|
>>> sorted(sc.parallelize([1, 1, 2, 3]).distinct().collect())
|
|
|
|
[1, 2, 3]
|
|
|
|
"""
|
2013-07-28 23:50:38 -04:00
|
|
|
return self.map(lambda x: (x, None)) \
|
2014-09-16 14:39:57 -04:00
|
|
|
.reduceByKey(lambda x, _: x, numPartitions) \
|
2015-04-16 19:20:57 -04:00
|
|
|
.map(lambda x: x[0])
|
2012-08-10 04:10:02 -04:00
|
|
|
|
SPARK-1438 RDD.sample() make seed param optional
copying form previous pull request https://github.com/apache/spark/pull/462
Its probably better to let the underlying language implementation take care of the default . This was easier to do with python as the default value for seed in random and numpy random is None.
In Scala/Java side it might mean propagating an Option or null(oh no!) down the chain until where the Random is constructed. But, looks like the convention in some other methods was to use System.nanoTime. So, followed that convention.
Conflict with overloaded method in sql.SchemaRDD.sample which also defines default params.
sample(fraction, withReplacement=false, seed=math.random)
Scala does not allow more than one overloaded to have default params. I believe the author intended to override the RDD.sample method and not overload it. So, changed it.
If backward compatible is important, 3 new method can be introduced (without default params) like this
sample(fraction)
sample(fraction, withReplacement)
sample(fraction, withReplacement, seed)
Added some tests for the scala RDD takeSample method.
Author: Arun Ramakrishnan <smartnut007@gmail.com>
This patch had conflicts when merged, resolved by
Committer: Matei Zaharia <matei@databricks.com>
Closes #477 from smartnut007/master and squashes the following commits:
07bb06e [Arun Ramakrishnan] SPARK-1438 fixing more space formatting issues
b9ebfe2 [Arun Ramakrishnan] SPARK-1438 removing redundant import of random in python rddsampler
8d05b1a [Arun Ramakrishnan] SPARK-1438 RDD . Replace System.nanoTime with a Random generated number. python: use a separate instance of Random instead of seeding language api global Random instance.
69619c6 [Arun Ramakrishnan] SPARK-1438 fix spacing issue
0c247db [Arun Ramakrishnan] SPARK-1438 RDD language apis to support optional seed in RDD methods sample/takeSample
2014-04-24 20:27:16 -04:00
|
|
|
def sample(self, withReplacement, fraction, seed=None):
|
2013-08-23 14:16:44 -04:00
|
|
|
"""
|
[SPARK-4477] [PySpark] remove numpy from RDDSampler
In RDDSampler, it try use numpy to gain better performance for possion(), but the number of call of random() is only (1+faction) * N in the pure python implementation of possion(), so there is no much performance gain from numpy.
numpy is not a dependent of pyspark, so it maybe introduce some problem, such as there is no numpy installed in slaves, but only installed master, as reported in SPARK-927.
It also complicate the code a lot, so we may should remove numpy from RDDSampler.
I also did some benchmark to verify that:
```
>>> from pyspark.mllib.random import RandomRDDs
>>> rdd = RandomRDDs.uniformRDD(sc, 1 << 20, 1).cache()
>>> rdd.count() # cache it
>>> rdd.sample(True, 0.9).count() # measure this line
```
the results:
|withReplacement | random | numpy.random |
------- | ------------ | -------
|True | 1.5 s| 1.4 s|
|False| 0.6 s | 0.8 s|
closes #2313
Note: this patch including some commits that not mirrored to github, it will be OK after it catches up.
Author: Davies Liu <davies@databricks.com>
Author: Xiangrui Meng <meng@databricks.com>
Closes #3351 from davies/numpy and squashes the following commits:
5c438d7 [Davies Liu] fix comment
c5b9252 [Davies Liu] Merge pull request #1 from mengxr/SPARK-4477
98eb31b [Xiangrui Meng] make poisson sampling slightly faster
ee17d78 [Davies Liu] remove = for float
13f7b05 [Davies Liu] Merge branch 'master' of http://git-wip-us.apache.org/repos/asf/spark into numpy
f583023 [Davies Liu] fix tests
51649f5 [Davies Liu] remove numpy in RDDSampler
78bf997 [Davies Liu] fix tests, do not use numpy in randomSplit, no performance gain
f5fdf63 [Davies Liu] fix bug with int in weights
4dfa2cd [Davies Liu] refactor
f866bcf [Davies Liu] remove unneeded change
c7a2007 [Davies Liu] switch to python implementation
95a48ac [Davies Liu] Merge branch 'master' of github.com:apache/spark into randomSplit
0d9b256 [Davies Liu] refactor
1715ee3 [Davies Liu] address comments
41fce54 [Davies Liu] randomSplit()
2014-11-20 19:40:25 -05:00
|
|
|
Return a sampled subset of this RDD.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
withReplacement : bool
|
|
|
|
can elements be sampled multiple times (replaced when sampled out)
|
|
|
|
fraction : float
|
|
|
|
expected size of the sample as a fraction of this RDD's size
|
2015-03-20 14:30:45 -04:00
|
|
|
without replacement: probability that each element is chosen; fraction must be [0, 1]
|
|
|
|
with replacement: expected number of times each element is chosen; fraction must be >= 0
|
2020-11-15 20:21:50 -05:00
|
|
|
seed : int, optional
|
|
|
|
seed for the random number generator
|
2015-03-20 14:30:45 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This is not guaranteed to provide exactly the fraction specified of the total
|
|
|
|
count of the given :class:`DataFrame`.
|
2016-11-17 06:34:55 -05:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
[SPARK-4477] [PySpark] remove numpy from RDDSampler
In RDDSampler, it try use numpy to gain better performance for possion(), but the number of call of random() is only (1+faction) * N in the pure python implementation of possion(), so there is no much performance gain from numpy.
numpy is not a dependent of pyspark, so it maybe introduce some problem, such as there is no numpy installed in slaves, but only installed master, as reported in SPARK-927.
It also complicate the code a lot, so we may should remove numpy from RDDSampler.
I also did some benchmark to verify that:
```
>>> from pyspark.mllib.random import RandomRDDs
>>> rdd = RandomRDDs.uniformRDD(sc, 1 << 20, 1).cache()
>>> rdd.count() # cache it
>>> rdd.sample(True, 0.9).count() # measure this line
```
the results:
|withReplacement | random | numpy.random |
------- | ------------ | -------
|True | 1.5 s| 1.4 s|
|False| 0.6 s | 0.8 s|
closes #2313
Note: this patch including some commits that not mirrored to github, it will be OK after it catches up.
Author: Davies Liu <davies@databricks.com>
Author: Xiangrui Meng <meng@databricks.com>
Closes #3351 from davies/numpy and squashes the following commits:
5c438d7 [Davies Liu] fix comment
c5b9252 [Davies Liu] Merge pull request #1 from mengxr/SPARK-4477
98eb31b [Xiangrui Meng] make poisson sampling slightly faster
ee17d78 [Davies Liu] remove = for float
13f7b05 [Davies Liu] Merge branch 'master' of http://git-wip-us.apache.org/repos/asf/spark into numpy
f583023 [Davies Liu] fix tests
51649f5 [Davies Liu] remove numpy in RDDSampler
78bf997 [Davies Liu] fix tests, do not use numpy in randomSplit, no performance gain
f5fdf63 [Davies Liu] fix bug with int in weights
4dfa2cd [Davies Liu] refactor
f866bcf [Davies Liu] remove unneeded change
c7a2007 [Davies Liu] switch to python implementation
95a48ac [Davies Liu] Merge branch 'master' of github.com:apache/spark into randomSplit
0d9b256 [Davies Liu] refactor
1715ee3 [Davies Liu] address comments
41fce54 [Davies Liu] randomSplit()
2014-11-20 19:40:25 -05:00
|
|
|
>>> rdd = sc.parallelize(range(100), 4)
|
2015-04-16 19:20:57 -04:00
|
|
|
>>> 6 <= rdd.sample(False, 0.1, 81).count() <= 14
|
|
|
|
True
|
2013-08-23 14:16:44 -04:00
|
|
|
"""
|
2014-07-25 02:42:08 -04:00
|
|
|
assert fraction >= 0.0, "Negative fraction value: %s" % fraction
|
2014-01-23 23:01:36 -05:00
|
|
|
return self.mapPartitionsWithIndex(RDDSampler(withReplacement, fraction, seed).func, True)
|
2013-08-23 14:16:44 -04:00
|
|
|
|
[SPARK-4327] [PySpark] Python API for RDD.randomSplit()
```
pyspark.RDD.randomSplit(self, weights, seed=None)
Randomly splits this RDD with the provided weights.
:param weights: weights for splits, will be normalized if they don't sum to 1
:param seed: random seed
:return: split RDDs in an list
>>> rdd = sc.parallelize(range(10), 1)
>>> rdd1, rdd2, rdd3 = rdd.randomSplit([0.4, 0.6, 1.0], 11)
>>> rdd1.collect()
[3, 6]
>>> rdd2.collect()
[0, 5, 7]
>>> rdd3.collect()
[1, 2, 4, 8, 9]
```
Author: Davies Liu <davies@databricks.com>
Closes #3193 from davies/randomSplit and squashes the following commits:
78bf997 [Davies Liu] fix tests, do not use numpy in randomSplit, no performance gain
f5fdf63 [Davies Liu] fix bug with int in weights
4dfa2cd [Davies Liu] refactor
f866bcf [Davies Liu] remove unneeded change
c7a2007 [Davies Liu] switch to python implementation
95a48ac [Davies Liu] Merge branch 'master' of github.com:apache/spark into randomSplit
0d9b256 [Davies Liu] refactor
1715ee3 [Davies Liu] address comments
41fce54 [Davies Liu] randomSplit()
2014-11-18 19:37:35 -05:00
|
|
|
def randomSplit(self, weights, seed=None):
|
|
|
|
"""
|
|
|
|
Randomly splits this RDD with the provided weights.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
weights : list
|
|
|
|
weights for splits, will be normalized if they don't sum to 1
|
|
|
|
seed : int, optional
|
|
|
|
random seed
|
[SPARK-4327] [PySpark] Python API for RDD.randomSplit()
```
pyspark.RDD.randomSplit(self, weights, seed=None)
Randomly splits this RDD with the provided weights.
:param weights: weights for splits, will be normalized if they don't sum to 1
:param seed: random seed
:return: split RDDs in an list
>>> rdd = sc.parallelize(range(10), 1)
>>> rdd1, rdd2, rdd3 = rdd.randomSplit([0.4, 0.6, 1.0], 11)
>>> rdd1.collect()
[3, 6]
>>> rdd2.collect()
[0, 5, 7]
>>> rdd3.collect()
[1, 2, 4, 8, 9]
```
Author: Davies Liu <davies@databricks.com>
Closes #3193 from davies/randomSplit and squashes the following commits:
78bf997 [Davies Liu] fix tests, do not use numpy in randomSplit, no performance gain
f5fdf63 [Davies Liu] fix bug with int in weights
4dfa2cd [Davies Liu] refactor
f866bcf [Davies Liu] remove unneeded change
c7a2007 [Davies Liu] switch to python implementation
95a48ac [Davies Liu] Merge branch 'master' of github.com:apache/spark into randomSplit
0d9b256 [Davies Liu] refactor
1715ee3 [Davies Liu] address comments
41fce54 [Davies Liu] randomSplit()
2014-11-18 19:37:35 -05:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Returns
|
|
|
|
-------
|
|
|
|
list
|
|
|
|
split RDDs in a list
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2015-04-16 19:20:57 -04:00
|
|
|
>>> rdd = sc.parallelize(range(500), 1)
|
[SPARK-4327] [PySpark] Python API for RDD.randomSplit()
```
pyspark.RDD.randomSplit(self, weights, seed=None)
Randomly splits this RDD with the provided weights.
:param weights: weights for splits, will be normalized if they don't sum to 1
:param seed: random seed
:return: split RDDs in an list
>>> rdd = sc.parallelize(range(10), 1)
>>> rdd1, rdd2, rdd3 = rdd.randomSplit([0.4, 0.6, 1.0], 11)
>>> rdd1.collect()
[3, 6]
>>> rdd2.collect()
[0, 5, 7]
>>> rdd3.collect()
[1, 2, 4, 8, 9]
```
Author: Davies Liu <davies@databricks.com>
Closes #3193 from davies/randomSplit and squashes the following commits:
78bf997 [Davies Liu] fix tests, do not use numpy in randomSplit, no performance gain
f5fdf63 [Davies Liu] fix bug with int in weights
4dfa2cd [Davies Liu] refactor
f866bcf [Davies Liu] remove unneeded change
c7a2007 [Davies Liu] switch to python implementation
95a48ac [Davies Liu] Merge branch 'master' of github.com:apache/spark into randomSplit
0d9b256 [Davies Liu] refactor
1715ee3 [Davies Liu] address comments
41fce54 [Davies Liu] randomSplit()
2014-11-18 19:37:35 -05:00
|
|
|
>>> rdd1, rdd2 = rdd.randomSplit([2, 3], 17)
|
2015-04-16 19:20:57 -04:00
|
|
|
>>> len(rdd1.collect() + rdd2.collect())
|
|
|
|
500
|
|
|
|
>>> 150 < rdd1.count() < 250
|
|
|
|
True
|
|
|
|
>>> 250 < rdd2.count() < 350
|
|
|
|
True
|
[SPARK-4327] [PySpark] Python API for RDD.randomSplit()
```
pyspark.RDD.randomSplit(self, weights, seed=None)
Randomly splits this RDD with the provided weights.
:param weights: weights for splits, will be normalized if they don't sum to 1
:param seed: random seed
:return: split RDDs in an list
>>> rdd = sc.parallelize(range(10), 1)
>>> rdd1, rdd2, rdd3 = rdd.randomSplit([0.4, 0.6, 1.0], 11)
>>> rdd1.collect()
[3, 6]
>>> rdd2.collect()
[0, 5, 7]
>>> rdd3.collect()
[1, 2, 4, 8, 9]
```
Author: Davies Liu <davies@databricks.com>
Closes #3193 from davies/randomSplit and squashes the following commits:
78bf997 [Davies Liu] fix tests, do not use numpy in randomSplit, no performance gain
f5fdf63 [Davies Liu] fix bug with int in weights
4dfa2cd [Davies Liu] refactor
f866bcf [Davies Liu] remove unneeded change
c7a2007 [Davies Liu] switch to python implementation
95a48ac [Davies Liu] Merge branch 'master' of github.com:apache/spark into randomSplit
0d9b256 [Davies Liu] refactor
1715ee3 [Davies Liu] address comments
41fce54 [Davies Liu] randomSplit()
2014-11-18 19:37:35 -05:00
|
|
|
"""
|
|
|
|
s = float(sum(weights))
|
|
|
|
cweights = [0.0]
|
|
|
|
for w in weights:
|
|
|
|
cweights.append(cweights[-1] + w / s)
|
|
|
|
if seed is None:
|
|
|
|
seed = random.randint(0, 2 ** 32 - 1)
|
|
|
|
return [self.mapPartitionsWithIndex(RDDRangeSampler(lb, ub, seed).func, True)
|
|
|
|
for lb, ub in zip(cweights, cweights[1:])]
|
|
|
|
|
2013-08-23 14:16:44 -04:00
|
|
|
# this is ported from scala/spark/RDD.scala
|
SPARK-1438 RDD.sample() make seed param optional
copying form previous pull request https://github.com/apache/spark/pull/462
Its probably better to let the underlying language implementation take care of the default . This was easier to do with python as the default value for seed in random and numpy random is None.
In Scala/Java side it might mean propagating an Option or null(oh no!) down the chain until where the Random is constructed. But, looks like the convention in some other methods was to use System.nanoTime. So, followed that convention.
Conflict with overloaded method in sql.SchemaRDD.sample which also defines default params.
sample(fraction, withReplacement=false, seed=math.random)
Scala does not allow more than one overloaded to have default params. I believe the author intended to override the RDD.sample method and not overload it. So, changed it.
If backward compatible is important, 3 new method can be introduced (without default params) like this
sample(fraction)
sample(fraction, withReplacement)
sample(fraction, withReplacement, seed)
Added some tests for the scala RDD takeSample method.
Author: Arun Ramakrishnan <smartnut007@gmail.com>
This patch had conflicts when merged, resolved by
Committer: Matei Zaharia <matei@databricks.com>
Closes #477 from smartnut007/master and squashes the following commits:
07bb06e [Arun Ramakrishnan] SPARK-1438 fixing more space formatting issues
b9ebfe2 [Arun Ramakrishnan] SPARK-1438 removing redundant import of random in python rddsampler
8d05b1a [Arun Ramakrishnan] SPARK-1438 RDD . Replace System.nanoTime with a Random generated number. python: use a separate instance of Random instead of seeding language api global Random instance.
69619c6 [Arun Ramakrishnan] SPARK-1438 fix spacing issue
0c247db [Arun Ramakrishnan] SPARK-1438 RDD language apis to support optional seed in RDD methods sample/takeSample
2014-04-24 20:27:16 -04:00
|
|
|
def takeSample(self, withReplacement, num, seed=None):
|
2013-08-23 14:16:44 -04:00
|
|
|
"""
|
[SPARK-4477] [PySpark] remove numpy from RDDSampler
In RDDSampler, it try use numpy to gain better performance for possion(), but the number of call of random() is only (1+faction) * N in the pure python implementation of possion(), so there is no much performance gain from numpy.
numpy is not a dependent of pyspark, so it maybe introduce some problem, such as there is no numpy installed in slaves, but only installed master, as reported in SPARK-927.
It also complicate the code a lot, so we may should remove numpy from RDDSampler.
I also did some benchmark to verify that:
```
>>> from pyspark.mllib.random import RandomRDDs
>>> rdd = RandomRDDs.uniformRDD(sc, 1 << 20, 1).cache()
>>> rdd.count() # cache it
>>> rdd.sample(True, 0.9).count() # measure this line
```
the results:
|withReplacement | random | numpy.random |
------- | ------------ | -------
|True | 1.5 s| 1.4 s|
|False| 0.6 s | 0.8 s|
closes #2313
Note: this patch including some commits that not mirrored to github, it will be OK after it catches up.
Author: Davies Liu <davies@databricks.com>
Author: Xiangrui Meng <meng@databricks.com>
Closes #3351 from davies/numpy and squashes the following commits:
5c438d7 [Davies Liu] fix comment
c5b9252 [Davies Liu] Merge pull request #1 from mengxr/SPARK-4477
98eb31b [Xiangrui Meng] make poisson sampling slightly faster
ee17d78 [Davies Liu] remove = for float
13f7b05 [Davies Liu] Merge branch 'master' of http://git-wip-us.apache.org/repos/asf/spark into numpy
f583023 [Davies Liu] fix tests
51649f5 [Davies Liu] remove numpy in RDDSampler
78bf997 [Davies Liu] fix tests, do not use numpy in randomSplit, no performance gain
f5fdf63 [Davies Liu] fix bug with int in weights
4dfa2cd [Davies Liu] refactor
f866bcf [Davies Liu] remove unneeded change
c7a2007 [Davies Liu] switch to python implementation
95a48ac [Davies Liu] Merge branch 'master' of github.com:apache/spark into randomSplit
0d9b256 [Davies Liu] refactor
1715ee3 [Davies Liu] address comments
41fce54 [Davies Liu] randomSplit()
2014-11-20 19:40:25 -05:00
|
|
|
Return a fixed-size sampled subset of this RDD.
|
2013-08-23 14:16:44 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This method should only be used if the resulting array is expected
|
|
|
|
to be small, as all the data is loaded into the driver's memory.
|
2016-02-06 12:29:09 -05:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2014-06-12 22:44:27 -04:00
|
|
|
>>> rdd = sc.parallelize(range(0, 10))
|
|
|
|
>>> len(rdd.takeSample(True, 20, 1))
|
|
|
|
20
|
|
|
|
>>> len(rdd.takeSample(False, 5, 2))
|
|
|
|
5
|
|
|
|
>>> len(rdd.takeSample(False, 15, 3))
|
|
|
|
10
|
2013-08-23 14:16:44 -04:00
|
|
|
"""
|
2014-06-12 22:44:27 -04:00
|
|
|
numStDev = 10.0
|
|
|
|
|
|
|
|
if num < 0:
|
|
|
|
raise ValueError("Sample size cannot be negative.")
|
|
|
|
elif num == 0:
|
|
|
|
return []
|
2013-08-23 14:16:44 -04:00
|
|
|
|
|
|
|
initialCount = self.count()
|
2014-06-12 22:44:27 -04:00
|
|
|
if initialCount == 0:
|
|
|
|
return []
|
2013-08-23 14:16:44 -04:00
|
|
|
|
[SPARK-4327] [PySpark] Python API for RDD.randomSplit()
```
pyspark.RDD.randomSplit(self, weights, seed=None)
Randomly splits this RDD with the provided weights.
:param weights: weights for splits, will be normalized if they don't sum to 1
:param seed: random seed
:return: split RDDs in an list
>>> rdd = sc.parallelize(range(10), 1)
>>> rdd1, rdd2, rdd3 = rdd.randomSplit([0.4, 0.6, 1.0], 11)
>>> rdd1.collect()
[3, 6]
>>> rdd2.collect()
[0, 5, 7]
>>> rdd3.collect()
[1, 2, 4, 8, 9]
```
Author: Davies Liu <davies@databricks.com>
Closes #3193 from davies/randomSplit and squashes the following commits:
78bf997 [Davies Liu] fix tests, do not use numpy in randomSplit, no performance gain
f5fdf63 [Davies Liu] fix bug with int in weights
4dfa2cd [Davies Liu] refactor
f866bcf [Davies Liu] remove unneeded change
c7a2007 [Davies Liu] switch to python implementation
95a48ac [Davies Liu] Merge branch 'master' of github.com:apache/spark into randomSplit
0d9b256 [Davies Liu] refactor
1715ee3 [Davies Liu] address comments
41fce54 [Davies Liu] randomSplit()
2014-11-18 19:37:35 -05:00
|
|
|
rand = random.Random(seed)
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2014-06-12 22:44:27 -04:00
|
|
|
if (not withReplacement) and num >= initialCount:
|
|
|
|
# shuffle current RDD and return
|
|
|
|
samples = self.collect()
|
|
|
|
rand.shuffle(samples)
|
|
|
|
return samples
|
2014-03-17 01:14:59 -04:00
|
|
|
|
2015-04-16 19:20:57 -04:00
|
|
|
maxSampleSize = sys.maxsize - int(numStDev * sqrt(sys.maxsize))
|
2014-06-12 22:44:27 -04:00
|
|
|
if num > maxSampleSize:
|
2014-07-14 03:42:59 -04:00
|
|
|
raise ValueError(
|
|
|
|
"Sample size cannot be greater than %d." % maxSampleSize)
|
2013-08-23 14:16:44 -04:00
|
|
|
|
2014-07-14 03:42:59 -04:00
|
|
|
fraction = RDD._computeFractionForSampleSize(
|
|
|
|
num, initialCount, withReplacement)
|
2013-08-23 14:16:44 -04:00
|
|
|
samples = self.sample(withReplacement, fraction, seed).collect()
|
2013-10-09 15:08:04 -04:00
|
|
|
|
2013-08-23 14:16:44 -04:00
|
|
|
# If the first sample didn't turn out large enough, keep trying to take samples;
|
|
|
|
# this shouldn't happen often because we use a big multiplier for their initial size.
|
|
|
|
# See: scala/spark/RDD.scala
|
2014-06-12 22:44:27 -04:00
|
|
|
while len(samples) < num:
|
|
|
|
# TODO: add log warning for when more than one iteration was run
|
2015-04-16 19:20:57 -04:00
|
|
|
seed = rand.randint(0, sys.maxsize)
|
2014-06-12 22:44:27 -04:00
|
|
|
samples = self.sample(withReplacement, fraction, seed).collect()
|
|
|
|
|
|
|
|
rand.shuffle(samples)
|
|
|
|
|
|
|
|
return samples[0:num]
|
|
|
|
|
|
|
|
@staticmethod
|
|
|
|
def _computeFractionForSampleSize(sampleSizeLowerBound, total, withReplacement):
|
|
|
|
"""
|
|
|
|
Returns a sampling rate that guarantees a sample of
|
|
|
|
size >= sampleSizeLowerBound 99.99% of the time.
|
|
|
|
|
|
|
|
How the sampling rate is determined:
|
|
|
|
Let p = num / total, where num is the sample size and total is the
|
|
|
|
total number of data points in the RDD. We're trying to compute
|
|
|
|
q > p such that
|
|
|
|
- when sampling with replacement, we're drawing each data point
|
|
|
|
with prob_i ~ Pois(q), where we want to guarantee
|
|
|
|
Pr[s < num] < 0.0001 for s = sum(prob_i for i from 0 to
|
|
|
|
total), i.e. the failure rate of not having a sufficiently large
|
|
|
|
sample < 0.0001. Setting q = p + 5 * sqrt(p/total) is sufficient
|
|
|
|
to guarantee 0.9999 success rate for num > 12, but we need a
|
|
|
|
slightly larger q (9 empirically determined).
|
|
|
|
- when sampling without replacement, we're drawing each data point
|
|
|
|
with prob_i ~ Binomial(total, fraction) and our choice of q
|
|
|
|
guarantees 1-delta, or 0.9999 success rate, where success rate is
|
|
|
|
defined the same as in sampling with replacement.
|
|
|
|
"""
|
|
|
|
fraction = float(sampleSizeLowerBound) / total
|
|
|
|
if withReplacement:
|
|
|
|
numStDev = 5
|
|
|
|
if (sampleSizeLowerBound < 12):
|
|
|
|
numStDev = 9
|
|
|
|
return fraction + numStDev * sqrt(fraction / total)
|
|
|
|
else:
|
|
|
|
delta = 0.00005
|
|
|
|
gamma = - log(delta) / total
|
|
|
|
return min(1, fraction + gamma + sqrt(gamma * gamma + 2 * gamma * fraction))
|
2012-08-10 04:10:02 -04:00
|
|
|
|
|
|
|
def union(self, other):
|
|
|
|
"""
|
2012-12-27 20:55:33 -05:00
|
|
|
Return the union of this RDD and another one.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2012-08-10 04:10:02 -04:00
|
|
|
>>> rdd = sc.parallelize([1, 1, 2, 3])
|
|
|
|
>>> rdd.union(rdd).collect()
|
|
|
|
[1, 1, 2, 3, 1, 1, 2, 3]
|
|
|
|
"""
|
2013-11-05 20:52:39 -05:00
|
|
|
if self._jrdd_deserializer == other._jrdd_deserializer:
|
|
|
|
rdd = RDD(self._jrdd.union(other._jrdd), self.ctx,
|
|
|
|
self._jrdd_deserializer)
|
|
|
|
else:
|
|
|
|
# These RDDs contain data in different serialized formats, so we
|
|
|
|
# must normalize them to the default serializer.
|
|
|
|
self_copy = self._reserialize()
|
|
|
|
other_copy = other._reserialize()
|
2015-02-17 19:54:57 -05:00
|
|
|
rdd = RDD(self_copy._jrdd.union(other_copy._jrdd), self.ctx,
|
|
|
|
self.ctx.serializer)
|
|
|
|
if (self.partitioner == other.partitioner and
|
|
|
|
self.getNumPartitions() == rdd.getNumPartitions()):
|
|
|
|
rdd.partitioner = self.partitioner
|
|
|
|
return rdd
|
2013-11-05 20:52:39 -05:00
|
|
|
|
2014-03-07 21:48:07 -05:00
|
|
|
def intersection(self, other):
|
|
|
|
"""
|
2014-06-12 22:44:27 -04:00
|
|
|
Return the intersection of this RDD and another one. The output will
|
|
|
|
not contain any duplicate elements, even if the input RDDs did.
|
2014-06-09 03:08:40 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This method performs a shuffle internally.
|
2014-03-07 21:48:07 -05:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2014-03-07 21:48:07 -05:00
|
|
|
>>> rdd1 = sc.parallelize([1, 10, 2, 3, 4, 5])
|
|
|
|
>>> rdd2 = sc.parallelize([1, 6, 2, 3, 7, 8])
|
|
|
|
>>> rdd1.intersection(rdd2).collect()
|
|
|
|
[1, 2, 3]
|
|
|
|
"""
|
|
|
|
return self.map(lambda v: (v, None)) \
|
|
|
|
.cogroup(other.map(lambda v: (v, None))) \
|
2015-04-16 19:20:57 -04:00
|
|
|
.filter(lambda k_vs: all(k_vs[1])) \
|
2014-03-07 21:48:07 -05:00
|
|
|
.keys()
|
|
|
|
|
2014-06-03 21:18:25 -04:00
|
|
|
def _reserialize(self, serializer=None):
|
|
|
|
serializer = serializer or self.ctx.serializer
|
2014-11-04 02:56:14 -05:00
|
|
|
if self._jrdd_deserializer != serializer:
|
2014-12-16 01:58:26 -05:00
|
|
|
self = self.map(lambda x: x, preservesPartitioning=True)
|
2014-11-04 02:56:14 -05:00
|
|
|
self._jrdd_deserializer = serializer
|
|
|
|
return self
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2012-08-25 02:09:15 -04:00
|
|
|
def __add__(self, other):
|
|
|
|
"""
|
2012-12-27 20:55:33 -05:00
|
|
|
Return the union of this RDD and another one.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2012-08-25 02:09:15 -04:00
|
|
|
>>> rdd = sc.parallelize([1, 1, 2, 3])
|
|
|
|
>>> (rdd + rdd).collect()
|
|
|
|
[1, 1, 2, 3, 1, 1, 2, 3]
|
|
|
|
"""
|
|
|
|
if not isinstance(other, RDD):
|
|
|
|
raise TypeError
|
|
|
|
return self.union(other)
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2014-09-08 14:20:00 -04:00
|
|
|
def repartitionAndSortWithinPartitions(self, numPartitions=None, partitionFunc=portable_hash,
|
|
|
|
ascending=True, keyfunc=lambda x: x):
|
|
|
|
"""
|
|
|
|
Repartition the RDD according to the given partitioner and, within each resulting partition,
|
|
|
|
sort records by their keys.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2014-09-08 14:20:00 -04:00
|
|
|
>>> rdd = sc.parallelize([(0, 5), (3, 8), (2, 6), (0, 8), (3, 8), (1, 3)])
|
2017-07-10 21:56:54 -04:00
|
|
|
>>> rdd2 = rdd.repartitionAndSortWithinPartitions(2, lambda x: x % 2, True)
|
2014-09-08 14:20:00 -04:00
|
|
|
>>> rdd2.glom().collect()
|
|
|
|
[[(0, 5), (0, 8), (2, 6)], [(1, 3), (3, 8), (3, 8)]]
|
|
|
|
"""
|
|
|
|
if numPartitions is None:
|
|
|
|
numPartitions = self._defaultReducePartitions()
|
|
|
|
|
2019-07-27 14:58:50 -04:00
|
|
|
memory = self._memory_limit()
|
2014-09-08 14:20:00 -04:00
|
|
|
serializer = self._jrdd_deserializer
|
|
|
|
|
|
|
|
def sortPartition(iterator):
|
2015-09-20 00:40:21 -04:00
|
|
|
sort = ExternalSorter(memory * 0.9, serializer).sorted
|
2015-04-16 19:20:57 -04:00
|
|
|
return iter(sort(iterator, key=lambda k_v: keyfunc(k_v[0]), reverse=(not ascending)))
|
2014-09-08 14:20:00 -04:00
|
|
|
|
|
|
|
return self.partitionBy(numPartitions, partitionFunc).mapPartitions(sortPartition, True)
|
|
|
|
|
2014-07-14 03:42:59 -04:00
|
|
|
def sortByKey(self, ascending=True, numPartitions=None, keyfunc=lambda x: x):
|
2013-10-07 13:42:39 -04:00
|
|
|
"""
|
|
|
|
Sorts this RDD, which is assumed to consist of (key, value) pairs.
|
2014-08-06 17:12:21 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2013-10-07 13:42:39 -04:00
|
|
|
>>> tmp = [('a', 1), ('b', 2), ('1', 3), ('d', 4), ('2', 5)]
|
2014-08-20 01:43:49 -04:00
|
|
|
>>> sc.parallelize(tmp).sortByKey().first()
|
|
|
|
('1', 3)
|
2014-08-13 17:57:12 -04:00
|
|
|
>>> sc.parallelize(tmp).sortByKey(True, 1).collect()
|
|
|
|
[('1', 3), ('2', 5), ('a', 1), ('b', 2), ('d', 4)]
|
2013-10-07 13:42:39 -04:00
|
|
|
>>> sc.parallelize(tmp).sortByKey(True, 2).collect()
|
|
|
|
[('1', 3), ('2', 5), ('a', 1), ('b', 2), ('d', 4)]
|
|
|
|
>>> tmp2 = [('Mary', 1), ('had', 2), ('a', 3), ('little', 4), ('lamb', 5)]
|
|
|
|
>>> tmp2.extend([('whose', 6), ('fleece', 7), ('was', 8), ('white', 9)])
|
|
|
|
>>> sc.parallelize(tmp2).sortByKey(True, 3, keyfunc=lambda k: k.lower()).collect()
|
2014-07-14 03:42:59 -04:00
|
|
|
[('a', 3), ('fleece', 7), ('had', 2), ('lamb', 5),...('white', 9), ('whose', 6)]
|
2013-10-07 13:42:39 -04:00
|
|
|
"""
|
|
|
|
if numPartitions is None:
|
2014-06-20 03:06:57 -04:00
|
|
|
numPartitions = self._defaultReducePartitions()
|
2013-10-07 13:42:39 -04:00
|
|
|
|
[SPARK-3074] [PySpark] support groupByKey() with single huge key
This patch change groupByKey() to use external sort based approach, so it can support single huge key.
For example, it can group by a dataset including one hot key with 40 millions values (strings), using 500M memory for Python worker, finished in about 2 minutes. (it will need 6G memory in hash based approach).
During groupByKey(), it will do in-memory groupBy first. If the dataset can not fit in memory, then data will be partitioned by hash. If one partition still can not fit in memory, it will switch to sort based groupBy().
Author: Davies Liu <davies.liu@gmail.com>
Author: Davies Liu <davies@databricks.com>
Closes #1977 from davies/groupby and squashes the following commits:
af3713a [Davies Liu] make sure it's iterator
67772dd [Davies Liu] fix tests
e78c15c [Davies Liu] address comments
0b0fde8 [Davies Liu] address comments
0dcf320 [Davies Liu] address comments, rollback changes in ResultIterable
e3b8eab [Davies Liu] fix narrow dependency
2a1857a [Davies Liu] typo
d2f053b [Davies Liu] add repr for FlattedValuesSerializer
c6a2f8d [Davies Liu] address comments
9e2df24 [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
2b9c261 [Davies Liu] fix typo in comments
70aadcd [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
a14b4bd [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
ab5515b [Davies Liu] Merge branch 'master' into groupby
651f891 [Davies Liu] simplify GroupByKey
1578f2e [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
1f69f93 [Davies Liu] fix tests
0d3395f [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
341f1e0 [Davies Liu] add comments, refactor
47918b8 [Davies Liu] remove unused code
6540948 [Davies Liu] address comments:
17f4ec6 [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
4d4bc86 [Davies Liu] bugfix
8ef965e [Davies Liu] Merge branch 'master' into groupby
fbc504a [Davies Liu] Merge branch 'master' into groupby
779ed03 [Davies Liu] fix merge conflict
2c1d05b [Davies Liu] refactor, minor turning
b48cda5 [Davies Liu] Merge branch 'master' into groupby
85138e6 [Davies Liu] Merge branch 'master' into groupby
acd8e1b [Davies Liu] fix memory when groupByKey().count()
905b233 [Davies Liu] Merge branch 'sort' into groupby
1f075ed [Davies Liu] Merge branch 'master' into sort
4b07d39 [Davies Liu] compress the data while spilling
0a081c6 [Davies Liu] Merge branch 'master' into groupby
f157fe7 [Davies Liu] Merge branch 'sort' into groupby
eb53ca6 [Davies Liu] Merge branch 'master' into sort
b2dc3bf [Davies Liu] Merge branch 'sort' into groupby
644abaf [Davies Liu] add license in LICENSE
19f7873 [Davies Liu] improve tests
11ba318 [Davies Liu] typo
085aef8 [Davies Liu] Merge branch 'master' into groupby
3ee58e5 [Davies Liu] switch to sort based groupBy, based on size of data
1ea0669 [Davies Liu] choose sort based groupByKey() automatically
b40bae7 [Davies Liu] bugfix
efa23df [Davies Liu] refactor, add spark.shuffle.sort=False
250be4e [Davies Liu] flatten the combined values when dumping into disks
d05060d [Davies Liu] group the same key before shuffle, reduce the comparison during sorting
083d842 [Davies Liu] sorted based groupByKey()
55602ee [Davies Liu] use external sort in sortBy() and sortByKey()
2015-04-09 20:07:23 -04:00
|
|
|
memory = self._memory_limit()
|
2014-08-26 19:57:40 -04:00
|
|
|
serializer = self._jrdd_deserializer
|
|
|
|
|
2014-08-20 01:43:49 -04:00
|
|
|
def sortPartition(iterator):
|
2015-09-20 00:40:21 -04:00
|
|
|
sort = ExternalSorter(memory * 0.9, serializer).sorted
|
2015-04-16 19:20:57 -04:00
|
|
|
return iter(sort(iterator, key=lambda kv: keyfunc(kv[0]), reverse=(not ascending)))
|
2014-08-20 01:43:49 -04:00
|
|
|
|
2014-08-13 17:57:12 -04:00
|
|
|
if numPartitions == 1:
|
|
|
|
if self.getNumPartitions() > 1:
|
|
|
|
self = self.coalesce(1)
|
2014-08-27 16:18:33 -04:00
|
|
|
return self.mapPartitions(sortPartition, True)
|
2013-10-07 13:42:39 -04:00
|
|
|
|
|
|
|
# first compute the boundary of each part via sampling: we want to partition
|
|
|
|
# the key-space into bins such that the bins have roughly the same
|
|
|
|
# number of (key, value) pairs falling into them
|
2014-08-13 17:57:12 -04:00
|
|
|
rddSize = self.count()
|
2014-11-07 23:53:03 -05:00
|
|
|
if not rddSize:
|
|
|
|
return self # empty RDD
|
2014-08-13 17:57:12 -04:00
|
|
|
maxSampleSize = numPartitions * 20.0 # constant from Spark's RangePartitioner
|
|
|
|
fraction = min(maxSampleSize / max(rddSize, 1), 1.0)
|
2015-04-16 19:20:57 -04:00
|
|
|
samples = self.sample(False, fraction, 1).map(lambda kv: kv[0]).collect()
|
2015-04-10 16:50:32 -04:00
|
|
|
samples = sorted(samples, key=keyfunc)
|
2014-08-13 17:57:12 -04:00
|
|
|
|
|
|
|
# we have numPartitions many parts but one of the them has
|
|
|
|
# an implicit boundary
|
2015-04-16 19:20:57 -04:00
|
|
|
bounds = [samples[int(len(samples) * (i + 1) / numPartitions)]
|
2014-08-13 17:57:12 -04:00
|
|
|
for i in range(0, numPartitions - 1)]
|
2013-10-07 13:42:39 -04:00
|
|
|
|
2014-08-20 01:43:49 -04:00
|
|
|
def rangePartitioner(k):
|
2014-08-13 17:57:12 -04:00
|
|
|
p = bisect.bisect_left(bounds, keyfunc(k))
|
2013-10-07 13:42:39 -04:00
|
|
|
if ascending:
|
|
|
|
return p
|
|
|
|
else:
|
2014-07-14 03:42:59 -04:00
|
|
|
return numPartitions - 1 - p
|
2013-10-07 13:42:39 -04:00
|
|
|
|
2014-08-20 01:43:49 -04:00
|
|
|
return self.partitionBy(numPartitions, rangePartitioner).mapPartitions(sortPartition, True)
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2014-06-17 14:47:48 -04:00
|
|
|
def sortBy(self, keyfunc, ascending=True, numPartitions=None):
|
|
|
|
"""
|
|
|
|
Sorts this RDD by the given keyfunc
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2014-06-17 14:47:48 -04:00
|
|
|
>>> tmp = [('a', 1), ('b', 2), ('1', 3), ('d', 4), ('2', 5)]
|
|
|
|
>>> sc.parallelize(tmp).sortBy(lambda x: x[0]).collect()
|
|
|
|
[('1', 3), ('2', 5), ('a', 1), ('b', 2), ('d', 4)]
|
|
|
|
>>> sc.parallelize(tmp).sortBy(lambda x: x[1]).collect()
|
|
|
|
[('a', 1), ('b', 2), ('1', 3), ('d', 4), ('2', 5)]
|
|
|
|
"""
|
|
|
|
return self.keyBy(keyfunc).sortByKey(ascending, numPartitions).values()
|
|
|
|
|
2012-08-25 01:51:45 -04:00
|
|
|
def glom(self):
|
|
|
|
"""
|
2012-12-27 20:55:33 -05:00
|
|
|
Return an RDD created by coalescing all elements within each partition
|
|
|
|
into a list.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2012-08-25 01:51:45 -04:00
|
|
|
>>> rdd = sc.parallelize([1, 2, 3, 4], 2)
|
2012-12-29 01:19:12 -05:00
|
|
|
>>> sorted(rdd.glom().collect())
|
|
|
|
[[1, 2], [3, 4]]
|
2012-08-25 01:51:45 -04:00
|
|
|
"""
|
2014-07-14 03:42:59 -04:00
|
|
|
def func(iterator):
|
|
|
|
yield list(iterator)
|
2012-08-25 17:19:07 -04:00
|
|
|
return self.mapPartitions(func)
|
2012-08-10 04:10:02 -04:00
|
|
|
|
|
|
|
def cartesian(self, other):
|
|
|
|
"""
|
2012-12-27 20:55:33 -05:00
|
|
|
Return the Cartesian product of this RDD and another one, that is, the
|
2019-07-05 13:08:22 -04:00
|
|
|
RDD of all pairs of elements ``(a, b)`` where ``a`` is in `self` and
|
|
|
|
``b`` is in `other`.
|
2012-12-27 20:55:33 -05:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2012-08-10 04:10:02 -04:00
|
|
|
>>> rdd = sc.parallelize([1, 2])
|
|
|
|
>>> sorted(rdd.cartesian(rdd).collect())
|
|
|
|
[(1, 1), (1, 2), (2, 1), (2, 2)]
|
|
|
|
"""
|
2012-12-29 18:34:57 -05:00
|
|
|
# Due to batching, we can't use the Java cartesian method.
|
2013-11-05 20:52:39 -05:00
|
|
|
deserializer = CartesianDeserializer(self._jrdd_deserializer,
|
|
|
|
other._jrdd_deserializer)
|
|
|
|
return RDD(self._jrdd.cartesian(other._jrdd), self.ctx, deserializer)
|
2012-08-10 04:10:02 -04:00
|
|
|
|
[SPARK-9821] [PYSPARK] pyspark-reduceByKey-should-take-a-custom-partitioner
from the issue:
In Scala, I can supply a custom partitioner to reduceByKey (and other aggregation/repartitioning methods like aggregateByKey and combinedByKey), but as far as I can tell from the Pyspark API, there's no way to do the same in Python.
Here's an example of my code in Scala:
weblogs.map(s => (getFileType(s), 1)).reduceByKey(new FileTypePartitioner(),_+_)
But I can't figure out how to do the same in Python. The closest I can get is to call repartition before reduceByKey like so:
weblogs.map(lambda s: (getFileType(s), 1)).partitionBy(3,hash_filetype).reduceByKey(lambda v1,v2: v1+v2).collect()
But that defeats the purpose, because I'm shuffling twice instead of once, so my performance is worse instead of better.
Author: Holden Karau <holden@pigscanfly.ca>
Closes #8569 from holdenk/SPARK-9821-pyspark-reduceByKey-should-take-a-custom-partitioner.
2015-09-22 02:21:24 -04:00
|
|
|
def groupBy(self, f, numPartitions=None, partitionFunc=portable_hash):
|
2012-08-10 04:10:02 -04:00
|
|
|
"""
|
2012-12-27 20:55:33 -05:00
|
|
|
Return an RDD of grouped items.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2012-08-10 04:10:02 -04:00
|
|
|
>>> rdd = sc.parallelize([1, 1, 2, 3, 5, 8])
|
2012-08-18 19:07:10 -04:00
|
|
|
>>> result = rdd.groupBy(lambda x: x % 2).collect()
|
|
|
|
>>> sorted([(x, sorted(y)) for (x, y) in result])
|
2012-08-10 04:10:02 -04:00
|
|
|
[(0, [2, 8]), (1, [1, 1, 3, 5])]
|
|
|
|
"""
|
[SPARK-9821] [PYSPARK] pyspark-reduceByKey-should-take-a-custom-partitioner
from the issue:
In Scala, I can supply a custom partitioner to reduceByKey (and other aggregation/repartitioning methods like aggregateByKey and combinedByKey), but as far as I can tell from the Pyspark API, there's no way to do the same in Python.
Here's an example of my code in Scala:
weblogs.map(s => (getFileType(s), 1)).reduceByKey(new FileTypePartitioner(),_+_)
But I can't figure out how to do the same in Python. The closest I can get is to call repartition before reduceByKey like so:
weblogs.map(lambda s: (getFileType(s), 1)).partitionBy(3,hash_filetype).reduceByKey(lambda v1,v2: v1+v2).collect()
But that defeats the purpose, because I'm shuffling twice instead of once, so my performance is worse instead of better.
Author: Holden Karau <holden@pigscanfly.ca>
Closes #8569 from holdenk/SPARK-9821-pyspark-reduceByKey-should-take-a-custom-partitioner.
2015-09-22 02:21:24 -04:00
|
|
|
return self.map(lambda x: (f(x), x)).groupByKey(numPartitions, partitionFunc)
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2015-08-14 15:46:05 -04:00
|
|
|
def pipe(self, command, env=None, checkCode=False):
|
2012-08-25 21:00:25 -04:00
|
|
|
"""
|
2012-12-27 20:55:33 -05:00
|
|
|
Return an RDD created by piping elements to a forked external process.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
command : str
|
|
|
|
command to run.
|
|
|
|
env : dict, optional
|
|
|
|
environment variables to set.
|
|
|
|
checkCode : bool, optional
|
|
|
|
whether or not to check the return value of the shell command.
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2014-04-29 21:06:45 -04:00
|
|
|
>>> sc.parallelize(['1', '2', '', '3']).pipe('cat').collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
['1', '2', '', '3']
|
2012-08-25 21:00:25 -04:00
|
|
|
"""
|
2015-08-14 15:46:05 -04:00
|
|
|
if env is None:
|
|
|
|
env = dict()
|
|
|
|
|
2012-08-25 21:00:25 -04:00
|
|
|
def func(iterator):
|
2014-07-14 03:42:59 -04:00
|
|
|
pipe = Popen(
|
|
|
|
shlex.split(command), env=env, stdin=PIPE, stdout=PIPE)
|
|
|
|
|
2012-08-25 21:00:25 -04:00
|
|
|
def pipe_objs(out):
|
|
|
|
for obj in iterator:
|
2020-07-13 22:22:44 -04:00
|
|
|
s = str(obj).rstrip('\n') + '\n'
|
2015-04-16 19:20:57 -04:00
|
|
|
out.write(s.encode('utf-8'))
|
2012-08-25 21:00:25 -04:00
|
|
|
out.close()
|
|
|
|
Thread(target=pipe_objs, args=[pipe.stdin]).start()
|
2015-07-10 22:29:32 -04:00
|
|
|
|
|
|
|
def check_return_code():
|
|
|
|
pipe.wait()
|
|
|
|
if checkCode and pipe.returncode:
|
|
|
|
raise Exception("Pipe function `%s' exited "
|
|
|
|
"with error code %d" % (command, pipe.returncode))
|
|
|
|
else:
|
|
|
|
for i in range(0):
|
|
|
|
yield i
|
|
|
|
return (x.rstrip(b'\n').decode('utf-8') for x in
|
|
|
|
chain(iter(pipe.stdout.readline, b''), check_return_code()))
|
2012-08-25 21:00:25 -04:00
|
|
|
return self.mapPartitions(func)
|
2012-08-10 04:10:02 -04:00
|
|
|
|
|
|
|
def foreach(self, f):
|
|
|
|
"""
|
2012-12-27 20:55:33 -05:00
|
|
|
Applies a function to all elements of this RDD.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2015-04-16 19:20:57 -04:00
|
|
|
>>> def f(x): print(x)
|
2012-08-10 04:10:02 -04:00
|
|
|
>>> sc.parallelize([1, 2, 3, 4, 5]).foreach(f)
|
|
|
|
"""
|
2018-05-30 06:11:33 -04:00
|
|
|
f = fail_on_stopiteration(f)
|
|
|
|
|
2013-07-28 23:38:56 -04:00
|
|
|
def processPartition(iterator):
|
|
|
|
for x in iterator:
|
|
|
|
f(x)
|
2014-08-27 16:18:33 -04:00
|
|
|
return iter([])
|
|
|
|
self.mapPartitions(processPartition).count() # Force evaluation
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2014-02-06 17:58:35 -05:00
|
|
|
def foreachPartition(self, f):
|
|
|
|
"""
|
|
|
|
Applies a function to each partition of this RDD.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2014-06-09 03:08:40 -04:00
|
|
|
>>> def f(iterator):
|
[MINOR][PYSPARK][DOCS] Fix examples in PySpark documentation
## What changes were proposed in this pull request?
This PR proposes to fix wrongly indented examples in PySpark documentation
```
- >>> json_sdf = spark.readStream.format("json")\
- .schema(sdf_schema)\
- .load(tempfile.mkdtemp())
+ >>> json_sdf = spark.readStream.format("json") \\
+ ... .schema(sdf_schema) \\
+ ... .load(tempfile.mkdtemp())
```
```
- people.filter(people.age > 30).join(department, people.deptId == department.id)\
+ people.filter(people.age > 30).join(department, people.deptId == department.id) \\
```
```
- >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, 1.23), (2, 4.56)])), \
- LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))]
+ >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, 1.23), (2, 4.56)])),
+ ... LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))]
```
```
- >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, -1.23), (2, 4.56e-7)])), \
- LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))]
+ >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, -1.23), (2, 4.56e-7)])),
+ ... LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))]
```
```
- ... for x in iterator:
- ... print(x)
+ ... for x in iterator:
+ ... print(x)
```
## How was this patch tested?
Manually tested.
**Before**
![2016-09-26 8 36 02](https://cloud.githubusercontent.com/assets/6477701/18834471/05c7a478-8431-11e6-94bb-09aa37b12ddb.png)
![2016-09-26 9 22 16](https://cloud.githubusercontent.com/assets/6477701/18834472/06c8735c-8431-11e6-8775-78631eab0411.png)
<img width="601" alt="2016-09-27 2 29 27" src="https://cloud.githubusercontent.com/assets/6477701/18861294/29c0d5b4-84bf-11e6-99c5-3c9d913c125d.png">
<img width="1056" alt="2016-09-27 2 29 58" src="https://cloud.githubusercontent.com/assets/6477701/18861298/31694cd8-84bf-11e6-9e61-9888cb8c2089.png">
<img width="1079" alt="2016-09-27 2 30 05" src="https://cloud.githubusercontent.com/assets/6477701/18861301/359722da-84bf-11e6-97f9-5f5365582d14.png">
**After**
![2016-09-26 9 29 47](https://cloud.githubusercontent.com/assets/6477701/18834467/0367f9da-8431-11e6-86d9-a490d3297339.png)
![2016-09-26 9 30 24](https://cloud.githubusercontent.com/assets/6477701/18834463/f870fae0-8430-11e6-9482-01fc47898492.png)
<img width="515" alt="2016-09-27 2 28 19" src="https://cloud.githubusercontent.com/assets/6477701/18861305/3ff88b88-84bf-11e6-902c-9f725e8a8b10.png">
<img width="652" alt="2016-09-27 3 50 59" src="https://cloud.githubusercontent.com/assets/6477701/18863053/592fbc74-84ca-11e6-8dbf-99cf57947de8.png">
<img width="709" alt="2016-09-27 3 51 03" src="https://cloud.githubusercontent.com/assets/6477701/18863060/601607be-84ca-11e6-80aa-a401df41c321.png">
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #15242 from HyukjinKwon/minor-example-pyspark.
2016-09-28 06:19:04 -04:00
|
|
|
... for x in iterator:
|
|
|
|
... print(x)
|
2014-02-06 17:58:35 -05:00
|
|
|
>>> sc.parallelize([1, 2, 3, 4, 5]).foreachPartition(f)
|
|
|
|
"""
|
2014-08-27 16:18:33 -04:00
|
|
|
def func(it):
|
|
|
|
r = f(it)
|
|
|
|
try:
|
|
|
|
return iter(r)
|
|
|
|
except TypeError:
|
|
|
|
return iter([])
|
|
|
|
self.mapPartitions(func).count() # Force evaluation
|
2014-06-09 03:08:40 -04:00
|
|
|
|
2012-08-10 04:10:02 -04:00
|
|
|
def collect(self):
|
2012-12-27 20:55:33 -05:00
|
|
|
"""
|
|
|
|
Return a list that contains all of the elements in this RDD.
|
2016-11-22 06:40:18 -05:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This method should only be used if the resulting array is expected
|
|
|
|
to be small, as all the data is loaded into the driver's memory.
|
2012-12-27 20:55:33 -05:00
|
|
|
"""
|
2014-09-15 22:28:17 -04:00
|
|
|
with SCCallSiteSync(self.context) as css:
|
2018-04-13 17:28:24 -04:00
|
|
|
sock_info = self.ctx._jvm.PythonRDD.collectAndServe(self._jrdd.rdd())
|
|
|
|
return list(_load_from_socket(sock_info, self._jrdd_deserializer))
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2020-04-30 21:08:16 -04:00
|
|
|
def collectWithJobGroup(self, groupId, description, interruptOnCancel=False):
|
|
|
|
"""
|
|
|
|
When collect rdd, use this method to specify job group.
|
|
|
|
|
|
|
|
.. versionadded:: 3.0.0
|
2020-11-15 20:21:50 -05:00
|
|
|
.. deprecated:: 3.1.0
|
|
|
|
Use :class:`pyspark.InheritableThread` with the pinned thread mode enabled.
|
2020-04-30 21:08:16 -04:00
|
|
|
"""
|
[SPARK-32010][PYTHON][CORE] Add InheritableThread for local properties and fixing a thread leak issue in pinned thread mode
### What changes were proposed in this pull request?
This PR proposes:
1. To introduce `InheritableThread` class, that works identically with `threading.Thread` but it can inherit the inheritable attributes of a JVM thread such as `InheritableThreadLocal`.
This was a problem from the pinned thread mode, see also https://github.com/apache/spark/pull/24898. Now it works as below:
```python
import pyspark
spark.sparkContext.setLocalProperty("a", "hi")
def print_prop():
print(spark.sparkContext.getLocalProperty("a"))
pyspark.InheritableThread(target=print_prop).start()
```
```
hi
```
2. Also, it adds the resource leak fix into `InheritableThread`. Py4J leaks the thread and does not close the connection from Python to JVM. In `InheritableThread`, it manually closes the connections when PVM garbage collection happens. So, JVM threads finish safely. I manually verified by profiling but there's also another easy way to verify:
```bash
PYSPARK_PIN_THREAD=true ./bin/pyspark
```
```python
>>> from threading import Thread
>>> Thread(target=lambda: spark.range(1000).collect()).start()
>>> Thread(target=lambda: spark.range(1000).collect()).start()
>>> Thread(target=lambda: spark.range(1000).collect()).start()
>>> spark._jvm._gateway_client.deque
deque([<py4j.clientserver.ClientServerConnection object at 0x119f7aba8>, <py4j.clientserver.ClientServerConnection object at 0x119fc9b70>, <py4j.clientserver.ClientServerConnection object at 0x119fc9e10>, <py4j.clientserver.ClientServerConnection object at 0x11a015358>, <py4j.clientserver.ClientServerConnection object at 0x119fc00f0>])
>>> Thread(target=lambda: spark.range(1000).collect()).start()
>>> spark._jvm._gateway_client.deque
deque([<py4j.clientserver.ClientServerConnection object at 0x119f7aba8>, <py4j.clientserver.ClientServerConnection object at 0x119fc9b70>, <py4j.clientserver.ClientServerConnection object at 0x119fc9e10>, <py4j.clientserver.ClientServerConnection object at 0x11a015358>, <py4j.clientserver.ClientServerConnection object at 0x119fc08d0>, <py4j.clientserver.ClientServerConnection object at 0x119fc00f0>])
```
This issue is fixed now.
3. Because now we have a fix for the issue here, it also proposes to deprecate `collectWithJobGroup` which was a temporary workaround added to avoid this leak issue.
### Why are the changes needed?
To support pinned thread mode properly without a resource leak, and a proper inheritable local properties.
### Does this PR introduce _any_ user-facing change?
Yes, it adds an API `InheritableThread` class for pinned thread mode.
### How was this patch tested?
Manually tested as described above, and unit test was added as well.
Closes #28968 from HyukjinKwon/SPARK-32010.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-29 21:15:25 -04:00
|
|
|
warnings.warn(
|
|
|
|
"Deprecated in 3.1, Use pyspark.InheritableThread with "
|
|
|
|
"the pinned thread mode enabled.",
|
|
|
|
DeprecationWarning)
|
|
|
|
|
2020-04-30 21:08:16 -04:00
|
|
|
with SCCallSiteSync(self.context) as css:
|
|
|
|
sock_info = self.ctx._jvm.PythonRDD.collectAndServeWithJobGroup(
|
|
|
|
self._jrdd.rdd(), groupId, description, interruptOnCancel)
|
|
|
|
return list(_load_from_socket(sock_info, self._jrdd_deserializer))
|
|
|
|
|
2012-08-18 19:07:10 -04:00
|
|
|
def reduce(self, f):
|
2012-08-10 04:10:02 -04:00
|
|
|
"""
|
2013-02-24 16:25:09 -05:00
|
|
|
Reduces the elements of this RDD using the specified commutative and
|
2014-04-25 02:07:54 -04:00
|
|
|
associative binary operator. Currently reduces partitions locally.
|
2012-12-27 20:55:33 -05:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2012-08-18 19:07:10 -04:00
|
|
|
>>> from operator import add
|
|
|
|
>>> sc.parallelize([1, 2, 3, 4, 5]).reduce(add)
|
2012-08-10 04:10:02 -04:00
|
|
|
15
|
2012-08-18 19:07:10 -04:00
|
|
|
>>> sc.parallelize((2 for _ in range(10))).map(lambda x: 1).cache().reduce(add)
|
|
|
|
10
|
2014-08-27 16:18:33 -04:00
|
|
|
>>> sc.parallelize([]).reduce(add)
|
|
|
|
Traceback (most recent call last):
|
|
|
|
...
|
|
|
|
ValueError: Can not reduce() empty RDD
|
2012-08-10 04:10:02 -04:00
|
|
|
"""
|
2018-05-30 06:11:33 -04:00
|
|
|
f = fail_on_stopiteration(f)
|
|
|
|
|
2012-08-24 22:38:50 -04:00
|
|
|
def func(iterator):
|
2014-08-27 16:18:33 -04:00
|
|
|
iterator = iter(iterator)
|
|
|
|
try:
|
|
|
|
initial = next(iterator)
|
|
|
|
except StopIteration:
|
|
|
|
return
|
|
|
|
yield reduce(f, iterator, initial)
|
|
|
|
|
2012-08-25 17:19:07 -04:00
|
|
|
vals = self.mapPartitions(func).collect()
|
2014-08-27 16:18:33 -04:00
|
|
|
if vals:
|
|
|
|
return reduce(f, vals)
|
|
|
|
raise ValueError("Can not reduce() empty RDD")
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2015-01-28 20:26:03 -05:00
|
|
|
def treeReduce(self, f, depth=2):
|
|
|
|
"""
|
|
|
|
Reduces the elements of this RDD in a multi-level tree pattern.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
f : function
|
|
|
|
depth : int, optional
|
|
|
|
suggested depth of the tree (default: 2)
|
2015-01-28 20:26:03 -05:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2015-01-28 20:26:03 -05:00
|
|
|
>>> add = lambda x, y: x + y
|
|
|
|
>>> rdd = sc.parallelize([-5, -4, -3, -2, -1, 1, 2, 3, 4], 10)
|
|
|
|
>>> rdd.treeReduce(add)
|
|
|
|
-5
|
|
|
|
>>> rdd.treeReduce(add, 1)
|
|
|
|
-5
|
|
|
|
>>> rdd.treeReduce(add, 2)
|
|
|
|
-5
|
|
|
|
>>> rdd.treeReduce(add, 5)
|
|
|
|
-5
|
|
|
|
>>> rdd.treeReduce(add, 10)
|
|
|
|
-5
|
|
|
|
"""
|
|
|
|
if depth < 1:
|
|
|
|
raise ValueError("Depth cannot be smaller than 1 but got %d." % depth)
|
|
|
|
|
|
|
|
zeroValue = None, True # Use the second entry to indicate whether this is a dummy value.
|
|
|
|
|
|
|
|
def op(x, y):
|
|
|
|
if x[1]:
|
|
|
|
return y
|
|
|
|
elif y[1]:
|
|
|
|
return x
|
|
|
|
else:
|
|
|
|
return f(x[0], y[0]), False
|
|
|
|
|
|
|
|
reduced = self.map(lambda x: (x, False)).treeAggregate(zeroValue, op, op, depth)
|
|
|
|
if reduced[1]:
|
|
|
|
raise ValueError("Cannot reduce empty RDD.")
|
|
|
|
return reduced[0]
|
|
|
|
|
2012-08-25 00:07:26 -04:00
|
|
|
def fold(self, zeroValue, op):
|
|
|
|
"""
|
|
|
|
Aggregate the elements of each partition, and then the results for all
|
2016-02-19 05:26:38 -05:00
|
|
|
the partitions, using a given associative function and a neutral "zero value."
|
2012-12-27 20:55:33 -05:00
|
|
|
|
2019-07-05 13:08:22 -04:00
|
|
|
The function ``op(t1, t2)`` is allowed to modify ``t1`` and return it
|
2012-08-25 00:07:26 -04:00
|
|
|
as its result value to avoid object allocation; however, it should not
|
2019-07-05 13:08:22 -04:00
|
|
|
modify ``t2``.
|
2012-08-25 00:07:26 -04:00
|
|
|
|
2015-05-21 14:42:51 -04:00
|
|
|
This behaves somewhat differently from fold operations implemented
|
|
|
|
for non-distributed collections in functional languages like Scala.
|
|
|
|
This fold operation may be applied to partitions individually, and then
|
|
|
|
fold those results into the final result, rather than apply the fold
|
|
|
|
to each element sequentially in some defined ordering. For functions
|
|
|
|
that are not commutative, the result may differ from that of a fold
|
|
|
|
applied to a non-distributed collection.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2012-08-25 00:07:26 -04:00
|
|
|
>>> from operator import add
|
|
|
|
>>> sc.parallelize([1, 2, 3, 4, 5]).fold(0, add)
|
|
|
|
15
|
|
|
|
"""
|
2018-05-30 06:11:33 -04:00
|
|
|
op = fail_on_stopiteration(op)
|
|
|
|
|
2012-08-25 00:07:26 -04:00
|
|
|
def func(iterator):
|
|
|
|
acc = zeroValue
|
|
|
|
for obj in iterator:
|
2016-01-19 04:34:49 -05:00
|
|
|
acc = op(acc, obj)
|
2012-08-25 00:07:26 -04:00
|
|
|
yield acc
|
[SPARK-9021] [PYSPARK] Change RDD.aggregate() to do reduce(mapPartitions()) instead of mapPartitions.fold()
I'm relatively new to Spark and functional programming, so forgive me if this pull request is just a result of my misunderstanding of how Spark should be used.
Currently, if one happens to use a mutable object as `zeroValue` for `RDD.aggregate()`, possibly unexpected behavior can occur.
This is because pyspark's current implementation of `RDD.aggregate()` does not serialize or make a copy of `zeroValue` before handing it off to `RDD.mapPartitions(...).fold(...)`. This results in a single reference to `zeroValue` being used for both `RDD.mapPartitions()` and `RDD.fold()` on each partition. This can result in strange accumulator values being fed into each partition's call to `RDD.fold()`, as the `zeroValue` may have been changed in-place during the `RDD.mapPartitions()` call.
As an illustrative example, submit the following to `spark-submit`:
```
from pyspark import SparkConf, SparkContext
import collections
def updateCounter(acc, val):
print 'update acc:', acc
print 'update val:', val
acc[val] += 1
return acc
def comboCounter(acc1, acc2):
print 'combo acc1:', acc1
print 'combo acc2:', acc2
acc1.update(acc2)
return acc1
def main():
conf = SparkConf().setMaster("local").setAppName("Aggregate with Counter")
sc = SparkContext(conf = conf)
print '======= AGGREGATING with ONE PARTITION ======='
print sc.parallelize(range(1,10), 1).aggregate(collections.Counter(), updateCounter, comboCounter)
print '======= AGGREGATING with TWO PARTITIONS ======='
print sc.parallelize(range(1,10), 2).aggregate(collections.Counter(), updateCounter, comboCounter)
if __name__ == "__main__":
main()
```
One probably expects this to output the following:
```
Counter({1: 1, 2: 1, 3: 1, 4: 1, 5: 1, 6: 1, 7: 1, 8: 1, 9: 1})
```
But it instead outputs this (regardless of the number of partitions):
```
Counter({1: 2, 2: 2, 3: 2, 4: 2, 5: 2, 6: 2, 7: 2, 8: 2, 9: 2})
```
This is because (I believe) `zeroValue` gets passed correctly to each partition, but after `RDD.mapPartitions()` completes, the `zeroValue` object has been updated and is then passed to `RDD.fold()`, which results in all items being double-counted within each partition before being finally reduced at the calling node.
I realize that this type of calculation is typically done by `RDD.mapPartitions(...).reduceByKey(...)`, but hopefully this illustrates some potentially confusing behavior. I also noticed that other `RDD` methods use this `deepcopy` approach to creating unique copies of `zeroValue` (i.e., `RDD.aggregateByKey()` and `RDD.foldByKey()`), and that the Scala implementations do seem to serialize the `zeroValue` object appropriately to prevent this type of behavior.
Author: Nicholas Hwang <moogling@gmail.com>
Closes #7378 from njhwang/master and squashes the following commits:
659bb27 [Nicholas Hwang] Fixed RDD.aggregate() to perform a reduce operation on collected mapPartitions results, similar to how fold currently is implemented. This prevents an initial combOp being performed on each partition with zeroValue (which leads to unexpected behavior if zeroValue is a mutable object) before being combOp'ed with other partition results.
8d8d694 [Nicholas Hwang] Changed dict construction to be compatible with Python 2.6 (cannot use list comprehensions to make dicts)
56eb2ab [Nicholas Hwang] Fixed whitespace after colon to conform with PEP8
391de4a [Nicholas Hwang] Removed used of collections.Counter from RDD tests for Python 2.6 compatibility; used defaultdict(int) instead. Merged treeAggregate test with mutable zero value into aggregate test to reduce code duplication.
2fa4e4b [Nicholas Hwang] Merge branch 'master' of https://github.com/njhwang/spark
ba528bd [Nicholas Hwang] Updated comments regarding protection of zeroValue from mutation in RDD.aggregate(). Added regression tests for aggregate(), fold(), aggregateByKey(), foldByKey(), and treeAggregate(), all with both 1 and 2 partition RDDs. Confirmed that aggregate() is the only problematic implementation as of commit 257236c3e17906098f801cbc2059e7a9054e8cab. Also replaced some parallelizations of ranges with xranges, per the documentation's recommendations of preferring xrange over range.
7820391 [Nicholas Hwang] Updated comments regarding protection of zeroValue from mutation in RDD.aggregate(). Added regression tests for aggregate(), fold(), aggregateByKey(), foldByKey(), and treeAggregate(), all with both 1 and 2 partition RDDs. Confirmed that aggregate() is the only problematic implementation as of commit 257236c3e17906098f801cbc2059e7a9054e8cab.
90d1544 [Nicholas Hwang] Made sure RDD.aggregate() makes a deepcopy of zeroValue for all partitions; this ensures that the mapPartitions call works with unique copies of zeroValue in each partition, and prevents a single reference to zeroValue being used for both map and fold calls on each partition (resulting in possibly unexpected behavior).
2015-07-19 13:30:28 -04:00
|
|
|
# collecting result of mapPartitions here ensures that the copy of
|
|
|
|
# zeroValue provided to each partition is unique from the one provided
|
|
|
|
# to the final reduce call
|
2012-08-25 17:19:07 -04:00
|
|
|
vals = self.mapPartitions(func).collect()
|
2012-08-25 00:07:26 -04:00
|
|
|
return reduce(op, vals, zeroValue)
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2014-04-25 02:07:54 -04:00
|
|
|
def aggregate(self, zeroValue, seqOp, combOp):
|
|
|
|
"""
|
|
|
|
Aggregate the elements of each partition, and then the results for all
|
|
|
|
the partitions, using a given combine functions and a neutral "zero
|
|
|
|
value."
|
|
|
|
|
2019-07-05 13:08:22 -04:00
|
|
|
The functions ``op(t1, t2)`` is allowed to modify ``t1`` and return it
|
2014-04-25 02:07:54 -04:00
|
|
|
as its result value to avoid object allocation; however, it should not
|
2019-07-05 13:08:22 -04:00
|
|
|
modify ``t2``.
|
2014-04-25 02:07:54 -04:00
|
|
|
|
|
|
|
The first function (seqOp) can return a different result type, U, than
|
2014-06-12 22:44:27 -04:00
|
|
|
the type of this RDD. Thus, we need one operation for merging a T into
|
|
|
|
an U and one operation for merging two U
|
2014-04-25 02:07:54 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2014-04-25 02:07:54 -04:00
|
|
|
>>> seqOp = (lambda x, y: (x[0] + y, x[1] + 1))
|
|
|
|
>>> combOp = (lambda x, y: (x[0] + y[0], x[1] + y[1]))
|
|
|
|
>>> sc.parallelize([1, 2, 3, 4]).aggregate((0, 0), seqOp, combOp)
|
|
|
|
(10, 4)
|
|
|
|
>>> sc.parallelize([]).aggregate((0, 0), seqOp, combOp)
|
|
|
|
(0, 0)
|
|
|
|
"""
|
2018-05-30 06:11:33 -04:00
|
|
|
seqOp = fail_on_stopiteration(seqOp)
|
|
|
|
combOp = fail_on_stopiteration(combOp)
|
|
|
|
|
2014-04-25 02:07:54 -04:00
|
|
|
def func(iterator):
|
|
|
|
acc = zeroValue
|
|
|
|
for obj in iterator:
|
|
|
|
acc = seqOp(acc, obj)
|
|
|
|
yield acc
|
[SPARK-9021] [PYSPARK] Change RDD.aggregate() to do reduce(mapPartitions()) instead of mapPartitions.fold()
I'm relatively new to Spark and functional programming, so forgive me if this pull request is just a result of my misunderstanding of how Spark should be used.
Currently, if one happens to use a mutable object as `zeroValue` for `RDD.aggregate()`, possibly unexpected behavior can occur.
This is because pyspark's current implementation of `RDD.aggregate()` does not serialize or make a copy of `zeroValue` before handing it off to `RDD.mapPartitions(...).fold(...)`. This results in a single reference to `zeroValue` being used for both `RDD.mapPartitions()` and `RDD.fold()` on each partition. This can result in strange accumulator values being fed into each partition's call to `RDD.fold()`, as the `zeroValue` may have been changed in-place during the `RDD.mapPartitions()` call.
As an illustrative example, submit the following to `spark-submit`:
```
from pyspark import SparkConf, SparkContext
import collections
def updateCounter(acc, val):
print 'update acc:', acc
print 'update val:', val
acc[val] += 1
return acc
def comboCounter(acc1, acc2):
print 'combo acc1:', acc1
print 'combo acc2:', acc2
acc1.update(acc2)
return acc1
def main():
conf = SparkConf().setMaster("local").setAppName("Aggregate with Counter")
sc = SparkContext(conf = conf)
print '======= AGGREGATING with ONE PARTITION ======='
print sc.parallelize(range(1,10), 1).aggregate(collections.Counter(), updateCounter, comboCounter)
print '======= AGGREGATING with TWO PARTITIONS ======='
print sc.parallelize(range(1,10), 2).aggregate(collections.Counter(), updateCounter, comboCounter)
if __name__ == "__main__":
main()
```
One probably expects this to output the following:
```
Counter({1: 1, 2: 1, 3: 1, 4: 1, 5: 1, 6: 1, 7: 1, 8: 1, 9: 1})
```
But it instead outputs this (regardless of the number of partitions):
```
Counter({1: 2, 2: 2, 3: 2, 4: 2, 5: 2, 6: 2, 7: 2, 8: 2, 9: 2})
```
This is because (I believe) `zeroValue` gets passed correctly to each partition, but after `RDD.mapPartitions()` completes, the `zeroValue` object has been updated and is then passed to `RDD.fold()`, which results in all items being double-counted within each partition before being finally reduced at the calling node.
I realize that this type of calculation is typically done by `RDD.mapPartitions(...).reduceByKey(...)`, but hopefully this illustrates some potentially confusing behavior. I also noticed that other `RDD` methods use this `deepcopy` approach to creating unique copies of `zeroValue` (i.e., `RDD.aggregateByKey()` and `RDD.foldByKey()`), and that the Scala implementations do seem to serialize the `zeroValue` object appropriately to prevent this type of behavior.
Author: Nicholas Hwang <moogling@gmail.com>
Closes #7378 from njhwang/master and squashes the following commits:
659bb27 [Nicholas Hwang] Fixed RDD.aggregate() to perform a reduce operation on collected mapPartitions results, similar to how fold currently is implemented. This prevents an initial combOp being performed on each partition with zeroValue (which leads to unexpected behavior if zeroValue is a mutable object) before being combOp'ed with other partition results.
8d8d694 [Nicholas Hwang] Changed dict construction to be compatible with Python 2.6 (cannot use list comprehensions to make dicts)
56eb2ab [Nicholas Hwang] Fixed whitespace after colon to conform with PEP8
391de4a [Nicholas Hwang] Removed used of collections.Counter from RDD tests for Python 2.6 compatibility; used defaultdict(int) instead. Merged treeAggregate test with mutable zero value into aggregate test to reduce code duplication.
2fa4e4b [Nicholas Hwang] Merge branch 'master' of https://github.com/njhwang/spark
ba528bd [Nicholas Hwang] Updated comments regarding protection of zeroValue from mutation in RDD.aggregate(). Added regression tests for aggregate(), fold(), aggregateByKey(), foldByKey(), and treeAggregate(), all with both 1 and 2 partition RDDs. Confirmed that aggregate() is the only problematic implementation as of commit 257236c3e17906098f801cbc2059e7a9054e8cab. Also replaced some parallelizations of ranges with xranges, per the documentation's recommendations of preferring xrange over range.
7820391 [Nicholas Hwang] Updated comments regarding protection of zeroValue from mutation in RDD.aggregate(). Added regression tests for aggregate(), fold(), aggregateByKey(), foldByKey(), and treeAggregate(), all with both 1 and 2 partition RDDs. Confirmed that aggregate() is the only problematic implementation as of commit 257236c3e17906098f801cbc2059e7a9054e8cab.
90d1544 [Nicholas Hwang] Made sure RDD.aggregate() makes a deepcopy of zeroValue for all partitions; this ensures that the mapPartitions call works with unique copies of zeroValue in each partition, and prevents a single reference to zeroValue being used for both map and fold calls on each partition (resulting in possibly unexpected behavior).
2015-07-19 13:30:28 -04:00
|
|
|
# collecting result of mapPartitions here ensures that the copy of
|
|
|
|
# zeroValue provided to each partition is unique from the one provided
|
|
|
|
# to the final reduce call
|
|
|
|
vals = self.mapPartitions(func).collect()
|
|
|
|
return reduce(combOp, vals, zeroValue)
|
2014-06-09 03:08:40 -04:00
|
|
|
|
2015-01-28 20:26:03 -05:00
|
|
|
def treeAggregate(self, zeroValue, seqOp, combOp, depth=2):
|
|
|
|
"""
|
|
|
|
Aggregates the elements of this RDD in a multi-level tree
|
|
|
|
pattern.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
depth : int, optional
|
|
|
|
suggested depth of the tree (default: 2)
|
2015-01-28 20:26:03 -05:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2015-01-28 20:26:03 -05:00
|
|
|
>>> add = lambda x, y: x + y
|
|
|
|
>>> rdd = sc.parallelize([-5, -4, -3, -2, -1, 1, 2, 3, 4], 10)
|
|
|
|
>>> rdd.treeAggregate(0, add, add)
|
|
|
|
-5
|
|
|
|
>>> rdd.treeAggregate(0, add, add, 1)
|
|
|
|
-5
|
|
|
|
>>> rdd.treeAggregate(0, add, add, 2)
|
|
|
|
-5
|
|
|
|
>>> rdd.treeAggregate(0, add, add, 5)
|
|
|
|
-5
|
|
|
|
>>> rdd.treeAggregate(0, add, add, 10)
|
|
|
|
-5
|
|
|
|
"""
|
|
|
|
if depth < 1:
|
|
|
|
raise ValueError("Depth cannot be smaller than 1 but got %d." % depth)
|
|
|
|
|
|
|
|
if self.getNumPartitions() == 0:
|
|
|
|
return zeroValue
|
|
|
|
|
|
|
|
def aggregatePartition(iterator):
|
|
|
|
acc = zeroValue
|
|
|
|
for obj in iterator:
|
|
|
|
acc = seqOp(acc, obj)
|
|
|
|
yield acc
|
|
|
|
|
|
|
|
partiallyAggregated = self.mapPartitions(aggregatePartition)
|
|
|
|
numPartitions = partiallyAggregated.getNumPartitions()
|
|
|
|
scale = max(int(ceil(pow(numPartitions, 1.0 / depth))), 2)
|
|
|
|
# If creating an extra level doesn't help reduce the wall-clock time, we stop the tree
|
|
|
|
# aggregation.
|
|
|
|
while numPartitions > scale + numPartitions / scale:
|
|
|
|
numPartitions /= scale
|
2015-04-16 19:20:57 -04:00
|
|
|
curNumPartitions = int(numPartitions)
|
2015-01-28 20:26:03 -05:00
|
|
|
|
|
|
|
def mapPartition(i, iterator):
|
|
|
|
for obj in iterator:
|
|
|
|
yield (i % curNumPartitions, obj)
|
|
|
|
|
|
|
|
partiallyAggregated = partiallyAggregated \
|
|
|
|
.mapPartitionsWithIndex(mapPartition) \
|
|
|
|
.reduceByKey(combOp, curNumPartitions) \
|
|
|
|
.values()
|
|
|
|
|
|
|
|
return partiallyAggregated.reduce(combOp)
|
|
|
|
|
[SPARK-2871] [PySpark] add `key` argument for max(), min() and top(n)
RDD.max(key=None)
param key: A function used to generate key for comparing
>>> rdd = sc.parallelize([1.0, 5.0, 43.0, 10.0])
>>> rdd.max()
43.0
>>> rdd.max(key=str)
5.0
RDD.min(key=None)
Find the minimum item in this RDD.
param key: A function used to generate key for comparing
>>> rdd = sc.parallelize([2.0, 5.0, 43.0, 10.0])
>>> rdd.min()
2.0
>>> rdd.min(key=str)
10.0
RDD.top(num, key=None)
Get the top N elements from a RDD.
Note: It returns the list sorted in descending order.
>>> sc.parallelize([10, 4, 2, 12, 3]).top(1)
[12]
>>> sc.parallelize([2, 3, 4, 5, 6], 2).top(2)
[6, 5]
>>> sc.parallelize([10, 4, 2, 12, 3]).top(3, key=str)
[4, 3, 2]
Author: Davies Liu <davies.liu@gmail.com>
Closes #2094 from davies/cmp and squashes the following commits:
ccbaf25 [Davies Liu] add `key` to top()
ad7e374 [Davies Liu] fix tests
2f63512 [Davies Liu] change `comp` to `key` in min/max
dd91e08 [Davies Liu] add `comp` argument for RDD.max() and RDD.min()
2014-08-23 21:55:13 -04:00
|
|
|
def max(self, key=None):
|
2014-03-18 03:45:47 -04:00
|
|
|
"""
|
|
|
|
Find the maximum item in this RDD.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
key : function, optional
|
|
|
|
A function used to generate key for comparing
|
[SPARK-2871] [PySpark] add `key` argument for max(), min() and top(n)
RDD.max(key=None)
param key: A function used to generate key for comparing
>>> rdd = sc.parallelize([1.0, 5.0, 43.0, 10.0])
>>> rdd.max()
43.0
>>> rdd.max(key=str)
5.0
RDD.min(key=None)
Find the minimum item in this RDD.
param key: A function used to generate key for comparing
>>> rdd = sc.parallelize([2.0, 5.0, 43.0, 10.0])
>>> rdd.min()
2.0
>>> rdd.min(key=str)
10.0
RDD.top(num, key=None)
Get the top N elements from a RDD.
Note: It returns the list sorted in descending order.
>>> sc.parallelize([10, 4, 2, 12, 3]).top(1)
[12]
>>> sc.parallelize([2, 3, 4, 5, 6], 2).top(2)
[6, 5]
>>> sc.parallelize([10, 4, 2, 12, 3]).top(3, key=str)
[4, 3, 2]
Author: Davies Liu <davies.liu@gmail.com>
Closes #2094 from davies/cmp and squashes the following commits:
ccbaf25 [Davies Liu] add `key` to top()
ad7e374 [Davies Liu] fix tests
2f63512 [Davies Liu] change `comp` to `key` in min/max
dd91e08 [Davies Liu] add `comp` argument for RDD.max() and RDD.min()
2014-08-23 21:55:13 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
[SPARK-2871] [PySpark] add `key` argument for max(), min() and top(n)
RDD.max(key=None)
param key: A function used to generate key for comparing
>>> rdd = sc.parallelize([1.0, 5.0, 43.0, 10.0])
>>> rdd.max()
43.0
>>> rdd.max(key=str)
5.0
RDD.min(key=None)
Find the minimum item in this RDD.
param key: A function used to generate key for comparing
>>> rdd = sc.parallelize([2.0, 5.0, 43.0, 10.0])
>>> rdd.min()
2.0
>>> rdd.min(key=str)
10.0
RDD.top(num, key=None)
Get the top N elements from a RDD.
Note: It returns the list sorted in descending order.
>>> sc.parallelize([10, 4, 2, 12, 3]).top(1)
[12]
>>> sc.parallelize([2, 3, 4, 5, 6], 2).top(2)
[6, 5]
>>> sc.parallelize([10, 4, 2, 12, 3]).top(3, key=str)
[4, 3, 2]
Author: Davies Liu <davies.liu@gmail.com>
Closes #2094 from davies/cmp and squashes the following commits:
ccbaf25 [Davies Liu] add `key` to top()
ad7e374 [Davies Liu] fix tests
2f63512 [Davies Liu] change `comp` to `key` in min/max
dd91e08 [Davies Liu] add `comp` argument for RDD.max() and RDD.min()
2014-08-23 21:55:13 -04:00
|
|
|
>>> rdd = sc.parallelize([1.0, 5.0, 43.0, 10.0])
|
|
|
|
>>> rdd.max()
|
2014-03-18 03:45:47 -04:00
|
|
|
43.0
|
[SPARK-2871] [PySpark] add `key` argument for max(), min() and top(n)
RDD.max(key=None)
param key: A function used to generate key for comparing
>>> rdd = sc.parallelize([1.0, 5.0, 43.0, 10.0])
>>> rdd.max()
43.0
>>> rdd.max(key=str)
5.0
RDD.min(key=None)
Find the minimum item in this RDD.
param key: A function used to generate key for comparing
>>> rdd = sc.parallelize([2.0, 5.0, 43.0, 10.0])
>>> rdd.min()
2.0
>>> rdd.min(key=str)
10.0
RDD.top(num, key=None)
Get the top N elements from a RDD.
Note: It returns the list sorted in descending order.
>>> sc.parallelize([10, 4, 2, 12, 3]).top(1)
[12]
>>> sc.parallelize([2, 3, 4, 5, 6], 2).top(2)
[6, 5]
>>> sc.parallelize([10, 4, 2, 12, 3]).top(3, key=str)
[4, 3, 2]
Author: Davies Liu <davies.liu@gmail.com>
Closes #2094 from davies/cmp and squashes the following commits:
ccbaf25 [Davies Liu] add `key` to top()
ad7e374 [Davies Liu] fix tests
2f63512 [Davies Liu] change `comp` to `key` in min/max
dd91e08 [Davies Liu] add `comp` argument for RDD.max() and RDD.min()
2014-08-23 21:55:13 -04:00
|
|
|
>>> rdd.max(key=str)
|
|
|
|
5.0
|
2014-03-18 03:45:47 -04:00
|
|
|
"""
|
[SPARK-2871] [PySpark] add `key` argument for max(), min() and top(n)
RDD.max(key=None)
param key: A function used to generate key for comparing
>>> rdd = sc.parallelize([1.0, 5.0, 43.0, 10.0])
>>> rdd.max()
43.0
>>> rdd.max(key=str)
5.0
RDD.min(key=None)
Find the minimum item in this RDD.
param key: A function used to generate key for comparing
>>> rdd = sc.parallelize([2.0, 5.0, 43.0, 10.0])
>>> rdd.min()
2.0
>>> rdd.min(key=str)
10.0
RDD.top(num, key=None)
Get the top N elements from a RDD.
Note: It returns the list sorted in descending order.
>>> sc.parallelize([10, 4, 2, 12, 3]).top(1)
[12]
>>> sc.parallelize([2, 3, 4, 5, 6], 2).top(2)
[6, 5]
>>> sc.parallelize([10, 4, 2, 12, 3]).top(3, key=str)
[4, 3, 2]
Author: Davies Liu <davies.liu@gmail.com>
Closes #2094 from davies/cmp and squashes the following commits:
ccbaf25 [Davies Liu] add `key` to top()
ad7e374 [Davies Liu] fix tests
2f63512 [Davies Liu] change `comp` to `key` in min/max
dd91e08 [Davies Liu] add `comp` argument for RDD.max() and RDD.min()
2014-08-23 21:55:13 -04:00
|
|
|
if key is None:
|
|
|
|
return self.reduce(max)
|
|
|
|
return self.reduce(lambda a, b: max(a, b, key=key))
|
2014-03-18 03:45:47 -04:00
|
|
|
|
[SPARK-2871] [PySpark] add `key` argument for max(), min() and top(n)
RDD.max(key=None)
param key: A function used to generate key for comparing
>>> rdd = sc.parallelize([1.0, 5.0, 43.0, 10.0])
>>> rdd.max()
43.0
>>> rdd.max(key=str)
5.0
RDD.min(key=None)
Find the minimum item in this RDD.
param key: A function used to generate key for comparing
>>> rdd = sc.parallelize([2.0, 5.0, 43.0, 10.0])
>>> rdd.min()
2.0
>>> rdd.min(key=str)
10.0
RDD.top(num, key=None)
Get the top N elements from a RDD.
Note: It returns the list sorted in descending order.
>>> sc.parallelize([10, 4, 2, 12, 3]).top(1)
[12]
>>> sc.parallelize([2, 3, 4, 5, 6], 2).top(2)
[6, 5]
>>> sc.parallelize([10, 4, 2, 12, 3]).top(3, key=str)
[4, 3, 2]
Author: Davies Liu <davies.liu@gmail.com>
Closes #2094 from davies/cmp and squashes the following commits:
ccbaf25 [Davies Liu] add `key` to top()
ad7e374 [Davies Liu] fix tests
2f63512 [Davies Liu] change `comp` to `key` in min/max
dd91e08 [Davies Liu] add `comp` argument for RDD.max() and RDD.min()
2014-08-23 21:55:13 -04:00
|
|
|
def min(self, key=None):
|
2014-03-18 03:45:47 -04:00
|
|
|
"""
|
2014-06-12 11:10:51 -04:00
|
|
|
Find the minimum item in this RDD.
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
key : function, optional
|
|
|
|
A function used to generate key for comparing
|
[SPARK-2871] [PySpark] add `key` argument for max(), min() and top(n)
RDD.max(key=None)
param key: A function used to generate key for comparing
>>> rdd = sc.parallelize([1.0, 5.0, 43.0, 10.0])
>>> rdd.max()
43.0
>>> rdd.max(key=str)
5.0
RDD.min(key=None)
Find the minimum item in this RDD.
param key: A function used to generate key for comparing
>>> rdd = sc.parallelize([2.0, 5.0, 43.0, 10.0])
>>> rdd.min()
2.0
>>> rdd.min(key=str)
10.0
RDD.top(num, key=None)
Get the top N elements from a RDD.
Note: It returns the list sorted in descending order.
>>> sc.parallelize([10, 4, 2, 12, 3]).top(1)
[12]
>>> sc.parallelize([2, 3, 4, 5, 6], 2).top(2)
[6, 5]
>>> sc.parallelize([10, 4, 2, 12, 3]).top(3, key=str)
[4, 3, 2]
Author: Davies Liu <davies.liu@gmail.com>
Closes #2094 from davies/cmp and squashes the following commits:
ccbaf25 [Davies Liu] add `key` to top()
ad7e374 [Davies Liu] fix tests
2f63512 [Davies Liu] change `comp` to `key` in min/max
dd91e08 [Davies Liu] add `comp` argument for RDD.max() and RDD.min()
2014-08-23 21:55:13 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
[SPARK-2871] [PySpark] add `key` argument for max(), min() and top(n)
RDD.max(key=None)
param key: A function used to generate key for comparing
>>> rdd = sc.parallelize([1.0, 5.0, 43.0, 10.0])
>>> rdd.max()
43.0
>>> rdd.max(key=str)
5.0
RDD.min(key=None)
Find the minimum item in this RDD.
param key: A function used to generate key for comparing
>>> rdd = sc.parallelize([2.0, 5.0, 43.0, 10.0])
>>> rdd.min()
2.0
>>> rdd.min(key=str)
10.0
RDD.top(num, key=None)
Get the top N elements from a RDD.
Note: It returns the list sorted in descending order.
>>> sc.parallelize([10, 4, 2, 12, 3]).top(1)
[12]
>>> sc.parallelize([2, 3, 4, 5, 6], 2).top(2)
[6, 5]
>>> sc.parallelize([10, 4, 2, 12, 3]).top(3, key=str)
[4, 3, 2]
Author: Davies Liu <davies.liu@gmail.com>
Closes #2094 from davies/cmp and squashes the following commits:
ccbaf25 [Davies Liu] add `key` to top()
ad7e374 [Davies Liu] fix tests
2f63512 [Davies Liu] change `comp` to `key` in min/max
dd91e08 [Davies Liu] add `comp` argument for RDD.max() and RDD.min()
2014-08-23 21:55:13 -04:00
|
|
|
>>> rdd = sc.parallelize([2.0, 5.0, 43.0, 10.0])
|
|
|
|
>>> rdd.min()
|
|
|
|
2.0
|
|
|
|
>>> rdd.min(key=str)
|
|
|
|
10.0
|
2014-03-18 03:45:47 -04:00
|
|
|
"""
|
[SPARK-2871] [PySpark] add `key` argument for max(), min() and top(n)
RDD.max(key=None)
param key: A function used to generate key for comparing
>>> rdd = sc.parallelize([1.0, 5.0, 43.0, 10.0])
>>> rdd.max()
43.0
>>> rdd.max(key=str)
5.0
RDD.min(key=None)
Find the minimum item in this RDD.
param key: A function used to generate key for comparing
>>> rdd = sc.parallelize([2.0, 5.0, 43.0, 10.0])
>>> rdd.min()
2.0
>>> rdd.min(key=str)
10.0
RDD.top(num, key=None)
Get the top N elements from a RDD.
Note: It returns the list sorted in descending order.
>>> sc.parallelize([10, 4, 2, 12, 3]).top(1)
[12]
>>> sc.parallelize([2, 3, 4, 5, 6], 2).top(2)
[6, 5]
>>> sc.parallelize([10, 4, 2, 12, 3]).top(3, key=str)
[4, 3, 2]
Author: Davies Liu <davies.liu@gmail.com>
Closes #2094 from davies/cmp and squashes the following commits:
ccbaf25 [Davies Liu] add `key` to top()
ad7e374 [Davies Liu] fix tests
2f63512 [Davies Liu] change `comp` to `key` in min/max
dd91e08 [Davies Liu] add `comp` argument for RDD.max() and RDD.min()
2014-08-23 21:55:13 -04:00
|
|
|
if key is None:
|
|
|
|
return self.reduce(min)
|
|
|
|
return self.reduce(lambda a, b: min(a, b, key=key))
|
2014-06-09 03:08:40 -04:00
|
|
|
|
2012-12-26 20:34:24 -05:00
|
|
|
def sum(self):
|
|
|
|
"""
|
2012-12-27 20:55:33 -05:00
|
|
|
Add up the elements in this RDD.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2012-12-26 20:34:24 -05:00
|
|
|
>>> sc.parallelize([1.0, 2.0, 3.0]).sum()
|
|
|
|
6.0
|
|
|
|
"""
|
2015-06-17 16:59:39 -04:00
|
|
|
return self.mapPartitions(lambda x: [sum(x)]).fold(0, operator.add)
|
2012-12-26 20:34:24 -05:00
|
|
|
|
2012-08-10 04:10:02 -04:00
|
|
|
def count(self):
|
|
|
|
"""
|
2012-12-27 20:55:33 -05:00
|
|
|
Return the number of elements in this RDD.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2012-08-10 04:10:02 -04:00
|
|
|
>>> sc.parallelize([2, 3, 4]).count()
|
2012-12-26 20:34:24 -05:00
|
|
|
3
|
2012-08-10 04:10:02 -04:00
|
|
|
"""
|
2012-12-26 20:34:24 -05:00
|
|
|
return self.mapPartitions(lambda i: [sum(1 for _ in i)]).sum()
|
2013-10-09 15:08:04 -04:00
|
|
|
|
2013-08-20 16:22:06 -04:00
|
|
|
def stats(self):
|
|
|
|
"""
|
2019-07-05 13:08:22 -04:00
|
|
|
Return a :class:`StatCounter` object that captures the mean, variance
|
2013-08-20 16:22:06 -04:00
|
|
|
and count of the RDD's elements in one operation.
|
|
|
|
"""
|
|
|
|
def redFunc(left_counter, right_counter):
|
|
|
|
return left_counter.mergeStats(right_counter)
|
|
|
|
|
|
|
|
return self.mapPartitions(lambda i: [StatCounter(i)]).reduce(redFunc)
|
|
|
|
|
[SPARK-2871] [PySpark] add histgram() API
RDD.histogram(buckets)
Compute a histogram using the provided buckets. The buckets
are all open to the right except for the last which is closed.
e.g. [1,10,20,50] means the buckets are [1,10) [10,20) [20,50],
which means 1<=x<10, 10<=x<20, 20<=x<=50. And on the input of 1
and 50 we would have a histogram of 1,0,1.
If your histogram is evenly spaced (e.g. [0, 10, 20, 30]),
this can be switched from an O(log n) inseration to O(1) per
element(where n = # buckets).
Buckets must be sorted and not contain any duplicates, must be
at least two elements.
If `buckets` is a number, it will generates buckets which is
evenly spaced between the minimum and maximum of the RDD. For
example, if the min value is 0 and the max is 100, given buckets
as 2, the resulting buckets will be [0,50) [50,100]. buckets must
be at least 1 If the RDD contains infinity, NaN throws an exception
If the elements in RDD do not vary (max == min) always returns
a single bucket.
It will return an tuple of buckets and histogram.
>>> rdd = sc.parallelize(range(51))
>>> rdd.histogram(2)
([0, 25, 50], [25, 26])
>>> rdd.histogram([0, 5, 25, 50])
([0, 5, 25, 50], [5, 20, 26])
>>> rdd.histogram([0, 15, 30, 45, 60], True)
([0, 15, 30, 45, 60], [15, 15, 15, 6])
>>> rdd = sc.parallelize(["ab", "ac", "b", "bd", "ef"])
>>> rdd.histogram(("a", "b", "c"))
(('a', 'b', 'c'), [2, 2])
closes #122, it's duplicated.
Author: Davies Liu <davies.liu@gmail.com>
Closes #2091 from davies/histgram and squashes the following commits:
a322f8a [Davies Liu] fix deprecation of e.message
84e85fa [Davies Liu] remove evenBuckets, add more tests (including str)
d9a0722 [Davies Liu] address comments
0e18a2d [Davies Liu] add histgram() API
2014-08-26 16:04:30 -04:00
|
|
|
def histogram(self, buckets):
|
|
|
|
"""
|
|
|
|
Compute a histogram using the provided buckets. The buckets
|
|
|
|
are all open to the right except for the last which is closed.
|
|
|
|
e.g. [1,10,20,50] means the buckets are [1,10) [10,20) [20,50],
|
|
|
|
which means 1<=x<10, 10<=x<20, 20<=x<=50. And on the input of 1
|
|
|
|
and 50 we would have a histogram of 1,0,1.
|
|
|
|
|
|
|
|
If your histogram is evenly spaced (e.g. [0, 10, 20, 30]),
|
2020-11-27 11:22:45 -05:00
|
|
|
this can be switched from an O(log n) insertion to O(1) per
|
2016-07-19 02:49:47 -04:00
|
|
|
element (where n is the number of buckets).
|
[SPARK-2871] [PySpark] add histgram() API
RDD.histogram(buckets)
Compute a histogram using the provided buckets. The buckets
are all open to the right except for the last which is closed.
e.g. [1,10,20,50] means the buckets are [1,10) [10,20) [20,50],
which means 1<=x<10, 10<=x<20, 20<=x<=50. And on the input of 1
and 50 we would have a histogram of 1,0,1.
If your histogram is evenly spaced (e.g. [0, 10, 20, 30]),
this can be switched from an O(log n) inseration to O(1) per
element(where n = # buckets).
Buckets must be sorted and not contain any duplicates, must be
at least two elements.
If `buckets` is a number, it will generates buckets which is
evenly spaced between the minimum and maximum of the RDD. For
example, if the min value is 0 and the max is 100, given buckets
as 2, the resulting buckets will be [0,50) [50,100]. buckets must
be at least 1 If the RDD contains infinity, NaN throws an exception
If the elements in RDD do not vary (max == min) always returns
a single bucket.
It will return an tuple of buckets and histogram.
>>> rdd = sc.parallelize(range(51))
>>> rdd.histogram(2)
([0, 25, 50], [25, 26])
>>> rdd.histogram([0, 5, 25, 50])
([0, 5, 25, 50], [5, 20, 26])
>>> rdd.histogram([0, 15, 30, 45, 60], True)
([0, 15, 30, 45, 60], [15, 15, 15, 6])
>>> rdd = sc.parallelize(["ab", "ac", "b", "bd", "ef"])
>>> rdd.histogram(("a", "b", "c"))
(('a', 'b', 'c'), [2, 2])
closes #122, it's duplicated.
Author: Davies Liu <davies.liu@gmail.com>
Closes #2091 from davies/histgram and squashes the following commits:
a322f8a [Davies Liu] fix deprecation of e.message
84e85fa [Davies Liu] remove evenBuckets, add more tests (including str)
d9a0722 [Davies Liu] address comments
0e18a2d [Davies Liu] add histgram() API
2014-08-26 16:04:30 -04:00
|
|
|
|
2016-07-19 02:49:47 -04:00
|
|
|
Buckets must be sorted, not contain any duplicates, and have
|
[SPARK-2871] [PySpark] add histgram() API
RDD.histogram(buckets)
Compute a histogram using the provided buckets. The buckets
are all open to the right except for the last which is closed.
e.g. [1,10,20,50] means the buckets are [1,10) [10,20) [20,50],
which means 1<=x<10, 10<=x<20, 20<=x<=50. And on the input of 1
and 50 we would have a histogram of 1,0,1.
If your histogram is evenly spaced (e.g. [0, 10, 20, 30]),
this can be switched from an O(log n) inseration to O(1) per
element(where n = # buckets).
Buckets must be sorted and not contain any duplicates, must be
at least two elements.
If `buckets` is a number, it will generates buckets which is
evenly spaced between the minimum and maximum of the RDD. For
example, if the min value is 0 and the max is 100, given buckets
as 2, the resulting buckets will be [0,50) [50,100]. buckets must
be at least 1 If the RDD contains infinity, NaN throws an exception
If the elements in RDD do not vary (max == min) always returns
a single bucket.
It will return an tuple of buckets and histogram.
>>> rdd = sc.parallelize(range(51))
>>> rdd.histogram(2)
([0, 25, 50], [25, 26])
>>> rdd.histogram([0, 5, 25, 50])
([0, 5, 25, 50], [5, 20, 26])
>>> rdd.histogram([0, 15, 30, 45, 60], True)
([0, 15, 30, 45, 60], [15, 15, 15, 6])
>>> rdd = sc.parallelize(["ab", "ac", "b", "bd", "ef"])
>>> rdd.histogram(("a", "b", "c"))
(('a', 'b', 'c'), [2, 2])
closes #122, it's duplicated.
Author: Davies Liu <davies.liu@gmail.com>
Closes #2091 from davies/histgram and squashes the following commits:
a322f8a [Davies Liu] fix deprecation of e.message
84e85fa [Davies Liu] remove evenBuckets, add more tests (including str)
d9a0722 [Davies Liu] address comments
0e18a2d [Davies Liu] add histgram() API
2014-08-26 16:04:30 -04:00
|
|
|
at least two elements.
|
|
|
|
|
2016-07-19 02:49:47 -04:00
|
|
|
If `buckets` is a number, it will generate buckets which are
|
[SPARK-2871] [PySpark] add histgram() API
RDD.histogram(buckets)
Compute a histogram using the provided buckets. The buckets
are all open to the right except for the last which is closed.
e.g. [1,10,20,50] means the buckets are [1,10) [10,20) [20,50],
which means 1<=x<10, 10<=x<20, 20<=x<=50. And on the input of 1
and 50 we would have a histogram of 1,0,1.
If your histogram is evenly spaced (e.g. [0, 10, 20, 30]),
this can be switched from an O(log n) inseration to O(1) per
element(where n = # buckets).
Buckets must be sorted and not contain any duplicates, must be
at least two elements.
If `buckets` is a number, it will generates buckets which is
evenly spaced between the minimum and maximum of the RDD. For
example, if the min value is 0 and the max is 100, given buckets
as 2, the resulting buckets will be [0,50) [50,100]. buckets must
be at least 1 If the RDD contains infinity, NaN throws an exception
If the elements in RDD do not vary (max == min) always returns
a single bucket.
It will return an tuple of buckets and histogram.
>>> rdd = sc.parallelize(range(51))
>>> rdd.histogram(2)
([0, 25, 50], [25, 26])
>>> rdd.histogram([0, 5, 25, 50])
([0, 5, 25, 50], [5, 20, 26])
>>> rdd.histogram([0, 15, 30, 45, 60], True)
([0, 15, 30, 45, 60], [15, 15, 15, 6])
>>> rdd = sc.parallelize(["ab", "ac", "b", "bd", "ef"])
>>> rdd.histogram(("a", "b", "c"))
(('a', 'b', 'c'), [2, 2])
closes #122, it's duplicated.
Author: Davies Liu <davies.liu@gmail.com>
Closes #2091 from davies/histgram and squashes the following commits:
a322f8a [Davies Liu] fix deprecation of e.message
84e85fa [Davies Liu] remove evenBuckets, add more tests (including str)
d9a0722 [Davies Liu] address comments
0e18a2d [Davies Liu] add histgram() API
2014-08-26 16:04:30 -04:00
|
|
|
evenly spaced between the minimum and maximum of the RDD. For
|
2016-07-19 02:49:47 -04:00
|
|
|
example, if the min value is 0 and the max is 100, given `buckets`
|
|
|
|
as 2, the resulting buckets will be [0,50) [50,100]. `buckets` must
|
|
|
|
be at least 1. An exception is raised if the RDD contains infinity.
|
|
|
|
If the elements in the RDD do not vary (max == min), a single bucket
|
|
|
|
will be used.
|
[SPARK-2871] [PySpark] add histgram() API
RDD.histogram(buckets)
Compute a histogram using the provided buckets. The buckets
are all open to the right except for the last which is closed.
e.g. [1,10,20,50] means the buckets are [1,10) [10,20) [20,50],
which means 1<=x<10, 10<=x<20, 20<=x<=50. And on the input of 1
and 50 we would have a histogram of 1,0,1.
If your histogram is evenly spaced (e.g. [0, 10, 20, 30]),
this can be switched from an O(log n) inseration to O(1) per
element(where n = # buckets).
Buckets must be sorted and not contain any duplicates, must be
at least two elements.
If `buckets` is a number, it will generates buckets which is
evenly spaced between the minimum and maximum of the RDD. For
example, if the min value is 0 and the max is 100, given buckets
as 2, the resulting buckets will be [0,50) [50,100]. buckets must
be at least 1 If the RDD contains infinity, NaN throws an exception
If the elements in RDD do not vary (max == min) always returns
a single bucket.
It will return an tuple of buckets and histogram.
>>> rdd = sc.parallelize(range(51))
>>> rdd.histogram(2)
([0, 25, 50], [25, 26])
>>> rdd.histogram([0, 5, 25, 50])
([0, 5, 25, 50], [5, 20, 26])
>>> rdd.histogram([0, 15, 30, 45, 60], True)
([0, 15, 30, 45, 60], [15, 15, 15, 6])
>>> rdd = sc.parallelize(["ab", "ac", "b", "bd", "ef"])
>>> rdd.histogram(("a", "b", "c"))
(('a', 'b', 'c'), [2, 2])
closes #122, it's duplicated.
Author: Davies Liu <davies.liu@gmail.com>
Closes #2091 from davies/histgram and squashes the following commits:
a322f8a [Davies Liu] fix deprecation of e.message
84e85fa [Davies Liu] remove evenBuckets, add more tests (including str)
d9a0722 [Davies Liu] address comments
0e18a2d [Davies Liu] add histgram() API
2014-08-26 16:04:30 -04:00
|
|
|
|
2016-07-19 02:49:47 -04:00
|
|
|
The return value is a tuple of buckets and histogram.
|
[SPARK-2871] [PySpark] add histgram() API
RDD.histogram(buckets)
Compute a histogram using the provided buckets. The buckets
are all open to the right except for the last which is closed.
e.g. [1,10,20,50] means the buckets are [1,10) [10,20) [20,50],
which means 1<=x<10, 10<=x<20, 20<=x<=50. And on the input of 1
and 50 we would have a histogram of 1,0,1.
If your histogram is evenly spaced (e.g. [0, 10, 20, 30]),
this can be switched from an O(log n) inseration to O(1) per
element(where n = # buckets).
Buckets must be sorted and not contain any duplicates, must be
at least two elements.
If `buckets` is a number, it will generates buckets which is
evenly spaced between the minimum and maximum of the RDD. For
example, if the min value is 0 and the max is 100, given buckets
as 2, the resulting buckets will be [0,50) [50,100]. buckets must
be at least 1 If the RDD contains infinity, NaN throws an exception
If the elements in RDD do not vary (max == min) always returns
a single bucket.
It will return an tuple of buckets and histogram.
>>> rdd = sc.parallelize(range(51))
>>> rdd.histogram(2)
([0, 25, 50], [25, 26])
>>> rdd.histogram([0, 5, 25, 50])
([0, 5, 25, 50], [5, 20, 26])
>>> rdd.histogram([0, 15, 30, 45, 60], True)
([0, 15, 30, 45, 60], [15, 15, 15, 6])
>>> rdd = sc.parallelize(["ab", "ac", "b", "bd", "ef"])
>>> rdd.histogram(("a", "b", "c"))
(('a', 'b', 'c'), [2, 2])
closes #122, it's duplicated.
Author: Davies Liu <davies.liu@gmail.com>
Closes #2091 from davies/histgram and squashes the following commits:
a322f8a [Davies Liu] fix deprecation of e.message
84e85fa [Davies Liu] remove evenBuckets, add more tests (including str)
d9a0722 [Davies Liu] address comments
0e18a2d [Davies Liu] add histgram() API
2014-08-26 16:04:30 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
[SPARK-2871] [PySpark] add histgram() API
RDD.histogram(buckets)
Compute a histogram using the provided buckets. The buckets
are all open to the right except for the last which is closed.
e.g. [1,10,20,50] means the buckets are [1,10) [10,20) [20,50],
which means 1<=x<10, 10<=x<20, 20<=x<=50. And on the input of 1
and 50 we would have a histogram of 1,0,1.
If your histogram is evenly spaced (e.g. [0, 10, 20, 30]),
this can be switched from an O(log n) inseration to O(1) per
element(where n = # buckets).
Buckets must be sorted and not contain any duplicates, must be
at least two elements.
If `buckets` is a number, it will generates buckets which is
evenly spaced between the minimum and maximum of the RDD. For
example, if the min value is 0 and the max is 100, given buckets
as 2, the resulting buckets will be [0,50) [50,100]. buckets must
be at least 1 If the RDD contains infinity, NaN throws an exception
If the elements in RDD do not vary (max == min) always returns
a single bucket.
It will return an tuple of buckets and histogram.
>>> rdd = sc.parallelize(range(51))
>>> rdd.histogram(2)
([0, 25, 50], [25, 26])
>>> rdd.histogram([0, 5, 25, 50])
([0, 5, 25, 50], [5, 20, 26])
>>> rdd.histogram([0, 15, 30, 45, 60], True)
([0, 15, 30, 45, 60], [15, 15, 15, 6])
>>> rdd = sc.parallelize(["ab", "ac", "b", "bd", "ef"])
>>> rdd.histogram(("a", "b", "c"))
(('a', 'b', 'c'), [2, 2])
closes #122, it's duplicated.
Author: Davies Liu <davies.liu@gmail.com>
Closes #2091 from davies/histgram and squashes the following commits:
a322f8a [Davies Liu] fix deprecation of e.message
84e85fa [Davies Liu] remove evenBuckets, add more tests (including str)
d9a0722 [Davies Liu] address comments
0e18a2d [Davies Liu] add histgram() API
2014-08-26 16:04:30 -04:00
|
|
|
>>> rdd = sc.parallelize(range(51))
|
|
|
|
>>> rdd.histogram(2)
|
|
|
|
([0, 25, 50], [25, 26])
|
|
|
|
>>> rdd.histogram([0, 5, 25, 50])
|
|
|
|
([0, 5, 25, 50], [5, 20, 26])
|
|
|
|
>>> rdd.histogram([0, 15, 30, 45, 60]) # evenly spaced buckets
|
|
|
|
([0, 15, 30, 45, 60], [15, 15, 15, 6])
|
|
|
|
>>> rdd = sc.parallelize(["ab", "ac", "b", "bd", "ef"])
|
|
|
|
>>> rdd.histogram(("a", "b", "c"))
|
|
|
|
(('a', 'b', 'c'), [2, 2])
|
|
|
|
"""
|
|
|
|
|
2015-04-16 19:20:57 -04:00
|
|
|
if isinstance(buckets, int):
|
[SPARK-2871] [PySpark] add histgram() API
RDD.histogram(buckets)
Compute a histogram using the provided buckets. The buckets
are all open to the right except for the last which is closed.
e.g. [1,10,20,50] means the buckets are [1,10) [10,20) [20,50],
which means 1<=x<10, 10<=x<20, 20<=x<=50. And on the input of 1
and 50 we would have a histogram of 1,0,1.
If your histogram is evenly spaced (e.g. [0, 10, 20, 30]),
this can be switched from an O(log n) inseration to O(1) per
element(where n = # buckets).
Buckets must be sorted and not contain any duplicates, must be
at least two elements.
If `buckets` is a number, it will generates buckets which is
evenly spaced between the minimum and maximum of the RDD. For
example, if the min value is 0 and the max is 100, given buckets
as 2, the resulting buckets will be [0,50) [50,100]. buckets must
be at least 1 If the RDD contains infinity, NaN throws an exception
If the elements in RDD do not vary (max == min) always returns
a single bucket.
It will return an tuple of buckets and histogram.
>>> rdd = sc.parallelize(range(51))
>>> rdd.histogram(2)
([0, 25, 50], [25, 26])
>>> rdd.histogram([0, 5, 25, 50])
([0, 5, 25, 50], [5, 20, 26])
>>> rdd.histogram([0, 15, 30, 45, 60], True)
([0, 15, 30, 45, 60], [15, 15, 15, 6])
>>> rdd = sc.parallelize(["ab", "ac", "b", "bd", "ef"])
>>> rdd.histogram(("a", "b", "c"))
(('a', 'b', 'c'), [2, 2])
closes #122, it's duplicated.
Author: Davies Liu <davies.liu@gmail.com>
Closes #2091 from davies/histgram and squashes the following commits:
a322f8a [Davies Liu] fix deprecation of e.message
84e85fa [Davies Liu] remove evenBuckets, add more tests (including str)
d9a0722 [Davies Liu] address comments
0e18a2d [Davies Liu] add histgram() API
2014-08-26 16:04:30 -04:00
|
|
|
if buckets < 1:
|
|
|
|
raise ValueError("number of buckets must be >= 1")
|
|
|
|
|
|
|
|
# filter out non-comparable elements
|
|
|
|
def comparable(x):
|
|
|
|
if x is None:
|
|
|
|
return False
|
|
|
|
if type(x) is float and isnan(x):
|
|
|
|
return False
|
|
|
|
return True
|
|
|
|
|
|
|
|
filtered = self.filter(comparable)
|
|
|
|
|
|
|
|
# faster than stats()
|
|
|
|
def minmax(a, b):
|
|
|
|
return min(a[0], b[0]), max(a[1], b[1])
|
|
|
|
try:
|
|
|
|
minv, maxv = filtered.map(lambda x: (x, x)).reduce(minmax)
|
|
|
|
except TypeError as e:
|
|
|
|
if " empty " in str(e):
|
|
|
|
raise ValueError("can not generate buckets from empty RDD")
|
|
|
|
raise
|
|
|
|
|
|
|
|
if minv == maxv or buckets == 1:
|
|
|
|
return [minv, maxv], [filtered.count()]
|
|
|
|
|
|
|
|
try:
|
|
|
|
inc = (maxv - minv) / buckets
|
|
|
|
except TypeError:
|
|
|
|
raise TypeError("Can not generate buckets with non-number in RDD")
|
|
|
|
|
|
|
|
if isinf(inc):
|
|
|
|
raise ValueError("Can not generate buckets with infinite value")
|
|
|
|
|
|
|
|
# keep them as integer if possible
|
2015-04-16 19:20:57 -04:00
|
|
|
inc = int(inc)
|
[SPARK-2871] [PySpark] add histgram() API
RDD.histogram(buckets)
Compute a histogram using the provided buckets. The buckets
are all open to the right except for the last which is closed.
e.g. [1,10,20,50] means the buckets are [1,10) [10,20) [20,50],
which means 1<=x<10, 10<=x<20, 20<=x<=50. And on the input of 1
and 50 we would have a histogram of 1,0,1.
If your histogram is evenly spaced (e.g. [0, 10, 20, 30]),
this can be switched from an O(log n) inseration to O(1) per
element(where n = # buckets).
Buckets must be sorted and not contain any duplicates, must be
at least two elements.
If `buckets` is a number, it will generates buckets which is
evenly spaced between the minimum and maximum of the RDD. For
example, if the min value is 0 and the max is 100, given buckets
as 2, the resulting buckets will be [0,50) [50,100]. buckets must
be at least 1 If the RDD contains infinity, NaN throws an exception
If the elements in RDD do not vary (max == min) always returns
a single bucket.
It will return an tuple of buckets and histogram.
>>> rdd = sc.parallelize(range(51))
>>> rdd.histogram(2)
([0, 25, 50], [25, 26])
>>> rdd.histogram([0, 5, 25, 50])
([0, 5, 25, 50], [5, 20, 26])
>>> rdd.histogram([0, 15, 30, 45, 60], True)
([0, 15, 30, 45, 60], [15, 15, 15, 6])
>>> rdd = sc.parallelize(["ab", "ac", "b", "bd", "ef"])
>>> rdd.histogram(("a", "b", "c"))
(('a', 'b', 'c'), [2, 2])
closes #122, it's duplicated.
Author: Davies Liu <davies.liu@gmail.com>
Closes #2091 from davies/histgram and squashes the following commits:
a322f8a [Davies Liu] fix deprecation of e.message
84e85fa [Davies Liu] remove evenBuckets, add more tests (including str)
d9a0722 [Davies Liu] address comments
0e18a2d [Davies Liu] add histgram() API
2014-08-26 16:04:30 -04:00
|
|
|
if inc * buckets != maxv - minv:
|
|
|
|
inc = (maxv - minv) * 1.0 / buckets
|
|
|
|
|
|
|
|
buckets = [i * inc + minv for i in range(buckets)]
|
|
|
|
buckets.append(maxv) # fix accumulated error
|
|
|
|
even = True
|
|
|
|
|
|
|
|
elif isinstance(buckets, (list, tuple)):
|
|
|
|
if len(buckets) < 2:
|
|
|
|
raise ValueError("buckets should have more than one value")
|
|
|
|
|
|
|
|
if any(i is None or isinstance(i, float) and isnan(i) for i in buckets):
|
|
|
|
raise ValueError("can not have None or NaN in buckets")
|
|
|
|
|
|
|
|
if sorted(buckets) != list(buckets):
|
|
|
|
raise ValueError("buckets should be sorted")
|
|
|
|
|
|
|
|
if len(set(buckets)) != len(buckets):
|
|
|
|
raise ValueError("buckets should not contain duplicated values")
|
|
|
|
|
|
|
|
minv = buckets[0]
|
|
|
|
maxv = buckets[-1]
|
|
|
|
even = False
|
|
|
|
inc = None
|
|
|
|
try:
|
|
|
|
steps = [buckets[i + 1] - buckets[i] for i in range(len(buckets) - 1)]
|
|
|
|
except TypeError:
|
|
|
|
pass # objects in buckets do not support '-'
|
|
|
|
else:
|
|
|
|
if max(steps) - min(steps) < 1e-10: # handle precision errors
|
|
|
|
even = True
|
|
|
|
inc = (maxv - minv) / (len(buckets) - 1)
|
|
|
|
|
|
|
|
else:
|
|
|
|
raise TypeError("buckets should be a list or tuple or number(int or long)")
|
|
|
|
|
|
|
|
def histogram(iterator):
|
|
|
|
counters = [0] * len(buckets)
|
|
|
|
for i in iterator:
|
|
|
|
if i is None or (type(i) is float and isnan(i)) or i > maxv or i < minv:
|
|
|
|
continue
|
|
|
|
t = (int((i - minv) / inc) if even
|
|
|
|
else bisect.bisect_right(buckets, i) - 1)
|
|
|
|
counters[t] += 1
|
|
|
|
# add last two together
|
|
|
|
last = counters.pop()
|
|
|
|
counters[-1] += last
|
|
|
|
return [counters]
|
|
|
|
|
|
|
|
def mergeCounters(a, b):
|
|
|
|
return [i + j for i, j in zip(a, b)]
|
|
|
|
|
|
|
|
return buckets, self.mapPartitions(histogram).reduce(mergeCounters)
|
|
|
|
|
2013-08-20 16:22:06 -04:00
|
|
|
def mean(self):
|
|
|
|
"""
|
|
|
|
Compute the mean of this RDD's elements.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2013-08-20 16:22:06 -04:00
|
|
|
>>> sc.parallelize([1, 2, 3]).mean()
|
|
|
|
2.0
|
|
|
|
"""
|
|
|
|
return self.stats().mean()
|
|
|
|
|
|
|
|
def variance(self):
|
|
|
|
"""
|
|
|
|
Compute the variance of this RDD's elements.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2013-08-20 16:22:06 -04:00
|
|
|
>>> sc.parallelize([1, 2, 3]).variance()
|
|
|
|
0.666...
|
|
|
|
"""
|
|
|
|
return self.stats().variance()
|
|
|
|
|
|
|
|
def stdev(self):
|
|
|
|
"""
|
|
|
|
Compute the standard deviation of this RDD's elements.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2013-08-20 16:22:06 -04:00
|
|
|
>>> sc.parallelize([1, 2, 3]).stdev()
|
|
|
|
0.816...
|
|
|
|
"""
|
|
|
|
return self.stats().stdev()
|
2013-10-09 15:08:04 -04:00
|
|
|
|
2013-08-20 16:22:06 -04:00
|
|
|
def sampleStdev(self):
|
|
|
|
"""
|
2014-06-12 22:44:27 -04:00
|
|
|
Compute the sample standard deviation of this RDD's elements (which
|
|
|
|
corrects for bias in estimating the standard deviation by dividing by
|
|
|
|
N-1 instead of N).
|
2013-08-20 16:22:06 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2013-08-20 16:22:06 -04:00
|
|
|
>>> sc.parallelize([1, 2, 3]).sampleStdev()
|
|
|
|
1.0
|
|
|
|
"""
|
|
|
|
return self.stats().sampleStdev()
|
|
|
|
|
|
|
|
def sampleVariance(self):
|
|
|
|
"""
|
2014-06-12 22:44:27 -04:00
|
|
|
Compute the sample variance of this RDD's elements (which corrects
|
|
|
|
for bias in estimating the variance by dividing by N-1 instead of N).
|
2013-08-20 16:22:06 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2013-08-20 16:22:06 -04:00
|
|
|
>>> sc.parallelize([1, 2, 3]).sampleVariance()
|
|
|
|
1.0
|
|
|
|
"""
|
|
|
|
return self.stats().sampleVariance()
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2012-08-25 01:51:45 -04:00
|
|
|
def countByValue(self):
|
|
|
|
"""
|
2012-12-27 20:55:33 -05:00
|
|
|
Return the count of each unique value in this RDD as a dictionary of
|
|
|
|
(value, count) pairs.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2012-09-17 03:08:50 -04:00
|
|
|
>>> sorted(sc.parallelize([1, 2, 1, 2, 2], 2).countByValue().items())
|
|
|
|
[(1, 2), (2, 3)]
|
2012-08-25 01:51:45 -04:00
|
|
|
"""
|
|
|
|
def countPartition(iterator):
|
2012-09-17 03:08:50 -04:00
|
|
|
counts = defaultdict(int)
|
|
|
|
for obj in iterator:
|
|
|
|
counts[obj] += 1
|
|
|
|
yield counts
|
2014-07-14 03:42:59 -04:00
|
|
|
|
2012-08-25 01:51:45 -04:00
|
|
|
def mergeMaps(m1, m2):
|
2015-04-16 19:20:57 -04:00
|
|
|
for k, v in m2.items():
|
2012-09-17 03:08:50 -04:00
|
|
|
m1[k] += v
|
|
|
|
return m1
|
2012-08-25 01:51:45 -04:00
|
|
|
return self.mapPartitions(countPartition).reduce(mergeMaps)
|
2014-06-09 03:08:40 -04:00
|
|
|
|
[SPARK-2871] [PySpark] add `key` argument for max(), min() and top(n)
RDD.max(key=None)
param key: A function used to generate key for comparing
>>> rdd = sc.parallelize([1.0, 5.0, 43.0, 10.0])
>>> rdd.max()
43.0
>>> rdd.max(key=str)
5.0
RDD.min(key=None)
Find the minimum item in this RDD.
param key: A function used to generate key for comparing
>>> rdd = sc.parallelize([2.0, 5.0, 43.0, 10.0])
>>> rdd.min()
2.0
>>> rdd.min(key=str)
10.0
RDD.top(num, key=None)
Get the top N elements from a RDD.
Note: It returns the list sorted in descending order.
>>> sc.parallelize([10, 4, 2, 12, 3]).top(1)
[12]
>>> sc.parallelize([2, 3, 4, 5, 6], 2).top(2)
[6, 5]
>>> sc.parallelize([10, 4, 2, 12, 3]).top(3, key=str)
[4, 3, 2]
Author: Davies Liu <davies.liu@gmail.com>
Closes #2094 from davies/cmp and squashes the following commits:
ccbaf25 [Davies Liu] add `key` to top()
ad7e374 [Davies Liu] fix tests
2f63512 [Davies Liu] change `comp` to `key` in min/max
dd91e08 [Davies Liu] add `comp` argument for RDD.max() and RDD.min()
2014-08-23 21:55:13 -04:00
|
|
|
def top(self, num, key=None):
|
2014-03-12 18:57:44 -04:00
|
|
|
"""
|
2016-11-19 06:24:15 -05:00
|
|
|
Get the top N elements from an RDD.
|
2014-03-12 18:57:44 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This method should only be used if the resulting array is expected
|
|
|
|
to be small, as all the data is loaded into the driver's memory.
|
2016-02-06 12:29:09 -05:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
It returns the list sorted in descending order.
|
2014-09-12 12:46:21 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2014-03-12 18:57:44 -04:00
|
|
|
>>> sc.parallelize([10, 4, 2, 12, 3]).top(1)
|
|
|
|
[12]
|
2014-06-17 15:03:22 -04:00
|
|
|
>>> sc.parallelize([2, 3, 4, 5, 6], 2).top(2)
|
2014-03-26 12:16:37 -04:00
|
|
|
[6, 5]
|
[SPARK-2871] [PySpark] add `key` argument for max(), min() and top(n)
RDD.max(key=None)
param key: A function used to generate key for comparing
>>> rdd = sc.parallelize([1.0, 5.0, 43.0, 10.0])
>>> rdd.max()
43.0
>>> rdd.max(key=str)
5.0
RDD.min(key=None)
Find the minimum item in this RDD.
param key: A function used to generate key for comparing
>>> rdd = sc.parallelize([2.0, 5.0, 43.0, 10.0])
>>> rdd.min()
2.0
>>> rdd.min(key=str)
10.0
RDD.top(num, key=None)
Get the top N elements from a RDD.
Note: It returns the list sorted in descending order.
>>> sc.parallelize([10, 4, 2, 12, 3]).top(1)
[12]
>>> sc.parallelize([2, 3, 4, 5, 6], 2).top(2)
[6, 5]
>>> sc.parallelize([10, 4, 2, 12, 3]).top(3, key=str)
[4, 3, 2]
Author: Davies Liu <davies.liu@gmail.com>
Closes #2094 from davies/cmp and squashes the following commits:
ccbaf25 [Davies Liu] add `key` to top()
ad7e374 [Davies Liu] fix tests
2f63512 [Davies Liu] change `comp` to `key` in min/max
dd91e08 [Davies Liu] add `comp` argument for RDD.max() and RDD.min()
2014-08-23 21:55:13 -04:00
|
|
|
>>> sc.parallelize([10, 4, 2, 12, 3]).top(3, key=str)
|
|
|
|
[4, 3, 2]
|
2014-03-12 18:57:44 -04:00
|
|
|
"""
|
|
|
|
def topIterator(iterator):
|
[SPARK-2871] [PySpark] add `key` argument for max(), min() and top(n)
RDD.max(key=None)
param key: A function used to generate key for comparing
>>> rdd = sc.parallelize([1.0, 5.0, 43.0, 10.0])
>>> rdd.max()
43.0
>>> rdd.max(key=str)
5.0
RDD.min(key=None)
Find the minimum item in this RDD.
param key: A function used to generate key for comparing
>>> rdd = sc.parallelize([2.0, 5.0, 43.0, 10.0])
>>> rdd.min()
2.0
>>> rdd.min(key=str)
10.0
RDD.top(num, key=None)
Get the top N elements from a RDD.
Note: It returns the list sorted in descending order.
>>> sc.parallelize([10, 4, 2, 12, 3]).top(1)
[12]
>>> sc.parallelize([2, 3, 4, 5, 6], 2).top(2)
[6, 5]
>>> sc.parallelize([10, 4, 2, 12, 3]).top(3, key=str)
[4, 3, 2]
Author: Davies Liu <davies.liu@gmail.com>
Closes #2094 from davies/cmp and squashes the following commits:
ccbaf25 [Davies Liu] add `key` to top()
ad7e374 [Davies Liu] fix tests
2f63512 [Davies Liu] change `comp` to `key` in min/max
dd91e08 [Davies Liu] add `comp` argument for RDD.max() and RDD.min()
2014-08-23 21:55:13 -04:00
|
|
|
yield heapq.nlargest(num, iterator, key=key)
|
2014-03-12 18:57:44 -04:00
|
|
|
|
|
|
|
def merge(a, b):
|
[SPARK-2871] [PySpark] add `key` argument for max(), min() and top(n)
RDD.max(key=None)
param key: A function used to generate key for comparing
>>> rdd = sc.parallelize([1.0, 5.0, 43.0, 10.0])
>>> rdd.max()
43.0
>>> rdd.max(key=str)
5.0
RDD.min(key=None)
Find the minimum item in this RDD.
param key: A function used to generate key for comparing
>>> rdd = sc.parallelize([2.0, 5.0, 43.0, 10.0])
>>> rdd.min()
2.0
>>> rdd.min(key=str)
10.0
RDD.top(num, key=None)
Get the top N elements from a RDD.
Note: It returns the list sorted in descending order.
>>> sc.parallelize([10, 4, 2, 12, 3]).top(1)
[12]
>>> sc.parallelize([2, 3, 4, 5, 6], 2).top(2)
[6, 5]
>>> sc.parallelize([10, 4, 2, 12, 3]).top(3, key=str)
[4, 3, 2]
Author: Davies Liu <davies.liu@gmail.com>
Closes #2094 from davies/cmp and squashes the following commits:
ccbaf25 [Davies Liu] add `key` to top()
ad7e374 [Davies Liu] fix tests
2f63512 [Davies Liu] change `comp` to `key` in min/max
dd91e08 [Davies Liu] add `comp` argument for RDD.max() and RDD.min()
2014-08-23 21:55:13 -04:00
|
|
|
return heapq.nlargest(num, a + b, key=key)
|
2014-03-12 18:57:44 -04:00
|
|
|
|
[SPARK-2871] [PySpark] add `key` argument for max(), min() and top(n)
RDD.max(key=None)
param key: A function used to generate key for comparing
>>> rdd = sc.parallelize([1.0, 5.0, 43.0, 10.0])
>>> rdd.max()
43.0
>>> rdd.max(key=str)
5.0
RDD.min(key=None)
Find the minimum item in this RDD.
param key: A function used to generate key for comparing
>>> rdd = sc.parallelize([2.0, 5.0, 43.0, 10.0])
>>> rdd.min()
2.0
>>> rdd.min(key=str)
10.0
RDD.top(num, key=None)
Get the top N elements from a RDD.
Note: It returns the list sorted in descending order.
>>> sc.parallelize([10, 4, 2, 12, 3]).top(1)
[12]
>>> sc.parallelize([2, 3, 4, 5, 6], 2).top(2)
[6, 5]
>>> sc.parallelize([10, 4, 2, 12, 3]).top(3, key=str)
[4, 3, 2]
Author: Davies Liu <davies.liu@gmail.com>
Closes #2094 from davies/cmp and squashes the following commits:
ccbaf25 [Davies Liu] add `key` to top()
ad7e374 [Davies Liu] fix tests
2f63512 [Davies Liu] change `comp` to `key` in min/max
dd91e08 [Davies Liu] add `comp` argument for RDD.max() and RDD.min()
2014-08-23 21:55:13 -04:00
|
|
|
return self.mapPartitions(topIterator).reduce(merge)
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2014-04-03 18:42:17 -04:00
|
|
|
def takeOrdered(self, num, key=None):
|
|
|
|
"""
|
2016-11-19 06:24:15 -05:00
|
|
|
Get the N elements from an RDD ordered in ascending order or as
|
2014-06-12 22:44:27 -04:00
|
|
|
specified by the optional key function.
|
2014-04-03 18:42:17 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This method should only be used if the resulting array is expected
|
|
|
|
to be small, as all the data is loaded into the driver's memory.
|
2016-02-06 12:29:09 -05:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2014-04-03 18:42:17 -04:00
|
|
|
>>> sc.parallelize([10, 1, 2, 9, 3, 4, 5, 6, 7]).takeOrdered(6)
|
|
|
|
[1, 2, 3, 4, 5, 6]
|
|
|
|
>>> sc.parallelize([10, 1, 2, 9, 3, 4, 5, 6, 7], 2).takeOrdered(6, key=lambda x: -x)
|
|
|
|
[10, 9, 7, 6, 5, 4]
|
|
|
|
"""
|
|
|
|
|
|
|
|
def merge(a, b):
|
2014-08-27 16:18:33 -04:00
|
|
|
return heapq.nsmallest(num, a + b, key)
|
|
|
|
|
|
|
|
return self.mapPartitions(lambda it: [heapq.nsmallest(num, it, key)]).reduce(merge)
|
2014-04-03 18:42:17 -04:00
|
|
|
|
2012-08-10 04:10:02 -04:00
|
|
|
def take(self, num):
|
|
|
|
"""
|
2012-12-27 20:55:33 -05:00
|
|
|
Take the first num elements of the RDD.
|
|
|
|
|
2014-05-31 16:04:57 -04:00
|
|
|
It works by first scanning one partition, and use the results from
|
|
|
|
that partition to estimate the number of additional partitions needed
|
|
|
|
to satisfy the limit.
|
|
|
|
|
|
|
|
Translated from the Scala implementation in RDD#take().
|
2012-12-27 20:55:33 -05:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This method should only be used if the resulting array is expected
|
|
|
|
to be small, as all the data is loaded into the driver's memory.
|
2016-11-22 06:40:18 -05:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2013-01-03 17:52:21 -05:00
|
|
|
>>> sc.parallelize([2, 3, 4, 5, 6]).cache().take(2)
|
2012-08-10 04:10:02 -04:00
|
|
|
[2, 3]
|
2012-12-29 01:19:12 -05:00
|
|
|
>>> sc.parallelize([2, 3, 4, 5, 6]).take(10)
|
|
|
|
[2, 3, 4, 5, 6]
|
2014-05-31 16:04:57 -04:00
|
|
|
>>> sc.parallelize(range(100), 100).filter(lambda x: x > 90).take(3)
|
|
|
|
[91, 92, 93]
|
2012-12-29 01:19:12 -05:00
|
|
|
"""
|
|
|
|
items = []
|
2015-04-15 15:58:02 -04:00
|
|
|
totalParts = self.getNumPartitions()
|
2014-05-31 16:04:57 -04:00
|
|
|
partsScanned = 0
|
|
|
|
|
|
|
|
while len(items) < num and partsScanned < totalParts:
|
|
|
|
# The number of partitions to try in this iteration.
|
|
|
|
# It is ok for this number to be greater than totalParts because
|
|
|
|
# we actually cap it at totalParts in runJob.
|
|
|
|
numPartsToTry = 1
|
|
|
|
if partsScanned > 0:
|
2014-09-05 21:52:05 -04:00
|
|
|
# If we didn't find any rows after the previous iteration,
|
|
|
|
# quadruple and retry. Otherwise, interpolate the number of
|
|
|
|
# partitions we need to try, but overestimate it by 50%.
|
2014-10-13 16:11:55 -04:00
|
|
|
# We also cap the estimation in the end.
|
2014-05-31 16:04:57 -04:00
|
|
|
if len(items) == 0:
|
2014-09-05 21:52:05 -04:00
|
|
|
numPartsToTry = partsScanned * 4
|
2014-05-31 16:04:57 -04:00
|
|
|
else:
|
2018-08-11 22:23:36 -04:00
|
|
|
# the first parameter of max is >=1 whenever partsScanned >= 2
|
2014-10-13 16:11:55 -04:00
|
|
|
numPartsToTry = int(1.5 * num * partsScanned / len(items)) - partsScanned
|
|
|
|
numPartsToTry = min(max(numPartsToTry, 1), partsScanned * 4)
|
2014-05-31 16:04:57 -04:00
|
|
|
|
|
|
|
left = num - len(items)
|
|
|
|
|
|
|
|
def takeUpToNumLeft(iterator):
|
2014-08-27 16:18:33 -04:00
|
|
|
iterator = iter(iterator)
|
2014-05-31 16:04:57 -04:00
|
|
|
taken = 0
|
|
|
|
while taken < left:
|
2018-07-06 23:37:41 -04:00
|
|
|
try:
|
|
|
|
yield next(iterator)
|
|
|
|
except StopIteration:
|
|
|
|
return
|
2014-05-31 16:04:57 -04:00
|
|
|
taken += 1
|
|
|
|
|
2014-08-27 16:18:33 -04:00
|
|
|
p = range(partsScanned, min(partsScanned + numPartsToTry, totalParts))
|
2015-07-23 00:04:04 -04:00
|
|
|
res = self.context.runJob(self, takeUpToNumLeft, p)
|
2014-05-31 16:04:57 -04:00
|
|
|
|
|
|
|
items += res
|
|
|
|
partsScanned += numPartsToTry
|
|
|
|
|
2012-12-29 01:19:12 -05:00
|
|
|
return items[:num]
|
2012-08-10 04:10:02 -04:00
|
|
|
|
|
|
|
def first(self):
|
|
|
|
"""
|
2012-12-27 20:55:33 -05:00
|
|
|
Return the first element in this RDD.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2012-08-10 04:10:02 -04:00
|
|
|
>>> sc.parallelize([2, 3, 4]).first()
|
|
|
|
2
|
2014-08-27 16:18:33 -04:00
|
|
|
>>> sc.parallelize([]).first()
|
|
|
|
Traceback (most recent call last):
|
|
|
|
...
|
|
|
|
ValueError: RDD is empty
|
2012-08-10 04:10:02 -04:00
|
|
|
"""
|
2014-08-27 16:18:33 -04:00
|
|
|
rs = self.take(1)
|
|
|
|
if rs:
|
|
|
|
return rs[0]
|
|
|
|
raise ValueError("RDD is empty")
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2015-01-20 01:50:44 -05:00
|
|
|
def isEmpty(self):
|
|
|
|
"""
|
2016-11-22 06:40:18 -05:00
|
|
|
Returns true if and only if the RDD contains no elements at all.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
An RDD may be empty even when it has at least 1 partition.
|
2015-01-20 01:50:44 -05:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2015-01-20 01:50:44 -05:00
|
|
|
>>> sc.parallelize([]).isEmpty()
|
|
|
|
True
|
|
|
|
>>> sc.parallelize([1]).isEmpty()
|
|
|
|
False
|
|
|
|
"""
|
2015-04-15 15:58:02 -04:00
|
|
|
return self.getNumPartitions() == 0 or len(self.take(1)) == 0
|
2015-01-20 01:50:44 -05:00
|
|
|
|
2014-07-30 16:19:05 -04:00
|
|
|
def saveAsNewAPIHadoopDataset(self, conf, keyConverter=None, valueConverter=None):
|
|
|
|
"""
|
2019-07-05 13:08:22 -04:00
|
|
|
Output a Python RDD of key-value pairs (of form ``RDD[(K, V)]``) to any Hadoop file
|
2014-07-30 16:19:05 -04:00
|
|
|
system, using the new Hadoop OutputFormat API (mapreduce package). Keys/values are
|
|
|
|
converted for output using either user specified converters or, by default,
|
2019-07-05 13:08:22 -04:00
|
|
|
"org.apache.spark.api.python.JavaToWritableConverter".
|
2014-07-30 16:19:05 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
conf : dict
|
|
|
|
Hadoop job configuration
|
|
|
|
keyConverter : str, optional
|
|
|
|
fully qualified classname of key converter (None by default)
|
|
|
|
valueConverter : str, optional
|
|
|
|
fully qualified classname of value converter (None by default)
|
2014-07-30 16:19:05 -04:00
|
|
|
"""
|
|
|
|
jconf = self.ctx._dictToJavaMap(conf)
|
2014-11-04 02:56:14 -05:00
|
|
|
pickledRDD = self._pickled()
|
|
|
|
self.ctx._jvm.PythonRDD.saveAsHadoopDataset(pickledRDD._jrdd, True, jconf,
|
2014-07-30 16:19:05 -04:00
|
|
|
keyConverter, valueConverter, True)
|
|
|
|
|
|
|
|
def saveAsNewAPIHadoopFile(self, path, outputFormatClass, keyClass=None, valueClass=None,
|
|
|
|
keyConverter=None, valueConverter=None, conf=None):
|
|
|
|
"""
|
2019-07-05 13:08:22 -04:00
|
|
|
Output a Python RDD of key-value pairs (of form ``RDD[(K, V)]``) to any Hadoop file
|
2014-07-30 16:19:05 -04:00
|
|
|
system, using the new Hadoop OutputFormat API (mapreduce package). Key and value types
|
|
|
|
will be inferred if not specified. Keys and values are converted for output using either
|
2019-07-05 13:08:22 -04:00
|
|
|
user specified converters or "org.apache.spark.api.python.JavaToWritableConverter". The
|
|
|
|
`conf` is applied on top of the base Hadoop conf associated with the SparkContext
|
2014-07-30 16:19:05 -04:00
|
|
|
of this RDD to create a merged Hadoop MapReduce job configuration for saving the data.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
path : str
|
|
|
|
path to Hadoop file
|
|
|
|
outputFormatClass : str
|
|
|
|
fully qualified classname of Hadoop OutputFormat
|
|
|
|
(e.g. "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat")
|
|
|
|
keyClass : str, optional
|
|
|
|
fully qualified classname of key Writable class
|
|
|
|
(e.g. "org.apache.hadoop.io.IntWritable", None by default)
|
|
|
|
valueClass : str, optional
|
|
|
|
fully qualified classname of value Writable class
|
|
|
|
(e.g. "org.apache.hadoop.io.Text", None by default)
|
|
|
|
keyConverter : str, optional
|
|
|
|
fully qualified classname of key converter (None by default)
|
|
|
|
valueConverter : str, optional
|
|
|
|
fully qualified classname of value converter (None by default)
|
|
|
|
conf : dict, optional
|
|
|
|
Hadoop job configuration (None by default)
|
2014-07-30 16:19:05 -04:00
|
|
|
"""
|
|
|
|
jconf = self.ctx._dictToJavaMap(conf)
|
2014-11-04 02:56:14 -05:00
|
|
|
pickledRDD = self._pickled()
|
|
|
|
self.ctx._jvm.PythonRDD.saveAsNewAPIHadoopFile(pickledRDD._jrdd, True, path,
|
2014-08-06 15:58:24 -04:00
|
|
|
outputFormatClass,
|
|
|
|
keyClass, valueClass,
|
|
|
|
keyConverter, valueConverter, jconf)
|
2014-07-30 16:19:05 -04:00
|
|
|
|
|
|
|
def saveAsHadoopDataset(self, conf, keyConverter=None, valueConverter=None):
|
|
|
|
"""
|
2019-07-05 13:08:22 -04:00
|
|
|
Output a Python RDD of key-value pairs (of form ``RDD[(K, V)]``) to any Hadoop file
|
2014-07-30 16:19:05 -04:00
|
|
|
system, using the old Hadoop OutputFormat API (mapred package). Keys/values are
|
|
|
|
converted for output using either user specified converters or, by default,
|
2019-07-05 13:08:22 -04:00
|
|
|
"org.apache.spark.api.python.JavaToWritableConverter".
|
2014-07-30 16:19:05 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
conf : dict
|
|
|
|
Hadoop job configuration
|
|
|
|
keyConverter : str, optional
|
|
|
|
fully qualified classname of key converter (None by default)
|
|
|
|
valueConverter : str, optional
|
|
|
|
fully qualified classname of value converter (None by default)
|
2014-07-30 16:19:05 -04:00
|
|
|
"""
|
|
|
|
jconf = self.ctx._dictToJavaMap(conf)
|
2014-11-04 02:56:14 -05:00
|
|
|
pickledRDD = self._pickled()
|
|
|
|
self.ctx._jvm.PythonRDD.saveAsHadoopDataset(pickledRDD._jrdd, True, jconf,
|
2014-07-30 16:19:05 -04:00
|
|
|
keyConverter, valueConverter, False)
|
|
|
|
|
|
|
|
def saveAsHadoopFile(self, path, outputFormatClass, keyClass=None, valueClass=None,
|
|
|
|
keyConverter=None, valueConverter=None, conf=None,
|
|
|
|
compressionCodecClass=None):
|
|
|
|
"""
|
2019-07-05 13:08:22 -04:00
|
|
|
Output a Python RDD of key-value pairs (of form ``RDD[(K, V)]``) to any Hadoop file
|
2014-07-30 16:19:05 -04:00
|
|
|
system, using the old Hadoop OutputFormat API (mapred package). Key and value types
|
|
|
|
will be inferred if not specified. Keys and values are converted for output using either
|
2019-07-05 13:08:22 -04:00
|
|
|
user specified converters or "org.apache.spark.api.python.JavaToWritableConverter". The
|
|
|
|
`conf` is applied on top of the base Hadoop conf associated with the SparkContext
|
2014-07-30 16:19:05 -04:00
|
|
|
of this RDD to create a merged Hadoop MapReduce job configuration for saving the data.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
path : str
|
|
|
|
path to Hadoop file
|
|
|
|
outputFormatClass : str
|
|
|
|
fully qualified classname of Hadoop OutputFormat
|
|
|
|
(e.g. "org.apache.hadoop.mapred.SequenceFileOutputFormat")
|
|
|
|
keyClass : str, optional
|
|
|
|
fully qualified classname of key Writable class
|
|
|
|
(e.g. "org.apache.hadoop.io.IntWritable", None by default)
|
|
|
|
valueClass : str, optional
|
|
|
|
fully qualified classname of value Writable class
|
|
|
|
(e.g. "org.apache.hadoop.io.Text", None by default)
|
|
|
|
keyConverter : str, optional
|
|
|
|
fully qualified classname of key converter (None by default)
|
|
|
|
valueConverter : str, optional
|
|
|
|
fully qualified classname of value converter (None by default)
|
|
|
|
conf : dict, optional
|
|
|
|
(None by default)
|
|
|
|
compressionCodecClass : str
|
|
|
|
fully qualified classname of the compression codec class
|
|
|
|
i.e. "org.apache.hadoop.io.compress.GzipCodec" (None by default)
|
2014-07-30 16:19:05 -04:00
|
|
|
"""
|
|
|
|
jconf = self.ctx._dictToJavaMap(conf)
|
2014-11-04 02:56:14 -05:00
|
|
|
pickledRDD = self._pickled()
|
|
|
|
self.ctx._jvm.PythonRDD.saveAsHadoopFile(pickledRDD._jrdd, True, path,
|
2014-08-06 15:58:24 -04:00
|
|
|
outputFormatClass,
|
|
|
|
keyClass, valueClass,
|
|
|
|
keyConverter, valueConverter,
|
|
|
|
jconf, compressionCodecClass)
|
2014-07-30 16:19:05 -04:00
|
|
|
|
|
|
|
def saveAsSequenceFile(self, path, compressionCodecClass=None):
|
|
|
|
"""
|
2019-07-05 13:08:22 -04:00
|
|
|
Output a Python RDD of key-value pairs (of form ``RDD[(K, V)]``) to any Hadoop file
|
|
|
|
system, using the "org.apache.hadoop.io.Writable" types that we convert from the
|
2014-07-30 16:19:05 -04:00
|
|
|
RDD's key and value types. The mechanism is as follows:
|
2014-10-06 17:08:40 -04:00
|
|
|
|
2014-07-30 16:19:05 -04:00
|
|
|
1. Pyrolite is used to convert pickled Python RDD into RDD of Java objects.
|
|
|
|
2. Keys and values of this Java RDD are converted to Writables and written out.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
path : str
|
|
|
|
path to sequence file
|
|
|
|
compressionCodecClass : str, optional
|
|
|
|
fully qualified classname of the compression codec class
|
|
|
|
i.e. "org.apache.hadoop.io.compress.GzipCodec" (None by default)
|
2014-07-30 16:19:05 -04:00
|
|
|
"""
|
2014-11-04 02:56:14 -05:00
|
|
|
pickledRDD = self._pickled()
|
|
|
|
self.ctx._jvm.PythonRDD.saveAsSequenceFile(pickledRDD._jrdd, True,
|
2014-07-30 16:19:05 -04:00
|
|
|
path, compressionCodecClass)
|
|
|
|
|
2014-06-03 21:18:25 -04:00
|
|
|
def saveAsPickleFile(self, path, batchSize=10):
|
|
|
|
"""
|
2014-06-12 22:44:27 -04:00
|
|
|
Save this RDD as a SequenceFile of serialized objects. The serializer
|
2019-07-05 13:08:22 -04:00
|
|
|
used is :class:`pyspark.serializers.PickleSerializer`, default batch size
|
2014-06-12 22:44:27 -04:00
|
|
|
is 10.
|
2014-06-03 21:18:25 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2020-08-08 11:51:57 -04:00
|
|
|
>>> from tempfile import NamedTemporaryFile
|
2014-06-03 21:18:25 -04:00
|
|
|
>>> tmpFile = NamedTemporaryFile(delete=True)
|
|
|
|
>>> tmpFile.close()
|
|
|
|
>>> sc.parallelize([1, 2, 'spark', 'rdd']).saveAsPickleFile(tmpFile.name, 3)
|
2015-04-16 19:20:57 -04:00
|
|
|
>>> sorted(sc.pickleFile(tmpFile.name, 5).map(str).collect())
|
|
|
|
['1', '2', 'rdd', 'spark']
|
2014-06-03 21:18:25 -04:00
|
|
|
"""
|
2014-11-04 02:56:14 -05:00
|
|
|
if batchSize == 0:
|
|
|
|
ser = AutoBatchedSerializer(PickleSerializer())
|
|
|
|
else:
|
|
|
|
ser = BatchedSerializer(PickleSerializer(), batchSize)
|
|
|
|
self._reserialize(ser)._jrdd.saveAsObjectFile(path)
|
2014-06-03 21:18:25 -04:00
|
|
|
|
2015-02-06 16:55:02 -05:00
|
|
|
def saveAsTextFile(self, path, compressionCodecClass=None):
|
2012-12-27 20:55:33 -05:00
|
|
|
"""
|
|
|
|
Save this RDD as a text file, using string representations of elements.
|
2012-12-29 20:06:50 -05:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
path : str
|
|
|
|
path to text file
|
|
|
|
compressionCodecClass : str, optional
|
|
|
|
fully qualified classname of the compression codec class
|
|
|
|
i.e. "org.apache.hadoop.io.compress.GzipCodec" (None by default)
|
2015-02-06 16:55:02 -05:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2020-08-08 11:51:57 -04:00
|
|
|
>>> from tempfile import NamedTemporaryFile
|
2012-12-29 20:06:50 -05:00
|
|
|
>>> tempFile = NamedTemporaryFile(delete=True)
|
|
|
|
>>> tempFile.close()
|
|
|
|
>>> sc.parallelize(range(10)).saveAsTextFile(tempFile.name)
|
|
|
|
>>> from fileinput import input
|
|
|
|
>>> from glob import glob
|
2013-04-02 14:58:40 -04:00
|
|
|
>>> ''.join(sorted(input(glob(tempFile.name + "/part-0000*"))))
|
2012-12-29 20:06:50 -05:00
|
|
|
'0\\n1\\n2\\n3\\n4\\n5\\n6\\n7\\n8\\n9\\n'
|
2014-05-10 17:01:08 -04:00
|
|
|
|
|
|
|
Empty lines are tolerated when saving to text files.
|
|
|
|
|
2020-08-08 11:51:57 -04:00
|
|
|
>>> from tempfile import NamedTemporaryFile
|
2014-05-10 17:01:08 -04:00
|
|
|
>>> tempFile2 = NamedTemporaryFile(delete=True)
|
|
|
|
>>> tempFile2.close()
|
|
|
|
>>> sc.parallelize(['', 'foo', '', 'bar', '']).saveAsTextFile(tempFile2.name)
|
|
|
|
>>> ''.join(sorted(input(glob(tempFile2.name + "/part-0000*"))))
|
|
|
|
'\\n\\n\\nbar\\nfoo\\n'
|
2015-02-06 16:55:02 -05:00
|
|
|
|
|
|
|
Using compressionCodecClass
|
|
|
|
|
2020-08-08 11:51:57 -04:00
|
|
|
>>> from tempfile import NamedTemporaryFile
|
2015-02-06 16:55:02 -05:00
|
|
|
>>> tempFile3 = NamedTemporaryFile(delete=True)
|
|
|
|
>>> tempFile3.close()
|
|
|
|
>>> codec = "org.apache.hadoop.io.compress.GzipCodec"
|
|
|
|
>>> sc.parallelize(['foo', 'bar']).saveAsTextFile(tempFile3.name, codec)
|
|
|
|
>>> from fileinput import input, hook_compressed
|
2015-04-16 19:20:57 -04:00
|
|
|
>>> result = sorted(input(glob(tempFile3.name + "/part*.gz"), openhook=hook_compressed))
|
|
|
|
>>> b''.join(result).decode('utf-8')
|
2020-07-13 22:22:44 -04:00
|
|
|
'bar\\nfoo\\n'
|
2012-12-27 20:55:33 -05:00
|
|
|
"""
|
2013-01-08 19:04:41 -05:00
|
|
|
def func(split, iterator):
|
2013-11-29 02:44:56 -05:00
|
|
|
for x in iterator:
|
2020-07-13 22:22:44 -04:00
|
|
|
if not isinstance(x, (str, bytes)):
|
|
|
|
x = str(x)
|
|
|
|
if isinstance(x, str):
|
2014-08-18 16:58:35 -04:00
|
|
|
x = x.encode("utf-8")
|
|
|
|
yield x
|
[SPARK-2010] [PySpark] [SQL] support nested structure in SchemaRDD
Convert Row in JavaSchemaRDD into Array[Any] and unpickle them as tuple in Python, then convert them into namedtuple, so use can access fields just like attributes.
This will let nested structure can be accessed as object, also it will reduce the size of serialized data and better performance.
root
|-- field1: integer (nullable = true)
|-- field2: string (nullable = true)
|-- field3: struct (nullable = true)
| |-- field4: integer (nullable = true)
| |-- field5: array (nullable = true)
| | |-- element: integer (containsNull = false)
|-- field6: array (nullable = true)
| |-- element: struct (containsNull = false)
| | |-- field7: string (nullable = true)
Then we can access them by row.field3.field5[0] or row.field6[5].field7
It also will infer the schema in Python, convert Row/dict/namedtuple/objects into tuple before serialization, then call applySchema in JVM. During inferSchema(), the top level of dict in row will be StructType, but any nested dictionary will be MapType.
You can use pyspark.sql.Row to convert unnamed structure into Row object, make the RDD can be inferable. Such as:
ctx.inferSchema(rdd.map(lambda x: Row(a=x[0], b=x[1]))
Or you could use Row to create a class just like namedtuple, for example:
Person = Row("name", "age")
ctx.inferSchema(rdd.map(lambda x: Person(*x)))
Also, you can call applySchema to apply an schema to a RDD of tuple/list and turn it into a SchemaRDD. The `schema` should be StructType, see the API docs for details.
schema = StructType([StructField("name, StringType, True),
StructType("age", IntegerType, True)])
ctx.applySchema(rdd, schema)
PS: In order to use namedtuple to inferSchema, you should make namedtuple picklable.
Author: Davies Liu <davies.liu@gmail.com>
Closes #1598 from davies/nested and squashes the following commits:
f1d15b6 [Davies Liu] verify schema with the first few rows
8852aaf [Davies Liu] check type of schema
abe9e6e [Davies Liu] address comments
61b2292 [Davies Liu] add @deprecated to pythonToJavaMap
1e5b801 [Davies Liu] improve cache of classes
51aa135 [Davies Liu] use Row to infer schema
e9c0d5c [Davies Liu] remove string typed schema
353a3f2 [Davies Liu] fix code style
63de8f8 [Davies Liu] fix typo
c79ca67 [Davies Liu] fix serialization of nested data
6b258b5 [Davies Liu] fix pep8
9d8447c [Davies Liu] apply schema provided by string of names
f5df97f [Davies Liu] refactor, address comments
9d9af55 [Davies Liu] use arrry to applySchema and infer schema in Python
84679b3 [Davies Liu] Merge branch 'master' of github.com:apache/spark into nested
0eaaf56 [Davies Liu] fix doc tests
b3559b4 [Davies Liu] use generated Row instead of namedtuple
c4ddc30 [Davies Liu] fix conflict between name of fields and variables
7f6f251 [Davies Liu] address all comments
d69d397 [Davies Liu] refactor
2cc2d45 [Davies Liu] refactor
182fb46 [Davies Liu] refactor
bc6e9e1 [Davies Liu] switch to new Schema API
547bf3e [Davies Liu] Merge branch 'master' into nested
a435b5a [Davies Liu] add docs and code refactor
2c8debc [Davies Liu] Merge branch 'master' into nested
644665a [Davies Liu] use tuple and namedtuple for schemardd
2014-08-01 21:47:41 -04:00
|
|
|
keyed = self.mapPartitionsWithIndex(func)
|
2012-08-25 21:00:25 -04:00
|
|
|
keyed._bypass_serializer = True
|
2015-02-06 16:55:02 -05:00
|
|
|
if compressionCodecClass:
|
|
|
|
compressionCodec = self.ctx._jvm.java.lang.Class.forName(compressionCodecClass)
|
|
|
|
keyed._jrdd.map(self.ctx._jvm.BytesToString()).saveAsTextFile(path, compressionCodec)
|
|
|
|
else:
|
|
|
|
keyed._jrdd.map(self.ctx._jvm.BytesToString()).saveAsTextFile(path)
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2012-08-18 19:07:10 -04:00
|
|
|
# Pair functions
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2012-08-18 19:07:10 -04:00
|
|
|
def collectAsMap(self):
|
|
|
|
"""
|
2012-12-27 20:55:33 -05:00
|
|
|
Return the key-value pairs in this RDD to the master as a dictionary.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This method should only be used if the resulting data is expected
|
|
|
|
to be small, as all the data is loaded into the driver's memory.
|
2016-02-06 12:29:09 -05:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2012-08-18 19:07:10 -04:00
|
|
|
>>> m = sc.parallelize([(1, 2), (3, 4)]).collectAsMap()
|
|
|
|
>>> m[1]
|
|
|
|
2
|
|
|
|
>>> m[3]
|
|
|
|
4
|
|
|
|
"""
|
|
|
|
return dict(self.collect())
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2014-02-06 17:58:35 -05:00
|
|
|
def keys(self):
|
|
|
|
"""
|
|
|
|
Return an RDD with the keys of each tuple.
|
2014-08-06 17:12:21 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2014-02-06 17:58:35 -05:00
|
|
|
>>> m = sc.parallelize([(1, 2), (3, 4)]).keys()
|
|
|
|
>>> m.collect()
|
|
|
|
[1, 3]
|
|
|
|
"""
|
2015-04-16 19:20:57 -04:00
|
|
|
return self.map(lambda x: x[0])
|
2014-02-06 17:58:35 -05:00
|
|
|
|
|
|
|
def values(self):
|
|
|
|
"""
|
|
|
|
Return an RDD with the values of each tuple.
|
2014-08-06 17:12:21 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2014-02-06 17:58:35 -05:00
|
|
|
>>> m = sc.parallelize([(1, 2), (3, 4)]).values()
|
|
|
|
>>> m.collect()
|
|
|
|
[2, 4]
|
|
|
|
"""
|
2015-04-16 19:20:57 -04:00
|
|
|
return self.map(lambda x: x[1])
|
2014-02-06 17:58:35 -05:00
|
|
|
|
[SPARK-9821] [PYSPARK] pyspark-reduceByKey-should-take-a-custom-partitioner
from the issue:
In Scala, I can supply a custom partitioner to reduceByKey (and other aggregation/repartitioning methods like aggregateByKey and combinedByKey), but as far as I can tell from the Pyspark API, there's no way to do the same in Python.
Here's an example of my code in Scala:
weblogs.map(s => (getFileType(s), 1)).reduceByKey(new FileTypePartitioner(),_+_)
But I can't figure out how to do the same in Python. The closest I can get is to call repartition before reduceByKey like so:
weblogs.map(lambda s: (getFileType(s), 1)).partitionBy(3,hash_filetype).reduceByKey(lambda v1,v2: v1+v2).collect()
But that defeats the purpose, because I'm shuffling twice instead of once, so my performance is worse instead of better.
Author: Holden Karau <holden@pigscanfly.ca>
Closes #8569 from holdenk/SPARK-9821-pyspark-reduceByKey-should-take-a-custom-partitioner.
2015-09-22 02:21:24 -04:00
|
|
|
def reduceByKey(self, func, numPartitions=None, partitionFunc=portable_hash):
|
2012-08-10 04:10:02 -04:00
|
|
|
"""
|
2016-02-19 05:26:38 -05:00
|
|
|
Merge the values for each key using an associative and commutative reduce function.
|
2012-12-27 20:55:33 -05:00
|
|
|
|
|
|
|
This will also perform the merging locally on each mapper before
|
|
|
|
sending results to a reducer, similarly to a "combiner" in MapReduce.
|
|
|
|
|
2019-07-05 13:08:22 -04:00
|
|
|
Output will be partitioned with `numPartitions` partitions, or
|
|
|
|
the default parallelism level if `numPartitions` is not specified.
|
[SPARK-9821] [PYSPARK] pyspark-reduceByKey-should-take-a-custom-partitioner
from the issue:
In Scala, I can supply a custom partitioner to reduceByKey (and other aggregation/repartitioning methods like aggregateByKey and combinedByKey), but as far as I can tell from the Pyspark API, there's no way to do the same in Python.
Here's an example of my code in Scala:
weblogs.map(s => (getFileType(s), 1)).reduceByKey(new FileTypePartitioner(),_+_)
But I can't figure out how to do the same in Python. The closest I can get is to call repartition before reduceByKey like so:
weblogs.map(lambda s: (getFileType(s), 1)).partitionBy(3,hash_filetype).reduceByKey(lambda v1,v2: v1+v2).collect()
But that defeats the purpose, because I'm shuffling twice instead of once, so my performance is worse instead of better.
Author: Holden Karau <holden@pigscanfly.ca>
Closes #8569 from holdenk/SPARK-9821-pyspark-reduceByKey-should-take-a-custom-partitioner.
2015-09-22 02:21:24 -04:00
|
|
|
Default partitioner is hash-partition.
|
2012-12-27 20:55:33 -05:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2012-08-25 02:09:15 -04:00
|
|
|
>>> from operator import add
|
|
|
|
>>> rdd = sc.parallelize([("a", 1), ("b", 1), ("a", 1)])
|
|
|
|
>>> sorted(rdd.reduceByKey(add).collect())
|
2012-08-10 04:10:02 -04:00
|
|
|
[('a', 2), ('b', 1)]
|
|
|
|
"""
|
[SPARK-9821] [PYSPARK] pyspark-reduceByKey-should-take-a-custom-partitioner
from the issue:
In Scala, I can supply a custom partitioner to reduceByKey (and other aggregation/repartitioning methods like aggregateByKey and combinedByKey), but as far as I can tell from the Pyspark API, there's no way to do the same in Python.
Here's an example of my code in Scala:
weblogs.map(s => (getFileType(s), 1)).reduceByKey(new FileTypePartitioner(),_+_)
But I can't figure out how to do the same in Python. The closest I can get is to call repartition before reduceByKey like so:
weblogs.map(lambda s: (getFileType(s), 1)).partitionBy(3,hash_filetype).reduceByKey(lambda v1,v2: v1+v2).collect()
But that defeats the purpose, because I'm shuffling twice instead of once, so my performance is worse instead of better.
Author: Holden Karau <holden@pigscanfly.ca>
Closes #8569 from holdenk/SPARK-9821-pyspark-reduceByKey-should-take-a-custom-partitioner.
2015-09-22 02:21:24 -04:00
|
|
|
return self.combineByKey(lambda x: x, func, func, numPartitions, partitionFunc)
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2012-08-25 02:09:15 -04:00
|
|
|
def reduceByKeyLocally(self, func):
|
|
|
|
"""
|
2016-02-19 05:26:38 -05:00
|
|
|
Merge the values for each key using an associative and commutative reduce function, but
|
2012-12-27 20:55:33 -05:00
|
|
|
return the results immediately to the master as a dictionary.
|
|
|
|
|
|
|
|
This will also perform the merging locally on each mapper before
|
|
|
|
sending results to a reducer, similarly to a "combiner" in MapReduce.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2012-08-25 02:09:15 -04:00
|
|
|
>>> from operator import add
|
|
|
|
>>> rdd = sc.parallelize([("a", 1), ("b", 1), ("a", 1)])
|
|
|
|
>>> sorted(rdd.reduceByKeyLocally(add).items())
|
|
|
|
[('a', 2), ('b', 1)]
|
|
|
|
"""
|
2018-05-30 06:11:33 -04:00
|
|
|
func = fail_on_stopiteration(func)
|
|
|
|
|
2012-08-25 02:09:15 -04:00
|
|
|
def reducePartition(iterator):
|
|
|
|
m = {}
|
2014-08-27 16:18:33 -04:00
|
|
|
for k, v in iterator:
|
|
|
|
m[k] = func(m[k], v) if k in m else v
|
2012-08-25 02:09:15 -04:00
|
|
|
yield m
|
2014-07-14 03:42:59 -04:00
|
|
|
|
2012-08-25 02:09:15 -04:00
|
|
|
def mergeMaps(m1, m2):
|
2015-04-16 19:20:57 -04:00
|
|
|
for k, v in m2.items():
|
2014-08-27 16:18:33 -04:00
|
|
|
m1[k] = func(m1[k], v) if k in m1 else v
|
2012-08-25 02:09:15 -04:00
|
|
|
return m1
|
|
|
|
return self.mapPartitions(reducePartition).reduce(mergeMaps)
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2012-08-25 02:09:15 -04:00
|
|
|
def countByKey(self):
|
|
|
|
"""
|
2012-12-27 20:55:33 -05:00
|
|
|
Count the number of elements for each key, and return the result to the
|
|
|
|
master as a dictionary.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2012-08-25 02:09:15 -04:00
|
|
|
>>> rdd = sc.parallelize([("a", 1), ("b", 1), ("a", 1)])
|
2012-09-17 03:08:50 -04:00
|
|
|
>>> sorted(rdd.countByKey().items())
|
2012-08-25 02:09:15 -04:00
|
|
|
[('a', 2), ('b', 1)]
|
|
|
|
"""
|
|
|
|
return self.map(lambda x: x[0]).countByValue()
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2013-02-24 16:25:09 -05:00
|
|
|
def join(self, other, numPartitions=None):
|
2012-08-10 04:10:02 -04:00
|
|
|
"""
|
2012-12-27 20:55:33 -05:00
|
|
|
Return an RDD containing all pairs of elements with matching keys in
|
2019-07-05 13:08:22 -04:00
|
|
|
`self` and `other`.
|
2012-12-27 20:55:33 -05:00
|
|
|
|
|
|
|
Each pair of elements will be returned as a (k, (v1, v2)) tuple, where
|
2019-07-05 13:08:22 -04:00
|
|
|
(k, v1) is in `self` and (k, v2) is in `other`.
|
2012-12-27 20:55:33 -05:00
|
|
|
|
|
|
|
Performs a hash join across the cluster.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2012-08-18 19:07:10 -04:00
|
|
|
>>> x = sc.parallelize([("a", 1), ("b", 4)])
|
|
|
|
>>> y = sc.parallelize([("a", 2), ("a", 3)])
|
|
|
|
>>> sorted(x.join(y).collect())
|
2012-08-10 04:10:02 -04:00
|
|
|
[('a', (1, 2)), ('a', (1, 3))]
|
|
|
|
"""
|
2013-02-24 16:25:09 -05:00
|
|
|
return python_join(self, other, numPartitions)
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2013-02-24 16:25:09 -05:00
|
|
|
def leftOuterJoin(self, other, numPartitions=None):
|
2012-08-10 04:10:02 -04:00
|
|
|
"""
|
2019-07-05 13:08:22 -04:00
|
|
|
Perform a left outer join of `self` and `other`.
|
2012-12-27 20:55:33 -05:00
|
|
|
|
2019-07-05 13:08:22 -04:00
|
|
|
For each element (k, v) in `self`, the resulting RDD will either
|
|
|
|
contain all pairs (k, (v, w)) for w in `other`, or the pair
|
|
|
|
(k, (v, None)) if no elements in `other` have key k.
|
2012-12-27 20:55:33 -05:00
|
|
|
|
|
|
|
Hash-partitions the resulting RDD into the given number of partitions.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2012-08-18 19:07:10 -04:00
|
|
|
>>> x = sc.parallelize([("a", 1), ("b", 4)])
|
|
|
|
>>> y = sc.parallelize([("a", 2)])
|
2012-08-10 04:10:02 -04:00
|
|
|
>>> sorted(x.leftOuterJoin(y).collect())
|
|
|
|
[('a', (1, 2)), ('b', (4, None))]
|
|
|
|
"""
|
2013-02-24 16:25:09 -05:00
|
|
|
return python_left_outer_join(self, other, numPartitions)
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2013-02-24 16:25:09 -05:00
|
|
|
def rightOuterJoin(self, other, numPartitions=None):
|
2012-08-10 04:10:02 -04:00
|
|
|
"""
|
2019-07-05 13:08:22 -04:00
|
|
|
Perform a right outer join of `self` and `other`.
|
2012-12-27 20:55:33 -05:00
|
|
|
|
2019-07-05 13:08:22 -04:00
|
|
|
For each element (k, w) in `other`, the resulting RDD will either
|
2012-12-27 20:55:33 -05:00
|
|
|
contain all pairs (k, (v, w)) for v in this, or the pair (k, (None, w))
|
2019-07-05 13:08:22 -04:00
|
|
|
if no elements in `self` have key k.
|
2012-12-27 20:55:33 -05:00
|
|
|
|
|
|
|
Hash-partitions the resulting RDD into the given number of partitions.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2012-08-18 19:07:10 -04:00
|
|
|
>>> x = sc.parallelize([("a", 1), ("b", 4)])
|
|
|
|
>>> y = sc.parallelize([("a", 2)])
|
2012-08-10 04:10:02 -04:00
|
|
|
>>> sorted(y.rightOuterJoin(x).collect())
|
|
|
|
[('a', (2, 1)), ('b', (None, 4))]
|
|
|
|
"""
|
2013-02-24 16:25:09 -05:00
|
|
|
return python_right_outer_join(self, other, numPartitions)
|
2012-08-18 19:07:10 -04:00
|
|
|
|
2014-09-24 23:39:09 -04:00
|
|
|
def fullOuterJoin(self, other, numPartitions=None):
|
|
|
|
"""
|
2019-07-05 13:08:22 -04:00
|
|
|
Perform a right outer join of `self` and `other`.
|
2014-09-24 23:39:09 -04:00
|
|
|
|
2019-07-05 13:08:22 -04:00
|
|
|
For each element (k, v) in `self`, the resulting RDD will either
|
|
|
|
contain all pairs (k, (v, w)) for w in `other`, or the pair
|
|
|
|
(k, (v, None)) if no elements in `other` have key k.
|
2014-09-24 23:39:09 -04:00
|
|
|
|
2019-07-05 13:08:22 -04:00
|
|
|
Similarly, for each element (k, w) in `other`, the resulting RDD will
|
|
|
|
either contain all pairs (k, (v, w)) for v in `self`, or the pair
|
|
|
|
(k, (None, w)) if no elements in `self` have key k.
|
2014-09-24 23:39:09 -04:00
|
|
|
|
|
|
|
Hash-partitions the resulting RDD into the given number of partitions.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2014-09-24 23:39:09 -04:00
|
|
|
>>> x = sc.parallelize([("a", 1), ("b", 4)])
|
|
|
|
>>> y = sc.parallelize([("a", 2), ("c", 8)])
|
|
|
|
>>> sorted(x.fullOuterJoin(y).collect())
|
|
|
|
[('a', (1, 2)), ('b', (4, None)), ('c', (None, 8))]
|
|
|
|
"""
|
|
|
|
return python_full_outer_join(self, other, numPartitions)
|
|
|
|
|
2012-12-27 20:55:33 -05:00
|
|
|
# TODO: add option to control map-side combining
|
2014-07-21 14:59:54 -04:00
|
|
|
# portable_hash is used as default, because builtin hash of None is different
|
|
|
|
# cross machines.
|
|
|
|
def partitionBy(self, numPartitions, partitionFunc=portable_hash):
|
2012-10-29 01:30:28 -04:00
|
|
|
"""
|
2012-12-27 20:55:33 -05:00
|
|
|
Return a copy of the RDD partitioned using the specified partitioner.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2012-10-29 01:30:28 -04:00
|
|
|
>>> pairs = sc.parallelize([1, 2, 3, 4, 2, 4, 1]).map(lambda x: (x, x))
|
|
|
|
>>> sets = pairs.partitionBy(2).glom().collect()
|
2015-04-16 19:20:57 -04:00
|
|
|
>>> len(set(sets[0]).intersection(set(sets[1])))
|
|
|
|
0
|
2012-10-29 01:30:28 -04:00
|
|
|
"""
|
2013-02-24 16:25:09 -05:00
|
|
|
if numPartitions is None:
|
2014-06-20 03:06:57 -04:00
|
|
|
numPartitions = self._defaultReducePartitions()
|
2015-02-17 19:54:57 -05:00
|
|
|
partitioner = Partitioner(numPartitions, partitionFunc)
|
|
|
|
if self.partitioner == partitioner:
|
|
|
|
return self
|
2014-06-03 16:31:16 -04:00
|
|
|
|
2014-07-25 01:53:47 -04:00
|
|
|
# Transferring O(n) objects to Java is too expensive.
|
|
|
|
# Instead, we'll form the hash buckets in Python,
|
|
|
|
# transferring O(numPartitions) objects to Java.
|
|
|
|
# Each object is a (splitNumber, [objects]) pair.
|
|
|
|
# In order to avoid too huge objects, the objects are
|
|
|
|
# grouped into chunks.
|
2013-11-05 20:52:39 -05:00
|
|
|
outputSerializer = self.ctx._unbatched_serializer
|
2014-07-14 03:42:59 -04:00
|
|
|
|
2019-07-27 14:58:50 -04:00
|
|
|
limit = (self._memory_limit() / 2)
|
2014-07-25 01:53:47 -04:00
|
|
|
|
2013-01-08 19:04:41 -05:00
|
|
|
def add_shuffle_key(split, iterator):
|
2013-10-04 14:56:47 -04:00
|
|
|
|
2012-12-24 18:01:13 -05:00
|
|
|
buckets = defaultdict(list)
|
2014-07-25 01:53:47 -04:00
|
|
|
c, batch = 0, min(10 * numPartitions, 1000)
|
2013-10-04 14:56:47 -04:00
|
|
|
|
2014-08-27 16:18:33 -04:00
|
|
|
for k, v in iterator:
|
2013-02-24 16:25:09 -05:00
|
|
|
buckets[partitionFunc(k) % numPartitions].append((k, v))
|
2014-07-25 01:53:47 -04:00
|
|
|
c += 1
|
|
|
|
|
|
|
|
# check used memory and avg size of chunk of objects
|
|
|
|
if (c % 1000 == 0 and get_used_memory() > limit
|
|
|
|
or c > batch):
|
|
|
|
n, size = len(buckets), 0
|
2015-04-16 19:20:57 -04:00
|
|
|
for split in list(buckets.keys()):
|
2014-07-25 01:53:47 -04:00
|
|
|
yield pack_long(split)
|
|
|
|
d = outputSerializer.dumps(buckets[split])
|
|
|
|
del buckets[split]
|
|
|
|
yield d
|
|
|
|
size += len(d)
|
|
|
|
|
2015-04-16 19:20:57 -04:00
|
|
|
avg = int(size / n) >> 20
|
2014-07-25 01:53:47 -04:00
|
|
|
# let 1M < avg < 10M
|
|
|
|
if avg < 1:
|
|
|
|
batch *= 1.5
|
|
|
|
elif avg > 10:
|
2015-04-16 19:20:57 -04:00
|
|
|
batch = max(int(batch / 1.5), 1)
|
2014-07-25 01:53:47 -04:00
|
|
|
c = 0
|
|
|
|
|
2015-04-16 19:20:57 -04:00
|
|
|
for split, items in buckets.items():
|
2013-10-04 14:56:47 -04:00
|
|
|
yield pack_long(split)
|
2013-11-10 20:48:27 -05:00
|
|
|
yield outputSerializer.dumps(items)
|
2014-07-25 01:53:47 -04:00
|
|
|
|
2015-02-17 19:54:57 -05:00
|
|
|
keyed = self.mapPartitionsWithIndex(add_shuffle_key, preservesPartitioning=True)
|
2012-08-25 19:46:07 -04:00
|
|
|
keyed._bypass_serializer = True
|
2014-09-15 22:28:17 -04:00
|
|
|
with SCCallSiteSync(self.context) as css:
|
2014-07-14 03:42:59 -04:00
|
|
|
pairRDD = self.ctx._jvm.PairwiseRDD(
|
|
|
|
keyed._jrdd.rdd()).asJavaPairRDD()
|
2015-02-17 19:54:57 -05:00
|
|
|
jpartitioner = self.ctx._jvm.PythonPartitioner(numPartitions,
|
|
|
|
id(partitionFunc))
|
|
|
|
jrdd = self.ctx._jvm.PythonRDD.valueOfPair(pairRDD.partitionBy(jpartitioner))
|
2013-11-05 20:52:39 -05:00
|
|
|
rdd = RDD(jrdd, self.ctx, BatchedSerializer(outputSerializer))
|
2015-02-17 19:54:57 -05:00
|
|
|
rdd.partitioner = partitioner
|
2013-01-14 18:30:42 -05:00
|
|
|
return rdd
|
2012-08-18 19:07:10 -04:00
|
|
|
|
2012-12-27 20:55:33 -05:00
|
|
|
# TODO: add control over map-side aggregation
|
2012-08-10 04:10:02 -04:00
|
|
|
def combineByKey(self, createCombiner, mergeValue, mergeCombiners,
|
[SPARK-9821] [PYSPARK] pyspark-reduceByKey-should-take-a-custom-partitioner
from the issue:
In Scala, I can supply a custom partitioner to reduceByKey (and other aggregation/repartitioning methods like aggregateByKey and combinedByKey), but as far as I can tell from the Pyspark API, there's no way to do the same in Python.
Here's an example of my code in Scala:
weblogs.map(s => (getFileType(s), 1)).reduceByKey(new FileTypePartitioner(),_+_)
But I can't figure out how to do the same in Python. The closest I can get is to call repartition before reduceByKey like so:
weblogs.map(lambda s: (getFileType(s), 1)).partitionBy(3,hash_filetype).reduceByKey(lambda v1,v2: v1+v2).collect()
But that defeats the purpose, because I'm shuffling twice instead of once, so my performance is worse instead of better.
Author: Holden Karau <holden@pigscanfly.ca>
Closes #8569 from holdenk/SPARK-9821-pyspark-reduceByKey-should-take-a-custom-partitioner.
2015-09-22 02:21:24 -04:00
|
|
|
numPartitions=None, partitionFunc=portable_hash):
|
2012-08-10 04:10:02 -04:00
|
|
|
"""
|
2012-12-27 20:55:33 -05:00
|
|
|
Generic function to combine the elements for each key using a custom
|
|
|
|
set of aggregation functions.
|
|
|
|
|
|
|
|
Turns an RDD[(K, V)] into a result of type RDD[(K, C)], for a "combined
|
2016-11-22 06:40:18 -05:00
|
|
|
type" C.
|
2012-12-27 20:55:33 -05:00
|
|
|
|
|
|
|
Users provide three functions:
|
|
|
|
|
2019-07-05 13:08:22 -04:00
|
|
|
- `createCombiner`, which turns a V into a C (e.g., creates
|
2012-12-27 20:55:33 -05:00
|
|
|
a one-element list)
|
2019-07-05 13:08:22 -04:00
|
|
|
- `mergeValue`, to merge a V into a C (e.g., adds it to the end of
|
2012-12-27 20:55:33 -05:00
|
|
|
a list)
|
2019-07-05 13:08:22 -04:00
|
|
|
- `mergeCombiners`, to combine two C's into a single one (e.g., merges
|
2017-04-13 15:43:28 -04:00
|
|
|
the lists)
|
|
|
|
|
|
|
|
To avoid memory allocation, both mergeValue and mergeCombiners are allowed to
|
|
|
|
modify and return their first argument instead of creating a new C.
|
2012-12-27 20:55:33 -05:00
|
|
|
|
|
|
|
In addition, users can control the partitioning of the output RDD.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
V and C can be different -- for example, one might group an RDD of type
|
2016-11-22 06:40:18 -05:00
|
|
|
(Int, Int) into an RDD of type (Int, List[Int]).
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2017-04-13 15:43:28 -04:00
|
|
|
>>> x = sc.parallelize([("a", 1), ("b", 1), ("a", 2)])
|
|
|
|
>>> def to_list(a):
|
|
|
|
... return [a]
|
|
|
|
...
|
|
|
|
>>> def append(a, b):
|
|
|
|
... a.append(b)
|
|
|
|
... return a
|
|
|
|
...
|
|
|
|
>>> def extend(a, b):
|
|
|
|
... a.extend(b)
|
|
|
|
... return a
|
|
|
|
...
|
|
|
|
>>> sorted(x.combineByKey(to_list, append, extend).collect())
|
|
|
|
[('a', [1, 2]), ('b', [1])]
|
2012-08-10 04:10:02 -04:00
|
|
|
"""
|
2013-02-24 16:25:09 -05:00
|
|
|
if numPartitions is None:
|
2014-06-20 03:06:57 -04:00
|
|
|
numPartitions = self._defaultReducePartitions()
|
2014-07-14 03:42:59 -04:00
|
|
|
|
2014-07-25 01:53:47 -04:00
|
|
|
serializer = self.ctx.serializer
|
[SPARK-3074] [PySpark] support groupByKey() with single huge key
This patch change groupByKey() to use external sort based approach, so it can support single huge key.
For example, it can group by a dataset including one hot key with 40 millions values (strings), using 500M memory for Python worker, finished in about 2 minutes. (it will need 6G memory in hash based approach).
During groupByKey(), it will do in-memory groupBy first. If the dataset can not fit in memory, then data will be partitioned by hash. If one partition still can not fit in memory, it will switch to sort based groupBy().
Author: Davies Liu <davies.liu@gmail.com>
Author: Davies Liu <davies@databricks.com>
Closes #1977 from davies/groupby and squashes the following commits:
af3713a [Davies Liu] make sure it's iterator
67772dd [Davies Liu] fix tests
e78c15c [Davies Liu] address comments
0b0fde8 [Davies Liu] address comments
0dcf320 [Davies Liu] address comments, rollback changes in ResultIterable
e3b8eab [Davies Liu] fix narrow dependency
2a1857a [Davies Liu] typo
d2f053b [Davies Liu] add repr for FlattedValuesSerializer
c6a2f8d [Davies Liu] address comments
9e2df24 [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
2b9c261 [Davies Liu] fix typo in comments
70aadcd [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
a14b4bd [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
ab5515b [Davies Liu] Merge branch 'master' into groupby
651f891 [Davies Liu] simplify GroupByKey
1578f2e [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
1f69f93 [Davies Liu] fix tests
0d3395f [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
341f1e0 [Davies Liu] add comments, refactor
47918b8 [Davies Liu] remove unused code
6540948 [Davies Liu] address comments:
17f4ec6 [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
4d4bc86 [Davies Liu] bugfix
8ef965e [Davies Liu] Merge branch 'master' into groupby
fbc504a [Davies Liu] Merge branch 'master' into groupby
779ed03 [Davies Liu] fix merge conflict
2c1d05b [Davies Liu] refactor, minor turning
b48cda5 [Davies Liu] Merge branch 'master' into groupby
85138e6 [Davies Liu] Merge branch 'master' into groupby
acd8e1b [Davies Liu] fix memory when groupByKey().count()
905b233 [Davies Liu] Merge branch 'sort' into groupby
1f075ed [Davies Liu] Merge branch 'master' into sort
4b07d39 [Davies Liu] compress the data while spilling
0a081c6 [Davies Liu] Merge branch 'master' into groupby
f157fe7 [Davies Liu] Merge branch 'sort' into groupby
eb53ca6 [Davies Liu] Merge branch 'master' into sort
b2dc3bf [Davies Liu] Merge branch 'sort' into groupby
644abaf [Davies Liu] add license in LICENSE
19f7873 [Davies Liu] improve tests
11ba318 [Davies Liu] typo
085aef8 [Davies Liu] Merge branch 'master' into groupby
3ee58e5 [Davies Liu] switch to sort based groupBy, based on size of data
1ea0669 [Davies Liu] choose sort based groupByKey() automatically
b40bae7 [Davies Liu] bugfix
efa23df [Davies Liu] refactor, add spark.shuffle.sort=False
250be4e [Davies Liu] flatten the combined values when dumping into disks
d05060d [Davies Liu] group the same key before shuffle, reduce the comparison during sorting
083d842 [Davies Liu] sorted based groupByKey()
55602ee [Davies Liu] use external sort in sortBy() and sortByKey()
2015-04-09 20:07:23 -04:00
|
|
|
memory = self._memory_limit()
|
2014-07-25 01:53:47 -04:00
|
|
|
agg = Aggregator(createCombiner, mergeValue, mergeCombiners)
|
|
|
|
|
2012-08-25 17:19:07 -04:00
|
|
|
def combineLocally(iterator):
|
2015-09-20 00:40:21 -04:00
|
|
|
merger = ExternalMerger(agg, memory * 0.9, serializer)
|
2014-07-25 01:53:47 -04:00
|
|
|
merger.mergeValues(iterator)
|
2015-04-16 19:20:57 -04:00
|
|
|
return merger.items()
|
2014-07-25 01:53:47 -04:00
|
|
|
|
2015-02-17 19:54:57 -05:00
|
|
|
locally_combined = self.mapPartitions(combineLocally, preservesPartitioning=True)
|
[SPARK-9821] [PYSPARK] pyspark-reduceByKey-should-take-a-custom-partitioner
from the issue:
In Scala, I can supply a custom partitioner to reduceByKey (and other aggregation/repartitioning methods like aggregateByKey and combinedByKey), but as far as I can tell from the Pyspark API, there's no way to do the same in Python.
Here's an example of my code in Scala:
weblogs.map(s => (getFileType(s), 1)).reduceByKey(new FileTypePartitioner(),_+_)
But I can't figure out how to do the same in Python. The closest I can get is to call repartition before reduceByKey like so:
weblogs.map(lambda s: (getFileType(s), 1)).partitionBy(3,hash_filetype).reduceByKey(lambda v1,v2: v1+v2).collect()
But that defeats the purpose, because I'm shuffling twice instead of once, so my performance is worse instead of better.
Author: Holden Karau <holden@pigscanfly.ca>
Closes #8569 from holdenk/SPARK-9821-pyspark-reduceByKey-should-take-a-custom-partitioner.
2015-09-22 02:21:24 -04:00
|
|
|
shuffled = locally_combined.partitionBy(numPartitions, partitionFunc)
|
2014-07-14 03:42:59 -04:00
|
|
|
|
2012-08-25 17:19:07 -04:00
|
|
|
def _mergeCombiners(iterator):
|
2015-09-20 00:40:21 -04:00
|
|
|
merger = ExternalMerger(agg, memory, serializer)
|
2014-07-25 01:53:47 -04:00
|
|
|
merger.mergeCombiners(iterator)
|
2015-04-16 19:20:57 -04:00
|
|
|
return merger.items()
|
2014-07-25 01:53:47 -04:00
|
|
|
|
2015-02-17 19:54:57 -05:00
|
|
|
return shuffled.mapPartitions(_mergeCombiners, preservesPartitioning=True)
|
2014-06-20 14:03:03 -04:00
|
|
|
|
[SPARK-9821] [PYSPARK] pyspark-reduceByKey-should-take-a-custom-partitioner
from the issue:
In Scala, I can supply a custom partitioner to reduceByKey (and other aggregation/repartitioning methods like aggregateByKey and combinedByKey), but as far as I can tell from the Pyspark API, there's no way to do the same in Python.
Here's an example of my code in Scala:
weblogs.map(s => (getFileType(s), 1)).reduceByKey(new FileTypePartitioner(),_+_)
But I can't figure out how to do the same in Python. The closest I can get is to call repartition before reduceByKey like so:
weblogs.map(lambda s: (getFileType(s), 1)).partitionBy(3,hash_filetype).reduceByKey(lambda v1,v2: v1+v2).collect()
But that defeats the purpose, because I'm shuffling twice instead of once, so my performance is worse instead of better.
Author: Holden Karau <holden@pigscanfly.ca>
Closes #8569 from holdenk/SPARK-9821-pyspark-reduceByKey-should-take-a-custom-partitioner.
2015-09-22 02:21:24 -04:00
|
|
|
def aggregateByKey(self, zeroValue, seqFunc, combFunc, numPartitions=None,
|
|
|
|
partitionFunc=portable_hash):
|
2014-06-12 11:14:25 -04:00
|
|
|
"""
|
2014-07-14 03:42:59 -04:00
|
|
|
Aggregate the values of each key, using given combine functions and a neutral
|
|
|
|
"zero value". This function can return a different result type, U, than the type
|
|
|
|
of the values in this RDD, V. Thus, we need one operation for merging a V into
|
|
|
|
a U and one operation for merging two U's, The former operation is used for merging
|
|
|
|
values within a partition, and the latter is used for merging values between
|
|
|
|
partitions. To avoid memory allocation, both of these functions are
|
2014-06-12 11:14:25 -04:00
|
|
|
allowed to modify and return their first argument instead of creating a new U.
|
|
|
|
"""
|
|
|
|
def createZero():
|
2014-07-14 03:42:59 -04:00
|
|
|
return copy.deepcopy(zeroValue)
|
2014-06-20 14:03:03 -04:00
|
|
|
|
2014-07-14 03:42:59 -04:00
|
|
|
return self.combineByKey(
|
[SPARK-9821] [PYSPARK] pyspark-reduceByKey-should-take-a-custom-partitioner
from the issue:
In Scala, I can supply a custom partitioner to reduceByKey (and other aggregation/repartitioning methods like aggregateByKey and combinedByKey), but as far as I can tell from the Pyspark API, there's no way to do the same in Python.
Here's an example of my code in Scala:
weblogs.map(s => (getFileType(s), 1)).reduceByKey(new FileTypePartitioner(),_+_)
But I can't figure out how to do the same in Python. The closest I can get is to call repartition before reduceByKey like so:
weblogs.map(lambda s: (getFileType(s), 1)).partitionBy(3,hash_filetype).reduceByKey(lambda v1,v2: v1+v2).collect()
But that defeats the purpose, because I'm shuffling twice instead of once, so my performance is worse instead of better.
Author: Holden Karau <holden@pigscanfly.ca>
Closes #8569 from holdenk/SPARK-9821-pyspark-reduceByKey-should-take-a-custom-partitioner.
2015-09-22 02:21:24 -04:00
|
|
|
lambda v: seqFunc(createZero(), v), seqFunc, combFunc, numPartitions, partitionFunc)
|
2014-06-09 03:08:40 -04:00
|
|
|
|
[SPARK-9821] [PYSPARK] pyspark-reduceByKey-should-take-a-custom-partitioner
from the issue:
In Scala, I can supply a custom partitioner to reduceByKey (and other aggregation/repartitioning methods like aggregateByKey and combinedByKey), but as far as I can tell from the Pyspark API, there's no way to do the same in Python.
Here's an example of my code in Scala:
weblogs.map(s => (getFileType(s), 1)).reduceByKey(new FileTypePartitioner(),_+_)
But I can't figure out how to do the same in Python. The closest I can get is to call repartition before reduceByKey like so:
weblogs.map(lambda s: (getFileType(s), 1)).partitionBy(3,hash_filetype).reduceByKey(lambda v1,v2: v1+v2).collect()
But that defeats the purpose, because I'm shuffling twice instead of once, so my performance is worse instead of better.
Author: Holden Karau <holden@pigscanfly.ca>
Closes #8569 from holdenk/SPARK-9821-pyspark-reduceByKey-should-take-a-custom-partitioner.
2015-09-22 02:21:24 -04:00
|
|
|
def foldByKey(self, zeroValue, func, numPartitions=None, partitionFunc=portable_hash):
|
2014-03-10 16:37:11 -04:00
|
|
|
"""
|
2014-06-12 22:44:27 -04:00
|
|
|
Merge the values for each key using an associative function "func"
|
|
|
|
and a neutral "zeroValue" which may be added to the result an
|
|
|
|
arbitrary number of times, and must not change the result
|
|
|
|
(e.g., 0 for addition, or 1 for multiplication.).
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2014-03-10 16:37:11 -04:00
|
|
|
>>> rdd = sc.parallelize([("a", 1), ("b", 1), ("a", 1)])
|
|
|
|
>>> from operator import add
|
2015-04-16 19:20:57 -04:00
|
|
|
>>> sorted(rdd.foldByKey(0, add).collect())
|
2014-03-10 16:37:11 -04:00
|
|
|
[('a', 2), ('b', 1)]
|
|
|
|
"""
|
2014-06-12 11:14:25 -04:00
|
|
|
def createZero():
|
2014-07-14 03:42:59 -04:00
|
|
|
return copy.deepcopy(zeroValue)
|
2014-06-12 11:14:25 -04:00
|
|
|
|
[SPARK-9821] [PYSPARK] pyspark-reduceByKey-should-take-a-custom-partitioner
from the issue:
In Scala, I can supply a custom partitioner to reduceByKey (and other aggregation/repartitioning methods like aggregateByKey and combinedByKey), but as far as I can tell from the Pyspark API, there's no way to do the same in Python.
Here's an example of my code in Scala:
weblogs.map(s => (getFileType(s), 1)).reduceByKey(new FileTypePartitioner(),_+_)
But I can't figure out how to do the same in Python. The closest I can get is to call repartition before reduceByKey like so:
weblogs.map(lambda s: (getFileType(s), 1)).partitionBy(3,hash_filetype).reduceByKey(lambda v1,v2: v1+v2).collect()
But that defeats the purpose, because I'm shuffling twice instead of once, so my performance is worse instead of better.
Author: Holden Karau <holden@pigscanfly.ca>
Closes #8569 from holdenk/SPARK-9821-pyspark-reduceByKey-should-take-a-custom-partitioner.
2015-09-22 02:21:24 -04:00
|
|
|
return self.combineByKey(lambda v: func(createZero(), v), func, func, numPartitions,
|
|
|
|
partitionFunc)
|
2014-06-09 03:08:40 -04:00
|
|
|
|
[SPARK-3074] [PySpark] support groupByKey() with single huge key
This patch change groupByKey() to use external sort based approach, so it can support single huge key.
For example, it can group by a dataset including one hot key with 40 millions values (strings), using 500M memory for Python worker, finished in about 2 minutes. (it will need 6G memory in hash based approach).
During groupByKey(), it will do in-memory groupBy first. If the dataset can not fit in memory, then data will be partitioned by hash. If one partition still can not fit in memory, it will switch to sort based groupBy().
Author: Davies Liu <davies.liu@gmail.com>
Author: Davies Liu <davies@databricks.com>
Closes #1977 from davies/groupby and squashes the following commits:
af3713a [Davies Liu] make sure it's iterator
67772dd [Davies Liu] fix tests
e78c15c [Davies Liu] address comments
0b0fde8 [Davies Liu] address comments
0dcf320 [Davies Liu] address comments, rollback changes in ResultIterable
e3b8eab [Davies Liu] fix narrow dependency
2a1857a [Davies Liu] typo
d2f053b [Davies Liu] add repr for FlattedValuesSerializer
c6a2f8d [Davies Liu] address comments
9e2df24 [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
2b9c261 [Davies Liu] fix typo in comments
70aadcd [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
a14b4bd [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
ab5515b [Davies Liu] Merge branch 'master' into groupby
651f891 [Davies Liu] simplify GroupByKey
1578f2e [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
1f69f93 [Davies Liu] fix tests
0d3395f [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
341f1e0 [Davies Liu] add comments, refactor
47918b8 [Davies Liu] remove unused code
6540948 [Davies Liu] address comments:
17f4ec6 [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
4d4bc86 [Davies Liu] bugfix
8ef965e [Davies Liu] Merge branch 'master' into groupby
fbc504a [Davies Liu] Merge branch 'master' into groupby
779ed03 [Davies Liu] fix merge conflict
2c1d05b [Davies Liu] refactor, minor turning
b48cda5 [Davies Liu] Merge branch 'master' into groupby
85138e6 [Davies Liu] Merge branch 'master' into groupby
acd8e1b [Davies Liu] fix memory when groupByKey().count()
905b233 [Davies Liu] Merge branch 'sort' into groupby
1f075ed [Davies Liu] Merge branch 'master' into sort
4b07d39 [Davies Liu] compress the data while spilling
0a081c6 [Davies Liu] Merge branch 'master' into groupby
f157fe7 [Davies Liu] Merge branch 'sort' into groupby
eb53ca6 [Davies Liu] Merge branch 'master' into sort
b2dc3bf [Davies Liu] Merge branch 'sort' into groupby
644abaf [Davies Liu] add license in LICENSE
19f7873 [Davies Liu] improve tests
11ba318 [Davies Liu] typo
085aef8 [Davies Liu] Merge branch 'master' into groupby
3ee58e5 [Davies Liu] switch to sort based groupBy, based on size of data
1ea0669 [Davies Liu] choose sort based groupByKey() automatically
b40bae7 [Davies Liu] bugfix
efa23df [Davies Liu] refactor, add spark.shuffle.sort=False
250be4e [Davies Liu] flatten the combined values when dumping into disks
d05060d [Davies Liu] group the same key before shuffle, reduce the comparison during sorting
083d842 [Davies Liu] sorted based groupByKey()
55602ee [Davies Liu] use external sort in sortBy() and sortByKey()
2015-04-09 20:07:23 -04:00
|
|
|
def _memory_limit(self):
|
|
|
|
return _parse_memory(self.ctx._conf.get("spark.python.worker.memory", "512m"))
|
|
|
|
|
2012-12-27 20:55:33 -05:00
|
|
|
# TODO: support variant with custom partitioner
|
[SPARK-9821] [PYSPARK] pyspark-reduceByKey-should-take-a-custom-partitioner
from the issue:
In Scala, I can supply a custom partitioner to reduceByKey (and other aggregation/repartitioning methods like aggregateByKey and combinedByKey), but as far as I can tell from the Pyspark API, there's no way to do the same in Python.
Here's an example of my code in Scala:
weblogs.map(s => (getFileType(s), 1)).reduceByKey(new FileTypePartitioner(),_+_)
But I can't figure out how to do the same in Python. The closest I can get is to call repartition before reduceByKey like so:
weblogs.map(lambda s: (getFileType(s), 1)).partitionBy(3,hash_filetype).reduceByKey(lambda v1,v2: v1+v2).collect()
But that defeats the purpose, because I'm shuffling twice instead of once, so my performance is worse instead of better.
Author: Holden Karau <holden@pigscanfly.ca>
Closes #8569 from holdenk/SPARK-9821-pyspark-reduceByKey-should-take-a-custom-partitioner.
2015-09-22 02:21:24 -04:00
|
|
|
def groupByKey(self, numPartitions=None, partitionFunc=portable_hash):
|
2012-08-10 04:10:02 -04:00
|
|
|
"""
|
2012-12-27 20:55:33 -05:00
|
|
|
Group the values for each key in the RDD into a single sequence.
|
[SPARK-3074] [PySpark] support groupByKey() with single huge key
This patch change groupByKey() to use external sort based approach, so it can support single huge key.
For example, it can group by a dataset including one hot key with 40 millions values (strings), using 500M memory for Python worker, finished in about 2 minutes. (it will need 6G memory in hash based approach).
During groupByKey(), it will do in-memory groupBy first. If the dataset can not fit in memory, then data will be partitioned by hash. If one partition still can not fit in memory, it will switch to sort based groupBy().
Author: Davies Liu <davies.liu@gmail.com>
Author: Davies Liu <davies@databricks.com>
Closes #1977 from davies/groupby and squashes the following commits:
af3713a [Davies Liu] make sure it's iterator
67772dd [Davies Liu] fix tests
e78c15c [Davies Liu] address comments
0b0fde8 [Davies Liu] address comments
0dcf320 [Davies Liu] address comments, rollback changes in ResultIterable
e3b8eab [Davies Liu] fix narrow dependency
2a1857a [Davies Liu] typo
d2f053b [Davies Liu] add repr for FlattedValuesSerializer
c6a2f8d [Davies Liu] address comments
9e2df24 [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
2b9c261 [Davies Liu] fix typo in comments
70aadcd [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
a14b4bd [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
ab5515b [Davies Liu] Merge branch 'master' into groupby
651f891 [Davies Liu] simplify GroupByKey
1578f2e [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
1f69f93 [Davies Liu] fix tests
0d3395f [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
341f1e0 [Davies Liu] add comments, refactor
47918b8 [Davies Liu] remove unused code
6540948 [Davies Liu] address comments:
17f4ec6 [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
4d4bc86 [Davies Liu] bugfix
8ef965e [Davies Liu] Merge branch 'master' into groupby
fbc504a [Davies Liu] Merge branch 'master' into groupby
779ed03 [Davies Liu] fix merge conflict
2c1d05b [Davies Liu] refactor, minor turning
b48cda5 [Davies Liu] Merge branch 'master' into groupby
85138e6 [Davies Liu] Merge branch 'master' into groupby
acd8e1b [Davies Liu] fix memory when groupByKey().count()
905b233 [Davies Liu] Merge branch 'sort' into groupby
1f075ed [Davies Liu] Merge branch 'master' into sort
4b07d39 [Davies Liu] compress the data while spilling
0a081c6 [Davies Liu] Merge branch 'master' into groupby
f157fe7 [Davies Liu] Merge branch 'sort' into groupby
eb53ca6 [Davies Liu] Merge branch 'master' into sort
b2dc3bf [Davies Liu] Merge branch 'sort' into groupby
644abaf [Davies Liu] add license in LICENSE
19f7873 [Davies Liu] improve tests
11ba318 [Davies Liu] typo
085aef8 [Davies Liu] Merge branch 'master' into groupby
3ee58e5 [Davies Liu] switch to sort based groupBy, based on size of data
1ea0669 [Davies Liu] choose sort based groupByKey() automatically
b40bae7 [Davies Liu] bugfix
efa23df [Davies Liu] refactor, add spark.shuffle.sort=False
250be4e [Davies Liu] flatten the combined values when dumping into disks
d05060d [Davies Liu] group the same key before shuffle, reduce the comparison during sorting
083d842 [Davies Liu] sorted based groupByKey()
55602ee [Davies Liu] use external sort in sortBy() and sortByKey()
2015-04-09 20:07:23 -04:00
|
|
|
Hash-partitions the resulting RDD with numPartitions partitions.
|
2012-12-27 20:55:33 -05:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
If you are grouping in order to perform an aggregation (such as a
|
|
|
|
sum or average) over each key, using reduceByKey or aggregateByKey will
|
|
|
|
provide much better performance.
|
2014-05-15 01:24:04 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2015-04-16 19:20:57 -04:00
|
|
|
>>> rdd = sc.parallelize([("a", 1), ("b", 1), ("a", 1)])
|
|
|
|
>>> sorted(rdd.groupByKey().mapValues(len).collect())
|
[SPARK-3074] [PySpark] support groupByKey() with single huge key
This patch change groupByKey() to use external sort based approach, so it can support single huge key.
For example, it can group by a dataset including one hot key with 40 millions values (strings), using 500M memory for Python worker, finished in about 2 minutes. (it will need 6G memory in hash based approach).
During groupByKey(), it will do in-memory groupBy first. If the dataset can not fit in memory, then data will be partitioned by hash. If one partition still can not fit in memory, it will switch to sort based groupBy().
Author: Davies Liu <davies.liu@gmail.com>
Author: Davies Liu <davies@databricks.com>
Closes #1977 from davies/groupby and squashes the following commits:
af3713a [Davies Liu] make sure it's iterator
67772dd [Davies Liu] fix tests
e78c15c [Davies Liu] address comments
0b0fde8 [Davies Liu] address comments
0dcf320 [Davies Liu] address comments, rollback changes in ResultIterable
e3b8eab [Davies Liu] fix narrow dependency
2a1857a [Davies Liu] typo
d2f053b [Davies Liu] add repr for FlattedValuesSerializer
c6a2f8d [Davies Liu] address comments
9e2df24 [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
2b9c261 [Davies Liu] fix typo in comments
70aadcd [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
a14b4bd [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
ab5515b [Davies Liu] Merge branch 'master' into groupby
651f891 [Davies Liu] simplify GroupByKey
1578f2e [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
1f69f93 [Davies Liu] fix tests
0d3395f [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
341f1e0 [Davies Liu] add comments, refactor
47918b8 [Davies Liu] remove unused code
6540948 [Davies Liu] address comments:
17f4ec6 [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
4d4bc86 [Davies Liu] bugfix
8ef965e [Davies Liu] Merge branch 'master' into groupby
fbc504a [Davies Liu] Merge branch 'master' into groupby
779ed03 [Davies Liu] fix merge conflict
2c1d05b [Davies Liu] refactor, minor turning
b48cda5 [Davies Liu] Merge branch 'master' into groupby
85138e6 [Davies Liu] Merge branch 'master' into groupby
acd8e1b [Davies Liu] fix memory when groupByKey().count()
905b233 [Davies Liu] Merge branch 'sort' into groupby
1f075ed [Davies Liu] Merge branch 'master' into sort
4b07d39 [Davies Liu] compress the data while spilling
0a081c6 [Davies Liu] Merge branch 'master' into groupby
f157fe7 [Davies Liu] Merge branch 'sort' into groupby
eb53ca6 [Davies Liu] Merge branch 'master' into sort
b2dc3bf [Davies Liu] Merge branch 'sort' into groupby
644abaf [Davies Liu] add license in LICENSE
19f7873 [Davies Liu] improve tests
11ba318 [Davies Liu] typo
085aef8 [Davies Liu] Merge branch 'master' into groupby
3ee58e5 [Davies Liu] switch to sort based groupBy, based on size of data
1ea0669 [Davies Liu] choose sort based groupByKey() automatically
b40bae7 [Davies Liu] bugfix
efa23df [Davies Liu] refactor, add spark.shuffle.sort=False
250be4e [Davies Liu] flatten the combined values when dumping into disks
d05060d [Davies Liu] group the same key before shuffle, reduce the comparison during sorting
083d842 [Davies Liu] sorted based groupByKey()
55602ee [Davies Liu] use external sort in sortBy() and sortByKey()
2015-04-09 20:07:23 -04:00
|
|
|
[('a', 2), ('b', 1)]
|
2015-04-16 19:20:57 -04:00
|
|
|
>>> sorted(rdd.groupByKey().mapValues(list).collect())
|
2012-08-10 04:10:02 -04:00
|
|
|
[('a', [1, 1]), ('b', [1])]
|
|
|
|
"""
|
|
|
|
def createCombiner(x):
|
|
|
|
return [x]
|
|
|
|
|
|
|
|
def mergeValue(xs, x):
|
|
|
|
xs.append(x)
|
|
|
|
return xs
|
|
|
|
|
|
|
|
def mergeCombiners(a, b):
|
2014-07-25 01:53:47 -04:00
|
|
|
a.extend(b)
|
|
|
|
return a
|
2012-08-10 04:10:02 -04:00
|
|
|
|
[SPARK-3074] [PySpark] support groupByKey() with single huge key
This patch change groupByKey() to use external sort based approach, so it can support single huge key.
For example, it can group by a dataset including one hot key with 40 millions values (strings), using 500M memory for Python worker, finished in about 2 minutes. (it will need 6G memory in hash based approach).
During groupByKey(), it will do in-memory groupBy first. If the dataset can not fit in memory, then data will be partitioned by hash. If one partition still can not fit in memory, it will switch to sort based groupBy().
Author: Davies Liu <davies.liu@gmail.com>
Author: Davies Liu <davies@databricks.com>
Closes #1977 from davies/groupby and squashes the following commits:
af3713a [Davies Liu] make sure it's iterator
67772dd [Davies Liu] fix tests
e78c15c [Davies Liu] address comments
0b0fde8 [Davies Liu] address comments
0dcf320 [Davies Liu] address comments, rollback changes in ResultIterable
e3b8eab [Davies Liu] fix narrow dependency
2a1857a [Davies Liu] typo
d2f053b [Davies Liu] add repr for FlattedValuesSerializer
c6a2f8d [Davies Liu] address comments
9e2df24 [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
2b9c261 [Davies Liu] fix typo in comments
70aadcd [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
a14b4bd [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
ab5515b [Davies Liu] Merge branch 'master' into groupby
651f891 [Davies Liu] simplify GroupByKey
1578f2e [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
1f69f93 [Davies Liu] fix tests
0d3395f [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
341f1e0 [Davies Liu] add comments, refactor
47918b8 [Davies Liu] remove unused code
6540948 [Davies Liu] address comments:
17f4ec6 [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
4d4bc86 [Davies Liu] bugfix
8ef965e [Davies Liu] Merge branch 'master' into groupby
fbc504a [Davies Liu] Merge branch 'master' into groupby
779ed03 [Davies Liu] fix merge conflict
2c1d05b [Davies Liu] refactor, minor turning
b48cda5 [Davies Liu] Merge branch 'master' into groupby
85138e6 [Davies Liu] Merge branch 'master' into groupby
acd8e1b [Davies Liu] fix memory when groupByKey().count()
905b233 [Davies Liu] Merge branch 'sort' into groupby
1f075ed [Davies Liu] Merge branch 'master' into sort
4b07d39 [Davies Liu] compress the data while spilling
0a081c6 [Davies Liu] Merge branch 'master' into groupby
f157fe7 [Davies Liu] Merge branch 'sort' into groupby
eb53ca6 [Davies Liu] Merge branch 'master' into sort
b2dc3bf [Davies Liu] Merge branch 'sort' into groupby
644abaf [Davies Liu] add license in LICENSE
19f7873 [Davies Liu] improve tests
11ba318 [Davies Liu] typo
085aef8 [Davies Liu] Merge branch 'master' into groupby
3ee58e5 [Davies Liu] switch to sort based groupBy, based on size of data
1ea0669 [Davies Liu] choose sort based groupByKey() automatically
b40bae7 [Davies Liu] bugfix
efa23df [Davies Liu] refactor, add spark.shuffle.sort=False
250be4e [Davies Liu] flatten the combined values when dumping into disks
d05060d [Davies Liu] group the same key before shuffle, reduce the comparison during sorting
083d842 [Davies Liu] sorted based groupByKey()
55602ee [Davies Liu] use external sort in sortBy() and sortByKey()
2015-04-09 20:07:23 -04:00
|
|
|
memory = self._memory_limit()
|
|
|
|
serializer = self._jrdd_deserializer
|
|
|
|
agg = Aggregator(createCombiner, mergeValue, mergeCombiners)
|
|
|
|
|
|
|
|
def combine(iterator):
|
2015-09-20 00:40:21 -04:00
|
|
|
merger = ExternalMerger(agg, memory * 0.9, serializer)
|
[SPARK-3074] [PySpark] support groupByKey() with single huge key
This patch change groupByKey() to use external sort based approach, so it can support single huge key.
For example, it can group by a dataset including one hot key with 40 millions values (strings), using 500M memory for Python worker, finished in about 2 minutes. (it will need 6G memory in hash based approach).
During groupByKey(), it will do in-memory groupBy first. If the dataset can not fit in memory, then data will be partitioned by hash. If one partition still can not fit in memory, it will switch to sort based groupBy().
Author: Davies Liu <davies.liu@gmail.com>
Author: Davies Liu <davies@databricks.com>
Closes #1977 from davies/groupby and squashes the following commits:
af3713a [Davies Liu] make sure it's iterator
67772dd [Davies Liu] fix tests
e78c15c [Davies Liu] address comments
0b0fde8 [Davies Liu] address comments
0dcf320 [Davies Liu] address comments, rollback changes in ResultIterable
e3b8eab [Davies Liu] fix narrow dependency
2a1857a [Davies Liu] typo
d2f053b [Davies Liu] add repr for FlattedValuesSerializer
c6a2f8d [Davies Liu] address comments
9e2df24 [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
2b9c261 [Davies Liu] fix typo in comments
70aadcd [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
a14b4bd [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
ab5515b [Davies Liu] Merge branch 'master' into groupby
651f891 [Davies Liu] simplify GroupByKey
1578f2e [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
1f69f93 [Davies Liu] fix tests
0d3395f [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
341f1e0 [Davies Liu] add comments, refactor
47918b8 [Davies Liu] remove unused code
6540948 [Davies Liu] address comments:
17f4ec6 [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
4d4bc86 [Davies Liu] bugfix
8ef965e [Davies Liu] Merge branch 'master' into groupby
fbc504a [Davies Liu] Merge branch 'master' into groupby
779ed03 [Davies Liu] fix merge conflict
2c1d05b [Davies Liu] refactor, minor turning
b48cda5 [Davies Liu] Merge branch 'master' into groupby
85138e6 [Davies Liu] Merge branch 'master' into groupby
acd8e1b [Davies Liu] fix memory when groupByKey().count()
905b233 [Davies Liu] Merge branch 'sort' into groupby
1f075ed [Davies Liu] Merge branch 'master' into sort
4b07d39 [Davies Liu] compress the data while spilling
0a081c6 [Davies Liu] Merge branch 'master' into groupby
f157fe7 [Davies Liu] Merge branch 'sort' into groupby
eb53ca6 [Davies Liu] Merge branch 'master' into sort
b2dc3bf [Davies Liu] Merge branch 'sort' into groupby
644abaf [Davies Liu] add license in LICENSE
19f7873 [Davies Liu] improve tests
11ba318 [Davies Liu] typo
085aef8 [Davies Liu] Merge branch 'master' into groupby
3ee58e5 [Davies Liu] switch to sort based groupBy, based on size of data
1ea0669 [Davies Liu] choose sort based groupByKey() automatically
b40bae7 [Davies Liu] bugfix
efa23df [Davies Liu] refactor, add spark.shuffle.sort=False
250be4e [Davies Liu] flatten the combined values when dumping into disks
d05060d [Davies Liu] group the same key before shuffle, reduce the comparison during sorting
083d842 [Davies Liu] sorted based groupByKey()
55602ee [Davies Liu] use external sort in sortBy() and sortByKey()
2015-04-09 20:07:23 -04:00
|
|
|
merger.mergeValues(iterator)
|
2015-04-16 19:20:57 -04:00
|
|
|
return merger.items()
|
[SPARK-3074] [PySpark] support groupByKey() with single huge key
This patch change groupByKey() to use external sort based approach, so it can support single huge key.
For example, it can group by a dataset including one hot key with 40 millions values (strings), using 500M memory for Python worker, finished in about 2 minutes. (it will need 6G memory in hash based approach).
During groupByKey(), it will do in-memory groupBy first. If the dataset can not fit in memory, then data will be partitioned by hash. If one partition still can not fit in memory, it will switch to sort based groupBy().
Author: Davies Liu <davies.liu@gmail.com>
Author: Davies Liu <davies@databricks.com>
Closes #1977 from davies/groupby and squashes the following commits:
af3713a [Davies Liu] make sure it's iterator
67772dd [Davies Liu] fix tests
e78c15c [Davies Liu] address comments
0b0fde8 [Davies Liu] address comments
0dcf320 [Davies Liu] address comments, rollback changes in ResultIterable
e3b8eab [Davies Liu] fix narrow dependency
2a1857a [Davies Liu] typo
d2f053b [Davies Liu] add repr for FlattedValuesSerializer
c6a2f8d [Davies Liu] address comments
9e2df24 [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
2b9c261 [Davies Liu] fix typo in comments
70aadcd [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
a14b4bd [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
ab5515b [Davies Liu] Merge branch 'master' into groupby
651f891 [Davies Liu] simplify GroupByKey
1578f2e [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
1f69f93 [Davies Liu] fix tests
0d3395f [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
341f1e0 [Davies Liu] add comments, refactor
47918b8 [Davies Liu] remove unused code
6540948 [Davies Liu] address comments:
17f4ec6 [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
4d4bc86 [Davies Liu] bugfix
8ef965e [Davies Liu] Merge branch 'master' into groupby
fbc504a [Davies Liu] Merge branch 'master' into groupby
779ed03 [Davies Liu] fix merge conflict
2c1d05b [Davies Liu] refactor, minor turning
b48cda5 [Davies Liu] Merge branch 'master' into groupby
85138e6 [Davies Liu] Merge branch 'master' into groupby
acd8e1b [Davies Liu] fix memory when groupByKey().count()
905b233 [Davies Liu] Merge branch 'sort' into groupby
1f075ed [Davies Liu] Merge branch 'master' into sort
4b07d39 [Davies Liu] compress the data while spilling
0a081c6 [Davies Liu] Merge branch 'master' into groupby
f157fe7 [Davies Liu] Merge branch 'sort' into groupby
eb53ca6 [Davies Liu] Merge branch 'master' into sort
b2dc3bf [Davies Liu] Merge branch 'sort' into groupby
644abaf [Davies Liu] add license in LICENSE
19f7873 [Davies Liu] improve tests
11ba318 [Davies Liu] typo
085aef8 [Davies Liu] Merge branch 'master' into groupby
3ee58e5 [Davies Liu] switch to sort based groupBy, based on size of data
1ea0669 [Davies Liu] choose sort based groupByKey() automatically
b40bae7 [Davies Liu] bugfix
efa23df [Davies Liu] refactor, add spark.shuffle.sort=False
250be4e [Davies Liu] flatten the combined values when dumping into disks
d05060d [Davies Liu] group the same key before shuffle, reduce the comparison during sorting
083d842 [Davies Liu] sorted based groupByKey()
55602ee [Davies Liu] use external sort in sortBy() and sortByKey()
2015-04-09 20:07:23 -04:00
|
|
|
|
|
|
|
locally_combined = self.mapPartitions(combine, preservesPartitioning=True)
|
[SPARK-9821] [PYSPARK] pyspark-reduceByKey-should-take-a-custom-partitioner
from the issue:
In Scala, I can supply a custom partitioner to reduceByKey (and other aggregation/repartitioning methods like aggregateByKey and combinedByKey), but as far as I can tell from the Pyspark API, there's no way to do the same in Python.
Here's an example of my code in Scala:
weblogs.map(s => (getFileType(s), 1)).reduceByKey(new FileTypePartitioner(),_+_)
But I can't figure out how to do the same in Python. The closest I can get is to call repartition before reduceByKey like so:
weblogs.map(lambda s: (getFileType(s), 1)).partitionBy(3,hash_filetype).reduceByKey(lambda v1,v2: v1+v2).collect()
But that defeats the purpose, because I'm shuffling twice instead of once, so my performance is worse instead of better.
Author: Holden Karau <holden@pigscanfly.ca>
Closes #8569 from holdenk/SPARK-9821-pyspark-reduceByKey-should-take-a-custom-partitioner.
2015-09-22 02:21:24 -04:00
|
|
|
shuffled = locally_combined.partitionBy(numPartitions, partitionFunc)
|
[SPARK-3074] [PySpark] support groupByKey() with single huge key
This patch change groupByKey() to use external sort based approach, so it can support single huge key.
For example, it can group by a dataset including one hot key with 40 millions values (strings), using 500M memory for Python worker, finished in about 2 minutes. (it will need 6G memory in hash based approach).
During groupByKey(), it will do in-memory groupBy first. If the dataset can not fit in memory, then data will be partitioned by hash. If one partition still can not fit in memory, it will switch to sort based groupBy().
Author: Davies Liu <davies.liu@gmail.com>
Author: Davies Liu <davies@databricks.com>
Closes #1977 from davies/groupby and squashes the following commits:
af3713a [Davies Liu] make sure it's iterator
67772dd [Davies Liu] fix tests
e78c15c [Davies Liu] address comments
0b0fde8 [Davies Liu] address comments
0dcf320 [Davies Liu] address comments, rollback changes in ResultIterable
e3b8eab [Davies Liu] fix narrow dependency
2a1857a [Davies Liu] typo
d2f053b [Davies Liu] add repr for FlattedValuesSerializer
c6a2f8d [Davies Liu] address comments
9e2df24 [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
2b9c261 [Davies Liu] fix typo in comments
70aadcd [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
a14b4bd [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
ab5515b [Davies Liu] Merge branch 'master' into groupby
651f891 [Davies Liu] simplify GroupByKey
1578f2e [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
1f69f93 [Davies Liu] fix tests
0d3395f [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
341f1e0 [Davies Liu] add comments, refactor
47918b8 [Davies Liu] remove unused code
6540948 [Davies Liu] address comments:
17f4ec6 [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
4d4bc86 [Davies Liu] bugfix
8ef965e [Davies Liu] Merge branch 'master' into groupby
fbc504a [Davies Liu] Merge branch 'master' into groupby
779ed03 [Davies Liu] fix merge conflict
2c1d05b [Davies Liu] refactor, minor turning
b48cda5 [Davies Liu] Merge branch 'master' into groupby
85138e6 [Davies Liu] Merge branch 'master' into groupby
acd8e1b [Davies Liu] fix memory when groupByKey().count()
905b233 [Davies Liu] Merge branch 'sort' into groupby
1f075ed [Davies Liu] Merge branch 'master' into sort
4b07d39 [Davies Liu] compress the data while spilling
0a081c6 [Davies Liu] Merge branch 'master' into groupby
f157fe7 [Davies Liu] Merge branch 'sort' into groupby
eb53ca6 [Davies Liu] Merge branch 'master' into sort
b2dc3bf [Davies Liu] Merge branch 'sort' into groupby
644abaf [Davies Liu] add license in LICENSE
19f7873 [Davies Liu] improve tests
11ba318 [Davies Liu] typo
085aef8 [Davies Liu] Merge branch 'master' into groupby
3ee58e5 [Davies Liu] switch to sort based groupBy, based on size of data
1ea0669 [Davies Liu] choose sort based groupByKey() automatically
b40bae7 [Davies Liu] bugfix
efa23df [Davies Liu] refactor, add spark.shuffle.sort=False
250be4e [Davies Liu] flatten the combined values when dumping into disks
d05060d [Davies Liu] group the same key before shuffle, reduce the comparison during sorting
083d842 [Davies Liu] sorted based groupByKey()
55602ee [Davies Liu] use external sort in sortBy() and sortByKey()
2015-04-09 20:07:23 -04:00
|
|
|
|
|
|
|
def groupByKey(it):
|
2015-09-20 00:40:21 -04:00
|
|
|
merger = ExternalGroupBy(agg, memory, serializer)
|
[SPARK-3074] [PySpark] support groupByKey() with single huge key
This patch change groupByKey() to use external sort based approach, so it can support single huge key.
For example, it can group by a dataset including one hot key with 40 millions values (strings), using 500M memory for Python worker, finished in about 2 minutes. (it will need 6G memory in hash based approach).
During groupByKey(), it will do in-memory groupBy first. If the dataset can not fit in memory, then data will be partitioned by hash. If one partition still can not fit in memory, it will switch to sort based groupBy().
Author: Davies Liu <davies.liu@gmail.com>
Author: Davies Liu <davies@databricks.com>
Closes #1977 from davies/groupby and squashes the following commits:
af3713a [Davies Liu] make sure it's iterator
67772dd [Davies Liu] fix tests
e78c15c [Davies Liu] address comments
0b0fde8 [Davies Liu] address comments
0dcf320 [Davies Liu] address comments, rollback changes in ResultIterable
e3b8eab [Davies Liu] fix narrow dependency
2a1857a [Davies Liu] typo
d2f053b [Davies Liu] add repr for FlattedValuesSerializer
c6a2f8d [Davies Liu] address comments
9e2df24 [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
2b9c261 [Davies Liu] fix typo in comments
70aadcd [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
a14b4bd [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
ab5515b [Davies Liu] Merge branch 'master' into groupby
651f891 [Davies Liu] simplify GroupByKey
1578f2e [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
1f69f93 [Davies Liu] fix tests
0d3395f [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
341f1e0 [Davies Liu] add comments, refactor
47918b8 [Davies Liu] remove unused code
6540948 [Davies Liu] address comments:
17f4ec6 [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
4d4bc86 [Davies Liu] bugfix
8ef965e [Davies Liu] Merge branch 'master' into groupby
fbc504a [Davies Liu] Merge branch 'master' into groupby
779ed03 [Davies Liu] fix merge conflict
2c1d05b [Davies Liu] refactor, minor turning
b48cda5 [Davies Liu] Merge branch 'master' into groupby
85138e6 [Davies Liu] Merge branch 'master' into groupby
acd8e1b [Davies Liu] fix memory when groupByKey().count()
905b233 [Davies Liu] Merge branch 'sort' into groupby
1f075ed [Davies Liu] Merge branch 'master' into sort
4b07d39 [Davies Liu] compress the data while spilling
0a081c6 [Davies Liu] Merge branch 'master' into groupby
f157fe7 [Davies Liu] Merge branch 'sort' into groupby
eb53ca6 [Davies Liu] Merge branch 'master' into sort
b2dc3bf [Davies Liu] Merge branch 'sort' into groupby
644abaf [Davies Liu] add license in LICENSE
19f7873 [Davies Liu] improve tests
11ba318 [Davies Liu] typo
085aef8 [Davies Liu] Merge branch 'master' into groupby
3ee58e5 [Davies Liu] switch to sort based groupBy, based on size of data
1ea0669 [Davies Liu] choose sort based groupByKey() automatically
b40bae7 [Davies Liu] bugfix
efa23df [Davies Liu] refactor, add spark.shuffle.sort=False
250be4e [Davies Liu] flatten the combined values when dumping into disks
d05060d [Davies Liu] group the same key before shuffle, reduce the comparison during sorting
083d842 [Davies Liu] sorted based groupByKey()
55602ee [Davies Liu] use external sort in sortBy() and sortByKey()
2015-04-09 20:07:23 -04:00
|
|
|
merger.mergeCombiners(it)
|
2015-04-16 19:20:57 -04:00
|
|
|
return merger.items()
|
[SPARK-3074] [PySpark] support groupByKey() with single huge key
This patch change groupByKey() to use external sort based approach, so it can support single huge key.
For example, it can group by a dataset including one hot key with 40 millions values (strings), using 500M memory for Python worker, finished in about 2 minutes. (it will need 6G memory in hash based approach).
During groupByKey(), it will do in-memory groupBy first. If the dataset can not fit in memory, then data will be partitioned by hash. If one partition still can not fit in memory, it will switch to sort based groupBy().
Author: Davies Liu <davies.liu@gmail.com>
Author: Davies Liu <davies@databricks.com>
Closes #1977 from davies/groupby and squashes the following commits:
af3713a [Davies Liu] make sure it's iterator
67772dd [Davies Liu] fix tests
e78c15c [Davies Liu] address comments
0b0fde8 [Davies Liu] address comments
0dcf320 [Davies Liu] address comments, rollback changes in ResultIterable
e3b8eab [Davies Liu] fix narrow dependency
2a1857a [Davies Liu] typo
d2f053b [Davies Liu] add repr for FlattedValuesSerializer
c6a2f8d [Davies Liu] address comments
9e2df24 [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
2b9c261 [Davies Liu] fix typo in comments
70aadcd [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
a14b4bd [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
ab5515b [Davies Liu] Merge branch 'master' into groupby
651f891 [Davies Liu] simplify GroupByKey
1578f2e [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
1f69f93 [Davies Liu] fix tests
0d3395f [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
341f1e0 [Davies Liu] add comments, refactor
47918b8 [Davies Liu] remove unused code
6540948 [Davies Liu] address comments:
17f4ec6 [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
4d4bc86 [Davies Liu] bugfix
8ef965e [Davies Liu] Merge branch 'master' into groupby
fbc504a [Davies Liu] Merge branch 'master' into groupby
779ed03 [Davies Liu] fix merge conflict
2c1d05b [Davies Liu] refactor, minor turning
b48cda5 [Davies Liu] Merge branch 'master' into groupby
85138e6 [Davies Liu] Merge branch 'master' into groupby
acd8e1b [Davies Liu] fix memory when groupByKey().count()
905b233 [Davies Liu] Merge branch 'sort' into groupby
1f075ed [Davies Liu] Merge branch 'master' into sort
4b07d39 [Davies Liu] compress the data while spilling
0a081c6 [Davies Liu] Merge branch 'master' into groupby
f157fe7 [Davies Liu] Merge branch 'sort' into groupby
eb53ca6 [Davies Liu] Merge branch 'master' into sort
b2dc3bf [Davies Liu] Merge branch 'sort' into groupby
644abaf [Davies Liu] add license in LICENSE
19f7873 [Davies Liu] improve tests
11ba318 [Davies Liu] typo
085aef8 [Davies Liu] Merge branch 'master' into groupby
3ee58e5 [Davies Liu] switch to sort based groupBy, based on size of data
1ea0669 [Davies Liu] choose sort based groupByKey() automatically
b40bae7 [Davies Liu] bugfix
efa23df [Davies Liu] refactor, add spark.shuffle.sort=False
250be4e [Davies Liu] flatten the combined values when dumping into disks
d05060d [Davies Liu] group the same key before shuffle, reduce the comparison during sorting
083d842 [Davies Liu] sorted based groupByKey()
55602ee [Davies Liu] use external sort in sortBy() and sortByKey()
2015-04-09 20:07:23 -04:00
|
|
|
|
|
|
|
return shuffled.mapPartitions(groupByKey, True).mapValues(ResultIterable)
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2012-08-18 19:07:10 -04:00
|
|
|
def flatMapValues(self, f):
|
2012-12-27 20:55:33 -05:00
|
|
|
"""
|
|
|
|
Pass each value in the key-value pair RDD through a flatMap function
|
|
|
|
without changing the keys; this also retains the original RDD's
|
|
|
|
partitioning.
|
2014-02-22 18:10:31 -05:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2014-02-22 18:10:31 -05:00
|
|
|
>>> x = sc.parallelize([("a", ["x", "y", "z"]), ("b", ["p", "r"])])
|
|
|
|
>>> def f(x): return x
|
|
|
|
>>> x.flatMapValues(f).collect()
|
|
|
|
[('a', 'x'), ('a', 'y'), ('a', 'z'), ('b', 'p'), ('b', 'r')]
|
2012-12-27 20:55:33 -05:00
|
|
|
"""
|
2015-04-16 19:20:57 -04:00
|
|
|
flat_map_fn = lambda kv: ((kv[0], x) for x in f(kv[1]))
|
2012-12-27 20:55:33 -05:00
|
|
|
return self.flatMap(flat_map_fn, preservesPartitioning=True)
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2012-08-18 19:07:10 -04:00
|
|
|
def mapValues(self, f):
|
2012-12-27 20:55:33 -05:00
|
|
|
"""
|
|
|
|
Pass each value in the key-value pair RDD through a map function
|
|
|
|
without changing the keys; this also retains the original RDD's
|
|
|
|
partitioning.
|
2014-02-22 18:10:31 -05:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2014-02-22 18:10:31 -05:00
|
|
|
>>> x = sc.parallelize([("a", ["apple", "banana", "lemon"]), ("b", ["grapes"])])
|
|
|
|
>>> def f(x): return len(x)
|
|
|
|
>>> x.mapValues(f).collect()
|
|
|
|
[('a', 3), ('b', 1)]
|
2012-12-27 20:55:33 -05:00
|
|
|
"""
|
2015-04-16 19:20:57 -04:00
|
|
|
map_values_fn = lambda kv: (kv[0], f(kv[1]))
|
2012-08-18 19:07:10 -04:00
|
|
|
return self.map(map_values_fn, preservesPartitioning=True)
|
|
|
|
|
2014-06-20 14:03:03 -04:00
|
|
|
def groupWith(self, other, *others):
|
2012-12-27 20:55:33 -05:00
|
|
|
"""
|
2014-06-20 14:03:03 -04:00
|
|
|
Alias for cogroup but with support for multiple RDDs.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2014-06-20 14:03:03 -04:00
|
|
|
>>> w = sc.parallelize([("a", 5), ("b", 6)])
|
|
|
|
>>> x = sc.parallelize([("a", 1), ("b", 4)])
|
|
|
|
>>> y = sc.parallelize([("a", 2)])
|
|
|
|
>>> z = sc.parallelize([("b", 42)])
|
2015-04-16 19:20:57 -04:00
|
|
|
>>> [(x, tuple(map(list, y))) for x, y in sorted(list(w.groupWith(x, y, z).collect()))]
|
2014-06-20 14:03:03 -04:00
|
|
|
[('a', ([5], [1], [2], [])), ('b', ([6], [4], [], [42]))]
|
|
|
|
|
2012-12-27 20:55:33 -05:00
|
|
|
"""
|
2014-06-20 14:03:03 -04:00
|
|
|
return python_cogroup((self, other) + others, numPartitions=None)
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2020-11-27 11:22:45 -05:00
|
|
|
# TODO: add variant with custom partitioner
|
2013-02-24 16:25:09 -05:00
|
|
|
def cogroup(self, other, numPartitions=None):
|
2012-08-10 04:10:02 -04:00
|
|
|
"""
|
2019-07-05 13:08:22 -04:00
|
|
|
For each key k in `self` or `other`, return a resulting RDD that
|
|
|
|
contains a tuple with the list of values for that key in `self` as
|
|
|
|
well as `other`.
|
2012-12-27 20:55:33 -05:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2012-08-18 19:07:10 -04:00
|
|
|
>>> x = sc.parallelize([("a", 1), ("b", 4)])
|
|
|
|
>>> y = sc.parallelize([("a", 2)])
|
2015-04-16 19:20:57 -04:00
|
|
|
>>> [(x, tuple(map(list, y))) for x, y in sorted(list(x.cogroup(y).collect()))]
|
2012-08-10 04:10:02 -04:00
|
|
|
[('a', ([1], [2])), ('b', ([4], []))]
|
|
|
|
"""
|
2014-06-20 14:03:03 -04:00
|
|
|
return python_cogroup((self, other), numPartitions)
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2014-07-25 02:42:08 -04:00
|
|
|
def sampleByKey(self, withReplacement, fractions, seed=None):
|
|
|
|
"""
|
|
|
|
Return a subset of this RDD sampled by key (via stratified sampling).
|
|
|
|
Create a sample of this RDD using variable sampling rates for
|
|
|
|
different keys as specified by fractions, a key to sampling rate map.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2014-07-25 02:42:08 -04:00
|
|
|
>>> fractions = {"a": 0.2, "b": 0.1}
|
|
|
|
>>> rdd = sc.parallelize(fractions.keys()).cartesian(sc.parallelize(range(0, 1000)))
|
|
|
|
>>> sample = dict(rdd.sampleByKey(False, fractions, 2).groupByKey().collect())
|
|
|
|
>>> 100 < len(sample["a"]) < 300 and 50 < len(sample["b"]) < 150
|
|
|
|
True
|
|
|
|
>>> max(sample["a"]) <= 999 and min(sample["a"]) >= 0
|
|
|
|
True
|
|
|
|
>>> max(sample["b"]) <= 999 and min(sample["b"]) >= 0
|
|
|
|
True
|
|
|
|
"""
|
|
|
|
for fraction in fractions.values():
|
|
|
|
assert fraction >= 0.0, "Negative fraction value: %s" % fraction
|
2014-08-06 15:58:24 -04:00
|
|
|
return self.mapPartitionsWithIndex(
|
2014-07-25 02:42:08 -04:00
|
|
|
RDDStratifiedSampler(withReplacement, fractions, seed).func, True)
|
|
|
|
|
2013-08-28 19:10:38 -04:00
|
|
|
def subtractByKey(self, other, numPartitions=None):
|
|
|
|
"""
|
2019-07-05 13:08:22 -04:00
|
|
|
Return each (key, value) pair in `self` that has no pair with matching
|
|
|
|
key in `other`.
|
2013-08-28 19:10:38 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2013-08-28 19:10:38 -04:00
|
|
|
>>> x = sc.parallelize([("a", 1), ("b", 4), ("b", 5), ("a", 2)])
|
|
|
|
>>> y = sc.parallelize([("a", 3), ("c", None)])
|
|
|
|
>>> sorted(x.subtractByKey(y).collect())
|
|
|
|
[('b', 4), ('b', 5)]
|
2013-10-09 15:08:04 -04:00
|
|
|
"""
|
2015-04-16 19:20:57 -04:00
|
|
|
def filter_func(pair):
|
|
|
|
key, (val1, val2) = pair
|
|
|
|
return val1 and not val2
|
2014-08-27 16:18:33 -04:00
|
|
|
return self.cogroup(other, numPartitions).filter(filter_func).flatMapValues(lambda x: x[0])
|
2013-08-28 19:10:38 -04:00
|
|
|
|
|
|
|
def subtract(self, other, numPartitions=None):
|
|
|
|
"""
|
2019-07-05 13:08:22 -04:00
|
|
|
Return each value in `self` that is not contained in `other`.
|
2013-08-28 19:10:38 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2013-08-28 19:10:38 -04:00
|
|
|
>>> x = sc.parallelize([("a", 1), ("b", 4), ("b", 5), ("a", 3)])
|
|
|
|
>>> y = sc.parallelize([("a", 3), ("c", None)])
|
|
|
|
>>> sorted(x.subtract(y).collect())
|
|
|
|
[('a', 1), ('b', 4), ('b', 5)]
|
|
|
|
"""
|
2014-07-14 03:42:59 -04:00
|
|
|
# note: here 'True' is just a placeholder
|
|
|
|
rdd = other.map(lambda x: (x, True))
|
2014-08-27 16:18:33 -04:00
|
|
|
return self.map(lambda x: (x, True)).subtractByKey(rdd, numPartitions).keys()
|
2013-08-28 19:10:38 -04:00
|
|
|
|
|
|
|
def keyBy(self, f):
|
|
|
|
"""
|
2019-07-05 13:08:22 -04:00
|
|
|
Creates tuples of the elements in this RDD by applying `f`.
|
2013-08-28 19:10:38 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2013-08-28 19:10:38 -04:00
|
|
|
>>> x = sc.parallelize(range(0,3)).keyBy(lambda x: x*x)
|
|
|
|
>>> y = sc.parallelize(zip(range(0,5), range(0,5)))
|
2015-04-16 19:20:57 -04:00
|
|
|
>>> [(x, list(map(list, y))) for x, y in sorted(x.cogroup(y).collect())]
|
|
|
|
[(0, [[0], [0]]), (1, [[1], [1]]), (2, [[], [2]]), (3, [[], [3]]), (4, [[2], [4]])]
|
2013-08-28 19:10:38 -04:00
|
|
|
"""
|
|
|
|
return self.map(lambda x: (f(x), x))
|
|
|
|
|
2014-02-06 17:58:35 -05:00
|
|
|
def repartition(self, numPartitions):
|
|
|
|
"""
|
|
|
|
Return a new RDD that has exactly numPartitions partitions.
|
2014-06-09 03:08:40 -04:00
|
|
|
|
2014-06-12 22:44:27 -04:00
|
|
|
Can increase or decrease the level of parallelism in this RDD.
|
|
|
|
Internally, this uses a shuffle to redistribute data.
|
|
|
|
If you are decreasing the number of partitions in this RDD, consider
|
|
|
|
using `coalesce`, which can avoid performing a shuffle.
|
2014-08-06 17:12:21 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2014-02-06 17:58:35 -05:00
|
|
|
>>> rdd = sc.parallelize([1,2,3,4,5,6,7], 4)
|
|
|
|
>>> sorted(rdd.glom().collect())
|
|
|
|
[[1], [2, 3], [4, 5], [6, 7]]
|
|
|
|
>>> len(rdd.repartition(2).glom().collect())
|
|
|
|
2
|
|
|
|
>>> len(rdd.repartition(10).glom().collect())
|
|
|
|
10
|
|
|
|
"""
|
2016-10-11 14:43:24 -04:00
|
|
|
return self.coalesce(numPartitions, shuffle=True)
|
2014-02-06 17:58:35 -05:00
|
|
|
|
|
|
|
def coalesce(self, numPartitions, shuffle=False):
|
|
|
|
"""
|
|
|
|
Return a new RDD that is reduced into `numPartitions` partitions.
|
2014-08-06 17:12:21 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2014-02-06 17:58:35 -05:00
|
|
|
>>> sc.parallelize([1, 2, 3, 4, 5], 3).glom().collect()
|
|
|
|
[[1], [2, 3], [4, 5]]
|
|
|
|
>>> sc.parallelize([1, 2, 3, 4, 5], 3).coalesce(1).glom().collect()
|
|
|
|
[[1, 2, 3, 4, 5]]
|
|
|
|
"""
|
2016-10-11 14:43:24 -04:00
|
|
|
if shuffle:
|
2016-10-18 17:25:10 -04:00
|
|
|
# Decrease the batch size in order to distribute evenly the elements across output
|
|
|
|
# partitions. Otherwise, repartition will possibly produce highly skewed partitions.
|
|
|
|
batchSize = min(10, self.ctx._batchSize or 1024)
|
|
|
|
ser = BatchedSerializer(PickleSerializer(), batchSize)
|
|
|
|
selfCopy = self._reserialize(ser)
|
[SPARK-19872] [PYTHON] Use the correct deserializer for RDD construction for coalesce/repartition
## What changes were proposed in this pull request?
This PR proposes to use the correct deserializer, `BatchedSerializer` for RDD construction for coalesce/repartition when the shuffle is enabled. Currently, it is passing `UTF8Deserializer` as is not `BatchedSerializer` from the copied one.
with the file, `text.txt` below:
```
a
b
d
e
f
g
h
i
j
k
l
```
- Before
```python
>>> sc.textFile('text.txt').repartition(1).collect()
```
```
UTF8Deserializer(True)
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File ".../spark/python/pyspark/rdd.py", line 811, in collect
return list(_load_from_socket(port, self._jrdd_deserializer))
File ".../spark/python/pyspark/serializers.py", line 549, in load_stream
yield self.loads(stream)
File ".../spark/python/pyspark/serializers.py", line 544, in loads
return s.decode("utf-8") if self.use_unicode else s
File "/System/Library/Frameworks/Python.framework/Versions/2.7/lib/python2.7/encodings/utf_8.py", line 16, in decode
return codecs.utf_8_decode(input, errors, True)
UnicodeDecodeError: 'utf8' codec can't decode byte 0x80 in position 0: invalid start byte
```
- After
```python
>>> sc.textFile('text.txt').repartition(1).collect()
```
```
[u'a', u'b', u'', u'd', u'e', u'f', u'g', u'h', u'i', u'j', u'k', u'l', u'']
```
## How was this patch tested?
Unit test in `python/pyspark/tests.py`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #17282 from HyukjinKwon/SPARK-19872.
2017-03-15 13:17:18 -04:00
|
|
|
jrdd_deserializer = selfCopy._jrdd_deserializer
|
2016-10-18 17:25:10 -04:00
|
|
|
jrdd = selfCopy._jrdd.coalesce(numPartitions, shuffle)
|
2016-10-11 14:43:24 -04:00
|
|
|
else:
|
[SPARK-19872] [PYTHON] Use the correct deserializer for RDD construction for coalesce/repartition
## What changes were proposed in this pull request?
This PR proposes to use the correct deserializer, `BatchedSerializer` for RDD construction for coalesce/repartition when the shuffle is enabled. Currently, it is passing `UTF8Deserializer` as is not `BatchedSerializer` from the copied one.
with the file, `text.txt` below:
```
a
b
d
e
f
g
h
i
j
k
l
```
- Before
```python
>>> sc.textFile('text.txt').repartition(1).collect()
```
```
UTF8Deserializer(True)
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File ".../spark/python/pyspark/rdd.py", line 811, in collect
return list(_load_from_socket(port, self._jrdd_deserializer))
File ".../spark/python/pyspark/serializers.py", line 549, in load_stream
yield self.loads(stream)
File ".../spark/python/pyspark/serializers.py", line 544, in loads
return s.decode("utf-8") if self.use_unicode else s
File "/System/Library/Frameworks/Python.framework/Versions/2.7/lib/python2.7/encodings/utf_8.py", line 16, in decode
return codecs.utf_8_decode(input, errors, True)
UnicodeDecodeError: 'utf8' codec can't decode byte 0x80 in position 0: invalid start byte
```
- After
```python
>>> sc.textFile('text.txt').repartition(1).collect()
```
```
[u'a', u'b', u'', u'd', u'e', u'f', u'g', u'h', u'i', u'j', u'k', u'l', u'']
```
## How was this patch tested?
Unit test in `python/pyspark/tests.py`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #17282 from HyukjinKwon/SPARK-19872.
2017-03-15 13:17:18 -04:00
|
|
|
jrdd_deserializer = self._jrdd_deserializer
|
2016-10-11 14:43:24 -04:00
|
|
|
jrdd = self._jrdd.coalesce(numPartitions, shuffle)
|
[SPARK-19872] [PYTHON] Use the correct deserializer for RDD construction for coalesce/repartition
## What changes were proposed in this pull request?
This PR proposes to use the correct deserializer, `BatchedSerializer` for RDD construction for coalesce/repartition when the shuffle is enabled. Currently, it is passing `UTF8Deserializer` as is not `BatchedSerializer` from the copied one.
with the file, `text.txt` below:
```
a
b
d
e
f
g
h
i
j
k
l
```
- Before
```python
>>> sc.textFile('text.txt').repartition(1).collect()
```
```
UTF8Deserializer(True)
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File ".../spark/python/pyspark/rdd.py", line 811, in collect
return list(_load_from_socket(port, self._jrdd_deserializer))
File ".../spark/python/pyspark/serializers.py", line 549, in load_stream
yield self.loads(stream)
File ".../spark/python/pyspark/serializers.py", line 544, in loads
return s.decode("utf-8") if self.use_unicode else s
File "/System/Library/Frameworks/Python.framework/Versions/2.7/lib/python2.7/encodings/utf_8.py", line 16, in decode
return codecs.utf_8_decode(input, errors, True)
UnicodeDecodeError: 'utf8' codec can't decode byte 0x80 in position 0: invalid start byte
```
- After
```python
>>> sc.textFile('text.txt').repartition(1).collect()
```
```
[u'a', u'b', u'', u'd', u'e', u'f', u'g', u'h', u'i', u'j', u'k', u'l', u'']
```
## How was this patch tested?
Unit test in `python/pyspark/tests.py`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #17282 from HyukjinKwon/SPARK-19872.
2017-03-15 13:17:18 -04:00
|
|
|
return RDD(jrdd, self.ctx, jrdd_deserializer)
|
2014-02-06 17:58:35 -05:00
|
|
|
|
2014-03-10 16:27:00 -04:00
|
|
|
def zip(self, other):
|
|
|
|
"""
|
2014-06-12 22:44:27 -04:00
|
|
|
Zips this RDD with another one, returning key-value pairs with the
|
|
|
|
first element in each RDD second element in each RDD, etc. Assumes
|
|
|
|
that the two RDDs have the same number of partitions and the same
|
|
|
|
number of elements in each partition (e.g. one was made through
|
2014-03-10 16:27:00 -04:00
|
|
|
a map on the other).
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2014-03-10 16:27:00 -04:00
|
|
|
>>> x = sc.parallelize(range(0,5))
|
|
|
|
>>> y = sc.parallelize(range(1000, 1005))
|
|
|
|
>>> x.zip(y).collect()
|
|
|
|
[(0, 1000), (1, 1001), (2, 1002), (3, 1003), (4, 1004)]
|
|
|
|
"""
|
2014-08-19 17:46:32 -04:00
|
|
|
def get_batch_size(ser):
|
|
|
|
if isinstance(ser, BatchedSerializer):
|
|
|
|
return ser.batchSize
|
2014-12-16 01:58:26 -05:00
|
|
|
return 1 # not batched
|
2014-08-19 17:46:32 -04:00
|
|
|
|
|
|
|
def batch_as(rdd, batchSize):
|
2014-12-16 01:58:26 -05:00
|
|
|
return rdd._reserialize(BatchedSerializer(PickleSerializer(), batchSize))
|
2014-08-19 17:46:32 -04:00
|
|
|
|
|
|
|
my_batch = get_batch_size(self._jrdd_deserializer)
|
|
|
|
other_batch = get_batch_size(other._jrdd_deserializer)
|
2015-02-24 17:50:00 -05:00
|
|
|
if my_batch != other_batch or not my_batch:
|
2014-12-16 01:58:26 -05:00
|
|
|
# use the smallest batchSize for both of them
|
|
|
|
batchSize = min(my_batch, other_batch)
|
|
|
|
if batchSize <= 0:
|
|
|
|
# auto batched or unlimited
|
|
|
|
batchSize = 100
|
|
|
|
other = batch_as(other, batchSize)
|
|
|
|
self = batch_as(self, batchSize)
|
2014-11-04 02:56:14 -05:00
|
|
|
|
|
|
|
if self.getNumPartitions() != other.getNumPartitions():
|
|
|
|
raise ValueError("Can only zip with RDD which has the same number of partitions")
|
2014-08-19 17:46:32 -04:00
|
|
|
|
|
|
|
# There will be an Exception in JVM if there are different number
|
|
|
|
# of items in each partitions.
|
2014-03-10 16:27:00 -04:00
|
|
|
pairRDD = self._jrdd.zip(other._jrdd)
|
|
|
|
deserializer = PairDeserializer(self._jrdd_deserializer,
|
2014-07-14 03:42:59 -04:00
|
|
|
other._jrdd_deserializer)
|
2014-03-10 16:27:00 -04:00
|
|
|
return RDD(pairRDD, self.ctx, deserializer)
|
|
|
|
|
[SPARK-2871] [PySpark] add zipWithIndex() and zipWithUniqueId()
RDD.zipWithIndex()
Zips this RDD with its element indices.
The ordering is first based on the partition index and then the
ordering of items within each partition. So the first item in
the first partition gets index 0, and the last item in the last
partition receives the largest index.
This method needs to trigger a spark job when this RDD contains
more than one partitions.
>>> sc.parallelize(range(4), 2).zipWithIndex().collect()
[(0, 0), (1, 1), (2, 2), (3, 3)]
RDD.zipWithUniqueId()
Zips this RDD with generated unique Long ids.
Items in the kth partition will get ids k, n+k, 2*n+k, ..., where
n is the number of partitions. So there may exist gaps, but this
method won't trigger a spark job, which is different from
L{zipWithIndex}
>>> sc.parallelize(range(4), 2).zipWithUniqueId().collect()
[(0, 0), (2, 1), (1, 2), (3, 3)]
Author: Davies Liu <davies.liu@gmail.com>
Closes #2092 from davies/zipWith and squashes the following commits:
cebe5bf [Davies Liu] improve test cases, reverse the order of index
0d2a128 [Davies Liu] add zipWithIndex() and zipWithUniqueId()
2014-08-25 00:16:05 -04:00
|
|
|
def zipWithIndex(self):
|
|
|
|
"""
|
|
|
|
Zips this RDD with its element indices.
|
|
|
|
|
|
|
|
The ordering is first based on the partition index and then the
|
|
|
|
ordering of items within each partition. So the first item in
|
|
|
|
the first partition gets index 0, and the last item in the last
|
|
|
|
partition receives the largest index.
|
|
|
|
|
|
|
|
This method needs to trigger a spark job when this RDD contains
|
|
|
|
more than one partitions.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
[SPARK-2871] [PySpark] add zipWithIndex() and zipWithUniqueId()
RDD.zipWithIndex()
Zips this RDD with its element indices.
The ordering is first based on the partition index and then the
ordering of items within each partition. So the first item in
the first partition gets index 0, and the last item in the last
partition receives the largest index.
This method needs to trigger a spark job when this RDD contains
more than one partitions.
>>> sc.parallelize(range(4), 2).zipWithIndex().collect()
[(0, 0), (1, 1), (2, 2), (3, 3)]
RDD.zipWithUniqueId()
Zips this RDD with generated unique Long ids.
Items in the kth partition will get ids k, n+k, 2*n+k, ..., where
n is the number of partitions. So there may exist gaps, but this
method won't trigger a spark job, which is different from
L{zipWithIndex}
>>> sc.parallelize(range(4), 2).zipWithUniqueId().collect()
[(0, 0), (2, 1), (1, 2), (3, 3)]
Author: Davies Liu <davies.liu@gmail.com>
Closes #2092 from davies/zipWith and squashes the following commits:
cebe5bf [Davies Liu] improve test cases, reverse the order of index
0d2a128 [Davies Liu] add zipWithIndex() and zipWithUniqueId()
2014-08-25 00:16:05 -04:00
|
|
|
>>> sc.parallelize(["a", "b", "c", "d"], 3).zipWithIndex().collect()
|
|
|
|
[('a', 0), ('b', 1), ('c', 2), ('d', 3)]
|
|
|
|
"""
|
|
|
|
starts = [0]
|
|
|
|
if self.getNumPartitions() > 1:
|
|
|
|
nums = self.mapPartitions(lambda it: [sum(1 for i in it)]).collect()
|
|
|
|
for i in range(len(nums) - 1):
|
|
|
|
starts.append(starts[-1] + nums[i])
|
|
|
|
|
|
|
|
def func(k, it):
|
|
|
|
for i, v in enumerate(it, starts[k]):
|
|
|
|
yield v, i
|
|
|
|
|
|
|
|
return self.mapPartitionsWithIndex(func)
|
|
|
|
|
|
|
|
def zipWithUniqueId(self):
|
|
|
|
"""
|
|
|
|
Zips this RDD with generated unique Long ids.
|
|
|
|
|
|
|
|
Items in the kth partition will get ids k, n+k, 2*n+k, ..., where
|
|
|
|
n is the number of partitions. So there may exist gaps, but this
|
|
|
|
method won't trigger a spark job, which is different from
|
2019-07-05 13:08:22 -04:00
|
|
|
:meth:`zipWithIndex`.
|
[SPARK-2871] [PySpark] add zipWithIndex() and zipWithUniqueId()
RDD.zipWithIndex()
Zips this RDD with its element indices.
The ordering is first based on the partition index and then the
ordering of items within each partition. So the first item in
the first partition gets index 0, and the last item in the last
partition receives the largest index.
This method needs to trigger a spark job when this RDD contains
more than one partitions.
>>> sc.parallelize(range(4), 2).zipWithIndex().collect()
[(0, 0), (1, 1), (2, 2), (3, 3)]
RDD.zipWithUniqueId()
Zips this RDD with generated unique Long ids.
Items in the kth partition will get ids k, n+k, 2*n+k, ..., where
n is the number of partitions. So there may exist gaps, but this
method won't trigger a spark job, which is different from
L{zipWithIndex}
>>> sc.parallelize(range(4), 2).zipWithUniqueId().collect()
[(0, 0), (2, 1), (1, 2), (3, 3)]
Author: Davies Liu <davies.liu@gmail.com>
Closes #2092 from davies/zipWith and squashes the following commits:
cebe5bf [Davies Liu] improve test cases, reverse the order of index
0d2a128 [Davies Liu] add zipWithIndex() and zipWithUniqueId()
2014-08-25 00:16:05 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
[SPARK-2871] [PySpark] add zipWithIndex() and zipWithUniqueId()
RDD.zipWithIndex()
Zips this RDD with its element indices.
The ordering is first based on the partition index and then the
ordering of items within each partition. So the first item in
the first partition gets index 0, and the last item in the last
partition receives the largest index.
This method needs to trigger a spark job when this RDD contains
more than one partitions.
>>> sc.parallelize(range(4), 2).zipWithIndex().collect()
[(0, 0), (1, 1), (2, 2), (3, 3)]
RDD.zipWithUniqueId()
Zips this RDD with generated unique Long ids.
Items in the kth partition will get ids k, n+k, 2*n+k, ..., where
n is the number of partitions. So there may exist gaps, but this
method won't trigger a spark job, which is different from
L{zipWithIndex}
>>> sc.parallelize(range(4), 2).zipWithUniqueId().collect()
[(0, 0), (2, 1), (1, 2), (3, 3)]
Author: Davies Liu <davies.liu@gmail.com>
Closes #2092 from davies/zipWith and squashes the following commits:
cebe5bf [Davies Liu] improve test cases, reverse the order of index
0d2a128 [Davies Liu] add zipWithIndex() and zipWithUniqueId()
2014-08-25 00:16:05 -04:00
|
|
|
>>> sc.parallelize(["a", "b", "c", "d", "e"], 3).zipWithUniqueId().collect()
|
|
|
|
[('a', 0), ('b', 1), ('c', 4), ('d', 2), ('e', 5)]
|
|
|
|
"""
|
|
|
|
n = self.getNumPartitions()
|
|
|
|
|
|
|
|
def func(k, it):
|
|
|
|
for i, v in enumerate(it):
|
|
|
|
yield v, i * n + k
|
|
|
|
|
|
|
|
return self.mapPartitionsWithIndex(func)
|
|
|
|
|
2014-03-12 02:57:05 -04:00
|
|
|
def name(self):
|
|
|
|
"""
|
|
|
|
Return the name of this RDD.
|
|
|
|
"""
|
2015-04-16 19:20:57 -04:00
|
|
|
n = self._jrdd.name()
|
|
|
|
if n:
|
|
|
|
return n
|
2014-03-12 02:57:05 -04:00
|
|
|
|
|
|
|
def setName(self, name):
|
|
|
|
"""
|
|
|
|
Assign a name to this RDD.
|
2014-08-06 17:12:21 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2015-04-16 19:20:57 -04:00
|
|
|
>>> rdd1 = sc.parallelize([1, 2])
|
2014-10-31 15:07:48 -04:00
|
|
|
>>> rdd1.setName('RDD1').name()
|
2020-07-13 22:22:44 -04:00
|
|
|
'RDD1'
|
2014-03-12 02:57:05 -04:00
|
|
|
"""
|
|
|
|
self._jrdd.setName(name)
|
2014-10-31 15:07:48 -04:00
|
|
|
return self
|
2014-03-12 02:57:05 -04:00
|
|
|
|
|
|
|
def toDebugString(self):
|
|
|
|
"""
|
|
|
|
A description of this RDD and its recursive dependencies for debugging.
|
|
|
|
"""
|
|
|
|
debug_string = self._jrdd.toDebugString()
|
2014-08-27 16:18:33 -04:00
|
|
|
if debug_string:
|
|
|
|
return debug_string.encode('utf-8')
|
2014-03-12 02:57:05 -04:00
|
|
|
|
|
|
|
def getStorageLevel(self):
|
|
|
|
"""
|
|
|
|
Get the RDD's current storage level.
|
2014-06-17 02:31:31 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2014-03-12 02:57:05 -04:00
|
|
|
>>> rdd1 = sc.parallelize([1,2])
|
|
|
|
>>> rdd1.getStorageLevel()
|
2014-04-04 23:36:24 -04:00
|
|
|
StorageLevel(False, False, False, False, 1)
|
2014-06-17 02:31:31 -04:00
|
|
|
>>> print(rdd1.getStorageLevel())
|
|
|
|
Serialized 1x Replicated
|
2014-03-12 02:57:05 -04:00
|
|
|
"""
|
|
|
|
java_storage_level = self._jrdd.getStorageLevel()
|
|
|
|
storage_level = StorageLevel(java_storage_level.useDisk(),
|
|
|
|
java_storage_level.useMemory(),
|
2014-04-04 23:36:24 -04:00
|
|
|
java_storage_level.useOffHeap(),
|
2014-03-12 02:57:05 -04:00
|
|
|
java_storage_level.deserialized(),
|
|
|
|
java_storage_level.replication())
|
|
|
|
return storage_level
|
2014-03-10 16:27:00 -04:00
|
|
|
|
2014-06-20 03:06:57 -04:00
|
|
|
def _defaultReducePartitions(self):
|
|
|
|
"""
|
|
|
|
Returns the default number of partitions to use during reduce tasks (e.g., groupBy).
|
|
|
|
If spark.default.parallelism is set, then we'll use the value from SparkContext
|
|
|
|
defaultParallelism, otherwise we'll use the number of partitions in this RDD.
|
|
|
|
|
|
|
|
This mirrors the behavior of the Scala Partitioner#defaultPartitioner, intended to reduce
|
|
|
|
the likelihood of OOMs. Once PySpark adopts Partitioner-based APIs, this behavior will
|
|
|
|
be inherent.
|
|
|
|
"""
|
|
|
|
if self.ctx._conf.contains("spark.default.parallelism"):
|
|
|
|
return self.ctx.defaultParallelism
|
|
|
|
else:
|
|
|
|
return self.getNumPartitions()
|
|
|
|
|
2014-08-27 16:18:33 -04:00
|
|
|
def lookup(self, key):
|
|
|
|
"""
|
|
|
|
Return the list of values in the RDD for key `key`. This operation
|
|
|
|
is done efficiently if the RDD has a known partitioner by only
|
|
|
|
searching the partition that the key maps to.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2014-08-27 16:18:33 -04:00
|
|
|
>>> l = range(1000)
|
|
|
|
>>> rdd = sc.parallelize(zip(l, l), 10)
|
|
|
|
>>> rdd.lookup(42) # slow
|
|
|
|
[42]
|
|
|
|
>>> sorted = rdd.sortByKey()
|
|
|
|
>>> sorted.lookup(42) # fast
|
|
|
|
[42]
|
|
|
|
>>> sorted.lookup(1024)
|
|
|
|
[]
|
2015-09-17 13:02:15 -04:00
|
|
|
>>> rdd2 = sc.parallelize([(('a', 'b'), 'c')]).groupByKey()
|
|
|
|
>>> list(rdd2.lookup(('a', 'b'))[0])
|
|
|
|
['c']
|
2014-08-27 16:18:33 -04:00
|
|
|
"""
|
2015-04-16 19:20:57 -04:00
|
|
|
values = self.filter(lambda kv: kv[0] == key).values()
|
2014-08-27 16:18:33 -04:00
|
|
|
|
2015-02-17 19:54:57 -05:00
|
|
|
if self.partitioner is not None:
|
2015-07-23 00:04:04 -04:00
|
|
|
return self.ctx.runJob(values, lambda x: x, [self.partitioner(key)])
|
2014-08-27 16:18:33 -04:00
|
|
|
|
|
|
|
return values.collect()
|
2012-08-10 04:10:02 -04:00
|
|
|
|
2014-09-02 18:47:47 -04:00
|
|
|
def _to_java_object_rdd(self):
|
2016-06-06 04:35:47 -04:00
|
|
|
""" Return a JavaRDD of Object by unpickling
|
[SPARK-2871] [PySpark] add approx API for RDD
RDD.countApprox(self, timeout, confidence=0.95)
:: Experimental ::
Approximate version of count() that returns a potentially incomplete
result within a timeout, even if not all tasks have finished.
>>> rdd = sc.parallelize(range(1000), 10)
>>> rdd.countApprox(1000, 1.0)
1000
RDD.sumApprox(self, timeout, confidence=0.95)
Approximate operation to return the sum within a timeout
or meet the confidence.
>>> rdd = sc.parallelize(range(1000), 10)
>>> r = sum(xrange(1000))
>>> (rdd.sumApprox(1000) - r) / r < 0.05
RDD.meanApprox(self, timeout, confidence=0.95)
:: Experimental ::
Approximate operation to return the mean within a timeout
or meet the confidence.
>>> rdd = sc.parallelize(range(1000), 10)
>>> r = sum(xrange(1000)) / 1000.0
>>> (rdd.meanApprox(1000) - r) / r < 0.05
True
Author: Davies Liu <davies.liu@gmail.com>
Closes #2095 from davies/approx and squashes the following commits:
e8c252b [Davies Liu] add approx API for RDD
2014-08-23 22:33:34 -04:00
|
|
|
|
|
|
|
It will convert each Python object into Java object by Pyrolite, whenever the
|
|
|
|
RDD is serialized in batch or not.
|
|
|
|
"""
|
2014-11-04 02:56:14 -05:00
|
|
|
rdd = self._pickled()
|
|
|
|
return self.ctx._jvm.SerDeUtil.pythonToJava(rdd._jrdd, True)
|
[SPARK-2871] [PySpark] add approx API for RDD
RDD.countApprox(self, timeout, confidence=0.95)
:: Experimental ::
Approximate version of count() that returns a potentially incomplete
result within a timeout, even if not all tasks have finished.
>>> rdd = sc.parallelize(range(1000), 10)
>>> rdd.countApprox(1000, 1.0)
1000
RDD.sumApprox(self, timeout, confidence=0.95)
Approximate operation to return the sum within a timeout
or meet the confidence.
>>> rdd = sc.parallelize(range(1000), 10)
>>> r = sum(xrange(1000))
>>> (rdd.sumApprox(1000) - r) / r < 0.05
RDD.meanApprox(self, timeout, confidence=0.95)
:: Experimental ::
Approximate operation to return the mean within a timeout
or meet the confidence.
>>> rdd = sc.parallelize(range(1000), 10)
>>> r = sum(xrange(1000)) / 1000.0
>>> (rdd.meanApprox(1000) - r) / r < 0.05
True
Author: Davies Liu <davies.liu@gmail.com>
Closes #2095 from davies/approx and squashes the following commits:
e8c252b [Davies Liu] add approx API for RDD
2014-08-23 22:33:34 -04:00
|
|
|
|
|
|
|
def countApprox(self, timeout, confidence=0.95):
|
|
|
|
"""
|
|
|
|
Approximate version of count() that returns a potentially incomplete
|
|
|
|
result within a timeout, even if not all tasks have finished.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
[SPARK-2871] [PySpark] add approx API for RDD
RDD.countApprox(self, timeout, confidence=0.95)
:: Experimental ::
Approximate version of count() that returns a potentially incomplete
result within a timeout, even if not all tasks have finished.
>>> rdd = sc.parallelize(range(1000), 10)
>>> rdd.countApprox(1000, 1.0)
1000
RDD.sumApprox(self, timeout, confidence=0.95)
Approximate operation to return the sum within a timeout
or meet the confidence.
>>> rdd = sc.parallelize(range(1000), 10)
>>> r = sum(xrange(1000))
>>> (rdd.sumApprox(1000) - r) / r < 0.05
RDD.meanApprox(self, timeout, confidence=0.95)
:: Experimental ::
Approximate operation to return the mean within a timeout
or meet the confidence.
>>> rdd = sc.parallelize(range(1000), 10)
>>> r = sum(xrange(1000)) / 1000.0
>>> (rdd.meanApprox(1000) - r) / r < 0.05
True
Author: Davies Liu <davies.liu@gmail.com>
Closes #2095 from davies/approx and squashes the following commits:
e8c252b [Davies Liu] add approx API for RDD
2014-08-23 22:33:34 -04:00
|
|
|
>>> rdd = sc.parallelize(range(1000), 10)
|
|
|
|
>>> rdd.countApprox(1000, 1.0)
|
|
|
|
1000
|
|
|
|
"""
|
|
|
|
drdd = self.mapPartitions(lambda it: [float(sum(1 for i in it))])
|
|
|
|
return int(drdd.sumApprox(timeout, confidence))
|
|
|
|
|
|
|
|
def sumApprox(self, timeout, confidence=0.95):
|
|
|
|
"""
|
|
|
|
Approximate operation to return the sum within a timeout
|
|
|
|
or meet the confidence.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
[SPARK-2871] [PySpark] add approx API for RDD
RDD.countApprox(self, timeout, confidence=0.95)
:: Experimental ::
Approximate version of count() that returns a potentially incomplete
result within a timeout, even if not all tasks have finished.
>>> rdd = sc.parallelize(range(1000), 10)
>>> rdd.countApprox(1000, 1.0)
1000
RDD.sumApprox(self, timeout, confidence=0.95)
Approximate operation to return the sum within a timeout
or meet the confidence.
>>> rdd = sc.parallelize(range(1000), 10)
>>> r = sum(xrange(1000))
>>> (rdd.sumApprox(1000) - r) / r < 0.05
RDD.meanApprox(self, timeout, confidence=0.95)
:: Experimental ::
Approximate operation to return the mean within a timeout
or meet the confidence.
>>> rdd = sc.parallelize(range(1000), 10)
>>> r = sum(xrange(1000)) / 1000.0
>>> (rdd.meanApprox(1000) - r) / r < 0.05
True
Author: Davies Liu <davies.liu@gmail.com>
Closes #2095 from davies/approx and squashes the following commits:
e8c252b [Davies Liu] add approx API for RDD
2014-08-23 22:33:34 -04:00
|
|
|
>>> rdd = sc.parallelize(range(1000), 10)
|
2015-04-16 19:20:57 -04:00
|
|
|
>>> r = sum(range(1000))
|
2015-06-23 02:37:56 -04:00
|
|
|
>>> abs(rdd.sumApprox(1000) - r) / r < 0.05
|
[SPARK-2871] [PySpark] add approx API for RDD
RDD.countApprox(self, timeout, confidence=0.95)
:: Experimental ::
Approximate version of count() that returns a potentially incomplete
result within a timeout, even if not all tasks have finished.
>>> rdd = sc.parallelize(range(1000), 10)
>>> rdd.countApprox(1000, 1.0)
1000
RDD.sumApprox(self, timeout, confidence=0.95)
Approximate operation to return the sum within a timeout
or meet the confidence.
>>> rdd = sc.parallelize(range(1000), 10)
>>> r = sum(xrange(1000))
>>> (rdd.sumApprox(1000) - r) / r < 0.05
RDD.meanApprox(self, timeout, confidence=0.95)
:: Experimental ::
Approximate operation to return the mean within a timeout
or meet the confidence.
>>> rdd = sc.parallelize(range(1000), 10)
>>> r = sum(xrange(1000)) / 1000.0
>>> (rdd.meanApprox(1000) - r) / r < 0.05
True
Author: Davies Liu <davies.liu@gmail.com>
Closes #2095 from davies/approx and squashes the following commits:
e8c252b [Davies Liu] add approx API for RDD
2014-08-23 22:33:34 -04:00
|
|
|
True
|
|
|
|
"""
|
2014-09-02 18:47:47 -04:00
|
|
|
jrdd = self.mapPartitions(lambda it: [float(sum(it))])._to_java_object_rdd()
|
[SPARK-2871] [PySpark] add approx API for RDD
RDD.countApprox(self, timeout, confidence=0.95)
:: Experimental ::
Approximate version of count() that returns a potentially incomplete
result within a timeout, even if not all tasks have finished.
>>> rdd = sc.parallelize(range(1000), 10)
>>> rdd.countApprox(1000, 1.0)
1000
RDD.sumApprox(self, timeout, confidence=0.95)
Approximate operation to return the sum within a timeout
or meet the confidence.
>>> rdd = sc.parallelize(range(1000), 10)
>>> r = sum(xrange(1000))
>>> (rdd.sumApprox(1000) - r) / r < 0.05
RDD.meanApprox(self, timeout, confidence=0.95)
:: Experimental ::
Approximate operation to return the mean within a timeout
or meet the confidence.
>>> rdd = sc.parallelize(range(1000), 10)
>>> r = sum(xrange(1000)) / 1000.0
>>> (rdd.meanApprox(1000) - r) / r < 0.05
True
Author: Davies Liu <davies.liu@gmail.com>
Closes #2095 from davies/approx and squashes the following commits:
e8c252b [Davies Liu] add approx API for RDD
2014-08-23 22:33:34 -04:00
|
|
|
jdrdd = self.ctx._jvm.JavaDoubleRDD.fromRDD(jrdd.rdd())
|
|
|
|
r = jdrdd.sumApprox(timeout, confidence).getFinalValue()
|
|
|
|
return BoundedFloat(r.mean(), r.confidence(), r.low(), r.high())
|
|
|
|
|
|
|
|
def meanApprox(self, timeout, confidence=0.95):
|
|
|
|
"""
|
|
|
|
Approximate operation to return the mean within a timeout
|
|
|
|
or meet the confidence.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
[SPARK-2871] [PySpark] add approx API for RDD
RDD.countApprox(self, timeout, confidence=0.95)
:: Experimental ::
Approximate version of count() that returns a potentially incomplete
result within a timeout, even if not all tasks have finished.
>>> rdd = sc.parallelize(range(1000), 10)
>>> rdd.countApprox(1000, 1.0)
1000
RDD.sumApprox(self, timeout, confidence=0.95)
Approximate operation to return the sum within a timeout
or meet the confidence.
>>> rdd = sc.parallelize(range(1000), 10)
>>> r = sum(xrange(1000))
>>> (rdd.sumApprox(1000) - r) / r < 0.05
RDD.meanApprox(self, timeout, confidence=0.95)
:: Experimental ::
Approximate operation to return the mean within a timeout
or meet the confidence.
>>> rdd = sc.parallelize(range(1000), 10)
>>> r = sum(xrange(1000)) / 1000.0
>>> (rdd.meanApprox(1000) - r) / r < 0.05
True
Author: Davies Liu <davies.liu@gmail.com>
Closes #2095 from davies/approx and squashes the following commits:
e8c252b [Davies Liu] add approx API for RDD
2014-08-23 22:33:34 -04:00
|
|
|
>>> rdd = sc.parallelize(range(1000), 10)
|
2015-04-16 19:20:57 -04:00
|
|
|
>>> r = sum(range(1000)) / 1000.0
|
2015-06-23 02:37:56 -04:00
|
|
|
>>> abs(rdd.meanApprox(1000) - r) / r < 0.05
|
[SPARK-2871] [PySpark] add approx API for RDD
RDD.countApprox(self, timeout, confidence=0.95)
:: Experimental ::
Approximate version of count() that returns a potentially incomplete
result within a timeout, even if not all tasks have finished.
>>> rdd = sc.parallelize(range(1000), 10)
>>> rdd.countApprox(1000, 1.0)
1000
RDD.sumApprox(self, timeout, confidence=0.95)
Approximate operation to return the sum within a timeout
or meet the confidence.
>>> rdd = sc.parallelize(range(1000), 10)
>>> r = sum(xrange(1000))
>>> (rdd.sumApprox(1000) - r) / r < 0.05
RDD.meanApprox(self, timeout, confidence=0.95)
:: Experimental ::
Approximate operation to return the mean within a timeout
or meet the confidence.
>>> rdd = sc.parallelize(range(1000), 10)
>>> r = sum(xrange(1000)) / 1000.0
>>> (rdd.meanApprox(1000) - r) / r < 0.05
True
Author: Davies Liu <davies.liu@gmail.com>
Closes #2095 from davies/approx and squashes the following commits:
e8c252b [Davies Liu] add approx API for RDD
2014-08-23 22:33:34 -04:00
|
|
|
True
|
|
|
|
"""
|
2014-09-02 18:47:47 -04:00
|
|
|
jrdd = self.map(float)._to_java_object_rdd()
|
[SPARK-2871] [PySpark] add approx API for RDD
RDD.countApprox(self, timeout, confidence=0.95)
:: Experimental ::
Approximate version of count() that returns a potentially incomplete
result within a timeout, even if not all tasks have finished.
>>> rdd = sc.parallelize(range(1000), 10)
>>> rdd.countApprox(1000, 1.0)
1000
RDD.sumApprox(self, timeout, confidence=0.95)
Approximate operation to return the sum within a timeout
or meet the confidence.
>>> rdd = sc.parallelize(range(1000), 10)
>>> r = sum(xrange(1000))
>>> (rdd.sumApprox(1000) - r) / r < 0.05
RDD.meanApprox(self, timeout, confidence=0.95)
:: Experimental ::
Approximate operation to return the mean within a timeout
or meet the confidence.
>>> rdd = sc.parallelize(range(1000), 10)
>>> r = sum(xrange(1000)) / 1000.0
>>> (rdd.meanApprox(1000) - r) / r < 0.05
True
Author: Davies Liu <davies.liu@gmail.com>
Closes #2095 from davies/approx and squashes the following commits:
e8c252b [Davies Liu] add approx API for RDD
2014-08-23 22:33:34 -04:00
|
|
|
jdrdd = self.ctx._jvm.JavaDoubleRDD.fromRDD(jrdd.rdd())
|
|
|
|
r = jdrdd.meanApprox(timeout, confidence).getFinalValue()
|
|
|
|
return BoundedFloat(r.mean(), r.confidence(), r.low(), r.high())
|
|
|
|
|
2014-09-02 18:47:47 -04:00
|
|
|
def countApproxDistinct(self, relativeSD=0.05):
|
|
|
|
"""
|
|
|
|
Return approximate number of distinct elements in the RDD.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
relativeSD : float, optional
|
|
|
|
Relative accuracy. Smaller values create
|
|
|
|
counters that require more space.
|
|
|
|
It must be greater than 0.000017.
|
|
|
|
|
|
|
|
Notes
|
|
|
|
-----
|
2014-09-02 18:47:47 -04:00
|
|
|
The algorithm used is based on streamlib's implementation of
|
2016-05-09 04:11:17 -04:00
|
|
|
`"HyperLogLog in Practice: Algorithmic Engineering of a State
|
|
|
|
of The Art Cardinality Estimation Algorithm", available here
|
2018-11-25 18:43:55 -05:00
|
|
|
<https://doi.org/10.1145/2452376.2452456>`_.
|
2014-09-02 18:47:47 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2014-09-02 18:47:47 -04:00
|
|
|
>>> n = sc.parallelize(range(1000)).map(str).countApproxDistinct()
|
2015-04-16 19:20:57 -04:00
|
|
|
>>> 900 < n < 1100
|
2014-09-02 18:47:47 -04:00
|
|
|
True
|
|
|
|
>>> n = sc.parallelize([i % 20 for i in range(1000)]).countApproxDistinct()
|
2015-04-16 19:20:57 -04:00
|
|
|
>>> 16 < n < 24
|
2014-09-02 18:47:47 -04:00
|
|
|
True
|
|
|
|
"""
|
|
|
|
if relativeSD < 0.000017:
|
|
|
|
raise ValueError("relativeSD should be greater than 0.000017")
|
|
|
|
# the hash space in Java is 2^32
|
|
|
|
hashRDD = self.map(lambda x: portable_hash(x) & 0xFFFFFFFF)
|
|
|
|
return hashRDD._to_java_object_rdd().countApproxDistinct(relativeSD)
|
|
|
|
|
2019-09-20 12:59:31 -04:00
|
|
|
def toLocalIterator(self, prefetchPartitions=False):
|
2015-01-28 15:47:12 -05:00
|
|
|
"""
|
|
|
|
Return an iterator that contains all of the elements in this RDD.
|
|
|
|
The iterator will consume as much memory as the largest partition in this RDD.
|
2019-09-20 12:59:31 -04:00
|
|
|
With prefetch it may consume up to the memory of the 2 largest partitions.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
prefetchPartitions : bool, optional
|
|
|
|
If Spark should pre-fetch the next partition
|
|
|
|
before it is needed.
|
2016-04-04 16:31:44 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2015-01-28 15:47:12 -05:00
|
|
|
>>> rdd = sc.parallelize(range(10))
|
|
|
|
>>> [x for x in rdd.toLocalIterator()]
|
|
|
|
[0, 1, 2, 3, 4, 5, 6, 7, 8, 9]
|
|
|
|
"""
|
2016-04-04 16:31:44 -04:00
|
|
|
with SCCallSiteSync(self.context) as css:
|
2019-09-20 12:59:31 -04:00
|
|
|
sock_info = self.ctx._jvm.PythonRDD.toLocalIteratorAndServe(
|
|
|
|
self._jrdd.rdd(),
|
|
|
|
prefetchPartitions)
|
[SPARK-23961][SPARK-27548][PYTHON] Fix error when toLocalIterator goes out of scope and properly raise errors from worker
## What changes were proposed in this pull request?
This fixes an error when a PySpark local iterator, for both RDD and DataFrames, goes out of scope and the connection is closed before fully consuming the iterator. The error occurs on the JVM in the serving thread, when Python closes the local socket while the JVM is writing to it. This usually happens when there is enough data to fill the socket read buffer, causing the write call to block.
Additionally, this fixes a problem when an error occurs in the Python worker and the collect job is cancelled with an exception. Previously, the Python driver was never notified of the error so the user could get a partial result (iteration until the error) and the application will continue. With this change, an error in the worker is sent to the Python iterator and is then raised.
The change here introduces a protocol for PySpark local iterators that work as follows:
1) The local socket connection is made when the iterator is created
2) When iterating, Python first sends a request for partition data as a non-zero integer
3) While the JVM local iterator over partitions has next, it triggers a job to collect the next partition
4) The JVM sends a nonzero response to indicate it has the next partition to send
5) The next partition is sent to Python and read by the PySpark deserializer
6) After sending the entire partition, an `END_OF_DATA_SECTION` is sent to Python which stops the deserializer and allows to make another request
7) When the JVM gets a request from Python but has already consumed it's local iterator, it will send a zero response to Python and both will close the socket cleanly
8) If an error occurs in the worker, a negative response is sent to Python followed by the error message. Python will then raise a RuntimeError with the message, stopping iteration.
9) When the PySpark local iterator is garbage-collected, it will read any remaining data from the current partition (this is data that has already been collected) and send a request of zero to tell the JVM to stop collection jobs and close the connection.
Steps 1, 3, 5, 6 are the same as before. Step 8 was completely missing before because errors in the worker were never communicated back to Python. The other steps add synchronization to allow for a clean closing of the socket, with a small trade-off in performance for each partition. This is mainly because the JVM does not start collecting partition data until it receives a request to do so, where before it would eagerly write all data until the socket receive buffer is full.
## How was this patch tested?
Added new unit tests for DataFrame and RDD `toLocalIterator` and tested not fully consuming the iterator. Manual tests with Python 2.7 and 3.6.
Closes #24070 from BryanCutler/pyspark-toLocalIterator-clean-stop-SPARK-23961.
Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-05-07 17:47:39 -04:00
|
|
|
return _local_iterator_from_socket(sock_info, self._jrdd_deserializer)
|
2015-01-28 15:47:12 -05:00
|
|
|
|
2018-08-11 09:44:45 -04:00
|
|
|
def barrier(self):
|
|
|
|
"""
|
2018-08-29 10:22:03 -04:00
|
|
|
Marks the current stage as a barrier stage, where Spark must launch all tasks together.
|
|
|
|
In case of a task failure, instead of only restarting the failed task, Spark will abort the
|
|
|
|
entire stage and relaunch all tasks for this stage.
|
|
|
|
The barrier execution mode feature is experimental and it only handles limited scenarios.
|
|
|
|
Please read the linked SPIP and design docs to understand the limitations and future plans.
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
.. versionadded:: 2.4.0
|
2018-08-29 10:22:03 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Returns
|
|
|
|
-------
|
|
|
|
:class:`RDDBarrier`
|
|
|
|
instance that provides actions within a barrier stage.
|
2018-08-11 09:44:45 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
See Also
|
|
|
|
--------
|
|
|
|
pyspark.BarrierTaskContext
|
|
|
|
|
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
For additional information see
|
|
|
|
|
|
|
|
- `SPIP: Barrier Execution Mode <http://jira.apache.org/jira/browse/SPARK-24374>`_
|
|
|
|
- `Design Doc <https://jira.apache.org/jira/browse/SPARK-24582>`_
|
|
|
|
|
|
|
|
This API is experimental
|
2018-08-11 09:44:45 -04:00
|
|
|
"""
|
|
|
|
return RDDBarrier(self)
|
|
|
|
|
|
|
|
def _is_barrier(self):
|
|
|
|
"""
|
|
|
|
Whether this RDD is in a barrier stage.
|
|
|
|
"""
|
|
|
|
return self._jrdd.rdd().isBarrier()
|
|
|
|
|
2020-04-22 21:20:39 -04:00
|
|
|
def withResources(self, profile):
|
|
|
|
"""
|
|
|
|
Specify a :class:`pyspark.resource.ResourceProfile` to use when calculating this RDD.
|
|
|
|
This is only supported on certain cluster managers and currently requires dynamic
|
|
|
|
allocation to be enabled. It will result in new executors with the resources specified
|
|
|
|
being acquired to calculate the RDD.
|
|
|
|
|
|
|
|
.. versionadded:: 3.1.0
|
2020-11-15 20:21:50 -05:00
|
|
|
|
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is experimental
|
2020-04-22 21:20:39 -04:00
|
|
|
"""
|
|
|
|
self.has_resource_profile = True
|
|
|
|
if profile._java_resource_profile is not None:
|
|
|
|
jrp = profile._java_resource_profile
|
|
|
|
else:
|
|
|
|
builder = self.ctx._jvm.org.apache.spark.resource.ResourceProfileBuilder()
|
|
|
|
ereqs = ExecutorResourceRequests(self.ctx._jvm, profile._executor_resource_requests)
|
|
|
|
treqs = TaskResourceRequests(self.ctx._jvm, profile._task_resource_requests)
|
|
|
|
builder.require(ereqs._java_executor_resource_requests)
|
|
|
|
builder.require(treqs._java_task_resource_requests)
|
|
|
|
jrp = builder.build()
|
|
|
|
|
|
|
|
self._jrdd.withResources(jrp)
|
|
|
|
return self
|
|
|
|
|
|
|
|
def getResourceProfile(self):
|
|
|
|
"""
|
|
|
|
Get the :class:`pyspark.resource.ResourceProfile` specified with this RDD or None
|
|
|
|
if it wasn't specified.
|
|
|
|
|
|
|
|
.. versionadded:: 3.1.0
|
2020-11-15 20:21:50 -05:00
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
|
|
|
:py:class:`pyspark.resource.ResourceProfile`
|
|
|
|
The the user specified profile or None if none were specified
|
|
|
|
|
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is experimental
|
2020-04-22 21:20:39 -04:00
|
|
|
"""
|
|
|
|
rp = self._jrdd.getResourceProfile()
|
|
|
|
if rp is not None:
|
|
|
|
return ResourceProfile(_java_resource_profile=rp)
|
|
|
|
else:
|
|
|
|
return None
|
|
|
|
|
2014-07-14 03:42:59 -04:00
|
|
|
|
2016-02-24 15:44:54 -05:00
|
|
|
def _prepare_for_python_RDD(sc, command):
|
2015-02-04 18:55:09 -05:00
|
|
|
# the serialized command will be compressed by broadcast
|
|
|
|
ser = CloudPickleSerializer()
|
2015-05-18 15:55:13 -04:00
|
|
|
pickled_command = ser.dumps(command)
|
2019-07-13 11:44:16 -04:00
|
|
|
if len(pickled_command) > sc._jvm.PythonUtils.getBroadcastThreshold(sc._jsc): # Default 1M
|
2015-04-15 15:58:02 -04:00
|
|
|
# The broadcast will have same life cycle as created PythonRDD
|
2015-02-04 18:55:09 -05:00
|
|
|
broadcast = sc.broadcast(pickled_command)
|
|
|
|
pickled_command = ser.dumps(broadcast)
|
2016-10-03 17:12:03 -04:00
|
|
|
broadcast_vars = [x._jbroadcast for x in sc._pickled_broadcast_vars]
|
2015-02-04 18:55:09 -05:00
|
|
|
sc._pickled_broadcast_vars.clear()
|
2016-10-03 17:12:03 -04:00
|
|
|
return pickled_command, broadcast_vars, sc.environment, sc._python_includes
|
2015-02-04 18:55:09 -05:00
|
|
|
|
|
|
|
|
2016-02-24 15:44:54 -05:00
|
|
|
def _wrap_function(sc, func, deserializer, serializer, profiler=None):
|
|
|
|
assert deserializer, "deserializer should not be empty"
|
|
|
|
assert serializer, "serializer should not be empty"
|
|
|
|
command = (func, profiler, deserializer, serializer)
|
|
|
|
pickled_command, broadcast_vars, env, includes = _prepare_for_python_RDD(sc, command)
|
|
|
|
return sc._jvm.PythonFunction(bytearray(pickled_command), env, includes, sc.pythonExec,
|
|
|
|
sc.pythonVer, broadcast_vars, sc._javaAccumulator)
|
|
|
|
|
|
|
|
|
2018-08-11 09:44:45 -04:00
|
|
|
class RDDBarrier(object):
|
|
|
|
|
|
|
|
"""
|
2018-08-29 10:22:03 -04:00
|
|
|
Wraps an RDD in a barrier stage, which forces Spark to launch tasks of this stage together.
|
|
|
|
:class:`RDDBarrier` instances are created by :func:`RDD.barrier`.
|
2018-08-11 09:44:45 -04:00
|
|
|
|
|
|
|
.. versionadded:: 2.4.0
|
2020-11-15 20:21:50 -05:00
|
|
|
|
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is experimental
|
2018-08-11 09:44:45 -04:00
|
|
|
"""
|
|
|
|
|
|
|
|
def __init__(self, rdd):
|
|
|
|
self.rdd = rdd
|
|
|
|
|
|
|
|
def mapPartitions(self, f, preservesPartitioning=False):
|
|
|
|
"""
|
2018-08-29 10:22:03 -04:00
|
|
|
Returns a new RDD by applying a function to each partition of the wrapped RDD,
|
|
|
|
where tasks are launched together in a barrier stage.
|
|
|
|
The interface is the same as :func:`RDD.mapPartitions`.
|
|
|
|
Please see the API doc there.
|
2018-08-11 09:44:45 -04:00
|
|
|
|
|
|
|
.. versionadded:: 2.4.0
|
2020-11-15 20:21:50 -05:00
|
|
|
|
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is experimental
|
2018-08-11 09:44:45 -04:00
|
|
|
"""
|
|
|
|
def func(s, iterator):
|
|
|
|
return f(iterator)
|
|
|
|
return PipelinedRDD(self.rdd, func, preservesPartitioning, isFromBarrier=True)
|
|
|
|
|
2019-10-23 07:46:09 -04:00
|
|
|
def mapPartitionsWithIndex(self, f, preservesPartitioning=False):
|
|
|
|
"""
|
|
|
|
Returns a new RDD by applying a function to each partition of the wrapped RDD, while
|
|
|
|
tracking the index of the original partition. And all tasks are launched together
|
|
|
|
in a barrier stage.
|
|
|
|
The interface is the same as :func:`RDD.mapPartitionsWithIndex`.
|
|
|
|
Please see the API doc there.
|
|
|
|
|
|
|
|
.. versionadded:: 3.0.0
|
2020-11-15 20:21:50 -05:00
|
|
|
|
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is experimental
|
2019-10-23 07:46:09 -04:00
|
|
|
"""
|
|
|
|
return PipelinedRDD(self.rdd, f, preservesPartitioning, isFromBarrier=True)
|
|
|
|
|
2018-08-11 09:44:45 -04:00
|
|
|
|
2012-08-24 22:38:50 -04:00
|
|
|
class PipelinedRDD(RDD):
|
2014-07-14 03:42:59 -04:00
|
|
|
|
2012-08-18 19:07:10 -04:00
|
|
|
"""
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2012-08-18 19:07:10 -04:00
|
|
|
Pipelined maps:
|
2014-08-06 17:12:21 -04:00
|
|
|
|
2012-08-18 19:07:10 -04:00
|
|
|
>>> rdd = sc.parallelize([1, 2, 3, 4])
|
|
|
|
>>> rdd.map(lambda x: 2 * x).cache().map(lambda x: 2 * x).collect()
|
|
|
|
[4, 8, 12, 16]
|
|
|
|
>>> rdd.map(lambda x: 2 * x).map(lambda x: 2 * x).collect()
|
|
|
|
[4, 8, 12, 16]
|
|
|
|
|
|
|
|
Pipelined reduces:
|
2020-11-15 20:21:50 -05:00
|
|
|
|
2012-08-18 19:07:10 -04:00
|
|
|
>>> from operator import add
|
|
|
|
>>> rdd.map(lambda x: 2 * x).reduce(add)
|
|
|
|
20
|
|
|
|
>>> rdd.flatMap(lambda x: [x, x]).reduce(add)
|
|
|
|
20
|
|
|
|
"""
|
2014-07-14 03:42:59 -04:00
|
|
|
|
2018-08-11 09:44:45 -04:00
|
|
|
def __init__(self, prev, func, preservesPartitioning=False, isFromBarrier=False):
|
2013-11-05 20:52:39 -05:00
|
|
|
if not isinstance(prev, PipelinedRDD) or not prev._is_pipelinable():
|
|
|
|
# This transformation is the first in its stage:
|
|
|
|
self.func = func
|
|
|
|
self.preservesPartitioning = preservesPartitioning
|
|
|
|
self._prev_jrdd = prev._jrdd
|
|
|
|
self._prev_jrdd_deserializer = prev._jrdd_deserializer
|
|
|
|
else:
|
2012-08-10 04:10:02 -04:00
|
|
|
prev_func = prev.func
|
2014-07-14 03:42:59 -04:00
|
|
|
|
2013-01-08 19:04:41 -05:00
|
|
|
def pipeline_func(split, iterator):
|
|
|
|
return func(split, prev_func(split, iterator))
|
2012-08-24 22:38:50 -04:00
|
|
|
self.func = pipeline_func
|
2012-08-10 04:10:02 -04:00
|
|
|
self.preservesPartitioning = \
|
|
|
|
prev.preservesPartitioning and preservesPartitioning
|
2013-11-05 20:52:39 -05:00
|
|
|
self._prev_jrdd = prev._prev_jrdd # maintain the pipeline
|
|
|
|
self._prev_jrdd_deserializer = prev._prev_jrdd_deserializer
|
2012-08-10 04:10:02 -04:00
|
|
|
self.is_cached = False
|
2020-04-22 21:20:39 -04:00
|
|
|
self.has_resource_profile = False
|
2013-01-20 16:59:45 -05:00
|
|
|
self.is_checkpointed = False
|
2012-08-10 04:10:02 -04:00
|
|
|
self.ctx = prev.ctx
|
|
|
|
self.prev = prev
|
|
|
|
self._jrdd_val = None
|
2014-09-06 19:12:29 -04:00
|
|
|
self._id = None
|
2013-11-05 20:52:39 -05:00
|
|
|
self._jrdd_deserializer = self.ctx.serializer
|
2012-08-25 19:46:07 -04:00
|
|
|
self._bypass_serializer = False
|
2015-02-17 19:54:57 -05:00
|
|
|
self.partitioner = prev.partitioner if self.preservesPartitioning else None
|
[SPARK-26776][PYTHON] Reduce Py4J communication cost in PySpark's execution barrier check
## What changes were proposed in this pull request?
I am investigating flaky tests. I realised that:
```
File "/home/jenkins/workspace/SparkPullRequestBuilder/python/pyspark/rdd.py", line 2512, in __init__
self.is_barrier = prev._is_barrier() or isFromBarrier
File "/home/jenkins/workspace/SparkPullRequestBuilder/python/pyspark/rdd.py", line 2412, in _is_barrier
return self._jrdd.rdd().isBarrier()
File "/home/jenkins/workspace/SparkPullRequestBuilder/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1286, in __call__
answer, self.gateway_client, self.target_id, self.name)
File "/home/jenkins/workspace/SparkPullRequestBuilder/python/lib/py4j-0.10.8.1-src.zip/py4j/protocol.py", line 342, in get_return_value
return OUTPUT_CONVERTER[type](answer[2:], gateway_client)
File "/home/jenkins/workspace/SparkPullRequestBuilder/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 2492, in <lambda>
lambda target_id, gateway_client: JavaObject(target_id, gateway_client))
File "/home/jenkins/workspace/SparkPullRequestBuilder/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1324, in __init__
ThreadSafeFinalizer.add_finalizer(key, value)
File "/home/jenkins/workspace/SparkPullRequestBuilder/python/lib/py4j-0.10.8.1-src.zip/py4j/finalizer.py", line 43, in add_finalizer
cls.finalizers[id] = weak_ref
File "/usr/lib64/pypy-2.5.1/lib-python/2.7/threading.py", line 216, in __exit__
self.release()
File "/usr/lib64/pypy-2.5.1/lib-python/2.7/threading.py", line 208, in release
self.__block.release()
error: release unlocked lock
```
I assume it might not be directly related with the test itself but I noticed that it `prev._is_barrier()` attempts to access via Py4J.
Accessing via Py4J is expensive. Therefore, this PR proposes to avoid Py4J access when `isFromBarrier` is `True`.
## How was this patch tested?
Unittests should cover this.
Closes #23690 from HyukjinKwon/minor-barrier.
Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-29 23:24:27 -05:00
|
|
|
self.is_barrier = isFromBarrier or prev._is_barrier()
|
2014-10-01 14:21:34 -04:00
|
|
|
|
2015-04-15 15:58:02 -04:00
|
|
|
def getNumPartitions(self):
|
|
|
|
return self._prev_jrdd.partitions().size()
|
2012-08-10 04:10:02 -04:00
|
|
|
|
|
|
|
@property
|
|
|
|
def _jrdd(self):
|
2012-08-25 19:46:07 -04:00
|
|
|
if self._jrdd_val:
|
|
|
|
return self._jrdd_val
|
2013-11-05 20:52:39 -05:00
|
|
|
if self._bypass_serializer:
|
2014-06-03 21:18:25 -04:00
|
|
|
self._jrdd_deserializer = NoOpSerializer()
|
2015-01-28 16:48:06 -05:00
|
|
|
|
|
|
|
if self.ctx.profiler_collector:
|
|
|
|
profiler = self.ctx.profiler_collector.new_profiler(self.ctx)
|
|
|
|
else:
|
|
|
|
profiler = None
|
|
|
|
|
2016-02-24 15:44:54 -05:00
|
|
|
wrapped_func = _wrap_function(self.ctx, self.func, self._prev_jrdd_deserializer,
|
|
|
|
self._jrdd_deserializer, profiler)
|
|
|
|
python_rdd = self.ctx._jvm.PythonRDD(self._prev_jrdd.rdd(), wrapped_func,
|
2018-08-11 09:44:45 -04:00
|
|
|
self.preservesPartitioning, self.is_barrier)
|
2012-08-25 19:46:07 -04:00
|
|
|
self._jrdd_val = python_rdd.asJavaRDD()
|
[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
|
|
|
|
2015-01-28 16:48:06 -05:00
|
|
|
if profiler:
|
[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
|
|
|
self._id = self._jrdd_val.id()
|
2015-01-28 16:48:06 -05:00
|
|
|
self.ctx.profiler_collector.add_profiler(self._id, profiler)
|
2012-08-10 04:10:02 -04:00
|
|
|
return self._jrdd_val
|
|
|
|
|
2014-09-06 19:12:29 -04:00
|
|
|
def id(self):
|
|
|
|
if self._id is None:
|
|
|
|
self._id = self._jrdd.id()
|
|
|
|
return self._id
|
|
|
|
|
2013-01-20 16:59:45 -05:00
|
|
|
def _is_pipelinable(self):
|
2020-04-22 21:20:39 -04:00
|
|
|
return not (self.is_cached or self.is_checkpointed or self.has_resource_profile)
|
2013-01-20 16:59:45 -05:00
|
|
|
|
2018-08-11 09:44:45 -04:00
|
|
|
def _is_barrier(self):
|
|
|
|
return self.is_barrier
|
|
|
|
|
2012-08-10 04:10:02 -04:00
|
|
|
|
|
|
|
def _test():
|
|
|
|
import doctest
|
|
|
|
from pyspark.context import SparkContext
|
|
|
|
globs = globals().copy()
|
2012-12-29 17:22:56 -05:00
|
|
|
# The small batch size here ensures that we see multiple batches,
|
|
|
|
# even in these small test examples:
|
2014-11-04 02:56:14 -05:00
|
|
|
globs['sc'] = SparkContext('local[4]', 'PythonTest')
|
2014-07-14 03:42:59 -04:00
|
|
|
(failure_count, test_count) = doctest.testmod(
|
|
|
|
globs=globs, optionflags=doctest.ELLIPSIS)
|
2012-08-10 04:10:02 -04:00
|
|
|
globs['sc'].stop()
|
2013-02-03 01:44:11 -05:00
|
|
|
if failure_count:
|
2018-03-08 06:38:34 -05:00
|
|
|
sys.exit(-1)
|
2012-08-10 04:10:02 -04:00
|
|
|
|
|
|
|
|
|
|
|
if __name__ == "__main__":
|
|
|
|
_test()
|