Skip to content

[SPARK-4327] [PySpark] Python API for RDD.randomSplit() #3193

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 9 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
30 changes: 27 additions & 3 deletions python/pyspark/rdd.py
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@
import warnings
import heapq
import bisect
from random import Random
import random
from math import sqrt, log, isinf, isnan

from pyspark.accumulators import PStatsParam
Expand All @@ -38,7 +38,7 @@
from pyspark.join import python_join, python_left_outer_join, \
python_right_outer_join, python_full_outer_join, python_cogroup
from pyspark.statcounter import StatCounter
from pyspark.rddsampler import RDDSampler, RDDStratifiedSampler
from pyspark.rddsampler import RDDSampler, RDDRangeSampler, RDDStratifiedSampler
from pyspark.storagelevel import StorageLevel
from pyspark.resultiterable import ResultIterable
from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, \
Expand Down Expand Up @@ -316,6 +316,30 @@ def sample(self, withReplacement, fraction, seed=None):
assert fraction >= 0.0, "Negative fraction value: %s" % fraction
return self.mapPartitionsWithIndex(RDDSampler(withReplacement, fraction, seed).func, True)

def randomSplit(self, weights, seed=None):
"""
Randomly splits this RDD with the provided weights.

:param weights: weights for splits, will be normalized if they don't sum to 1
:param seed: random seed
:return: split RDDs in a list

>>> rdd = sc.parallelize(range(5), 1)
>>> rdd1, rdd2 = rdd.randomSplit([2, 3], 17)
>>> rdd1.collect()
[1, 3]
>>> rdd2.collect()
[0, 2, 4]
"""
s = float(sum(weights))
cweights = [0.0]
for w in weights:
cweights.append(cweights[-1] + w / s)
if seed is None:
seed = random.randint(0, 2 ** 32 - 1)
return [self.mapPartitionsWithIndex(RDDRangeSampler(lb, ub, seed).func, True)
for lb, ub in zip(cweights, cweights[1:])]

# this is ported from scala/spark/RDD.scala
def takeSample(self, withReplacement, num, seed=None):
"""
Expand All @@ -341,7 +365,7 @@ def takeSample(self, withReplacement, num, seed=None):
if initialCount == 0:
return []

rand = Random(seed)
rand = random.Random(seed)

if (not withReplacement) and num >= initialCount:
# shuffle current RDD and return
Expand Down
14 changes: 14 additions & 0 deletions python/pyspark/rddsampler.py
Original file line number Diff line number Diff line change
Expand Up @@ -115,6 +115,20 @@ def func(self, split, iterator):
yield obj


class RDDRangeSampler(RDDSamplerBase):

def __init__(self, lowerBound, upperBound, seed=None):
RDDSamplerBase.__init__(self, False, seed)
self._use_numpy = False # no performance gain from numpy
self._lowerBound = lowerBound
self._upperBound = upperBound

def func(self, split, iterator):
for obj in iterator:
if self._lowerBound <= self.getUniformSample(split) < self._upperBound:
yield obj


class RDDStratifiedSampler(RDDSamplerBase):

def __init__(self, withReplacement, fractions, seed=None):
Expand Down