diff --git a/pyspark/.gitignore b/pyspark/.gitignore new file mode 100644 index 0000000000000000000000000000000000000000..5c56e638f923ad45ca0ea3ebe821ef7aa9d09847 --- /dev/null +++ b/pyspark/.gitignore @@ -0,0 +1,2 @@ +*.pyc +docs/ diff --git a/pyspark/pyspark/rdd.py b/pyspark/pyspark/rdd.py index 111476d2740bfa6b0ab79a18e8c40a552ef5ad25..20f84b2dd08e3394db6bfffbc10dd591630993bb 100644 --- a/pyspark/pyspark/rdd.py +++ b/pyspark/pyspark/rdd.py @@ -695,7 +695,9 @@ def _test(): import doctest from pyspark.context import SparkContext globs = globals().copy() - globs['sc'] = SparkContext('local[4]', 'PythonTest') + # The small batch size here ensures that we see multiple batches, + # even in these small test examples: + globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) doctest.testmod(globs=globs) globs['sc'].stop() diff --git a/pyspark/pyspark/serializers.py b/pyspark/pyspark/serializers.py index 4ed925697c3390ef04add09f515923438cf27b60..8b08f7ef0f982173774d1d59abdfb8da28ebb9b5 100644 --- a/pyspark/pyspark/serializers.py +++ b/pyspark/pyspark/serializers.py @@ -24,7 +24,7 @@ def batched(iterator, batchSize): if count == batchSize: yield Batch(items) items = [] - count = [] + count = 0 if items: yield Batch(items)