spark-instrumented-optimizer/python/pyspark/tests.py

1591 lines
64 KiB
Python
Raw Normal View History

#
# 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.
"""
[SPARK-2024] Add saveAsSequenceFile to PySpark JIRA issue: https://issues.apache.org/jira/browse/SPARK-2024 This PR is a followup to #455 and adds capabilities for saving PySpark RDDs using SequenceFile or any Hadoop OutputFormats. * Added RDD methods ```saveAsSequenceFile```, ```saveAsHadoopFile``` and ```saveAsHadoopDataset```, for both old and new MapReduce APIs. * Default converter for converting common data types to Writables. Users may specify custom converters to convert to desired data types. * No out-of-box support for reading/writing arrays, since ArrayWritable itself doesn't have a no-arg constructor for creating an empty instance upon reading. Users need to provide ArrayWritable subtypes. Custom converters for converting arrays to suitable ArrayWritable subtypes are also needed when writing. When reading, the default converter will convert any custom ArrayWritable subtypes to ```Object[]``` and they get pickled to Python tuples. * Added HBase and Cassandra output examples to show how custom output formats and converters can be used. cc MLnick mateiz ahirreddy pwendell Author: Kan Zhang <kzhang@apache.org> Closes #1338 from kanzhang/SPARK-2024 and squashes the following commits: c01e3ef [Kan Zhang] [SPARK-2024] code formatting 6591e37 [Kan Zhang] [SPARK-2024] renaming pickled -> pickledRDD d998ad6 [Kan Zhang] [SPARK-2024] refectoring to get method params below 10 57a7a5e [Kan Zhang] [SPARK-2024] correcting typo 75ca5bd [Kan Zhang] [SPARK-2024] Better type checking for batch serialized RDD 0bdec55 [Kan Zhang] [SPARK-2024] Refactoring newly added tests 9f39ff4 [Kan Zhang] [SPARK-2024] Adding 2 saveAsHadoopDataset tests 0c134f3 [Kan Zhang] [SPARK-2024] Test refactoring and adding couple unbatched cases 7a176df [Kan Zhang] [SPARK-2024] Add saveAsSequenceFile to PySpark
2014-07-30 16:19:05 -04:00
from array import array
from fileinput import input
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
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
import random
import threading
2014-11-18 19:17:51 -05:00
import hashlib
2013-01-16 22:15:14 -05:00
if sys.version_info[:2] <= (2, 6):
try:
import unittest2 as unittest
except ImportError:
sys.stderr.write('Please install unittest2 to test with Python 2.6 or earlier')
sys.exit(1)
else:
import unittest
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
from pyspark.files import SparkFiles
[SPARK-3094] [PySpark] compatitable with PyPy After this patch, we can run PySpark in PyPy (testing with PyPy 2.3.1 in Mac 10.9), for example: ``` PYSPARK_PYTHON=pypy ./bin/spark-submit wordcount.py ``` The performance speed up will depend on work load (from 20% to 3000%). Here are some benchmarks: Job | CPython 2.7 | PyPy 2.3.1 | Speed up ------- | ------------ | ------------- | ------- Word Count | 41s | 15s | 2.7x Sort | 46s | 44s | 1.05x Stats | 174s | 3.6s | 48x Here is the code used for benchmark: ```python rdd = sc.textFile("text") def wordcount(): rdd.flatMap(lambda x:x.split('/'))\ .map(lambda x:(x,1)).reduceByKey(lambda x,y:x+y).collectAsMap() def sort(): rdd.sortBy(lambda x:x, 1).count() def stats(): sc.parallelize(range(1024), 20).flatMap(lambda x: xrange(5024)).stats() ``` Author: Davies Liu <davies.liu@gmail.com> Closes #2144 from davies/pypy and squashes the following commits: 9aed6c5 [Davies Liu] use protocol 2 in CloudPickle 4bc1f04 [Davies Liu] refactor b20ab3a [Davies Liu] pickle sys.stdout and stderr in portable way 3ca2351 [Davies Liu] Merge branch 'master' into pypy fae8b19 [Davies Liu] improve attrgetter, add tests 591f830 [Davies Liu] try to run tests with PyPy in run-tests c8d62ba [Davies Liu] cleanup f651fd0 [Davies Liu] fix tests using array with PyPy 1b98fb3 [Davies Liu] serialize itemgetter/attrgetter in portable ways 3c1dbfe [Davies Liu] Merge branch 'master' into pypy 42fb5fa [Davies Liu] Merge branch 'master' into pypy cb2d724 [Davies Liu] fix tests 9986692 [Davies Liu] Merge branch 'master' into pypy 25b4ca7 [Davies Liu] support PyPy
2014-09-12 21:42:50 -04:00
from pyspark.serializers import read_int, BatchedSerializer, MarshalSerializer, PickleSerializer, \
[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
CloudPickleSerializer, CompressedSerializer, UTF8Deserializer, NoOpSerializer
from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, ExternalSorter
from pyspark import shuffle
from pyspark.profiler import BasicProfiler
2013-01-16 22:15:14 -05:00
_have_scipy = False
_have_numpy = False
try:
import scipy.sparse
_have_scipy = True
except:
# No SciPy, but that's okay, we'll skip those tests
pass
try:
import numpy as np
_have_numpy = True
except:
# No NumPy, but that's okay, we'll skip those tests
pass
2013-01-16 22:15:14 -05:00
SPARK_HOME = os.environ["SPARK_HOME"]
class MergerTests(unittest.TestCase):
[SPARK-2538] [PySpark] Hash based disk spilling aggregation During aggregation in Python worker, if the memory usage is above spark.executor.memory, it will do disk spilling aggregation. It will split the aggregation into multiple stage, in each stage, it will partition the aggregated data by hash and dump them into disks. After all the data are aggregated, it will merge all the stages together (partition by partition). Author: Davies Liu <davies.liu@gmail.com> Closes #1460 from davies/spill and squashes the following commits: cad91bf [Davies Liu] call gc.collect() after data.clear() to release memory as much as possible. 37d71f7 [Davies Liu] balance the partitions 902f036 [Davies Liu] add shuffle.py into run-tests dcf03a9 [Davies Liu] fix memory_info() of psutil 67e6eba [Davies Liu] comment for MAX_TOTAL_PARTITIONS f6bd5d6 [Davies Liu] rollback next_limit() again, the performance difference is huge: e74b785 [Davies Liu] fix code style and change next_limit to memory_limit 400be01 [Davies Liu] address all the comments 6178844 [Davies Liu] refactor and improve docs fdd0a49 [Davies Liu] add long doc string for ExternalMerger 1a97ce4 [Davies Liu] limit used memory and size of objects in partitionBy() e6cc7f9 [Davies Liu] Merge branch 'master' into spill 3652583 [Davies Liu] address comments e78a0a0 [Davies Liu] fix style 24cec6a [Davies Liu] get local directory by SPARK_LOCAL_DIR 57ee7ef [Davies Liu] update docs 286aaff [Davies Liu] let spilled aggregation in Python configurable e9a40f6 [Davies Liu] recursive merger 6edbd1f [Davies Liu] Hash based disk spilling aggregation
2014-07-25 01:53:47 -04:00
def setUp(self):
self.N = 1 << 14
[SPARK-2538] [PySpark] Hash based disk spilling aggregation During aggregation in Python worker, if the memory usage is above spark.executor.memory, it will do disk spilling aggregation. It will split the aggregation into multiple stage, in each stage, it will partition the aggregated data by hash and dump them into disks. After all the data are aggregated, it will merge all the stages together (partition by partition). Author: Davies Liu <davies.liu@gmail.com> Closes #1460 from davies/spill and squashes the following commits: cad91bf [Davies Liu] call gc.collect() after data.clear() to release memory as much as possible. 37d71f7 [Davies Liu] balance the partitions 902f036 [Davies Liu] add shuffle.py into run-tests dcf03a9 [Davies Liu] fix memory_info() of psutil 67e6eba [Davies Liu] comment for MAX_TOTAL_PARTITIONS f6bd5d6 [Davies Liu] rollback next_limit() again, the performance difference is huge: e74b785 [Davies Liu] fix code style and change next_limit to memory_limit 400be01 [Davies Liu] address all the comments 6178844 [Davies Liu] refactor and improve docs fdd0a49 [Davies Liu] add long doc string for ExternalMerger 1a97ce4 [Davies Liu] limit used memory and size of objects in partitionBy() e6cc7f9 [Davies Liu] Merge branch 'master' into spill 3652583 [Davies Liu] address comments e78a0a0 [Davies Liu] fix style 24cec6a [Davies Liu] get local directory by SPARK_LOCAL_DIR 57ee7ef [Davies Liu] update docs 286aaff [Davies Liu] let spilled aggregation in Python configurable e9a40f6 [Davies Liu] recursive merger 6edbd1f [Davies Liu] Hash based disk spilling aggregation
2014-07-25 01:53:47 -04:00
self.l = [i for i in xrange(self.N)]
self.data = zip(self.l, self.l)
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
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)
[SPARK-2538] [PySpark] Hash based disk spilling aggregation During aggregation in Python worker, if the memory usage is above spark.executor.memory, it will do disk spilling aggregation. It will split the aggregation into multiple stage, in each stage, it will partition the aggregated data by hash and dump them into disks. After all the data are aggregated, it will merge all the stages together (partition by partition). Author: Davies Liu <davies.liu@gmail.com> Closes #1460 from davies/spill and squashes the following commits: cad91bf [Davies Liu] call gc.collect() after data.clear() to release memory as much as possible. 37d71f7 [Davies Liu] balance the partitions 902f036 [Davies Liu] add shuffle.py into run-tests dcf03a9 [Davies Liu] fix memory_info() of psutil 67e6eba [Davies Liu] comment for MAX_TOTAL_PARTITIONS f6bd5d6 [Davies Liu] rollback next_limit() again, the performance difference is huge: e74b785 [Davies Liu] fix code style and change next_limit to memory_limit 400be01 [Davies Liu] address all the comments 6178844 [Davies Liu] refactor and improve docs fdd0a49 [Davies Liu] add long doc string for ExternalMerger 1a97ce4 [Davies Liu] limit used memory and size of objects in partitionBy() e6cc7f9 [Davies Liu] Merge branch 'master' into spill 3652583 [Davies Liu] address comments e78a0a0 [Davies Liu] fix style 24cec6a [Davies Liu] get local directory by SPARK_LOCAL_DIR 57ee7ef [Davies Liu] update docs 286aaff [Davies Liu] let spilled aggregation in Python configurable e9a40f6 [Davies Liu] recursive merger 6edbd1f [Davies Liu] Hash based disk spilling aggregation
2014-07-25 01:53:47 -04:00
def test_in_memory(self):
m = InMemoryMerger(self.agg)
m.mergeValues(self.data)
self.assertEqual(sum(sum(v) for k, v in m.iteritems()),
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
sum(xrange(self.N)))
[SPARK-2538] [PySpark] Hash based disk spilling aggregation During aggregation in Python worker, if the memory usage is above spark.executor.memory, it will do disk spilling aggregation. It will split the aggregation into multiple stage, in each stage, it will partition the aggregated data by hash and dump them into disks. After all the data are aggregated, it will merge all the stages together (partition by partition). Author: Davies Liu <davies.liu@gmail.com> Closes #1460 from davies/spill and squashes the following commits: cad91bf [Davies Liu] call gc.collect() after data.clear() to release memory as much as possible. 37d71f7 [Davies Liu] balance the partitions 902f036 [Davies Liu] add shuffle.py into run-tests dcf03a9 [Davies Liu] fix memory_info() of psutil 67e6eba [Davies Liu] comment for MAX_TOTAL_PARTITIONS f6bd5d6 [Davies Liu] rollback next_limit() again, the performance difference is huge: e74b785 [Davies Liu] fix code style and change next_limit to memory_limit 400be01 [Davies Liu] address all the comments 6178844 [Davies Liu] refactor and improve docs fdd0a49 [Davies Liu] add long doc string for ExternalMerger 1a97ce4 [Davies Liu] limit used memory and size of objects in partitionBy() e6cc7f9 [Davies Liu] Merge branch 'master' into spill 3652583 [Davies Liu] address comments e78a0a0 [Davies Liu] fix style 24cec6a [Davies Liu] get local directory by SPARK_LOCAL_DIR 57ee7ef [Davies Liu] update docs 286aaff [Davies Liu] let spilled aggregation in Python configurable e9a40f6 [Davies Liu] recursive merger 6edbd1f [Davies Liu] Hash based disk spilling aggregation
2014-07-25 01:53:47 -04:00
m = InMemoryMerger(self.agg)
m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data))
self.assertEqual(sum(sum(v) for k, v in m.iteritems()),
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
sum(xrange(self.N)))
[SPARK-2538] [PySpark] Hash based disk spilling aggregation During aggregation in Python worker, if the memory usage is above spark.executor.memory, it will do disk spilling aggregation. It will split the aggregation into multiple stage, in each stage, it will partition the aggregated data by hash and dump them into disks. After all the data are aggregated, it will merge all the stages together (partition by partition). Author: Davies Liu <davies.liu@gmail.com> Closes #1460 from davies/spill and squashes the following commits: cad91bf [Davies Liu] call gc.collect() after data.clear() to release memory as much as possible. 37d71f7 [Davies Liu] balance the partitions 902f036 [Davies Liu] add shuffle.py into run-tests dcf03a9 [Davies Liu] fix memory_info() of psutil 67e6eba [Davies Liu] comment for MAX_TOTAL_PARTITIONS f6bd5d6 [Davies Liu] rollback next_limit() again, the performance difference is huge: e74b785 [Davies Liu] fix code style and change next_limit to memory_limit 400be01 [Davies Liu] address all the comments 6178844 [Davies Liu] refactor and improve docs fdd0a49 [Davies Liu] add long doc string for ExternalMerger 1a97ce4 [Davies Liu] limit used memory and size of objects in partitionBy() e6cc7f9 [Davies Liu] Merge branch 'master' into spill 3652583 [Davies Liu] address comments e78a0a0 [Davies Liu] fix style 24cec6a [Davies Liu] get local directory by SPARK_LOCAL_DIR 57ee7ef [Davies Liu] update docs 286aaff [Davies Liu] let spilled aggregation in Python configurable e9a40f6 [Davies Liu] recursive merger 6edbd1f [Davies Liu] Hash based disk spilling aggregation
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)
self.assertEqual(sum(sum(v) for k, v in m.iteritems()),
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
sum(xrange(self.N)))
[SPARK-2538] [PySpark] Hash based disk spilling aggregation During aggregation in Python worker, if the memory usage is above spark.executor.memory, it will do disk spilling aggregation. It will split the aggregation into multiple stage, in each stage, it will partition the aggregated data by hash and dump them into disks. After all the data are aggregated, it will merge all the stages together (partition by partition). Author: Davies Liu <davies.liu@gmail.com> Closes #1460 from davies/spill and squashes the following commits: cad91bf [Davies Liu] call gc.collect() after data.clear() to release memory as much as possible. 37d71f7 [Davies Liu] balance the partitions 902f036 [Davies Liu] add shuffle.py into run-tests dcf03a9 [Davies Liu] fix memory_info() of psutil 67e6eba [Davies Liu] comment for MAX_TOTAL_PARTITIONS f6bd5d6 [Davies Liu] rollback next_limit() again, the performance difference is huge: e74b785 [Davies Liu] fix code style and change next_limit to memory_limit 400be01 [Davies Liu] address all the comments 6178844 [Davies Liu] refactor and improve docs fdd0a49 [Davies Liu] add long doc string for ExternalMerger 1a97ce4 [Davies Liu] limit used memory and size of objects in partitionBy() e6cc7f9 [Davies Liu] Merge branch 'master' into spill 3652583 [Davies Liu] address comments e78a0a0 [Davies Liu] fix style 24cec6a [Davies Liu] get local directory by SPARK_LOCAL_DIR 57ee7ef [Davies Liu] update docs 286aaff [Davies Liu] let spilled aggregation in Python configurable e9a40f6 [Davies Liu] recursive merger 6edbd1f [Davies Liu] Hash based disk spilling aggregation
2014-07-25 01:53:47 -04:00
m = ExternalMerger(self.agg, 1000)
m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data))
self.assertEqual(m.spills, 0)
self.assertEqual(sum(sum(v) for k, v in m.iteritems()),
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
sum(xrange(self.N)))
[SPARK-2538] [PySpark] Hash based disk spilling aggregation During aggregation in Python worker, if the memory usage is above spark.executor.memory, it will do disk spilling aggregation. It will split the aggregation into multiple stage, in each stage, it will partition the aggregated data by hash and dump them into disks. After all the data are aggregated, it will merge all the stages together (partition by partition). Author: Davies Liu <davies.liu@gmail.com> Closes #1460 from davies/spill and squashes the following commits: cad91bf [Davies Liu] call gc.collect() after data.clear() to release memory as much as possible. 37d71f7 [Davies Liu] balance the partitions 902f036 [Davies Liu] add shuffle.py into run-tests dcf03a9 [Davies Liu] fix memory_info() of psutil 67e6eba [Davies Liu] comment for MAX_TOTAL_PARTITIONS f6bd5d6 [Davies Liu] rollback next_limit() again, the performance difference is huge: e74b785 [Davies Liu] fix code style and change next_limit to memory_limit 400be01 [Davies Liu] address all the comments 6178844 [Davies Liu] refactor and improve docs fdd0a49 [Davies Liu] add long doc string for ExternalMerger 1a97ce4 [Davies Liu] limit used memory and size of objects in partitionBy() e6cc7f9 [Davies Liu] Merge branch 'master' into spill 3652583 [Davies Liu] address comments e78a0a0 [Davies Liu] fix style 24cec6a [Davies Liu] get local directory by SPARK_LOCAL_DIR 57ee7ef [Davies Liu] update docs 286aaff [Davies Liu] let spilled aggregation in Python configurable e9a40f6 [Davies Liu] recursive merger 6edbd1f [Davies Liu] Hash based disk spilling aggregation
2014-07-25 01:53:47 -04:00
def test_medium_dataset(self):
m = ExternalMerger(self.agg, 10)
m.mergeValues(self.data)
self.assertTrue(m.spills >= 1)
self.assertEqual(sum(sum(v) for k, v in m.iteritems()),
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
sum(xrange(self.N)))
[SPARK-2538] [PySpark] Hash based disk spilling aggregation During aggregation in Python worker, if the memory usage is above spark.executor.memory, it will do disk spilling aggregation. It will split the aggregation into multiple stage, in each stage, it will partition the aggregated data by hash and dump them into disks. After all the data are aggregated, it will merge all the stages together (partition by partition). Author: Davies Liu <davies.liu@gmail.com> Closes #1460 from davies/spill and squashes the following commits: cad91bf [Davies Liu] call gc.collect() after data.clear() to release memory as much as possible. 37d71f7 [Davies Liu] balance the partitions 902f036 [Davies Liu] add shuffle.py into run-tests dcf03a9 [Davies Liu] fix memory_info() of psutil 67e6eba [Davies Liu] comment for MAX_TOTAL_PARTITIONS f6bd5d6 [Davies Liu] rollback next_limit() again, the performance difference is huge: e74b785 [Davies Liu] fix code style and change next_limit to memory_limit 400be01 [Davies Liu] address all the comments 6178844 [Davies Liu] refactor and improve docs fdd0a49 [Davies Liu] add long doc string for ExternalMerger 1a97ce4 [Davies Liu] limit used memory and size of objects in partitionBy() e6cc7f9 [Davies Liu] Merge branch 'master' into spill 3652583 [Davies Liu] address comments e78a0a0 [Davies Liu] fix style 24cec6a [Davies Liu] get local directory by SPARK_LOCAL_DIR 57ee7ef [Davies Liu] update docs 286aaff [Davies Liu] let spilled aggregation in Python configurable e9a40f6 [Davies Liu] recursive merger 6edbd1f [Davies Liu] Hash based disk spilling aggregation
2014-07-25 01:53:47 -04:00
m = ExternalMerger(self.agg, 10)
m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data * 3))
self.assertTrue(m.spills >= 1)
self.assertEqual(sum(sum(v) for k, v in m.iteritems()),
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
sum(xrange(self.N)) * 3)
[SPARK-2538] [PySpark] Hash based disk spilling aggregation During aggregation in Python worker, if the memory usage is above spark.executor.memory, it will do disk spilling aggregation. It will split the aggregation into multiple stage, in each stage, it will partition the aggregated data by hash and dump them into disks. After all the data are aggregated, it will merge all the stages together (partition by partition). Author: Davies Liu <davies.liu@gmail.com> Closes #1460 from davies/spill and squashes the following commits: cad91bf [Davies Liu] call gc.collect() after data.clear() to release memory as much as possible. 37d71f7 [Davies Liu] balance the partitions 902f036 [Davies Liu] add shuffle.py into run-tests dcf03a9 [Davies Liu] fix memory_info() of psutil 67e6eba [Davies Liu] comment for MAX_TOTAL_PARTITIONS f6bd5d6 [Davies Liu] rollback next_limit() again, the performance difference is huge: e74b785 [Davies Liu] fix code style and change next_limit to memory_limit 400be01 [Davies Liu] address all the comments 6178844 [Davies Liu] refactor and improve docs fdd0a49 [Davies Liu] add long doc string for ExternalMerger 1a97ce4 [Davies Liu] limit used memory and size of objects in partitionBy() e6cc7f9 [Davies Liu] Merge branch 'master' into spill 3652583 [Davies Liu] address comments e78a0a0 [Davies Liu] fix style 24cec6a [Davies Liu] get local directory by SPARK_LOCAL_DIR 57ee7ef [Davies Liu] update docs 286aaff [Davies Liu] let spilled aggregation in Python configurable e9a40f6 [Davies Liu] recursive merger 6edbd1f [Davies Liu] Hash based disk spilling aggregation
2014-07-25 01:53:47 -04:00
def test_huge_dataset(self):
m = ExternalMerger(self.agg, 10, partitions=3)
[SPARK-2538] [PySpark] Hash based disk spilling aggregation During aggregation in Python worker, if the memory usage is above spark.executor.memory, it will do disk spilling aggregation. It will split the aggregation into multiple stage, in each stage, it will partition the aggregated data by hash and dump them into disks. After all the data are aggregated, it will merge all the stages together (partition by partition). Author: Davies Liu <davies.liu@gmail.com> Closes #1460 from davies/spill and squashes the following commits: cad91bf [Davies Liu] call gc.collect() after data.clear() to release memory as much as possible. 37d71f7 [Davies Liu] balance the partitions 902f036 [Davies Liu] add shuffle.py into run-tests dcf03a9 [Davies Liu] fix memory_info() of psutil 67e6eba [Davies Liu] comment for MAX_TOTAL_PARTITIONS f6bd5d6 [Davies Liu] rollback next_limit() again, the performance difference is huge: e74b785 [Davies Liu] fix code style and change next_limit to memory_limit 400be01 [Davies Liu] address all the comments 6178844 [Davies Liu] refactor and improve docs fdd0a49 [Davies Liu] add long doc string for ExternalMerger 1a97ce4 [Davies Liu] limit used memory and size of objects in partitionBy() e6cc7f9 [Davies Liu] Merge branch 'master' into spill 3652583 [Davies Liu] address comments e78a0a0 [Davies Liu] fix style 24cec6a [Davies Liu] get local directory by SPARK_LOCAL_DIR 57ee7ef [Davies Liu] update docs 286aaff [Davies Liu] let spilled aggregation in Python configurable e9a40f6 [Davies Liu] recursive merger 6edbd1f [Davies Liu] Hash based disk spilling aggregation
2014-07-25 01:53:47 -04:00
m.mergeCombiners(map(lambda (k, v): (k, [str(v)]), self.data * 10))
self.assertTrue(m.spills >= 1)
self.assertEqual(sum(len(v) for k, v in m._recursive_merged_items(0)),
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
self.N * 10)
[SPARK-2538] [PySpark] Hash based disk spilling aggregation During aggregation in Python worker, if the memory usage is above spark.executor.memory, it will do disk spilling aggregation. It will split the aggregation into multiple stage, in each stage, it will partition the aggregated data by hash and dump them into disks. After all the data are aggregated, it will merge all the stages together (partition by partition). Author: Davies Liu <davies.liu@gmail.com> Closes #1460 from davies/spill and squashes the following commits: cad91bf [Davies Liu] call gc.collect() after data.clear() to release memory as much as possible. 37d71f7 [Davies Liu] balance the partitions 902f036 [Davies Liu] add shuffle.py into run-tests dcf03a9 [Davies Liu] fix memory_info() of psutil 67e6eba [Davies Liu] comment for MAX_TOTAL_PARTITIONS f6bd5d6 [Davies Liu] rollback next_limit() again, the performance difference is huge: e74b785 [Davies Liu] fix code style and change next_limit to memory_limit 400be01 [Davies Liu] address all the comments 6178844 [Davies Liu] refactor and improve docs fdd0a49 [Davies Liu] add long doc string for ExternalMerger 1a97ce4 [Davies Liu] limit used memory and size of objects in partitionBy() e6cc7f9 [Davies Liu] Merge branch 'master' into spill 3652583 [Davies Liu] address comments e78a0a0 [Davies Liu] fix style 24cec6a [Davies Liu] get local directory by SPARK_LOCAL_DIR 57ee7ef [Davies Liu] update docs 286aaff [Davies Liu] let spilled aggregation in Python configurable e9a40f6 [Davies Liu] recursive merger 6edbd1f [Davies Liu] Hash based disk spilling aggregation
2014-07-25 01:53:47 -04:00
m._cleanup()
class SorterTests(unittest.TestCase):
def test_in_memory_sort(self):
l = range(1024)
random.shuffle(l)
sorter = ExternalSorter(1024)
self.assertEquals(sorted(l), list(sorter.sorted(l)))
self.assertEquals(sorted(l, reverse=True), list(sorter.sorted(l, reverse=True)))
self.assertEquals(sorted(l, key=lambda x: -x), list(sorter.sorted(l, key=lambda x: -x)))
self.assertEquals(sorted(l, key=lambda x: -x, reverse=True),
list(sorter.sorted(l, key=lambda x: -x, reverse=True)))
def test_external_sort(self):
l = range(1024)
random.shuffle(l)
sorter = ExternalSorter(1)
self.assertEquals(sorted(l), list(sorter.sorted(l)))
self.assertGreater(shuffle.DiskBytesSpilled, 0)
last = shuffle.DiskBytesSpilled
self.assertEquals(sorted(l, reverse=True), list(sorter.sorted(l, reverse=True)))
self.assertGreater(shuffle.DiskBytesSpilled, last)
last = shuffle.DiskBytesSpilled
self.assertEquals(sorted(l, key=lambda x: -x), list(sorter.sorted(l, key=lambda x: -x)))
self.assertGreater(shuffle.DiskBytesSpilled, last)
last = shuffle.DiskBytesSpilled
self.assertEquals(sorted(l, key=lambda x: -x, reverse=True),
list(sorter.sorted(l, key=lambda x: -x, reverse=True)))
self.assertGreater(shuffle.DiskBytesSpilled, last)
def test_external_sort_in_rdd(self):
conf = SparkConf().set("spark.python.worker.memory", "1m")
sc = SparkContext(conf=conf)
l = range(10240)
random.shuffle(l)
rdd = sc.parallelize(l, 10)
self.assertEquals(sorted(l), rdd.sortBy(lambda x: x).collect())
sc.stop()
class SerializationTestCase(unittest.TestCase):
def test_namedtuple(self):
from collections import namedtuple
from cPickle import dumps, loads
P = namedtuple("P", "x y")
p1 = P(1, 3)
p2 = loads(dumps(p1, 2))
self.assertEquals(p1, p2)
[SPARK-3094] [PySpark] compatitable with PyPy After this patch, we can run PySpark in PyPy (testing with PyPy 2.3.1 in Mac 10.9), for example: ``` PYSPARK_PYTHON=pypy ./bin/spark-submit wordcount.py ``` The performance speed up will depend on work load (from 20% to 3000%). Here are some benchmarks: Job | CPython 2.7 | PyPy 2.3.1 | Speed up ------- | ------------ | ------------- | ------- Word Count | 41s | 15s | 2.7x Sort | 46s | 44s | 1.05x Stats | 174s | 3.6s | 48x Here is the code used for benchmark: ```python rdd = sc.textFile("text") def wordcount(): rdd.flatMap(lambda x:x.split('/'))\ .map(lambda x:(x,1)).reduceByKey(lambda x,y:x+y).collectAsMap() def sort(): rdd.sortBy(lambda x:x, 1).count() def stats(): sc.parallelize(range(1024), 20).flatMap(lambda x: xrange(5024)).stats() ``` Author: Davies Liu <davies.liu@gmail.com> Closes #2144 from davies/pypy and squashes the following commits: 9aed6c5 [Davies Liu] use protocol 2 in CloudPickle 4bc1f04 [Davies Liu] refactor b20ab3a [Davies Liu] pickle sys.stdout and stderr in portable way 3ca2351 [Davies Liu] Merge branch 'master' into pypy fae8b19 [Davies Liu] improve attrgetter, add tests 591f830 [Davies Liu] try to run tests with PyPy in run-tests c8d62ba [Davies Liu] cleanup f651fd0 [Davies Liu] fix tests using array with PyPy 1b98fb3 [Davies Liu] serialize itemgetter/attrgetter in portable ways 3c1dbfe [Davies Liu] Merge branch 'master' into pypy 42fb5fa [Davies Liu] Merge branch 'master' into pypy cb2d724 [Davies Liu] fix tests 9986692 [Davies Liu] Merge branch 'master' into pypy 25b4ca7 [Davies Liu] support PyPy
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))
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
def test_pickling_file_handles(self):
[SPARK-3094] [PySpark] compatitable with PyPy After this patch, we can run PySpark in PyPy (testing with PyPy 2.3.1 in Mac 10.9), for example: ``` PYSPARK_PYTHON=pypy ./bin/spark-submit wordcount.py ``` The performance speed up will depend on work load (from 20% to 3000%). Here are some benchmarks: Job | CPython 2.7 | PyPy 2.3.1 | Speed up ------- | ------------ | ------------- | ------- Word Count | 41s | 15s | 2.7x Sort | 46s | 44s | 1.05x Stats | 174s | 3.6s | 48x Here is the code used for benchmark: ```python rdd = sc.textFile("text") def wordcount(): rdd.flatMap(lambda x:x.split('/'))\ .map(lambda x:(x,1)).reduceByKey(lambda x,y:x+y).collectAsMap() def sort(): rdd.sortBy(lambda x:x, 1).count() def stats(): sc.parallelize(range(1024), 20).flatMap(lambda x: xrange(5024)).stats() ``` Author: Davies Liu <davies.liu@gmail.com> Closes #2144 from davies/pypy and squashes the following commits: 9aed6c5 [Davies Liu] use protocol 2 in CloudPickle 4bc1f04 [Davies Liu] refactor b20ab3a [Davies Liu] pickle sys.stdout and stderr in portable way 3ca2351 [Davies Liu] Merge branch 'master' into pypy fae8b19 [Davies Liu] improve attrgetter, add tests 591f830 [Davies Liu] try to run tests with PyPy in run-tests c8d62ba [Davies Liu] cleanup f651fd0 [Davies Liu] fix tests using array with PyPy 1b98fb3 [Davies Liu] serialize itemgetter/attrgetter in portable ways 3c1dbfe [Davies Liu] Merge branch 'master' into pypy 42fb5fa [Davies Liu] Merge branch 'master' into pypy cb2d724 [Davies Liu] fix tests 9986692 [Davies Liu] Merge branch 'master' into pypy 25b4ca7 [Davies Liu] support PyPy
2014-09-12 21:42:50 -04:00
ser = CloudPickleSerializer()
out1 = sys.stderr
[SPARK-3094] [PySpark] compatitable with PyPy After this patch, we can run PySpark in PyPy (testing with PyPy 2.3.1 in Mac 10.9), for example: ``` PYSPARK_PYTHON=pypy ./bin/spark-submit wordcount.py ``` The performance speed up will depend on work load (from 20% to 3000%). Here are some benchmarks: Job | CPython 2.7 | PyPy 2.3.1 | Speed up ------- | ------------ | ------------- | ------- Word Count | 41s | 15s | 2.7x Sort | 46s | 44s | 1.05x Stats | 174s | 3.6s | 48x Here is the code used for benchmark: ```python rdd = sc.textFile("text") def wordcount(): rdd.flatMap(lambda x:x.split('/'))\ .map(lambda x:(x,1)).reduceByKey(lambda x,y:x+y).collectAsMap() def sort(): rdd.sortBy(lambda x:x, 1).count() def stats(): sc.parallelize(range(1024), 20).flatMap(lambda x: xrange(5024)).stats() ``` Author: Davies Liu <davies.liu@gmail.com> Closes #2144 from davies/pypy and squashes the following commits: 9aed6c5 [Davies Liu] use protocol 2 in CloudPickle 4bc1f04 [Davies Liu] refactor b20ab3a [Davies Liu] pickle sys.stdout and stderr in portable way 3ca2351 [Davies Liu] Merge branch 'master' into pypy fae8b19 [Davies Liu] improve attrgetter, add tests 591f830 [Davies Liu] try to run tests with PyPy in run-tests c8d62ba [Davies Liu] cleanup f651fd0 [Davies Liu] fix tests using array with PyPy 1b98fb3 [Davies Liu] serialize itemgetter/attrgetter in portable ways 3c1dbfe [Davies Liu] Merge branch 'master' into pypy 42fb5fa [Davies Liu] Merge branch 'master' into pypy cb2d724 [Davies Liu] fix tests 9986692 [Davies Liu] Merge branch 'master' into pypy 25b4ca7 [Davies Liu] support PyPy
2014-09-12 21:42:50 -04:00
out2 = ser.loads(ser.dumps(out1))
self.assertEquals(out1, out2)
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)
self.assertTrue("exit" in foo.func_code.co_names)
ser.dumps(foo)
[SPARK-4548] []SPARK-4517] improve performance of python broadcast Re-implement the Python broadcast using file: 1) serialize the python object using cPickle, write into disks. 2) Create a wrapper in JVM (for the dumped file), it read data from during serialization 3) Using TorrentBroadcast or HttpBroadcast to transfer the data (compressed) into executors 4) During deserialization, writing the data into disk. 5) Passing the path into Python worker, read data from disk and unpickle it into python object, until the first access. It fixes the performance regression introduced in #2659, has similar performance as 1.1, but support object larger than 2G, also improve the memory efficiency (only one compressed copy in driver and executor). Testing with a 500M broadcast and 4 tasks (excluding the benefit from reused worker in 1.2): name | 1.1 | 1.2 with this patch | improvement ---------|--------|---------|-------- python-broadcast-w-bytes | 25.20 | 9.33 | 170.13% | python-broadcast-w-set | 4.13 | 4.50 | -8.35% | Testing with 100 tasks (16 CPUs): name | 1.1 | 1.2 with this patch | improvement ---------|--------|---------|-------- python-broadcast-w-bytes | 38.16 | 8.40 | 353.98% python-broadcast-w-set | 23.29 | 9.59 | 142.80% Author: Davies Liu <davies@databricks.com> Closes #3417 from davies/pybroadcast and squashes the following commits: 50a58e0 [Davies Liu] address comments b98de1d [Davies Liu] disable gc while unpickle e5ee6b9 [Davies Liu] support large string 09303b8 [Davies Liu] read all data into memory dde02dd [Davies Liu] improve performance of python broadcast
2014-11-24 20:17:03 -05:00
def test_compressed_serializer(self):
ser = CompressedSerializer(PickleSerializer())
2014-11-18 19:17:51 -05:00
from StringIO import StringIO
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)
[SPARK-4548] []SPARK-4517] improve performance of python broadcast Re-implement the Python broadcast using file: 1) serialize the python object using cPickle, write into disks. 2) Create a wrapper in JVM (for the dumped file), it read data from during serialization 3) Using TorrentBroadcast or HttpBroadcast to transfer the data (compressed) into executors 4) During deserialization, writing the data into disk. 5) Passing the path into Python worker, read data from disk and unpickle it into python object, until the first access. It fixes the performance regression introduced in #2659, has similar performance as 1.1, but support object larger than 2G, also improve the memory efficiency (only one compressed copy in driver and executor). Testing with a 500M broadcast and 4 tasks (excluding the benefit from reused worker in 1.2): name | 1.1 | 1.2 with this patch | improvement ---------|--------|---------|-------- python-broadcast-w-bytes | 25.20 | 9.33 | 170.13% | python-broadcast-w-set | 4.13 | 4.50 | -8.35% | Testing with 100 tasks (16 CPUs): name | 1.1 | 1.2 with this patch | improvement ---------|--------|---------|-------- python-broadcast-w-bytes | 38.16 | 8.40 | 353.98% python-broadcast-w-set | 23.29 | 9.59 | 142.80% Author: Davies Liu <davies@databricks.com> Closes #3417 from davies/pybroadcast and squashes the following commits: 50a58e0 [Davies Liu] address comments b98de1d [Davies Liu] disable gc while unpickle e5ee6b9 [Davies Liu] support large string 09303b8 [Davies Liu] read all data into memory dde02dd [Davies Liu] improve performance of python broadcast
2014-11-24 20:17:03 -05:00
self.assertEqual(["abc", u"123", range(5)] + range(1000), list(ser.load_stream(io)))
2014-11-18 19:17:51 -05:00
class PySparkTestCase(unittest.TestCase):
2013-01-16 22:15:14 -05:00
def setUp(self):
self._old_sys_path = list(sys.path)
class_name = self.__class__.__name__
self.sc = SparkContext('local[4]', class_name)
2013-01-16 22:15:14 -05:00
def tearDown(self):
self.sc.stop()
sys.path = self._old_sys_path
[SPARK-2470] PEP8 fixes to PySpark This pull request aims to resolve all outstanding PEP8 violations in PySpark. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1505 from nchammas/master and squashes the following commits: 98171af [Nicholas Chammas] [SPARK-2470] revert PEP 8 fixes to cloudpickle cba7768 [Nicholas Chammas] [SPARK-2470] wrap expression list in parentheses e178dbe [Nicholas Chammas] [SPARK-2470] style - change position of line break 9127d2b [Nicholas Chammas] [SPARK-2470] wrap expression lists in parentheses 22132a4 [Nicholas Chammas] [SPARK-2470] wrap conditionals in parentheses 24639bc [Nicholas Chammas] [SPARK-2470] fix whitespace for doctest 7d557b7 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to tests.py 8f8e4c0 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to storagelevel.py b3b96cf [Nicholas Chammas] [SPARK-2470] PEP8 fixes to statcounter.py d644477 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to worker.py aa3a7b6 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to sql.py 1916859 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to shell.py 95d1d95 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to serializers.py a0fec2e [Nicholas Chammas] [SPARK-2470] PEP8 fixes to mllib c85e1e5 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to join.py d14f2f1 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to __init__.py 81fcb20 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to resultiterable.py 1bde265 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to java_gateway.py 7fc849c [Nicholas Chammas] [SPARK-2470] PEP8 fixes to daemon.py ca2d28b [Nicholas Chammas] [SPARK-2470] PEP8 fixes to context.py f4e0039 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to conf.py a6d5e4b [Nicholas Chammas] [SPARK-2470] PEP8 fixes to cloudpickle.py f0a7ebf [Nicholas Chammas] [SPARK-2470] PEP8 fixes to rddsampler.py 4dd148f [nchammas] Merge pull request #5 from apache/master f7e4581 [Nicholas Chammas] unrelated pep8 fix a36eed0 [Nicholas Chammas] name ec2 instances and security groups consistently de7292a [nchammas] Merge pull request #4 from apache/master 2e4fe00 [nchammas] Merge pull request #3 from apache/master 89fde08 [nchammas] Merge pull request #2 from apache/master 69f6e22 [Nicholas Chammas] PEP8 fixes 2627247 [Nicholas Chammas] broke up lines before they hit 100 chars 6544b7e [Nicholas Chammas] [SPARK-2065] give launched instances names 69da6cf [nchammas] Merge pull request #1 from apache/master
2014-07-22 01:30:53 -04:00
class ReusedPySparkTestCase(unittest.TestCase):
@classmethod
def setUpClass(cls):
cls.sc = SparkContext('local[4]', cls.__name__)
@classmethod
def tearDownClass(cls):
cls.sc.stop()
class CheckpointTests(ReusedPySparkTestCase):
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)
os.unlink(self.checkpointDir.name)
self.sc.setCheckpointDir(self.checkpointDir.name)
def tearDown(self):
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)
self.assertEqual("file:" + self.checkpointDir.name,
os.path.dirname(os.path.dirname(flatMappedRDD.getCheckpointFile())))
2013-01-16 22:15:14 -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)
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)
recovered = self.sc._checkpointFile(flatMappedRDD.getCheckpointFile(),
flatMappedRDD._jrdd_deserializer)
self.assertEquals([1, 2, 3, 4], recovered.collect())
2013-01-16 22:15:14 -05:00
class AddFileTests(PySparkTestCase):
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:
# disable logging in log4j temporarily
log4j = self.sc._jvm.org.apache.log4j
old_level = log4j.LogManager.getRootLogger().getLevel()
log4j.LogManager.getRootLogger().setLevel(log4j.Level.FATAL)
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
def func(x):
from userlibrary import UserClass
return UserClass().hello()
self.assertRaises(Exception,
self.sc.parallelize(range(2)).map(func).first)
log4j.LogManager.getRootLogger().setLevel(old_level)
# 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)
def test_add_file_locally(self):
path = os.path.join(SPARK_HOME, "python/test_support/hello.txt")
self.sc.addFile(path)
download_path = SparkFiles.get("hello.txt")
self.assertNotEqual(path, download_path)
with open(download_path) as test_file:
self.assertEquals("Hello World!\n", test_file.readline())
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")
self.sc.addFile(path)
from userlibrary import UserClass
self.assertEqual("Hello World!", UserClass().hello())
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)
path = os.path.join(SPARK_HOME, "python/test_support/userlib-0.1-py2.7.egg")
self.sc.addPyFile(path)
from userlib import UserClass
self.assertEqual("Hello World from inside a package!", UserClass().hello())
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())
class RDDTests(ReusedPySparkTestCase):
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())
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)
tempFile.close()
data.saveAsTextFile(tempFile.name)
raw_contents = ''.join(input(glob(tempFile.name + "/part-0000*")))
self.assertEqual(x, unicode(raw_contents.strip(), "utf-8"))
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)
raw_contents = ''.join(input(glob(tempFile.name + "/part-0000*")))
self.assertEqual(x, unicode(raw_contents.strip(), "utf-8"))
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)
result = cart.map(lambda (x, y): x + y).collect()
def test_transforming_pickle_file(self):
# Regression test for SPARK-2601
data = self.sc.parallelize(["Hello", "World!"])
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()
def test_cartesian_on_textfile(self):
# Regression test for
path = os.path.join(SPARK_HOME, "python/test_support/hello.txt")
a = self.sc.textFile(path)
result = a.cartesian(a).collect()
(x, y) = result[0]
self.assertEqual("Hello World!", x.strip())
self.assertEqual("Hello World!", y.strip())
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)
tempFile.write("Hello World!")
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)
self.assertRaises(Exception, lambda: filtered_data.count())
def test_sampling_default_seed(self):
# Test for SPARK-3995 (default seed setting)
data = self.sc.parallelize(range(1000), 1)
subset = data.takeSample(False, 10)
self.assertEqual(len(subset), 10)
2014-11-18 19:17:51 -05:00
def test_aggregate_by_key(self):
data = self.sc.parallelize([(1, 1), (1, 1), (3, 2), (5, 1), (5, 3)], 2)
[SPARK-2470] PEP8 fixes to PySpark This pull request aims to resolve all outstanding PEP8 violations in PySpark. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1505 from nchammas/master and squashes the following commits: 98171af [Nicholas Chammas] [SPARK-2470] revert PEP 8 fixes to cloudpickle cba7768 [Nicholas Chammas] [SPARK-2470] wrap expression list in parentheses e178dbe [Nicholas Chammas] [SPARK-2470] style - change position of line break 9127d2b [Nicholas Chammas] [SPARK-2470] wrap expression lists in parentheses 22132a4 [Nicholas Chammas] [SPARK-2470] wrap conditionals in parentheses 24639bc [Nicholas Chammas] [SPARK-2470] fix whitespace for doctest 7d557b7 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to tests.py 8f8e4c0 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to storagelevel.py b3b96cf [Nicholas Chammas] [SPARK-2470] PEP8 fixes to statcounter.py d644477 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to worker.py aa3a7b6 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to sql.py 1916859 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to shell.py 95d1d95 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to serializers.py a0fec2e [Nicholas Chammas] [SPARK-2470] PEP8 fixes to mllib c85e1e5 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to join.py d14f2f1 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to __init__.py 81fcb20 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to resultiterable.py 1bde265 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to java_gateway.py 7fc849c [Nicholas Chammas] [SPARK-2470] PEP8 fixes to daemon.py ca2d28b [Nicholas Chammas] [SPARK-2470] PEP8 fixes to context.py f4e0039 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to conf.py a6d5e4b [Nicholas Chammas] [SPARK-2470] PEP8 fixes to cloudpickle.py f0a7ebf [Nicholas Chammas] [SPARK-2470] PEP8 fixes to rddsampler.py 4dd148f [nchammas] Merge pull request #5 from apache/master f7e4581 [Nicholas Chammas] unrelated pep8 fix a36eed0 [Nicholas Chammas] name ec2 instances and security groups consistently de7292a [nchammas] Merge pull request #4 from apache/master 2e4fe00 [nchammas] Merge pull request #3 from apache/master 89fde08 [nchammas] Merge pull request #2 from apache/master 69f6e22 [Nicholas Chammas] PEP8 fixes 2627247 [Nicholas Chammas] broke up lines before they hit 100 chars 6544b7e [Nicholas Chammas] [SPARK-2065] give launched instances names 69da6cf [nchammas] Merge pull request #1 from apache/master
2014-07-22 01:30:53 -04:00
def seqOp(x, y):
x.add(y)
return x
def combOp(x, y):
x |= y
return x
[SPARK-2470] PEP8 fixes to PySpark This pull request aims to resolve all outstanding PEP8 violations in PySpark. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1505 from nchammas/master and squashes the following commits: 98171af [Nicholas Chammas] [SPARK-2470] revert PEP 8 fixes to cloudpickle cba7768 [Nicholas Chammas] [SPARK-2470] wrap expression list in parentheses e178dbe [Nicholas Chammas] [SPARK-2470] style - change position of line break 9127d2b [Nicholas Chammas] [SPARK-2470] wrap expression lists in parentheses 22132a4 [Nicholas Chammas] [SPARK-2470] wrap conditionals in parentheses 24639bc [Nicholas Chammas] [SPARK-2470] fix whitespace for doctest 7d557b7 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to tests.py 8f8e4c0 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to storagelevel.py b3b96cf [Nicholas Chammas] [SPARK-2470] PEP8 fixes to statcounter.py d644477 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to worker.py aa3a7b6 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to sql.py 1916859 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to shell.py 95d1d95 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to serializers.py a0fec2e [Nicholas Chammas] [SPARK-2470] PEP8 fixes to mllib c85e1e5 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to join.py d14f2f1 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to __init__.py 81fcb20 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to resultiterable.py 1bde265 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to java_gateway.py 7fc849c [Nicholas Chammas] [SPARK-2470] PEP8 fixes to daemon.py ca2d28b [Nicholas Chammas] [SPARK-2470] PEP8 fixes to context.py f4e0039 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to conf.py a6d5e4b [Nicholas Chammas] [SPARK-2470] PEP8 fixes to cloudpickle.py f0a7ebf [Nicholas Chammas] [SPARK-2470] PEP8 fixes to rddsampler.py 4dd148f [nchammas] Merge pull request #5 from apache/master f7e4581 [Nicholas Chammas] unrelated pep8 fix a36eed0 [Nicholas Chammas] name ec2 instances and security groups consistently de7292a [nchammas] Merge pull request #4 from apache/master 2e4fe00 [nchammas] Merge pull request #3 from apache/master 89fde08 [nchammas] Merge pull request #2 from apache/master 69f6e22 [Nicholas Chammas] PEP8 fixes 2627247 [Nicholas Chammas] broke up lines before they hit 100 chars 6544b7e [Nicholas Chammas] [SPARK-2065] give launched instances names 69da6cf [nchammas] Merge pull request #1 from apache/master
2014-07-22 01:30:53 -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])
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())
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])
self.assertEquals([jon, jane], theDoes.collect())
def test_large_broadcast(self):
N = 100000
data = [[float(i) for i in range(300)] for i in range(N)]
bdata = self.sc.broadcast(data) # 270MB
m = self.sc.parallelize(range(1), 1).map(lambda x: len(bdata.value)).sum()
self.assertEquals(N, m)
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
r = range(1 << 15)
random.shuffle(r)
s = str(r)
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)
s = str(r)
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)
def test_large_closure(self):
N = 1000000
data = [float(i) for i in xrange(N)]
rdd = self.sc.parallelize(range(1), 1).map(lambda x: len(data))
self.assertEquals(N, rdd.first())
self.assertTrue(rdd._broadcast is not None)
rdd = self.sc.parallelize(range(1), 1).map(lambda x: 1)
self.assertEqual(1, rdd.first())
self.assertTrue(rdd._broadcast is None)
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)])
# regression test for SPARK-4841
path = os.path.join(SPARK_HOME, "python/test_support/hello.txt")
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())
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))
# 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.assertEquals(a.count(), b.count())
self.assertRaises(Exception, lambda: a.zip(b).count())
def test_count_approx_distinct(self):
rdd = self.sc.parallelize(range(1000))
self.assertTrue(950 < rdd.countApproxDistinct(0.04) < 1050)
self.assertTrue(950 < rdd.map(float).countApproxDistinct(0.04) < 1050)
self.assertTrue(950 < rdd.map(str).countApproxDistinct(0.04) < 1050)
self.assertTrue(950 < rdd.map(lambda x: (x, -x)).countApproxDistinct(0.04) < 1050)
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))
self.assertRaises(ValueError, lambda: rdd.countApproxDistinct(0.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
def test_histogram(self):
# empty
rdd = self.sc.parallelize([])
self.assertEquals([0], rdd.histogram([0, 10])[1])
self.assertEquals([0, 0], rdd.histogram([0, 4, 10])[1])
self.assertRaises(ValueError, lambda: rdd.histogram(1))
# out of range
rdd = self.sc.parallelize([10.01, -0.01])
self.assertEquals([0], rdd.histogram([0, 10])[1])
self.assertEquals([0, 0], rdd.histogram((0, 4, 10))[1])
# in range with one bucket
rdd = self.sc.parallelize(range(1, 5))
self.assertEquals([4], rdd.histogram([0, 10])[1])
self.assertEquals([3, 1], rdd.histogram([0, 4, 10])[1])
# in range with one bucket exact match
self.assertEquals([4], rdd.histogram([1, 4])[1])
# out of range with two buckets
rdd = self.sc.parallelize([10.01, -0.01])
self.assertEquals([0, 0], rdd.histogram([0, 5, 10])[1])
# out of range with two uneven buckets
rdd = self.sc.parallelize([10.01, -0.01])
self.assertEquals([0, 0], rdd.histogram([0, 4, 10])[1])
# in range with two buckets
rdd = self.sc.parallelize([1, 2, 3, 5, 6])
self.assertEquals([3, 2], rdd.histogram([0, 5, 10])[1])
# in range with two bucket and None
rdd = self.sc.parallelize([1, 2, 3, 5, 6, None, float('nan')])
self.assertEquals([3, 2], rdd.histogram([0, 5, 10])[1])
# in range with two uneven buckets
rdd = self.sc.parallelize([1, 2, 3, 5, 6])
self.assertEquals([3, 2], rdd.histogram([0, 5, 11])[1])
# mixed range with two uneven buckets
rdd = self.sc.parallelize([-0.01, 0.0, 1, 2, 3, 5, 6, 11.0, 11.01])
self.assertEquals([4, 3], rdd.histogram([0, 5, 11])[1])
# 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])
self.assertEquals([4, 2, 1, 3], rdd.histogram([0.0, 5.0, 11.0, 12.0, 200.0])[1])
# 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')])
self.assertEquals([4, 2, 1, 3], rdd.histogram([0.0, 5.0, 11.0, 12.0, 200.0])[1])
# out of range with infinite buckets
rdd = self.sc.parallelize([10.01, -0.01, float('nan'), float("inf")])
self.assertEquals([1, 2], rdd.histogram([float('-inf'), 0, float('inf')])[1])
# 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))
self.assertEquals(([1, 4], [4]), rdd.histogram(1))
# without buckets single element
rdd = self.sc.parallelize([1])
self.assertEquals(([1, 1], [1]), rdd.histogram(1))
# without bucket no range
rdd = self.sc.parallelize([1] * 4)
self.assertEquals(([1, 1], [4]), rdd.histogram(1))
# without buckets basic two
rdd = self.sc.parallelize(range(1, 5))
self.assertEquals(([1, 2.5, 4], [2, 2]), rdd.histogram(2))
# 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]
self.assertEquals((buckets, hist), rdd.histogram(5))
# 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)
self.assertEquals([2, 2], rdd.histogram(["a", "b", "c"])[1])
self.assertEquals((["ab", "ef"], [5]), rdd.histogram(1))
self.assertRaises(TypeError, lambda: rdd.histogram(2))
# mixed RDD
rdd = self.sc.parallelize([1, 4, "ab", "ac", "b"], 2)
self.assertEquals([1, 1], rdd.histogram([0, 4, 10])[1])
self.assertEquals([2, 1], rdd.histogram(["a", "b", "c"])[1])
self.assertEquals(([1, "b"], [5]), rdd.histogram(1))
self.assertRaises(TypeError, lambda: rdd.histogram(2))
def test_repartitionAndSortWithinPartitions(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)
partitions = repartitioned.glom().collect()
self.assertEquals(partitions[0], [(0, 5), (0, 8), (2, 6)])
self.assertEquals(partitions[1], [(1, 3), (3, 8), (3, 8)])
def test_distinct(self):
rdd = self.sc.parallelize((1, 2, 3)*10, 10)
self.assertEquals(rdd.getNumPartitions(), 10)
self.assertEquals(rdd.distinct().count(), 3)
result = rdd.distinct(5)
self.assertEquals(result.getNumPartitions(), 5)
self.assertEquals(result.count(), 3)
def test_sort_on_empty_rdd(self):
self.assertEqual([], self.sc.parallelize(zip([], [])).sortByKey().collect())
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())
self.assertEqual(["a", None, "b"], rdd.collect())
[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'})
]
from pyspark.rdd import RDD
data_rdd = self.sc.parallelize(data)
data_java_rdd = data_rdd._to_java_object_rdd()
data_python_rdd = self.sc._jvm.SerDe.javaToPython(data_java_rdd)
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()
data_python_rdd = self.sc._jvm.SerDe.javaToPython(data_java_rdd)
converted_rdd = RDD(data_python_rdd, self.sc)
self.assertEqual(2, converted_rdd.count())
[SPARK-2470] PEP8 fixes to PySpark This pull request aims to resolve all outstanding PEP8 violations in PySpark. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1505 from nchammas/master and squashes the following commits: 98171af [Nicholas Chammas] [SPARK-2470] revert PEP 8 fixes to cloudpickle cba7768 [Nicholas Chammas] [SPARK-2470] wrap expression list in parentheses e178dbe [Nicholas Chammas] [SPARK-2470] style - change position of line break 9127d2b [Nicholas Chammas] [SPARK-2470] wrap expression lists in parentheses 22132a4 [Nicholas Chammas] [SPARK-2470] wrap conditionals in parentheses 24639bc [Nicholas Chammas] [SPARK-2470] fix whitespace for doctest 7d557b7 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to tests.py 8f8e4c0 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to storagelevel.py b3b96cf [Nicholas Chammas] [SPARK-2470] PEP8 fixes to statcounter.py d644477 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to worker.py aa3a7b6 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to sql.py 1916859 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to shell.py 95d1d95 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to serializers.py a0fec2e [Nicholas Chammas] [SPARK-2470] PEP8 fixes to mllib c85e1e5 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to join.py d14f2f1 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to __init__.py 81fcb20 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to resultiterable.py 1bde265 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to java_gateway.py 7fc849c [Nicholas Chammas] [SPARK-2470] PEP8 fixes to daemon.py ca2d28b [Nicholas Chammas] [SPARK-2470] PEP8 fixes to context.py f4e0039 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to conf.py a6d5e4b [Nicholas Chammas] [SPARK-2470] PEP8 fixes to cloudpickle.py f0a7ebf [Nicholas Chammas] [SPARK-2470] PEP8 fixes to rddsampler.py 4dd148f [nchammas] Merge pull request #5 from apache/master f7e4581 [Nicholas Chammas] unrelated pep8 fix a36eed0 [Nicholas Chammas] name ec2 instances and security groups consistently de7292a [nchammas] Merge pull request #4 from apache/master 2e4fe00 [nchammas] Merge pull request #3 from apache/master 89fde08 [nchammas] Merge pull request #2 from apache/master 69f6e22 [Nicholas Chammas] PEP8 fixes 2627247 [Nicholas Chammas] broke up lines before they hit 100 chars 6544b7e [Nicholas Chammas] [SPARK-2065] give launched instances names 69da6cf [nchammas] Merge pull request #1 from apache/master
2014-07-22 01:30: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")
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):
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
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)
self.sc.show_profiles()
d = tempfile.gettempdir()
self.sc.dump_profiles(d)
self.assertTrue("rdd_%d.pstats" % id in os.listdir(d))
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):
for i in range(1 << 20):
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
class InputFormatTests(ReusedPySparkTestCase):
2013-02-01 03:25:19 -05: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
@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
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)
[SPARK-2024] Add saveAsSequenceFile to PySpark JIRA issue: https://issues.apache.org/jira/browse/SPARK-2024 This PR is a followup to #455 and adds capabilities for saving PySpark RDDs using SequenceFile or any Hadoop OutputFormats. * Added RDD methods ```saveAsSequenceFile```, ```saveAsHadoopFile``` and ```saveAsHadoopDataset```, for both old and new MapReduce APIs. * Default converter for converting common data types to Writables. Users may specify custom converters to convert to desired data types. * No out-of-box support for reading/writing arrays, since ArrayWritable itself doesn't have a no-arg constructor for creating an empty instance upon reading. Users need to provide ArrayWritable subtypes. Custom converters for converting arrays to suitable ArrayWritable subtypes are also needed when writing. When reading, the default converter will convert any custom ArrayWritable subtypes to ```Object[]``` and they get pickled to Python tuples. * Added HBase and Cassandra output examples to show how custom output formats and converters can be used. cc MLnick mateiz ahirreddy pwendell Author: Kan Zhang <kzhang@apache.org> Closes #1338 from kanzhang/SPARK-2024 and squashes the following commits: c01e3ef [Kan Zhang] [SPARK-2024] code formatting 6591e37 [Kan Zhang] [SPARK-2024] renaming pickled -> pickledRDD d998ad6 [Kan Zhang] [SPARK-2024] refectoring to get method params below 10 57a7a5e [Kan Zhang] [SPARK-2024] correcting typo 75ca5bd [Kan Zhang] [SPARK-2024] Better type checking for batch serialized RDD 0bdec55 [Kan Zhang] [SPARK-2024] Refactoring newly added tests 9f39ff4 [Kan Zhang] [SPARK-2024] Adding 2 saveAsHadoopDataset tests 0c134f3 [Kan Zhang] [SPARK-2024] Test refactoring and adding couple unbatched cases 7a176df [Kan Zhang] [SPARK-2024] Add saveAsSequenceFile to PySpark
2014-07-30 16:19:05 -04:00
bytes = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfbytes/",
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
"org.apache.hadoop.io.IntWritable",
"org.apache.hadoop.io.BytesWritable").collect())
[SPARK-2024] Add saveAsSequenceFile to PySpark JIRA issue: https://issues.apache.org/jira/browse/SPARK-2024 This PR is a followup to #455 and adds capabilities for saving PySpark RDDs using SequenceFile or any Hadoop OutputFormats. * Added RDD methods ```saveAsSequenceFile```, ```saveAsHadoopFile``` and ```saveAsHadoopDataset```, for both old and new MapReduce APIs. * Default converter for converting common data types to Writables. Users may specify custom converters to convert to desired data types. * No out-of-box support for reading/writing arrays, since ArrayWritable itself doesn't have a no-arg constructor for creating an empty instance upon reading. Users need to provide ArrayWritable subtypes. Custom converters for converting arrays to suitable ArrayWritable subtypes are also needed when writing. When reading, the default converter will convert any custom ArrayWritable subtypes to ```Object[]``` and they get pickled to Python tuples. * Added HBase and Cassandra output examples to show how custom output formats and converters can be used. cc MLnick mateiz ahirreddy pwendell Author: Kan Zhang <kzhang@apache.org> Closes #1338 from kanzhang/SPARK-2024 and squashes the following commits: c01e3ef [Kan Zhang] [SPARK-2024] code formatting 6591e37 [Kan Zhang] [SPARK-2024] renaming pickled -> pickledRDD d998ad6 [Kan Zhang] [SPARK-2024] refectoring to get method params below 10 57a7a5e [Kan Zhang] [SPARK-2024] correcting typo 75ca5bd [Kan Zhang] [SPARK-2024] Better type checking for batch serialized RDD 0bdec55 [Kan Zhang] [SPARK-2024] Refactoring newly added tests 9f39ff4 [Kan Zhang] [SPARK-2024] Adding 2 saveAsHadoopDataset tests 0c134f3 [Kan Zhang] [SPARK-2024] Test refactoring and adding couple unbatched cases 7a176df [Kan Zhang] [SPARK-2024] Add saveAsSequenceFile to PySpark
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)
maps = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfmap/",
"org.apache.hadoop.io.IntWritable",
"org.apache.hadoop.io.MapWritable").collect())
[SPARK-2024] Add saveAsSequenceFile to PySpark JIRA issue: https://issues.apache.org/jira/browse/SPARK-2024 This PR is a followup to #455 and adds capabilities for saving PySpark RDDs using SequenceFile or any Hadoop OutputFormats. * Added RDD methods ```saveAsSequenceFile```, ```saveAsHadoopFile``` and ```saveAsHadoopDataset```, for both old and new MapReduce APIs. * Default converter for converting common data types to Writables. Users may specify custom converters to convert to desired data types. * No out-of-box support for reading/writing arrays, since ArrayWritable itself doesn't have a no-arg constructor for creating an empty instance upon reading. Users need to provide ArrayWritable subtypes. Custom converters for converting arrays to suitable ArrayWritable subtypes are also needed when writing. When reading, the default converter will convert any custom ArrayWritable subtypes to ```Object[]``` and they get pickled to Python tuples. * Added HBase and Cassandra output examples to show how custom output formats and converters can be used. cc MLnick mateiz ahirreddy pwendell Author: Kan Zhang <kzhang@apache.org> Closes #1338 from kanzhang/SPARK-2024 and squashes the following commits: c01e3ef [Kan Zhang] [SPARK-2024] code formatting 6591e37 [Kan Zhang] [SPARK-2024] renaming pickled -> pickledRDD d998ad6 [Kan Zhang] [SPARK-2024] refectoring to get method params below 10 57a7a5e [Kan Zhang] [SPARK-2024] correcting typo 75ca5bd [Kan Zhang] [SPARK-2024] Better type checking for batch serialized RDD 0bdec55 [Kan Zhang] [SPARK-2024] Refactoring newly added tests 9f39ff4 [Kan Zhang] [SPARK-2024] Adding 2 saveAsHadoopDataset tests 0c134f3 [Kan Zhang] [SPARK-2024] Test refactoring and adding couple unbatched cases 7a176df [Kan Zhang] [SPARK-2024] Add saveAsSequenceFile to PySpark
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'})]
self.assertEqual(maps, em)
[SPARK-2024] Add saveAsSequenceFile to PySpark JIRA issue: https://issues.apache.org/jira/browse/SPARK-2024 This PR is a followup to #455 and adds capabilities for saving PySpark RDDs using SequenceFile or any Hadoop OutputFormats. * Added RDD methods ```saveAsSequenceFile```, ```saveAsHadoopFile``` and ```saveAsHadoopDataset```, for both old and new MapReduce APIs. * Default converter for converting common data types to Writables. Users may specify custom converters to convert to desired data types. * No out-of-box support for reading/writing arrays, since ArrayWritable itself doesn't have a no-arg constructor for creating an empty instance upon reading. Users need to provide ArrayWritable subtypes. Custom converters for converting arrays to suitable ArrayWritable subtypes are also needed when writing. When reading, the default converter will convert any custom ArrayWritable subtypes to ```Object[]``` and they get pickled to Python tuples. * Added HBase and Cassandra output examples to show how custom output formats and converters can be used. cc MLnick mateiz ahirreddy pwendell Author: Kan Zhang <kzhang@apache.org> Closes #1338 from kanzhang/SPARK-2024 and squashes the following commits: c01e3ef [Kan Zhang] [SPARK-2024] code formatting 6591e37 [Kan Zhang] [SPARK-2024] renaming pickled -> pickledRDD d998ad6 [Kan Zhang] [SPARK-2024] refectoring to get method params below 10 57a7a5e [Kan Zhang] [SPARK-2024] correcting typo 75ca5bd [Kan Zhang] [SPARK-2024] Better type checking for batch serialized RDD 0bdec55 [Kan Zhang] [SPARK-2024] Refactoring newly added tests 9f39ff4 [Kan Zhang] [SPARK-2024] Adding 2 saveAsHadoopDataset tests 0c134f3 [Kan Zhang] [SPARK-2024] Test refactoring and adding couple unbatched cases 7a176df [Kan Zhang] [SPARK-2024] Add saveAsSequenceFile to PySpark
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())
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
[SPARK-2024] Add saveAsSequenceFile to PySpark JIRA issue: https://issues.apache.org/jira/browse/SPARK-2024 This PR is a followup to #455 and adds capabilities for saving PySpark RDDs using SequenceFile or any Hadoop OutputFormats. * Added RDD methods ```saveAsSequenceFile```, ```saveAsHadoopFile``` and ```saveAsHadoopDataset```, for both old and new MapReduce APIs. * Default converter for converting common data types to Writables. Users may specify custom converters to convert to desired data types. * No out-of-box support for reading/writing arrays, since ArrayWritable itself doesn't have a no-arg constructor for creating an empty instance upon reading. Users need to provide ArrayWritable subtypes. Custom converters for converting arrays to suitable ArrayWritable subtypes are also needed when writing. When reading, the default converter will convert any custom ArrayWritable subtypes to ```Object[]``` and they get pickled to Python tuples. * Added HBase and Cassandra output examples to show how custom output formats and converters can be used. cc MLnick mateiz ahirreddy pwendell Author: Kan Zhang <kzhang@apache.org> Closes #1338 from kanzhang/SPARK-2024 and squashes the following commits: c01e3ef [Kan Zhang] [SPARK-2024] code formatting 6591e37 [Kan Zhang] [SPARK-2024] renaming pickled -> pickledRDD d998ad6 [Kan Zhang] [SPARK-2024] refectoring to get method params below 10 57a7a5e [Kan Zhang] [SPARK-2024] correcting typo 75ca5bd [Kan Zhang] [SPARK-2024] Better type checking for batch serialized RDD 0bdec55 [Kan Zhang] [SPARK-2024] Refactoring newly added tests 9f39ff4 [Kan Zhang] [SPARK-2024] Adding 2 saveAsHadoopDataset tests 0c134f3 [Kan Zhang] [SPARK-2024] Test refactoring and adding couple unbatched cases 7a176df [Kan Zhang] [SPARK-2024] Add saveAsSequenceFile to PySpark
2014-07-30 16:19:05 -04:00
unbatched_clazz = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfclass/",
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
"org.apache.hadoop.io.Text",
"org.apache.spark.api.python.TestWritable",
).collect())
self.assertEqual(unbatched_clazz, ec)
[SPARK-2024] Add saveAsSequenceFile to PySpark JIRA issue: https://issues.apache.org/jira/browse/SPARK-2024 This PR is a followup to #455 and adds capabilities for saving PySpark RDDs using SequenceFile or any Hadoop OutputFormats. * Added RDD methods ```saveAsSequenceFile```, ```saveAsHadoopFile``` and ```saveAsHadoopDataset```, for both old and new MapReduce APIs. * Default converter for converting common data types to Writables. Users may specify custom converters to convert to desired data types. * No out-of-box support for reading/writing arrays, since ArrayWritable itself doesn't have a no-arg constructor for creating an empty instance upon reading. Users need to provide ArrayWritable subtypes. Custom converters for converting arrays to suitable ArrayWritable subtypes are also needed when writing. When reading, the default converter will convert any custom ArrayWritable subtypes to ```Object[]``` and they get pickled to Python tuples. * Added HBase and Cassandra output examples to show how custom output formats and converters can be used. cc MLnick mateiz ahirreddy pwendell Author: Kan Zhang <kzhang@apache.org> Closes #1338 from kanzhang/SPARK-2024 and squashes the following commits: c01e3ef [Kan Zhang] [SPARK-2024] code formatting 6591e37 [Kan Zhang] [SPARK-2024] renaming pickled -> pickledRDD d998ad6 [Kan Zhang] [SPARK-2024] refectoring to get method params below 10 57a7a5e [Kan Zhang] [SPARK-2024] correcting typo 75ca5bd [Kan Zhang] [SPARK-2024] Better type checking for batch serialized RDD 0bdec55 [Kan Zhang] [SPARK-2024] Refactoring newly added tests 9f39ff4 [Kan Zhang] [SPARK-2024] Adding 2 saveAsHadoopDataset tests 0c134f3 [Kan Zhang] [SPARK-2024] Test refactoring and adding couple unbatched cases 7a176df [Kan Zhang] [SPARK-2024] Add saveAsSequenceFile to PySpark
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)
hellopath = os.path.join(SPARK_HOME, "python/test_support/hello.txt")
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
oldconf = {"mapred.input.dir": hellopath}
[SPARK-2024] Add saveAsSequenceFile to PySpark JIRA issue: https://issues.apache.org/jira/browse/SPARK-2024 This PR is a followup to #455 and adds capabilities for saving PySpark RDDs using SequenceFile or any Hadoop OutputFormats. * Added RDD methods ```saveAsSequenceFile```, ```saveAsHadoopFile``` and ```saveAsHadoopDataset```, for both old and new MapReduce APIs. * Default converter for converting common data types to Writables. Users may specify custom converters to convert to desired data types. * No out-of-box support for reading/writing arrays, since ArrayWritable itself doesn't have a no-arg constructor for creating an empty instance upon reading. Users need to provide ArrayWritable subtypes. Custom converters for converting arrays to suitable ArrayWritable subtypes are also needed when writing. When reading, the default converter will convert any custom ArrayWritable subtypes to ```Object[]``` and they get pickled to Python tuples. * Added HBase and Cassandra output examples to show how custom output formats and converters can be used. cc MLnick mateiz ahirreddy pwendell Author: Kan Zhang <kzhang@apache.org> Closes #1338 from kanzhang/SPARK-2024 and squashes the following commits: c01e3ef [Kan Zhang] [SPARK-2024] code formatting 6591e37 [Kan Zhang] [SPARK-2024] renaming pickled -> pickledRDD d998ad6 [Kan Zhang] [SPARK-2024] refectoring to get method params below 10 57a7a5e [Kan Zhang] [SPARK-2024] correcting typo 75ca5bd [Kan Zhang] [SPARK-2024] Better type checking for batch serialized RDD 0bdec55 [Kan Zhang] [SPARK-2024] Refactoring newly added tests 9f39ff4 [Kan Zhang] [SPARK-2024] Adding 2 saveAsHadoopDataset tests 0c134f3 [Kan Zhang] [SPARK-2024] Test refactoring and adding couple unbatched cases 7a176df [Kan Zhang] [SPARK-2024] Add saveAsSequenceFile to PySpark
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)
hellopath = os.path.join(SPARK_HOME, "python/test_support/hello.txt")
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
newconf = {"mapred.input.dir": hellopath}
[SPARK-2024] Add saveAsSequenceFile to PySpark JIRA issue: https://issues.apache.org/jira/browse/SPARK-2024 This PR is a followup to #455 and adds capabilities for saving PySpark RDDs using SequenceFile or any Hadoop OutputFormats. * Added RDD methods ```saveAsSequenceFile```, ```saveAsHadoopFile``` and ```saveAsHadoopDataset```, for both old and new MapReduce APIs. * Default converter for converting common data types to Writables. Users may specify custom converters to convert to desired data types. * No out-of-box support for reading/writing arrays, since ArrayWritable itself doesn't have a no-arg constructor for creating an empty instance upon reading. Users need to provide ArrayWritable subtypes. Custom converters for converting arrays to suitable ArrayWritable subtypes are also needed when writing. When reading, the default converter will convert any custom ArrayWritable subtypes to ```Object[]``` and they get pickled to Python tuples. * Added HBase and Cassandra output examples to show how custom output formats and converters can be used. cc MLnick mateiz ahirreddy pwendell Author: Kan Zhang <kzhang@apache.org> Closes #1338 from kanzhang/SPARK-2024 and squashes the following commits: c01e3ef [Kan Zhang] [SPARK-2024] code formatting 6591e37 [Kan Zhang] [SPARK-2024] renaming pickled -> pickledRDD d998ad6 [Kan Zhang] [SPARK-2024] refectoring to get method params below 10 57a7a5e [Kan Zhang] [SPARK-2024] correcting typo 75ca5bd [Kan Zhang] [SPARK-2024] Better type checking for batch serialized RDD 0bdec55 [Kan Zhang] [SPARK-2024] Refactoring newly added tests 9f39ff4 [Kan Zhang] [SPARK-2024] Adding 2 saveAsHadoopDataset tests 0c134f3 [Kan Zhang] [SPARK-2024] Test refactoring and adding couple unbatched cases 7a176df [Kan Zhang] [SPARK-2024] Add saveAsSequenceFile to PySpark
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"))
[SPARK-2024] Add saveAsSequenceFile to PySpark JIRA issue: https://issues.apache.org/jira/browse/SPARK-2024 This PR is a followup to #455 and adds capabilities for saving PySpark RDDs using SequenceFile or any Hadoop OutputFormats. * Added RDD methods ```saveAsSequenceFile```, ```saveAsHadoopFile``` and ```saveAsHadoopDataset```, for both old and new MapReduce APIs. * Default converter for converting common data types to Writables. Users may specify custom converters to convert to desired data types. * No out-of-box support for reading/writing arrays, since ArrayWritable itself doesn't have a no-arg constructor for creating an empty instance upon reading. Users need to provide ArrayWritable subtypes. Custom converters for converting arrays to suitable ArrayWritable subtypes are also needed when writing. When reading, the default converter will convert any custom ArrayWritable subtypes to ```Object[]``` and they get pickled to Python tuples. * Added HBase and Cassandra output examples to show how custom output formats and converters can be used. cc MLnick mateiz ahirreddy pwendell Author: Kan Zhang <kzhang@apache.org> Closes #1338 from kanzhang/SPARK-2024 and squashes the following commits: c01e3ef [Kan Zhang] [SPARK-2024] code formatting 6591e37 [Kan Zhang] [SPARK-2024] renaming pickled -> pickledRDD d998ad6 [Kan Zhang] [SPARK-2024] refectoring to get method params below 10 57a7a5e [Kan Zhang] [SPARK-2024] correcting typo 75ca5bd [Kan Zhang] [SPARK-2024] Better type checking for batch serialized RDD 0bdec55 [Kan Zhang] [SPARK-2024] Refactoring newly added tests 9f39ff4 [Kan Zhang] [SPARK-2024] Adding 2 saveAsHadoopDataset tests 0c134f3 [Kan Zhang] [SPARK-2024] Test refactoring and adding couple unbatched cases 7a176df [Kan Zhang] [SPARK-2024] Add saveAsSequenceFile to PySpark
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",
[SPARK-2024] Add saveAsSequenceFile to PySpark JIRA issue: https://issues.apache.org/jira/browse/SPARK-2024 This PR is a followup to #455 and adds capabilities for saving PySpark RDDs using SequenceFile or any Hadoop OutputFormats. * Added RDD methods ```saveAsSequenceFile```, ```saveAsHadoopFile``` and ```saveAsHadoopDataset```, for both old and new MapReduce APIs. * Default converter for converting common data types to Writables. Users may specify custom converters to convert to desired data types. * No out-of-box support for reading/writing arrays, since ArrayWritable itself doesn't have a no-arg constructor for creating an empty instance upon reading. Users need to provide ArrayWritable subtypes. Custom converters for converting arrays to suitable ArrayWritable subtypes are also needed when writing. When reading, the default converter will convert any custom ArrayWritable subtypes to ```Object[]``` and they get pickled to Python tuples. * Added HBase and Cassandra output examples to show how custom output formats and converters can be used. cc MLnick mateiz ahirreddy pwendell Author: Kan Zhang <kzhang@apache.org> Closes #1338 from kanzhang/SPARK-2024 and squashes the following commits: c01e3ef [Kan Zhang] [SPARK-2024] code formatting 6591e37 [Kan Zhang] [SPARK-2024] renaming pickled -> pickledRDD d998ad6 [Kan Zhang] [SPARK-2024] refectoring to get method params below 10 57a7a5e [Kan Zhang] [SPARK-2024] correcting typo 75ca5bd [Kan Zhang] [SPARK-2024] Better type checking for batch serialized RDD 0bdec55 [Kan Zhang] [SPARK-2024] Refactoring newly added tests 9f39ff4 [Kan Zhang] [SPARK-2024] Adding 2 saveAsHadoopDataset tests 0c134f3 [Kan Zhang] [SPARK-2024] Test refactoring and adding couple unbatched cases 7a176df [Kan Zhang] [SPARK-2024] Add saveAsSequenceFile to PySpark
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)
def test_binary_files(self):
path = os.path.join(self.tempdir.name, "binaryfiles")
os.mkdir(path)
data = "short binary data"
with open(os.path.join(path, "part-0000"), 'w') as f:
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()
self.assertEqual(range(100), result)
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
class OutputFormatTests(ReusedPySparkTestCase):
[SPARK-2024] Add saveAsSequenceFile to PySpark JIRA issue: https://issues.apache.org/jira/browse/SPARK-2024 This PR is a followup to #455 and adds capabilities for saving PySpark RDDs using SequenceFile or any Hadoop OutputFormats. * Added RDD methods ```saveAsSequenceFile```, ```saveAsHadoopFile``` and ```saveAsHadoopDataset```, for both old and new MapReduce APIs. * Default converter for converting common data types to Writables. Users may specify custom converters to convert to desired data types. * No out-of-box support for reading/writing arrays, since ArrayWritable itself doesn't have a no-arg constructor for creating an empty instance upon reading. Users need to provide ArrayWritable subtypes. Custom converters for converting arrays to suitable ArrayWritable subtypes are also needed when writing. When reading, the default converter will convert any custom ArrayWritable subtypes to ```Object[]``` and they get pickled to Python tuples. * Added HBase and Cassandra output examples to show how custom output formats and converters can be used. cc MLnick mateiz ahirreddy pwendell Author: Kan Zhang <kzhang@apache.org> Closes #1338 from kanzhang/SPARK-2024 and squashes the following commits: c01e3ef [Kan Zhang] [SPARK-2024] code formatting 6591e37 [Kan Zhang] [SPARK-2024] renaming pickled -> pickledRDD d998ad6 [Kan Zhang] [SPARK-2024] refectoring to get method params below 10 57a7a5e [Kan Zhang] [SPARK-2024] correcting typo 75ca5bd [Kan Zhang] [SPARK-2024] Better type checking for batch serialized RDD 0bdec55 [Kan Zhang] [SPARK-2024] Refactoring newly added tests 9f39ff4 [Kan Zhang] [SPARK-2024] Adding 2 saveAsHadoopDataset tests 0c134f3 [Kan Zhang] [SPARK-2024] Test refactoring and adding couple unbatched cases 7a176df [Kan Zhang] [SPARK-2024] Add saveAsSequenceFile to PySpark
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)
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/")
maps = sorted(self.sc.sequenceFile(basepath + "/sfmap/").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
self.assertEqual(maps, em)
[SPARK-2024] Add saveAsSequenceFile to PySpark JIRA issue: https://issues.apache.org/jira/browse/SPARK-2024 This PR is a followup to #455 and adds capabilities for saving PySpark RDDs using SequenceFile or any Hadoop OutputFormats. * Added RDD methods ```saveAsSequenceFile```, ```saveAsHadoopFile``` and ```saveAsHadoopDataset```, for both old and new MapReduce APIs. * Default converter for converting common data types to Writables. Users may specify custom converters to convert to desired data types. * No out-of-box support for reading/writing arrays, since ArrayWritable itself doesn't have a no-arg constructor for creating an empty instance upon reading. Users need to provide ArrayWritable subtypes. Custom converters for converting arrays to suitable ArrayWritable subtypes are also needed when writing. When reading, the default converter will convert any custom ArrayWritable subtypes to ```Object[]``` and they get pickled to Python tuples. * Added HBase and Cassandra output examples to show how custom output formats and converters can be used. cc MLnick mateiz ahirreddy pwendell Author: Kan Zhang <kzhang@apache.org> Closes #1338 from kanzhang/SPARK-2024 and squashes the following commits: c01e3ef [Kan Zhang] [SPARK-2024] code formatting 6591e37 [Kan Zhang] [SPARK-2024] renaming pickled -> pickledRDD d998ad6 [Kan Zhang] [SPARK-2024] refectoring to get method params below 10 57a7a5e [Kan Zhang] [SPARK-2024] correcting typo 75ca5bd [Kan Zhang] [SPARK-2024] Better type checking for batch serialized RDD 0bdec55 [Kan Zhang] [SPARK-2024] Refactoring newly added tests 9f39ff4 [Kan Zhang] [SPARK-2024] Adding 2 saveAsHadoopDataset tests 0c134f3 [Kan Zhang] [SPARK-2024] Test refactoring and adding couple unbatched cases 7a176df [Kan Zhang] [SPARK-2024] Add saveAsSequenceFile to PySpark
2014-07-30 16:19:05 -04:00
def test_oldhadoop(self):
basepath = self.tempdir.name
dict_data = [(1, {}),
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
(1, {"row1": 1.0}),
(2, {"row2": 2.0})]
[SPARK-2024] Add saveAsSequenceFile to PySpark JIRA issue: https://issues.apache.org/jira/browse/SPARK-2024 This PR is a followup to #455 and adds capabilities for saving PySpark RDDs using SequenceFile or any Hadoop OutputFormats. * Added RDD methods ```saveAsSequenceFile```, ```saveAsHadoopFile``` and ```saveAsHadoopDataset```, for both old and new MapReduce APIs. * Default converter for converting common data types to Writables. Users may specify custom converters to convert to desired data types. * No out-of-box support for reading/writing arrays, since ArrayWritable itself doesn't have a no-arg constructor for creating an empty instance upon reading. Users need to provide ArrayWritable subtypes. Custom converters for converting arrays to suitable ArrayWritable subtypes are also needed when writing. When reading, the default converter will convert any custom ArrayWritable subtypes to ```Object[]``` and they get pickled to Python tuples. * Added HBase and Cassandra output examples to show how custom output formats and converters can be used. cc MLnick mateiz ahirreddy pwendell Author: Kan Zhang <kzhang@apache.org> Closes #1338 from kanzhang/SPARK-2024 and squashes the following commits: c01e3ef [Kan Zhang] [SPARK-2024] code formatting 6591e37 [Kan Zhang] [SPARK-2024] renaming pickled -> pickledRDD d998ad6 [Kan Zhang] [SPARK-2024] refectoring to get method params below 10 57a7a5e [Kan Zhang] [SPARK-2024] correcting typo 75ca5bd [Kan Zhang] [SPARK-2024] Better type checking for batch serialized RDD 0bdec55 [Kan Zhang] [SPARK-2024] Refactoring newly added tests 9f39ff4 [Kan Zhang] [SPARK-2024] Adding 2 saveAsHadoopDataset tests 0c134f3 [Kan Zhang] [SPARK-2024] Test refactoring and adding couple unbatched cases 7a176df [Kan Zhang] [SPARK-2024] Add saveAsSequenceFile to PySpark
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")
result = sorted(self.sc.hadoopFile(
basepath + "/oldhadoop/",
"org.apache.hadoop.mapred.SequenceFileInputFormat",
"org.apache.hadoop.io.IntWritable",
"org.apache.hadoop.io.MapWritable").collect())
self.assertEqual(result, dict_data)
conf = {
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
"mapred.output.format.class": "org.apache.hadoop.mapred.SequenceFileOutputFormat",
"mapred.output.key.class": "org.apache.hadoop.io.IntWritable",
"mapred.output.value.class": "org.apache.hadoop.io.MapWritable",
"mapred.output.dir": basepath + "/olddataset/"
}
[SPARK-2024] Add saveAsSequenceFile to PySpark JIRA issue: https://issues.apache.org/jira/browse/SPARK-2024 This PR is a followup to #455 and adds capabilities for saving PySpark RDDs using SequenceFile or any Hadoop OutputFormats. * Added RDD methods ```saveAsSequenceFile```, ```saveAsHadoopFile``` and ```saveAsHadoopDataset```, for both old and new MapReduce APIs. * Default converter for converting common data types to Writables. Users may specify custom converters to convert to desired data types. * No out-of-box support for reading/writing arrays, since ArrayWritable itself doesn't have a no-arg constructor for creating an empty instance upon reading. Users need to provide ArrayWritable subtypes. Custom converters for converting arrays to suitable ArrayWritable subtypes are also needed when writing. When reading, the default converter will convert any custom ArrayWritable subtypes to ```Object[]``` and they get pickled to Python tuples. * Added HBase and Cassandra output examples to show how custom output formats and converters can be used. cc MLnick mateiz ahirreddy pwendell Author: Kan Zhang <kzhang@apache.org> Closes #1338 from kanzhang/SPARK-2024 and squashes the following commits: c01e3ef [Kan Zhang] [SPARK-2024] code formatting 6591e37 [Kan Zhang] [SPARK-2024] renaming pickled -> pickledRDD d998ad6 [Kan Zhang] [SPARK-2024] refectoring to get method params below 10 57a7a5e [Kan Zhang] [SPARK-2024] correcting typo 75ca5bd [Kan Zhang] [SPARK-2024] Better type checking for batch serialized RDD 0bdec55 [Kan Zhang] [SPARK-2024] Refactoring newly added tests 9f39ff4 [Kan Zhang] [SPARK-2024] Adding 2 saveAsHadoopDataset tests 0c134f3 [Kan Zhang] [SPARK-2024] Test refactoring and adding couple unbatched cases 7a176df [Kan Zhang] [SPARK-2024] Add saveAsSequenceFile to PySpark
2014-07-30 16:19:05 -04:00
self.sc.parallelize(dict_data).saveAsHadoopDataset(conf)
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
input_conf = {"mapred.input.dir": basepath + "/olddataset/"}
[SPARK-2024] Add saveAsSequenceFile to PySpark JIRA issue: https://issues.apache.org/jira/browse/SPARK-2024 This PR is a followup to #455 and adds capabilities for saving PySpark RDDs using SequenceFile or any Hadoop OutputFormats. * Added RDD methods ```saveAsSequenceFile```, ```saveAsHadoopFile``` and ```saveAsHadoopDataset```, for both old and new MapReduce APIs. * Default converter for converting common data types to Writables. Users may specify custom converters to convert to desired data types. * No out-of-box support for reading/writing arrays, since ArrayWritable itself doesn't have a no-arg constructor for creating an empty instance upon reading. Users need to provide ArrayWritable subtypes. Custom converters for converting arrays to suitable ArrayWritable subtypes are also needed when writing. When reading, the default converter will convert any custom ArrayWritable subtypes to ```Object[]``` and they get pickled to Python tuples. * Added HBase and Cassandra output examples to show how custom output formats and converters can be used. cc MLnick mateiz ahirreddy pwendell Author: Kan Zhang <kzhang@apache.org> Closes #1338 from kanzhang/SPARK-2024 and squashes the following commits: c01e3ef [Kan Zhang] [SPARK-2024] code formatting 6591e37 [Kan Zhang] [SPARK-2024] renaming pickled -> pickledRDD d998ad6 [Kan Zhang] [SPARK-2024] refectoring to get method params below 10 57a7a5e [Kan Zhang] [SPARK-2024] correcting typo 75ca5bd [Kan Zhang] [SPARK-2024] Better type checking for batch serialized RDD 0bdec55 [Kan Zhang] [SPARK-2024] Refactoring newly added tests 9f39ff4 [Kan Zhang] [SPARK-2024] Adding 2 saveAsHadoopDataset tests 0c134f3 [Kan Zhang] [SPARK-2024] Test refactoring and adding couple unbatched cases 7a176df [Kan Zhang] [SPARK-2024] Add saveAsSequenceFile to PySpark
2014-07-30 16:19:05 -04:00
old_dataset = sorted(self.sc.hadoopRDD(
"org.apache.hadoop.mapred.SequenceFileInputFormat",
"org.apache.hadoop.io.IntWritable",
"org.apache.hadoop.io.MapWritable",
conf=input_conf).collect())
self.assertEqual(old_dataset, dict_data)
def test_newhadoop(self):
[SPARK-3094] [PySpark] compatitable with PyPy After this patch, we can run PySpark in PyPy (testing with PyPy 2.3.1 in Mac 10.9), for example: ``` PYSPARK_PYTHON=pypy ./bin/spark-submit wordcount.py ``` The performance speed up will depend on work load (from 20% to 3000%). Here are some benchmarks: Job | CPython 2.7 | PyPy 2.3.1 | Speed up ------- | ------------ | ------------- | ------- Word Count | 41s | 15s | 2.7x Sort | 46s | 44s | 1.05x Stats | 174s | 3.6s | 48x Here is the code used for benchmark: ```python rdd = sc.textFile("text") def wordcount(): rdd.flatMap(lambda x:x.split('/'))\ .map(lambda x:(x,1)).reduceByKey(lambda x,y:x+y).collectAsMap() def sort(): rdd.sortBy(lambda x:x, 1).count() def stats(): sc.parallelize(range(1024), 20).flatMap(lambda x: xrange(5024)).stats() ``` Author: Davies Liu <davies.liu@gmail.com> Closes #2144 from davies/pypy and squashes the following commits: 9aed6c5 [Davies Liu] use protocol 2 in CloudPickle 4bc1f04 [Davies Liu] refactor b20ab3a [Davies Liu] pickle sys.stdout and stderr in portable way 3ca2351 [Davies Liu] Merge branch 'master' into pypy fae8b19 [Davies Liu] improve attrgetter, add tests 591f830 [Davies Liu] try to run tests with PyPy in run-tests c8d62ba [Davies Liu] cleanup f651fd0 [Davies Liu] fix tests using array with PyPy 1b98fb3 [Davies Liu] serialize itemgetter/attrgetter in portable ways 3c1dbfe [Davies Liu] Merge branch 'master' into pypy 42fb5fa [Davies Liu] Merge branch 'master' into pypy cb2d724 [Davies Liu] fix tests 9986692 [Davies Liu] Merge branch 'master' into pypy 25b4ca7 [Davies Liu] support PyPy
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 = {
"mapreduce.outputformat.class":
"org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat",
"mapred.output.key.class": "org.apache.hadoop.io.IntWritable",
"mapred.output.value.class": "org.apache.hadoop.io.Text",
"mapred.output.dir": basepath + "/newdataset/"
}
self.sc.parallelize(data).saveAsNewAPIHadoopDataset(conf)
input_conf = {"mapred.input.dir": basepath + "/newdataset/"}
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)
def test_newhadoop_with_array(self):
[SPARK-2024] Add saveAsSequenceFile to PySpark JIRA issue: https://issues.apache.org/jira/browse/SPARK-2024 This PR is a followup to #455 and adds capabilities for saving PySpark RDDs using SequenceFile or any Hadoop OutputFormats. * Added RDD methods ```saveAsSequenceFile```, ```saveAsHadoopFile``` and ```saveAsHadoopDataset```, for both old and new MapReduce APIs. * Default converter for converting common data types to Writables. Users may specify custom converters to convert to desired data types. * No out-of-box support for reading/writing arrays, since ArrayWritable itself doesn't have a no-arg constructor for creating an empty instance upon reading. Users need to provide ArrayWritable subtypes. Custom converters for converting arrays to suitable ArrayWritable subtypes are also needed when writing. When reading, the default converter will convert any custom ArrayWritable subtypes to ```Object[]``` and they get pickled to Python tuples. * Added HBase and Cassandra output examples to show how custom output formats and converters can be used. cc MLnick mateiz ahirreddy pwendell Author: Kan Zhang <kzhang@apache.org> Closes #1338 from kanzhang/SPARK-2024 and squashes the following commits: c01e3ef [Kan Zhang] [SPARK-2024] code formatting 6591e37 [Kan Zhang] [SPARK-2024] renaming pickled -> pickledRDD d998ad6 [Kan Zhang] [SPARK-2024] refectoring to get method params below 10 57a7a5e [Kan Zhang] [SPARK-2024] correcting typo 75ca5bd [Kan Zhang] [SPARK-2024] Better type checking for batch serialized RDD 0bdec55 [Kan Zhang] [SPARK-2024] Refactoring newly added tests 9f39ff4 [Kan Zhang] [SPARK-2024] Adding 2 saveAsHadoopDataset tests 0c134f3 [Kan Zhang] [SPARK-2024] Test refactoring and adding couple unbatched cases 7a176df [Kan Zhang] [SPARK-2024] Add saveAsSequenceFile to PySpark
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)
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
conf = {
"mapreduce.outputformat.class":
"org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat",
"mapred.output.key.class": "org.apache.hadoop.io.IntWritable",
"mapred.output.value.class": "org.apache.spark.api.python.DoubleArrayWritable",
"mapred.output.dir": basepath + "/newdataset/"
}
self.sc.parallelize(array_data).saveAsNewAPIHadoopDataset(
conf,
[SPARK-2024] Add saveAsSequenceFile to PySpark JIRA issue: https://issues.apache.org/jira/browse/SPARK-2024 This PR is a followup to #455 and adds capabilities for saving PySpark RDDs using SequenceFile or any Hadoop OutputFormats. * Added RDD methods ```saveAsSequenceFile```, ```saveAsHadoopFile``` and ```saveAsHadoopDataset```, for both old and new MapReduce APIs. * Default converter for converting common data types to Writables. Users may specify custom converters to convert to desired data types. * No out-of-box support for reading/writing arrays, since ArrayWritable itself doesn't have a no-arg constructor for creating an empty instance upon reading. Users need to provide ArrayWritable subtypes. Custom converters for converting arrays to suitable ArrayWritable subtypes are also needed when writing. When reading, the default converter will convert any custom ArrayWritable subtypes to ```Object[]``` and they get pickled to Python tuples. * Added HBase and Cassandra output examples to show how custom output formats and converters can be used. cc MLnick mateiz ahirreddy pwendell Author: Kan Zhang <kzhang@apache.org> Closes #1338 from kanzhang/SPARK-2024 and squashes the following commits: c01e3ef [Kan Zhang] [SPARK-2024] code formatting 6591e37 [Kan Zhang] [SPARK-2024] renaming pickled -> pickledRDD d998ad6 [Kan Zhang] [SPARK-2024] refectoring to get method params below 10 57a7a5e [Kan Zhang] [SPARK-2024] correcting typo 75ca5bd [Kan Zhang] [SPARK-2024] Better type checking for batch serialized RDD 0bdec55 [Kan Zhang] [SPARK-2024] Refactoring newly added tests 9f39ff4 [Kan Zhang] [SPARK-2024] Adding 2 saveAsHadoopDataset tests 0c134f3 [Kan Zhang] [SPARK-2024] Test refactoring and adding couple unbatched cases 7a176df [Kan Zhang] [SPARK-2024] Add saveAsSequenceFile to PySpark
2014-07-30 16:19:05 -04:00
valueConverter="org.apache.spark.api.python.DoubleArrayToWritableConverter")
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
input_conf = {"mapred.input.dir": basepath + "/newdataset/"}
[SPARK-2024] Add saveAsSequenceFile to PySpark JIRA issue: https://issues.apache.org/jira/browse/SPARK-2024 This PR is a followup to #455 and adds capabilities for saving PySpark RDDs using SequenceFile or any Hadoop OutputFormats. * Added RDD methods ```saveAsSequenceFile```, ```saveAsHadoopFile``` and ```saveAsHadoopDataset```, for both old and new MapReduce APIs. * Default converter for converting common data types to Writables. Users may specify custom converters to convert to desired data types. * No out-of-box support for reading/writing arrays, since ArrayWritable itself doesn't have a no-arg constructor for creating an empty instance upon reading. Users need to provide ArrayWritable subtypes. Custom converters for converting arrays to suitable ArrayWritable subtypes are also needed when writing. When reading, the default converter will convert any custom ArrayWritable subtypes to ```Object[]``` and they get pickled to Python tuples. * Added HBase and Cassandra output examples to show how custom output formats and converters can be used. cc MLnick mateiz ahirreddy pwendell Author: Kan Zhang <kzhang@apache.org> Closes #1338 from kanzhang/SPARK-2024 and squashes the following commits: c01e3ef [Kan Zhang] [SPARK-2024] code formatting 6591e37 [Kan Zhang] [SPARK-2024] renaming pickled -> pickledRDD d998ad6 [Kan Zhang] [SPARK-2024] refectoring to get method params below 10 57a7a5e [Kan Zhang] [SPARK-2024] correcting typo 75ca5bd [Kan Zhang] [SPARK-2024] Better type checking for batch serialized RDD 0bdec55 [Kan Zhang] [SPARK-2024] Refactoring newly added tests 9f39ff4 [Kan Zhang] [SPARK-2024] Adding 2 saveAsHadoopDataset tests 0c134f3 [Kan Zhang] [SPARK-2024] Test refactoring and adding couple unbatched cases 7a176df [Kan Zhang] [SPARK-2024] Add saveAsSequenceFile to PySpark
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
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
rdd = self.sc.parallelize(range(1, 4)).map(lambda x: (x, "a" * x))
[SPARK-2024] Add saveAsSequenceFile to PySpark JIRA issue: https://issues.apache.org/jira/browse/SPARK-2024 This PR is a followup to #455 and adds capabilities for saving PySpark RDDs using SequenceFile or any Hadoop OutputFormats. * Added RDD methods ```saveAsSequenceFile```, ```saveAsHadoopFile``` and ```saveAsHadoopDataset```, for both old and new MapReduce APIs. * Default converter for converting common data types to Writables. Users may specify custom converters to convert to desired data types. * No out-of-box support for reading/writing arrays, since ArrayWritable itself doesn't have a no-arg constructor for creating an empty instance upon reading. Users need to provide ArrayWritable subtypes. Custom converters for converting arrays to suitable ArrayWritable subtypes are also needed when writing. When reading, the default converter will convert any custom ArrayWritable subtypes to ```Object[]``` and they get pickled to Python tuples. * Added HBase and Cassandra output examples to show how custom output formats and converters can be used. cc MLnick mateiz ahirreddy pwendell Author: Kan Zhang <kzhang@apache.org> Closes #1338 from kanzhang/SPARK-2024 and squashes the following commits: c01e3ef [Kan Zhang] [SPARK-2024] code formatting 6591e37 [Kan Zhang] [SPARK-2024] renaming pickled -> pickledRDD d998ad6 [Kan Zhang] [SPARK-2024] refectoring to get method params below 10 57a7a5e [Kan Zhang] [SPARK-2024] correcting typo 75ca5bd [Kan Zhang] [SPARK-2024] Better type checking for batch serialized RDD 0bdec55 [Kan Zhang] [SPARK-2024] Refactoring newly added tests 9f39ff4 [Kan Zhang] [SPARK-2024] Adding 2 saveAsHadoopDataset tests 0c134f3 [Kan Zhang] [SPARK-2024] Test refactoring and adding couple unbatched cases 7a176df [Kan Zhang] [SPARK-2024] Add saveAsSequenceFile to PySpark
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
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
rdd = self.sc.parallelize(range(1, 4)).map(lambda x: (x, "a" * x))
[SPARK-2024] Add saveAsSequenceFile to PySpark JIRA issue: https://issues.apache.org/jira/browse/SPARK-2024 This PR is a followup to #455 and adds capabilities for saving PySpark RDDs using SequenceFile or any Hadoop OutputFormats. * Added RDD methods ```saveAsSequenceFile```, ```saveAsHadoopFile``` and ```saveAsHadoopDataset```, for both old and new MapReduce APIs. * Default converter for converting common data types to Writables. Users may specify custom converters to convert to desired data types. * No out-of-box support for reading/writing arrays, since ArrayWritable itself doesn't have a no-arg constructor for creating an empty instance upon reading. Users need to provide ArrayWritable subtypes. Custom converters for converting arrays to suitable ArrayWritable subtypes are also needed when writing. When reading, the default converter will convert any custom ArrayWritable subtypes to ```Object[]``` and they get pickled to Python tuples. * Added HBase and Cassandra output examples to show how custom output formats and converters can be used. cc MLnick mateiz ahirreddy pwendell Author: Kan Zhang <kzhang@apache.org> Closes #1338 from kanzhang/SPARK-2024 and squashes the following commits: c01e3ef [Kan Zhang] [SPARK-2024] code formatting 6591e37 [Kan Zhang] [SPARK-2024] renaming pickled -> pickledRDD d998ad6 [Kan Zhang] [SPARK-2024] refectoring to get method params below 10 57a7a5e [Kan Zhang] [SPARK-2024] correcting typo 75ca5bd [Kan Zhang] [SPARK-2024] Better type checking for batch serialized RDD 0bdec55 [Kan Zhang] [SPARK-2024] Refactoring newly added tests 9f39ff4 [Kan Zhang] [SPARK-2024] Adding 2 saveAsHadoopDataset tests 0c134f3 [Kan Zhang] [SPARK-2024] Test refactoring and adding couple unbatched cases 7a176df [Kan Zhang] [SPARK-2024] Add saveAsSequenceFile to PySpark
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)
data = zip(x, y)
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)
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
rdd.saveAsHadoopFile(
basepath + "/reserialize/hadoop",
"org.apache.hadoop.mapred.SequenceFileOutputFormat")
[SPARK-2024] Add saveAsSequenceFile to PySpark JIRA issue: https://issues.apache.org/jira/browse/SPARK-2024 This PR is a followup to #455 and adds capabilities for saving PySpark RDDs using SequenceFile or any Hadoop OutputFormats. * Added RDD methods ```saveAsSequenceFile```, ```saveAsHadoopFile``` and ```saveAsHadoopDataset```, for both old and new MapReduce APIs. * Default converter for converting common data types to Writables. Users may specify custom converters to convert to desired data types. * No out-of-box support for reading/writing arrays, since ArrayWritable itself doesn't have a no-arg constructor for creating an empty instance upon reading. Users need to provide ArrayWritable subtypes. Custom converters for converting arrays to suitable ArrayWritable subtypes are also needed when writing. When reading, the default converter will convert any custom ArrayWritable subtypes to ```Object[]``` and they get pickled to Python tuples. * Added HBase and Cassandra output examples to show how custom output formats and converters can be used. cc MLnick mateiz ahirreddy pwendell Author: Kan Zhang <kzhang@apache.org> Closes #1338 from kanzhang/SPARK-2024 and squashes the following commits: c01e3ef [Kan Zhang] [SPARK-2024] code formatting 6591e37 [Kan Zhang] [SPARK-2024] renaming pickled -> pickledRDD d998ad6 [Kan Zhang] [SPARK-2024] refectoring to get method params below 10 57a7a5e [Kan Zhang] [SPARK-2024] correcting typo 75ca5bd [Kan Zhang] [SPARK-2024] Better type checking for batch serialized RDD 0bdec55 [Kan Zhang] [SPARK-2024] Refactoring newly added tests 9f39ff4 [Kan Zhang] [SPARK-2024] Adding 2 saveAsHadoopDataset tests 0c134f3 [Kan Zhang] [SPARK-2024] Test refactoring and adding couple unbatched cases 7a176df [Kan Zhang] [SPARK-2024] Add saveAsSequenceFile to PySpark
2014-07-30 16:19:05 -04:00
result2 = sorted(self.sc.sequenceFile(basepath + "/reserialize/hadoop").collect())
self.assertEqual(result2, data)
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
rdd.saveAsNewAPIHadoopFile(
basepath + "/reserialize/newhadoop",
"org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat")
[SPARK-2024] Add saveAsSequenceFile to PySpark JIRA issue: https://issues.apache.org/jira/browse/SPARK-2024 This PR is a followup to #455 and adds capabilities for saving PySpark RDDs using SequenceFile or any Hadoop OutputFormats. * Added RDD methods ```saveAsSequenceFile```, ```saveAsHadoopFile``` and ```saveAsHadoopDataset```, for both old and new MapReduce APIs. * Default converter for converting common data types to Writables. Users may specify custom converters to convert to desired data types. * No out-of-box support for reading/writing arrays, since ArrayWritable itself doesn't have a no-arg constructor for creating an empty instance upon reading. Users need to provide ArrayWritable subtypes. Custom converters for converting arrays to suitable ArrayWritable subtypes are also needed when writing. When reading, the default converter will convert any custom ArrayWritable subtypes to ```Object[]``` and they get pickled to Python tuples. * Added HBase and Cassandra output examples to show how custom output formats and converters can be used. cc MLnick mateiz ahirreddy pwendell Author: Kan Zhang <kzhang@apache.org> Closes #1338 from kanzhang/SPARK-2024 and squashes the following commits: c01e3ef [Kan Zhang] [SPARK-2024] code formatting 6591e37 [Kan Zhang] [SPARK-2024] renaming pickled -> pickledRDD d998ad6 [Kan Zhang] [SPARK-2024] refectoring to get method params below 10 57a7a5e [Kan Zhang] [SPARK-2024] correcting typo 75ca5bd [Kan Zhang] [SPARK-2024] Better type checking for batch serialized RDD 0bdec55 [Kan Zhang] [SPARK-2024] Refactoring newly added tests 9f39ff4 [Kan Zhang] [SPARK-2024] Adding 2 saveAsHadoopDataset tests 0c134f3 [Kan Zhang] [SPARK-2024] Test refactoring and adding couple unbatched cases 7a176df [Kan Zhang] [SPARK-2024] Add saveAsSequenceFile to PySpark
2014-07-30 16:19:05 -04:00
result3 = sorted(self.sc.sequenceFile(basepath + "/reserialize/newhadoop").collect())
self.assertEqual(result3, data)
conf4 = {
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
"mapred.output.format.class": "org.apache.hadoop.mapred.SequenceFileOutputFormat",
"mapred.output.key.class": "org.apache.hadoop.io.IntWritable",
"mapred.output.value.class": "org.apache.hadoop.io.IntWritable",
"mapred.output.dir": basepath + "/reserialize/dataset"}
[SPARK-2024] Add saveAsSequenceFile to PySpark JIRA issue: https://issues.apache.org/jira/browse/SPARK-2024 This PR is a followup to #455 and adds capabilities for saving PySpark RDDs using SequenceFile or any Hadoop OutputFormats. * Added RDD methods ```saveAsSequenceFile```, ```saveAsHadoopFile``` and ```saveAsHadoopDataset```, for both old and new MapReduce APIs. * Default converter for converting common data types to Writables. Users may specify custom converters to convert to desired data types. * No out-of-box support for reading/writing arrays, since ArrayWritable itself doesn't have a no-arg constructor for creating an empty instance upon reading. Users need to provide ArrayWritable subtypes. Custom converters for converting arrays to suitable ArrayWritable subtypes are also needed when writing. When reading, the default converter will convert any custom ArrayWritable subtypes to ```Object[]``` and they get pickled to Python tuples. * Added HBase and Cassandra output examples to show how custom output formats and converters can be used. cc MLnick mateiz ahirreddy pwendell Author: Kan Zhang <kzhang@apache.org> Closes #1338 from kanzhang/SPARK-2024 and squashes the following commits: c01e3ef [Kan Zhang] [SPARK-2024] code formatting 6591e37 [Kan Zhang] [SPARK-2024] renaming pickled -> pickledRDD d998ad6 [Kan Zhang] [SPARK-2024] refectoring to get method params below 10 57a7a5e [Kan Zhang] [SPARK-2024] correcting typo 75ca5bd [Kan Zhang] [SPARK-2024] Better type checking for batch serialized RDD 0bdec55 [Kan Zhang] [SPARK-2024] Refactoring newly added tests 9f39ff4 [Kan Zhang] [SPARK-2024] Adding 2 saveAsHadoopDataset tests 0c134f3 [Kan Zhang] [SPARK-2024] Test refactoring and adding couple unbatched cases 7a176df [Kan Zhang] [SPARK-2024] Add saveAsSequenceFile to PySpark
2014-07-30 16:19:05 -04:00
rdd.saveAsHadoopDataset(conf4)
result4 = sorted(self.sc.sequenceFile(basepath + "/reserialize/dataset").collect())
self.assertEqual(result4, data)
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
conf5 = {"mapreduce.outputformat.class":
"org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat",
"mapred.output.key.class": "org.apache.hadoop.io.IntWritable",
"mapred.output.value.class": "org.apache.hadoop.io.IntWritable",
"mapred.output.dir": basepath + "/reserialize/newdataset"}
[SPARK-2024] Add saveAsSequenceFile to PySpark JIRA issue: https://issues.apache.org/jira/browse/SPARK-2024 This PR is a followup to #455 and adds capabilities for saving PySpark RDDs using SequenceFile or any Hadoop OutputFormats. * Added RDD methods ```saveAsSequenceFile```, ```saveAsHadoopFile``` and ```saveAsHadoopDataset```, for both old and new MapReduce APIs. * Default converter for converting common data types to Writables. Users may specify custom converters to convert to desired data types. * No out-of-box support for reading/writing arrays, since ArrayWritable itself doesn't have a no-arg constructor for creating an empty instance upon reading. Users need to provide ArrayWritable subtypes. Custom converters for converting arrays to suitable ArrayWritable subtypes are also needed when writing. When reading, the default converter will convert any custom ArrayWritable subtypes to ```Object[]``` and they get pickled to Python tuples. * Added HBase and Cassandra output examples to show how custom output formats and converters can be used. cc MLnick mateiz ahirreddy pwendell Author: Kan Zhang <kzhang@apache.org> Closes #1338 from kanzhang/SPARK-2024 and squashes the following commits: c01e3ef [Kan Zhang] [SPARK-2024] code formatting 6591e37 [Kan Zhang] [SPARK-2024] renaming pickled -> pickledRDD d998ad6 [Kan Zhang] [SPARK-2024] refectoring to get method params below 10 57a7a5e [Kan Zhang] [SPARK-2024] correcting typo 75ca5bd [Kan Zhang] [SPARK-2024] Better type checking for batch serialized RDD 0bdec55 [Kan Zhang] [SPARK-2024] Refactoring newly added tests 9f39ff4 [Kan Zhang] [SPARK-2024] Adding 2 saveAsHadoopDataset tests 0c134f3 [Kan Zhang] [SPARK-2024] Test refactoring and adding couple unbatched cases 7a176df [Kan Zhang] [SPARK-2024] Add saveAsSequenceFile to PySpark
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")]]
rdd = self.sc.parallelize(data, len(data))
[SPARK-2024] Add saveAsSequenceFile to PySpark JIRA issue: https://issues.apache.org/jira/browse/SPARK-2024 This PR is a followup to #455 and adds capabilities for saving PySpark RDDs using SequenceFile or any Hadoop OutputFormats. * Added RDD methods ```saveAsSequenceFile```, ```saveAsHadoopFile``` and ```saveAsHadoopDataset```, for both old and new MapReduce APIs. * Default converter for converting common data types to Writables. Users may specify custom converters to convert to desired data types. * No out-of-box support for reading/writing arrays, since ArrayWritable itself doesn't have a no-arg constructor for creating an empty instance upon reading. Users need to provide ArrayWritable subtypes. Custom converters for converting arrays to suitable ArrayWritable subtypes are also needed when writing. When reading, the default converter will convert any custom ArrayWritable subtypes to ```Object[]``` and they get pickled to Python tuples. * Added HBase and Cassandra output examples to show how custom output formats and converters can be used. cc MLnick mateiz ahirreddy pwendell Author: Kan Zhang <kzhang@apache.org> Closes #1338 from kanzhang/SPARK-2024 and squashes the following commits: c01e3ef [Kan Zhang] [SPARK-2024] code formatting 6591e37 [Kan Zhang] [SPARK-2024] renaming pickled -> pickledRDD d998ad6 [Kan Zhang] [SPARK-2024] refectoring to get method params below 10 57a7a5e [Kan Zhang] [SPARK-2024] correcting typo 75ca5bd [Kan Zhang] [SPARK-2024] Better type checking for batch serialized RDD 0bdec55 [Kan Zhang] [SPARK-2024] Refactoring newly added tests 9f39ff4 [Kan Zhang] [SPARK-2024] Adding 2 saveAsHadoopDataset tests 0c134f3 [Kan Zhang] [SPARK-2024] Test refactoring and adding couple unbatched cases 7a176df [Kan Zhang] [SPARK-2024] Add saveAsSequenceFile to PySpark
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
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
class DaemonTests(unittest.TestCase):
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
sock.send("\xFF\xFF\xFF\xFF")
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")
daemon = Popen([sys.executable, daemon_path], stdin=PIPE, stdout=PIPE)
# 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:
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")
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
class WorkerTests(PySparkTestCase):
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
def test_cancel_task(self):
temp = tempfile.NamedTemporaryFile(delete=True)
temp.close()
path = temp.name
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
def sleep(x):
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
import os
import time
with open(path, 'w') as f:
f.write("%d %d" % (os.getppid(), os.getpid()))
time.sleep(100)
# start job in background thread
def run():
self.sc.parallelize(range(1)).foreach(sleep)
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):
data = open(path).read().split(' ')
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:
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
break # worker was killed
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")
[SPARK-3030] [PySpark] Reuse Python worker Reuse Python worker to avoid the overhead of fork() Python process for each tasks. It also tracks the broadcasts for each worker, avoid sending repeated broadcasts. This can reduce the time for dummy task from 22ms to 13ms (-40%). It can help to reduce the latency for Spark Streaming. For a job with broadcast (43M after compress): ``` b = sc.broadcast(set(range(30000000))) print sc.parallelize(range(24000), 100).filter(lambda x: x in b.value).count() ``` It will finish in 281s without reused worker, and it will finish in 65s with reused worker(4 CPUs). After reusing the worker, it can save about 9 seconds for transfer and deserialize the broadcast for each tasks. It's enabled by default, could be disabled by `spark.python.worker.reuse = false`. Author: Davies Liu <davies.liu@gmail.com> Closes #2259 from davies/reuse-worker and squashes the following commits: f11f617 [Davies Liu] Merge branch 'master' into reuse-worker 3939f20 [Davies Liu] fix bug in serializer in mllib cf1c55e [Davies Liu] address comments 3133a60 [Davies Liu] fix accumulator with reused worker 760ab1f [Davies Liu] do not reuse worker if there are any exceptions 7abb224 [Davies Liu] refactor: sychronized with itself ac3206e [Davies Liu] renaming 8911f44 [Davies Liu] synchronized getWorkerBroadcasts() 6325fc1 [Davies Liu] bugfix: bid >= 0 e0131a2 [Davies Liu] fix name of config 583716e [Davies Liu] only reuse completed and not interrupted worker ace2917 [Davies Liu] kill python worker after timeout 6123d0f [Davies Liu] track broadcasts for each worker 8d2f08c [Davies Liu] reuse python worker
2014-09-13 19:22:04 -04:00
# run a normal job
rdd = self.sc.parallelize(range(100), 1)
self.assertEqual(100, rdd.map(str).count())
def test_after_exception(self):
def raise_exception(_):
raise Exception()
rdd = self.sc.parallelize(range(100), 1)
self.assertRaises(Exception, lambda: rdd.foreach(raise_exception))
self.assertEqual(100, rdd.map(str).count())
def test_after_jvm_exception(self):
tempFile = tempfile.NamedTemporaryFile(delete=False)
tempFile.write("Hello World!")
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)
self.assertRaises(Exception, lambda: filtered_data.count())
rdd = self.sc.parallelize(range(100), 1)
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)
self.sc.parallelize(range(100), 20).foreach(lambda x: acc1.add(x))
self.assertEqual(sum(range(100)), acc1.value)
acc2 = self.sc.accumulator(0, INT_ACCUMULATOR_PARAM)
self.sc.parallelize(range(100), 20).foreach(lambda x: acc2.add(x))
self.assertEqual(sum(range(100)), acc2.value)
self.assertEqual(sum(range(100)), acc1.value)
def test_reuse_worker_after_take(self):
rdd = self.sc.parallelize(range(100000), 1)
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())
class SparkSubmitTests(unittest.TestCase):
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
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()
self.sparkSubmit = os.path.join(os.environ.get("SPARK_HOME"), "bin", "spark-submit")
def tearDown(self):
shutil.rmtree(self.programDir)
def createTempFile(self, name, content):
"""
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())
path = os.path.join(self.programDir, name)
with open(path, "w") as f:
f.write(content)
return path
def createFileInZip(self, name, content):
"""
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())
path = os.path.join(self.programDir, name + ".zip")
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
def test_single_script(self):
"""Submit and test a single script file"""
script = self.createTempFile("test.py", """
|from pyspark import SparkContext
|
|sc = SparkContext()
|print sc.parallelize([1, 2, 3]).map(lambda x: x * 2).collect()
""")
proc = subprocess.Popen([self.sparkSubmit, script], stdout=subprocess.PIPE)
out, err = proc.communicate()
self.assertEqual(0, proc.returncode)
self.assertIn("[2, 4, 6]", out)
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()
|print sc.parallelize([1, 2, 3]).map(foo).collect()
""")
proc = subprocess.Popen([self.sparkSubmit, script], stdout=subprocess.PIPE)
out, err = proc.communicate()
self.assertEqual(0, proc.returncode)
self.assertIn("[3, 6, 9]", out)
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()
|print sc.parallelize([1, 2, 3]).map(myfunc).collect()
""")
zip = self.createFileInZip("mylib.py", """
|def myfunc(x):
| return x + 1
""")
proc = subprocess.Popen([self.sparkSubmit, "--py-files", zip, script],
[SPARK-2470] PEP8 fixes to PySpark This pull request aims to resolve all outstanding PEP8 violations in PySpark. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1505 from nchammas/master and squashes the following commits: 98171af [Nicholas Chammas] [SPARK-2470] revert PEP 8 fixes to cloudpickle cba7768 [Nicholas Chammas] [SPARK-2470] wrap expression list in parentheses e178dbe [Nicholas Chammas] [SPARK-2470] style - change position of line break 9127d2b [Nicholas Chammas] [SPARK-2470] wrap expression lists in parentheses 22132a4 [Nicholas Chammas] [SPARK-2470] wrap conditionals in parentheses 24639bc [Nicholas Chammas] [SPARK-2470] fix whitespace for doctest 7d557b7 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to tests.py 8f8e4c0 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to storagelevel.py b3b96cf [Nicholas Chammas] [SPARK-2470] PEP8 fixes to statcounter.py d644477 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to worker.py aa3a7b6 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to sql.py 1916859 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to shell.py 95d1d95 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to serializers.py a0fec2e [Nicholas Chammas] [SPARK-2470] PEP8 fixes to mllib c85e1e5 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to join.py d14f2f1 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to __init__.py 81fcb20 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to resultiterable.py 1bde265 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to java_gateway.py 7fc849c [Nicholas Chammas] [SPARK-2470] PEP8 fixes to daemon.py ca2d28b [Nicholas Chammas] [SPARK-2470] PEP8 fixes to context.py f4e0039 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to conf.py a6d5e4b [Nicholas Chammas] [SPARK-2470] PEP8 fixes to cloudpickle.py f0a7ebf [Nicholas Chammas] [SPARK-2470] PEP8 fixes to rddsampler.py 4dd148f [nchammas] Merge pull request #5 from apache/master f7e4581 [Nicholas Chammas] unrelated pep8 fix a36eed0 [Nicholas Chammas] name ec2 instances and security groups consistently de7292a [nchammas] Merge pull request #4 from apache/master 2e4fe00 [nchammas] Merge pull request #3 from apache/master 89fde08 [nchammas] Merge pull request #2 from apache/master 69f6e22 [Nicholas Chammas] PEP8 fixes 2627247 [Nicholas Chammas] broke up lines before they hit 100 chars 6544b7e [Nicholas Chammas] [SPARK-2065] give launched instances names 69da6cf [nchammas] Merge pull request #1 from apache/master
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)
self.assertIn("[2, 3, 4]", out)
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()
|print sc.parallelize([1, 2, 3]).map(myfunc).collect()
""")
zip = self.createFileInZip("mylib.py", """
|def myfunc(x):
| return x + 1
""")
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
proc = subprocess.Popen([self.sparkSubmit, "--py-files", zip, "--master",
"local-cluster[1,1,512]", 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)
self.assertIn("[2, 3, 4]", out)
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()
|print sc.parallelize([1, 2, 3]).map(foo).collect()
""")
# 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(
[self.sparkSubmit, "--master", "local-cluster[1,1,512]", script],
stdout=subprocess.PIPE)
out, err = proc.communicate()
self.assertEqual(0, proc.returncode)
self.assertIn("[2, 4, 6]", out)
class ContextTests(unittest.TestCase):
def test_failed_sparkcontext_creation(self):
# Regression test for SPARK-1550
self.assertRaises(Exception, lambda: SparkContext("an-invalid-master-name"))
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)
@unittest.skipIf(not _have_scipy, "SciPy not installed")
class SciPyTests(PySparkTestCase):
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
"""General PySpark tests that depend on scipy """
def test_serialize(self):
from scipy.special import gammaln
x = range(1, 5)
expected = map(gammaln, x)
observed = self.sc.parallelize(x).map(gammaln).collect()
self.assertEqual(expected, observed)
@unittest.skipIf(not _have_numpy, "NumPy not installed")
class NumPyTests(PySparkTestCase):
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 15:58:24 -04:00
"""General PySpark tests that depend on numpy """
def test_statcounter_array(self):
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
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])])
s = x.stats()
[SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent
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())
2013-01-16 22:15:14 -05:00
if __name__ == "__main__":
if not _have_scipy:
print "NOTE: Skipping SciPy tests as it does not seem to be installed"
if not _have_numpy:
print "NOTE: Skipping NumPy tests as it does not seem to be installed"
2013-01-16 22:15:14 -05:00
unittest.main()
if not _have_scipy:
print "NOTE: SciPy tests were skipped as it does not seem to be installed"
if not _have_numpy:
print "NOTE: NumPy tests were skipped as it does not seem to be installed"