diff --git a/pyspark/pyspark/rdd.py b/pyspark/pyspark/rdd.py index 21dda31c4e435d2af0766d7a5770729374315ab4..cbffb6cc1f2995a14ee53e04281a807cab32c335 100644 --- a/pyspark/pyspark/rdd.py +++ b/pyspark/pyspark/rdd.py @@ -351,10 +351,17 @@ class RDD(object): """ return self.take(1)[0] - # TODO: add test and fix for use with Batch def saveAsTextFile(self, path): """ Save this RDD as a text file, using string representations of elements. + + >>> tempFile = NamedTemporaryFile(delete=True) + >>> tempFile.close() + >>> sc.parallelize(range(10)).saveAsTextFile(tempFile.name) + >>> from fileinput import input + >>> from glob import glob + >>> ''.join(input(glob(tempFile.name + "/part-0000*"))) + '0\\n1\\n2\\n3\\n4\\n5\\n6\\n7\\n8\\n9\\n' """ def func(iterator): return (str(x).encode("utf-8") for x in iterator)