Skip to content
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
64 changes: 57 additions & 7 deletions python/pyspark/sql/dataframe.py
Original file line number Diff line number Diff line change
Expand Up @@ -659,19 +659,69 @@ def distinct(self):
return DataFrame(self._jdf.distinct(), self.sql_ctx)

@since(1.3)
def sample(self, withReplacement, fraction, seed=None):
def sample(self, withReplacement=None, fraction=None, seed=None):
"""Returns a sampled subset of this :class:`DataFrame`.

:param withReplacement: Sample with replacement or not (default False).
:param fraction: Fraction of rows to generate, range [0.0, 1.0].
:param seed: Seed for sampling (default a random seed).

.. note:: This is not guaranteed to provide exactly the fraction specified of the total
count of the given :class:`DataFrame`.

>>> df.sample(False, 0.5, 42).count()
2
.. note:: `fraction` is required and, `withReplacement` and `seed` are optional.

>>> df = spark.range(10)
>>> df.sample(0.5, 3).count()
4
>>> df.sample(fraction=0.5, seed=3).count()
4
>>> df.sample(withReplacement=True, fraction=0.5, seed=3).count()
1
>>> df.sample(1.0).count()
10
>>> df.sample(fraction=1.0).count()
10
>>> df.sample(False, fraction=1.0).count()
10
"""
assert fraction >= 0.0, "Negative fraction value: %s" % fraction
seed = seed if seed is not None else random.randint(0, sys.maxsize)
rdd = self._jdf.sample(withReplacement, fraction, long(seed))
return DataFrame(rdd, self.sql_ctx)

# For the cases below:
# sample(True, 0.5 [, seed])
# sample(True, fraction=0.5 [, seed])
# sample(withReplacement=False, fraction=0.5 [, seed])
is_withReplacement_set = \
type(withReplacement) == bool and isinstance(fraction, float)

# For the case below:
# sample(faction=0.5 [, seed])
is_withReplacement_omitted_kwargs = \
withReplacement is None and isinstance(fraction, float)

# For the case below:
# sample(0.5 [, seed])
is_withReplacement_omitted_args = isinstance(withReplacement, float)

if not (is_withReplacement_set
or is_withReplacement_omitted_kwargs
or is_withReplacement_omitted_args):
argtypes = [
str(type(arg)) for arg in [withReplacement, fraction, seed] if arg is not None]
raise TypeError(
"withReplacement (optional), fraction (required) and seed (optional)"
" should be a bool, float and number; however, "
"got [%s]." % ", ".join(argtypes))

if is_withReplacement_omitted_args:
if fraction is not None:
seed = fraction
fraction = withReplacement
withReplacement = None

seed = long(seed) if seed is not None else None
args = [arg for arg in [withReplacement, fraction, seed] if arg is not None]
jdf = self._jdf.sample(*args)
return DataFrame(jdf, self.sql_ctx)

@since(1.5)
def sampleBy(self, col, fractions, seed=None):
Expand Down
18 changes: 18 additions & 0 deletions python/pyspark/sql/tests.py
Original file line number Diff line number Diff line change
Expand Up @@ -2083,6 +2083,24 @@ def test_generic_hints(self):
plan = df1.join(df2.hint("broadcast"), "id")._jdf.queryExecution().executedPlan()
self.assertEqual(1, plan.toString().count("BroadcastHashJoin"))

def test_sample(self):
self.assertRaisesRegexp(
TypeError,
"should be a bool, float and number",
lambda: self.spark.range(1).sample())

self.assertRaises(
TypeError,
lambda: self.spark.range(1).sample("a"))

self.assertRaises(
TypeError,
lambda: self.spark.range(1).sample(seed="abc"))

self.assertRaises(
IllegalArgumentException,
lambda: self.spark.range(1).sample(-1.0))

def test_toDF_with_schema_string(self):
data = [Row(key=i, value=str(i)) for i in range(100)]
rdd = self.sc.parallelize(data, 5)
Expand Down
3 changes: 2 additions & 1 deletion sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
Original file line number Diff line number Diff line change
Expand Up @@ -1866,7 +1866,8 @@ class Dataset[T] private[sql](
}

/**
* Returns a new [[Dataset]] by sampling a fraction of rows (without replacement).
* Returns a new [[Dataset]] by sampling a fraction of rows (without replacement),
* using a random seed.
*
* @param fraction Fraction of rows to generate, range [0.0, 1.0].
*
Expand Down