Skip to content

Commit

Permalink
add sampleBy to DataFrame
Browse files Browse the repository at this point in the history
  • Loading branch information
mengxr committed Jun 11, 2015
1 parent 7d669a5 commit 832f7cc
Show file tree
Hide file tree
Showing 3 changed files with 67 additions and 6 deletions.
35 changes: 35 additions & 0 deletions python/pyspark/sql/dataframe.py
Original file line number Diff line number Diff line change
Expand Up @@ -444,6 +444,41 @@ def sample(self, withReplacement, fraction, seed=None):
rdd = self._jdf.sample(withReplacement, fraction, long(seed))
return DataFrame(rdd, self.sql_ctx)

@since(1.5)
def sampleBy(self, col, fractions, seed=None):
"""
Returns a stratified sample without replacement based on the
fraction given on each stratum.
:param col: column that defines strata
:param fractions:
sampling fraction for each stratum. If a stratum is not
specified, we treat its fraction as zero.
:param seed: random seed
:return: a new DataFrame that represents the stratified sample
>>> from pyspark.sql.functions import col
>>> dataset = sqlContext.range(0, 100).select((col("id") % 3).alias("key"))
>>> sampled = dataset.sampleBy("key", fractions={0: 0.1, 1: 0.2}, seed=0L)
>>> sampled.groupBy("key").count().orderBy("key").show()
+---+-----+
|key|count|
+---+-----+
| 0| 5|
| 1| 8|
+---+-----+
"""
if not isinstance(col, str):
raise ValueError("col must be a string, but got %r" % type(col))
if not isinstance(fractions, dict):
raise ValueError("fractions must be a dict but got %r" % type(fractions))
for k, v in fractions.items():
if not isinstance(k, (float, int, long, basestring)):
raise ValueError("key must be float, int, long, or string, but got %r" % type(k))
fractions[k] = float(v)
seed = seed if seed is not None else random.randint(0, sys.maxsize)
return DataFrame(self._jdf.sampleBy(col, self._jmap(fractions), seed), self.sql_ctx)

@since(1.4)
def randomSplit(self, weights, seed=None):
"""Randomly splits this :class:`DataFrame` with the provided weights.
Expand Down
30 changes: 24 additions & 6 deletions sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
Original file line number Diff line number Diff line change
Expand Up @@ -18,9 +18,8 @@
package org.apache.spark.sql

import java.io.CharArrayWriter
import java.util.Properties
import java.util.{Properties, UUID}

import scala.collection.JavaConversions._
import scala.language.implicitConversions
import scala.reflect.ClassTag
import scala.reflect.runtime.universe.TypeTag
Expand All @@ -33,19 +32,18 @@ import org.apache.spark.annotation.{DeveloperApi, Experimental}
import org.apache.spark.api.java.JavaRDD
import org.apache.spark.api.python.SerDeUtil
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.analysis.{MultiAlias, ResolvedStar, UnresolvedAttribute, UnresolvedRelation}
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, ScalaReflection, SqlParser}
import org.apache.spark.sql.catalyst.analysis.{MultiAlias, ResolvedStar, UnresolvedAttribute}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical.{Filter, _}
import org.apache.spark.sql.catalyst.plans.{Inner, JoinType}
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, ScalaReflection, SqlParser}
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.execution.{EvaluatePython, ExplainCommand, LogicalRDD}
import org.apache.spark.sql.json.JacksonGenerator
import org.apache.spark.sql.sources.CreateTableUsingAsSelect
import org.apache.spark.sql.types._
import org.apache.spark.storage.StorageLevel
import org.apache.spark.util.Utils


private[sql] object DataFrame {
def apply(sqlContext: SQLContext, logicalPlan: LogicalPlan): DataFrame = {
new DataFrame(sqlContext, logicalPlan)
Expand Down Expand Up @@ -947,6 +945,26 @@ class DataFrame private[sql](
sample(withReplacement, fraction, Utils.random.nextLong)
}

/**
* Returns a stratified sample without replacement based on the fraction given on each stratum.
* @param col column that defines strata
* @param fractions sampling fraction for each stratum. If a stratum is not specified, we treat
* its fraction as zero.
* @param seed random seed
* @return a new [[DataFrame]] that represents the stratified sample
*/
def sampleBy(col: String, fractions: Map[Any, Double], seed: Long): DataFrame = {
require(fractions.values.forall(p => p >= 0.0 && p <= 1.0),
s"Fractions must be in [0, 1], but got $fractions.")
import org.apache.spark.sql.functions.rand
val c = Column(col)
val r = rand(seed).as("rand_" + UUID.randomUUID().toString.take(8))
val expr = fractions.toSeq.map { case (k, v) =>
(c === k) && (r < v)
}.reduce(_ || _) || false
this.filter(expr)
}

/**
* Randomly splits this [[DataFrame]] with the provided weights.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -635,4 +635,12 @@ class DataFrameSuite extends QueryTest {
val res11 = ctx.range(-1).select("id")
assert(res11.count == 0)
}

test("sampleBy") {
val df = ctx.range(0, 100).select((col("id") % 3).as("key"))
val sampled = df.sampleBy("key", Map(0 -> 0.1, 1 -> 0.2), 0L)
checkAnswer(
sampled.groupBy("key").count().orderBy("key"),
Seq(Row(0, 4), Row(1, 9)))
}
}

0 comments on commit 832f7cc

Please sign in to comment.