aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark/rdd.py
diff options
context:
space:
mode:
authorKan Zhang <kzhang@apache.org>2014-07-30 13:19:05 -0700
committerJosh Rosen <joshrosen@apache.org>2014-07-30 13:19:05 -0700
commit94d1f46fc43c0cb85125f757fb40db9271caf1f4 (patch)
tree8878443a963ad6ce5ba3af679567d893c8df70cc /python/pyspark/rdd.py
parent437dc8c5b54f0dcf9564c1fb07e8dce9e771c8cd (diff)
downloadspark-94d1f46fc43c0cb85125f757fb40db9271caf1f4.tar.gz
spark-94d1f46fc43c0cb85125f757fb40db9271caf1f4.tar.bz2
spark-94d1f46fc43c0cb85125f757fb40db9271caf1f4.zip
[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
Diffstat (limited to 'python/pyspark/rdd.py')
-rw-r--r--python/pyspark/rdd.py114
1 files changed, 114 insertions, 0 deletions
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index b84d976114..e8fcc900ef 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -231,6 +231,13 @@ class RDD(object):
self._jrdd_deserializer = jrdd_deserializer
self._id = jrdd.id()
+ def _toPickleSerialization(self):
+ if (self._jrdd_deserializer == PickleSerializer() or
+ self._jrdd_deserializer == BatchedSerializer(PickleSerializer())):
+ return self
+ else:
+ return self._reserialize(BatchedSerializer(PickleSerializer(), 10))
+
def id(self):
"""
A unique ID for this RDD (within its SparkContext).
@@ -1030,6 +1037,113 @@ class RDD(object):
"""
return self.take(1)[0]
+ def saveAsNewAPIHadoopDataset(self, conf, keyConverter=None, valueConverter=None):
+ """
+ Output a Python RDD of key-value pairs (of form C{RDD[(K, V)]}) to any Hadoop file
+ system, using the new Hadoop OutputFormat API (mapreduce package). Keys/values are
+ converted for output using either user specified converters or, by default,
+ L{org.apache.spark.api.python.JavaToWritableConverter}.
+
+ @param conf: Hadoop job configuration, passed in as a dict
+ @param keyConverter: (None by default)
+ @param valueConverter: (None by default)
+ """
+ jconf = self.ctx._dictToJavaMap(conf)
+ pickledRDD = self._toPickleSerialization()
+ batched = isinstance(pickledRDD._jrdd_deserializer, BatchedSerializer)
+ self.ctx._jvm.PythonRDD.saveAsHadoopDataset(pickledRDD._jrdd, batched, jconf,
+ keyConverter, valueConverter, True)
+
+ def saveAsNewAPIHadoopFile(self, path, outputFormatClass, keyClass=None, valueClass=None,
+ keyConverter=None, valueConverter=None, conf=None):
+ """
+ Output a Python RDD of key-value pairs (of form C{RDD[(K, V)]}) to any Hadoop file
+ system, using the new Hadoop OutputFormat API (mapreduce package). Key and value types
+ will be inferred if not specified. Keys and values are converted for output using either
+ user specified converters or L{org.apache.spark.api.python.JavaToWritableConverter}. The
+ C{conf} is applied on top of the base Hadoop conf associated with the SparkContext
+ of this RDD to create a merged Hadoop MapReduce job configuration for saving the data.
+
+ @param path: path to Hadoop file
+ @param outputFormatClass: fully qualified classname of Hadoop OutputFormat
+ (e.g. "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat")
+ @param keyClass: fully qualified classname of key Writable class
+ (e.g. "org.apache.hadoop.io.IntWritable", None by default)
+ @param valueClass: fully qualified classname of value Writable class
+ (e.g. "org.apache.hadoop.io.Text", None by default)
+ @param keyConverter: (None by default)
+ @param valueConverter: (None by default)
+ @param conf: Hadoop job configuration, passed in as a dict (None by default)
+ """
+ jconf = self.ctx._dictToJavaMap(conf)
+ pickledRDD = self._toPickleSerialization()
+ batched = isinstance(pickledRDD._jrdd_deserializer, BatchedSerializer)
+ self.ctx._jvm.PythonRDD.saveAsNewAPIHadoopFile(pickledRDD._jrdd, batched, path,
+ outputFormatClass, keyClass, valueClass, keyConverter, valueConverter, jconf)
+
+ def saveAsHadoopDataset(self, conf, keyConverter=None, valueConverter=None):
+ """
+ Output a Python RDD of key-value pairs (of form C{RDD[(K, V)]}) to any Hadoop file
+ system, using the old Hadoop OutputFormat API (mapred package). Keys/values are
+ converted for output using either user specified converters or, by default,
+ L{org.apache.spark.api.python.JavaToWritableConverter}.
+
+ @param conf: Hadoop job configuration, passed in as a dict
+ @param keyConverter: (None by default)
+ @param valueConverter: (None by default)
+ """
+ jconf = self.ctx._dictToJavaMap(conf)
+ pickledRDD = self._toPickleSerialization()
+ batched = isinstance(pickledRDD._jrdd_deserializer, BatchedSerializer)
+ self.ctx._jvm.PythonRDD.saveAsHadoopDataset(pickledRDD._jrdd, batched, jconf,
+ keyConverter, valueConverter, False)
+
+ def saveAsHadoopFile(self, path, outputFormatClass, keyClass=None, valueClass=None,
+ keyConverter=None, valueConverter=None, conf=None,
+ compressionCodecClass=None):
+ """
+ Output a Python RDD of key-value pairs (of form C{RDD[(K, V)]}) to any Hadoop file
+ system, using the old Hadoop OutputFormat API (mapred package). Key and value types
+ will be inferred if not specified. Keys and values are converted for output using either
+ user specified converters or L{org.apache.spark.api.python.JavaToWritableConverter}. The
+ C{conf} is applied on top of the base Hadoop conf associated with the SparkContext
+ of this RDD to create a merged Hadoop MapReduce job configuration for saving the data.
+
+ @param path: path to Hadoop file
+ @param outputFormatClass: fully qualified classname of Hadoop OutputFormat
+ (e.g. "org.apache.hadoop.mapred.SequenceFileOutputFormat")
+ @param keyClass: fully qualified classname of key Writable class
+ (e.g. "org.apache.hadoop.io.IntWritable", None by default)
+ @param valueClass: fully qualified classname of value Writable class
+ (e.g. "org.apache.hadoop.io.Text", None by default)
+ @param keyConverter: (None by default)
+ @param valueConverter: (None by default)
+ @param conf: (None by default)
+ @param compressionCodecClass: (None by default)
+ """
+ jconf = self.ctx._dictToJavaMap(conf)
+ pickledRDD = self._toPickleSerialization()
+ batched = isinstance(pickledRDD._jrdd_deserializer, BatchedSerializer)
+ self.ctx._jvm.PythonRDD.saveAsHadoopFile(pickledRDD._jrdd, batched, path,
+ outputFormatClass, keyClass, valueClass, keyConverter, valueConverter,
+ jconf, compressionCodecClass)
+
+ def saveAsSequenceFile(self, path, compressionCodecClass=None):
+ """
+ Output a Python RDD of key-value pairs (of form C{RDD[(K, V)]}) to any Hadoop file
+ system, using the L{org.apache.hadoop.io.Writable} types that we convert from the
+ RDD's key and value types. The mechanism is as follows:
+ 1. Pyrolite is used to convert pickled Python RDD into RDD of Java objects.
+ 2. Keys and values of this Java RDD are converted to Writables and written out.
+
+ @param path: path to sequence file
+ @param compressionCodecClass: (None by default)
+ """
+ pickledRDD = self._toPickleSerialization()
+ batched = isinstance(pickledRDD._jrdd_deserializer, BatchedSerializer)
+ self.ctx._jvm.PythonRDD.saveAsSequenceFile(pickledRDD._jrdd, batched,
+ path, compressionCodecClass)
+
def saveAsPickleFile(self, path, batchSize=10):
"""
Save this RDD as a SequenceFile of serialized objects. The serializer