diff --git a/pyspark/pyspark/rdd.py b/pyspark/pyspark/rdd.py
index 85a24c685406a780d286ba7c6b28a4d2628a5a3e..708ea6eb559c5f062721d7ffbc2992c015b15586 100644
--- a/pyspark/pyspark/rdd.py
+++ b/pyspark/pyspark/rdd.py
@@ -145,8 +145,10 @@ class RDD(object):
         self.map(f).collect()  # Force evaluation
 
     def collect(self):
-        pickle = self.ctx.arrayAsPickle(self._jrdd.rdd().collect())
-        return load_pickle(bytes(pickle))
+        def asList(iterator):
+            yield list(iterator)
+        pickles = self.mapPartitions(asList)._jrdd.rdd().collect()
+        return list(chain.from_iterable(load_pickle(bytes(p)) for p in pickles))
 
     def reduce(self, f):
         """
@@ -319,16 +321,23 @@ class RDD(object):
         if numSplits is None:
             numSplits = self.ctx.defaultParallelism
         def add_shuffle_key(iterator):
+            buckets = defaultdict(list)
             for (k, v) in iterator:
-                yield str(hashFunc(k))
-                yield dump_pickle((k, v))
+                buckets[hashFunc(k) % numSplits].append((k, v))
+            for (split, items) in buckets.iteritems():
+                yield str(split)
+                yield dump_pickle(items)
         keyed = PipelinedRDD(self, add_shuffle_key)
         keyed._bypass_serializer = True
         pairRDD = self.ctx.jvm.PairwiseRDD(keyed._jrdd.rdd()).asJavaPairRDD()
         partitioner = self.ctx.jvm.spark.api.python.PythonPartitioner(numSplits)
+        # Transferring O(n) objects to Java is too expensive.  Instead, we'll
+        # form the hash buckets in Python, transferring O(numSplits) objects
+        # to Java.  Each object is a (splitNumber, [objects]) pair.
         jrdd = pairRDD.partitionBy(partitioner)
         jrdd = jrdd.map(self.ctx.jvm.ExtractValue())
-        return RDD(jrdd, self.ctx)
+        # Flatten the resulting RDD:
+        return RDD(jrdd, self.ctx).flatMap(lambda items: items)
 
     def combineByKey(self, createCombiner, mergeValue, mergeCombiners,
                      numSplits=None):