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.
|
|
|
|
#
|
|
|
|
|
2013-01-16 22:15:14 -05:00
|
|
|
"""
|
|
|
|
Unit tests for PySpark; additional tests are implemented as doctests in
|
|
|
|
individual modules.
|
|
|
|
"""
|
2015-04-16 19:20:57 -04:00
|
|
|
|
2014-07-30 16:19:05 -04:00
|
|
|
from array import array
|
2013-11-29 02:44:56 -05:00
|
|
|
from glob import glob
|
2013-01-16 22:15:14 -05:00
|
|
|
import os
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
import re
|
2013-01-16 22:15:14 -05:00
|
|
|
import shutil
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
import subprocess
|
2013-01-23 13:36:18 -05:00
|
|
|
import sys
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
import tempfile
|
2013-01-16 22:15:14 -05:00
|
|
|
import time
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
import zipfile
|
2014-08-26 19:57:40 -04:00
|
|
|
import random
|
2014-10-23 20:20:00 -04:00
|
|
|
import threading
|
2014-11-18 19:17:51 -05:00
|
|
|
import hashlib
|
2013-01-16 22:15:14 -05:00
|
|
|
|
2015-04-10 17:04:53 -04:00
|
|
|
from py4j.protocol import Py4JJavaError
|
2015-10-22 18:27:11 -04:00
|
|
|
try:
|
|
|
|
import xmlrunner
|
|
|
|
except ImportError:
|
|
|
|
xmlrunner = None
|
2015-04-10 17:04:53 -04:00
|
|
|
|
2014-08-11 14:54:09 -04:00
|
|
|
if sys.version_info[:2] <= (2, 6):
|
2014-10-11 14:26:17 -04:00
|
|
|
try:
|
|
|
|
import unittest2 as unittest
|
|
|
|
except ImportError:
|
|
|
|
sys.stderr.write('Please install unittest2 to test with Python 2.6 or earlier')
|
|
|
|
sys.exit(1)
|
2014-08-11 14:54:09 -04:00
|
|
|
else:
|
|
|
|
import unittest
|
2015-04-16 19:20:57 -04:00
|
|
|
if sys.version_info[0] >= 3:
|
|
|
|
xrange = range
|
|
|
|
basestring = str
|
2014-08-11 14:54:09 -04:00
|
|
|
|
2015-04-21 20:49:55 -04:00
|
|
|
if sys.version >= "3":
|
|
|
|
from io import StringIO
|
|
|
|
else:
|
|
|
|
from StringIO import StringIO
|
|
|
|
|
2014-08-11 14:54:09 -04:00
|
|
|
|
2017-03-03 19:43:45 -05:00
|
|
|
from pyspark import keyword_only
|
2014-08-26 19:57:40 -04:00
|
|
|
from pyspark.conf import SparkConf
|
2013-01-16 22:15:14 -05:00
|
|
|
from pyspark.context import SparkContext
|
[SPARK-5154] [PySpark] [Streaming] Kafka streaming support in Python
This PR brings the Python API for Spark Streaming Kafka data source.
```
class KafkaUtils(__builtin__.object)
| Static methods defined here:
|
| createStream(ssc, zkQuorum, groupId, topics, storageLevel=StorageLevel(True, True, False, False,
2), keyDecoder=<function utf8_decoder>, valueDecoder=<function utf8_decoder>)
| Create an input stream that pulls messages from a Kafka Broker.
|
| :param ssc: StreamingContext object
| :param zkQuorum: Zookeeper quorum (hostname:port,hostname:port,..).
| :param groupId: The group id for this consumer.
| :param topics: Dict of (topic_name -> numPartitions) to consume.
| Each partition is consumed in its own thread.
| :param storageLevel: RDD storage level.
| :param keyDecoder: A function used to decode key
| :param valueDecoder: A function used to decode value
| :return: A DStream object
```
run the example:
```
bin/spark-submit --driver-class-path external/kafka-assembly/target/scala-*/spark-streaming-kafka-assembly-*.jar examples/src/main/python/streaming/kafka_wordcount.py localhost:2181 test
```
Author: Davies Liu <davies@databricks.com>
Author: Tathagata Das <tdas@databricks.com>
Closes #3715 from davies/kafka and squashes the following commits:
d93bfe0 [Davies Liu] Update make-distribution.sh
4280d04 [Davies Liu] address comments
e6d0427 [Davies Liu] Merge branch 'master' of github.com:apache/spark into kafka
f257071 [Davies Liu] add tests for null in RDD
23b039a [Davies Liu] address comments
9af51c4 [Davies Liu] Merge branch 'kafka' of github.com:davies/spark into kafka
a74da87 [Davies Liu] address comments
dc1eed0 [Davies Liu] Update kafka_wordcount.py
31e2317 [Davies Liu] Update kafka_wordcount.py
370ba61 [Davies Liu] Update kafka.py
97386b3 [Davies Liu] address comment
2c567a5 [Davies Liu] update logging and comment
33730d1 [Davies Liu] Merge branch 'master' of github.com:apache/spark into kafka
adeeb38 [Davies Liu] Merge pull request #3 from tdas/kafka-python-api
aea8953 [Tathagata Das] Kafka-assembly for Python API
eea16a7 [Davies Liu] refactor
f6ce899 [Davies Liu] add example and fix bugs
98c8d17 [Davies Liu] fix python style
5697a01 [Davies Liu] bypass decoder in scala
048dbe6 [Davies Liu] fix python style
75d485e [Davies Liu] add mqtt
07923c4 [Davies Liu] support kafka in Python
2015-02-02 22:16:27 -05:00
|
|
|
from pyspark.rdd import RDD
|
2013-01-23 13:36:18 -05:00
|
|
|
from pyspark.files import SparkFiles
|
2014-09-12 21:42:50 -04:00
|
|
|
from pyspark.serializers import read_int, BatchedSerializer, MarshalSerializer, PickleSerializer, \
|
2015-04-16 19:20:57 -04:00
|
|
|
CloudPickleSerializer, CompressedSerializer, UTF8Deserializer, NoOpSerializer, \
|
|
|
|
PairDeserializer, CartesianDeserializer, AutoBatchedSerializer, AutoSerializer, \
|
|
|
|
FlattenedValuesSerializer
|
2015-09-20 00:40:21 -04:00
|
|
|
from pyspark.shuffle import Aggregator, ExternalMerger, ExternalSorter
|
2014-09-14 01:31:21 -04:00
|
|
|
from pyspark import shuffle
|
2015-01-28 16:48:06 -05:00
|
|
|
from pyspark.profiler import BasicProfiler
|
2016-12-20 18:51:21 -05:00
|
|
|
from pyspark.taskcontext import TaskContext
|
2013-01-16 22:15:14 -05:00
|
|
|
|
2014-05-31 17:59:09 -04:00
|
|
|
_have_scipy = False
|
StatCounter on NumPy arrays [PYSPARK][SPARK-2012]
These changes allow StatCounters to work properly on NumPy arrays, to fix the issue reported here (https://issues.apache.org/jira/browse/SPARK-2012).
If NumPy is installed, the NumPy functions ``maximum``, ``minimum``, and ``sqrt``, which work on arrays, are used to merge statistics. If not, we fall back on scalar operators, so it will work on arrays with NumPy, but will also work without NumPy.
New unit tests added, along with a check for NumPy in the tests.
Author: Jeremy Freeman <the.freeman.lab@gmail.com>
Closes #1725 from freeman-lab/numpy-max-statcounter and squashes the following commits:
fe973b1 [Jeremy Freeman] Avoid duplicate array import in tests
7f0e397 [Jeremy Freeman] Refactored check for numpy
8e764dd [Jeremy Freeman] Explicit numpy imports
875414c [Jeremy Freeman] Fixed indents
1c8a832 [Jeremy Freeman] Unit tests for StatCounter with NumPy arrays
176a127 [Jeremy Freeman] Use numpy arrays in StatCounter
2014-08-02 01:33:25 -04:00
|
|
|
_have_numpy = False
|
2014-05-31 17:59:09 -04:00
|
|
|
try:
|
|
|
|
import scipy.sparse
|
|
|
|
_have_scipy = True
|
|
|
|
except:
|
|
|
|
# No SciPy, but that's okay, we'll skip those tests
|
|
|
|
pass
|
StatCounter on NumPy arrays [PYSPARK][SPARK-2012]
These changes allow StatCounters to work properly on NumPy arrays, to fix the issue reported here (https://issues.apache.org/jira/browse/SPARK-2012).
If NumPy is installed, the NumPy functions ``maximum``, ``minimum``, and ``sqrt``, which work on arrays, are used to merge statistics. If not, we fall back on scalar operators, so it will work on arrays with NumPy, but will also work without NumPy.
New unit tests added, along with a check for NumPy in the tests.
Author: Jeremy Freeman <the.freeman.lab@gmail.com>
Closes #1725 from freeman-lab/numpy-max-statcounter and squashes the following commits:
fe973b1 [Jeremy Freeman] Avoid duplicate array import in tests
7f0e397 [Jeremy Freeman] Refactored check for numpy
8e764dd [Jeremy Freeman] Explicit numpy imports
875414c [Jeremy Freeman] Fixed indents
1c8a832 [Jeremy Freeman] Unit tests for StatCounter with NumPy arrays
176a127 [Jeremy Freeman] Use numpy arrays in StatCounter
2014-08-02 01:33:25 -04:00
|
|
|
try:
|
|
|
|
import numpy as np
|
|
|
|
_have_numpy = True
|
|
|
|
except:
|
|
|
|
# No NumPy, but that's okay, we'll skip those tests
|
|
|
|
pass
|
2014-05-31 17:59:09 -04:00
|
|
|
|
2013-01-16 22:15:14 -05:00
|
|
|
|
2014-04-30 02:24:34 -04:00
|
|
|
SPARK_HOME = os.environ["SPARK_HOME"]
|
|
|
|
|
|
|
|
|
2014-10-06 17:07:53 -04:00
|
|
|
class MergerTests(unittest.TestCase):
|
2014-07-25 01:53:47 -04:00
|
|
|
|
|
|
|
def setUp(self):
|
[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
|
|
|
self.N = 1 << 12
|
2014-07-25 01:53:47 -04:00
|
|
|
self.l = [i for i in xrange(self.N)]
|
2015-04-16 19:20:57 -04:00
|
|
|
self.data = list(zip(self.l, self.l))
|
2014-08-06 15:58:24 -04:00
|
|
|
self.agg = Aggregator(lambda x: [x],
|
|
|
|
lambda x, y: x.append(y) or x,
|
|
|
|
lambda x, y: x.extend(y) or x)
|
2014-07-25 01:53:47 -04:00
|
|
|
|
|
|
|
def test_small_dataset(self):
|
|
|
|
m = ExternalMerger(self.agg, 1000)
|
|
|
|
m.mergeValues(self.data)
|
|
|
|
self.assertEqual(m.spills, 0)
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual(sum(sum(v) for k, v in m.items()),
|
2014-08-06 15:58:24 -04:00
|
|
|
sum(xrange(self.N)))
|
2014-07-25 01:53:47 -04:00
|
|
|
|
|
|
|
m = ExternalMerger(self.agg, 1000)
|
2015-04-16 19:20:57 -04:00
|
|
|
m.mergeCombiners(map(lambda x_y1: (x_y1[0], [x_y1[1]]), self.data))
|
2014-07-25 01:53:47 -04:00
|
|
|
self.assertEqual(m.spills, 0)
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual(sum(sum(v) for k, v in m.items()),
|
2014-08-06 15:58:24 -04:00
|
|
|
sum(xrange(self.N)))
|
2014-07-25 01:53:47 -04:00
|
|
|
|
|
|
|
def test_medium_dataset(self):
|
2015-04-16 19:20:57 -04:00
|
|
|
m = ExternalMerger(self.agg, 20)
|
2014-07-25 01:53:47 -04:00
|
|
|
m.mergeValues(self.data)
|
|
|
|
self.assertTrue(m.spills >= 1)
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual(sum(sum(v) for k, v in m.items()),
|
2014-08-06 15:58:24 -04:00
|
|
|
sum(xrange(self.N)))
|
2014-07-25 01:53:47 -04:00
|
|
|
|
|
|
|
m = ExternalMerger(self.agg, 10)
|
2015-04-16 19:20:57 -04:00
|
|
|
m.mergeCombiners(map(lambda x_y2: (x_y2[0], [x_y2[1]]), self.data * 3))
|
2014-07-25 01:53:47 -04:00
|
|
|
self.assertTrue(m.spills >= 1)
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual(sum(sum(v) for k, v in m.items()),
|
2014-08-06 15:58:24 -04:00
|
|
|
sum(xrange(self.N)) * 3)
|
2014-07-25 01:53:47 -04:00
|
|
|
|
|
|
|
def test_huge_dataset(self):
|
2015-04-16 19:20:57 -04:00
|
|
|
m = ExternalMerger(self.agg, 5, partitions=3)
|
|
|
|
m.mergeCombiners(map(lambda k_v: (k_v[0], [str(k_v[1])]), self.data * 10))
|
2014-07-25 01:53:47 -04:00
|
|
|
self.assertTrue(m.spills >= 1)
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual(sum(len(v) for k, v in m.items()),
|
2014-08-06 15:58:24 -04:00
|
|
|
self.N * 10)
|
2014-07-25 01:53:47 -04:00
|
|
|
m._cleanup()
|
|
|
|
|
[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 test_group_by_key(self):
|
|
|
|
|
|
|
|
def gen_data(N, step):
|
|
|
|
for i in range(1, N + 1, step):
|
|
|
|
for j in range(i):
|
|
|
|
yield (i, [j])
|
|
|
|
|
|
|
|
def gen_gs(N, step=1):
|
|
|
|
return shuffle.GroupByKey(gen_data(N, step))
|
|
|
|
|
|
|
|
self.assertEqual(1, len(list(gen_gs(1))))
|
|
|
|
self.assertEqual(2, len(list(gen_gs(2))))
|
|
|
|
self.assertEqual(100, len(list(gen_gs(100))))
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual(list(range(1, 101)), [k for k, _ in gen_gs(100)])
|
|
|
|
self.assertTrue(all(list(range(k)) == list(vs) for k, vs in gen_gs(100)))
|
[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
|
|
|
|
|
|
|
for k, vs in gen_gs(50002, 10000):
|
|
|
|
self.assertEqual(k, len(vs))
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual(list(range(k)), list(vs))
|
[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
|
|
|
|
|
|
|
ser = PickleSerializer()
|
|
|
|
l = ser.loads(ser.dumps(list(gen_gs(50002, 30000))))
|
|
|
|
for k, vs in l:
|
|
|
|
self.assertEqual(k, len(vs))
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual(list(range(k)), list(vs))
|
[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
|
|
|
|
2018-05-30 06:11:33 -04:00
|
|
|
def test_stopiteration_is_raised(self):
|
|
|
|
|
|
|
|
def stopit(*args, **kwargs):
|
|
|
|
raise StopIteration()
|
|
|
|
|
|
|
|
def legit_create_combiner(x):
|
|
|
|
return [x]
|
|
|
|
|
|
|
|
def legit_merge_value(x, y):
|
|
|
|
return x.append(y) or x
|
|
|
|
|
|
|
|
def legit_merge_combiners(x, y):
|
|
|
|
return x.extend(y) or x
|
|
|
|
|
|
|
|
data = [(x % 2, x) for x in range(100)]
|
|
|
|
|
|
|
|
# wrong create combiner
|
|
|
|
m = ExternalMerger(Aggregator(stopit, legit_merge_value, legit_merge_combiners), 20)
|
|
|
|
with self.assertRaises((Py4JJavaError, RuntimeError)) as cm:
|
|
|
|
m.mergeValues(data)
|
|
|
|
|
|
|
|
# wrong merge value
|
|
|
|
m = ExternalMerger(Aggregator(legit_create_combiner, stopit, legit_merge_combiners), 20)
|
|
|
|
with self.assertRaises((Py4JJavaError, RuntimeError)) as cm:
|
|
|
|
m.mergeValues(data)
|
|
|
|
|
|
|
|
# wrong merge combiners
|
|
|
|
m = ExternalMerger(Aggregator(legit_create_combiner, legit_merge_value, stopit), 20)
|
|
|
|
with self.assertRaises((Py4JJavaError, RuntimeError)) as cm:
|
|
|
|
m.mergeCombiners(map(lambda x_y1: (x_y1[0], [x_y1[1]]), data))
|
|
|
|
|
2014-07-25 01:53:47 -04:00
|
|
|
|
2014-10-06 17:07:53 -04:00
|
|
|
class SorterTests(unittest.TestCase):
|
2014-08-26 19:57:40 -04:00
|
|
|
def test_in_memory_sort(self):
|
2015-04-16 19:20:57 -04:00
|
|
|
l = list(range(1024))
|
2014-08-26 19:57:40 -04:00
|
|
|
random.shuffle(l)
|
|
|
|
sorter = ExternalSorter(1024)
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual(sorted(l), list(sorter.sorted(l)))
|
|
|
|
self.assertEqual(sorted(l, reverse=True), list(sorter.sorted(l, reverse=True)))
|
|
|
|
self.assertEqual(sorted(l, key=lambda x: -x), list(sorter.sorted(l, key=lambda x: -x)))
|
|
|
|
self.assertEqual(sorted(l, key=lambda x: -x, reverse=True),
|
|
|
|
list(sorter.sorted(l, key=lambda x: -x, reverse=True)))
|
2014-08-26 19:57:40 -04:00
|
|
|
|
|
|
|
def test_external_sort(self):
|
2015-06-18 16:45:58 -04:00
|
|
|
class CustomizedSorter(ExternalSorter):
|
|
|
|
def _next_limit(self):
|
|
|
|
return self.memory_limit
|
2015-04-16 19:20:57 -04:00
|
|
|
l = list(range(1024))
|
2014-08-26 19:57:40 -04:00
|
|
|
random.shuffle(l)
|
2015-06-18 16:45:58 -04:00
|
|
|
sorter = CustomizedSorter(1)
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual(sorted(l), list(sorter.sorted(l)))
|
2014-09-14 01:31:21 -04:00
|
|
|
self.assertGreater(shuffle.DiskBytesSpilled, 0)
|
|
|
|
last = shuffle.DiskBytesSpilled
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual(sorted(l, reverse=True), list(sorter.sorted(l, reverse=True)))
|
2014-09-14 01:31:21 -04:00
|
|
|
self.assertGreater(shuffle.DiskBytesSpilled, last)
|
|
|
|
last = shuffle.DiskBytesSpilled
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual(sorted(l, key=lambda x: -x), list(sorter.sorted(l, key=lambda x: -x)))
|
2014-09-14 01:31:21 -04:00
|
|
|
self.assertGreater(shuffle.DiskBytesSpilled, last)
|
|
|
|
last = shuffle.DiskBytesSpilled
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual(sorted(l, key=lambda x: -x, reverse=True),
|
|
|
|
list(sorter.sorted(l, key=lambda x: -x, reverse=True)))
|
2014-09-14 01:31:21 -04:00
|
|
|
self.assertGreater(shuffle.DiskBytesSpilled, last)
|
2014-08-26 19:57:40 -04:00
|
|
|
|
|
|
|
def test_external_sort_in_rdd(self):
|
|
|
|
conf = SparkConf().set("spark.python.worker.memory", "1m")
|
|
|
|
sc = SparkContext(conf=conf)
|
2015-04-16 19:20:57 -04:00
|
|
|
l = list(range(10240))
|
2014-08-26 19:57:40 -04:00
|
|
|
random.shuffle(l)
|
2015-04-21 20:49:55 -04:00
|
|
|
rdd = sc.parallelize(l, 4)
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual(sorted(l), rdd.sortBy(lambda x: x).collect())
|
2014-08-26 19:57:40 -04:00
|
|
|
sc.stop()
|
|
|
|
|
|
|
|
|
2014-08-04 15:13:41 -04:00
|
|
|
class SerializationTestCase(unittest.TestCase):
|
|
|
|
|
|
|
|
def test_namedtuple(self):
|
|
|
|
from collections import namedtuple
|
2015-04-16 19:20:57 -04:00
|
|
|
from pickle import dumps, loads
|
2014-08-04 15:13:41 -04:00
|
|
|
P = namedtuple("P", "x y")
|
|
|
|
p1 = P(1, 3)
|
|
|
|
p2 = loads(dumps(p1, 2))
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual(p1, p2)
|
2014-08-04 15:13:41 -04:00
|
|
|
|
2015-09-14 22:46:34 -04:00
|
|
|
from pyspark.cloudpickle import dumps
|
|
|
|
P2 = loads(dumps(P))
|
|
|
|
p3 = P2(1, 3)
|
|
|
|
self.assertEqual(p1, p3)
|
|
|
|
|
2014-09-12 21:42:50 -04:00
|
|
|
def test_itemgetter(self):
|
|
|
|
from operator import itemgetter
|
|
|
|
ser = CloudPickleSerializer()
|
|
|
|
d = range(10)
|
|
|
|
getter = itemgetter(1)
|
|
|
|
getter2 = ser.loads(ser.dumps(getter))
|
|
|
|
self.assertEqual(getter(d), getter2(d))
|
|
|
|
|
|
|
|
getter = itemgetter(0, 3)
|
|
|
|
getter2 = ser.loads(ser.dumps(getter))
|
|
|
|
self.assertEqual(getter(d), getter2(d))
|
|
|
|
|
[SPARK-13697] [PYSPARK] Fix the missing module name of TransformFunctionSerializer.loads
## What changes were proposed in this pull request?
Set the function's module name to `__main__` if it's missing in `TransformFunctionSerializer.loads`.
## How was this patch tested?
Manually test in the shell.
Before this patch:
```
>>> from pyspark.streaming import StreamingContext
>>> from pyspark.streaming.util import TransformFunction
>>> ssc = StreamingContext(sc, 1)
>>> func = TransformFunction(sc, lambda x: x, sc.serializer)
>>> func.rdd_wrapper(lambda x: x)
TransformFunction(<function <lambda> at 0x106ac8b18>)
>>> bytes = bytearray(ssc._transformerSerializer.serializer.dumps((func.func, func.rdd_wrap_func, func.deserializers)))
>>> func2 = ssc._transformerSerializer.loads(bytes)
>>> print(func2.func.__module__)
None
>>> print(func2.rdd_wrap_func.__module__)
None
>>>
```
After this patch:
```
>>> from pyspark.streaming import StreamingContext
>>> from pyspark.streaming.util import TransformFunction
>>> ssc = StreamingContext(sc, 1)
>>> func = TransformFunction(sc, lambda x: x, sc.serializer)
>>> func.rdd_wrapper(lambda x: x)
TransformFunction(<function <lambda> at 0x108bf1b90>)
>>> bytes = bytearray(ssc._transformerSerializer.serializer.dumps((func.func, func.rdd_wrap_func, func.deserializers)))
>>> func2 = ssc._transformerSerializer.loads(bytes)
>>> print(func2.func.__module__)
__main__
>>> print(func2.rdd_wrap_func.__module__)
__main__
>>>
```
Author: Shixiong Zhu <shixiong@databricks.com>
Closes #11535 from zsxwing/loads-module.
2016-03-06 11:57:01 -05:00
|
|
|
def test_function_module_name(self):
|
|
|
|
ser = CloudPickleSerializer()
|
|
|
|
func = lambda x: x
|
|
|
|
func2 = ser.loads(ser.dumps(func))
|
|
|
|
self.assertEqual(func.__module__, func2.__module__)
|
|
|
|
|
2014-09-12 21:42:50 -04:00
|
|
|
def test_attrgetter(self):
|
|
|
|
from operator import attrgetter
|
|
|
|
ser = CloudPickleSerializer()
|
|
|
|
|
|
|
|
class C(object):
|
|
|
|
def __getattr__(self, item):
|
|
|
|
return item
|
|
|
|
d = C()
|
|
|
|
getter = attrgetter("a")
|
|
|
|
getter2 = ser.loads(ser.dumps(getter))
|
|
|
|
self.assertEqual(getter(d), getter2(d))
|
|
|
|
getter = attrgetter("a", "b")
|
|
|
|
getter2 = ser.loads(ser.dumps(getter))
|
|
|
|
self.assertEqual(getter(d), getter2(d))
|
|
|
|
|
|
|
|
d.e = C()
|
|
|
|
getter = attrgetter("e.a")
|
|
|
|
getter2 = ser.loads(ser.dumps(getter))
|
|
|
|
self.assertEqual(getter(d), getter2(d))
|
|
|
|
getter = attrgetter("e.a", "e.b")
|
|
|
|
getter2 = ser.loads(ser.dumps(getter))
|
|
|
|
self.assertEqual(getter(d), getter2(d))
|
|
|
|
|
|
|
|
# Regression test for SPARK-3415
|
2014-09-07 21:54:36 -04:00
|
|
|
def test_pickling_file_handles(self):
|
2015-10-22 18:27:11 -04:00
|
|
|
# to be corrected with SPARK-11160
|
|
|
|
if not xmlrunner:
|
|
|
|
ser = CloudPickleSerializer()
|
|
|
|
out1 = sys.stderr
|
|
|
|
out2 = ser.loads(ser.dumps(out1))
|
|
|
|
self.assertEqual(out1, out2)
|
2014-09-07 21:54:36 -04:00
|
|
|
|
2014-09-24 16:00:05 -04:00
|
|
|
def test_func_globals(self):
|
|
|
|
|
|
|
|
class Unpicklable(object):
|
|
|
|
def __reduce__(self):
|
|
|
|
raise Exception("not picklable")
|
|
|
|
|
|
|
|
global exit
|
|
|
|
exit = Unpicklable()
|
|
|
|
|
|
|
|
ser = CloudPickleSerializer()
|
|
|
|
self.assertRaises(Exception, lambda: ser.dumps(exit))
|
|
|
|
|
|
|
|
def foo():
|
|
|
|
sys.exit(0)
|
|
|
|
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertTrue("exit" in foo.__code__.co_names)
|
2014-09-24 16:00:05 -04:00
|
|
|
ser.dumps(foo)
|
|
|
|
|
2014-11-24 20:17:03 -05:00
|
|
|
def test_compressed_serializer(self):
|
|
|
|
ser = CompressedSerializer(PickleSerializer())
|
2015-04-16 19:20:57 -04:00
|
|
|
try:
|
|
|
|
from StringIO import StringIO
|
|
|
|
except ImportError:
|
|
|
|
from io import BytesIO as StringIO
|
2014-11-18 19:17:51 -05:00
|
|
|
io = StringIO()
|
|
|
|
ser.dump_stream(["abc", u"123", range(5)], io)
|
|
|
|
io.seek(0)
|
|
|
|
self.assertEqual(["abc", u"123", range(5)], list(ser.load_stream(io)))
|
|
|
|
ser.dump_stream(range(1000), io)
|
|
|
|
io.seek(0)
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual(["abc", u"123", range(5)] + list(range(1000)), list(ser.load_stream(io)))
|
|
|
|
io.close()
|
|
|
|
|
|
|
|
def test_hash_serializer(self):
|
|
|
|
hash(NoOpSerializer())
|
|
|
|
hash(UTF8Deserializer())
|
|
|
|
hash(PickleSerializer())
|
|
|
|
hash(MarshalSerializer())
|
|
|
|
hash(AutoSerializer())
|
|
|
|
hash(BatchedSerializer(PickleSerializer()))
|
|
|
|
hash(AutoBatchedSerializer(MarshalSerializer()))
|
|
|
|
hash(PairDeserializer(NoOpSerializer(), UTF8Deserializer()))
|
|
|
|
hash(CartesianDeserializer(NoOpSerializer(), UTF8Deserializer()))
|
|
|
|
hash(CompressedSerializer(PickleSerializer()))
|
|
|
|
hash(FlattenedValuesSerializer(PickleSerializer()))
|
2014-11-18 19:17:51 -05:00
|
|
|
|
2014-09-07 21:54:36 -04:00
|
|
|
|
2015-04-21 20:49:55 -04:00
|
|
|
class QuietTest(object):
|
|
|
|
def __init__(self, sc):
|
|
|
|
self.log4j = sc._jvm.org.apache.log4j
|
|
|
|
|
|
|
|
def __enter__(self):
|
|
|
|
self.old_level = self.log4j.LogManager.getRootLogger().getLevel()
|
|
|
|
self.log4j.LogManager.getRootLogger().setLevel(self.log4j.Level.FATAL)
|
|
|
|
|
|
|
|
def __exit__(self, exc_type, exc_val, exc_tb):
|
|
|
|
self.log4j.LogManager.getRootLogger().setLevel(self.old_level)
|
|
|
|
|
|
|
|
|
2013-01-22 20:54:11 -05:00
|
|
|
class PySparkTestCase(unittest.TestCase):
|
2013-01-16 22:15:14 -05:00
|
|
|
|
|
|
|
def setUp(self):
|
2013-01-23 13:36:18 -05:00
|
|
|
self._old_sys_path = list(sys.path)
|
2013-01-22 20:54:11 -05:00
|
|
|
class_name = self.__class__.__name__
|
2014-11-04 02:56:14 -05:00
|
|
|
self.sc = SparkContext('local[4]', class_name)
|
2013-01-16 22:15:14 -05:00
|
|
|
|
|
|
|
def tearDown(self):
|
|
|
|
self.sc.stop()
|
2013-01-23 13:36:18 -05:00
|
|
|
sys.path = self._old_sys_path
|
2013-01-22 20:54:11 -05:00
|
|
|
|
2014-07-22 01:30:53 -04:00
|
|
|
|
2014-10-06 17:07:53 -04:00
|
|
|
class ReusedPySparkTestCase(unittest.TestCase):
|
|
|
|
|
|
|
|
@classmethod
|
|
|
|
def setUpClass(cls):
|
2014-11-04 02:56:14 -05:00
|
|
|
cls.sc = SparkContext('local[4]', cls.__name__)
|
2014-10-06 17:07:53 -04:00
|
|
|
|
|
|
|
@classmethod
|
|
|
|
def tearDownClass(cls):
|
|
|
|
cls.sc.stop()
|
|
|
|
|
|
|
|
|
|
|
|
class CheckpointTests(ReusedPySparkTestCase):
|
2013-01-22 20:54:11 -05:00
|
|
|
|
|
|
|
def setUp(self):
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
self.checkpointDir = tempfile.NamedTemporaryFile(delete=False)
|
2013-01-22 20:54:11 -05:00
|
|
|
os.unlink(self.checkpointDir.name)
|
|
|
|
self.sc.setCheckpointDir(self.checkpointDir.name)
|
|
|
|
|
|
|
|
def tearDown(self):
|
2013-01-20 18:38:11 -05:00
|
|
|
shutil.rmtree(self.checkpointDir.name)
|
2013-01-16 22:15:14 -05:00
|
|
|
|
|
|
|
def test_basic_checkpointing(self):
|
|
|
|
parCollection = self.sc.parallelize([1, 2, 3, 4])
|
|
|
|
flatMappedRDD = parCollection.flatMap(lambda x: range(1, x + 1))
|
|
|
|
|
|
|
|
self.assertFalse(flatMappedRDD.isCheckpointed())
|
2013-08-14 18:12:12 -04:00
|
|
|
self.assertTrue(flatMappedRDD.getCheckpointFile() is None)
|
2013-01-16 22:15:14 -05:00
|
|
|
|
|
|
|
flatMappedRDD.checkpoint()
|
|
|
|
result = flatMappedRDD.collect()
|
|
|
|
time.sleep(1) # 1 second
|
|
|
|
self.assertTrue(flatMappedRDD.isCheckpointed())
|
|
|
|
self.assertEqual(flatMappedRDD.collect(), result)
|
2013-12-24 17:01:13 -05:00
|
|
|
self.assertEqual("file:" + self.checkpointDir.name,
|
|
|
|
os.path.dirname(os.path.dirname(flatMappedRDD.getCheckpointFile())))
|
2013-01-16 22:15:14 -05:00
|
|
|
|
2013-01-20 16:59:45 -05:00
|
|
|
def test_checkpoint_and_restore(self):
|
|
|
|
parCollection = self.sc.parallelize([1, 2, 3, 4])
|
|
|
|
flatMappedRDD = parCollection.flatMap(lambda x: [x])
|
|
|
|
|
|
|
|
self.assertFalse(flatMappedRDD.isCheckpointed())
|
2013-08-14 18:12:12 -04:00
|
|
|
self.assertTrue(flatMappedRDD.getCheckpointFile() is None)
|
2013-01-20 16:59:45 -05:00
|
|
|
|
|
|
|
flatMappedRDD.checkpoint()
|
|
|
|
flatMappedRDD.count() # forces a checkpoint to be computed
|
|
|
|
time.sleep(1) # 1 second
|
|
|
|
|
2013-08-14 18:12:12 -04:00
|
|
|
self.assertTrue(flatMappedRDD.getCheckpointFile() is not None)
|
2013-11-05 20:52:39 -05:00
|
|
|
recovered = self.sc._checkpointFile(flatMappedRDD.getCheckpointFile(),
|
|
|
|
flatMappedRDD._jrdd_deserializer)
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual([1, 2, 3, 4], recovered.collect())
|
2013-01-20 16:59:45 -05:00
|
|
|
|
2013-01-16 22:15:14 -05:00
|
|
|
|
2016-11-21 16:08:34 -05:00
|
|
|
class LocalCheckpointTests(ReusedPySparkTestCase):
|
|
|
|
|
|
|
|
def test_basic_localcheckpointing(self):
|
|
|
|
parCollection = self.sc.parallelize([1, 2, 3, 4])
|
|
|
|
flatMappedRDD = parCollection.flatMap(lambda x: range(1, x + 1))
|
|
|
|
|
|
|
|
self.assertFalse(flatMappedRDD.isCheckpointed())
|
|
|
|
self.assertFalse(flatMappedRDD.isLocallyCheckpointed())
|
|
|
|
|
|
|
|
flatMappedRDD.localCheckpoint()
|
|
|
|
result = flatMappedRDD.collect()
|
|
|
|
time.sleep(1) # 1 second
|
|
|
|
self.assertTrue(flatMappedRDD.isCheckpointed())
|
|
|
|
self.assertTrue(flatMappedRDD.isLocallyCheckpointed())
|
|
|
|
self.assertEqual(flatMappedRDD.collect(), result)
|
|
|
|
|
|
|
|
|
2014-10-06 17:07:53 -04:00
|
|
|
class AddFileTests(PySparkTestCase):
|
2013-01-22 20:54:11 -05:00
|
|
|
|
|
|
|
def test_add_py_file(self):
|
|
|
|
# To ensure that we're actually testing addPyFile's effects, check that
|
|
|
|
# this job fails due to `userlibrary` not being on the Python path:
|
2014-07-29 03:15:45 -04:00
|
|
|
# disable logging in log4j temporarily
|
2013-01-22 20:54:11 -05:00
|
|
|
def func(x):
|
|
|
|
from userlibrary import UserClass
|
|
|
|
return UserClass().hello()
|
2015-04-21 20:49:55 -04:00
|
|
|
with QuietTest(self.sc):
|
|
|
|
self.assertRaises(Exception, self.sc.parallelize(range(2)).map(func).first)
|
2014-07-29 03:15:45 -04:00
|
|
|
|
2013-01-22 20:54:11 -05:00
|
|
|
# Add the file, so the job should now succeed:
|
|
|
|
path = os.path.join(SPARK_HOME, "python/test_support/userlibrary.py")
|
|
|
|
self.sc.addPyFile(path)
|
|
|
|
res = self.sc.parallelize(range(2)).map(func).first()
|
|
|
|
self.assertEqual("Hello World!", res)
|
|
|
|
|
2013-01-23 13:36:18 -05:00
|
|
|
def test_add_file_locally(self):
|
2016-09-21 04:37:03 -04:00
|
|
|
path = os.path.join(SPARK_HOME, "python/test_support/hello/hello.txt")
|
2013-01-23 13:36:18 -05:00
|
|
|
self.sc.addFile(path)
|
|
|
|
download_path = SparkFiles.get("hello.txt")
|
|
|
|
self.assertNotEqual(path, download_path)
|
|
|
|
with open(download_path) as test_file:
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual("Hello World!\n", test_file.readline())
|
2013-01-23 13:36:18 -05:00
|
|
|
|
2016-09-21 04:37:03 -04:00
|
|
|
def test_add_file_recursively_locally(self):
|
|
|
|
path = os.path.join(SPARK_HOME, "python/test_support/hello")
|
|
|
|
self.sc.addFile(path, True)
|
|
|
|
download_path = SparkFiles.get("hello")
|
|
|
|
self.assertNotEqual(path, download_path)
|
|
|
|
with open(download_path + "/hello.txt") as test_file:
|
|
|
|
self.assertEqual("Hello World!\n", test_file.readline())
|
|
|
|
with open(download_path + "/sub_hello/sub_hello.txt") as test_file:
|
|
|
|
self.assertEqual("Sub Hello World!\n", test_file.readline())
|
|
|
|
|
2013-01-23 13:36:18 -05:00
|
|
|
def test_add_py_file_locally(self):
|
|
|
|
# To ensure that we're actually testing addPyFile's effects, check that
|
|
|
|
# this fails due to `userlibrary` not being on the Python path:
|
|
|
|
def func():
|
|
|
|
from userlibrary import UserClass
|
|
|
|
self.assertRaises(ImportError, func)
|
|
|
|
path = os.path.join(SPARK_HOME, "python/test_support/userlibrary.py")
|
2015-04-16 19:20:57 -04:00
|
|
|
self.sc.addPyFile(path)
|
2013-01-23 13:36:18 -05:00
|
|
|
from userlibrary import UserClass
|
|
|
|
self.assertEqual("Hello World!", UserClass().hello())
|
|
|
|
|
2013-08-15 19:01:19 -04:00
|
|
|
def test_add_egg_file_locally(self):
|
|
|
|
# To ensure that we're actually testing addPyFile's effects, check that
|
|
|
|
# this fails due to `userlibrary` not being on the Python path:
|
|
|
|
def func():
|
|
|
|
from userlib import UserClass
|
|
|
|
self.assertRaises(ImportError, func)
|
2015-04-16 19:20:57 -04:00
|
|
|
path = os.path.join(SPARK_HOME, "python/test_support/userlib-0.1.zip")
|
2013-08-15 19:01:19 -04:00
|
|
|
self.sc.addPyFile(path)
|
|
|
|
from userlib import UserClass
|
|
|
|
self.assertEqual("Hello World from inside a package!", UserClass().hello())
|
|
|
|
|
2014-09-24 15:10:09 -04:00
|
|
|
def test_overwrite_system_module(self):
|
|
|
|
self.sc.addPyFile(os.path.join(SPARK_HOME, "python/test_support/SimpleHTTPServer.py"))
|
|
|
|
|
|
|
|
import SimpleHTTPServer
|
|
|
|
self.assertEqual("My Server", SimpleHTTPServer.__name__)
|
|
|
|
|
|
|
|
def func(x):
|
|
|
|
import SimpleHTTPServer
|
|
|
|
return SimpleHTTPServer.__name__
|
|
|
|
|
|
|
|
self.assertEqual(["My Server"], self.sc.parallelize(range(1)).map(func).collect())
|
|
|
|
|
2013-01-22 20:54:11 -05:00
|
|
|
|
2016-12-20 18:51:21 -05:00
|
|
|
class TaskContextTests(PySparkTestCase):
|
|
|
|
|
|
|
|
def setUp(self):
|
|
|
|
self._old_sys_path = list(sys.path)
|
|
|
|
class_name = self.__class__.__name__
|
|
|
|
# Allow retries even though they are normally disabled in local mode
|
|
|
|
self.sc = SparkContext('local[4, 2]', class_name)
|
|
|
|
|
|
|
|
def test_stage_id(self):
|
|
|
|
"""Test the stage ids are available and incrementing as expected."""
|
|
|
|
rdd = self.sc.parallelize(range(10))
|
|
|
|
stage1 = rdd.map(lambda x: TaskContext.get().stageId()).take(1)[0]
|
|
|
|
stage2 = rdd.map(lambda x: TaskContext.get().stageId()).take(1)[0]
|
|
|
|
# Test using the constructor directly rather than the get()
|
|
|
|
stage3 = rdd.map(lambda x: TaskContext().stageId()).take(1)[0]
|
|
|
|
self.assertEqual(stage1 + 1, stage2)
|
|
|
|
self.assertEqual(stage1 + 2, stage3)
|
|
|
|
self.assertEqual(stage2 + 1, stage3)
|
|
|
|
|
|
|
|
def test_partition_id(self):
|
|
|
|
"""Test the partition id."""
|
|
|
|
rdd1 = self.sc.parallelize(range(10), 1)
|
|
|
|
rdd2 = self.sc.parallelize(range(10), 2)
|
|
|
|
pids1 = rdd1.map(lambda x: TaskContext.get().partitionId()).collect()
|
|
|
|
pids2 = rdd2.map(lambda x: TaskContext.get().partitionId()).collect()
|
|
|
|
self.assertEqual(0, pids1[0])
|
|
|
|
self.assertEqual(0, pids1[9])
|
|
|
|
self.assertEqual(0, pids2[0])
|
|
|
|
self.assertEqual(1, pids2[9])
|
|
|
|
|
|
|
|
def test_attempt_number(self):
|
|
|
|
"""Verify the attempt numbers are correctly reported."""
|
|
|
|
rdd = self.sc.parallelize(range(10))
|
|
|
|
# Verify a simple job with no failures
|
|
|
|
attempt_numbers = rdd.map(lambda x: TaskContext.get().attemptNumber()).collect()
|
|
|
|
map(lambda attempt: self.assertEqual(0, attempt), attempt_numbers)
|
|
|
|
|
|
|
|
def fail_on_first(x):
|
|
|
|
"""Fail on the first attempt so we get a positive attempt number"""
|
|
|
|
tc = TaskContext.get()
|
|
|
|
attempt_number = tc.attemptNumber()
|
|
|
|
partition_id = tc.partitionId()
|
|
|
|
attempt_id = tc.taskAttemptId()
|
|
|
|
if attempt_number == 0 and partition_id == 0:
|
|
|
|
raise Exception("Failing on first attempt")
|
|
|
|
else:
|
|
|
|
return [x, partition_id, attempt_number, attempt_id]
|
|
|
|
result = rdd.map(fail_on_first).collect()
|
|
|
|
# We should re-submit the first partition to it but other partitions should be attempt 0
|
|
|
|
self.assertEqual([0, 0, 1], result[0][0:3])
|
|
|
|
self.assertEqual([9, 3, 0], result[9][0:3])
|
|
|
|
first_partition = filter(lambda x: x[1] == 0, result)
|
|
|
|
map(lambda x: self.assertEqual(1, x[2]), first_partition)
|
|
|
|
other_partitions = filter(lambda x: x[1] != 0, result)
|
|
|
|
map(lambda x: self.assertEqual(0, x[2]), other_partitions)
|
|
|
|
# The task attempt id should be different
|
|
|
|
self.assertTrue(result[0][3] != result[9][3])
|
|
|
|
|
|
|
|
def test_tc_on_driver(self):
|
|
|
|
"""Verify that getting the TaskContext on the driver returns None."""
|
|
|
|
tc = TaskContext.get()
|
|
|
|
self.assertTrue(tc is None)
|
|
|
|
|
2018-05-31 14:23:57 -04:00
|
|
|
def test_get_local_property(self):
|
|
|
|
"""Verify that local properties set on the driver are available in TaskContext."""
|
|
|
|
key = "testkey"
|
|
|
|
value = "testvalue"
|
|
|
|
self.sc.setLocalProperty(key, value)
|
|
|
|
try:
|
|
|
|
rdd = self.sc.parallelize(range(1), 1)
|
2018-06-15 15:56:39 -04:00
|
|
|
prop1 = rdd.map(lambda _: TaskContext.get().getLocalProperty(key)).collect()[0]
|
2018-05-31 14:23:57 -04:00
|
|
|
self.assertEqual(prop1, value)
|
2018-06-15 15:56:39 -04:00
|
|
|
prop2 = rdd.map(lambda _: TaskContext.get().getLocalProperty("otherkey")).collect()[0]
|
2018-05-31 14:23:57 -04:00
|
|
|
self.assertTrue(prop2 is None)
|
|
|
|
finally:
|
|
|
|
self.sc.setLocalProperty(key, None)
|
|
|
|
|
2016-12-20 18:51:21 -05:00
|
|
|
|
2014-10-06 17:07:53 -04:00
|
|
|
class RDDTests(ReusedPySparkTestCase):
|
2013-11-29 02:44:56 -05:00
|
|
|
|
2015-05-19 00:43:12 -04:00
|
|
|
def test_range(self):
|
|
|
|
self.assertEqual(self.sc.range(1, 1).count(), 0)
|
|
|
|
self.assertEqual(self.sc.range(1, 0, -1).count(), 1)
|
|
|
|
self.assertEqual(self.sc.range(0, 1 << 40, 1 << 39).count(), 2)
|
|
|
|
|
2014-09-06 19:12:29 -04:00
|
|
|
def test_id(self):
|
|
|
|
rdd = self.sc.parallelize(range(10))
|
|
|
|
id = rdd.id()
|
|
|
|
self.assertEqual(id, rdd.id())
|
|
|
|
rdd2 = rdd.map(str).filter(bool)
|
|
|
|
id2 = rdd2.id()
|
|
|
|
self.assertEqual(id + 1, id2)
|
|
|
|
self.assertEqual(id2, rdd2.id())
|
|
|
|
|
2015-06-17 16:59:39 -04:00
|
|
|
def test_empty_rdd(self):
|
|
|
|
rdd = self.sc.emptyRDD()
|
|
|
|
self.assertTrue(rdd.isEmpty())
|
|
|
|
|
|
|
|
def test_sum(self):
|
|
|
|
self.assertEqual(0, self.sc.emptyRDD().sum())
|
|
|
|
self.assertEqual(6, self.sc.parallelize([1, 2, 3]).sum())
|
|
|
|
|
2016-12-20 16:12:16 -05:00
|
|
|
def test_to_localiterator(self):
|
|
|
|
from time import sleep
|
|
|
|
rdd = self.sc.parallelize([1, 2, 3])
|
|
|
|
it = rdd.toLocalIterator()
|
|
|
|
sleep(5)
|
|
|
|
self.assertEqual([1, 2, 3], sorted(it))
|
|
|
|
|
|
|
|
rdd2 = rdd.repartition(1000)
|
|
|
|
it2 = rdd2.toLocalIterator()
|
|
|
|
sleep(5)
|
|
|
|
self.assertEqual([1, 2, 3], sorted(it2))
|
|
|
|
|
2013-11-29 02:44:56 -05:00
|
|
|
def test_save_as_textfile_with_unicode(self):
|
|
|
|
# Regression test for SPARK-970
|
|
|
|
x = u"\u00A1Hola, mundo!"
|
|
|
|
data = self.sc.parallelize([x])
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
tempFile = tempfile.NamedTemporaryFile(delete=True)
|
2013-11-29 02:44:56 -05:00
|
|
|
tempFile.close()
|
|
|
|
data.saveAsTextFile(tempFile.name)
|
2015-04-16 19:20:57 -04:00
|
|
|
raw_contents = b''.join(open(p, 'rb').read()
|
|
|
|
for p in glob(tempFile.name + "/part-0000*"))
|
|
|
|
self.assertEqual(x, raw_contents.strip().decode("utf-8"))
|
2013-11-29 02:44:56 -05:00
|
|
|
|
2014-08-18 16:58:35 -04:00
|
|
|
def test_save_as_textfile_with_utf8(self):
|
|
|
|
x = u"\u00A1Hola, mundo!"
|
|
|
|
data = self.sc.parallelize([x.encode("utf-8")])
|
|
|
|
tempFile = tempfile.NamedTemporaryFile(delete=True)
|
|
|
|
tempFile.close()
|
|
|
|
data.saveAsTextFile(tempFile.name)
|
2015-04-16 19:20:57 -04:00
|
|
|
raw_contents = b''.join(open(p, 'rb').read()
|
|
|
|
for p in glob(tempFile.name + "/part-0000*"))
|
|
|
|
self.assertEqual(x, raw_contents.strip().decode('utf8'))
|
2014-08-18 16:58:35 -04:00
|
|
|
|
2014-01-23 16:05:59 -05:00
|
|
|
def test_transforming_cartesian_result(self):
|
|
|
|
# Regression test for SPARK-1034
|
|
|
|
rdd1 = self.sc.parallelize([1, 2])
|
|
|
|
rdd2 = self.sc.parallelize([3, 4])
|
|
|
|
cart = rdd1.cartesian(rdd2)
|
2015-04-16 19:20:57 -04:00
|
|
|
result = cart.map(lambda x_y3: x_y3[0] + x_y3[1]).collect()
|
2014-01-23 16:05:59 -05:00
|
|
|
|
2014-07-26 20:37:05 -04:00
|
|
|
def test_transforming_pickle_file(self):
|
|
|
|
# Regression test for SPARK-2601
|
2015-04-16 19:20:57 -04:00
|
|
|
data = self.sc.parallelize([u"Hello", u"World!"])
|
2014-07-26 20:37:05 -04:00
|
|
|
tempFile = tempfile.NamedTemporaryFile(delete=True)
|
|
|
|
tempFile.close()
|
|
|
|
data.saveAsPickleFile(tempFile.name)
|
|
|
|
pickled_file = self.sc.pickleFile(tempFile.name)
|
|
|
|
pickled_file.map(lambda x: x).collect()
|
|
|
|
|
2014-01-23 18:09:19 -05:00
|
|
|
def test_cartesian_on_textfile(self):
|
|
|
|
# Regression test for
|
2016-09-21 04:37:03 -04:00
|
|
|
path = os.path.join(SPARK_HOME, "python/test_support/hello/hello.txt")
|
2014-01-23 18:09:19 -05:00
|
|
|
a = self.sc.textFile(path)
|
|
|
|
result = a.cartesian(a).collect()
|
|
|
|
(x, y) = result[0]
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual(u"Hello World!", x.strip())
|
|
|
|
self.assertEqual(u"Hello World!", y.strip())
|
2014-01-23 18:09:19 -05:00
|
|
|
|
2016-12-08 14:08:12 -05:00
|
|
|
def test_cartesian_chaining(self):
|
|
|
|
# Tests for SPARK-16589
|
|
|
|
rdd = self.sc.parallelize(range(10), 2)
|
|
|
|
self.assertSetEqual(
|
|
|
|
set(rdd.cartesian(rdd).cartesian(rdd).collect()),
|
|
|
|
set([((x, y), z) for x in range(10) for y in range(10) for z in range(10)])
|
|
|
|
)
|
|
|
|
|
|
|
|
self.assertSetEqual(
|
|
|
|
set(rdd.cartesian(rdd.cartesian(rdd)).collect()),
|
|
|
|
set([(x, (y, z)) for x in range(10) for y in range(10) for z in range(10)])
|
|
|
|
)
|
|
|
|
|
|
|
|
self.assertSetEqual(
|
|
|
|
set(rdd.cartesian(rdd.zip(rdd)).collect()),
|
|
|
|
set([(x, (y, y)) for x in range(10) for y in range(10)])
|
|
|
|
)
|
|
|
|
|
2017-09-17 13:46:27 -04:00
|
|
|
def test_zip_chaining(self):
|
|
|
|
# Tests for SPARK-21985
|
|
|
|
rdd = self.sc.parallelize('abc', 2)
|
|
|
|
self.assertSetEqual(
|
|
|
|
set(rdd.zip(rdd).zip(rdd).collect()),
|
|
|
|
set([((x, x), x) for x in 'abc'])
|
|
|
|
)
|
|
|
|
self.assertSetEqual(
|
|
|
|
set(rdd.zip(rdd.zip(rdd)).collect()),
|
|
|
|
set([(x, (x, x)) for x in 'abc'])
|
|
|
|
)
|
|
|
|
|
2014-01-23 21:10:16 -05:00
|
|
|
def test_deleting_input_files(self):
|
|
|
|
# Regression test for SPARK-1025
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
tempFile = tempfile.NamedTemporaryFile(delete=False)
|
2015-04-16 19:20:57 -04:00
|
|
|
tempFile.write(b"Hello World!")
|
2014-01-23 21:10:16 -05:00
|
|
|
tempFile.close()
|
|
|
|
data = self.sc.textFile(tempFile.name)
|
|
|
|
filtered_data = data.filter(lambda x: True)
|
|
|
|
self.assertEqual(1, filtered_data.count())
|
|
|
|
os.unlink(tempFile.name)
|
2015-04-21 20:49:55 -04:00
|
|
|
with QuietTest(self.sc):
|
|
|
|
self.assertRaises(Exception, lambda: filtered_data.count())
|
2014-01-23 21:10:16 -05:00
|
|
|
|
2014-10-22 12:33:12 -04:00
|
|
|
def test_sampling_default_seed(self):
|
|
|
|
# Test for SPARK-3995 (default seed setting)
|
[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
|
|
|
data = self.sc.parallelize(xrange(1000), 1)
|
2014-10-22 12:33:12 -04:00
|
|
|
subset = data.takeSample(False, 10)
|
|
|
|
self.assertEqual(len(subset), 10)
|
|
|
|
|
[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
|
|
|
def test_aggregate_mutable_zero_value(self):
|
|
|
|
# Test for SPARK-9021; uses aggregate and treeAggregate to build dict
|
|
|
|
# representing a counter of ints
|
|
|
|
# NOTE: dict is used instead of collections.Counter for Python 2.6
|
|
|
|
# compatibility
|
|
|
|
from collections import defaultdict
|
|
|
|
|
|
|
|
# Show that single or multiple partitions work
|
|
|
|
data1 = self.sc.range(10, numSlices=1)
|
|
|
|
data2 = self.sc.range(10, numSlices=2)
|
|
|
|
|
|
|
|
def seqOp(x, y):
|
|
|
|
x[y] += 1
|
|
|
|
return x
|
|
|
|
|
|
|
|
def comboOp(x, y):
|
|
|
|
for key, val in y.items():
|
|
|
|
x[key] += val
|
|
|
|
return x
|
|
|
|
|
|
|
|
counts1 = data1.aggregate(defaultdict(int), seqOp, comboOp)
|
|
|
|
counts2 = data2.aggregate(defaultdict(int), seqOp, comboOp)
|
|
|
|
counts3 = data1.treeAggregate(defaultdict(int), seqOp, comboOp, 2)
|
|
|
|
counts4 = data2.treeAggregate(defaultdict(int), seqOp, comboOp, 2)
|
|
|
|
|
|
|
|
ground_truth = defaultdict(int, dict((i, 1) for i in range(10)))
|
|
|
|
self.assertEqual(counts1, ground_truth)
|
|
|
|
self.assertEqual(counts2, ground_truth)
|
|
|
|
self.assertEqual(counts3, ground_truth)
|
|
|
|
self.assertEqual(counts4, ground_truth)
|
|
|
|
|
|
|
|
def test_aggregate_by_key_mutable_zero_value(self):
|
|
|
|
# Test for SPARK-9021; uses aggregateByKey to make a pair RDD that
|
|
|
|
# contains lists of all values for each key in the original RDD
|
|
|
|
|
|
|
|
# list(range(...)) for Python 3.x compatibility (can't use * operator
|
|
|
|
# on a range object)
|
|
|
|
# list(zip(...)) for Python 3.x compatibility (want to parallelize a
|
|
|
|
# collection, not a zip object)
|
|
|
|
tuples = list(zip(list(range(10))*2, [1]*20))
|
|
|
|
# Show that single or multiple partitions work
|
|
|
|
data1 = self.sc.parallelize(tuples, 1)
|
|
|
|
data2 = self.sc.parallelize(tuples, 2)
|
|
|
|
|
|
|
|
def seqOp(x, y):
|
|
|
|
x.append(y)
|
|
|
|
return x
|
|
|
|
|
|
|
|
def comboOp(x, y):
|
|
|
|
x.extend(y)
|
|
|
|
return x
|
|
|
|
|
|
|
|
values1 = data1.aggregateByKey([], seqOp, comboOp).collect()
|
|
|
|
values2 = data2.aggregateByKey([], seqOp, comboOp).collect()
|
|
|
|
# Sort lists to ensure clean comparison with ground_truth
|
|
|
|
values1.sort()
|
|
|
|
values2.sort()
|
|
|
|
|
|
|
|
ground_truth = [(i, [1]*2) for i in range(10)]
|
|
|
|
self.assertEqual(values1, ground_truth)
|
|
|
|
self.assertEqual(values2, ground_truth)
|
|
|
|
|
|
|
|
def test_fold_mutable_zero_value(self):
|
|
|
|
# Test for SPARK-9021; uses fold to merge an RDD of dict counters into
|
|
|
|
# a single dict
|
|
|
|
# NOTE: dict is used instead of collections.Counter for Python 2.6
|
|
|
|
# compatibility
|
|
|
|
from collections import defaultdict
|
|
|
|
|
|
|
|
counts1 = defaultdict(int, dict((i, 1) for i in range(10)))
|
|
|
|
counts2 = defaultdict(int, dict((i, 1) for i in range(3, 8)))
|
|
|
|
counts3 = defaultdict(int, dict((i, 1) for i in range(4, 7)))
|
|
|
|
counts4 = defaultdict(int, dict((i, 1) for i in range(5, 6)))
|
|
|
|
all_counts = [counts1, counts2, counts3, counts4]
|
|
|
|
# Show that single or multiple partitions work
|
|
|
|
data1 = self.sc.parallelize(all_counts, 1)
|
|
|
|
data2 = self.sc.parallelize(all_counts, 2)
|
|
|
|
|
|
|
|
def comboOp(x, y):
|
|
|
|
for key, val in y.items():
|
|
|
|
x[key] += val
|
|
|
|
return x
|
|
|
|
|
|
|
|
fold1 = data1.fold(defaultdict(int), comboOp)
|
|
|
|
fold2 = data2.fold(defaultdict(int), comboOp)
|
|
|
|
|
|
|
|
ground_truth = defaultdict(int)
|
|
|
|
for counts in all_counts:
|
|
|
|
for key, val in counts.items():
|
|
|
|
ground_truth[key] += val
|
|
|
|
self.assertEqual(fold1, ground_truth)
|
|
|
|
self.assertEqual(fold2, ground_truth)
|
|
|
|
|
|
|
|
def test_fold_by_key_mutable_zero_value(self):
|
|
|
|
# Test for SPARK-9021; uses foldByKey to make a pair RDD that contains
|
|
|
|
# lists of all values for each key in the original RDD
|
|
|
|
|
|
|
|
tuples = [(i, range(i)) for i in range(10)]*2
|
|
|
|
# Show that single or multiple partitions work
|
|
|
|
data1 = self.sc.parallelize(tuples, 1)
|
|
|
|
data2 = self.sc.parallelize(tuples, 2)
|
|
|
|
|
|
|
|
def comboOp(x, y):
|
|
|
|
x.extend(y)
|
|
|
|
return x
|
|
|
|
|
|
|
|
values1 = data1.foldByKey([], comboOp).collect()
|
|
|
|
values2 = data2.foldByKey([], comboOp).collect()
|
|
|
|
# Sort lists to ensure clean comparison with ground_truth
|
|
|
|
values1.sort()
|
|
|
|
values2.sort()
|
|
|
|
|
|
|
|
# list(range(...)) for Python 3.x compatibility
|
|
|
|
ground_truth = [(i, list(range(i))*2) for i in range(10)]
|
|
|
|
self.assertEqual(values1, ground_truth)
|
|
|
|
self.assertEqual(values2, ground_truth)
|
|
|
|
|
2014-11-18 19:17:51 -05:00
|
|
|
def test_aggregate_by_key(self):
|
2014-06-12 11:14:25 -04:00
|
|
|
data = self.sc.parallelize([(1, 1), (1, 1), (3, 2), (5, 1), (5, 3)], 2)
|
2014-07-22 01:30:53 -04:00
|
|
|
|
2014-06-12 11:14:25 -04:00
|
|
|
def seqOp(x, y):
|
|
|
|
x.add(y)
|
|
|
|
return x
|
|
|
|
|
|
|
|
def combOp(x, y):
|
|
|
|
x |= y
|
|
|
|
return x
|
2014-07-22 01:30:53 -04:00
|
|
|
|
2014-06-12 11:14:25 -04:00
|
|
|
sets = dict(data.aggregateByKey(set(), seqOp, combOp).collect())
|
|
|
|
self.assertEqual(3, len(sets))
|
|
|
|
self.assertEqual(set([1]), sets[1])
|
|
|
|
self.assertEqual(set([2]), sets[3])
|
|
|
|
self.assertEqual(set([1, 3]), sets[5])
|
2013-11-29 02:44:56 -05:00
|
|
|
|
2014-07-29 04:02:18 -04:00
|
|
|
def test_itemgetter(self):
|
|
|
|
rdd = self.sc.parallelize([range(10)])
|
|
|
|
from operator import itemgetter
|
|
|
|
self.assertEqual([1], rdd.map(itemgetter(1)).collect())
|
|
|
|
self.assertEqual([(2, 3)], rdd.map(itemgetter(2, 3)).collect())
|
|
|
|
|
2014-08-04 15:13:41 -04:00
|
|
|
def test_namedtuple_in_rdd(self):
|
|
|
|
from collections import namedtuple
|
|
|
|
Person = namedtuple("Person", "id firstName lastName")
|
|
|
|
jon = Person(1, "Jon", "Doe")
|
|
|
|
jane = Person(2, "Jane", "Doe")
|
|
|
|
theDoes = self.sc.parallelize([jon, jane])
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual([jon, jane], theDoes.collect())
|
2014-08-04 15:13:41 -04:00
|
|
|
|
2014-08-16 19:59:34 -04:00
|
|
|
def test_large_broadcast(self):
|
2015-04-21 20:49:55 -04:00
|
|
|
N = 10000
|
2014-08-16 19:59:34 -04:00
|
|
|
data = [[float(i) for i in range(300)] for i in range(N)]
|
2015-04-21 20:49:55 -04:00
|
|
|
bdata = self.sc.broadcast(data) # 27MB
|
2014-08-16 19:59:34 -04:00
|
|
|
m = self.sc.parallelize(range(1), 1).map(lambda x: len(bdata.value)).sum()
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual(N, m)
|
2014-08-16 19:59:34 -04:00
|
|
|
|
2016-04-06 13:46:34 -04:00
|
|
|
def test_unpersist(self):
|
|
|
|
N = 1000
|
|
|
|
data = [[float(i) for i in range(300)] for i in range(N)]
|
|
|
|
bdata = self.sc.broadcast(data) # 3MB
|
|
|
|
bdata.unpersist()
|
|
|
|
m = self.sc.parallelize(range(1), 1).map(lambda x: len(bdata.value)).sum()
|
|
|
|
self.assertEqual(N, m)
|
|
|
|
bdata.destroy()
|
|
|
|
try:
|
|
|
|
self.sc.parallelize(range(1), 1).map(lambda x: len(bdata.value)).sum()
|
|
|
|
except Exception as e:
|
|
|
|
pass
|
|
|
|
else:
|
|
|
|
raise Exception("job should fail after destroy the broadcast")
|
|
|
|
|
2014-11-18 19:17:51 -05:00
|
|
|
def test_multiple_broadcasts(self):
|
|
|
|
N = 1 << 21
|
|
|
|
b1 = self.sc.broadcast(set(range(N))) # multiple blocks in JVM
|
2015-04-16 19:20:57 -04:00
|
|
|
r = list(range(1 << 15))
|
2014-11-18 19:17:51 -05:00
|
|
|
random.shuffle(r)
|
2015-04-16 19:20:57 -04:00
|
|
|
s = str(r).encode()
|
2014-11-18 19:17:51 -05:00
|
|
|
checksum = hashlib.md5(s).hexdigest()
|
|
|
|
b2 = self.sc.broadcast(s)
|
|
|
|
r = list(set(self.sc.parallelize(range(10), 10).map(
|
|
|
|
lambda x: (len(b1.value), hashlib.md5(b2.value).hexdigest())).collect()))
|
|
|
|
self.assertEqual(1, len(r))
|
|
|
|
size, csum = r[0]
|
|
|
|
self.assertEqual(N, size)
|
|
|
|
self.assertEqual(checksum, csum)
|
|
|
|
|
|
|
|
random.shuffle(r)
|
2015-04-16 19:20:57 -04:00
|
|
|
s = str(r).encode()
|
2014-11-18 19:17:51 -05:00
|
|
|
checksum = hashlib.md5(s).hexdigest()
|
|
|
|
b2 = self.sc.broadcast(s)
|
|
|
|
r = list(set(self.sc.parallelize(range(10), 10).map(
|
|
|
|
lambda x: (len(b1.value), hashlib.md5(b2.value).hexdigest())).collect()))
|
|
|
|
self.assertEqual(1, len(r))
|
|
|
|
size, csum = r[0]
|
|
|
|
self.assertEqual(N, size)
|
|
|
|
self.assertEqual(checksum, csum)
|
|
|
|
|
2017-08-01 18:12:23 -04:00
|
|
|
def test_multithread_broadcast_pickle(self):
|
|
|
|
import threading
|
|
|
|
|
|
|
|
b1 = self.sc.broadcast(list(range(3)))
|
|
|
|
b2 = self.sc.broadcast(list(range(3)))
|
|
|
|
|
|
|
|
def f1():
|
|
|
|
return b1.value
|
|
|
|
|
|
|
|
def f2():
|
|
|
|
return b2.value
|
|
|
|
|
|
|
|
funcs_num_pickled = {f1: None, f2: None}
|
|
|
|
|
|
|
|
def do_pickle(f, sc):
|
|
|
|
command = (f, None, sc.serializer, sc.serializer)
|
|
|
|
ser = CloudPickleSerializer()
|
|
|
|
ser.dumps(command)
|
|
|
|
|
|
|
|
def process_vars(sc):
|
|
|
|
broadcast_vars = list(sc._pickled_broadcast_vars)
|
|
|
|
num_pickled = len(broadcast_vars)
|
|
|
|
sc._pickled_broadcast_vars.clear()
|
|
|
|
return num_pickled
|
|
|
|
|
|
|
|
def run(f, sc):
|
|
|
|
do_pickle(f, sc)
|
|
|
|
funcs_num_pickled[f] = process_vars(sc)
|
|
|
|
|
|
|
|
# pickle f1, adds b1 to sc._pickled_broadcast_vars in main thread local storage
|
|
|
|
do_pickle(f1, self.sc)
|
|
|
|
|
|
|
|
# run all for f2, should only add/count/clear b2 from worker thread local storage
|
|
|
|
t = threading.Thread(target=run, args=(f2, self.sc))
|
|
|
|
t.start()
|
|
|
|
t.join()
|
|
|
|
|
|
|
|
# count number of vars pickled in main thread, only b1 should be counted and cleared
|
|
|
|
funcs_num_pickled[f1] = process_vars(self.sc)
|
|
|
|
|
|
|
|
self.assertEqual(funcs_num_pickled[f1], 1)
|
|
|
|
self.assertEqual(funcs_num_pickled[f2], 1)
|
|
|
|
self.assertEqual(len(list(self.sc._pickled_broadcast_vars)), 0)
|
|
|
|
|
2014-09-18 21:11:48 -04:00
|
|
|
def test_large_closure(self):
|
2015-04-21 20:49:55 -04:00
|
|
|
N = 200000
|
2014-09-18 21:11:48 -04:00
|
|
|
data = [float(i) for i in xrange(N)]
|
2014-10-01 14:21:34 -04:00
|
|
|
rdd = self.sc.parallelize(range(1), 1).map(lambda x: len(data))
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual(N, rdd.first())
|
2015-04-15 15:58:02 -04:00
|
|
|
# regression test for SPARK-6886
|
|
|
|
self.assertEqual(1, rdd.map(lambda x: (x, 1)).groupByKey().count())
|
2014-09-18 21:11:48 -04:00
|
|
|
|
2014-08-19 17:46:32 -04:00
|
|
|
def test_zip_with_different_serializers(self):
|
|
|
|
a = self.sc.parallelize(range(5))
|
|
|
|
b = self.sc.parallelize(range(100, 105))
|
|
|
|
self.assertEqual(a.zip(b).collect(), [(0, 100), (1, 101), (2, 102), (3, 103), (4, 104)])
|
|
|
|
a = a._reserialize(BatchedSerializer(PickleSerializer(), 2))
|
|
|
|
b = b._reserialize(MarshalSerializer())
|
|
|
|
self.assertEqual(a.zip(b).collect(), [(0, 100), (1, 101), (2, 102), (3, 103), (4, 104)])
|
2014-12-16 01:58:26 -05:00
|
|
|
# regression test for SPARK-4841
|
2016-09-21 04:37:03 -04:00
|
|
|
path = os.path.join(SPARK_HOME, "python/test_support/hello/hello.txt")
|
2014-12-16 01:58:26 -05:00
|
|
|
t = self.sc.textFile(path)
|
|
|
|
cnt = t.count()
|
|
|
|
self.assertEqual(cnt, t.zip(t).count())
|
|
|
|
rdd = t.map(str)
|
|
|
|
self.assertEqual(cnt, t.zip(rdd).count())
|
|
|
|
# regression test for bug in _reserializer()
|
|
|
|
self.assertEqual(cnt, t.zip(rdd).count())
|
2014-08-19 17:46:32 -04:00
|
|
|
|
2015-02-24 17:50:00 -05:00
|
|
|
def test_zip_with_different_object_sizes(self):
|
|
|
|
# regress test for SPARK-5973
|
[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
|
|
|
a = self.sc.parallelize(xrange(10000)).map(lambda i: '*' * i)
|
|
|
|
b = self.sc.parallelize(xrange(10000, 20000)).map(lambda i: '*' * i)
|
2015-02-24 17:50:00 -05:00
|
|
|
self.assertEqual(10000, a.zip(b).count())
|
|
|
|
|
2014-08-19 17:46:32 -04:00
|
|
|
def test_zip_with_different_number_of_items(self):
|
|
|
|
a = self.sc.parallelize(range(5), 2)
|
|
|
|
# different number of partitions
|
|
|
|
b = self.sc.parallelize(range(100, 106), 3)
|
|
|
|
self.assertRaises(ValueError, lambda: a.zip(b))
|
2015-04-21 20:49:55 -04:00
|
|
|
with QuietTest(self.sc):
|
|
|
|
# different number of batched items in JVM
|
|
|
|
b = self.sc.parallelize(range(100, 104), 2)
|
|
|
|
self.assertRaises(Exception, lambda: a.zip(b).count())
|
|
|
|
# different number of items in one pair
|
|
|
|
b = self.sc.parallelize(range(100, 106), 2)
|
|
|
|
self.assertRaises(Exception, lambda: a.zip(b).count())
|
|
|
|
# same total number of items, but different distributions
|
|
|
|
a = self.sc.parallelize([2, 3], 2).flatMap(range)
|
|
|
|
b = self.sc.parallelize([3, 2], 2).flatMap(range)
|
|
|
|
self.assertEqual(a.count(), b.count())
|
|
|
|
self.assertRaises(Exception, lambda: a.zip(b).count())
|
2014-08-19 17:46:32 -04:00
|
|
|
|
2014-09-02 18:47:47 -04:00
|
|
|
def test_count_approx_distinct(self):
|
[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
|
|
|
rdd = self.sc.parallelize(xrange(1000))
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertTrue(950 < rdd.countApproxDistinct(0.03) < 1050)
|
|
|
|
self.assertTrue(950 < rdd.map(float).countApproxDistinct(0.03) < 1050)
|
|
|
|
self.assertTrue(950 < rdd.map(str).countApproxDistinct(0.03) < 1050)
|
|
|
|
self.assertTrue(950 < rdd.map(lambda x: (x, -x)).countApproxDistinct(0.03) < 1050)
|
2014-09-02 18:47:47 -04:00
|
|
|
|
|
|
|
rdd = self.sc.parallelize([i % 20 for i in range(1000)], 7)
|
|
|
|
self.assertTrue(18 < rdd.countApproxDistinct() < 22)
|
|
|
|
self.assertTrue(18 < rdd.map(float).countApproxDistinct() < 22)
|
|
|
|
self.assertTrue(18 < rdd.map(str).countApproxDistinct() < 22)
|
|
|
|
self.assertTrue(18 < rdd.map(lambda x: (x, -x)).countApproxDistinct() < 22)
|
|
|
|
|
|
|
|
self.assertRaises(ValueError, lambda: rdd.countApproxDistinct(0.00000001))
|
|
|
|
|
[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 test_histogram(self):
|
|
|
|
# empty
|
|
|
|
rdd = self.sc.parallelize([])
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual([0], rdd.histogram([0, 10])[1])
|
|
|
|
self.assertEqual([0, 0], rdd.histogram([0, 4, 10])[1])
|
[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
|
|
|
self.assertRaises(ValueError, lambda: rdd.histogram(1))
|
|
|
|
|
|
|
|
# out of range
|
|
|
|
rdd = self.sc.parallelize([10.01, -0.01])
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual([0], rdd.histogram([0, 10])[1])
|
|
|
|
self.assertEqual([0, 0], rdd.histogram((0, 4, 10))[1])
|
[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
|
|
|
|
|
|
|
# in range with one bucket
|
|
|
|
rdd = self.sc.parallelize(range(1, 5))
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual([4], rdd.histogram([0, 10])[1])
|
|
|
|
self.assertEqual([3, 1], rdd.histogram([0, 4, 10])[1])
|
[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
|
|
|
|
|
|
|
# in range with one bucket exact match
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual([4], rdd.histogram([1, 4])[1])
|
[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
|
|
|
|
|
|
|
# out of range with two buckets
|
|
|
|
rdd = self.sc.parallelize([10.01, -0.01])
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual([0, 0], rdd.histogram([0, 5, 10])[1])
|
[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
|
|
|
|
|
|
|
# out of range with two uneven buckets
|
|
|
|
rdd = self.sc.parallelize([10.01, -0.01])
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual([0, 0], rdd.histogram([0, 4, 10])[1])
|
[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
|
|
|
|
|
|
|
# in range with two buckets
|
|
|
|
rdd = self.sc.parallelize([1, 2, 3, 5, 6])
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual([3, 2], rdd.histogram([0, 5, 10])[1])
|
[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
|
|
|
|
|
|
|
# in range with two bucket and None
|
|
|
|
rdd = self.sc.parallelize([1, 2, 3, 5, 6, None, float('nan')])
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual([3, 2], rdd.histogram([0, 5, 10])[1])
|
[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
|
|
|
|
|
|
|
# in range with two uneven buckets
|
|
|
|
rdd = self.sc.parallelize([1, 2, 3, 5, 6])
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual([3, 2], rdd.histogram([0, 5, 11])[1])
|
[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
|
|
|
|
|
|
|
# mixed range with two uneven buckets
|
|
|
|
rdd = self.sc.parallelize([-0.01, 0.0, 1, 2, 3, 5, 6, 11.0, 11.01])
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual([4, 3], rdd.histogram([0, 5, 11])[1])
|
[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
|
|
|
|
|
|
|
# mixed range with four uneven buckets
|
|
|
|
rdd = self.sc.parallelize([-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0, 200.0, 200.1])
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual([4, 2, 1, 3], rdd.histogram([0.0, 5.0, 11.0, 12.0, 200.0])[1])
|
[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
|
|
|
|
|
|
|
# mixed range with uneven buckets and NaN
|
|
|
|
rdd = self.sc.parallelize([-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0,
|
|
|
|
199.0, 200.0, 200.1, None, float('nan')])
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual([4, 2, 1, 3], rdd.histogram([0.0, 5.0, 11.0, 12.0, 200.0])[1])
|
[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
|
|
|
|
|
|
|
# out of range with infinite buckets
|
|
|
|
rdd = self.sc.parallelize([10.01, -0.01, float('nan'), float("inf")])
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual([1, 2], rdd.histogram([float('-inf'), 0, float('inf')])[1])
|
[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
|
|
|
|
|
|
|
# invalid buckets
|
|
|
|
self.assertRaises(ValueError, lambda: rdd.histogram([]))
|
|
|
|
self.assertRaises(ValueError, lambda: rdd.histogram([1]))
|
|
|
|
self.assertRaises(ValueError, lambda: rdd.histogram(0))
|
|
|
|
self.assertRaises(TypeError, lambda: rdd.histogram({}))
|
|
|
|
|
|
|
|
# without buckets
|
|
|
|
rdd = self.sc.parallelize(range(1, 5))
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual(([1, 4], [4]), rdd.histogram(1))
|
[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
|
|
|
|
|
|
|
# without buckets single element
|
|
|
|
rdd = self.sc.parallelize([1])
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual(([1, 1], [1]), rdd.histogram(1))
|
[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
|
|
|
|
|
|
|
# without bucket no range
|
|
|
|
rdd = self.sc.parallelize([1] * 4)
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual(([1, 1], [4]), rdd.histogram(1))
|
[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
|
|
|
|
|
|
|
# without buckets basic two
|
|
|
|
rdd = self.sc.parallelize(range(1, 5))
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual(([1, 2.5, 4], [2, 2]), rdd.histogram(2))
|
[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
|
|
|
|
|
|
|
# without buckets with more requested than elements
|
|
|
|
rdd = self.sc.parallelize([1, 2])
|
|
|
|
buckets = [1 + 0.2 * i for i in range(6)]
|
|
|
|
hist = [1, 0, 0, 0, 1]
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual((buckets, hist), rdd.histogram(5))
|
[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
|
|
|
|
|
|
|
# invalid RDDs
|
|
|
|
rdd = self.sc.parallelize([1, float('inf')])
|
|
|
|
self.assertRaises(ValueError, lambda: rdd.histogram(2))
|
|
|
|
rdd = self.sc.parallelize([float('nan')])
|
|
|
|
self.assertRaises(ValueError, lambda: rdd.histogram(2))
|
|
|
|
|
|
|
|
# string
|
|
|
|
rdd = self.sc.parallelize(["ab", "ac", "b", "bd", "ef"], 2)
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual([2, 2], rdd.histogram(["a", "b", "c"])[1])
|
|
|
|
self.assertEqual((["ab", "ef"], [5]), rdd.histogram(1))
|
[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
|
|
|
self.assertRaises(TypeError, lambda: rdd.histogram(2))
|
|
|
|
|
2017-07-10 21:56:54 -04:00
|
|
|
def test_repartitionAndSortWithinPartitions_asc(self):
|
2014-09-08 14:20:00 -04:00
|
|
|
rdd = self.sc.parallelize([(0, 5), (3, 8), (2, 6), (0, 8), (3, 8), (1, 3)], 2)
|
|
|
|
|
2017-07-10 21:56:54 -04:00
|
|
|
repartitioned = rdd.repartitionAndSortWithinPartitions(2, lambda key: key % 2, True)
|
2014-09-08 14:20:00 -04:00
|
|
|
partitions = repartitioned.glom().collect()
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual(partitions[0], [(0, 5), (0, 8), (2, 6)])
|
|
|
|
self.assertEqual(partitions[1], [(1, 3), (3, 8), (3, 8)])
|
2014-09-08 14:20:00 -04:00
|
|
|
|
2017-07-10 21:56:54 -04:00
|
|
|
def test_repartitionAndSortWithinPartitions_desc(self):
|
|
|
|
rdd = self.sc.parallelize([(0, 5), (3, 8), (2, 6), (0, 8), (3, 8), (1, 3)], 2)
|
|
|
|
|
|
|
|
repartitioned = rdd.repartitionAndSortWithinPartitions(2, lambda key: key % 2, False)
|
|
|
|
partitions = repartitioned.glom().collect()
|
|
|
|
self.assertEqual(partitions[0], [(2, 6), (0, 5), (0, 8)])
|
|
|
|
self.assertEqual(partitions[1], [(3, 8), (3, 8), (1, 3)])
|
|
|
|
|
2016-10-11 14:43:24 -04:00
|
|
|
def test_repartition_no_skewed(self):
|
|
|
|
num_partitions = 20
|
|
|
|
a = self.sc.parallelize(range(int(1000)), 2)
|
|
|
|
l = a.repartition(num_partitions).glom().map(len).collect()
|
|
|
|
zeros = len([x for x in l if x == 0])
|
|
|
|
self.assertTrue(zeros == 0)
|
|
|
|
l = a.coalesce(num_partitions, True).glom().map(len).collect()
|
|
|
|
zeros = len([x for x in l if x == 0])
|
|
|
|
self.assertTrue(zeros == 0)
|
|
|
|
|
[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
|
|
|
def test_repartition_on_textfile(self):
|
|
|
|
path = os.path.join(SPARK_HOME, "python/test_support/hello/hello.txt")
|
|
|
|
rdd = self.sc.textFile(path)
|
|
|
|
result = rdd.repartition(1).collect()
|
|
|
|
self.assertEqual(u"Hello World!", result[0])
|
|
|
|
|
2014-09-16 14:39:57 -04:00
|
|
|
def test_distinct(self):
|
|
|
|
rdd = self.sc.parallelize((1, 2, 3)*10, 10)
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual(rdd.getNumPartitions(), 10)
|
|
|
|
self.assertEqual(rdd.distinct().count(), 3)
|
2014-09-16 14:39:57 -04:00
|
|
|
result = rdd.distinct(5)
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual(result.getNumPartitions(), 5)
|
|
|
|
self.assertEqual(result.count(), 3)
|
2014-09-16 14:39:57 -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 test_external_group_by_key(self):
|
2015-04-16 19:20:57 -04:00
|
|
|
self.sc._conf.set("spark.python.worker.memory", "1m")
|
[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
|
|
|
N = 200001
|
[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
|
|
|
kv = self.sc.parallelize(xrange(N)).map(lambda x: (x % 3, x))
|
[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
|
|
|
gkv = kv.groupByKey().cache()
|
|
|
|
self.assertEqual(3, gkv.count())
|
2015-04-16 19:20:57 -04:00
|
|
|
filtered = gkv.filter(lambda kv: kv[0] == 1)
|
[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
|
|
|
self.assertEqual(1, filtered.count())
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual([(1, N // 3)], filtered.mapValues(len).collect())
|
|
|
|
self.assertEqual([(N // 3, N // 3)],
|
[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
|
|
|
filtered.values().map(lambda x: (len(x), len(list(x)))).collect())
|
|
|
|
result = filtered.collect()[0][1]
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual(N // 3, len(result))
|
|
|
|
self.assertTrue(isinstance(result.data, shuffle.ExternalListOfList))
|
[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
|
|
|
|
2014-11-07 23:53:03 -05:00
|
|
|
def test_sort_on_empty_rdd(self):
|
|
|
|
self.assertEqual([], self.sc.parallelize(zip([], [])).sortByKey().collect())
|
|
|
|
|
2014-11-03 15:24:24 -05:00
|
|
|
def test_sample(self):
|
|
|
|
rdd = self.sc.parallelize(range(0, 100), 4)
|
|
|
|
wo = rdd.sample(False, 0.1, 2).collect()
|
|
|
|
wo_dup = rdd.sample(False, 0.1, 2).collect()
|
|
|
|
self.assertSetEqual(set(wo), set(wo_dup))
|
|
|
|
wr = rdd.sample(True, 0.2, 5).collect()
|
|
|
|
wr_dup = rdd.sample(True, 0.2, 5).collect()
|
|
|
|
self.assertSetEqual(set(wr), set(wr_dup))
|
|
|
|
wo_s10 = rdd.sample(False, 0.3, 10).collect()
|
|
|
|
wo_s20 = rdd.sample(False, 0.3, 20).collect()
|
|
|
|
self.assertNotEqual(set(wo_s10), set(wo_s20))
|
|
|
|
wr_s11 = rdd.sample(True, 0.4, 11).collect()
|
|
|
|
wr_s21 = rdd.sample(True, 0.4, 21).collect()
|
|
|
|
self.assertNotEqual(set(wr_s11), set(wr_s21))
|
|
|
|
|
[SPARK-5154] [PySpark] [Streaming] Kafka streaming support in Python
This PR brings the Python API for Spark Streaming Kafka data source.
```
class KafkaUtils(__builtin__.object)
| Static methods defined here:
|
| createStream(ssc, zkQuorum, groupId, topics, storageLevel=StorageLevel(True, True, False, False,
2), keyDecoder=<function utf8_decoder>, valueDecoder=<function utf8_decoder>)
| Create an input stream that pulls messages from a Kafka Broker.
|
| :param ssc: StreamingContext object
| :param zkQuorum: Zookeeper quorum (hostname:port,hostname:port,..).
| :param groupId: The group id for this consumer.
| :param topics: Dict of (topic_name -> numPartitions) to consume.
| Each partition is consumed in its own thread.
| :param storageLevel: RDD storage level.
| :param keyDecoder: A function used to decode key
| :param valueDecoder: A function used to decode value
| :return: A DStream object
```
run the example:
```
bin/spark-submit --driver-class-path external/kafka-assembly/target/scala-*/spark-streaming-kafka-assembly-*.jar examples/src/main/python/streaming/kafka_wordcount.py localhost:2181 test
```
Author: Davies Liu <davies@databricks.com>
Author: Tathagata Das <tdas@databricks.com>
Closes #3715 from davies/kafka and squashes the following commits:
d93bfe0 [Davies Liu] Update make-distribution.sh
4280d04 [Davies Liu] address comments
e6d0427 [Davies Liu] Merge branch 'master' of github.com:apache/spark into kafka
f257071 [Davies Liu] add tests for null in RDD
23b039a [Davies Liu] address comments
9af51c4 [Davies Liu] Merge branch 'kafka' of github.com:davies/spark into kafka
a74da87 [Davies Liu] address comments
dc1eed0 [Davies Liu] Update kafka_wordcount.py
31e2317 [Davies Liu] Update kafka_wordcount.py
370ba61 [Davies Liu] Update kafka.py
97386b3 [Davies Liu] address comment
2c567a5 [Davies Liu] update logging and comment
33730d1 [Davies Liu] Merge branch 'master' of github.com:apache/spark into kafka
adeeb38 [Davies Liu] Merge pull request #3 from tdas/kafka-python-api
aea8953 [Tathagata Das] Kafka-assembly for Python API
eea16a7 [Davies Liu] refactor
f6ce899 [Davies Liu] add example and fix bugs
98c8d17 [Davies Liu] fix python style
5697a01 [Davies Liu] bypass decoder in scala
048dbe6 [Davies Liu] fix python style
75d485e [Davies Liu] add mqtt
07923c4 [Davies Liu] support kafka in Python
2015-02-02 22:16:27 -05:00
|
|
|
def test_null_in_rdd(self):
|
|
|
|
jrdd = self.sc._jvm.PythonUtils.generateRDDWithNull(self.sc._jsc)
|
|
|
|
rdd = RDD(jrdd, self.sc, UTF8Deserializer())
|
|
|
|
self.assertEqual([u"a", None, u"b"], rdd.collect())
|
|
|
|
rdd = RDD(jrdd, self.sc, NoOpSerializer())
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual([b"a", None, b"b"], rdd.collect())
|
[SPARK-5154] [PySpark] [Streaming] Kafka streaming support in Python
This PR brings the Python API for Spark Streaming Kafka data source.
```
class KafkaUtils(__builtin__.object)
| Static methods defined here:
|
| createStream(ssc, zkQuorum, groupId, topics, storageLevel=StorageLevel(True, True, False, False,
2), keyDecoder=<function utf8_decoder>, valueDecoder=<function utf8_decoder>)
| Create an input stream that pulls messages from a Kafka Broker.
|
| :param ssc: StreamingContext object
| :param zkQuorum: Zookeeper quorum (hostname:port,hostname:port,..).
| :param groupId: The group id for this consumer.
| :param topics: Dict of (topic_name -> numPartitions) to consume.
| Each partition is consumed in its own thread.
| :param storageLevel: RDD storage level.
| :param keyDecoder: A function used to decode key
| :param valueDecoder: A function used to decode value
| :return: A DStream object
```
run the example:
```
bin/spark-submit --driver-class-path external/kafka-assembly/target/scala-*/spark-streaming-kafka-assembly-*.jar examples/src/main/python/streaming/kafka_wordcount.py localhost:2181 test
```
Author: Davies Liu <davies@databricks.com>
Author: Tathagata Das <tdas@databricks.com>
Closes #3715 from davies/kafka and squashes the following commits:
d93bfe0 [Davies Liu] Update make-distribution.sh
4280d04 [Davies Liu] address comments
e6d0427 [Davies Liu] Merge branch 'master' of github.com:apache/spark into kafka
f257071 [Davies Liu] add tests for null in RDD
23b039a [Davies Liu] address comments
9af51c4 [Davies Liu] Merge branch 'kafka' of github.com:davies/spark into kafka
a74da87 [Davies Liu] address comments
dc1eed0 [Davies Liu] Update kafka_wordcount.py
31e2317 [Davies Liu] Update kafka_wordcount.py
370ba61 [Davies Liu] Update kafka.py
97386b3 [Davies Liu] address comment
2c567a5 [Davies Liu] update logging and comment
33730d1 [Davies Liu] Merge branch 'master' of github.com:apache/spark into kafka
adeeb38 [Davies Liu] Merge pull request #3 from tdas/kafka-python-api
aea8953 [Tathagata Das] Kafka-assembly for Python API
eea16a7 [Davies Liu] refactor
f6ce899 [Davies Liu] add example and fix bugs
98c8d17 [Davies Liu] fix python style
5697a01 [Davies Liu] bypass decoder in scala
048dbe6 [Davies Liu] fix python style
75d485e [Davies Liu] add mqtt
07923c4 [Davies Liu] support kafka in Python
2015-02-02 22:16:27 -05:00
|
|
|
|
[SPARK-5361]Multiple Java RDD <-> Python RDD conversions not working correctly
This is found through reading RDD from `sc.newAPIHadoopRDD` and writing it back using `rdd.saveAsNewAPIHadoopFile` in pyspark.
It turns out that whenever there are multiple RDD conversions from JavaRDD to PythonRDD then back to JavaRDD, the exception below happens:
```
15/01/16 10:28:31 ERROR Executor: Exception in task 0.0 in stage 3.0 (TID 7)
java.lang.ClassCastException: [Ljava.lang.Object; cannot be cast to java.util.ArrayList
at org.apache.spark.api.python.SerDeUtil$$anonfun$pythonToJava$1$$anonfun$apply$1.apply(SerDeUtil.scala:157)
at org.apache.spark.api.python.SerDeUtil$$anonfun$pythonToJava$1$$anonfun$apply$1.apply(SerDeUtil.scala:153)
at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:371)
at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:308)
```
The test case code below reproduces it:
```
from pyspark.rdd import RDD
dl = [
(u'2', {u'director': u'David Lean'}),
(u'7', {u'director': u'Andrew Dominik'})
]
dl_rdd = sc.parallelize(dl)
tmp = dl_rdd._to_java_object_rdd()
tmp2 = sc._jvm.SerDe.javaToPython(tmp)
t = RDD(tmp2, sc)
t.count()
tmp = t._to_java_object_rdd()
tmp2 = sc._jvm.SerDe.javaToPython(tmp)
t = RDD(tmp2, sc)
t.count() # it blows up here during the 2nd time of conversion
```
Author: Winston Chen <wchen@quid.com>
Closes #4146 from wingchen/master and squashes the following commits:
903df7d [Winston Chen] SPARK-5361, update to toSeq based on the PR
5d90a83 [Winston Chen] SPARK-5361, make python pretty, so to pass PEP 8 checks
126be6b [Winston Chen] SPARK-5361, add in test case
4cf1187 [Winston Chen] SPARK-5361, add in test case
9f1a097 [Winston Chen] add in tuple handling while converting form python RDD back to JavaRDD
2015-01-28 14:08:44 -05:00
|
|
|
def test_multiple_python_java_RDD_conversions(self):
|
|
|
|
# Regression test for SPARK-5361
|
|
|
|
data = [
|
|
|
|
(u'1', {u'director': u'David Lean'}),
|
|
|
|
(u'2', {u'director': u'Andrew Dominik'})
|
|
|
|
]
|
|
|
|
data_rdd = self.sc.parallelize(data)
|
|
|
|
data_java_rdd = data_rdd._to_java_object_rdd()
|
2016-05-24 13:10:41 -04:00
|
|
|
data_python_rdd = self.sc._jvm.SerDeUtil.javaToPython(data_java_rdd)
|
[SPARK-5361]Multiple Java RDD <-> Python RDD conversions not working correctly
This is found through reading RDD from `sc.newAPIHadoopRDD` and writing it back using `rdd.saveAsNewAPIHadoopFile` in pyspark.
It turns out that whenever there are multiple RDD conversions from JavaRDD to PythonRDD then back to JavaRDD, the exception below happens:
```
15/01/16 10:28:31 ERROR Executor: Exception in task 0.0 in stage 3.0 (TID 7)
java.lang.ClassCastException: [Ljava.lang.Object; cannot be cast to java.util.ArrayList
at org.apache.spark.api.python.SerDeUtil$$anonfun$pythonToJava$1$$anonfun$apply$1.apply(SerDeUtil.scala:157)
at org.apache.spark.api.python.SerDeUtil$$anonfun$pythonToJava$1$$anonfun$apply$1.apply(SerDeUtil.scala:153)
at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:371)
at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:308)
```
The test case code below reproduces it:
```
from pyspark.rdd import RDD
dl = [
(u'2', {u'director': u'David Lean'}),
(u'7', {u'director': u'Andrew Dominik'})
]
dl_rdd = sc.parallelize(dl)
tmp = dl_rdd._to_java_object_rdd()
tmp2 = sc._jvm.SerDe.javaToPython(tmp)
t = RDD(tmp2, sc)
t.count()
tmp = t._to_java_object_rdd()
tmp2 = sc._jvm.SerDe.javaToPython(tmp)
t = RDD(tmp2, sc)
t.count() # it blows up here during the 2nd time of conversion
```
Author: Winston Chen <wchen@quid.com>
Closes #4146 from wingchen/master and squashes the following commits:
903df7d [Winston Chen] SPARK-5361, update to toSeq based on the PR
5d90a83 [Winston Chen] SPARK-5361, make python pretty, so to pass PEP 8 checks
126be6b [Winston Chen] SPARK-5361, add in test case
4cf1187 [Winston Chen] SPARK-5361, add in test case
9f1a097 [Winston Chen] add in tuple handling while converting form python RDD back to JavaRDD
2015-01-28 14:08:44 -05:00
|
|
|
converted_rdd = RDD(data_python_rdd, self.sc)
|
|
|
|
self.assertEqual(2, converted_rdd.count())
|
|
|
|
|
|
|
|
# conversion between python and java RDD threw exceptions
|
|
|
|
data_java_rdd = converted_rdd._to_java_object_rdd()
|
2016-05-24 13:10:41 -04:00
|
|
|
data_python_rdd = self.sc._jvm.SerDeUtil.javaToPython(data_java_rdd)
|
[SPARK-5361]Multiple Java RDD <-> Python RDD conversions not working correctly
This is found through reading RDD from `sc.newAPIHadoopRDD` and writing it back using `rdd.saveAsNewAPIHadoopFile` in pyspark.
It turns out that whenever there are multiple RDD conversions from JavaRDD to PythonRDD then back to JavaRDD, the exception below happens:
```
15/01/16 10:28:31 ERROR Executor: Exception in task 0.0 in stage 3.0 (TID 7)
java.lang.ClassCastException: [Ljava.lang.Object; cannot be cast to java.util.ArrayList
at org.apache.spark.api.python.SerDeUtil$$anonfun$pythonToJava$1$$anonfun$apply$1.apply(SerDeUtil.scala:157)
at org.apache.spark.api.python.SerDeUtil$$anonfun$pythonToJava$1$$anonfun$apply$1.apply(SerDeUtil.scala:153)
at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:371)
at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:308)
```
The test case code below reproduces it:
```
from pyspark.rdd import RDD
dl = [
(u'2', {u'director': u'David Lean'}),
(u'7', {u'director': u'Andrew Dominik'})
]
dl_rdd = sc.parallelize(dl)
tmp = dl_rdd._to_java_object_rdd()
tmp2 = sc._jvm.SerDe.javaToPython(tmp)
t = RDD(tmp2, sc)
t.count()
tmp = t._to_java_object_rdd()
tmp2 = sc._jvm.SerDe.javaToPython(tmp)
t = RDD(tmp2, sc)
t.count() # it blows up here during the 2nd time of conversion
```
Author: Winston Chen <wchen@quid.com>
Closes #4146 from wingchen/master and squashes the following commits:
903df7d [Winston Chen] SPARK-5361, update to toSeq based on the PR
5d90a83 [Winston Chen] SPARK-5361, make python pretty, so to pass PEP 8 checks
126be6b [Winston Chen] SPARK-5361, add in test case
4cf1187 [Winston Chen] SPARK-5361, add in test case
9f1a097 [Winston Chen] add in tuple handling while converting form python RDD back to JavaRDD
2015-01-28 14:08:44 -05:00
|
|
|
converted_rdd = RDD(data_python_rdd, self.sc)
|
|
|
|
self.assertEqual(2, converted_rdd.count())
|
|
|
|
|
2015-02-17 19:54:57 -05:00
|
|
|
def test_narrow_dependency_in_join(self):
|
|
|
|
rdd = self.sc.parallelize(range(10)).map(lambda x: (x, x))
|
|
|
|
parted = rdd.partitionBy(2)
|
|
|
|
self.assertEqual(2, parted.union(parted).getNumPartitions())
|
|
|
|
self.assertEqual(rdd.getNumPartitions() + 2, parted.union(rdd).getNumPartitions())
|
|
|
|
self.assertEqual(rdd.getNumPartitions() + 2, rdd.union(parted).getNumPartitions())
|
|
|
|
|
|
|
|
tracker = self.sc.statusTracker()
|
|
|
|
|
[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
|
|
|
self.sc.setJobGroup("test1", "test", True)
|
2015-02-17 19:54:57 -05:00
|
|
|
d = sorted(parted.join(parted).collect())
|
|
|
|
self.assertEqual(10, len(d))
|
|
|
|
self.assertEqual((0, (0, 0)), d[0])
|
|
|
|
jobId = tracker.getJobIdsForGroup("test1")[0]
|
|
|
|
self.assertEqual(2, len(tracker.getJobInfo(jobId).stageIds))
|
|
|
|
|
|
|
|
self.sc.setJobGroup("test2", "test", True)
|
|
|
|
d = sorted(parted.join(rdd).collect())
|
|
|
|
self.assertEqual(10, len(d))
|
|
|
|
self.assertEqual((0, (0, 0)), d[0])
|
|
|
|
jobId = tracker.getJobIdsForGroup("test2")[0]
|
|
|
|
self.assertEqual(3, len(tracker.getJobInfo(jobId).stageIds))
|
|
|
|
|
|
|
|
self.sc.setJobGroup("test3", "test", True)
|
|
|
|
d = sorted(parted.cogroup(parted).collect())
|
|
|
|
self.assertEqual(10, len(d))
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual([[0], [0]], list(map(list, d[0][1])))
|
2015-02-17 19:54:57 -05:00
|
|
|
jobId = tracker.getJobIdsForGroup("test3")[0]
|
|
|
|
self.assertEqual(2, len(tracker.getJobInfo(jobId).stageIds))
|
|
|
|
|
|
|
|
self.sc.setJobGroup("test4", "test", True)
|
|
|
|
d = sorted(parted.cogroup(rdd).collect())
|
|
|
|
self.assertEqual(10, len(d))
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual([[0], [0]], list(map(list, d[0][1])))
|
2015-02-17 19:54:57 -05:00
|
|
|
jobId = tracker.getJobIdsForGroup("test4")[0]
|
|
|
|
self.assertEqual(3, len(tracker.getJobInfo(jobId).stageIds))
|
|
|
|
|
2015-03-12 04:34:38 -04:00
|
|
|
# Regression test for SPARK-6294
|
|
|
|
def test_take_on_jrdd(self):
|
[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
|
|
|
rdd = self.sc.parallelize(xrange(1 << 20)).map(lambda x: str(x))
|
2015-03-12 04:34:38 -04:00
|
|
|
rdd._jrdd.first()
|
|
|
|
|
2015-04-10 16:50:32 -04:00
|
|
|
def test_sortByKey_uses_all_partitions_not_only_first_and_last(self):
|
|
|
|
# Regression test for SPARK-5969
|
|
|
|
seq = [(i * 59 % 101, i) for i in range(101)] # unsorted sequence
|
|
|
|
rdd = self.sc.parallelize(seq)
|
|
|
|
for ascending in [True, False]:
|
|
|
|
sort = rdd.sortByKey(ascending=ascending, numPartitions=5)
|
|
|
|
self.assertEqual(sort.collect(), sorted(seq, reverse=not ascending))
|
|
|
|
sizes = sort.glom().map(len).collect()
|
|
|
|
for size in sizes:
|
|
|
|
self.assertGreater(size, 0)
|
|
|
|
|
2015-07-10 22:29:32 -04:00
|
|
|
def test_pipe_functions(self):
|
|
|
|
data = ['1', '2', '3']
|
|
|
|
rdd = self.sc.parallelize(data)
|
|
|
|
with QuietTest(self.sc):
|
|
|
|
self.assertEqual([], rdd.pipe('cc').collect())
|
|
|
|
self.assertRaises(Py4JJavaError, rdd.pipe('cc', checkCode=True).collect)
|
|
|
|
result = rdd.pipe('cat').collect()
|
|
|
|
result.sort()
|
2015-07-15 11:25:53 -04:00
|
|
|
for x, y in zip(data, result):
|
|
|
|
self.assertEqual(x, y)
|
2015-07-10 22:29:32 -04:00
|
|
|
self.assertRaises(Py4JJavaError, rdd.pipe('grep 4', checkCode=True).collect)
|
|
|
|
self.assertEqual([], rdd.pipe('grep 4').collect())
|
|
|
|
|
2018-01-21 20:43:12 -05:00
|
|
|
def test_pipe_unicode(self):
|
|
|
|
# Regression test for SPARK-20947
|
|
|
|
data = [u'\u6d4b\u8bd5', '1']
|
|
|
|
rdd = self.sc.parallelize(data)
|
|
|
|
result = rdd.pipe('cat').collect()
|
|
|
|
self.assertEqual(data, result)
|
|
|
|
|
[SPARK-23754][PYTHON][FOLLOWUP] Move UDF stop iteration wrapping from driver to executor
## What changes were proposed in this pull request?
SPARK-23754 was fixed in #21383 by changing the UDF code to wrap the user function, but this required a hack to save its argspec. This PR reverts this change and fixes the `StopIteration` bug in the worker
## How does this work?
The root of the problem is that when an user-supplied function raises a `StopIteration`, pyspark might stop processing data, if this function is used in a for-loop. The solution is to catch `StopIteration`s exceptions and re-raise them as `RuntimeError`s, so that the execution fails and the error is reported to the user. This is done using the `fail_on_stopiteration` wrapper, in different ways depending on where the function is used:
- In RDDs, the user function is wrapped in the driver, because this function is also called in the driver itself.
- In SQL UDFs, the function is wrapped in the worker, since all processing happens there. Moreover, the worker needs the signature of the user function, which is lost when wrapping it, but passing this signature to the worker requires a not so nice hack.
## How was this patch tested?
Same tests, plus tests for pandas UDFs
Author: edorigatti <emilio.dorigatti@gmail.com>
Closes #21467 from e-dorigatti/fix_udf_hack.
2018-06-10 22:15:42 -04:00
|
|
|
def test_stopiteration_in_user_code(self):
|
2018-05-30 06:11:33 -04:00
|
|
|
|
|
|
|
def stopit(*x):
|
|
|
|
raise StopIteration()
|
|
|
|
|
|
|
|
seq_rdd = self.sc.parallelize(range(10))
|
|
|
|
keyed_rdd = self.sc.parallelize((x % 2, x) for x in range(10))
|
[SPARK-23754][PYTHON][FOLLOWUP] Move UDF stop iteration wrapping from driver to executor
## What changes were proposed in this pull request?
SPARK-23754 was fixed in #21383 by changing the UDF code to wrap the user function, but this required a hack to save its argspec. This PR reverts this change and fixes the `StopIteration` bug in the worker
## How does this work?
The root of the problem is that when an user-supplied function raises a `StopIteration`, pyspark might stop processing data, if this function is used in a for-loop. The solution is to catch `StopIteration`s exceptions and re-raise them as `RuntimeError`s, so that the execution fails and the error is reported to the user. This is done using the `fail_on_stopiteration` wrapper, in different ways depending on where the function is used:
- In RDDs, the user function is wrapped in the driver, because this function is also called in the driver itself.
- In SQL UDFs, the function is wrapped in the worker, since all processing happens there. Moreover, the worker needs the signature of the user function, which is lost when wrapping it, but passing this signature to the worker requires a not so nice hack.
## How was this patch tested?
Same tests, plus tests for pandas UDFs
Author: edorigatti <emilio.dorigatti@gmail.com>
Closes #21467 from e-dorigatti/fix_udf_hack.
2018-06-10 22:15:42 -04:00
|
|
|
msg = "Caught StopIteration thrown from user's code; failing the task"
|
|
|
|
|
|
|
|
self.assertRaisesRegexp(Py4JJavaError, msg, seq_rdd.map(stopit).collect)
|
|
|
|
self.assertRaisesRegexp(Py4JJavaError, msg, seq_rdd.filter(stopit).collect)
|
|
|
|
self.assertRaisesRegexp(Py4JJavaError, msg, seq_rdd.foreach, stopit)
|
|
|
|
self.assertRaisesRegexp(Py4JJavaError, msg, seq_rdd.reduce, stopit)
|
|
|
|
self.assertRaisesRegexp(Py4JJavaError, msg, seq_rdd.fold, 0, stopit)
|
|
|
|
self.assertRaisesRegexp(Py4JJavaError, msg, seq_rdd.foreach, stopit)
|
|
|
|
self.assertRaisesRegexp(Py4JJavaError, msg,
|
|
|
|
seq_rdd.cartesian(seq_rdd).flatMap(stopit).collect)
|
|
|
|
|
|
|
|
# these methods call the user function both in the driver and in the executor
|
|
|
|
# the exception raised is different according to where the StopIteration happens
|
|
|
|
# RuntimeError is raised if in the driver
|
|
|
|
# Py4JJavaError is raised if in the executor (wraps the RuntimeError raised in the worker)
|
|
|
|
self.assertRaisesRegexp((Py4JJavaError, RuntimeError), msg,
|
|
|
|
keyed_rdd.reduceByKeyLocally, stopit)
|
|
|
|
self.assertRaisesRegexp((Py4JJavaError, RuntimeError), msg,
|
|
|
|
seq_rdd.aggregate, 0, stopit, lambda *x: 1)
|
|
|
|
self.assertRaisesRegexp((Py4JJavaError, RuntimeError), msg,
|
|
|
|
seq_rdd.aggregate, 0, lambda *x: 1, stopit)
|
2018-05-30 06:11:33 -04:00
|
|
|
|
2014-07-22 01:30:53 -04:00
|
|
|
|
2014-10-06 17:07:53 -04:00
|
|
|
class ProfilerTests(PySparkTestCase):
|
[SPARK-3478] [PySpark] Profile the Python tasks
This patch add profiling support for PySpark, it will show the profiling results
before the driver exits, here is one example:
```
============================================================
Profile of RDD<id=3>
============================================================
5146507 function calls (5146487 primitive calls) in 71.094 seconds
Ordered by: internal time, cumulative time
ncalls tottime percall cumtime percall filename:lineno(function)
5144576 68.331 0.000 68.331 0.000 statcounter.py:44(merge)
20 2.735 0.137 71.071 3.554 statcounter.py:33(__init__)
20 0.017 0.001 0.017 0.001 {cPickle.dumps}
1024 0.003 0.000 0.003 0.000 t.py:16(<lambda>)
20 0.001 0.000 0.001 0.000 {reduce}
21 0.001 0.000 0.001 0.000 {cPickle.loads}
20 0.001 0.000 0.001 0.000 copy_reg.py:95(_slotnames)
41 0.001 0.000 0.001 0.000 serializers.py:461(read_int)
40 0.001 0.000 0.002 0.000 serializers.py:179(_batched)
62 0.000 0.000 0.000 0.000 {method 'read' of 'file' objects}
20 0.000 0.000 71.072 3.554 rdd.py:863(<lambda>)
20 0.000 0.000 0.001 0.000 serializers.py:198(load_stream)
40/20 0.000 0.000 71.072 3.554 rdd.py:2093(pipeline_func)
41 0.000 0.000 0.002 0.000 serializers.py:130(load_stream)
40 0.000 0.000 71.072 1.777 rdd.py:304(func)
20 0.000 0.000 71.094 3.555 worker.py:82(process)
```
Also, use can show profile result manually by `sc.show_profiles()` or dump it into disk
by `sc.dump_profiles(path)`, such as
```python
>>> sc._conf.set("spark.python.profile", "true")
>>> rdd = sc.parallelize(range(100)).map(str)
>>> rdd.count()
100
>>> sc.show_profiles()
============================================================
Profile of RDD<id=1>
============================================================
284 function calls (276 primitive calls) in 0.001 seconds
Ordered by: internal time, cumulative time
ncalls tottime percall cumtime percall filename:lineno(function)
4 0.000 0.000 0.000 0.000 serializers.py:198(load_stream)
4 0.000 0.000 0.000 0.000 {reduce}
12/4 0.000 0.000 0.001 0.000 rdd.py:2092(pipeline_func)
4 0.000 0.000 0.000 0.000 {cPickle.loads}
4 0.000 0.000 0.000 0.000 {cPickle.dumps}
104 0.000 0.000 0.000 0.000 rdd.py:852(<genexpr>)
8 0.000 0.000 0.000 0.000 serializers.py:461(read_int)
12 0.000 0.000 0.000 0.000 rdd.py:303(func)
```
The profiling is disabled by default, can be enabled by "spark.python.profile=true".
Also, users can dump the results into disks automatically for future analysis, by "spark.python.profile.dump=path_to_dump"
This is bugfix of #2351 cc JoshRosen
Author: Davies Liu <davies.liu@gmail.com>
Closes #2556 from davies/profiler and squashes the following commits:
e68df5a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
858e74c [Davies Liu] compatitable with python 2.6
7ef2aa0 [Davies Liu] bugfix, add tests for show_profiles and dump_profiles()
2b0daf2 [Davies Liu] fix docs
7a56c24 [Davies Liu] bugfix
cba9463 [Davies Liu] move show_profiles and dump_profiles to SparkContext
fb9565b [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
116d52a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
09d02c3 [Davies Liu] Merge branch 'master' into profiler
c23865c [Davies Liu] Merge branch 'master' into profiler
15d6f18 [Davies Liu] add docs for two configs
dadee1a [Davies Liu] add docs string and clear profiles after show or dump
4f8309d [Davies Liu] address comment, add tests
0a5b6eb [Davies Liu] fix Python UDF
4b20494 [Davies Liu] add profile for python
2014-09-30 21:24:57 -04:00
|
|
|
|
|
|
|
def setUp(self):
|
|
|
|
self._old_sys_path = list(sys.path)
|
|
|
|
class_name = self.__class__.__name__
|
|
|
|
conf = SparkConf().set("spark.python.profile", "true")
|
2014-11-04 02:56:14 -05:00
|
|
|
self.sc = SparkContext('local[4]', class_name, conf=conf)
|
[SPARK-3478] [PySpark] Profile the Python tasks
This patch add profiling support for PySpark, it will show the profiling results
before the driver exits, here is one example:
```
============================================================
Profile of RDD<id=3>
============================================================
5146507 function calls (5146487 primitive calls) in 71.094 seconds
Ordered by: internal time, cumulative time
ncalls tottime percall cumtime percall filename:lineno(function)
5144576 68.331 0.000 68.331 0.000 statcounter.py:44(merge)
20 2.735 0.137 71.071 3.554 statcounter.py:33(__init__)
20 0.017 0.001 0.017 0.001 {cPickle.dumps}
1024 0.003 0.000 0.003 0.000 t.py:16(<lambda>)
20 0.001 0.000 0.001 0.000 {reduce}
21 0.001 0.000 0.001 0.000 {cPickle.loads}
20 0.001 0.000 0.001 0.000 copy_reg.py:95(_slotnames)
41 0.001 0.000 0.001 0.000 serializers.py:461(read_int)
40 0.001 0.000 0.002 0.000 serializers.py:179(_batched)
62 0.000 0.000 0.000 0.000 {method 'read' of 'file' objects}
20 0.000 0.000 71.072 3.554 rdd.py:863(<lambda>)
20 0.000 0.000 0.001 0.000 serializers.py:198(load_stream)
40/20 0.000 0.000 71.072 3.554 rdd.py:2093(pipeline_func)
41 0.000 0.000 0.002 0.000 serializers.py:130(load_stream)
40 0.000 0.000 71.072 1.777 rdd.py:304(func)
20 0.000 0.000 71.094 3.555 worker.py:82(process)
```
Also, use can show profile result manually by `sc.show_profiles()` or dump it into disk
by `sc.dump_profiles(path)`, such as
```python
>>> sc._conf.set("spark.python.profile", "true")
>>> rdd = sc.parallelize(range(100)).map(str)
>>> rdd.count()
100
>>> sc.show_profiles()
============================================================
Profile of RDD<id=1>
============================================================
284 function calls (276 primitive calls) in 0.001 seconds
Ordered by: internal time, cumulative time
ncalls tottime percall cumtime percall filename:lineno(function)
4 0.000 0.000 0.000 0.000 serializers.py:198(load_stream)
4 0.000 0.000 0.000 0.000 {reduce}
12/4 0.000 0.000 0.001 0.000 rdd.py:2092(pipeline_func)
4 0.000 0.000 0.000 0.000 {cPickle.loads}
4 0.000 0.000 0.000 0.000 {cPickle.dumps}
104 0.000 0.000 0.000 0.000 rdd.py:852(<genexpr>)
8 0.000 0.000 0.000 0.000 serializers.py:461(read_int)
12 0.000 0.000 0.000 0.000 rdd.py:303(func)
```
The profiling is disabled by default, can be enabled by "spark.python.profile=true".
Also, users can dump the results into disks automatically for future analysis, by "spark.python.profile.dump=path_to_dump"
This is bugfix of #2351 cc JoshRosen
Author: Davies Liu <davies.liu@gmail.com>
Closes #2556 from davies/profiler and squashes the following commits:
e68df5a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
858e74c [Davies Liu] compatitable with python 2.6
7ef2aa0 [Davies Liu] bugfix, add tests for show_profiles and dump_profiles()
2b0daf2 [Davies Liu] fix docs
7a56c24 [Davies Liu] bugfix
cba9463 [Davies Liu] move show_profiles and dump_profiles to SparkContext
fb9565b [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
116d52a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
09d02c3 [Davies Liu] Merge branch 'master' into profiler
c23865c [Davies Liu] Merge branch 'master' into profiler
15d6f18 [Davies Liu] add docs for two configs
dadee1a [Davies Liu] add docs string and clear profiles after show or dump
4f8309d [Davies Liu] address comment, add tests
0a5b6eb [Davies Liu] fix Python UDF
4b20494 [Davies Liu] add profile for python
2014-09-30 21:24:57 -04:00
|
|
|
|
|
|
|
def test_profiler(self):
|
2015-01-28 16:48:06 -05:00
|
|
|
self.do_computation()
|
[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
|
|
|
profilers = self.sc.profiler_collector.profilers
|
|
|
|
self.assertEqual(1, len(profilers))
|
|
|
|
id, profiler, _ = profilers[0]
|
|
|
|
stats = profiler.stats()
|
[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.assertTrue(stats is not None)
|
|
|
|
width, stat_list = stats.get_print_list([])
|
|
|
|
func_names = [func_name for fname, n, func_name in stat_list]
|
|
|
|
self.assertTrue("heavy_foo" in func_names)
|
|
|
|
|
2015-04-21 20:49:55 -04:00
|
|
|
old_stdout = sys.stdout
|
|
|
|
sys.stdout = io = StringIO()
|
[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.sc.show_profiles()
|
2015-04-21 20:49:55 -04:00
|
|
|
self.assertTrue("heavy_foo" in io.getvalue())
|
|
|
|
sys.stdout = old_stdout
|
|
|
|
|
[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
|
|
|
d = tempfile.gettempdir()
|
|
|
|
self.sc.dump_profiles(d)
|
|
|
|
self.assertTrue("rdd_%d.pstats" % id in os.listdir(d))
|
|
|
|
|
2015-01-28 16:48:06 -05:00
|
|
|
def test_custom_profiler(self):
|
|
|
|
class TestCustomProfiler(BasicProfiler):
|
|
|
|
def show(self, id):
|
|
|
|
self.result = "Custom formatting"
|
|
|
|
|
|
|
|
self.sc.profiler_collector.profiler_cls = TestCustomProfiler
|
|
|
|
|
|
|
|
self.do_computation()
|
|
|
|
|
|
|
|
profilers = self.sc.profiler_collector.profilers
|
|
|
|
self.assertEqual(1, len(profilers))
|
|
|
|
_, profiler, _ = profilers[0]
|
|
|
|
self.assertTrue(isinstance(profiler, TestCustomProfiler))
|
|
|
|
|
|
|
|
self.sc.show_profiles()
|
|
|
|
self.assertEqual("Custom formatting", profiler.result)
|
|
|
|
|
|
|
|
def do_computation(self):
|
|
|
|
def heavy_foo(x):
|
2015-04-21 20:49:55 -04:00
|
|
|
for i in range(1 << 18):
|
2015-01-28 16:48:06 -05:00
|
|
|
x = 1
|
|
|
|
|
|
|
|
rdd = self.sc.parallelize(range(100))
|
|
|
|
rdd.foreach(heavy_foo)
|
|
|
|
|
[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
|
|
|
|
[SPARK-22043][PYTHON] Improves error message for show_profiles and dump_profiles
## What changes were proposed in this pull request?
This PR proposes to improve error message from:
```
>>> sc.show_profiles()
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File ".../spark/python/pyspark/context.py", line 1000, in show_profiles
self.profiler_collector.show_profiles()
AttributeError: 'NoneType' object has no attribute 'show_profiles'
>>> sc.dump_profiles("/tmp/abc")
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File ".../spark/python/pyspark/context.py", line 1005, in dump_profiles
self.profiler_collector.dump_profiles(path)
AttributeError: 'NoneType' object has no attribute 'dump_profiles'
```
to
```
>>> sc.show_profiles()
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File ".../spark/python/pyspark/context.py", line 1003, in show_profiles
raise RuntimeError("'spark.python.profile' configuration must be set "
RuntimeError: 'spark.python.profile' configuration must be set to 'true' to enable Python profile.
>>> sc.dump_profiles("/tmp/abc")
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File ".../spark/python/pyspark/context.py", line 1012, in dump_profiles
raise RuntimeError("'spark.python.profile' configuration must be set "
RuntimeError: 'spark.python.profile' configuration must be set to 'true' to enable Python profile.
```
## How was this patch tested?
Unit tests added in `python/pyspark/tests.py` and manual tests.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #19260 from HyukjinKwon/profile-errors.
2017-09-18 00:20:11 -04:00
|
|
|
class ProfilerTests2(unittest.TestCase):
|
|
|
|
def test_profiler_disabled(self):
|
|
|
|
sc = SparkContext(conf=SparkConf().set("spark.python.profile", "false"))
|
|
|
|
try:
|
|
|
|
self.assertRaisesRegexp(
|
|
|
|
RuntimeError,
|
|
|
|
"'spark.python.profile' configuration must be set",
|
|
|
|
lambda: sc.show_profiles())
|
|
|
|
self.assertRaisesRegexp(
|
|
|
|
RuntimeError,
|
|
|
|
"'spark.python.profile' configuration must be set",
|
|
|
|
lambda: sc.dump_profiles("/tmp/abc"))
|
|
|
|
finally:
|
|
|
|
sc.stop()
|
|
|
|
|
|
|
|
|
2014-10-06 17:07:53 -04:00
|
|
|
class InputFormatTests(ReusedPySparkTestCase):
|
2013-02-01 03:25:19 -05:00
|
|
|
|
2014-10-06 17:07:53 -04:00
|
|
|
@classmethod
|
|
|
|
def setUpClass(cls):
|
|
|
|
ReusedPySparkTestCase.setUpClass()
|
|
|
|
cls.tempdir = tempfile.NamedTemporaryFile(delete=False)
|
|
|
|
os.unlink(cls.tempdir.name)
|
|
|
|
cls.sc._jvm.WriteInputFormatTestDataGenerator.generateData(cls.tempdir.name, cls.sc._jsc)
|
2013-02-01 03:25:19 -05:00
|
|
|
|
2014-10-06 17:07:53 -04:00
|
|
|
@classmethod
|
|
|
|
def tearDownClass(cls):
|
|
|
|
ReusedPySparkTestCase.tearDownClass()
|
|
|
|
shutil.rmtree(cls.tempdir.name)
|
SPARK-1416: PySpark support for SequenceFile and Hadoop InputFormats
So I finally resurrected this PR. It seems the old one against the incubator mirror is no longer available, so I cannot reference it.
This adds initial support for reading Hadoop ```SequenceFile```s, as well as arbitrary Hadoop ```InputFormat```s, in PySpark.
# Overview
The basics are as follows:
1. ```PythonRDD``` object contains the relevant methods, that are in turn invoked by ```SparkContext``` in PySpark
2. The SequenceFile or InputFormat is read on the Scala side and converted from ```Writable``` instances to the relevant Scala classes (in the case of primitives)
3. Pyrolite is used to serialize Java objects. If this fails, the fallback is ```toString```
4. ```PickleSerializer``` on the Python side deserializes.
This works "out the box" for simple ```Writable```s:
* ```Text```
* ```IntWritable```, ```DoubleWritable```, ```FloatWritable```
* ```NullWritable```
* ```BooleanWritable```
* ```BytesWritable```
* ```MapWritable```
It also works for simple, "struct-like" classes. Due to the way Pyrolite works, this requires that the classes satisfy the JavaBeans convenstions (i.e. with fields and a no-arg constructor and getters/setters). (Perhaps in future some sugar for case classes and reflection could be added).
I've tested it out with ```ESInputFormat``` as an example and it works very nicely:
```python
conf = {"es.resource" : "index/type" }
rdd = sc.newAPIHadoopRDD("org.elasticsearch.hadoop.mr.EsInputFormat", "org.apache.hadoop.io.NullWritable", "org.elasticsearch.hadoop.mr.LinkedMapWritable", conf=conf)
rdd.first()
```
I suspect for things like HBase/Cassandra it will be a bit trickier to get it to work out the box.
# Some things still outstanding:
1. ~~Requires ```msgpack-python``` and will fail without it. As originally discussed with Josh, add a ```as_strings``` argument that defaults to ```False```, that can be used if ```msgpack-python``` is not available~~
2. ~~I see from https://github.com/apache/spark/pull/363 that Pyrolite is being used there for SerDe between Scala and Python. @ahirreddy @mateiz what is the plan behind this - is Pyrolite preferred? It seems from a cursory glance that adapting the ```msgpack```-based SerDe here to use Pyrolite wouldn't be too hard~~
3. ~~Support the key and value "wrapper" that would allow a Scala/Java function to be plugged in that would transform whatever the key/value Writable class is into something that can be serialized (e.g. convert some custom Writable to a JavaBean or ```java.util.Map``` that can be easily serialized)~~
4. Support ```saveAsSequenceFile``` and ```saveAsHadoopFile``` etc. This would require SerDe in the reverse direction, that can be handled by Pyrolite. Will work on this as a separate PR
Author: Nick Pentreath <nick.pentreath@gmail.com>
Closes #455 from MLnick/pyspark-inputformats and squashes the following commits:
268df7e [Nick Pentreath] Documentation changes mer @pwendell comments
761269b [Nick Pentreath] Address @pwendell comments, simplify default writable conversions and remove registry.
4c972d8 [Nick Pentreath] Add license headers
d150431 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
cde6af9 [Nick Pentreath] Parameterize converter trait
5ebacfa [Nick Pentreath] Update docs for PySpark input formats
a985492 [Nick Pentreath] Move Converter examples to own package
365d0be [Nick Pentreath] Make classes private[python]. Add docs and @Experimental annotation to Converter interface.
eeb8205 [Nick Pentreath] Fix path relative to SPARK_HOME in tests
1eaa08b [Nick Pentreath] HBase -> Cassandra app name oversight
3f90c3e [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
2c18513 [Nick Pentreath] Add examples for reading HBase and Cassandra InputFormats from Python
b65606f [Nick Pentreath] Add converter interface
5757f6e [Nick Pentreath] Default key/value classes for sequenceFile asre None
085b55f [Nick Pentreath] Move input format tests to tests.py and clean up docs
43eb728 [Nick Pentreath] PySpark InputFormats docs into programming guide
94beedc [Nick Pentreath] Clean up args in PythonRDD. Set key/value converter defaults to None for PySpark context.py methods
1a4a1d6 [Nick Pentreath] Address @mateiz style comments
01e0813 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
15a7d07 [Nick Pentreath] Remove default args for key/value classes. Arg names to camelCase
9fe6bd5 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
84fe8e3 [Nick Pentreath] Python programming guide space formatting
d0f52b6 [Nick Pentreath] Python programming guide
7caa73a [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
93ef995 [Nick Pentreath] Add back context.py changes
9ef1896 [Nick Pentreath] Recover earlier changes lost in previous merge for serializers.py
077ecb2 [Nick Pentreath] Recover earlier changes lost in previous merge for context.py
5af4770 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
35b8e3a [Nick Pentreath] Another fix for test ordering
bef3afb [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
e001b94 [Nick Pentreath] Fix test failures due to ordering
78978d9 [Nick Pentreath] Add doc for SequenceFile and InputFormat support to Python programming guide
64eb051 [Nick Pentreath] Scalastyle fix
e7552fa [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
44f2857 [Nick Pentreath] Remove msgpack dependency and switch serialization to Pyrolite, plus some clean up and refactoring
c0ebfb6 [Nick Pentreath] Change sequencefile test data generator to easily be called from PySpark tests
1d7c17c [Nick Pentreath] Amend tests to auto-generate sequencefile data in temp dir
17a656b [Nick Pentreath] remove binary sequencefile for tests
f60959e [Nick Pentreath] Remove msgpack dependency and serializer from PySpark
450e0a2 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
31a2fff [Nick Pentreath] Scalastyle fixes
fc5099e [Nick Pentreath] Add Apache license headers
4e08983 [Nick Pentreath] Clean up docs for PySpark context methods
b20ec7e [Nick Pentreath] Clean up merge duplicate dependencies
951c117 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
f6aac55 [Nick Pentreath] Bring back msgpack
9d2256e [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
1bbbfb0 [Nick Pentreath] Clean up SparkBuild from merge
a67dfad [Nick Pentreath] Clean up Msgpack serialization and registering
7237263 [Nick Pentreath] Add back msgpack serializer and hadoop file code lost during merging
25da1ca [Nick Pentreath] Add generator for nulls, bools, bytes and maps
65360d5 [Nick Pentreath] Adding test SequenceFiles
0c612e5 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
d72bf18 [Nick Pentreath] msgpack
dd57922 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
e67212a [Nick Pentreath] Add back msgpack dependency
f2d76a0 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
41856a5 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
97ef708 [Nick Pentreath] Remove old writeToStream
2beeedb [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
795a763 [Nick Pentreath] Change name to WriteInputFormatTestDataGenerator. Cleanup some var names. Use SPARK_HOME in path for writing test sequencefile data.
174f520 [Nick Pentreath] Add back graphx settings
703ee65 [Nick Pentreath] Add back msgpack
619c0fa [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
1c8efbc [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
eb40036 [Nick Pentreath] Remove unused comment lines
4d7ef2e [Nick Pentreath] Fix indentation
f1d73e3 [Nick Pentreath] mergeConfs returns a copy rather than mutating one of the input arguments
0f5cd84 [Nick Pentreath] Remove unused pair UTF8 class. Add comments to msgpack deserializer
4294cbb [Nick Pentreath] Add old Hadoop api methods. Clean up and expand comments. Clean up argument names
818a1e6 [Nick Pentreath] Add seqencefile and Hadoop InputFormat support to PythonRDD
4e7c9e3 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
c304cc8 [Nick Pentreath] Adding supporting sequncefiles for tests. Cleaning up
4b0a43f [Nick Pentreath] Refactoring utils into own objects. Cleaning up old commented-out code
d86325f [Nick Pentreath] Initial WIP of PySpark support for SequenceFile and arbitrary Hadoop InputFormat
2014-06-10 01:21:03 -04:00
|
|
|
|
2015-04-16 19:20:57 -04:00
|
|
|
@unittest.skipIf(sys.version >= "3", "serialize array of byte")
|
SPARK-1416: PySpark support for SequenceFile and Hadoop InputFormats
So I finally resurrected this PR. It seems the old one against the incubator mirror is no longer available, so I cannot reference it.
This adds initial support for reading Hadoop ```SequenceFile```s, as well as arbitrary Hadoop ```InputFormat```s, in PySpark.
# Overview
The basics are as follows:
1. ```PythonRDD``` object contains the relevant methods, that are in turn invoked by ```SparkContext``` in PySpark
2. The SequenceFile or InputFormat is read on the Scala side and converted from ```Writable``` instances to the relevant Scala classes (in the case of primitives)
3. Pyrolite is used to serialize Java objects. If this fails, the fallback is ```toString```
4. ```PickleSerializer``` on the Python side deserializes.
This works "out the box" for simple ```Writable```s:
* ```Text```
* ```IntWritable```, ```DoubleWritable```, ```FloatWritable```
* ```NullWritable```
* ```BooleanWritable```
* ```BytesWritable```
* ```MapWritable```
It also works for simple, "struct-like" classes. Due to the way Pyrolite works, this requires that the classes satisfy the JavaBeans convenstions (i.e. with fields and a no-arg constructor and getters/setters). (Perhaps in future some sugar for case classes and reflection could be added).
I've tested it out with ```ESInputFormat``` as an example and it works very nicely:
```python
conf = {"es.resource" : "index/type" }
rdd = sc.newAPIHadoopRDD("org.elasticsearch.hadoop.mr.EsInputFormat", "org.apache.hadoop.io.NullWritable", "org.elasticsearch.hadoop.mr.LinkedMapWritable", conf=conf)
rdd.first()
```
I suspect for things like HBase/Cassandra it will be a bit trickier to get it to work out the box.
# Some things still outstanding:
1. ~~Requires ```msgpack-python``` and will fail without it. As originally discussed with Josh, add a ```as_strings``` argument that defaults to ```False```, that can be used if ```msgpack-python``` is not available~~
2. ~~I see from https://github.com/apache/spark/pull/363 that Pyrolite is being used there for SerDe between Scala and Python. @ahirreddy @mateiz what is the plan behind this - is Pyrolite preferred? It seems from a cursory glance that adapting the ```msgpack```-based SerDe here to use Pyrolite wouldn't be too hard~~
3. ~~Support the key and value "wrapper" that would allow a Scala/Java function to be plugged in that would transform whatever the key/value Writable class is into something that can be serialized (e.g. convert some custom Writable to a JavaBean or ```java.util.Map``` that can be easily serialized)~~
4. Support ```saveAsSequenceFile``` and ```saveAsHadoopFile``` etc. This would require SerDe in the reverse direction, that can be handled by Pyrolite. Will work on this as a separate PR
Author: Nick Pentreath <nick.pentreath@gmail.com>
Closes #455 from MLnick/pyspark-inputformats and squashes the following commits:
268df7e [Nick Pentreath] Documentation changes mer @pwendell comments
761269b [Nick Pentreath] Address @pwendell comments, simplify default writable conversions and remove registry.
4c972d8 [Nick Pentreath] Add license headers
d150431 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
cde6af9 [Nick Pentreath] Parameterize converter trait
5ebacfa [Nick Pentreath] Update docs for PySpark input formats
a985492 [Nick Pentreath] Move Converter examples to own package
365d0be [Nick Pentreath] Make classes private[python]. Add docs and @Experimental annotation to Converter interface.
eeb8205 [Nick Pentreath] Fix path relative to SPARK_HOME in tests
1eaa08b [Nick Pentreath] HBase -> Cassandra app name oversight
3f90c3e [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
2c18513 [Nick Pentreath] Add examples for reading HBase and Cassandra InputFormats from Python
b65606f [Nick Pentreath] Add converter interface
5757f6e [Nick Pentreath] Default key/value classes for sequenceFile asre None
085b55f [Nick Pentreath] Move input format tests to tests.py and clean up docs
43eb728 [Nick Pentreath] PySpark InputFormats docs into programming guide
94beedc [Nick Pentreath] Clean up args in PythonRDD. Set key/value converter defaults to None for PySpark context.py methods
1a4a1d6 [Nick Pentreath] Address @mateiz style comments
01e0813 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
15a7d07 [Nick Pentreath] Remove default args for key/value classes. Arg names to camelCase
9fe6bd5 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
84fe8e3 [Nick Pentreath] Python programming guide space formatting
d0f52b6 [Nick Pentreath] Python programming guide
7caa73a [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
93ef995 [Nick Pentreath] Add back context.py changes
9ef1896 [Nick Pentreath] Recover earlier changes lost in previous merge for serializers.py
077ecb2 [Nick Pentreath] Recover earlier changes lost in previous merge for context.py
5af4770 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
35b8e3a [Nick Pentreath] Another fix for test ordering
bef3afb [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
e001b94 [Nick Pentreath] Fix test failures due to ordering
78978d9 [Nick Pentreath] Add doc for SequenceFile and InputFormat support to Python programming guide
64eb051 [Nick Pentreath] Scalastyle fix
e7552fa [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
44f2857 [Nick Pentreath] Remove msgpack dependency and switch serialization to Pyrolite, plus some clean up and refactoring
c0ebfb6 [Nick Pentreath] Change sequencefile test data generator to easily be called from PySpark tests
1d7c17c [Nick Pentreath] Amend tests to auto-generate sequencefile data in temp dir
17a656b [Nick Pentreath] remove binary sequencefile for tests
f60959e [Nick Pentreath] Remove msgpack dependency and serializer from PySpark
450e0a2 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
31a2fff [Nick Pentreath] Scalastyle fixes
fc5099e [Nick Pentreath] Add Apache license headers
4e08983 [Nick Pentreath] Clean up docs for PySpark context methods
b20ec7e [Nick Pentreath] Clean up merge duplicate dependencies
951c117 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
f6aac55 [Nick Pentreath] Bring back msgpack
9d2256e [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
1bbbfb0 [Nick Pentreath] Clean up SparkBuild from merge
a67dfad [Nick Pentreath] Clean up Msgpack serialization and registering
7237263 [Nick Pentreath] Add back msgpack serializer and hadoop file code lost during merging
25da1ca [Nick Pentreath] Add generator for nulls, bools, bytes and maps
65360d5 [Nick Pentreath] Adding test SequenceFiles
0c612e5 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
d72bf18 [Nick Pentreath] msgpack
dd57922 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
e67212a [Nick Pentreath] Add back msgpack dependency
f2d76a0 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
41856a5 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
97ef708 [Nick Pentreath] Remove old writeToStream
2beeedb [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
795a763 [Nick Pentreath] Change name to WriteInputFormatTestDataGenerator. Cleanup some var names. Use SPARK_HOME in path for writing test sequencefile data.
174f520 [Nick Pentreath] Add back graphx settings
703ee65 [Nick Pentreath] Add back msgpack
619c0fa [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
1c8efbc [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
eb40036 [Nick Pentreath] Remove unused comment lines
4d7ef2e [Nick Pentreath] Fix indentation
f1d73e3 [Nick Pentreath] mergeConfs returns a copy rather than mutating one of the input arguments
0f5cd84 [Nick Pentreath] Remove unused pair UTF8 class. Add comments to msgpack deserializer
4294cbb [Nick Pentreath] Add old Hadoop api methods. Clean up and expand comments. Clean up argument names
818a1e6 [Nick Pentreath] Add seqencefile and Hadoop InputFormat support to PythonRDD
4e7c9e3 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
c304cc8 [Nick Pentreath] Adding supporting sequncefiles for tests. Cleaning up
4b0a43f [Nick Pentreath] Refactoring utils into own objects. Cleaning up old commented-out code
d86325f [Nick Pentreath] Initial WIP of PySpark support for SequenceFile and arbitrary Hadoop InputFormat
2014-06-10 01:21:03 -04:00
|
|
|
def test_sequencefiles(self):
|
|
|
|
basepath = self.tempdir.name
|
|
|
|
ints = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfint/",
|
|
|
|
"org.apache.hadoop.io.IntWritable",
|
|
|
|
"org.apache.hadoop.io.Text").collect())
|
|
|
|
ei = [(1, u'aa'), (1, u'aa'), (2, u'aa'), (2, u'bb'), (2, u'bb'), (3, u'cc')]
|
|
|
|
self.assertEqual(ints, ei)
|
|
|
|
|
|
|
|
doubles = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfdouble/",
|
|
|
|
"org.apache.hadoop.io.DoubleWritable",
|
|
|
|
"org.apache.hadoop.io.Text").collect())
|
|
|
|
ed = [(1.0, u'aa'), (1.0, u'aa'), (2.0, u'aa'), (2.0, u'bb'), (2.0, u'bb'), (3.0, u'cc')]
|
|
|
|
self.assertEqual(doubles, ed)
|
|
|
|
|
2014-07-30 16:19:05 -04:00
|
|
|
bytes = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfbytes/",
|
2014-08-06 15:58:24 -04:00
|
|
|
"org.apache.hadoop.io.IntWritable",
|
|
|
|
"org.apache.hadoop.io.BytesWritable").collect())
|
2014-07-30 16:19:05 -04:00
|
|
|
ebs = [(1, bytearray('aa', 'utf-8')),
|
|
|
|
(1, bytearray('aa', 'utf-8')),
|
|
|
|
(2, bytearray('aa', 'utf-8')),
|
|
|
|
(2, bytearray('bb', 'utf-8')),
|
|
|
|
(2, bytearray('bb', 'utf-8')),
|
|
|
|
(3, bytearray('cc', 'utf-8'))]
|
|
|
|
self.assertEqual(bytes, ebs)
|
|
|
|
|
SPARK-1416: PySpark support for SequenceFile and Hadoop InputFormats
So I finally resurrected this PR. It seems the old one against the incubator mirror is no longer available, so I cannot reference it.
This adds initial support for reading Hadoop ```SequenceFile```s, as well as arbitrary Hadoop ```InputFormat```s, in PySpark.
# Overview
The basics are as follows:
1. ```PythonRDD``` object contains the relevant methods, that are in turn invoked by ```SparkContext``` in PySpark
2. The SequenceFile or InputFormat is read on the Scala side and converted from ```Writable``` instances to the relevant Scala classes (in the case of primitives)
3. Pyrolite is used to serialize Java objects. If this fails, the fallback is ```toString```
4. ```PickleSerializer``` on the Python side deserializes.
This works "out the box" for simple ```Writable```s:
* ```Text```
* ```IntWritable```, ```DoubleWritable```, ```FloatWritable```
* ```NullWritable```
* ```BooleanWritable```
* ```BytesWritable```
* ```MapWritable```
It also works for simple, "struct-like" classes. Due to the way Pyrolite works, this requires that the classes satisfy the JavaBeans convenstions (i.e. with fields and a no-arg constructor and getters/setters). (Perhaps in future some sugar for case classes and reflection could be added).
I've tested it out with ```ESInputFormat``` as an example and it works very nicely:
```python
conf = {"es.resource" : "index/type" }
rdd = sc.newAPIHadoopRDD("org.elasticsearch.hadoop.mr.EsInputFormat", "org.apache.hadoop.io.NullWritable", "org.elasticsearch.hadoop.mr.LinkedMapWritable", conf=conf)
rdd.first()
```
I suspect for things like HBase/Cassandra it will be a bit trickier to get it to work out the box.
# Some things still outstanding:
1. ~~Requires ```msgpack-python``` and will fail without it. As originally discussed with Josh, add a ```as_strings``` argument that defaults to ```False```, that can be used if ```msgpack-python``` is not available~~
2. ~~I see from https://github.com/apache/spark/pull/363 that Pyrolite is being used there for SerDe between Scala and Python. @ahirreddy @mateiz what is the plan behind this - is Pyrolite preferred? It seems from a cursory glance that adapting the ```msgpack```-based SerDe here to use Pyrolite wouldn't be too hard~~
3. ~~Support the key and value "wrapper" that would allow a Scala/Java function to be plugged in that would transform whatever the key/value Writable class is into something that can be serialized (e.g. convert some custom Writable to a JavaBean or ```java.util.Map``` that can be easily serialized)~~
4. Support ```saveAsSequenceFile``` and ```saveAsHadoopFile``` etc. This would require SerDe in the reverse direction, that can be handled by Pyrolite. Will work on this as a separate PR
Author: Nick Pentreath <nick.pentreath@gmail.com>
Closes #455 from MLnick/pyspark-inputformats and squashes the following commits:
268df7e [Nick Pentreath] Documentation changes mer @pwendell comments
761269b [Nick Pentreath] Address @pwendell comments, simplify default writable conversions and remove registry.
4c972d8 [Nick Pentreath] Add license headers
d150431 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
cde6af9 [Nick Pentreath] Parameterize converter trait
5ebacfa [Nick Pentreath] Update docs for PySpark input formats
a985492 [Nick Pentreath] Move Converter examples to own package
365d0be [Nick Pentreath] Make classes private[python]. Add docs and @Experimental annotation to Converter interface.
eeb8205 [Nick Pentreath] Fix path relative to SPARK_HOME in tests
1eaa08b [Nick Pentreath] HBase -> Cassandra app name oversight
3f90c3e [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
2c18513 [Nick Pentreath] Add examples for reading HBase and Cassandra InputFormats from Python
b65606f [Nick Pentreath] Add converter interface
5757f6e [Nick Pentreath] Default key/value classes for sequenceFile asre None
085b55f [Nick Pentreath] Move input format tests to tests.py and clean up docs
43eb728 [Nick Pentreath] PySpark InputFormats docs into programming guide
94beedc [Nick Pentreath] Clean up args in PythonRDD. Set key/value converter defaults to None for PySpark context.py methods
1a4a1d6 [Nick Pentreath] Address @mateiz style comments
01e0813 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
15a7d07 [Nick Pentreath] Remove default args for key/value classes. Arg names to camelCase
9fe6bd5 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
84fe8e3 [Nick Pentreath] Python programming guide space formatting
d0f52b6 [Nick Pentreath] Python programming guide
7caa73a [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
93ef995 [Nick Pentreath] Add back context.py changes
9ef1896 [Nick Pentreath] Recover earlier changes lost in previous merge for serializers.py
077ecb2 [Nick Pentreath] Recover earlier changes lost in previous merge for context.py
5af4770 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
35b8e3a [Nick Pentreath] Another fix for test ordering
bef3afb [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
e001b94 [Nick Pentreath] Fix test failures due to ordering
78978d9 [Nick Pentreath] Add doc for SequenceFile and InputFormat support to Python programming guide
64eb051 [Nick Pentreath] Scalastyle fix
e7552fa [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
44f2857 [Nick Pentreath] Remove msgpack dependency and switch serialization to Pyrolite, plus some clean up and refactoring
c0ebfb6 [Nick Pentreath] Change sequencefile test data generator to easily be called from PySpark tests
1d7c17c [Nick Pentreath] Amend tests to auto-generate sequencefile data in temp dir
17a656b [Nick Pentreath] remove binary sequencefile for tests
f60959e [Nick Pentreath] Remove msgpack dependency and serializer from PySpark
450e0a2 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
31a2fff [Nick Pentreath] Scalastyle fixes
fc5099e [Nick Pentreath] Add Apache license headers
4e08983 [Nick Pentreath] Clean up docs for PySpark context methods
b20ec7e [Nick Pentreath] Clean up merge duplicate dependencies
951c117 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
f6aac55 [Nick Pentreath] Bring back msgpack
9d2256e [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
1bbbfb0 [Nick Pentreath] Clean up SparkBuild from merge
a67dfad [Nick Pentreath] Clean up Msgpack serialization and registering
7237263 [Nick Pentreath] Add back msgpack serializer and hadoop file code lost during merging
25da1ca [Nick Pentreath] Add generator for nulls, bools, bytes and maps
65360d5 [Nick Pentreath] Adding test SequenceFiles
0c612e5 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
d72bf18 [Nick Pentreath] msgpack
dd57922 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
e67212a [Nick Pentreath] Add back msgpack dependency
f2d76a0 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
41856a5 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
97ef708 [Nick Pentreath] Remove old writeToStream
2beeedb [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
795a763 [Nick Pentreath] Change name to WriteInputFormatTestDataGenerator. Cleanup some var names. Use SPARK_HOME in path for writing test sequencefile data.
174f520 [Nick Pentreath] Add back graphx settings
703ee65 [Nick Pentreath] Add back msgpack
619c0fa [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
1c8efbc [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
eb40036 [Nick Pentreath] Remove unused comment lines
4d7ef2e [Nick Pentreath] Fix indentation
f1d73e3 [Nick Pentreath] mergeConfs returns a copy rather than mutating one of the input arguments
0f5cd84 [Nick Pentreath] Remove unused pair UTF8 class. Add comments to msgpack deserializer
4294cbb [Nick Pentreath] Add old Hadoop api methods. Clean up and expand comments. Clean up argument names
818a1e6 [Nick Pentreath] Add seqencefile and Hadoop InputFormat support to PythonRDD
4e7c9e3 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
c304cc8 [Nick Pentreath] Adding supporting sequncefiles for tests. Cleaning up
4b0a43f [Nick Pentreath] Refactoring utils into own objects. Cleaning up old commented-out code
d86325f [Nick Pentreath] Initial WIP of PySpark support for SequenceFile and arbitrary Hadoop InputFormat
2014-06-10 01:21:03 -04:00
|
|
|
text = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sftext/",
|
|
|
|
"org.apache.hadoop.io.Text",
|
|
|
|
"org.apache.hadoop.io.Text").collect())
|
|
|
|
et = [(u'1', u'aa'),
|
|
|
|
(u'1', u'aa'),
|
|
|
|
(u'2', u'aa'),
|
|
|
|
(u'2', u'bb'),
|
|
|
|
(u'2', u'bb'),
|
|
|
|
(u'3', u'cc')]
|
|
|
|
self.assertEqual(text, et)
|
|
|
|
|
|
|
|
bools = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfbool/",
|
|
|
|
"org.apache.hadoop.io.IntWritable",
|
|
|
|
"org.apache.hadoop.io.BooleanWritable").collect())
|
|
|
|
eb = [(1, False), (1, True), (2, False), (2, False), (2, True), (3, True)]
|
|
|
|
self.assertEqual(bools, eb)
|
|
|
|
|
|
|
|
nulls = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfnull/",
|
|
|
|
"org.apache.hadoop.io.IntWritable",
|
|
|
|
"org.apache.hadoop.io.BooleanWritable").collect())
|
|
|
|
en = [(1, None), (1, None), (2, None), (2, None), (2, None), (3, None)]
|
|
|
|
self.assertEqual(nulls, en)
|
|
|
|
|
2015-04-16 19:20:57 -04:00
|
|
|
maps = self.sc.sequenceFile(basepath + "/sftestdata/sfmap/",
|
|
|
|
"org.apache.hadoop.io.IntWritable",
|
|
|
|
"org.apache.hadoop.io.MapWritable").collect()
|
2014-07-30 16:19:05 -04:00
|
|
|
em = [(1, {}),
|
SPARK-1416: PySpark support for SequenceFile and Hadoop InputFormats
So I finally resurrected this PR. It seems the old one against the incubator mirror is no longer available, so I cannot reference it.
This adds initial support for reading Hadoop ```SequenceFile```s, as well as arbitrary Hadoop ```InputFormat```s, in PySpark.
# Overview
The basics are as follows:
1. ```PythonRDD``` object contains the relevant methods, that are in turn invoked by ```SparkContext``` in PySpark
2. The SequenceFile or InputFormat is read on the Scala side and converted from ```Writable``` instances to the relevant Scala classes (in the case of primitives)
3. Pyrolite is used to serialize Java objects. If this fails, the fallback is ```toString```
4. ```PickleSerializer``` on the Python side deserializes.
This works "out the box" for simple ```Writable```s:
* ```Text```
* ```IntWritable```, ```DoubleWritable```, ```FloatWritable```
* ```NullWritable```
* ```BooleanWritable```
* ```BytesWritable```
* ```MapWritable```
It also works for simple, "struct-like" classes. Due to the way Pyrolite works, this requires that the classes satisfy the JavaBeans convenstions (i.e. with fields and a no-arg constructor and getters/setters). (Perhaps in future some sugar for case classes and reflection could be added).
I've tested it out with ```ESInputFormat``` as an example and it works very nicely:
```python
conf = {"es.resource" : "index/type" }
rdd = sc.newAPIHadoopRDD("org.elasticsearch.hadoop.mr.EsInputFormat", "org.apache.hadoop.io.NullWritable", "org.elasticsearch.hadoop.mr.LinkedMapWritable", conf=conf)
rdd.first()
```
I suspect for things like HBase/Cassandra it will be a bit trickier to get it to work out the box.
# Some things still outstanding:
1. ~~Requires ```msgpack-python``` and will fail without it. As originally discussed with Josh, add a ```as_strings``` argument that defaults to ```False```, that can be used if ```msgpack-python``` is not available~~
2. ~~I see from https://github.com/apache/spark/pull/363 that Pyrolite is being used there for SerDe between Scala and Python. @ahirreddy @mateiz what is the plan behind this - is Pyrolite preferred? It seems from a cursory glance that adapting the ```msgpack```-based SerDe here to use Pyrolite wouldn't be too hard~~
3. ~~Support the key and value "wrapper" that would allow a Scala/Java function to be plugged in that would transform whatever the key/value Writable class is into something that can be serialized (e.g. convert some custom Writable to a JavaBean or ```java.util.Map``` that can be easily serialized)~~
4. Support ```saveAsSequenceFile``` and ```saveAsHadoopFile``` etc. This would require SerDe in the reverse direction, that can be handled by Pyrolite. Will work on this as a separate PR
Author: Nick Pentreath <nick.pentreath@gmail.com>
Closes #455 from MLnick/pyspark-inputformats and squashes the following commits:
268df7e [Nick Pentreath] Documentation changes mer @pwendell comments
761269b [Nick Pentreath] Address @pwendell comments, simplify default writable conversions and remove registry.
4c972d8 [Nick Pentreath] Add license headers
d150431 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
cde6af9 [Nick Pentreath] Parameterize converter trait
5ebacfa [Nick Pentreath] Update docs for PySpark input formats
a985492 [Nick Pentreath] Move Converter examples to own package
365d0be [Nick Pentreath] Make classes private[python]. Add docs and @Experimental annotation to Converter interface.
eeb8205 [Nick Pentreath] Fix path relative to SPARK_HOME in tests
1eaa08b [Nick Pentreath] HBase -> Cassandra app name oversight
3f90c3e [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
2c18513 [Nick Pentreath] Add examples for reading HBase and Cassandra InputFormats from Python
b65606f [Nick Pentreath] Add converter interface
5757f6e [Nick Pentreath] Default key/value classes for sequenceFile asre None
085b55f [Nick Pentreath] Move input format tests to tests.py and clean up docs
43eb728 [Nick Pentreath] PySpark InputFormats docs into programming guide
94beedc [Nick Pentreath] Clean up args in PythonRDD. Set key/value converter defaults to None for PySpark context.py methods
1a4a1d6 [Nick Pentreath] Address @mateiz style comments
01e0813 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
15a7d07 [Nick Pentreath] Remove default args for key/value classes. Arg names to camelCase
9fe6bd5 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
84fe8e3 [Nick Pentreath] Python programming guide space formatting
d0f52b6 [Nick Pentreath] Python programming guide
7caa73a [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
93ef995 [Nick Pentreath] Add back context.py changes
9ef1896 [Nick Pentreath] Recover earlier changes lost in previous merge for serializers.py
077ecb2 [Nick Pentreath] Recover earlier changes lost in previous merge for context.py
5af4770 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
35b8e3a [Nick Pentreath] Another fix for test ordering
bef3afb [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
e001b94 [Nick Pentreath] Fix test failures due to ordering
78978d9 [Nick Pentreath] Add doc for SequenceFile and InputFormat support to Python programming guide
64eb051 [Nick Pentreath] Scalastyle fix
e7552fa [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
44f2857 [Nick Pentreath] Remove msgpack dependency and switch serialization to Pyrolite, plus some clean up and refactoring
c0ebfb6 [Nick Pentreath] Change sequencefile test data generator to easily be called from PySpark tests
1d7c17c [Nick Pentreath] Amend tests to auto-generate sequencefile data in temp dir
17a656b [Nick Pentreath] remove binary sequencefile for tests
f60959e [Nick Pentreath] Remove msgpack dependency and serializer from PySpark
450e0a2 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
31a2fff [Nick Pentreath] Scalastyle fixes
fc5099e [Nick Pentreath] Add Apache license headers
4e08983 [Nick Pentreath] Clean up docs for PySpark context methods
b20ec7e [Nick Pentreath] Clean up merge duplicate dependencies
951c117 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
f6aac55 [Nick Pentreath] Bring back msgpack
9d2256e [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
1bbbfb0 [Nick Pentreath] Clean up SparkBuild from merge
a67dfad [Nick Pentreath] Clean up Msgpack serialization and registering
7237263 [Nick Pentreath] Add back msgpack serializer and hadoop file code lost during merging
25da1ca [Nick Pentreath] Add generator for nulls, bools, bytes and maps
65360d5 [Nick Pentreath] Adding test SequenceFiles
0c612e5 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
d72bf18 [Nick Pentreath] msgpack
dd57922 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
e67212a [Nick Pentreath] Add back msgpack dependency
f2d76a0 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
41856a5 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
97ef708 [Nick Pentreath] Remove old writeToStream
2beeedb [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
795a763 [Nick Pentreath] Change name to WriteInputFormatTestDataGenerator. Cleanup some var names. Use SPARK_HOME in path for writing test sequencefile data.
174f520 [Nick Pentreath] Add back graphx settings
703ee65 [Nick Pentreath] Add back msgpack
619c0fa [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
1c8efbc [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
eb40036 [Nick Pentreath] Remove unused comment lines
4d7ef2e [Nick Pentreath] Fix indentation
f1d73e3 [Nick Pentreath] mergeConfs returns a copy rather than mutating one of the input arguments
0f5cd84 [Nick Pentreath] Remove unused pair UTF8 class. Add comments to msgpack deserializer
4294cbb [Nick Pentreath] Add old Hadoop api methods. Clean up and expand comments. Clean up argument names
818a1e6 [Nick Pentreath] Add seqencefile and Hadoop InputFormat support to PythonRDD
4e7c9e3 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
c304cc8 [Nick Pentreath] Adding supporting sequncefiles for tests. Cleaning up
4b0a43f [Nick Pentreath] Refactoring utils into own objects. Cleaning up old commented-out code
d86325f [Nick Pentreath] Initial WIP of PySpark support for SequenceFile and arbitrary Hadoop InputFormat
2014-06-10 01:21:03 -04:00
|
|
|
(1, {3.0: u'bb'}),
|
|
|
|
(2, {1.0: u'aa'}),
|
|
|
|
(2, {1.0: u'cc'}),
|
|
|
|
(3, {2.0: u'dd'})]
|
2015-04-16 19:20:57 -04:00
|
|
|
for v in maps:
|
|
|
|
self.assertTrue(v in em)
|
SPARK-1416: PySpark support for SequenceFile and Hadoop InputFormats
So I finally resurrected this PR. It seems the old one against the incubator mirror is no longer available, so I cannot reference it.
This adds initial support for reading Hadoop ```SequenceFile```s, as well as arbitrary Hadoop ```InputFormat```s, in PySpark.
# Overview
The basics are as follows:
1. ```PythonRDD``` object contains the relevant methods, that are in turn invoked by ```SparkContext``` in PySpark
2. The SequenceFile or InputFormat is read on the Scala side and converted from ```Writable``` instances to the relevant Scala classes (in the case of primitives)
3. Pyrolite is used to serialize Java objects. If this fails, the fallback is ```toString```
4. ```PickleSerializer``` on the Python side deserializes.
This works "out the box" for simple ```Writable```s:
* ```Text```
* ```IntWritable```, ```DoubleWritable```, ```FloatWritable```
* ```NullWritable```
* ```BooleanWritable```
* ```BytesWritable```
* ```MapWritable```
It also works for simple, "struct-like" classes. Due to the way Pyrolite works, this requires that the classes satisfy the JavaBeans convenstions (i.e. with fields and a no-arg constructor and getters/setters). (Perhaps in future some sugar for case classes and reflection could be added).
I've tested it out with ```ESInputFormat``` as an example and it works very nicely:
```python
conf = {"es.resource" : "index/type" }
rdd = sc.newAPIHadoopRDD("org.elasticsearch.hadoop.mr.EsInputFormat", "org.apache.hadoop.io.NullWritable", "org.elasticsearch.hadoop.mr.LinkedMapWritable", conf=conf)
rdd.first()
```
I suspect for things like HBase/Cassandra it will be a bit trickier to get it to work out the box.
# Some things still outstanding:
1. ~~Requires ```msgpack-python``` and will fail without it. As originally discussed with Josh, add a ```as_strings``` argument that defaults to ```False```, that can be used if ```msgpack-python``` is not available~~
2. ~~I see from https://github.com/apache/spark/pull/363 that Pyrolite is being used there for SerDe between Scala and Python. @ahirreddy @mateiz what is the plan behind this - is Pyrolite preferred? It seems from a cursory glance that adapting the ```msgpack```-based SerDe here to use Pyrolite wouldn't be too hard~~
3. ~~Support the key and value "wrapper" that would allow a Scala/Java function to be plugged in that would transform whatever the key/value Writable class is into something that can be serialized (e.g. convert some custom Writable to a JavaBean or ```java.util.Map``` that can be easily serialized)~~
4. Support ```saveAsSequenceFile``` and ```saveAsHadoopFile``` etc. This would require SerDe in the reverse direction, that can be handled by Pyrolite. Will work on this as a separate PR
Author: Nick Pentreath <nick.pentreath@gmail.com>
Closes #455 from MLnick/pyspark-inputformats and squashes the following commits:
268df7e [Nick Pentreath] Documentation changes mer @pwendell comments
761269b [Nick Pentreath] Address @pwendell comments, simplify default writable conversions and remove registry.
4c972d8 [Nick Pentreath] Add license headers
d150431 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
cde6af9 [Nick Pentreath] Parameterize converter trait
5ebacfa [Nick Pentreath] Update docs for PySpark input formats
a985492 [Nick Pentreath] Move Converter examples to own package
365d0be [Nick Pentreath] Make classes private[python]. Add docs and @Experimental annotation to Converter interface.
eeb8205 [Nick Pentreath] Fix path relative to SPARK_HOME in tests
1eaa08b [Nick Pentreath] HBase -> Cassandra app name oversight
3f90c3e [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
2c18513 [Nick Pentreath] Add examples for reading HBase and Cassandra InputFormats from Python
b65606f [Nick Pentreath] Add converter interface
5757f6e [Nick Pentreath] Default key/value classes for sequenceFile asre None
085b55f [Nick Pentreath] Move input format tests to tests.py and clean up docs
43eb728 [Nick Pentreath] PySpark InputFormats docs into programming guide
94beedc [Nick Pentreath] Clean up args in PythonRDD. Set key/value converter defaults to None for PySpark context.py methods
1a4a1d6 [Nick Pentreath] Address @mateiz style comments
01e0813 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
15a7d07 [Nick Pentreath] Remove default args for key/value classes. Arg names to camelCase
9fe6bd5 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
84fe8e3 [Nick Pentreath] Python programming guide space formatting
d0f52b6 [Nick Pentreath] Python programming guide
7caa73a [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
93ef995 [Nick Pentreath] Add back context.py changes
9ef1896 [Nick Pentreath] Recover earlier changes lost in previous merge for serializers.py
077ecb2 [Nick Pentreath] Recover earlier changes lost in previous merge for context.py
5af4770 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
35b8e3a [Nick Pentreath] Another fix for test ordering
bef3afb [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
e001b94 [Nick Pentreath] Fix test failures due to ordering
78978d9 [Nick Pentreath] Add doc for SequenceFile and InputFormat support to Python programming guide
64eb051 [Nick Pentreath] Scalastyle fix
e7552fa [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
44f2857 [Nick Pentreath] Remove msgpack dependency and switch serialization to Pyrolite, plus some clean up and refactoring
c0ebfb6 [Nick Pentreath] Change sequencefile test data generator to easily be called from PySpark tests
1d7c17c [Nick Pentreath] Amend tests to auto-generate sequencefile data in temp dir
17a656b [Nick Pentreath] remove binary sequencefile for tests
f60959e [Nick Pentreath] Remove msgpack dependency and serializer from PySpark
450e0a2 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
31a2fff [Nick Pentreath] Scalastyle fixes
fc5099e [Nick Pentreath] Add Apache license headers
4e08983 [Nick Pentreath] Clean up docs for PySpark context methods
b20ec7e [Nick Pentreath] Clean up merge duplicate dependencies
951c117 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
f6aac55 [Nick Pentreath] Bring back msgpack
9d2256e [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
1bbbfb0 [Nick Pentreath] Clean up SparkBuild from merge
a67dfad [Nick Pentreath] Clean up Msgpack serialization and registering
7237263 [Nick Pentreath] Add back msgpack serializer and hadoop file code lost during merging
25da1ca [Nick Pentreath] Add generator for nulls, bools, bytes and maps
65360d5 [Nick Pentreath] Adding test SequenceFiles
0c612e5 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
d72bf18 [Nick Pentreath] msgpack
dd57922 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
e67212a [Nick Pentreath] Add back msgpack dependency
f2d76a0 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
41856a5 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
97ef708 [Nick Pentreath] Remove old writeToStream
2beeedb [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
795a763 [Nick Pentreath] Change name to WriteInputFormatTestDataGenerator. Cleanup some var names. Use SPARK_HOME in path for writing test sequencefile data.
174f520 [Nick Pentreath] Add back graphx settings
703ee65 [Nick Pentreath] Add back msgpack
619c0fa [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
1c8efbc [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
eb40036 [Nick Pentreath] Remove unused comment lines
4d7ef2e [Nick Pentreath] Fix indentation
f1d73e3 [Nick Pentreath] mergeConfs returns a copy rather than mutating one of the input arguments
0f5cd84 [Nick Pentreath] Remove unused pair UTF8 class. Add comments to msgpack deserializer
4294cbb [Nick Pentreath] Add old Hadoop api methods. Clean up and expand comments. Clean up argument names
818a1e6 [Nick Pentreath] Add seqencefile and Hadoop InputFormat support to PythonRDD
4e7c9e3 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
c304cc8 [Nick Pentreath] Adding supporting sequncefiles for tests. Cleaning up
4b0a43f [Nick Pentreath] Refactoring utils into own objects. Cleaning up old commented-out code
d86325f [Nick Pentreath] Initial WIP of PySpark support for SequenceFile and arbitrary Hadoop InputFormat
2014-06-10 01:21:03 -04:00
|
|
|
|
2014-07-30 16:19:05 -04:00
|
|
|
# arrays get pickled to tuples by default
|
|
|
|
tuples = sorted(self.sc.sequenceFile(
|
|
|
|
basepath + "/sftestdata/sfarray/",
|
|
|
|
"org.apache.hadoop.io.IntWritable",
|
|
|
|
"org.apache.spark.api.python.DoubleArrayWritable").collect())
|
|
|
|
et = [(1, ()),
|
|
|
|
(2, (3.0, 4.0, 5.0)),
|
|
|
|
(3, (4.0, 5.0, 6.0))]
|
|
|
|
self.assertEqual(tuples, et)
|
|
|
|
|
|
|
|
# with custom converters, primitive arrays can stay as arrays
|
|
|
|
arrays = sorted(self.sc.sequenceFile(
|
|
|
|
basepath + "/sftestdata/sfarray/",
|
|
|
|
"org.apache.hadoop.io.IntWritable",
|
|
|
|
"org.apache.spark.api.python.DoubleArrayWritable",
|
|
|
|
valueConverter="org.apache.spark.api.python.WritableToDoubleArrayConverter").collect())
|
|
|
|
ea = [(1, array('d')),
|
|
|
|
(2, array('d', [3.0, 4.0, 5.0])),
|
|
|
|
(3, array('d', [4.0, 5.0, 6.0]))]
|
|
|
|
self.assertEqual(arrays, ea)
|
|
|
|
|
SPARK-1416: PySpark support for SequenceFile and Hadoop InputFormats
So I finally resurrected this PR. It seems the old one against the incubator mirror is no longer available, so I cannot reference it.
This adds initial support for reading Hadoop ```SequenceFile```s, as well as arbitrary Hadoop ```InputFormat```s, in PySpark.
# Overview
The basics are as follows:
1. ```PythonRDD``` object contains the relevant methods, that are in turn invoked by ```SparkContext``` in PySpark
2. The SequenceFile or InputFormat is read on the Scala side and converted from ```Writable``` instances to the relevant Scala classes (in the case of primitives)
3. Pyrolite is used to serialize Java objects. If this fails, the fallback is ```toString```
4. ```PickleSerializer``` on the Python side deserializes.
This works "out the box" for simple ```Writable```s:
* ```Text```
* ```IntWritable```, ```DoubleWritable```, ```FloatWritable```
* ```NullWritable```
* ```BooleanWritable```
* ```BytesWritable```
* ```MapWritable```
It also works for simple, "struct-like" classes. Due to the way Pyrolite works, this requires that the classes satisfy the JavaBeans convenstions (i.e. with fields and a no-arg constructor and getters/setters). (Perhaps in future some sugar for case classes and reflection could be added).
I've tested it out with ```ESInputFormat``` as an example and it works very nicely:
```python
conf = {"es.resource" : "index/type" }
rdd = sc.newAPIHadoopRDD("org.elasticsearch.hadoop.mr.EsInputFormat", "org.apache.hadoop.io.NullWritable", "org.elasticsearch.hadoop.mr.LinkedMapWritable", conf=conf)
rdd.first()
```
I suspect for things like HBase/Cassandra it will be a bit trickier to get it to work out the box.
# Some things still outstanding:
1. ~~Requires ```msgpack-python``` and will fail without it. As originally discussed with Josh, add a ```as_strings``` argument that defaults to ```False```, that can be used if ```msgpack-python``` is not available~~
2. ~~I see from https://github.com/apache/spark/pull/363 that Pyrolite is being used there for SerDe between Scala and Python. @ahirreddy @mateiz what is the plan behind this - is Pyrolite preferred? It seems from a cursory glance that adapting the ```msgpack```-based SerDe here to use Pyrolite wouldn't be too hard~~
3. ~~Support the key and value "wrapper" that would allow a Scala/Java function to be plugged in that would transform whatever the key/value Writable class is into something that can be serialized (e.g. convert some custom Writable to a JavaBean or ```java.util.Map``` that can be easily serialized)~~
4. Support ```saveAsSequenceFile``` and ```saveAsHadoopFile``` etc. This would require SerDe in the reverse direction, that can be handled by Pyrolite. Will work on this as a separate PR
Author: Nick Pentreath <nick.pentreath@gmail.com>
Closes #455 from MLnick/pyspark-inputformats and squashes the following commits:
268df7e [Nick Pentreath] Documentation changes mer @pwendell comments
761269b [Nick Pentreath] Address @pwendell comments, simplify default writable conversions and remove registry.
4c972d8 [Nick Pentreath] Add license headers
d150431 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
cde6af9 [Nick Pentreath] Parameterize converter trait
5ebacfa [Nick Pentreath] Update docs for PySpark input formats
a985492 [Nick Pentreath] Move Converter examples to own package
365d0be [Nick Pentreath] Make classes private[python]. Add docs and @Experimental annotation to Converter interface.
eeb8205 [Nick Pentreath] Fix path relative to SPARK_HOME in tests
1eaa08b [Nick Pentreath] HBase -> Cassandra app name oversight
3f90c3e [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
2c18513 [Nick Pentreath] Add examples for reading HBase and Cassandra InputFormats from Python
b65606f [Nick Pentreath] Add converter interface
5757f6e [Nick Pentreath] Default key/value classes for sequenceFile asre None
085b55f [Nick Pentreath] Move input format tests to tests.py and clean up docs
43eb728 [Nick Pentreath] PySpark InputFormats docs into programming guide
94beedc [Nick Pentreath] Clean up args in PythonRDD. Set key/value converter defaults to None for PySpark context.py methods
1a4a1d6 [Nick Pentreath] Address @mateiz style comments
01e0813 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
15a7d07 [Nick Pentreath] Remove default args for key/value classes. Arg names to camelCase
9fe6bd5 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
84fe8e3 [Nick Pentreath] Python programming guide space formatting
d0f52b6 [Nick Pentreath] Python programming guide
7caa73a [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
93ef995 [Nick Pentreath] Add back context.py changes
9ef1896 [Nick Pentreath] Recover earlier changes lost in previous merge for serializers.py
077ecb2 [Nick Pentreath] Recover earlier changes lost in previous merge for context.py
5af4770 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
35b8e3a [Nick Pentreath] Another fix for test ordering
bef3afb [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
e001b94 [Nick Pentreath] Fix test failures due to ordering
78978d9 [Nick Pentreath] Add doc for SequenceFile and InputFormat support to Python programming guide
64eb051 [Nick Pentreath] Scalastyle fix
e7552fa [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
44f2857 [Nick Pentreath] Remove msgpack dependency and switch serialization to Pyrolite, plus some clean up and refactoring
c0ebfb6 [Nick Pentreath] Change sequencefile test data generator to easily be called from PySpark tests
1d7c17c [Nick Pentreath] Amend tests to auto-generate sequencefile data in temp dir
17a656b [Nick Pentreath] remove binary sequencefile for tests
f60959e [Nick Pentreath] Remove msgpack dependency and serializer from PySpark
450e0a2 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
31a2fff [Nick Pentreath] Scalastyle fixes
fc5099e [Nick Pentreath] Add Apache license headers
4e08983 [Nick Pentreath] Clean up docs for PySpark context methods
b20ec7e [Nick Pentreath] Clean up merge duplicate dependencies
951c117 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
f6aac55 [Nick Pentreath] Bring back msgpack
9d2256e [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
1bbbfb0 [Nick Pentreath] Clean up SparkBuild from merge
a67dfad [Nick Pentreath] Clean up Msgpack serialization and registering
7237263 [Nick Pentreath] Add back msgpack serializer and hadoop file code lost during merging
25da1ca [Nick Pentreath] Add generator for nulls, bools, bytes and maps
65360d5 [Nick Pentreath] Adding test SequenceFiles
0c612e5 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
d72bf18 [Nick Pentreath] msgpack
dd57922 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
e67212a [Nick Pentreath] Add back msgpack dependency
f2d76a0 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
41856a5 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
97ef708 [Nick Pentreath] Remove old writeToStream
2beeedb [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
795a763 [Nick Pentreath] Change name to WriteInputFormatTestDataGenerator. Cleanup some var names. Use SPARK_HOME in path for writing test sequencefile data.
174f520 [Nick Pentreath] Add back graphx settings
703ee65 [Nick Pentreath] Add back msgpack
619c0fa [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
1c8efbc [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
eb40036 [Nick Pentreath] Remove unused comment lines
4d7ef2e [Nick Pentreath] Fix indentation
f1d73e3 [Nick Pentreath] mergeConfs returns a copy rather than mutating one of the input arguments
0f5cd84 [Nick Pentreath] Remove unused pair UTF8 class. Add comments to msgpack deserializer
4294cbb [Nick Pentreath] Add old Hadoop api methods. Clean up and expand comments. Clean up argument names
818a1e6 [Nick Pentreath] Add seqencefile and Hadoop InputFormat support to PythonRDD
4e7c9e3 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
c304cc8 [Nick Pentreath] Adding supporting sequncefiles for tests. Cleaning up
4b0a43f [Nick Pentreath] Refactoring utils into own objects. Cleaning up old commented-out code
d86325f [Nick Pentreath] Initial WIP of PySpark support for SequenceFile and arbitrary Hadoop InputFormat
2014-06-10 01:21:03 -04:00
|
|
|
clazz = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfclass/",
|
|
|
|
"org.apache.hadoop.io.Text",
|
|
|
|
"org.apache.spark.api.python.TestWritable").collect())
|
2014-11-04 02:56:14 -05:00
|
|
|
cname = u'org.apache.spark.api.python.TestWritable'
|
|
|
|
ec = [(u'1', {u'__class__': cname, u'double': 1.0, u'int': 1, u'str': u'test1'}),
|
|
|
|
(u'2', {u'__class__': cname, u'double': 2.3, u'int': 2, u'str': u'test2'}),
|
|
|
|
(u'3', {u'__class__': cname, u'double': 3.1, u'int': 3, u'str': u'test3'}),
|
|
|
|
(u'4', {u'__class__': cname, u'double': 4.2, u'int': 4, u'str': u'test4'}),
|
|
|
|
(u'5', {u'__class__': cname, u'double': 5.5, u'int': 5, u'str': u'test56'})]
|
|
|
|
self.assertEqual(clazz, ec)
|
SPARK-1416: PySpark support for SequenceFile and Hadoop InputFormats
So I finally resurrected this PR. It seems the old one against the incubator mirror is no longer available, so I cannot reference it.
This adds initial support for reading Hadoop ```SequenceFile```s, as well as arbitrary Hadoop ```InputFormat```s, in PySpark.
# Overview
The basics are as follows:
1. ```PythonRDD``` object contains the relevant methods, that are in turn invoked by ```SparkContext``` in PySpark
2. The SequenceFile or InputFormat is read on the Scala side and converted from ```Writable``` instances to the relevant Scala classes (in the case of primitives)
3. Pyrolite is used to serialize Java objects. If this fails, the fallback is ```toString```
4. ```PickleSerializer``` on the Python side deserializes.
This works "out the box" for simple ```Writable```s:
* ```Text```
* ```IntWritable```, ```DoubleWritable```, ```FloatWritable```
* ```NullWritable```
* ```BooleanWritable```
* ```BytesWritable```
* ```MapWritable```
It also works for simple, "struct-like" classes. Due to the way Pyrolite works, this requires that the classes satisfy the JavaBeans convenstions (i.e. with fields and a no-arg constructor and getters/setters). (Perhaps in future some sugar for case classes and reflection could be added).
I've tested it out with ```ESInputFormat``` as an example and it works very nicely:
```python
conf = {"es.resource" : "index/type" }
rdd = sc.newAPIHadoopRDD("org.elasticsearch.hadoop.mr.EsInputFormat", "org.apache.hadoop.io.NullWritable", "org.elasticsearch.hadoop.mr.LinkedMapWritable", conf=conf)
rdd.first()
```
I suspect for things like HBase/Cassandra it will be a bit trickier to get it to work out the box.
# Some things still outstanding:
1. ~~Requires ```msgpack-python``` and will fail without it. As originally discussed with Josh, add a ```as_strings``` argument that defaults to ```False```, that can be used if ```msgpack-python``` is not available~~
2. ~~I see from https://github.com/apache/spark/pull/363 that Pyrolite is being used there for SerDe between Scala and Python. @ahirreddy @mateiz what is the plan behind this - is Pyrolite preferred? It seems from a cursory glance that adapting the ```msgpack```-based SerDe here to use Pyrolite wouldn't be too hard~~
3. ~~Support the key and value "wrapper" that would allow a Scala/Java function to be plugged in that would transform whatever the key/value Writable class is into something that can be serialized (e.g. convert some custom Writable to a JavaBean or ```java.util.Map``` that can be easily serialized)~~
4. Support ```saveAsSequenceFile``` and ```saveAsHadoopFile``` etc. This would require SerDe in the reverse direction, that can be handled by Pyrolite. Will work on this as a separate PR
Author: Nick Pentreath <nick.pentreath@gmail.com>
Closes #455 from MLnick/pyspark-inputformats and squashes the following commits:
268df7e [Nick Pentreath] Documentation changes mer @pwendell comments
761269b [Nick Pentreath] Address @pwendell comments, simplify default writable conversions and remove registry.
4c972d8 [Nick Pentreath] Add license headers
d150431 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
cde6af9 [Nick Pentreath] Parameterize converter trait
5ebacfa [Nick Pentreath] Update docs for PySpark input formats
a985492 [Nick Pentreath] Move Converter examples to own package
365d0be [Nick Pentreath] Make classes private[python]. Add docs and @Experimental annotation to Converter interface.
eeb8205 [Nick Pentreath] Fix path relative to SPARK_HOME in tests
1eaa08b [Nick Pentreath] HBase -> Cassandra app name oversight
3f90c3e [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
2c18513 [Nick Pentreath] Add examples for reading HBase and Cassandra InputFormats from Python
b65606f [Nick Pentreath] Add converter interface
5757f6e [Nick Pentreath] Default key/value classes for sequenceFile asre None
085b55f [Nick Pentreath] Move input format tests to tests.py and clean up docs
43eb728 [Nick Pentreath] PySpark InputFormats docs into programming guide
94beedc [Nick Pentreath] Clean up args in PythonRDD. Set key/value converter defaults to None for PySpark context.py methods
1a4a1d6 [Nick Pentreath] Address @mateiz style comments
01e0813 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
15a7d07 [Nick Pentreath] Remove default args for key/value classes. Arg names to camelCase
9fe6bd5 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
84fe8e3 [Nick Pentreath] Python programming guide space formatting
d0f52b6 [Nick Pentreath] Python programming guide
7caa73a [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
93ef995 [Nick Pentreath] Add back context.py changes
9ef1896 [Nick Pentreath] Recover earlier changes lost in previous merge for serializers.py
077ecb2 [Nick Pentreath] Recover earlier changes lost in previous merge for context.py
5af4770 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
35b8e3a [Nick Pentreath] Another fix for test ordering
bef3afb [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
e001b94 [Nick Pentreath] Fix test failures due to ordering
78978d9 [Nick Pentreath] Add doc for SequenceFile and InputFormat support to Python programming guide
64eb051 [Nick Pentreath] Scalastyle fix
e7552fa [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
44f2857 [Nick Pentreath] Remove msgpack dependency and switch serialization to Pyrolite, plus some clean up and refactoring
c0ebfb6 [Nick Pentreath] Change sequencefile test data generator to easily be called from PySpark tests
1d7c17c [Nick Pentreath] Amend tests to auto-generate sequencefile data in temp dir
17a656b [Nick Pentreath] remove binary sequencefile for tests
f60959e [Nick Pentreath] Remove msgpack dependency and serializer from PySpark
450e0a2 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
31a2fff [Nick Pentreath] Scalastyle fixes
fc5099e [Nick Pentreath] Add Apache license headers
4e08983 [Nick Pentreath] Clean up docs for PySpark context methods
b20ec7e [Nick Pentreath] Clean up merge duplicate dependencies
951c117 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
f6aac55 [Nick Pentreath] Bring back msgpack
9d2256e [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
1bbbfb0 [Nick Pentreath] Clean up SparkBuild from merge
a67dfad [Nick Pentreath] Clean up Msgpack serialization and registering
7237263 [Nick Pentreath] Add back msgpack serializer and hadoop file code lost during merging
25da1ca [Nick Pentreath] Add generator for nulls, bools, bytes and maps
65360d5 [Nick Pentreath] Adding test SequenceFiles
0c612e5 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
d72bf18 [Nick Pentreath] msgpack
dd57922 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
e67212a [Nick Pentreath] Add back msgpack dependency
f2d76a0 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
41856a5 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
97ef708 [Nick Pentreath] Remove old writeToStream
2beeedb [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
795a763 [Nick Pentreath] Change name to WriteInputFormatTestDataGenerator. Cleanup some var names. Use SPARK_HOME in path for writing test sequencefile data.
174f520 [Nick Pentreath] Add back graphx settings
703ee65 [Nick Pentreath] Add back msgpack
619c0fa [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
1c8efbc [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
eb40036 [Nick Pentreath] Remove unused comment lines
4d7ef2e [Nick Pentreath] Fix indentation
f1d73e3 [Nick Pentreath] mergeConfs returns a copy rather than mutating one of the input arguments
0f5cd84 [Nick Pentreath] Remove unused pair UTF8 class. Add comments to msgpack deserializer
4294cbb [Nick Pentreath] Add old Hadoop api methods. Clean up and expand comments. Clean up argument names
818a1e6 [Nick Pentreath] Add seqencefile and Hadoop InputFormat support to PythonRDD
4e7c9e3 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
c304cc8 [Nick Pentreath] Adding supporting sequncefiles for tests. Cleaning up
4b0a43f [Nick Pentreath] Refactoring utils into own objects. Cleaning up old commented-out code
d86325f [Nick Pentreath] Initial WIP of PySpark support for SequenceFile and arbitrary Hadoop InputFormat
2014-06-10 01:21:03 -04:00
|
|
|
|
2014-07-30 16:19:05 -04:00
|
|
|
unbatched_clazz = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfclass/",
|
2014-08-06 15:58:24 -04:00
|
|
|
"org.apache.hadoop.io.Text",
|
|
|
|
"org.apache.spark.api.python.TestWritable",
|
2014-11-04 02:56:14 -05:00
|
|
|
).collect())
|
|
|
|
self.assertEqual(unbatched_clazz, ec)
|
2014-07-30 16:19:05 -04:00
|
|
|
|
SPARK-1416: PySpark support for SequenceFile and Hadoop InputFormats
So I finally resurrected this PR. It seems the old one against the incubator mirror is no longer available, so I cannot reference it.
This adds initial support for reading Hadoop ```SequenceFile```s, as well as arbitrary Hadoop ```InputFormat```s, in PySpark.
# Overview
The basics are as follows:
1. ```PythonRDD``` object contains the relevant methods, that are in turn invoked by ```SparkContext``` in PySpark
2. The SequenceFile or InputFormat is read on the Scala side and converted from ```Writable``` instances to the relevant Scala classes (in the case of primitives)
3. Pyrolite is used to serialize Java objects. If this fails, the fallback is ```toString```
4. ```PickleSerializer``` on the Python side deserializes.
This works "out the box" for simple ```Writable```s:
* ```Text```
* ```IntWritable```, ```DoubleWritable```, ```FloatWritable```
* ```NullWritable```
* ```BooleanWritable```
* ```BytesWritable```
* ```MapWritable```
It also works for simple, "struct-like" classes. Due to the way Pyrolite works, this requires that the classes satisfy the JavaBeans convenstions (i.e. with fields and a no-arg constructor and getters/setters). (Perhaps in future some sugar for case classes and reflection could be added).
I've tested it out with ```ESInputFormat``` as an example and it works very nicely:
```python
conf = {"es.resource" : "index/type" }
rdd = sc.newAPIHadoopRDD("org.elasticsearch.hadoop.mr.EsInputFormat", "org.apache.hadoop.io.NullWritable", "org.elasticsearch.hadoop.mr.LinkedMapWritable", conf=conf)
rdd.first()
```
I suspect for things like HBase/Cassandra it will be a bit trickier to get it to work out the box.
# Some things still outstanding:
1. ~~Requires ```msgpack-python``` and will fail without it. As originally discussed with Josh, add a ```as_strings``` argument that defaults to ```False```, that can be used if ```msgpack-python``` is not available~~
2. ~~I see from https://github.com/apache/spark/pull/363 that Pyrolite is being used there for SerDe between Scala and Python. @ahirreddy @mateiz what is the plan behind this - is Pyrolite preferred? It seems from a cursory glance that adapting the ```msgpack```-based SerDe here to use Pyrolite wouldn't be too hard~~
3. ~~Support the key and value "wrapper" that would allow a Scala/Java function to be plugged in that would transform whatever the key/value Writable class is into something that can be serialized (e.g. convert some custom Writable to a JavaBean or ```java.util.Map``` that can be easily serialized)~~
4. Support ```saveAsSequenceFile``` and ```saveAsHadoopFile``` etc. This would require SerDe in the reverse direction, that can be handled by Pyrolite. Will work on this as a separate PR
Author: Nick Pentreath <nick.pentreath@gmail.com>
Closes #455 from MLnick/pyspark-inputformats and squashes the following commits:
268df7e [Nick Pentreath] Documentation changes mer @pwendell comments
761269b [Nick Pentreath] Address @pwendell comments, simplify default writable conversions and remove registry.
4c972d8 [Nick Pentreath] Add license headers
d150431 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
cde6af9 [Nick Pentreath] Parameterize converter trait
5ebacfa [Nick Pentreath] Update docs for PySpark input formats
a985492 [Nick Pentreath] Move Converter examples to own package
365d0be [Nick Pentreath] Make classes private[python]. Add docs and @Experimental annotation to Converter interface.
eeb8205 [Nick Pentreath] Fix path relative to SPARK_HOME in tests
1eaa08b [Nick Pentreath] HBase -> Cassandra app name oversight
3f90c3e [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
2c18513 [Nick Pentreath] Add examples for reading HBase and Cassandra InputFormats from Python
b65606f [Nick Pentreath] Add converter interface
5757f6e [Nick Pentreath] Default key/value classes for sequenceFile asre None
085b55f [Nick Pentreath] Move input format tests to tests.py and clean up docs
43eb728 [Nick Pentreath] PySpark InputFormats docs into programming guide
94beedc [Nick Pentreath] Clean up args in PythonRDD. Set key/value converter defaults to None for PySpark context.py methods
1a4a1d6 [Nick Pentreath] Address @mateiz style comments
01e0813 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
15a7d07 [Nick Pentreath] Remove default args for key/value classes. Arg names to camelCase
9fe6bd5 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
84fe8e3 [Nick Pentreath] Python programming guide space formatting
d0f52b6 [Nick Pentreath] Python programming guide
7caa73a [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
93ef995 [Nick Pentreath] Add back context.py changes
9ef1896 [Nick Pentreath] Recover earlier changes lost in previous merge for serializers.py
077ecb2 [Nick Pentreath] Recover earlier changes lost in previous merge for context.py
5af4770 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
35b8e3a [Nick Pentreath] Another fix for test ordering
bef3afb [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
e001b94 [Nick Pentreath] Fix test failures due to ordering
78978d9 [Nick Pentreath] Add doc for SequenceFile and InputFormat support to Python programming guide
64eb051 [Nick Pentreath] Scalastyle fix
e7552fa [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
44f2857 [Nick Pentreath] Remove msgpack dependency and switch serialization to Pyrolite, plus some clean up and refactoring
c0ebfb6 [Nick Pentreath] Change sequencefile test data generator to easily be called from PySpark tests
1d7c17c [Nick Pentreath] Amend tests to auto-generate sequencefile data in temp dir
17a656b [Nick Pentreath] remove binary sequencefile for tests
f60959e [Nick Pentreath] Remove msgpack dependency and serializer from PySpark
450e0a2 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
31a2fff [Nick Pentreath] Scalastyle fixes
fc5099e [Nick Pentreath] Add Apache license headers
4e08983 [Nick Pentreath] Clean up docs for PySpark context methods
b20ec7e [Nick Pentreath] Clean up merge duplicate dependencies
951c117 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
f6aac55 [Nick Pentreath] Bring back msgpack
9d2256e [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
1bbbfb0 [Nick Pentreath] Clean up SparkBuild from merge
a67dfad [Nick Pentreath] Clean up Msgpack serialization and registering
7237263 [Nick Pentreath] Add back msgpack serializer and hadoop file code lost during merging
25da1ca [Nick Pentreath] Add generator for nulls, bools, bytes and maps
65360d5 [Nick Pentreath] Adding test SequenceFiles
0c612e5 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
d72bf18 [Nick Pentreath] msgpack
dd57922 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
e67212a [Nick Pentreath] Add back msgpack dependency
f2d76a0 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
41856a5 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
97ef708 [Nick Pentreath] Remove old writeToStream
2beeedb [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
795a763 [Nick Pentreath] Change name to WriteInputFormatTestDataGenerator. Cleanup some var names. Use SPARK_HOME in path for writing test sequencefile data.
174f520 [Nick Pentreath] Add back graphx settings
703ee65 [Nick Pentreath] Add back msgpack
619c0fa [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
1c8efbc [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
eb40036 [Nick Pentreath] Remove unused comment lines
4d7ef2e [Nick Pentreath] Fix indentation
f1d73e3 [Nick Pentreath] mergeConfs returns a copy rather than mutating one of the input arguments
0f5cd84 [Nick Pentreath] Remove unused pair UTF8 class. Add comments to msgpack deserializer
4294cbb [Nick Pentreath] Add old Hadoop api methods. Clean up and expand comments. Clean up argument names
818a1e6 [Nick Pentreath] Add seqencefile and Hadoop InputFormat support to PythonRDD
4e7c9e3 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
c304cc8 [Nick Pentreath] Adding supporting sequncefiles for tests. Cleaning up
4b0a43f [Nick Pentreath] Refactoring utils into own objects. Cleaning up old commented-out code
d86325f [Nick Pentreath] Initial WIP of PySpark support for SequenceFile and arbitrary Hadoop InputFormat
2014-06-10 01:21:03 -04:00
|
|
|
def test_oldhadoop(self):
|
|
|
|
basepath = self.tempdir.name
|
|
|
|
ints = sorted(self.sc.hadoopFile(basepath + "/sftestdata/sfint/",
|
|
|
|
"org.apache.hadoop.mapred.SequenceFileInputFormat",
|
|
|
|
"org.apache.hadoop.io.IntWritable",
|
|
|
|
"org.apache.hadoop.io.Text").collect())
|
|
|
|
ei = [(1, u'aa'), (1, u'aa'), (2, u'aa'), (2, u'bb'), (2, u'bb'), (3, u'cc')]
|
|
|
|
self.assertEqual(ints, ei)
|
|
|
|
|
2016-09-21 04:37:03 -04:00
|
|
|
hellopath = os.path.join(SPARK_HOME, "python/test_support/hello/hello.txt")
|
2017-02-28 05:13:42 -05:00
|
|
|
oldconf = {"mapreduce.input.fileinputformat.inputdir": hellopath}
|
2014-07-30 16:19:05 -04:00
|
|
|
hello = self.sc.hadoopRDD("org.apache.hadoop.mapred.TextInputFormat",
|
|
|
|
"org.apache.hadoop.io.LongWritable",
|
|
|
|
"org.apache.hadoop.io.Text",
|
|
|
|
conf=oldconf).collect()
|
SPARK-1416: PySpark support for SequenceFile and Hadoop InputFormats
So I finally resurrected this PR. It seems the old one against the incubator mirror is no longer available, so I cannot reference it.
This adds initial support for reading Hadoop ```SequenceFile```s, as well as arbitrary Hadoop ```InputFormat```s, in PySpark.
# Overview
The basics are as follows:
1. ```PythonRDD``` object contains the relevant methods, that are in turn invoked by ```SparkContext``` in PySpark
2. The SequenceFile or InputFormat is read on the Scala side and converted from ```Writable``` instances to the relevant Scala classes (in the case of primitives)
3. Pyrolite is used to serialize Java objects. If this fails, the fallback is ```toString```
4. ```PickleSerializer``` on the Python side deserializes.
This works "out the box" for simple ```Writable```s:
* ```Text```
* ```IntWritable```, ```DoubleWritable```, ```FloatWritable```
* ```NullWritable```
* ```BooleanWritable```
* ```BytesWritable```
* ```MapWritable```
It also works for simple, "struct-like" classes. Due to the way Pyrolite works, this requires that the classes satisfy the JavaBeans convenstions (i.e. with fields and a no-arg constructor and getters/setters). (Perhaps in future some sugar for case classes and reflection could be added).
I've tested it out with ```ESInputFormat``` as an example and it works very nicely:
```python
conf = {"es.resource" : "index/type" }
rdd = sc.newAPIHadoopRDD("org.elasticsearch.hadoop.mr.EsInputFormat", "org.apache.hadoop.io.NullWritable", "org.elasticsearch.hadoop.mr.LinkedMapWritable", conf=conf)
rdd.first()
```
I suspect for things like HBase/Cassandra it will be a bit trickier to get it to work out the box.
# Some things still outstanding:
1. ~~Requires ```msgpack-python``` and will fail without it. As originally discussed with Josh, add a ```as_strings``` argument that defaults to ```False```, that can be used if ```msgpack-python``` is not available~~
2. ~~I see from https://github.com/apache/spark/pull/363 that Pyrolite is being used there for SerDe between Scala and Python. @ahirreddy @mateiz what is the plan behind this - is Pyrolite preferred? It seems from a cursory glance that adapting the ```msgpack```-based SerDe here to use Pyrolite wouldn't be too hard~~
3. ~~Support the key and value "wrapper" that would allow a Scala/Java function to be plugged in that would transform whatever the key/value Writable class is into something that can be serialized (e.g. convert some custom Writable to a JavaBean or ```java.util.Map``` that can be easily serialized)~~
4. Support ```saveAsSequenceFile``` and ```saveAsHadoopFile``` etc. This would require SerDe in the reverse direction, that can be handled by Pyrolite. Will work on this as a separate PR
Author: Nick Pentreath <nick.pentreath@gmail.com>
Closes #455 from MLnick/pyspark-inputformats and squashes the following commits:
268df7e [Nick Pentreath] Documentation changes mer @pwendell comments
761269b [Nick Pentreath] Address @pwendell comments, simplify default writable conversions and remove registry.
4c972d8 [Nick Pentreath] Add license headers
d150431 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
cde6af9 [Nick Pentreath] Parameterize converter trait
5ebacfa [Nick Pentreath] Update docs for PySpark input formats
a985492 [Nick Pentreath] Move Converter examples to own package
365d0be [Nick Pentreath] Make classes private[python]. Add docs and @Experimental annotation to Converter interface.
eeb8205 [Nick Pentreath] Fix path relative to SPARK_HOME in tests
1eaa08b [Nick Pentreath] HBase -> Cassandra app name oversight
3f90c3e [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
2c18513 [Nick Pentreath] Add examples for reading HBase and Cassandra InputFormats from Python
b65606f [Nick Pentreath] Add converter interface
5757f6e [Nick Pentreath] Default key/value classes for sequenceFile asre None
085b55f [Nick Pentreath] Move input format tests to tests.py and clean up docs
43eb728 [Nick Pentreath] PySpark InputFormats docs into programming guide
94beedc [Nick Pentreath] Clean up args in PythonRDD. Set key/value converter defaults to None for PySpark context.py methods
1a4a1d6 [Nick Pentreath] Address @mateiz style comments
01e0813 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
15a7d07 [Nick Pentreath] Remove default args for key/value classes. Arg names to camelCase
9fe6bd5 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
84fe8e3 [Nick Pentreath] Python programming guide space formatting
d0f52b6 [Nick Pentreath] Python programming guide
7caa73a [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
93ef995 [Nick Pentreath] Add back context.py changes
9ef1896 [Nick Pentreath] Recover earlier changes lost in previous merge for serializers.py
077ecb2 [Nick Pentreath] Recover earlier changes lost in previous merge for context.py
5af4770 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
35b8e3a [Nick Pentreath] Another fix for test ordering
bef3afb [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
e001b94 [Nick Pentreath] Fix test failures due to ordering
78978d9 [Nick Pentreath] Add doc for SequenceFile and InputFormat support to Python programming guide
64eb051 [Nick Pentreath] Scalastyle fix
e7552fa [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
44f2857 [Nick Pentreath] Remove msgpack dependency and switch serialization to Pyrolite, plus some clean up and refactoring
c0ebfb6 [Nick Pentreath] Change sequencefile test data generator to easily be called from PySpark tests
1d7c17c [Nick Pentreath] Amend tests to auto-generate sequencefile data in temp dir
17a656b [Nick Pentreath] remove binary sequencefile for tests
f60959e [Nick Pentreath] Remove msgpack dependency and serializer from PySpark
450e0a2 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
31a2fff [Nick Pentreath] Scalastyle fixes
fc5099e [Nick Pentreath] Add Apache license headers
4e08983 [Nick Pentreath] Clean up docs for PySpark context methods
b20ec7e [Nick Pentreath] Clean up merge duplicate dependencies
951c117 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
f6aac55 [Nick Pentreath] Bring back msgpack
9d2256e [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
1bbbfb0 [Nick Pentreath] Clean up SparkBuild from merge
a67dfad [Nick Pentreath] Clean up Msgpack serialization and registering
7237263 [Nick Pentreath] Add back msgpack serializer and hadoop file code lost during merging
25da1ca [Nick Pentreath] Add generator for nulls, bools, bytes and maps
65360d5 [Nick Pentreath] Adding test SequenceFiles
0c612e5 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
d72bf18 [Nick Pentreath] msgpack
dd57922 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
e67212a [Nick Pentreath] Add back msgpack dependency
f2d76a0 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
41856a5 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
97ef708 [Nick Pentreath] Remove old writeToStream
2beeedb [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
795a763 [Nick Pentreath] Change name to WriteInputFormatTestDataGenerator. Cleanup some var names. Use SPARK_HOME in path for writing test sequencefile data.
174f520 [Nick Pentreath] Add back graphx settings
703ee65 [Nick Pentreath] Add back msgpack
619c0fa [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
1c8efbc [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
eb40036 [Nick Pentreath] Remove unused comment lines
4d7ef2e [Nick Pentreath] Fix indentation
f1d73e3 [Nick Pentreath] mergeConfs returns a copy rather than mutating one of the input arguments
0f5cd84 [Nick Pentreath] Remove unused pair UTF8 class. Add comments to msgpack deserializer
4294cbb [Nick Pentreath] Add old Hadoop api methods. Clean up and expand comments. Clean up argument names
818a1e6 [Nick Pentreath] Add seqencefile and Hadoop InputFormat support to PythonRDD
4e7c9e3 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
c304cc8 [Nick Pentreath] Adding supporting sequncefiles for tests. Cleaning up
4b0a43f [Nick Pentreath] Refactoring utils into own objects. Cleaning up old commented-out code
d86325f [Nick Pentreath] Initial WIP of PySpark support for SequenceFile and arbitrary Hadoop InputFormat
2014-06-10 01:21:03 -04:00
|
|
|
result = [(0, u'Hello World!')]
|
|
|
|
self.assertEqual(hello, result)
|
|
|
|
|
|
|
|
def test_newhadoop(self):
|
|
|
|
basepath = self.tempdir.name
|
|
|
|
ints = sorted(self.sc.newAPIHadoopFile(
|
|
|
|
basepath + "/sftestdata/sfint/",
|
|
|
|
"org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat",
|
|
|
|
"org.apache.hadoop.io.IntWritable",
|
|
|
|
"org.apache.hadoop.io.Text").collect())
|
|
|
|
ei = [(1, u'aa'), (1, u'aa'), (2, u'aa'), (2, u'bb'), (2, u'bb'), (3, u'cc')]
|
|
|
|
self.assertEqual(ints, ei)
|
|
|
|
|
2016-09-21 04:37:03 -04:00
|
|
|
hellopath = os.path.join(SPARK_HOME, "python/test_support/hello/hello.txt")
|
2017-02-28 05:13:42 -05:00
|
|
|
newconf = {"mapreduce.input.fileinputformat.inputdir": hellopath}
|
2014-07-30 16:19:05 -04:00
|
|
|
hello = self.sc.newAPIHadoopRDD("org.apache.hadoop.mapreduce.lib.input.TextInputFormat",
|
|
|
|
"org.apache.hadoop.io.LongWritable",
|
|
|
|
"org.apache.hadoop.io.Text",
|
|
|
|
conf=newconf).collect()
|
SPARK-1416: PySpark support for SequenceFile and Hadoop InputFormats
So I finally resurrected this PR. It seems the old one against the incubator mirror is no longer available, so I cannot reference it.
This adds initial support for reading Hadoop ```SequenceFile```s, as well as arbitrary Hadoop ```InputFormat```s, in PySpark.
# Overview
The basics are as follows:
1. ```PythonRDD``` object contains the relevant methods, that are in turn invoked by ```SparkContext``` in PySpark
2. The SequenceFile or InputFormat is read on the Scala side and converted from ```Writable``` instances to the relevant Scala classes (in the case of primitives)
3. Pyrolite is used to serialize Java objects. If this fails, the fallback is ```toString```
4. ```PickleSerializer``` on the Python side deserializes.
This works "out the box" for simple ```Writable```s:
* ```Text```
* ```IntWritable```, ```DoubleWritable```, ```FloatWritable```
* ```NullWritable```
* ```BooleanWritable```
* ```BytesWritable```
* ```MapWritable```
It also works for simple, "struct-like" classes. Due to the way Pyrolite works, this requires that the classes satisfy the JavaBeans convenstions (i.e. with fields and a no-arg constructor and getters/setters). (Perhaps in future some sugar for case classes and reflection could be added).
I've tested it out with ```ESInputFormat``` as an example and it works very nicely:
```python
conf = {"es.resource" : "index/type" }
rdd = sc.newAPIHadoopRDD("org.elasticsearch.hadoop.mr.EsInputFormat", "org.apache.hadoop.io.NullWritable", "org.elasticsearch.hadoop.mr.LinkedMapWritable", conf=conf)
rdd.first()
```
I suspect for things like HBase/Cassandra it will be a bit trickier to get it to work out the box.
# Some things still outstanding:
1. ~~Requires ```msgpack-python``` and will fail without it. As originally discussed with Josh, add a ```as_strings``` argument that defaults to ```False```, that can be used if ```msgpack-python``` is not available~~
2. ~~I see from https://github.com/apache/spark/pull/363 that Pyrolite is being used there for SerDe between Scala and Python. @ahirreddy @mateiz what is the plan behind this - is Pyrolite preferred? It seems from a cursory glance that adapting the ```msgpack```-based SerDe here to use Pyrolite wouldn't be too hard~~
3. ~~Support the key and value "wrapper" that would allow a Scala/Java function to be plugged in that would transform whatever the key/value Writable class is into something that can be serialized (e.g. convert some custom Writable to a JavaBean or ```java.util.Map``` that can be easily serialized)~~
4. Support ```saveAsSequenceFile``` and ```saveAsHadoopFile``` etc. This would require SerDe in the reverse direction, that can be handled by Pyrolite. Will work on this as a separate PR
Author: Nick Pentreath <nick.pentreath@gmail.com>
Closes #455 from MLnick/pyspark-inputformats and squashes the following commits:
268df7e [Nick Pentreath] Documentation changes mer @pwendell comments
761269b [Nick Pentreath] Address @pwendell comments, simplify default writable conversions and remove registry.
4c972d8 [Nick Pentreath] Add license headers
d150431 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
cde6af9 [Nick Pentreath] Parameterize converter trait
5ebacfa [Nick Pentreath] Update docs for PySpark input formats
a985492 [Nick Pentreath] Move Converter examples to own package
365d0be [Nick Pentreath] Make classes private[python]. Add docs and @Experimental annotation to Converter interface.
eeb8205 [Nick Pentreath] Fix path relative to SPARK_HOME in tests
1eaa08b [Nick Pentreath] HBase -> Cassandra app name oversight
3f90c3e [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
2c18513 [Nick Pentreath] Add examples for reading HBase and Cassandra InputFormats from Python
b65606f [Nick Pentreath] Add converter interface
5757f6e [Nick Pentreath] Default key/value classes for sequenceFile asre None
085b55f [Nick Pentreath] Move input format tests to tests.py and clean up docs
43eb728 [Nick Pentreath] PySpark InputFormats docs into programming guide
94beedc [Nick Pentreath] Clean up args in PythonRDD. Set key/value converter defaults to None for PySpark context.py methods
1a4a1d6 [Nick Pentreath] Address @mateiz style comments
01e0813 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
15a7d07 [Nick Pentreath] Remove default args for key/value classes. Arg names to camelCase
9fe6bd5 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
84fe8e3 [Nick Pentreath] Python programming guide space formatting
d0f52b6 [Nick Pentreath] Python programming guide
7caa73a [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
93ef995 [Nick Pentreath] Add back context.py changes
9ef1896 [Nick Pentreath] Recover earlier changes lost in previous merge for serializers.py
077ecb2 [Nick Pentreath] Recover earlier changes lost in previous merge for context.py
5af4770 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
35b8e3a [Nick Pentreath] Another fix for test ordering
bef3afb [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
e001b94 [Nick Pentreath] Fix test failures due to ordering
78978d9 [Nick Pentreath] Add doc for SequenceFile and InputFormat support to Python programming guide
64eb051 [Nick Pentreath] Scalastyle fix
e7552fa [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
44f2857 [Nick Pentreath] Remove msgpack dependency and switch serialization to Pyrolite, plus some clean up and refactoring
c0ebfb6 [Nick Pentreath] Change sequencefile test data generator to easily be called from PySpark tests
1d7c17c [Nick Pentreath] Amend tests to auto-generate sequencefile data in temp dir
17a656b [Nick Pentreath] remove binary sequencefile for tests
f60959e [Nick Pentreath] Remove msgpack dependency and serializer from PySpark
450e0a2 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
31a2fff [Nick Pentreath] Scalastyle fixes
fc5099e [Nick Pentreath] Add Apache license headers
4e08983 [Nick Pentreath] Clean up docs for PySpark context methods
b20ec7e [Nick Pentreath] Clean up merge duplicate dependencies
951c117 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
f6aac55 [Nick Pentreath] Bring back msgpack
9d2256e [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
1bbbfb0 [Nick Pentreath] Clean up SparkBuild from merge
a67dfad [Nick Pentreath] Clean up Msgpack serialization and registering
7237263 [Nick Pentreath] Add back msgpack serializer and hadoop file code lost during merging
25da1ca [Nick Pentreath] Add generator for nulls, bools, bytes and maps
65360d5 [Nick Pentreath] Adding test SequenceFiles
0c612e5 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
d72bf18 [Nick Pentreath] msgpack
dd57922 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
e67212a [Nick Pentreath] Add back msgpack dependency
f2d76a0 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
41856a5 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
97ef708 [Nick Pentreath] Remove old writeToStream
2beeedb [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
795a763 [Nick Pentreath] Change name to WriteInputFormatTestDataGenerator. Cleanup some var names. Use SPARK_HOME in path for writing test sequencefile data.
174f520 [Nick Pentreath] Add back graphx settings
703ee65 [Nick Pentreath] Add back msgpack
619c0fa [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
1c8efbc [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
eb40036 [Nick Pentreath] Remove unused comment lines
4d7ef2e [Nick Pentreath] Fix indentation
f1d73e3 [Nick Pentreath] mergeConfs returns a copy rather than mutating one of the input arguments
0f5cd84 [Nick Pentreath] Remove unused pair UTF8 class. Add comments to msgpack deserializer
4294cbb [Nick Pentreath] Add old Hadoop api methods. Clean up and expand comments. Clean up argument names
818a1e6 [Nick Pentreath] Add seqencefile and Hadoop InputFormat support to PythonRDD
4e7c9e3 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
c304cc8 [Nick Pentreath] Adding supporting sequncefiles for tests. Cleaning up
4b0a43f [Nick Pentreath] Refactoring utils into own objects. Cleaning up old commented-out code
d86325f [Nick Pentreath] Initial WIP of PySpark support for SequenceFile and arbitrary Hadoop InputFormat
2014-06-10 01:21:03 -04:00
|
|
|
result = [(0, u'Hello World!')]
|
|
|
|
self.assertEqual(hello, result)
|
|
|
|
|
|
|
|
def test_newolderror(self):
|
|
|
|
basepath = self.tempdir.name
|
|
|
|
self.assertRaises(Exception, lambda: self.sc.hadoopFile(
|
|
|
|
basepath + "/sftestdata/sfint/",
|
|
|
|
"org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat",
|
|
|
|
"org.apache.hadoop.io.IntWritable",
|
|
|
|
"org.apache.hadoop.io.Text"))
|
|
|
|
|
|
|
|
self.assertRaises(Exception, lambda: self.sc.newAPIHadoopFile(
|
|
|
|
basepath + "/sftestdata/sfint/",
|
|
|
|
"org.apache.hadoop.mapred.SequenceFileInputFormat",
|
|
|
|
"org.apache.hadoop.io.IntWritable",
|
|
|
|
"org.apache.hadoop.io.Text"))
|
|
|
|
|
|
|
|
def test_bad_inputs(self):
|
|
|
|
basepath = self.tempdir.name
|
|
|
|
self.assertRaises(Exception, lambda: self.sc.sequenceFile(
|
|
|
|
basepath + "/sftestdata/sfint/",
|
|
|
|
"org.apache.hadoop.io.NotValidWritable",
|
|
|
|
"org.apache.hadoop.io.Text"))
|
|
|
|
self.assertRaises(Exception, lambda: self.sc.hadoopFile(
|
|
|
|
basepath + "/sftestdata/sfint/",
|
|
|
|
"org.apache.hadoop.mapred.NotValidInputFormat",
|
|
|
|
"org.apache.hadoop.io.IntWritable",
|
|
|
|
"org.apache.hadoop.io.Text"))
|
|
|
|
self.assertRaises(Exception, lambda: self.sc.newAPIHadoopFile(
|
|
|
|
basepath + "/sftestdata/sfint/",
|
|
|
|
"org.apache.hadoop.mapreduce.lib.input.NotValidInputFormat",
|
|
|
|
"org.apache.hadoop.io.IntWritable",
|
|
|
|
"org.apache.hadoop.io.Text"))
|
|
|
|
|
2014-07-30 16:19:05 -04:00
|
|
|
def test_converters(self):
|
|
|
|
# use of custom converters
|
SPARK-1416: PySpark support for SequenceFile and Hadoop InputFormats
So I finally resurrected this PR. It seems the old one against the incubator mirror is no longer available, so I cannot reference it.
This adds initial support for reading Hadoop ```SequenceFile```s, as well as arbitrary Hadoop ```InputFormat```s, in PySpark.
# Overview
The basics are as follows:
1. ```PythonRDD``` object contains the relevant methods, that are in turn invoked by ```SparkContext``` in PySpark
2. The SequenceFile or InputFormat is read on the Scala side and converted from ```Writable``` instances to the relevant Scala classes (in the case of primitives)
3. Pyrolite is used to serialize Java objects. If this fails, the fallback is ```toString```
4. ```PickleSerializer``` on the Python side deserializes.
This works "out the box" for simple ```Writable```s:
* ```Text```
* ```IntWritable```, ```DoubleWritable```, ```FloatWritable```
* ```NullWritable```
* ```BooleanWritable```
* ```BytesWritable```
* ```MapWritable```
It also works for simple, "struct-like" classes. Due to the way Pyrolite works, this requires that the classes satisfy the JavaBeans convenstions (i.e. with fields and a no-arg constructor and getters/setters). (Perhaps in future some sugar for case classes and reflection could be added).
I've tested it out with ```ESInputFormat``` as an example and it works very nicely:
```python
conf = {"es.resource" : "index/type" }
rdd = sc.newAPIHadoopRDD("org.elasticsearch.hadoop.mr.EsInputFormat", "org.apache.hadoop.io.NullWritable", "org.elasticsearch.hadoop.mr.LinkedMapWritable", conf=conf)
rdd.first()
```
I suspect for things like HBase/Cassandra it will be a bit trickier to get it to work out the box.
# Some things still outstanding:
1. ~~Requires ```msgpack-python``` and will fail without it. As originally discussed with Josh, add a ```as_strings``` argument that defaults to ```False```, that can be used if ```msgpack-python``` is not available~~
2. ~~I see from https://github.com/apache/spark/pull/363 that Pyrolite is being used there for SerDe between Scala and Python. @ahirreddy @mateiz what is the plan behind this - is Pyrolite preferred? It seems from a cursory glance that adapting the ```msgpack```-based SerDe here to use Pyrolite wouldn't be too hard~~
3. ~~Support the key and value "wrapper" that would allow a Scala/Java function to be plugged in that would transform whatever the key/value Writable class is into something that can be serialized (e.g. convert some custom Writable to a JavaBean or ```java.util.Map``` that can be easily serialized)~~
4. Support ```saveAsSequenceFile``` and ```saveAsHadoopFile``` etc. This would require SerDe in the reverse direction, that can be handled by Pyrolite. Will work on this as a separate PR
Author: Nick Pentreath <nick.pentreath@gmail.com>
Closes #455 from MLnick/pyspark-inputformats and squashes the following commits:
268df7e [Nick Pentreath] Documentation changes mer @pwendell comments
761269b [Nick Pentreath] Address @pwendell comments, simplify default writable conversions and remove registry.
4c972d8 [Nick Pentreath] Add license headers
d150431 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
cde6af9 [Nick Pentreath] Parameterize converter trait
5ebacfa [Nick Pentreath] Update docs for PySpark input formats
a985492 [Nick Pentreath] Move Converter examples to own package
365d0be [Nick Pentreath] Make classes private[python]. Add docs and @Experimental annotation to Converter interface.
eeb8205 [Nick Pentreath] Fix path relative to SPARK_HOME in tests
1eaa08b [Nick Pentreath] HBase -> Cassandra app name oversight
3f90c3e [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
2c18513 [Nick Pentreath] Add examples for reading HBase and Cassandra InputFormats from Python
b65606f [Nick Pentreath] Add converter interface
5757f6e [Nick Pentreath] Default key/value classes for sequenceFile asre None
085b55f [Nick Pentreath] Move input format tests to tests.py and clean up docs
43eb728 [Nick Pentreath] PySpark InputFormats docs into programming guide
94beedc [Nick Pentreath] Clean up args in PythonRDD. Set key/value converter defaults to None for PySpark context.py methods
1a4a1d6 [Nick Pentreath] Address @mateiz style comments
01e0813 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
15a7d07 [Nick Pentreath] Remove default args for key/value classes. Arg names to camelCase
9fe6bd5 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
84fe8e3 [Nick Pentreath] Python programming guide space formatting
d0f52b6 [Nick Pentreath] Python programming guide
7caa73a [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
93ef995 [Nick Pentreath] Add back context.py changes
9ef1896 [Nick Pentreath] Recover earlier changes lost in previous merge for serializers.py
077ecb2 [Nick Pentreath] Recover earlier changes lost in previous merge for context.py
5af4770 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
35b8e3a [Nick Pentreath] Another fix for test ordering
bef3afb [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
e001b94 [Nick Pentreath] Fix test failures due to ordering
78978d9 [Nick Pentreath] Add doc for SequenceFile and InputFormat support to Python programming guide
64eb051 [Nick Pentreath] Scalastyle fix
e7552fa [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
44f2857 [Nick Pentreath] Remove msgpack dependency and switch serialization to Pyrolite, plus some clean up and refactoring
c0ebfb6 [Nick Pentreath] Change sequencefile test data generator to easily be called from PySpark tests
1d7c17c [Nick Pentreath] Amend tests to auto-generate sequencefile data in temp dir
17a656b [Nick Pentreath] remove binary sequencefile for tests
f60959e [Nick Pentreath] Remove msgpack dependency and serializer from PySpark
450e0a2 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
31a2fff [Nick Pentreath] Scalastyle fixes
fc5099e [Nick Pentreath] Add Apache license headers
4e08983 [Nick Pentreath] Clean up docs for PySpark context methods
b20ec7e [Nick Pentreath] Clean up merge duplicate dependencies
951c117 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
f6aac55 [Nick Pentreath] Bring back msgpack
9d2256e [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
1bbbfb0 [Nick Pentreath] Clean up SparkBuild from merge
a67dfad [Nick Pentreath] Clean up Msgpack serialization and registering
7237263 [Nick Pentreath] Add back msgpack serializer and hadoop file code lost during merging
25da1ca [Nick Pentreath] Add generator for nulls, bools, bytes and maps
65360d5 [Nick Pentreath] Adding test SequenceFiles
0c612e5 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
d72bf18 [Nick Pentreath] msgpack
dd57922 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
e67212a [Nick Pentreath] Add back msgpack dependency
f2d76a0 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
41856a5 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
97ef708 [Nick Pentreath] Remove old writeToStream
2beeedb [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
795a763 [Nick Pentreath] Change name to WriteInputFormatTestDataGenerator. Cleanup some var names. Use SPARK_HOME in path for writing test sequencefile data.
174f520 [Nick Pentreath] Add back graphx settings
703ee65 [Nick Pentreath] Add back msgpack
619c0fa [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
1c8efbc [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
eb40036 [Nick Pentreath] Remove unused comment lines
4d7ef2e [Nick Pentreath] Fix indentation
f1d73e3 [Nick Pentreath] mergeConfs returns a copy rather than mutating one of the input arguments
0f5cd84 [Nick Pentreath] Remove unused pair UTF8 class. Add comments to msgpack deserializer
4294cbb [Nick Pentreath] Add old Hadoop api methods. Clean up and expand comments. Clean up argument names
818a1e6 [Nick Pentreath] Add seqencefile and Hadoop InputFormat support to PythonRDD
4e7c9e3 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
c304cc8 [Nick Pentreath] Adding supporting sequncefiles for tests. Cleaning up
4b0a43f [Nick Pentreath] Refactoring utils into own objects. Cleaning up old commented-out code
d86325f [Nick Pentreath] Initial WIP of PySpark support for SequenceFile and arbitrary Hadoop InputFormat
2014-06-10 01:21:03 -04:00
|
|
|
basepath = self.tempdir.name
|
|
|
|
maps = sorted(self.sc.sequenceFile(
|
|
|
|
basepath + "/sftestdata/sfmap/",
|
|
|
|
"org.apache.hadoop.io.IntWritable",
|
|
|
|
"org.apache.hadoop.io.MapWritable",
|
2014-07-30 16:19:05 -04:00
|
|
|
keyConverter="org.apache.spark.api.python.TestInputKeyConverter",
|
|
|
|
valueConverter="org.apache.spark.api.python.TestInputValueConverter").collect())
|
|
|
|
em = [(u'\x01', []),
|
|
|
|
(u'\x01', [3.0]),
|
|
|
|
(u'\x02', [1.0]),
|
|
|
|
(u'\x02', [1.0]),
|
|
|
|
(u'\x03', [2.0])]
|
|
|
|
self.assertEqual(maps, em)
|
|
|
|
|
[SPARK-4186] add binaryFiles and binaryRecords in Python
add binaryFiles() and binaryRecords() in Python
```
binaryFiles(self, path, minPartitions=None):
:: Developer API ::
Read a directory of binary files from HDFS, a local file system
(available on all nodes), or any Hadoop-supported file system URI
as a byte array. Each file is read as a single record and returned
in a key-value pair, where the key is the path of each file, the
value is the content of each file.
Note: Small files are preferred, large file is also allowable, but
may cause bad performance.
binaryRecords(self, path, recordLength):
Load data from a flat binary file, assuming each record is a set of numbers
with the specified numerical format (see ByteBuffer), and the number of
bytes per record is constant.
:param path: Directory to the input data files
:param recordLength: The length at which to split the records
```
Author: Davies Liu <davies@databricks.com>
Closes #3078 from davies/binary and squashes the following commits:
cd0bdbd [Davies Liu] Merge branch 'master' of github.com:apache/spark into binary
3aa349b [Davies Liu] add experimental notes
24e84b6 [Davies Liu] Merge branch 'master' of github.com:apache/spark into binary
5ceaa8a [Davies Liu] Merge branch 'master' of github.com:apache/spark into binary
1900085 [Davies Liu] bugfix
bb22442 [Davies Liu] add binaryFiles and binaryRecords in Python
2014-11-06 03:22:19 -05:00
|
|
|
def test_binary_files(self):
|
|
|
|
path = os.path.join(self.tempdir.name, "binaryfiles")
|
|
|
|
os.mkdir(path)
|
2015-04-16 19:20:57 -04:00
|
|
|
data = b"short binary data"
|
|
|
|
with open(os.path.join(path, "part-0000"), 'wb') as f:
|
[SPARK-4186] add binaryFiles and binaryRecords in Python
add binaryFiles() and binaryRecords() in Python
```
binaryFiles(self, path, minPartitions=None):
:: Developer API ::
Read a directory of binary files from HDFS, a local file system
(available on all nodes), or any Hadoop-supported file system URI
as a byte array. Each file is read as a single record and returned
in a key-value pair, where the key is the path of each file, the
value is the content of each file.
Note: Small files are preferred, large file is also allowable, but
may cause bad performance.
binaryRecords(self, path, recordLength):
Load data from a flat binary file, assuming each record is a set of numbers
with the specified numerical format (see ByteBuffer), and the number of
bytes per record is constant.
:param path: Directory to the input data files
:param recordLength: The length at which to split the records
```
Author: Davies Liu <davies@databricks.com>
Closes #3078 from davies/binary and squashes the following commits:
cd0bdbd [Davies Liu] Merge branch 'master' of github.com:apache/spark into binary
3aa349b [Davies Liu] add experimental notes
24e84b6 [Davies Liu] Merge branch 'master' of github.com:apache/spark into binary
5ceaa8a [Davies Liu] Merge branch 'master' of github.com:apache/spark into binary
1900085 [Davies Liu] bugfix
bb22442 [Davies Liu] add binaryFiles and binaryRecords in Python
2014-11-06 03:22:19 -05:00
|
|
|
f.write(data)
|
|
|
|
[(p, d)] = self.sc.binaryFiles(path).collect()
|
|
|
|
self.assertTrue(p.endswith("part-0000"))
|
|
|
|
self.assertEqual(d, data)
|
|
|
|
|
|
|
|
def test_binary_records(self):
|
|
|
|
path = os.path.join(self.tempdir.name, "binaryrecords")
|
|
|
|
os.mkdir(path)
|
|
|
|
with open(os.path.join(path, "part-0000"), 'w') as f:
|
|
|
|
for i in range(100):
|
|
|
|
f.write('%04d' % i)
|
|
|
|
result = self.sc.binaryRecords(path, 4).map(int).collect()
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertEqual(list(range(100)), result)
|
[SPARK-4186] add binaryFiles and binaryRecords in Python
add binaryFiles() and binaryRecords() in Python
```
binaryFiles(self, path, minPartitions=None):
:: Developer API ::
Read a directory of binary files from HDFS, a local file system
(available on all nodes), or any Hadoop-supported file system URI
as a byte array. Each file is read as a single record and returned
in a key-value pair, where the key is the path of each file, the
value is the content of each file.
Note: Small files are preferred, large file is also allowable, but
may cause bad performance.
binaryRecords(self, path, recordLength):
Load data from a flat binary file, assuming each record is a set of numbers
with the specified numerical format (see ByteBuffer), and the number of
bytes per record is constant.
:param path: Directory to the input data files
:param recordLength: The length at which to split the records
```
Author: Davies Liu <davies@databricks.com>
Closes #3078 from davies/binary and squashes the following commits:
cd0bdbd [Davies Liu] Merge branch 'master' of github.com:apache/spark into binary
3aa349b [Davies Liu] add experimental notes
24e84b6 [Davies Liu] Merge branch 'master' of github.com:apache/spark into binary
5ceaa8a [Davies Liu] Merge branch 'master' of github.com:apache/spark into binary
1900085 [Davies Liu] bugfix
bb22442 [Davies Liu] add binaryFiles and binaryRecords in Python
2014-11-06 03:22:19 -05:00
|
|
|
|
2014-08-06 15:58:24 -04:00
|
|
|
|
2014-10-06 17:07:53 -04:00
|
|
|
class OutputFormatTests(ReusedPySparkTestCase):
|
2014-07-30 16:19:05 -04:00
|
|
|
|
|
|
|
def setUp(self):
|
|
|
|
self.tempdir = tempfile.NamedTemporaryFile(delete=False)
|
|
|
|
os.unlink(self.tempdir.name)
|
|
|
|
|
|
|
|
def tearDown(self):
|
|
|
|
shutil.rmtree(self.tempdir.name, ignore_errors=True)
|
|
|
|
|
2015-04-16 19:20:57 -04:00
|
|
|
@unittest.skipIf(sys.version >= "3", "serialize array of byte")
|
2014-07-30 16:19:05 -04:00
|
|
|
def test_sequencefiles(self):
|
|
|
|
basepath = self.tempdir.name
|
|
|
|
ei = [(1, u'aa'), (1, u'aa'), (2, u'aa'), (2, u'bb'), (2, u'bb'), (3, u'cc')]
|
|
|
|
self.sc.parallelize(ei).saveAsSequenceFile(basepath + "/sfint/")
|
|
|
|
ints = sorted(self.sc.sequenceFile(basepath + "/sfint/").collect())
|
|
|
|
self.assertEqual(ints, ei)
|
|
|
|
|
|
|
|
ed = [(1.0, u'aa'), (1.0, u'aa'), (2.0, u'aa'), (2.0, u'bb'), (2.0, u'bb'), (3.0, u'cc')]
|
|
|
|
self.sc.parallelize(ed).saveAsSequenceFile(basepath + "/sfdouble/")
|
|
|
|
doubles = sorted(self.sc.sequenceFile(basepath + "/sfdouble/").collect())
|
|
|
|
self.assertEqual(doubles, ed)
|
|
|
|
|
|
|
|
ebs = [(1, bytearray(b'\x00\x07spam\x08')), (2, bytearray(b'\x00\x07spam\x08'))]
|
|
|
|
self.sc.parallelize(ebs).saveAsSequenceFile(basepath + "/sfbytes/")
|
|
|
|
bytes = sorted(self.sc.sequenceFile(basepath + "/sfbytes/").collect())
|
|
|
|
self.assertEqual(bytes, ebs)
|
|
|
|
|
|
|
|
et = [(u'1', u'aa'),
|
|
|
|
(u'2', u'bb'),
|
|
|
|
(u'3', u'cc')]
|
|
|
|
self.sc.parallelize(et).saveAsSequenceFile(basepath + "/sftext/")
|
|
|
|
text = sorted(self.sc.sequenceFile(basepath + "/sftext/").collect())
|
|
|
|
self.assertEqual(text, et)
|
|
|
|
|
|
|
|
eb = [(1, False), (1, True), (2, False), (2, False), (2, True), (3, True)]
|
|
|
|
self.sc.parallelize(eb).saveAsSequenceFile(basepath + "/sfbool/")
|
|
|
|
bools = sorted(self.sc.sequenceFile(basepath + "/sfbool/").collect())
|
|
|
|
self.assertEqual(bools, eb)
|
|
|
|
|
|
|
|
en = [(1, None), (1, None), (2, None), (2, None), (2, None), (3, None)]
|
|
|
|
self.sc.parallelize(en).saveAsSequenceFile(basepath + "/sfnull/")
|
|
|
|
nulls = sorted(self.sc.sequenceFile(basepath + "/sfnull/").collect())
|
|
|
|
self.assertEqual(nulls, en)
|
|
|
|
|
|
|
|
em = [(1, {}),
|
|
|
|
(1, {3.0: u'bb'}),
|
|
|
|
(2, {1.0: u'aa'}),
|
|
|
|
(2, {1.0: u'cc'}),
|
|
|
|
(3, {2.0: u'dd'})]
|
|
|
|
self.sc.parallelize(em).saveAsSequenceFile(basepath + "/sfmap/")
|
2015-04-16 19:20:57 -04:00
|
|
|
maps = self.sc.sequenceFile(basepath + "/sfmap/").collect()
|
|
|
|
for v in maps:
|
|
|
|
self.assertTrue(v, em)
|
SPARK-1416: PySpark support for SequenceFile and Hadoop InputFormats
So I finally resurrected this PR. It seems the old one against the incubator mirror is no longer available, so I cannot reference it.
This adds initial support for reading Hadoop ```SequenceFile```s, as well as arbitrary Hadoop ```InputFormat```s, in PySpark.
# Overview
The basics are as follows:
1. ```PythonRDD``` object contains the relevant methods, that are in turn invoked by ```SparkContext``` in PySpark
2. The SequenceFile or InputFormat is read on the Scala side and converted from ```Writable``` instances to the relevant Scala classes (in the case of primitives)
3. Pyrolite is used to serialize Java objects. If this fails, the fallback is ```toString```
4. ```PickleSerializer``` on the Python side deserializes.
This works "out the box" for simple ```Writable```s:
* ```Text```
* ```IntWritable```, ```DoubleWritable```, ```FloatWritable```
* ```NullWritable```
* ```BooleanWritable```
* ```BytesWritable```
* ```MapWritable```
It also works for simple, "struct-like" classes. Due to the way Pyrolite works, this requires that the classes satisfy the JavaBeans convenstions (i.e. with fields and a no-arg constructor and getters/setters). (Perhaps in future some sugar for case classes and reflection could be added).
I've tested it out with ```ESInputFormat``` as an example and it works very nicely:
```python
conf = {"es.resource" : "index/type" }
rdd = sc.newAPIHadoopRDD("org.elasticsearch.hadoop.mr.EsInputFormat", "org.apache.hadoop.io.NullWritable", "org.elasticsearch.hadoop.mr.LinkedMapWritable", conf=conf)
rdd.first()
```
I suspect for things like HBase/Cassandra it will be a bit trickier to get it to work out the box.
# Some things still outstanding:
1. ~~Requires ```msgpack-python``` and will fail without it. As originally discussed with Josh, add a ```as_strings``` argument that defaults to ```False```, that can be used if ```msgpack-python``` is not available~~
2. ~~I see from https://github.com/apache/spark/pull/363 that Pyrolite is being used there for SerDe between Scala and Python. @ahirreddy @mateiz what is the plan behind this - is Pyrolite preferred? It seems from a cursory glance that adapting the ```msgpack```-based SerDe here to use Pyrolite wouldn't be too hard~~
3. ~~Support the key and value "wrapper" that would allow a Scala/Java function to be plugged in that would transform whatever the key/value Writable class is into something that can be serialized (e.g. convert some custom Writable to a JavaBean or ```java.util.Map``` that can be easily serialized)~~
4. Support ```saveAsSequenceFile``` and ```saveAsHadoopFile``` etc. This would require SerDe in the reverse direction, that can be handled by Pyrolite. Will work on this as a separate PR
Author: Nick Pentreath <nick.pentreath@gmail.com>
Closes #455 from MLnick/pyspark-inputformats and squashes the following commits:
268df7e [Nick Pentreath] Documentation changes mer @pwendell comments
761269b [Nick Pentreath] Address @pwendell comments, simplify default writable conversions and remove registry.
4c972d8 [Nick Pentreath] Add license headers
d150431 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
cde6af9 [Nick Pentreath] Parameterize converter trait
5ebacfa [Nick Pentreath] Update docs for PySpark input formats
a985492 [Nick Pentreath] Move Converter examples to own package
365d0be [Nick Pentreath] Make classes private[python]. Add docs and @Experimental annotation to Converter interface.
eeb8205 [Nick Pentreath] Fix path relative to SPARK_HOME in tests
1eaa08b [Nick Pentreath] HBase -> Cassandra app name oversight
3f90c3e [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
2c18513 [Nick Pentreath] Add examples for reading HBase and Cassandra InputFormats from Python
b65606f [Nick Pentreath] Add converter interface
5757f6e [Nick Pentreath] Default key/value classes for sequenceFile asre None
085b55f [Nick Pentreath] Move input format tests to tests.py and clean up docs
43eb728 [Nick Pentreath] PySpark InputFormats docs into programming guide
94beedc [Nick Pentreath] Clean up args in PythonRDD. Set key/value converter defaults to None for PySpark context.py methods
1a4a1d6 [Nick Pentreath] Address @mateiz style comments
01e0813 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
15a7d07 [Nick Pentreath] Remove default args for key/value classes. Arg names to camelCase
9fe6bd5 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
84fe8e3 [Nick Pentreath] Python programming guide space formatting
d0f52b6 [Nick Pentreath] Python programming guide
7caa73a [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
93ef995 [Nick Pentreath] Add back context.py changes
9ef1896 [Nick Pentreath] Recover earlier changes lost in previous merge for serializers.py
077ecb2 [Nick Pentreath] Recover earlier changes lost in previous merge for context.py
5af4770 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
35b8e3a [Nick Pentreath] Another fix for test ordering
bef3afb [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
e001b94 [Nick Pentreath] Fix test failures due to ordering
78978d9 [Nick Pentreath] Add doc for SequenceFile and InputFormat support to Python programming guide
64eb051 [Nick Pentreath] Scalastyle fix
e7552fa [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
44f2857 [Nick Pentreath] Remove msgpack dependency and switch serialization to Pyrolite, plus some clean up and refactoring
c0ebfb6 [Nick Pentreath] Change sequencefile test data generator to easily be called from PySpark tests
1d7c17c [Nick Pentreath] Amend tests to auto-generate sequencefile data in temp dir
17a656b [Nick Pentreath] remove binary sequencefile for tests
f60959e [Nick Pentreath] Remove msgpack dependency and serializer from PySpark
450e0a2 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
31a2fff [Nick Pentreath] Scalastyle fixes
fc5099e [Nick Pentreath] Add Apache license headers
4e08983 [Nick Pentreath] Clean up docs for PySpark context methods
b20ec7e [Nick Pentreath] Clean up merge duplicate dependencies
951c117 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
f6aac55 [Nick Pentreath] Bring back msgpack
9d2256e [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
1bbbfb0 [Nick Pentreath] Clean up SparkBuild from merge
a67dfad [Nick Pentreath] Clean up Msgpack serialization and registering
7237263 [Nick Pentreath] Add back msgpack serializer and hadoop file code lost during merging
25da1ca [Nick Pentreath] Add generator for nulls, bools, bytes and maps
65360d5 [Nick Pentreath] Adding test SequenceFiles
0c612e5 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
d72bf18 [Nick Pentreath] msgpack
dd57922 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
e67212a [Nick Pentreath] Add back msgpack dependency
f2d76a0 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
41856a5 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
97ef708 [Nick Pentreath] Remove old writeToStream
2beeedb [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
795a763 [Nick Pentreath] Change name to WriteInputFormatTestDataGenerator. Cleanup some var names. Use SPARK_HOME in path for writing test sequencefile data.
174f520 [Nick Pentreath] Add back graphx settings
703ee65 [Nick Pentreath] Add back msgpack
619c0fa [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
1c8efbc [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
eb40036 [Nick Pentreath] Remove unused comment lines
4d7ef2e [Nick Pentreath] Fix indentation
f1d73e3 [Nick Pentreath] mergeConfs returns a copy rather than mutating one of the input arguments
0f5cd84 [Nick Pentreath] Remove unused pair UTF8 class. Add comments to msgpack deserializer
4294cbb [Nick Pentreath] Add old Hadoop api methods. Clean up and expand comments. Clean up argument names
818a1e6 [Nick Pentreath] Add seqencefile and Hadoop InputFormat support to PythonRDD
4e7c9e3 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
c304cc8 [Nick Pentreath] Adding supporting sequncefiles for tests. Cleaning up
4b0a43f [Nick Pentreath] Refactoring utils into own objects. Cleaning up old commented-out code
d86325f [Nick Pentreath] Initial WIP of PySpark support for SequenceFile and arbitrary Hadoop InputFormat
2014-06-10 01:21:03 -04:00
|
|
|
|
2014-07-30 16:19:05 -04:00
|
|
|
def test_oldhadoop(self):
|
|
|
|
basepath = self.tempdir.name
|
|
|
|
dict_data = [(1, {}),
|
2014-08-06 15:58:24 -04:00
|
|
|
(1, {"row1": 1.0}),
|
|
|
|
(2, {"row2": 2.0})]
|
2014-07-30 16:19:05 -04:00
|
|
|
self.sc.parallelize(dict_data).saveAsHadoopFile(
|
|
|
|
basepath + "/oldhadoop/",
|
|
|
|
"org.apache.hadoop.mapred.SequenceFileOutputFormat",
|
|
|
|
"org.apache.hadoop.io.IntWritable",
|
|
|
|
"org.apache.hadoop.io.MapWritable")
|
2015-04-16 19:20:57 -04:00
|
|
|
result = self.sc.hadoopFile(
|
2014-07-30 16:19:05 -04:00
|
|
|
basepath + "/oldhadoop/",
|
|
|
|
"org.apache.hadoop.mapred.SequenceFileInputFormat",
|
|
|
|
"org.apache.hadoop.io.IntWritable",
|
2015-04-16 19:20:57 -04:00
|
|
|
"org.apache.hadoop.io.MapWritable").collect()
|
|
|
|
for v in result:
|
|
|
|
self.assertTrue(v, dict_data)
|
2014-07-30 16:19:05 -04:00
|
|
|
|
|
|
|
conf = {
|
2014-08-06 15:58:24 -04:00
|
|
|
"mapred.output.format.class": "org.apache.hadoop.mapred.SequenceFileOutputFormat",
|
2017-02-28 05:13:42 -05:00
|
|
|
"mapreduce.job.output.key.class": "org.apache.hadoop.io.IntWritable",
|
|
|
|
"mapreduce.job.output.value.class": "org.apache.hadoop.io.MapWritable",
|
|
|
|
"mapreduce.output.fileoutputformat.outputdir": basepath + "/olddataset/"
|
2014-08-06 15:58:24 -04:00
|
|
|
}
|
2014-07-30 16:19:05 -04:00
|
|
|
self.sc.parallelize(dict_data).saveAsHadoopDataset(conf)
|
2017-02-28 05:13:42 -05:00
|
|
|
input_conf = {"mapreduce.input.fileinputformat.inputdir": basepath + "/olddataset/"}
|
2015-04-16 19:20:57 -04:00
|
|
|
result = self.sc.hadoopRDD(
|
2014-07-30 16:19:05 -04:00
|
|
|
"org.apache.hadoop.mapred.SequenceFileInputFormat",
|
|
|
|
"org.apache.hadoop.io.IntWritable",
|
|
|
|
"org.apache.hadoop.io.MapWritable",
|
2015-04-16 19:20:57 -04:00
|
|
|
conf=input_conf).collect()
|
|
|
|
for v in result:
|
|
|
|
self.assertTrue(v, dict_data)
|
2014-07-30 16:19:05 -04:00
|
|
|
|
|
|
|
def test_newhadoop(self):
|
2014-09-12 21:42:50 -04:00
|
|
|
basepath = self.tempdir.name
|
|
|
|
data = [(1, ""),
|
|
|
|
(1, "a"),
|
|
|
|
(2, "bcdf")]
|
|
|
|
self.sc.parallelize(data).saveAsNewAPIHadoopFile(
|
|
|
|
basepath + "/newhadoop/",
|
|
|
|
"org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat",
|
|
|
|
"org.apache.hadoop.io.IntWritable",
|
|
|
|
"org.apache.hadoop.io.Text")
|
|
|
|
result = sorted(self.sc.newAPIHadoopFile(
|
|
|
|
basepath + "/newhadoop/",
|
|
|
|
"org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat",
|
|
|
|
"org.apache.hadoop.io.IntWritable",
|
|
|
|
"org.apache.hadoop.io.Text").collect())
|
|
|
|
self.assertEqual(result, data)
|
|
|
|
|
|
|
|
conf = {
|
2017-02-28 05:13:42 -05:00
|
|
|
"mapreduce.job.outputformat.class":
|
2014-09-12 21:42:50 -04:00
|
|
|
"org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat",
|
2017-02-28 05:13:42 -05:00
|
|
|
"mapreduce.job.output.key.class": "org.apache.hadoop.io.IntWritable",
|
|
|
|
"mapreduce.job.output.value.class": "org.apache.hadoop.io.Text",
|
|
|
|
"mapreduce.output.fileoutputformat.outputdir": basepath + "/newdataset/"
|
2014-09-12 21:42:50 -04:00
|
|
|
}
|
|
|
|
self.sc.parallelize(data).saveAsNewAPIHadoopDataset(conf)
|
2017-02-28 05:13:42 -05:00
|
|
|
input_conf = {"mapreduce.input.fileinputformat.inputdir": basepath + "/newdataset/"}
|
2014-09-12 21:42:50 -04:00
|
|
|
new_dataset = sorted(self.sc.newAPIHadoopRDD(
|
|
|
|
"org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat",
|
|
|
|
"org.apache.hadoop.io.IntWritable",
|
|
|
|
"org.apache.hadoop.io.Text",
|
|
|
|
conf=input_conf).collect())
|
|
|
|
self.assertEqual(new_dataset, data)
|
|
|
|
|
2015-04-16 19:20:57 -04:00
|
|
|
@unittest.skipIf(sys.version >= "3", "serialize of array")
|
2014-09-12 21:42:50 -04:00
|
|
|
def test_newhadoop_with_array(self):
|
2014-07-30 16:19:05 -04:00
|
|
|
basepath = self.tempdir.name
|
|
|
|
# use custom ArrayWritable types and converters to handle arrays
|
|
|
|
array_data = [(1, array('d')),
|
|
|
|
(1, array('d', [1.0, 2.0, 3.0])),
|
|
|
|
(2, array('d', [3.0, 4.0, 5.0]))]
|
|
|
|
self.sc.parallelize(array_data).saveAsNewAPIHadoopFile(
|
|
|
|
basepath + "/newhadoop/",
|
|
|
|
"org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat",
|
|
|
|
"org.apache.hadoop.io.IntWritable",
|
|
|
|
"org.apache.spark.api.python.DoubleArrayWritable",
|
|
|
|
valueConverter="org.apache.spark.api.python.DoubleArrayToWritableConverter")
|
|
|
|
result = sorted(self.sc.newAPIHadoopFile(
|
|
|
|
basepath + "/newhadoop/",
|
|
|
|
"org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat",
|
|
|
|
"org.apache.hadoop.io.IntWritable",
|
|
|
|
"org.apache.spark.api.python.DoubleArrayWritable",
|
|
|
|
valueConverter="org.apache.spark.api.python.WritableToDoubleArrayConverter").collect())
|
|
|
|
self.assertEqual(result, array_data)
|
|
|
|
|
2014-08-06 15:58:24 -04:00
|
|
|
conf = {
|
2017-02-28 05:13:42 -05:00
|
|
|
"mapreduce.job.outputformat.class":
|
2014-08-06 15:58:24 -04:00
|
|
|
"org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat",
|
2017-02-28 05:13:42 -05:00
|
|
|
"mapreduce.job.output.key.class": "org.apache.hadoop.io.IntWritable",
|
|
|
|
"mapreduce.job.output.value.class": "org.apache.spark.api.python.DoubleArrayWritable",
|
|
|
|
"mapreduce.output.fileoutputformat.outputdir": basepath + "/newdataset/"
|
2014-08-06 15:58:24 -04:00
|
|
|
}
|
|
|
|
self.sc.parallelize(array_data).saveAsNewAPIHadoopDataset(
|
|
|
|
conf,
|
2014-07-30 16:19:05 -04:00
|
|
|
valueConverter="org.apache.spark.api.python.DoubleArrayToWritableConverter")
|
2017-02-28 05:13:42 -05:00
|
|
|
input_conf = {"mapreduce.input.fileinputformat.inputdir": basepath + "/newdataset/"}
|
2014-07-30 16:19:05 -04:00
|
|
|
new_dataset = sorted(self.sc.newAPIHadoopRDD(
|
|
|
|
"org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat",
|
|
|
|
"org.apache.hadoop.io.IntWritable",
|
|
|
|
"org.apache.spark.api.python.DoubleArrayWritable",
|
|
|
|
valueConverter="org.apache.spark.api.python.WritableToDoubleArrayConverter",
|
|
|
|
conf=input_conf).collect())
|
|
|
|
self.assertEqual(new_dataset, array_data)
|
|
|
|
|
|
|
|
def test_newolderror(self):
|
|
|
|
basepath = self.tempdir.name
|
2014-08-06 15:58:24 -04:00
|
|
|
rdd = self.sc.parallelize(range(1, 4)).map(lambda x: (x, "a" * x))
|
2014-07-30 16:19:05 -04:00
|
|
|
self.assertRaises(Exception, lambda: rdd.saveAsHadoopFile(
|
|
|
|
basepath + "/newolderror/saveAsHadoopFile/",
|
|
|
|
"org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat"))
|
|
|
|
self.assertRaises(Exception, lambda: rdd.saveAsNewAPIHadoopFile(
|
|
|
|
basepath + "/newolderror/saveAsNewAPIHadoopFile/",
|
|
|
|
"org.apache.hadoop.mapred.SequenceFileOutputFormat"))
|
|
|
|
|
|
|
|
def test_bad_inputs(self):
|
|
|
|
basepath = self.tempdir.name
|
2014-08-06 15:58:24 -04:00
|
|
|
rdd = self.sc.parallelize(range(1, 4)).map(lambda x: (x, "a" * x))
|
2014-07-30 16:19:05 -04:00
|
|
|
self.assertRaises(Exception, lambda: rdd.saveAsHadoopFile(
|
|
|
|
basepath + "/badinputs/saveAsHadoopFile/",
|
|
|
|
"org.apache.hadoop.mapred.NotValidOutputFormat"))
|
|
|
|
self.assertRaises(Exception, lambda: rdd.saveAsNewAPIHadoopFile(
|
|
|
|
basepath + "/badinputs/saveAsNewAPIHadoopFile/",
|
|
|
|
"org.apache.hadoop.mapreduce.lib.output.NotValidOutputFormat"))
|
|
|
|
|
|
|
|
def test_converters(self):
|
|
|
|
# use of custom converters
|
|
|
|
basepath = self.tempdir.name
|
|
|
|
data = [(1, {3.0: u'bb'}),
|
|
|
|
(2, {1.0: u'aa'}),
|
|
|
|
(3, {2.0: u'dd'})]
|
|
|
|
self.sc.parallelize(data).saveAsNewAPIHadoopFile(
|
|
|
|
basepath + "/converters/",
|
|
|
|
"org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat",
|
|
|
|
keyConverter="org.apache.spark.api.python.TestOutputKeyConverter",
|
|
|
|
valueConverter="org.apache.spark.api.python.TestOutputValueConverter")
|
|
|
|
converted = sorted(self.sc.sequenceFile(basepath + "/converters/").collect())
|
|
|
|
expected = [(u'1', 3.0),
|
|
|
|
(u'2', 1.0),
|
|
|
|
(u'3', 2.0)]
|
|
|
|
self.assertEqual(converted, expected)
|
|
|
|
|
|
|
|
def test_reserialization(self):
|
|
|
|
basepath = self.tempdir.name
|
|
|
|
x = range(1, 5)
|
|
|
|
y = range(1001, 1005)
|
2015-04-16 19:20:57 -04:00
|
|
|
data = list(zip(x, y))
|
2014-07-30 16:19:05 -04:00
|
|
|
rdd = self.sc.parallelize(x).zip(self.sc.parallelize(y))
|
|
|
|
rdd.saveAsSequenceFile(basepath + "/reserialize/sequence")
|
|
|
|
result1 = sorted(self.sc.sequenceFile(basepath + "/reserialize/sequence").collect())
|
|
|
|
self.assertEqual(result1, data)
|
|
|
|
|
2014-08-06 15:58:24 -04:00
|
|
|
rdd.saveAsHadoopFile(
|
|
|
|
basepath + "/reserialize/hadoop",
|
|
|
|
"org.apache.hadoop.mapred.SequenceFileOutputFormat")
|
2014-07-30 16:19:05 -04:00
|
|
|
result2 = sorted(self.sc.sequenceFile(basepath + "/reserialize/hadoop").collect())
|
|
|
|
self.assertEqual(result2, data)
|
|
|
|
|
2014-08-06 15:58:24 -04:00
|
|
|
rdd.saveAsNewAPIHadoopFile(
|
|
|
|
basepath + "/reserialize/newhadoop",
|
|
|
|
"org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat")
|
2014-07-30 16:19:05 -04:00
|
|
|
result3 = sorted(self.sc.sequenceFile(basepath + "/reserialize/newhadoop").collect())
|
|
|
|
self.assertEqual(result3, data)
|
|
|
|
|
|
|
|
conf4 = {
|
2014-08-06 15:58:24 -04:00
|
|
|
"mapred.output.format.class": "org.apache.hadoop.mapred.SequenceFileOutputFormat",
|
2017-02-28 05:13:42 -05:00
|
|
|
"mapreduce.job.output.key.class": "org.apache.hadoop.io.IntWritable",
|
|
|
|
"mapreduce.job.output.value.class": "org.apache.hadoop.io.IntWritable",
|
|
|
|
"mapreduce.output.fileoutputformat.outputdir": basepath + "/reserialize/dataset"}
|
2014-07-30 16:19:05 -04:00
|
|
|
rdd.saveAsHadoopDataset(conf4)
|
|
|
|
result4 = sorted(self.sc.sequenceFile(basepath + "/reserialize/dataset").collect())
|
|
|
|
self.assertEqual(result4, data)
|
|
|
|
|
2017-02-28 05:13:42 -05:00
|
|
|
conf5 = {"mapreduce.job.outputformat.class":
|
2014-08-06 15:58:24 -04:00
|
|
|
"org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat",
|
2017-02-28 05:13:42 -05:00
|
|
|
"mapreduce.job.output.key.class": "org.apache.hadoop.io.IntWritable",
|
|
|
|
"mapreduce.job.output.value.class": "org.apache.hadoop.io.IntWritable",
|
|
|
|
"mapreduce.output.fileoutputformat.outputdir": basepath + "/reserialize/newdataset"
|
|
|
|
}
|
2014-07-30 16:19:05 -04:00
|
|
|
rdd.saveAsNewAPIHadoopDataset(conf5)
|
|
|
|
result5 = sorted(self.sc.sequenceFile(basepath + "/reserialize/newdataset").collect())
|
|
|
|
self.assertEqual(result5, data)
|
|
|
|
|
|
|
|
def test_malformed_RDD(self):
|
|
|
|
basepath = self.tempdir.name
|
|
|
|
# non-batch-serialized RDD[[(K, V)]] should be rejected
|
|
|
|
data = [[(1, "a")], [(2, "aa")], [(3, "aaa")]]
|
2014-09-20 18:09:35 -04:00
|
|
|
rdd = self.sc.parallelize(data, len(data))
|
2014-07-30 16:19:05 -04:00
|
|
|
self.assertRaises(Exception, lambda: rdd.saveAsSequenceFile(
|
|
|
|
basepath + "/malformed/sequence"))
|
SPARK-1416: PySpark support for SequenceFile and Hadoop InputFormats
So I finally resurrected this PR. It seems the old one against the incubator mirror is no longer available, so I cannot reference it.
This adds initial support for reading Hadoop ```SequenceFile```s, as well as arbitrary Hadoop ```InputFormat```s, in PySpark.
# Overview
The basics are as follows:
1. ```PythonRDD``` object contains the relevant methods, that are in turn invoked by ```SparkContext``` in PySpark
2. The SequenceFile or InputFormat is read on the Scala side and converted from ```Writable``` instances to the relevant Scala classes (in the case of primitives)
3. Pyrolite is used to serialize Java objects. If this fails, the fallback is ```toString```
4. ```PickleSerializer``` on the Python side deserializes.
This works "out the box" for simple ```Writable```s:
* ```Text```
* ```IntWritable```, ```DoubleWritable```, ```FloatWritable```
* ```NullWritable```
* ```BooleanWritable```
* ```BytesWritable```
* ```MapWritable```
It also works for simple, "struct-like" classes. Due to the way Pyrolite works, this requires that the classes satisfy the JavaBeans convenstions (i.e. with fields and a no-arg constructor and getters/setters). (Perhaps in future some sugar for case classes and reflection could be added).
I've tested it out with ```ESInputFormat``` as an example and it works very nicely:
```python
conf = {"es.resource" : "index/type" }
rdd = sc.newAPIHadoopRDD("org.elasticsearch.hadoop.mr.EsInputFormat", "org.apache.hadoop.io.NullWritable", "org.elasticsearch.hadoop.mr.LinkedMapWritable", conf=conf)
rdd.first()
```
I suspect for things like HBase/Cassandra it will be a bit trickier to get it to work out the box.
# Some things still outstanding:
1. ~~Requires ```msgpack-python``` and will fail without it. As originally discussed with Josh, add a ```as_strings``` argument that defaults to ```False```, that can be used if ```msgpack-python``` is not available~~
2. ~~I see from https://github.com/apache/spark/pull/363 that Pyrolite is being used there for SerDe between Scala and Python. @ahirreddy @mateiz what is the plan behind this - is Pyrolite preferred? It seems from a cursory glance that adapting the ```msgpack```-based SerDe here to use Pyrolite wouldn't be too hard~~
3. ~~Support the key and value "wrapper" that would allow a Scala/Java function to be plugged in that would transform whatever the key/value Writable class is into something that can be serialized (e.g. convert some custom Writable to a JavaBean or ```java.util.Map``` that can be easily serialized)~~
4. Support ```saveAsSequenceFile``` and ```saveAsHadoopFile``` etc. This would require SerDe in the reverse direction, that can be handled by Pyrolite. Will work on this as a separate PR
Author: Nick Pentreath <nick.pentreath@gmail.com>
Closes #455 from MLnick/pyspark-inputformats and squashes the following commits:
268df7e [Nick Pentreath] Documentation changes mer @pwendell comments
761269b [Nick Pentreath] Address @pwendell comments, simplify default writable conversions and remove registry.
4c972d8 [Nick Pentreath] Add license headers
d150431 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
cde6af9 [Nick Pentreath] Parameterize converter trait
5ebacfa [Nick Pentreath] Update docs for PySpark input formats
a985492 [Nick Pentreath] Move Converter examples to own package
365d0be [Nick Pentreath] Make classes private[python]. Add docs and @Experimental annotation to Converter interface.
eeb8205 [Nick Pentreath] Fix path relative to SPARK_HOME in tests
1eaa08b [Nick Pentreath] HBase -> Cassandra app name oversight
3f90c3e [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
2c18513 [Nick Pentreath] Add examples for reading HBase and Cassandra InputFormats from Python
b65606f [Nick Pentreath] Add converter interface
5757f6e [Nick Pentreath] Default key/value classes for sequenceFile asre None
085b55f [Nick Pentreath] Move input format tests to tests.py and clean up docs
43eb728 [Nick Pentreath] PySpark InputFormats docs into programming guide
94beedc [Nick Pentreath] Clean up args in PythonRDD. Set key/value converter defaults to None for PySpark context.py methods
1a4a1d6 [Nick Pentreath] Address @mateiz style comments
01e0813 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
15a7d07 [Nick Pentreath] Remove default args for key/value classes. Arg names to camelCase
9fe6bd5 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
84fe8e3 [Nick Pentreath] Python programming guide space formatting
d0f52b6 [Nick Pentreath] Python programming guide
7caa73a [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
93ef995 [Nick Pentreath] Add back context.py changes
9ef1896 [Nick Pentreath] Recover earlier changes lost in previous merge for serializers.py
077ecb2 [Nick Pentreath] Recover earlier changes lost in previous merge for context.py
5af4770 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
35b8e3a [Nick Pentreath] Another fix for test ordering
bef3afb [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
e001b94 [Nick Pentreath] Fix test failures due to ordering
78978d9 [Nick Pentreath] Add doc for SequenceFile and InputFormat support to Python programming guide
64eb051 [Nick Pentreath] Scalastyle fix
e7552fa [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
44f2857 [Nick Pentreath] Remove msgpack dependency and switch serialization to Pyrolite, plus some clean up and refactoring
c0ebfb6 [Nick Pentreath] Change sequencefile test data generator to easily be called from PySpark tests
1d7c17c [Nick Pentreath] Amend tests to auto-generate sequencefile data in temp dir
17a656b [Nick Pentreath] remove binary sequencefile for tests
f60959e [Nick Pentreath] Remove msgpack dependency and serializer from PySpark
450e0a2 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
31a2fff [Nick Pentreath] Scalastyle fixes
fc5099e [Nick Pentreath] Add Apache license headers
4e08983 [Nick Pentreath] Clean up docs for PySpark context methods
b20ec7e [Nick Pentreath] Clean up merge duplicate dependencies
951c117 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
f6aac55 [Nick Pentreath] Bring back msgpack
9d2256e [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
1bbbfb0 [Nick Pentreath] Clean up SparkBuild from merge
a67dfad [Nick Pentreath] Clean up Msgpack serialization and registering
7237263 [Nick Pentreath] Add back msgpack serializer and hadoop file code lost during merging
25da1ca [Nick Pentreath] Add generator for nulls, bools, bytes and maps
65360d5 [Nick Pentreath] Adding test SequenceFiles
0c612e5 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
d72bf18 [Nick Pentreath] msgpack
dd57922 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
e67212a [Nick Pentreath] Add back msgpack dependency
f2d76a0 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
41856a5 [Nick Pentreath] Merge branch 'master' into pyspark-inputformats
97ef708 [Nick Pentreath] Remove old writeToStream
2beeedb [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
795a763 [Nick Pentreath] Change name to WriteInputFormatTestDataGenerator. Cleanup some var names. Use SPARK_HOME in path for writing test sequencefile data.
174f520 [Nick Pentreath] Add back graphx settings
703ee65 [Nick Pentreath] Add back msgpack
619c0fa [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
1c8efbc [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
eb40036 [Nick Pentreath] Remove unused comment lines
4d7ef2e [Nick Pentreath] Fix indentation
f1d73e3 [Nick Pentreath] mergeConfs returns a copy rather than mutating one of the input arguments
0f5cd84 [Nick Pentreath] Remove unused pair UTF8 class. Add comments to msgpack deserializer
4294cbb [Nick Pentreath] Add old Hadoop api methods. Clean up and expand comments. Clean up argument names
818a1e6 [Nick Pentreath] Add seqencefile and Hadoop InputFormat support to PythonRDD
4e7c9e3 [Nick Pentreath] Merge remote-tracking branch 'upstream/master' into pyspark-inputformats
c304cc8 [Nick Pentreath] Adding supporting sequncefiles for tests. Cleaning up
4b0a43f [Nick Pentreath] Refactoring utils into own objects. Cleaning up old commented-out code
d86325f [Nick Pentreath] Initial WIP of PySpark support for SequenceFile and arbitrary Hadoop InputFormat
2014-06-10 01:21:03 -04:00
|
|
|
|
2014-08-06 15:58:24 -04:00
|
|
|
|
2014-10-06 17:07:53 -04:00
|
|
|
class DaemonTests(unittest.TestCase):
|
2013-05-10 18:48:48 -04:00
|
|
|
def connect(self, port):
|
|
|
|
from socket import socket, AF_INET, SOCK_STREAM
|
|
|
|
sock = socket(AF_INET, SOCK_STREAM)
|
|
|
|
sock.connect(('127.0.0.1', port))
|
|
|
|
# send a split index of -1 to shutdown the worker
|
2015-04-16 19:20:57 -04:00
|
|
|
sock.send(b"\xFF\xFF\xFF\xFF")
|
2013-05-10 18:48:48 -04:00
|
|
|
sock.close()
|
|
|
|
return True
|
|
|
|
|
|
|
|
def do_termination_test(self, terminator):
|
|
|
|
from subprocess import Popen, PIPE
|
|
|
|
from errno import ECONNREFUSED
|
|
|
|
|
|
|
|
# start daemon
|
|
|
|
daemon_path = os.path.join(os.path.dirname(__file__), "daemon.py")
|
[SPARK-8583] [SPARK-5482] [BUILD] Refactor python/run-tests to integrate with dev/run-tests module system
This patch refactors the `python/run-tests` script:
- It's now written in Python instead of Bash.
- The descriptions of the tests to run are now stored in `dev/run-tests`'s modules. This allows the pull request builder to skip Python tests suites that were not affected by the pull request's changes. For example, we can now skip the PySpark Streaming test cases when only SQL files are changed.
- `python/run-tests` now supports command-line flags to make it easier to run individual test suites (this addresses SPARK-5482):
```
Usage: run-tests [options]
Options:
-h, --help show this help message and exit
--python-executables=PYTHON_EXECUTABLES
A comma-separated list of Python executables to test
against (default: python2.6,python3.4,pypy)
--modules=MODULES A comma-separated list of Python modules to test
(default: pyspark-core,pyspark-ml,pyspark-mllib
,pyspark-sql,pyspark-streaming)
```
- `dev/run-tests` has been split into multiple files: the module definitions and test utility functions are now stored inside of a `dev/sparktestsupport` Python module, allowing them to be re-used from the Python test runner script.
Author: Josh Rosen <joshrosen@databricks.com>
Closes #6967 from JoshRosen/run-tests-python-modules and squashes the following commits:
f578d6d [Josh Rosen] Fix print for Python 2.x
8233d61 [Josh Rosen] Add python/run-tests.py to Python lint checks
34c98d2 [Josh Rosen] Fix universal_newlines for Python 3
8f65ed0 [Josh Rosen] Fix handling of module in python/run-tests
37aff00 [Josh Rosen] Python 3 fix
27a389f [Josh Rosen] Skip MLLib tests for PyPy
c364ccf [Josh Rosen] Use which() to convert PYSPARK_PYTHON to an absolute path before shelling out to run tests
568a3fd [Josh Rosen] Fix hashbang
3b852ae [Josh Rosen] Fall back to PYSPARK_PYTHON when sys.executable is None (fixes a test)
f53db55 [Josh Rosen] Remove python2 flag, since the test runner script also works fine under Python 3
9c80469 [Josh Rosen] Fix passing of PYSPARK_PYTHON
d33e525 [Josh Rosen] Merge remote-tracking branch 'origin/master' into run-tests-python-modules
4f8902c [Josh Rosen] Python lint fixes.
8f3244c [Josh Rosen] Use universal_newlines to fix dev/run-tests doctest failures on Python 3.
f542ac5 [Josh Rosen] Fix lint check for Python 3
fff4d09 [Josh Rosen] Add dev/sparktestsupport to pep8 checks
2efd594 [Josh Rosen] Update dev/run-tests to use new Python test runner flags
b2ab027 [Josh Rosen] Add command-line options for running individual suites in python/run-tests
caeb040 [Josh Rosen] Fixes to PySpark test module definitions
d6a77d3 [Josh Rosen] Fix the tests of dev/run-tests
def2d8a [Josh Rosen] Two minor fixes
aec0b8f [Josh Rosen] Actually get the Kafka stuff to run properly
04015b9 [Josh Rosen] First attempt at getting PySpark Kafka test to work in new runner script
4c97136 [Josh Rosen] PYTHONPATH fixes
dcc9c09 [Josh Rosen] Fix time division
32660fc [Josh Rosen] Initial cut at Python test runner refactoring
311c6a9 [Josh Rosen] Move shell utility functions to own module.
1bdeb87 [Josh Rosen] Move module definitions to separate file.
2015-06-27 23:24:34 -04:00
|
|
|
python_exec = sys.executable or os.environ.get("PYSPARK_PYTHON")
|
|
|
|
daemon = Popen([python_exec, daemon_path], stdin=PIPE, stdout=PIPE)
|
2013-05-10 18:48:48 -04:00
|
|
|
|
|
|
|
# read the port number
|
|
|
|
port = read_int(daemon.stdout)
|
|
|
|
|
|
|
|
# daemon should accept connections
|
|
|
|
self.assertTrue(self.connect(port))
|
|
|
|
|
|
|
|
# request shutdown
|
|
|
|
terminator(daemon)
|
|
|
|
time.sleep(1)
|
|
|
|
|
|
|
|
# daemon should no longer accept connections
|
2013-08-14 18:12:12 -04:00
|
|
|
try:
|
2013-05-10 18:48:48 -04:00
|
|
|
self.connect(port)
|
2013-08-14 18:12:12 -04:00
|
|
|
except EnvironmentError as exception:
|
|
|
|
self.assertEqual(exception.errno, ECONNREFUSED)
|
|
|
|
else:
|
|
|
|
self.fail("Expected EnvironmentError to be raised")
|
2013-05-10 18:48:48 -04:00
|
|
|
|
|
|
|
def test_termination_stdin(self):
|
|
|
|
"""Ensure that daemon and workers terminate when stdin is closed."""
|
|
|
|
self.do_termination_test(lambda daemon: daemon.stdin.close())
|
|
|
|
|
|
|
|
def test_termination_sigterm(self):
|
|
|
|
"""Ensure that daemon and workers terminate on SIGTERM."""
|
|
|
|
from signal import SIGTERM
|
|
|
|
self.do_termination_test(lambda daemon: os.kill(daemon.pid, SIGTERM))
|
|
|
|
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
|
2015-04-21 20:49:55 -04:00
|
|
|
class WorkerTests(ReusedPySparkTestCase):
|
2014-08-03 18:52:00 -04:00
|
|
|
def test_cancel_task(self):
|
|
|
|
temp = tempfile.NamedTemporaryFile(delete=True)
|
|
|
|
temp.close()
|
|
|
|
path = temp.name
|
2014-08-06 15:58:24 -04:00
|
|
|
|
2014-08-03 18:52:00 -04:00
|
|
|
def sleep(x):
|
2014-08-06 15:58:24 -04:00
|
|
|
import os
|
|
|
|
import time
|
2014-08-03 18:52:00 -04:00
|
|
|
with open(path, 'w') as f:
|
|
|
|
f.write("%d %d" % (os.getppid(), os.getpid()))
|
|
|
|
time.sleep(100)
|
|
|
|
|
|
|
|
# start job in background thread
|
|
|
|
def run():
|
2015-04-21 20:49:55 -04:00
|
|
|
try:
|
|
|
|
self.sc.parallelize(range(1), 1).foreach(sleep)
|
|
|
|
except Exception:
|
|
|
|
pass
|
2014-08-03 18:52:00 -04:00
|
|
|
import threading
|
|
|
|
t = threading.Thread(target=run)
|
|
|
|
t.daemon = True
|
|
|
|
t.start()
|
|
|
|
|
|
|
|
daemon_pid, worker_pid = 0, 0
|
|
|
|
while True:
|
|
|
|
if os.path.exists(path):
|
2015-04-16 19:20:57 -04:00
|
|
|
with open(path) as f:
|
|
|
|
data = f.read().split(' ')
|
2014-08-03 18:52:00 -04:00
|
|
|
daemon_pid, worker_pid = map(int, data)
|
|
|
|
break
|
|
|
|
time.sleep(0.1)
|
|
|
|
|
|
|
|
# cancel jobs
|
|
|
|
self.sc.cancelAllJobs()
|
|
|
|
t.join()
|
|
|
|
|
|
|
|
for i in range(50):
|
|
|
|
try:
|
|
|
|
os.kill(worker_pid, 0)
|
|
|
|
time.sleep(0.1)
|
|
|
|
except OSError:
|
2014-08-06 15:58:24 -04:00
|
|
|
break # worker was killed
|
2014-08-03 18:52:00 -04:00
|
|
|
else:
|
|
|
|
self.fail("worker has not been killed after 5 seconds")
|
|
|
|
|
|
|
|
try:
|
|
|
|
os.kill(daemon_pid, 0)
|
|
|
|
except OSError:
|
|
|
|
self.fail("daemon had been killed")
|
|
|
|
|
2014-09-13 19:22:04 -04:00
|
|
|
# run a normal job
|
[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
|
|
|
rdd = self.sc.parallelize(xrange(100), 1)
|
2014-09-13 19:22:04 -04:00
|
|
|
self.assertEqual(100, rdd.map(str).count())
|
|
|
|
|
|
|
|
def test_after_exception(self):
|
|
|
|
def raise_exception(_):
|
|
|
|
raise Exception()
|
[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
|
|
|
rdd = self.sc.parallelize(xrange(100), 1)
|
2015-04-21 20:49:55 -04:00
|
|
|
with QuietTest(self.sc):
|
|
|
|
self.assertRaises(Exception, lambda: rdd.foreach(raise_exception))
|
2014-09-13 19:22:04 -04:00
|
|
|
self.assertEqual(100, rdd.map(str).count())
|
|
|
|
|
|
|
|
def test_after_jvm_exception(self):
|
|
|
|
tempFile = tempfile.NamedTemporaryFile(delete=False)
|
2015-04-16 19:20:57 -04:00
|
|
|
tempFile.write(b"Hello World!")
|
2014-09-13 19:22:04 -04:00
|
|
|
tempFile.close()
|
|
|
|
data = self.sc.textFile(tempFile.name, 1)
|
|
|
|
filtered_data = data.filter(lambda x: True)
|
|
|
|
self.assertEqual(1, filtered_data.count())
|
|
|
|
os.unlink(tempFile.name)
|
2015-04-21 20:49:55 -04:00
|
|
|
with QuietTest(self.sc):
|
|
|
|
self.assertRaises(Exception, lambda: filtered_data.count())
|
2014-09-13 19:22:04 -04:00
|
|
|
|
[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
|
|
|
rdd = self.sc.parallelize(xrange(100), 1)
|
2014-09-13 19:22:04 -04:00
|
|
|
self.assertEqual(100, rdd.map(str).count())
|
|
|
|
|
|
|
|
def test_accumulator_when_reuse_worker(self):
|
|
|
|
from pyspark.accumulators import INT_ACCUMULATOR_PARAM
|
|
|
|
acc1 = self.sc.accumulator(0, INT_ACCUMULATOR_PARAM)
|
[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
|
|
|
self.sc.parallelize(xrange(100), 20).foreach(lambda x: acc1.add(x))
|
2014-09-13 19:22:04 -04:00
|
|
|
self.assertEqual(sum(range(100)), acc1.value)
|
|
|
|
|
|
|
|
acc2 = self.sc.accumulator(0, INT_ACCUMULATOR_PARAM)
|
[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
|
|
|
self.sc.parallelize(xrange(100), 20).foreach(lambda x: acc2.add(x))
|
2014-09-13 19:22:04 -04:00
|
|
|
self.assertEqual(sum(range(100)), acc2.value)
|
|
|
|
self.assertEqual(sum(range(100)), acc1.value)
|
|
|
|
|
2014-10-23 20:20:00 -04:00
|
|
|
def test_reuse_worker_after_take(self):
|
[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
|
|
|
rdd = self.sc.parallelize(xrange(100000), 1)
|
2014-10-23 20:20:00 -04:00
|
|
|
self.assertEqual(0, rdd.first())
|
|
|
|
|
|
|
|
def count():
|
|
|
|
try:
|
|
|
|
rdd.count()
|
|
|
|
except Exception:
|
|
|
|
pass
|
|
|
|
|
|
|
|
t = threading.Thread(target=count)
|
|
|
|
t.daemon = True
|
|
|
|
t.start()
|
|
|
|
t.join(5)
|
|
|
|
self.assertTrue(not t.isAlive())
|
|
|
|
self.assertEqual(100000, rdd.count())
|
|
|
|
|
2015-04-10 17:04:53 -04:00
|
|
|
def test_with_different_versions_of_python(self):
|
|
|
|
rdd = self.sc.parallelize(range(10))
|
|
|
|
rdd.count()
|
2015-05-18 15:55:13 -04:00
|
|
|
version = self.sc.pythonVer
|
|
|
|
self.sc.pythonVer = "2.0"
|
2015-04-10 17:04:53 -04:00
|
|
|
try:
|
2015-04-21 20:49:55 -04:00
|
|
|
with QuietTest(self.sc):
|
|
|
|
self.assertRaises(Py4JJavaError, lambda: rdd.count())
|
2015-04-10 17:04:53 -04:00
|
|
|
finally:
|
2015-05-18 15:55:13 -04:00
|
|
|
self.sc.pythonVer = version
|
2015-04-10 17:04:53 -04:00
|
|
|
|
2014-08-03 18:52:00 -04:00
|
|
|
|
2014-10-06 17:07:53 -04:00
|
|
|
class SparkSubmitTests(unittest.TestCase):
|
2014-08-06 15:58:24 -04:00
|
|
|
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
def setUp(self):
|
|
|
|
self.programDir = tempfile.mkdtemp()
|
2018-05-07 01:00:18 -04:00
|
|
|
tmp_dir = tempfile.gettempdir()
|
|
|
|
self.sparkSubmit = [
|
|
|
|
os.path.join(os.environ.get("SPARK_HOME"), "bin", "spark-submit"),
|
|
|
|
"--conf", "spark.driver.extraJavaOptions=-Djava.io.tmpdir={0}".format(tmp_dir),
|
|
|
|
"--conf", "spark.executor.extraJavaOptions=-Djava.io.tmpdir={0}".format(tmp_dir),
|
|
|
|
]
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
|
|
|
|
def tearDown(self):
|
|
|
|
shutil.rmtree(self.programDir)
|
|
|
|
|
2015-02-17 20:15:43 -05:00
|
|
|
def createTempFile(self, name, content, dir=None):
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
"""
|
|
|
|
Create a temp file with the given name and content and return its path.
|
|
|
|
Strips leading spaces from content up to the first '|' in each line.
|
|
|
|
"""
|
|
|
|
pattern = re.compile(r'^ *\|', re.MULTILINE)
|
|
|
|
content = re.sub(pattern, '', content.strip())
|
2015-02-17 20:15:43 -05:00
|
|
|
if dir is None:
|
|
|
|
path = os.path.join(self.programDir, name)
|
|
|
|
else:
|
|
|
|
os.makedirs(os.path.join(self.programDir, dir))
|
|
|
|
path = os.path.join(self.programDir, dir, name)
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
with open(path, "w") as f:
|
|
|
|
f.write(content)
|
|
|
|
return path
|
|
|
|
|
2015-02-17 20:15:43 -05:00
|
|
|
def createFileInZip(self, name, content, ext=".zip", dir=None, zip_name=None):
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
"""
|
|
|
|
Create a zip archive containing a file with the given content and return its path.
|
|
|
|
Strips leading spaces from content up to the first '|' in each line.
|
|
|
|
"""
|
|
|
|
pattern = re.compile(r'^ *\|', re.MULTILINE)
|
|
|
|
content = re.sub(pattern, '', content.strip())
|
2015-02-17 20:15:43 -05:00
|
|
|
if dir is None:
|
|
|
|
path = os.path.join(self.programDir, name + ext)
|
|
|
|
else:
|
|
|
|
path = os.path.join(self.programDir, dir, zip_name + ext)
|
2014-08-11 14:54:09 -04:00
|
|
|
zip = zipfile.ZipFile(path, 'w')
|
|
|
|
zip.writestr(name, content)
|
|
|
|
zip.close()
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
return path
|
|
|
|
|
2015-02-17 20:15:43 -05:00
|
|
|
def create_spark_package(self, artifact_name):
|
|
|
|
group_id, artifact_id, version = artifact_name.split(":")
|
|
|
|
self.createTempFile("%s-%s.pom" % (artifact_id, version), ("""
|
|
|
|
|<?xml version="1.0" encoding="UTF-8"?>
|
|
|
|
|<project xmlns="http://maven.apache.org/POM/4.0.0"
|
|
|
|
| xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
|
|
|
| xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
|
|
|
|
| http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
|
|
|
| <modelVersion>4.0.0</modelVersion>
|
|
|
|
| <groupId>%s</groupId>
|
|
|
|
| <artifactId>%s</artifactId>
|
|
|
|
| <version>%s</version>
|
|
|
|
|</project>
|
|
|
|
""" % (group_id, artifact_id, version)).lstrip(),
|
|
|
|
os.path.join(group_id, artifact_id, version))
|
|
|
|
self.createFileInZip("%s.py" % artifact_id, """
|
|
|
|
|def myfunc(x):
|
|
|
|
| return x + 1
|
|
|
|
""", ".jar", os.path.join(group_id, artifact_id, version),
|
|
|
|
"%s-%s" % (artifact_id, version))
|
|
|
|
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
def test_single_script(self):
|
|
|
|
"""Submit and test a single script file"""
|
|
|
|
script = self.createTempFile("test.py", """
|
|
|
|
|from pyspark import SparkContext
|
|
|
|
|
|
|
|
|
|sc = SparkContext()
|
2015-04-16 19:20:57 -04:00
|
|
|
|print(sc.parallelize([1, 2, 3]).map(lambda x: x * 2).collect())
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
""")
|
2018-05-07 01:00:18 -04:00
|
|
|
proc = subprocess.Popen(self.sparkSubmit + [script], stdout=subprocess.PIPE)
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
out, err = proc.communicate()
|
|
|
|
self.assertEqual(0, proc.returncode)
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertIn("[2, 4, 6]", out.decode('utf-8'))
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
|
|
|
|
def test_script_with_local_functions(self):
|
|
|
|
"""Submit and test a single script file calling a global function"""
|
|
|
|
script = self.createTempFile("test.py", """
|
|
|
|
|from pyspark import SparkContext
|
|
|
|
|
|
|
|
|
|def foo(x):
|
|
|
|
| return x * 3
|
|
|
|
|
|
|
|
|
|sc = SparkContext()
|
2015-04-16 19:20:57 -04:00
|
|
|
|print(sc.parallelize([1, 2, 3]).map(foo).collect())
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
""")
|
2018-05-07 01:00:18 -04:00
|
|
|
proc = subprocess.Popen(self.sparkSubmit + [script], stdout=subprocess.PIPE)
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
out, err = proc.communicate()
|
|
|
|
self.assertEqual(0, proc.returncode)
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertIn("[3, 6, 9]", out.decode('utf-8'))
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
|
|
|
|
def test_module_dependency(self):
|
|
|
|
"""Submit and test a script with a dependency on another module"""
|
|
|
|
script = self.createTempFile("test.py", """
|
|
|
|
|from pyspark import SparkContext
|
|
|
|
|from mylib import myfunc
|
|
|
|
|
|
|
|
|
|sc = SparkContext()
|
2015-04-16 19:20:57 -04:00
|
|
|
|print(sc.parallelize([1, 2, 3]).map(myfunc).collect())
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
""")
|
|
|
|
zip = self.createFileInZip("mylib.py", """
|
|
|
|
|def myfunc(x):
|
|
|
|
| return x + 1
|
|
|
|
""")
|
2018-05-07 01:00:18 -04:00
|
|
|
proc = subprocess.Popen(self.sparkSubmit + ["--py-files", zip, script],
|
2014-07-22 01:30:53 -04:00
|
|
|
stdout=subprocess.PIPE)
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
out, err = proc.communicate()
|
|
|
|
self.assertEqual(0, proc.returncode)
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertIn("[2, 3, 4]", out.decode('utf-8'))
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
|
|
|
|
def test_module_dependency_on_cluster(self):
|
|
|
|
"""Submit and test a script with a dependency on another module on a cluster"""
|
|
|
|
script = self.createTempFile("test.py", """
|
|
|
|
|from pyspark import SparkContext
|
|
|
|
|from mylib import myfunc
|
|
|
|
|
|
|
|
|
|sc = SparkContext()
|
2015-04-16 19:20:57 -04:00
|
|
|
|print(sc.parallelize([1, 2, 3]).map(myfunc).collect())
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
""")
|
|
|
|
zip = self.createFileInZip("mylib.py", """
|
|
|
|
|def myfunc(x):
|
|
|
|
| return x + 1
|
|
|
|
""")
|
2018-05-07 01:00:18 -04:00
|
|
|
proc = subprocess.Popen(self.sparkSubmit + ["--py-files", zip, "--master",
|
2015-07-22 18:28:09 -04:00
|
|
|
"local-cluster[1,1,1024]", script],
|
2014-08-06 15:58:24 -04:00
|
|
|
stdout=subprocess.PIPE)
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
out, err = proc.communicate()
|
|
|
|
self.assertEqual(0, proc.returncode)
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertIn("[2, 3, 4]", out.decode('utf-8'))
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
|
2015-02-17 20:15:43 -05:00
|
|
|
def test_package_dependency(self):
|
|
|
|
"""Submit and test a script with a dependency on a Spark Package"""
|
|
|
|
script = self.createTempFile("test.py", """
|
|
|
|
|from pyspark import SparkContext
|
|
|
|
|from mylib import myfunc
|
|
|
|
|
|
|
|
|
|sc = SparkContext()
|
2015-04-16 19:20:57 -04:00
|
|
|
|print(sc.parallelize([1, 2, 3]).map(myfunc).collect())
|
2015-02-17 20:15:43 -05:00
|
|
|
""")
|
|
|
|
self.create_spark_package("a:mylib:0.1")
|
2018-05-07 01:00:18 -04:00
|
|
|
proc = subprocess.Popen(
|
|
|
|
self.sparkSubmit + ["--packages", "a:mylib:0.1", "--repositories",
|
|
|
|
"file:" + self.programDir, script],
|
|
|
|
stdout=subprocess.PIPE)
|
2015-02-17 20:15:43 -05:00
|
|
|
out, err = proc.communicate()
|
|
|
|
self.assertEqual(0, proc.returncode)
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertIn("[2, 3, 4]", out.decode('utf-8'))
|
2015-02-17 20:15:43 -05:00
|
|
|
|
|
|
|
def test_package_dependency_on_cluster(self):
|
|
|
|
"""Submit and test a script with a dependency on a Spark Package on a cluster"""
|
|
|
|
script = self.createTempFile("test.py", """
|
|
|
|
|from pyspark import SparkContext
|
|
|
|
|from mylib import myfunc
|
|
|
|
|
|
|
|
|
|sc = SparkContext()
|
2015-04-16 19:20:57 -04:00
|
|
|
|print(sc.parallelize([1, 2, 3]).map(myfunc).collect())
|
2015-02-17 20:15:43 -05:00
|
|
|
""")
|
|
|
|
self.create_spark_package("a:mylib:0.1")
|
2018-05-07 01:00:18 -04:00
|
|
|
proc = subprocess.Popen(
|
|
|
|
self.sparkSubmit + ["--packages", "a:mylib:0.1", "--repositories",
|
|
|
|
"file:" + self.programDir, "--master", "local-cluster[1,1,1024]",
|
|
|
|
script],
|
|
|
|
stdout=subprocess.PIPE)
|
2015-02-17 20:15:43 -05:00
|
|
|
out, err = proc.communicate()
|
|
|
|
self.assertEqual(0, proc.returncode)
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertIn("[2, 3, 4]", out.decode('utf-8'))
|
2015-02-17 20:15:43 -05:00
|
|
|
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
def test_single_script_on_cluster(self):
|
|
|
|
"""Submit and test a single script on a cluster"""
|
|
|
|
script = self.createTempFile("test.py", """
|
|
|
|
|from pyspark import SparkContext
|
|
|
|
|
|
|
|
|
|def foo(x):
|
|
|
|
| return x * 2
|
|
|
|
|
|
|
|
|
|sc = SparkContext()
|
2015-04-16 19:20:57 -04:00
|
|
|
|print(sc.parallelize([1, 2, 3]).map(foo).collect())
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
""")
|
2014-10-06 17:07:53 -04:00
|
|
|
# this will fail if you have different spark.executor.memory
|
|
|
|
# in conf/spark-defaults.conf
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
proc = subprocess.Popen(
|
2018-05-07 01:00:18 -04:00
|
|
|
self.sparkSubmit + ["--master", "local-cluster[1,1,1024]", script],
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
stdout=subprocess.PIPE)
|
|
|
|
out, err = proc.communicate()
|
|
|
|
self.assertEqual(0, proc.returncode)
|
2015-04-16 19:20:57 -04:00
|
|
|
self.assertIn("[2, 4, 6]", out.decode('utf-8'))
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
|
2017-01-25 15:08:08 -05:00
|
|
|
def test_user_configuration(self):
|
|
|
|
"""Make sure user configuration is respected (SPARK-19307)"""
|
|
|
|
script = self.createTempFile("test.py", """
|
|
|
|
|from pyspark import SparkConf, SparkContext
|
|
|
|
|
|
|
|
|
|conf = SparkConf().set("spark.test_config", "1")
|
|
|
|
|sc = SparkContext(conf = conf)
|
|
|
|
|try:
|
|
|
|
| if sc._conf.get("spark.test_config") != "1":
|
|
|
|
| raise Exception("Cannot find spark.test_config in SparkContext's conf.")
|
|
|
|
|finally:
|
|
|
|
| sc.stop()
|
|
|
|
""")
|
|
|
|
proc = subprocess.Popen(
|
2018-05-07 01:00:18 -04:00
|
|
|
self.sparkSubmit + ["--master", "local", script],
|
2017-01-25 15:08:08 -05:00
|
|
|
stdout=subprocess.PIPE,
|
|
|
|
stderr=subprocess.STDOUT)
|
|
|
|
out, err = proc.communicate()
|
|
|
|
self.assertEqual(0, proc.returncode, msg="Process failed with error:\n {0}".format(out))
|
|
|
|
|
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.
This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.
In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.
In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.
Author: Matei Zaharia <matei@databricks.com>
Closes #664 from mateiz/py-submit and squashes the following commits:
15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
|
|
|
|
2014-10-06 17:07:53 -04:00
|
|
|
class ContextTests(unittest.TestCase):
|
|
|
|
|
|
|
|
def test_failed_sparkcontext_creation(self):
|
|
|
|
# Regression test for SPARK-1550
|
|
|
|
self.assertRaises(Exception, lambda: SparkContext("an-invalid-master-name"))
|
2014-09-09 21:54:54 -04:00
|
|
|
|
2015-10-19 19:18:20 -04:00
|
|
|
def test_get_or_create(self):
|
|
|
|
with SparkContext.getOrCreate() as sc:
|
|
|
|
self.assertTrue(SparkContext.getOrCreate() is sc)
|
|
|
|
|
2016-04-09 21:34:54 -04:00
|
|
|
def test_parallelize_eager_cleanup(self):
|
|
|
|
with SparkContext() as sc:
|
|
|
|
temp_files = os.listdir(sc._temp_dir)
|
|
|
|
rdd = sc.parallelize([0, 1, 2])
|
|
|
|
post_parallalize_temp_files = os.listdir(sc._temp_dir)
|
|
|
|
self.assertEqual(temp_files, post_parallalize_temp_files)
|
|
|
|
|
2016-06-28 10:54:44 -04:00
|
|
|
def test_set_conf(self):
|
|
|
|
# This is for an internal use case. When there is an existing SparkContext,
|
|
|
|
# SparkSession's builder needs to set configs into SparkContext's conf.
|
|
|
|
sc = SparkContext()
|
|
|
|
sc._conf.set("spark.test.SPARK16224", "SPARK16224")
|
|
|
|
self.assertEqual(sc._jsc.sc().conf().get("spark.test.SPARK16224"), "SPARK16224")
|
|
|
|
sc.stop()
|
|
|
|
|
2014-09-09 21:54:54 -04:00
|
|
|
def test_stop(self):
|
|
|
|
sc = SparkContext()
|
|
|
|
self.assertNotEqual(SparkContext._active_spark_context, None)
|
|
|
|
sc.stop()
|
|
|
|
self.assertEqual(SparkContext._active_spark_context, None)
|
|
|
|
|
|
|
|
def test_with(self):
|
|
|
|
with SparkContext() as sc:
|
|
|
|
self.assertNotEqual(SparkContext._active_spark_context, None)
|
|
|
|
self.assertEqual(SparkContext._active_spark_context, None)
|
|
|
|
|
|
|
|
def test_with_exception(self):
|
|
|
|
try:
|
|
|
|
with SparkContext() as sc:
|
|
|
|
self.assertNotEqual(SparkContext._active_spark_context, None)
|
|
|
|
raise Exception()
|
|
|
|
except:
|
|
|
|
pass
|
|
|
|
self.assertEqual(SparkContext._active_spark_context, None)
|
|
|
|
|
|
|
|
def test_with_stop(self):
|
|
|
|
with SparkContext() as sc:
|
|
|
|
self.assertNotEqual(SparkContext._active_spark_context, None)
|
|
|
|
sc.stop()
|
|
|
|
self.assertEqual(SparkContext._active_spark_context, None)
|
|
|
|
|
2015-02-17 16:36:43 -05:00
|
|
|
def test_progress_api(self):
|
|
|
|
with SparkContext() as sc:
|
|
|
|
sc.setJobGroup('test_progress_api', '', True)
|
|
|
|
rdd = sc.parallelize(range(10)).map(lambda x: time.sleep(100))
|
2015-04-21 20:49:55 -04:00
|
|
|
|
|
|
|
def run():
|
|
|
|
try:
|
|
|
|
rdd.count()
|
|
|
|
except Exception:
|
|
|
|
pass
|
|
|
|
t = threading.Thread(target=run)
|
2015-02-17 16:36:43 -05:00
|
|
|
t.daemon = True
|
|
|
|
t.start()
|
|
|
|
# wait for scheduler to start
|
|
|
|
time.sleep(1)
|
|
|
|
|
|
|
|
tracker = sc.statusTracker()
|
|
|
|
jobIds = tracker.getJobIdsForGroup('test_progress_api')
|
|
|
|
self.assertEqual(1, len(jobIds))
|
|
|
|
job = tracker.getJobInfo(jobIds[0])
|
|
|
|
self.assertEqual(1, len(job.stageIds))
|
|
|
|
stage = tracker.getStageInfo(job.stageIds[0])
|
|
|
|
self.assertEqual(rdd.getNumPartitions(), stage.numTasks)
|
|
|
|
|
|
|
|
sc.cancelAllJobs()
|
|
|
|
t.join()
|
|
|
|
# wait for event listener to update the status
|
|
|
|
time.sleep(1)
|
|
|
|
|
|
|
|
job = tracker.getJobInfo(jobIds[0])
|
|
|
|
self.assertEqual('FAILED', job.status)
|
|
|
|
self.assertEqual([], tracker.getActiveJobsIds())
|
|
|
|
self.assertEqual([], tracker.getActiveStageIds())
|
|
|
|
|
|
|
|
sc.stop()
|
|
|
|
|
2015-05-21 17:08:57 -04:00
|
|
|
def test_startTime(self):
|
|
|
|
with SparkContext() as sc:
|
|
|
|
self.assertGreater(sc.startTime, 0)
|
|
|
|
|
2014-09-09 21:54:54 -04:00
|
|
|
|
2016-04-04 23:19:20 -04:00
|
|
|
class ConfTests(unittest.TestCase):
|
|
|
|
def test_memory_conf(self):
|
|
|
|
memoryList = ["1T", "1G", "1M", "1024K"]
|
|
|
|
for memory in memoryList:
|
|
|
|
sc = SparkContext(conf=SparkConf().set("spark.python.worker.memory", memory))
|
|
|
|
l = list(range(1024))
|
|
|
|
random.shuffle(l)
|
|
|
|
rdd = sc.parallelize(l, 4)
|
|
|
|
self.assertEqual(sorted(l), rdd.sortBy(lambda x: x).collect())
|
|
|
|
sc.stop()
|
|
|
|
|
|
|
|
|
2017-03-03 19:43:45 -05:00
|
|
|
class KeywordOnlyTests(unittest.TestCase):
|
|
|
|
class Wrapped(object):
|
|
|
|
@keyword_only
|
|
|
|
def set(self, x=None, y=None):
|
|
|
|
if "x" in self._input_kwargs:
|
|
|
|
self._x = self._input_kwargs["x"]
|
|
|
|
if "y" in self._input_kwargs:
|
|
|
|
self._y = self._input_kwargs["y"]
|
|
|
|
return x, y
|
|
|
|
|
|
|
|
def test_keywords(self):
|
|
|
|
w = self.Wrapped()
|
|
|
|
x, y = w.set(y=1)
|
|
|
|
self.assertEqual(y, 1)
|
|
|
|
self.assertEqual(y, w._y)
|
|
|
|
self.assertIsNone(x)
|
|
|
|
self.assertFalse(hasattr(w, "_x"))
|
|
|
|
|
|
|
|
def test_non_keywords(self):
|
|
|
|
w = self.Wrapped()
|
|
|
|
self.assertRaises(TypeError, lambda: w.set(0, y=1))
|
|
|
|
|
|
|
|
def test_kwarg_ownership(self):
|
|
|
|
# test _input_kwargs is owned by each class instance and not a shared static variable
|
|
|
|
class Setter(object):
|
|
|
|
@keyword_only
|
|
|
|
def set(self, x=None, other=None, other_x=None):
|
|
|
|
if "other" in self._input_kwargs:
|
|
|
|
self._input_kwargs["other"].set(x=self._input_kwargs["other_x"])
|
|
|
|
self._x = self._input_kwargs["x"]
|
|
|
|
|
|
|
|
a = Setter()
|
|
|
|
b = Setter()
|
|
|
|
a.set(x=1, other=b, other_x=2)
|
|
|
|
self.assertEqual(a._x, 1)
|
|
|
|
self.assertEqual(b._x, 2)
|
|
|
|
|
|
|
|
|
[SPARK-23517][PYTHON] Make `pyspark.util._exception_message` produce the trace from Java side by Py4JJavaError
## What changes were proposed in this pull request?
This PR proposes for `pyspark.util._exception_message` to produce the trace from Java side by `Py4JJavaError`.
Currently, in Python 2, it uses `message` attribute which `Py4JJavaError` didn't happen to have:
```python
>>> from pyspark.util import _exception_message
>>> try:
... sc._jvm.java.lang.String(None)
... except Exception as e:
... pass
...
>>> e.message
''
```
Seems we should use `str` instead for now:
https://github.com/bartdag/py4j/blob/aa6c53b59027925a426eb09b58c453de02c21b7c/py4j-python/src/py4j/protocol.py#L412
but this doesn't address the problem with non-ascii string from Java side -
`https://github.com/bartdag/py4j/issues/306`
So, we could directly call `__str__()`:
```python
>>> e.__str__()
u'An error occurred while calling None.java.lang.String.\n: java.lang.NullPointerException\n\tat java.lang.String.<init>(String.java:588)\n\tat sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)\n\tat sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)\n\tat sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)\n\tat java.lang.reflect.Constructor.newInstance(Constructor.java:422)\n\tat py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:247)\n\tat py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\n\tat py4j.Gateway.invoke(Gateway.java:238)\n\tat py4j.commands.ConstructorCommand.invokeConstructor(ConstructorCommand.java:80)\n\tat py4j.commands.ConstructorCommand.execute(ConstructorCommand.java:69)\n\tat py4j.GatewayConnection.run(GatewayConnection.java:214)\n\tat java.lang.Thread.run(Thread.java:745)\n'
```
which doesn't type coerce unicodes to `str` in Python 2.
This can be actually a problem:
```python
from pyspark.sql.functions import udf
spark.conf.set("spark.sql.execution.arrow.enabled", True)
spark.range(1).select(udf(lambda x: [[]])()).toPandas()
```
**Before**
```
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../spark/python/pyspark/sql/dataframe.py", line 2009, in toPandas
raise RuntimeError("%s\n%s" % (_exception_message(e), msg))
RuntimeError:
Note: toPandas attempted Arrow optimization because 'spark.sql.execution.arrow.enabled' is set to true. Please set it to false to disable this.
```
**After**
```
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../spark/python/pyspark/sql/dataframe.py", line 2009, in toPandas
raise RuntimeError("%s\n%s" % (_exception_message(e), msg))
RuntimeError: An error occurred while calling o47.collectAsArrowToPython.
: org.apache.spark.SparkException: Job aborted due to stage failure: Task 7 in stage 0.0 failed 1 times, most recent failure: Lost task 7.0 in stage 0.0 (TID 7, localhost, executor driver): org.apache.spark.api.python.PythonException: Traceback (most recent call last):
File "/.../spark/python/pyspark/worker.py", line 245, in main
process()
File "/.../spark/python/pyspark/worker.py", line 240, in process
...
Note: toPandas attempted Arrow optimization because 'spark.sql.execution.arrow.enabled' is set to true. Please set it to false to disable this.
```
## How was this patch tested?
Manually tested and unit tests were added.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #20680 from HyukjinKwon/SPARK-23517.
2018-02-28 10:44:13 -05:00
|
|
|
class UtilTests(PySparkTestCase):
|
|
|
|
def test_py4j_exception_message(self):
|
|
|
|
from pyspark.util import _exception_message
|
|
|
|
|
|
|
|
with self.assertRaises(Py4JJavaError) as context:
|
|
|
|
# This attempts java.lang.String(null) which throws an NPE.
|
|
|
|
self.sc._jvm.java.lang.String(None)
|
|
|
|
|
|
|
|
self.assertTrue('NullPointerException' in _exception_message(context.exception))
|
|
|
|
|
2018-05-08 09:22:54 -04:00
|
|
|
def test_parsing_version_string(self):
|
|
|
|
from pyspark.util import VersionUtils
|
|
|
|
self.assertRaises(ValueError, lambda: VersionUtils.majorMinorVersion("abced"))
|
|
|
|
|
[SPARK-23517][PYTHON] Make `pyspark.util._exception_message` produce the trace from Java side by Py4JJavaError
## What changes were proposed in this pull request?
This PR proposes for `pyspark.util._exception_message` to produce the trace from Java side by `Py4JJavaError`.
Currently, in Python 2, it uses `message` attribute which `Py4JJavaError` didn't happen to have:
```python
>>> from pyspark.util import _exception_message
>>> try:
... sc._jvm.java.lang.String(None)
... except Exception as e:
... pass
...
>>> e.message
''
```
Seems we should use `str` instead for now:
https://github.com/bartdag/py4j/blob/aa6c53b59027925a426eb09b58c453de02c21b7c/py4j-python/src/py4j/protocol.py#L412
but this doesn't address the problem with non-ascii string from Java side -
`https://github.com/bartdag/py4j/issues/306`
So, we could directly call `__str__()`:
```python
>>> e.__str__()
u'An error occurred while calling None.java.lang.String.\n: java.lang.NullPointerException\n\tat java.lang.String.<init>(String.java:588)\n\tat sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)\n\tat sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)\n\tat sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)\n\tat java.lang.reflect.Constructor.newInstance(Constructor.java:422)\n\tat py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:247)\n\tat py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\n\tat py4j.Gateway.invoke(Gateway.java:238)\n\tat py4j.commands.ConstructorCommand.invokeConstructor(ConstructorCommand.java:80)\n\tat py4j.commands.ConstructorCommand.execute(ConstructorCommand.java:69)\n\tat py4j.GatewayConnection.run(GatewayConnection.java:214)\n\tat java.lang.Thread.run(Thread.java:745)\n'
```
which doesn't type coerce unicodes to `str` in Python 2.
This can be actually a problem:
```python
from pyspark.sql.functions import udf
spark.conf.set("spark.sql.execution.arrow.enabled", True)
spark.range(1).select(udf(lambda x: [[]])()).toPandas()
```
**Before**
```
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../spark/python/pyspark/sql/dataframe.py", line 2009, in toPandas
raise RuntimeError("%s\n%s" % (_exception_message(e), msg))
RuntimeError:
Note: toPandas attempted Arrow optimization because 'spark.sql.execution.arrow.enabled' is set to true. Please set it to false to disable this.
```
**After**
```
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../spark/python/pyspark/sql/dataframe.py", line 2009, in toPandas
raise RuntimeError("%s\n%s" % (_exception_message(e), msg))
RuntimeError: An error occurred while calling o47.collectAsArrowToPython.
: org.apache.spark.SparkException: Job aborted due to stage failure: Task 7 in stage 0.0 failed 1 times, most recent failure: Lost task 7.0 in stage 0.0 (TID 7, localhost, executor driver): org.apache.spark.api.python.PythonException: Traceback (most recent call last):
File "/.../spark/python/pyspark/worker.py", line 245, in main
process()
File "/.../spark/python/pyspark/worker.py", line 240, in process
...
Note: toPandas attempted Arrow optimization because 'spark.sql.execution.arrow.enabled' is set to true. Please set it to false to disable this.
```
## How was this patch tested?
Manually tested and unit tests were added.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #20680 from HyukjinKwon/SPARK-23517.
2018-02-28 10:44:13 -05:00
|
|
|
|
2014-05-31 17:59:09 -04:00
|
|
|
@unittest.skipIf(not _have_scipy, "SciPy not installed")
|
|
|
|
class SciPyTests(PySparkTestCase):
|
2014-08-06 15:58:24 -04:00
|
|
|
|
2014-05-31 17:59:09 -04:00
|
|
|
"""General PySpark tests that depend on scipy """
|
|
|
|
|
|
|
|
def test_serialize(self):
|
|
|
|
from scipy.special import gammaln
|
|
|
|
x = range(1, 5)
|
2015-04-16 19:20:57 -04:00
|
|
|
expected = list(map(gammaln, x))
|
2014-05-31 17:59:09 -04:00
|
|
|
observed = self.sc.parallelize(x).map(gammaln).collect()
|
|
|
|
self.assertEqual(expected, observed)
|
|
|
|
|
|
|
|
|
StatCounter on NumPy arrays [PYSPARK][SPARK-2012]
These changes allow StatCounters to work properly on NumPy arrays, to fix the issue reported here (https://issues.apache.org/jira/browse/SPARK-2012).
If NumPy is installed, the NumPy functions ``maximum``, ``minimum``, and ``sqrt``, which work on arrays, are used to merge statistics. If not, we fall back on scalar operators, so it will work on arrays with NumPy, but will also work without NumPy.
New unit tests added, along with a check for NumPy in the tests.
Author: Jeremy Freeman <the.freeman.lab@gmail.com>
Closes #1725 from freeman-lab/numpy-max-statcounter and squashes the following commits:
fe973b1 [Jeremy Freeman] Avoid duplicate array import in tests
7f0e397 [Jeremy Freeman] Refactored check for numpy
8e764dd [Jeremy Freeman] Explicit numpy imports
875414c [Jeremy Freeman] Fixed indents
1c8a832 [Jeremy Freeman] Unit tests for StatCounter with NumPy arrays
176a127 [Jeremy Freeman] Use numpy arrays in StatCounter
2014-08-02 01:33:25 -04:00
|
|
|
@unittest.skipIf(not _have_numpy, "NumPy not installed")
|
|
|
|
class NumPyTests(PySparkTestCase):
|
2014-08-06 15:58:24 -04:00
|
|
|
|
StatCounter on NumPy arrays [PYSPARK][SPARK-2012]
These changes allow StatCounters to work properly on NumPy arrays, to fix the issue reported here (https://issues.apache.org/jira/browse/SPARK-2012).
If NumPy is installed, the NumPy functions ``maximum``, ``minimum``, and ``sqrt``, which work on arrays, are used to merge statistics. If not, we fall back on scalar operators, so it will work on arrays with NumPy, but will also work without NumPy.
New unit tests added, along with a check for NumPy in the tests.
Author: Jeremy Freeman <the.freeman.lab@gmail.com>
Closes #1725 from freeman-lab/numpy-max-statcounter and squashes the following commits:
fe973b1 [Jeremy Freeman] Avoid duplicate array import in tests
7f0e397 [Jeremy Freeman] Refactored check for numpy
8e764dd [Jeremy Freeman] Explicit numpy imports
875414c [Jeremy Freeman] Fixed indents
1c8a832 [Jeremy Freeman] Unit tests for StatCounter with NumPy arrays
176a127 [Jeremy Freeman] Use numpy arrays in StatCounter
2014-08-02 01:33:25 -04:00
|
|
|
"""General PySpark tests that depend on numpy """
|
|
|
|
|
|
|
|
def test_statcounter_array(self):
|
2014-08-06 15:58:24 -04:00
|
|
|
x = self.sc.parallelize([np.array([1.0, 1.0]), np.array([2.0, 2.0]), np.array([3.0, 3.0])])
|
StatCounter on NumPy arrays [PYSPARK][SPARK-2012]
These changes allow StatCounters to work properly on NumPy arrays, to fix the issue reported here (https://issues.apache.org/jira/browse/SPARK-2012).
If NumPy is installed, the NumPy functions ``maximum``, ``minimum``, and ``sqrt``, which work on arrays, are used to merge statistics. If not, we fall back on scalar operators, so it will work on arrays with NumPy, but will also work without NumPy.
New unit tests added, along with a check for NumPy in the tests.
Author: Jeremy Freeman <the.freeman.lab@gmail.com>
Closes #1725 from freeman-lab/numpy-max-statcounter and squashes the following commits:
fe973b1 [Jeremy Freeman] Avoid duplicate array import in tests
7f0e397 [Jeremy Freeman] Refactored check for numpy
8e764dd [Jeremy Freeman] Explicit numpy imports
875414c [Jeremy Freeman] Fixed indents
1c8a832 [Jeremy Freeman] Unit tests for StatCounter with NumPy arrays
176a127 [Jeremy Freeman] Use numpy arrays in StatCounter
2014-08-02 01:33:25 -04:00
|
|
|
s = x.stats()
|
2014-08-06 15:58:24 -04:00
|
|
|
self.assertSequenceEqual([2.0, 2.0], s.mean().tolist())
|
|
|
|
self.assertSequenceEqual([1.0, 1.0], s.min().tolist())
|
|
|
|
self.assertSequenceEqual([3.0, 3.0], s.max().tolist())
|
|
|
|
self.assertSequenceEqual([1.0, 1.0], s.sampleStdev().tolist())
|
StatCounter on NumPy arrays [PYSPARK][SPARK-2012]
These changes allow StatCounters to work properly on NumPy arrays, to fix the issue reported here (https://issues.apache.org/jira/browse/SPARK-2012).
If NumPy is installed, the NumPy functions ``maximum``, ``minimum``, and ``sqrt``, which work on arrays, are used to merge statistics. If not, we fall back on scalar operators, so it will work on arrays with NumPy, but will also work without NumPy.
New unit tests added, along with a check for NumPy in the tests.
Author: Jeremy Freeman <the.freeman.lab@gmail.com>
Closes #1725 from freeman-lab/numpy-max-statcounter and squashes the following commits:
fe973b1 [Jeremy Freeman] Avoid duplicate array import in tests
7f0e397 [Jeremy Freeman] Refactored check for numpy
8e764dd [Jeremy Freeman] Explicit numpy imports
875414c [Jeremy Freeman] Fixed indents
1c8a832 [Jeremy Freeman] Unit tests for StatCounter with NumPy arrays
176a127 [Jeremy Freeman] Use numpy arrays in StatCounter
2014-08-02 01:33:25 -04:00
|
|
|
|
2015-09-29 16:38:15 -04:00
|
|
|
stats_dict = s.asDict()
|
|
|
|
self.assertEqual(3, stats_dict['count'])
|
|
|
|
self.assertSequenceEqual([2.0, 2.0], stats_dict['mean'].tolist())
|
|
|
|
self.assertSequenceEqual([1.0, 1.0], stats_dict['min'].tolist())
|
|
|
|
self.assertSequenceEqual([3.0, 3.0], stats_dict['max'].tolist())
|
|
|
|
self.assertSequenceEqual([6.0, 6.0], stats_dict['sum'].tolist())
|
|
|
|
self.assertSequenceEqual([1.0, 1.0], stats_dict['stdev'].tolist())
|
|
|
|
self.assertSequenceEqual([1.0, 1.0], stats_dict['variance'].tolist())
|
|
|
|
|
|
|
|
stats_sample_dict = s.asDict(sample=True)
|
|
|
|
self.assertEqual(3, stats_dict['count'])
|
|
|
|
self.assertSequenceEqual([2.0, 2.0], stats_sample_dict['mean'].tolist())
|
|
|
|
self.assertSequenceEqual([1.0, 1.0], stats_sample_dict['min'].tolist())
|
|
|
|
self.assertSequenceEqual([3.0, 3.0], stats_sample_dict['max'].tolist())
|
|
|
|
self.assertSequenceEqual([6.0, 6.0], stats_sample_dict['sum'].tolist())
|
|
|
|
self.assertSequenceEqual(
|
|
|
|
[0.816496580927726, 0.816496580927726], stats_sample_dict['stdev'].tolist())
|
|
|
|
self.assertSequenceEqual(
|
|
|
|
[0.6666666666666666, 0.6666666666666666], stats_sample_dict['variance'].tolist())
|
|
|
|
|
StatCounter on NumPy arrays [PYSPARK][SPARK-2012]
These changes allow StatCounters to work properly on NumPy arrays, to fix the issue reported here (https://issues.apache.org/jira/browse/SPARK-2012).
If NumPy is installed, the NumPy functions ``maximum``, ``minimum``, and ``sqrt``, which work on arrays, are used to merge statistics. If not, we fall back on scalar operators, so it will work on arrays with NumPy, but will also work without NumPy.
New unit tests added, along with a check for NumPy in the tests.
Author: Jeremy Freeman <the.freeman.lab@gmail.com>
Closes #1725 from freeman-lab/numpy-max-statcounter and squashes the following commits:
fe973b1 [Jeremy Freeman] Avoid duplicate array import in tests
7f0e397 [Jeremy Freeman] Refactored check for numpy
8e764dd [Jeremy Freeman] Explicit numpy imports
875414c [Jeremy Freeman] Fixed indents
1c8a832 [Jeremy Freeman] Unit tests for StatCounter with NumPy arrays
176a127 [Jeremy Freeman] Use numpy arrays in StatCounter
2014-08-02 01:33:25 -04:00
|
|
|
|
2013-01-16 22:15:14 -05:00
|
|
|
if __name__ == "__main__":
|
2016-01-20 14:11:10 -05:00
|
|
|
from pyspark.tests import *
|
2015-10-22 18:27:11 -04:00
|
|
|
if xmlrunner:
|
2018-04-26 18:11:42 -04:00
|
|
|
unittest.main(testRunner=xmlrunner.XMLTestRunner(output='target/test-reports'), verbosity=2)
|
2015-10-22 18:27:11 -04:00
|
|
|
else:
|
2018-04-26 18:11:42 -04:00
|
|
|
unittest.main(verbosity=2)
|