From 14419775202e6eef1f0e1f0c74c7be9030aca73d Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Thu, 29 May 2014 15:22:14 -0700 Subject: [PATCH 01/28] SPARK-1939 Refactor takeSample method in RDD to use ScaSRS --- core/pom.xml | 4 ++ .../main/scala/org/apache/spark/rdd/RDD.scala | 32 +++++++++- .../spark/util/random/RandomSampler.scala | 2 +- .../scala/org/apache/spark/rdd/RDDSuite.scala | 63 ++++++++++++++----- pom.xml | 5 ++ project/SparkBuild.scala | 1 + python/pyspark/rdd.py | 15 ++++- 7 files changed, 100 insertions(+), 22 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index bab50f5ce288..6cb58dbd291c 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -67,6 +67,10 @@ org.apache.commons commons-lang3 + + org.apache.commons + commons-math3 + com.google.code.findbugs jsr305 diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index aa03e9276fb3..2fdf45a0c8b8 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -379,8 +379,17 @@ abstract class RDD[T: ClassTag]( }.toArray } - def takeSample(withReplacement: Boolean, num: Int, seed: Long = Utils.random.nextLong): Array[T] = - { + /** + * Return a fixed-size sampled subset of this RDD in an array + * + * @param withReplacement whether sampling is done with replacement + * @param num size of the returned sample + * @param seed seed for the random number generator + * @return sample of specified size in an array + */ + def takeSample(withReplacement: Boolean, + num: Int, + seed: Long = Utils.random.nextLong): Array[T] = { var fraction = 0.0 var total = 0 val multiplier = 3.0 @@ -402,10 +411,11 @@ abstract class RDD[T: ClassTag]( } if (num > initialCount && !withReplacement) { + // special case not covered in computeFraction total = maxSelected fraction = multiplier * (maxSelected + 1) / initialCount } else { - fraction = multiplier * (num + 1) / initialCount + fraction = computeFraction(num, initialCount, withReplacement) total = num } @@ -421,6 +431,22 @@ abstract class RDD[T: ClassTag]( Utils.randomizeInPlace(samples, rand).take(total) } + private[spark] def computeFraction(num: Int, total: Long, withReplacement: Boolean) : Double = { + val fraction = num.toDouble / total + if (withReplacement) { + var numStDev = 5 + if (num < 12) { + // special case to guarantee sample size for small s + numStDev = 9 + } + fraction + numStDev * math.sqrt(fraction / total) + } else { + val delta = 0.00005 + val gamma = - math.log(delta)/total + math.min(1, fraction + gamma + math.sqrt(gamma * gamma + 2 * gamma * fraction)) + } + } + /** * Return the union of this RDD and another one. Any identical elements will appear multiple * times (use `.distinct()` to eliminate them). diff --git a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala index 4dc8ada00a3e..e53103755b27 100644 --- a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala +++ b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala @@ -70,7 +70,7 @@ class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) } /** - * Return a sampler with is the complement of the range specified of the current sampler. + * Return a sampler which is the complement of the range specified of the current sampler. */ def cloneComplement(): BernoulliSampler[T] = new BernoulliSampler[T](lb, ub, !complement) diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index e686068f7a99..5bdcb9bef6d6 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -22,6 +22,7 @@ import scala.reflect.ClassTag import org.scalatest.FunSuite +import org.apache.commons.math3.distribution.PoissonDistribution import org.apache.spark._ import org.apache.spark.SparkContext._ import org.apache.spark.rdd._ @@ -494,56 +495,84 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(sortedTopK === nums.sorted(ord).take(5)) } + test("computeFraction") { + // test that the computed fraction guarantees enough datapoints in the sample with a failure rate <= 0.0001 + val data = new EmptyRDD[Int](sc) + val n = 100000 + + for (s <- 1 to 15) { + val frac = data.computeFraction(s, n, true) + val qpois = new PoissonDistribution(frac * n) + assert(qpois.inverseCumulativeProbability(0.0001) >= s, "Computed fraction is too low") + } + for (s <- 1 to 15) { + val frac = data.computeFraction(s, n, false) + val qpois = new PoissonDistribution(frac * n) + assert(qpois.inverseCumulativeProbability(0.0001) >= s, "Computed fraction is too low") + } + for (s <- List(1, 10, 100, 1000)) { + val frac = data.computeFraction(s, n, true) + val qpois = new PoissonDistribution(frac * n) + assert(qpois.inverseCumulativeProbability(0.0001) >= s, "Computed fraction is too low") + } + for (s <- List(1, 10, 100, 1000)) { + val frac = data.computeFraction(s, n, false) + val qpois = new PoissonDistribution(frac * n) + assert(qpois.inverseCumulativeProbability(0.0001) >= s, "Computed fraction is too low") + } + } + test("takeSample") { - val data = sc.parallelize(1 to 100, 2) + val n = 1000000 + val data = sc.parallelize(1 to n, 2) for (num <- List(5, 20, 100)) { val sample = data.takeSample(withReplacement=false, num=num) assert(sample.size === num) // Got exactly num elements assert(sample.toSet.size === num) // Elements are distinct - assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") + assert(sample.forall(x => 1 <= x && x <= n), "elements not in [1, 100]") } for (seed <- 1 to 5) { val sample = data.takeSample(withReplacement=false, 20, seed) assert(sample.size === 20) // Got exactly 20 elements assert(sample.toSet.size === 20) // Elements are distinct - assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") + assert(sample.forall(x => 1 <= x && x <= n), "elements not in [1, 100]") } for (seed <- 1 to 5) { - val sample = data.takeSample(withReplacement=false, 200, seed) + val sample = data.takeSample(withReplacement=false, 100, seed) assert(sample.size === 100) // Got only 100 elements assert(sample.toSet.size === 100) // Elements are distinct - assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") + assert(sample.forall(x => 1 <= x && x <= n), "elements not in [1, 100]") } for (seed <- 1 to 5) { val sample = data.takeSample(withReplacement=true, 20, seed) assert(sample.size === 20) // Got exactly 20 elements - assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") + assert(sample.forall(x => 1 <= x && x <= n), "elements not in [1, 100]") } { val sample = data.takeSample(withReplacement=true, num=20) assert(sample.size === 20) // Got exactly 100 elements assert(sample.toSet.size <= 20, "sampling with replacement returned all distinct elements") - assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") + assert(sample.forall(x => 1 <= x && x <= n), "elements not in [1, 100]") } { - val sample = data.takeSample(withReplacement=true, num=100) - assert(sample.size === 100) // Got exactly 100 elements + val sample = data.takeSample(withReplacement=true, num=n) + assert(sample.size === n) // Got exactly 100 elements // Chance of getting all distinct elements is astronomically low, so test we got < 100 - assert(sample.toSet.size < 100, "sampling with replacement returned all distinct elements") - assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") + assert(sample.toSet.size < n, "sampling with replacement returned all distinct elements") + assert(sample.forall(x => 1 <= x && x <= n), "elements not in [1, 100]") } for (seed <- 1 to 5) { - val sample = data.takeSample(withReplacement=true, 100, seed) - assert(sample.size === 100) // Got exactly 100 elements + val sample = data.takeSample(withReplacement=true, n, seed) + assert(sample.size === n) // Got exactly 100 elements // Chance of getting all distinct elements is astronomically low, so test we got < 100 - assert(sample.toSet.size < 100, "sampling with replacement returned all distinct elements") + assert(sample.toSet.size < n, "sampling with replacement returned all distinct elements") } for (seed <- 1 to 5) { - val sample = data.takeSample(withReplacement=true, 200, seed) - assert(sample.size === 200) // Got exactly 200 elements + val sample = data.takeSample(withReplacement=true, 2*n, seed) + assert(sample.size === 2*n) // Got exactly 200 elements // Chance of getting all distinct elements is still quite low, so test we got < 100 - assert(sample.toSet.size < 100, "sampling with replacement returned all distinct elements") + assert(sample.toSet.size < n, "sampling with replacement returned all distinct elements") } } diff --git a/pom.xml b/pom.xml index 7bf9f135fd34..01d6eef32be6 100644 --- a/pom.xml +++ b/pom.xml @@ -245,6 +245,11 @@ commons-codec 1.5 + + org.apache.commons + commons-math3 + 3.2 + com.google.code.findbugs jsr305 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 8ef1e91f609f..a6b6c26a4939 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -331,6 +331,7 @@ object SparkBuild extends Build { libraryDependencies ++= Seq( "com.google.guava" % "guava" % "14.0.1", "org.apache.commons" % "commons-lang3" % "3.3.2", + "org.apache.commons" % "commons-math3" % "3.2", "com.google.code.findbugs" % "jsr305" % "1.3.9", "log4j" % "log4j" % "1.2.17", "org.slf4j" % "slf4j-api" % slf4jVersion, diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 07578b8d937f..b400404ad97c 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -31,6 +31,7 @@ import warnings import heapq from random import Random +from math import sqrt, log, min from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ BatchedSerializer, CloudPickleSerializer, PairDeserializer, pack_long @@ -374,7 +375,7 @@ def takeSample(self, withReplacement, num, seed=None): total = maxSelected fraction = multiplier * (maxSelected + 1) / initialCount else: - fraction = multiplier * (num + 1) / initialCount + fraction = self._computeFraction(num, initialCount, withReplacement) total = num samples = self.sample(withReplacement, fraction, seed).collect() @@ -390,6 +391,18 @@ def takeSample(self, withReplacement, num, seed=None): sampler.shuffle(samples) return samples[0:total] + def _computeFraction(self, num, total, withReplacement): + fraction = float(num)/total + if withReplacement: + numStDev = 5 + if (num < 12): + numStDev = 9 + return fraction + numStDev * sqrt(fraction/total) + else: + delta = 0.00005 + gamma = - log(delta)/total + return min(1, fraction + gamma + sqrt(gamma * gamma + 2* gamma * fraction)) + def union(self, other): """ Return the union of this RDD and another one. From ffea61a67d228edb476d29ca13a84bb3f9a22887 Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Thu, 29 May 2014 17:55:54 -0700 Subject: [PATCH 02/28] SPARK-1939: Refactor takeSample method in RDD Reviewer comments addressed: - commons-math3 is now a test-only dependency. bumped up to v3.3 - comments added to explain what computeFraction is doing - fixed the unit for computeFraction to use BinomialDitro for without replacement sampling - stylistic fixes --- core/pom.xml | 1 + .../main/scala/org/apache/spark/rdd/RDD.scala | 33 +++++++++++------ .../spark/util/random/RandomSampler.scala | 2 +- .../scala/org/apache/spark/rdd/RDDSuite.scala | 36 +++++++++---------- pom.xml | 2 +- project/SparkBuild.scala | 2 +- 6 files changed, 44 insertions(+), 32 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 6cb58dbd291c..2b9f750e07d9 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -70,6 +70,7 @@ org.apache.commons commons-math3 + test com.google.code.findbugs diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 2fdf45a0c8b8..9a5cf13e3ba5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -388,8 +388,8 @@ abstract class RDD[T: ClassTag]( * @return sample of specified size in an array */ def takeSample(withReplacement: Boolean, - num: Int, - seed: Long = Utils.random.nextLong): Array[T] = { + num: Int, + seed: Long = Utils.random.nextLong): Array[T] = { var fraction = 0.0 var total = 0 val multiplier = 3.0 @@ -431,18 +431,31 @@ abstract class RDD[T: ClassTag]( Utils.randomizeInPlace(samples, rand).take(total) } - private[spark] def computeFraction(num: Int, total: Long, withReplacement: Boolean) : Double = { + /** + * Let p = num / total, where num is the sample size and total is the total number of + * datapoints in the RDD. We're trying to compute q > p such that + * - when sampling with replacement, we're drawing each datapoint with prob_i ~ Pois(q), + * where we want to guarantee Pr[s < num] < 0.0001 for s = sum(prob_i for i from 0 to total), + * i.e. the failure rate of not having a sufficiently large sample < 0.0001. + * Setting q = p + 5 * sqrt(p/total) is sufficient to guarantee 0.9999 success rate for + * num > 12, but we need a slightly larger q (9 empirically determined). + * - when sampling without replacement, we're drawing each datapoint with prob_i + * ~ Binomial(total, fraction) and our choice of q guarantees 1-delta, or 0.9999 success + * rate, where success rate is defined the same as in sampling with replacement. + * + * @param num sample size + * @param total size of RDD + * @param withReplacement whether sampling with replacement + * @return a sampling rate that guarantees sufficient sample size with 99.99% success rate + */ + private[rdd] def computeFraction(num: Int, total: Long, withReplacement: Boolean): Double = { val fraction = num.toDouble / total if (withReplacement) { - var numStDev = 5 - if (num < 12) { - // special case to guarantee sample size for small s - numStDev = 9 - } + val numStDev = if (num < 12) 9 else 5 fraction + numStDev * math.sqrt(fraction / total) } else { - val delta = 0.00005 - val gamma = - math.log(delta)/total + val delta = 1e-4 + val gamma = - math.log(delta) / total math.min(1, fraction + gamma + math.sqrt(gamma * gamma + 2 * gamma * fraction)) } } diff --git a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala index e53103755b27..247f10173f1e 100644 --- a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala +++ b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala @@ -70,7 +70,7 @@ class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) } /** - * Return a sampler which is the complement of the range specified of the current sampler. + * Return a sampler that is the complement of the range specified of the current sampler. */ def cloneComplement(): BernoulliSampler[T] = new BernoulliSampler[T](lb, ub, !complement) diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 5bdcb9bef6d6..08b3b93d6a31 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -22,7 +22,9 @@ import scala.reflect.ClassTag import org.scalatest.FunSuite +import org.apache.commons.math3.distribution.BinomialDistribution import org.apache.commons.math3.distribution.PoissonDistribution + import org.apache.spark._ import org.apache.spark.SparkContext._ import org.apache.spark.rdd._ @@ -496,29 +498,25 @@ class RDDSuite extends FunSuite with SharedSparkContext { } test("computeFraction") { - // test that the computed fraction guarantees enough datapoints in the sample with a failure rate <= 0.0001 + // test that the computed fraction guarantees enough datapoints + // in the sample with a failure rate <= 0.0001 val data = new EmptyRDD[Int](sc) val n = 100000 for (s <- 1 to 15) { val frac = data.computeFraction(s, n, true) - val qpois = new PoissonDistribution(frac * n) - assert(qpois.inverseCumulativeProbability(0.0001) >= s, "Computed fraction is too low") + val poisson = new PoissonDistribution(frac * n) + assert(poisson.inverseCumulativeProbability(0.0001) >= s, "Computed fraction is too low") } - for (s <- 1 to 15) { - val frac = data.computeFraction(s, n, false) - val qpois = new PoissonDistribution(frac * n) - assert(qpois.inverseCumulativeProbability(0.0001) >= s, "Computed fraction is too low") - } - for (s <- List(1, 10, 100, 1000)) { + for (s <- List(20, 100, 1000)) { val frac = data.computeFraction(s, n, true) - val qpois = new PoissonDistribution(frac * n) - assert(qpois.inverseCumulativeProbability(0.0001) >= s, "Computed fraction is too low") + val poisson = new PoissonDistribution(frac * n) + assert(poisson.inverseCumulativeProbability(0.0001) >= s, "Computed fraction is too low") } for (s <- List(1, 10, 100, 1000)) { val frac = data.computeFraction(s, n, false) - val qpois = new PoissonDistribution(frac * n) - assert(qpois.inverseCumulativeProbability(0.0001) >= s, "Computed fraction is too low") + val binomial = new BinomialDistribution(n, frac) + assert(binomial.inverseCumulativeProbability(0.0001)*n >= s, "Computed fraction is too low") } } @@ -530,37 +528,37 @@ class RDDSuite extends FunSuite with SharedSparkContext { val sample = data.takeSample(withReplacement=false, num=num) assert(sample.size === num) // Got exactly num elements assert(sample.toSet.size === num) // Elements are distinct - assert(sample.forall(x => 1 <= x && x <= n), "elements not in [1, 100]") + assert(sample.forall(x => 1 <= x && x <= n), s"elements not in [1, $n]") } for (seed <- 1 to 5) { val sample = data.takeSample(withReplacement=false, 20, seed) assert(sample.size === 20) // Got exactly 20 elements assert(sample.toSet.size === 20) // Elements are distinct - assert(sample.forall(x => 1 <= x && x <= n), "elements not in [1, 100]") + assert(sample.forall(x => 1 <= x && x <= n), s"elements not in [1, $n]") } for (seed <- 1 to 5) { val sample = data.takeSample(withReplacement=false, 100, seed) assert(sample.size === 100) // Got only 100 elements assert(sample.toSet.size === 100) // Elements are distinct - assert(sample.forall(x => 1 <= x && x <= n), "elements not in [1, 100]") + assert(sample.forall(x => 1 <= x && x <= n), s"elements not in [1, $n]") } for (seed <- 1 to 5) { val sample = data.takeSample(withReplacement=true, 20, seed) assert(sample.size === 20) // Got exactly 20 elements - assert(sample.forall(x => 1 <= x && x <= n), "elements not in [1, 100]") + assert(sample.forall(x => 1 <= x && x <= n), s"elements not in [1, $n]") } { val sample = data.takeSample(withReplacement=true, num=20) assert(sample.size === 20) // Got exactly 100 elements assert(sample.toSet.size <= 20, "sampling with replacement returned all distinct elements") - assert(sample.forall(x => 1 <= x && x <= n), "elements not in [1, 100]") + assert(sample.forall(x => 1 <= x && x <= n), s"elements not in [1, $n]") } { val sample = data.takeSample(withReplacement=true, num=n) assert(sample.size === n) // Got exactly 100 elements // Chance of getting all distinct elements is astronomically low, so test we got < 100 assert(sample.toSet.size < n, "sampling with replacement returned all distinct elements") - assert(sample.forall(x => 1 <= x && x <= n), "elements not in [1, 100]") + assert(sample.forall(x => 1 <= x && x <= n), s"elements not in [1, $n]") } for (seed <- 1 to 5) { val sample = data.takeSample(withReplacement=true, n, seed) diff --git a/pom.xml b/pom.xml index 01d6eef32be6..64c8cd3c7810 100644 --- a/pom.xml +++ b/pom.xml @@ -248,7 +248,7 @@ org.apache.commons commons-math3 - 3.2 + 3.3 com.google.code.findbugs diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index a6b6c26a4939..7314168fa8b6 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -331,7 +331,7 @@ object SparkBuild extends Build { libraryDependencies ++= Seq( "com.google.guava" % "guava" % "14.0.1", "org.apache.commons" % "commons-lang3" % "3.3.2", - "org.apache.commons" % "commons-math3" % "3.2", + "org.apache.commons" % "commons-math3" % "3.3" % "test", "com.google.code.findbugs" % "jsr305" % "1.3.9", "log4j" % "log4j" % "1.2.17", "org.slf4j" % "slf4j-api" % slf4jVersion, From 7cab53a3926f4351432e5e3600b0796b9a4146e4 Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Mon, 2 Jun 2014 12:00:38 -0700 Subject: [PATCH 03/28] fixed import bug in rdd.py --- python/pyspark/rdd.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index b400404ad97c..8f46e448e643 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -31,7 +31,7 @@ import warnings import heapq from random import Random -from math import sqrt, log, min +from math import sqrt, log from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ BatchedSerializer, CloudPickleSerializer, PairDeserializer, pack_long From 9ee94ee3c28e8d808063fef4e5d39f06ab738e0b Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Mon, 9 Jun 2014 13:15:23 -0700 Subject: [PATCH 04/28] [SPARK-2082] stratified sampling in PairRDDFunctions that guarantees exact sample size --- core/pom.xml | 1 - .../apache/spark/rdd/PairRDDFunctions.scala | 186 +++++++++++++++++- .../main/scala/org/apache/spark/rdd/RDD.scala | 18 ++ .../spark/util/random/SamplingUtils.scala | 56 ++++++ .../spark/rdd/PairRDDFunctionsSuite.scala | 88 ++++++++- .../scala/org/apache/spark/rdd/RDDSuite.scala | 32 +++ project/SparkBuild.scala | 2 +- 7 files changed, 379 insertions(+), 4 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala diff --git a/core/pom.xml b/core/pom.xml index e928cc556b55..96d1ca5a4cd7 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -70,7 +70,6 @@ org.apache.commons commons-math3 - test com.google.code.findbugs diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 223fef79261d..53219d1a94b7 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -27,8 +27,12 @@ import scala.collection.Map import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag +import scala.util.control.Breaks._ import com.clearspring.analytics.stream.cardinality.HyperLogLog + +import org.apache.commons.math3.random.RandomDataGenerator + import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.io.SequenceFile.CompressionType @@ -46,7 +50,8 @@ import org.apache.spark.Partitioner.defaultPartitioner import org.apache.spark.SparkContext._ import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.serializer.Serializer -import org.apache.spark.util.SerializableHyperLogLog +import org.apache.spark.util.{Utils, SerializableHyperLogLog} +import org.apache.spark.util.random.{PoissonBounds => PB} /** * Extra functions available on RDDs of (key, value) pairs through an implicit conversion. @@ -155,6 +160,182 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) foldByKey(zeroValue, defaultPartitioner(self))(func) } + /** + * Return a subset of this RDD sampled by key (via stratified sampling). + * We guarantee a sample size = math.ceil(fraction * S_i), where S_i is the size of the ith + * stratum. + * + * @param withReplacement whether to sample with or without replacement + * @param fraction sampling rate + * @param seed seed for the random number generator + * @return RDD containing the sampled subset + */ + def sampleByKey(withReplacement: Boolean, + fraction: Double, + seed: Long = Utils.random.nextLong): RDD[(K, V)]= { + + class Stratum(var numItems: Long = 0L, var numAccepted: Long = 0L) extends Serializable { + var waitList: ArrayBuffer[Double] = new ArrayBuffer[Double] + var q1: Option[Double] = None + var q2: Option[Double] = None + + def incrNumItems(by: Long = 1L) = numItems += by + + def incrNumAccepted(by: Long = 1L) = numAccepted += by + + def addToWaitList(elem: Double) = waitList += elem + + def addToWaitList(elems: ArrayBuffer[Double]) = waitList ++= elems + + override def toString() = { + "numItems: " + numItems + " numAccepted: " + numAccepted + " q1: " + q1 + " q2: " + q2 + + " waitListSize:" + waitList.size + } + } + + class Result(var resultMap: Map[K, Stratum], var cachedPartitionId: Option[Int] = None) + extends Serializable { + var rand: RandomDataGenerator = new RandomDataGenerator + + def getEntry(key: K, numItems: Long = 0L): Stratum = { + if (resultMap.get(key).isEmpty) { + resultMap += (key -> new Stratum(numItems)) + } + resultMap.get(key).get + } + + def getRand(partitionId: Int): RandomDataGenerator = { + if (cachedPartitionId.isEmpty || cachedPartitionId.get != partitionId) { + cachedPartitionId = Some(partitionId) + rand.reSeed(seed + partitionId) + } + rand + } + } + + // TODO implement the streaming version of sampling w/ replacement that doesn't require counts + // in order to save one pass over the RDD + val counts = if (withReplacement) Some(this.countByKey()) else None + + val seqOp = (U: (TaskContext, Result), item: (K, V)) => { + val delta = 5e-5 + val result = U._2 + val tc = U._1 + val rng = result.getRand(tc.partitionId) + val stratum = result.getEntry(item._1) + if (withReplacement) { + // compute q1 and q2 only if they haven't been computed already + // since they don't change from iteration to iteration. + // TODO change this to the streaming version + if (stratum.q1.isEmpty || stratum.q2.isEmpty) { + val n = counts.get(item._1) + val s = math.ceil(n * fraction).toLong + val lmbd1 = PB.getLambda1(s) + val minCount = PB.getMinCount(lmbd1) + val lmbd2 = if (lmbd1 == 0) PB.getLambda2(s) else PB.getLambda2(s - minCount) + val q1 = lmbd1 / n + val q2 = lmbd2 / n + stratum.q1 = Some(q1) + stratum.q2 = Some(q2) + } + val x1 = if (stratum.q1.get == 0) 0L else rng.nextPoisson(stratum.q1.get) + if (x1 > 0) { + stratum.incrNumAccepted(x1) + } + val x2 = rng.nextPoisson(stratum.q2.get).toInt + if (x2 > 0) { + stratum.addToWaitList(ArrayBuffer.fill(x2)(rng.nextUniform(0.0, 1.0))) + } + } else { + val g1 = - math.log(delta) / stratum.numItems + val g2 = (2.0 / 3.0) * g1 + val q1 = math.max(0, fraction + g2 - math.sqrt((g2 * g2 + 3 * g2 * fraction))) + val q2 = math.min(1, fraction + g1 + math.sqrt(g1 * g1 + 2 * g1 * fraction)) + + val x = rng.nextUniform(0.0, 1.0) + if (x < q1) { + stratum.incrNumAccepted() + } else if ( x < q2) { + stratum.addToWaitList(x) + } + stratum.q1 = Some(q1) + stratum.q2 = Some(q2) + } + stratum.incrNumItems() + result + } + + val combOp = (r1: Result, r2: Result) => { + //take union of both key sets in case one partion doesn't contain all keys + val keyUnion = r1.resultMap.keys.toSet.union(r2.resultMap.keys.toSet) + + //Use r2 to keep the combined result since r1 is usual empty + for (key <- keyUnion) { + val entry1 = r1.resultMap.get(key) + val entry2 = r2.resultMap.get(key) + if (entry2.isEmpty && entry1.isDefined) { + r2.resultMap += (key -> entry1.get) + } else if (entry1.isDefined && entry2.isDefined) { + entry2.get.addToWaitList(entry1.get.waitList) + entry2.get.incrNumAccepted(entry1.get.numAccepted) + entry2.get.incrNumItems(entry1.get.numItems) + } + } + r2 + } + + val zeroU = new Result(Map[K, Stratum]()) + + //determine threshold for each stratum and resample + val finalResult = self.aggregateWithContext(zeroU)(seqOp, combOp).resultMap + val thresholdByKey = new mutable.HashMap[K, Double]() + for ((key, stratum) <- finalResult) { + val s = math.ceil(stratum.numItems * fraction).toLong + breakable { + if (stratum.numAccepted > s) { + logWarning("Pre-accepted too many") + thresholdByKey += (key -> stratum.q1.get) + break + } + val numWaitListAccepted = (s - stratum.numAccepted).toInt + if (numWaitListAccepted >= stratum.waitList.size) { + logWarning("WaitList too short") + thresholdByKey += (key -> stratum.q2.get) + } else { + thresholdByKey += (key -> stratum.waitList.sorted.apply(numWaitListAccepted)) + } + } + } + + if (withReplacement) { + // Poisson sampler + self.mapPartitionsWithIndex((idx: Int, iter: Iterator[(K, V)]) => { + val random = new RandomDataGenerator() + random.reSeed(seed + idx) + iter.flatMap { t => + val q1 = finalResult.get(t._1).get.q1.get + val q2 = finalResult.get(t._1).get.q2.get + val x1 = if (q1 == 0) 0L else random.nextPoisson(q1) + val x2 = random.nextPoisson(q2).toInt + val x = x1 + (0 until x2).filter(i => random.nextUniform(0.0, 1.0) < + thresholdByKey.get(t._1).get).size + if (x > 0) { + Iterator.fill(x.toInt)(t) + } else { + Iterator.empty + } + } + }, preservesPartitioning = true) + } else { + // Bernoulli sampler + self.mapPartitionsWithIndex((idx: Int, iter: Iterator[(K, V)]) => { + val random = new RandomDataGenerator + random.reSeed(seed+idx) + iter.filter(t => random.nextUniform(0.0, 1.0) < thresholdByKey.get(t._1).get) + }, preservesPartitioning = true) + } + } + /** * Merge the values for each key using an associative reduce function. This will also perform * the merging locally on each mapper before sending results to a reducer, similarly to a @@ -442,6 +623,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) /** * Return the key-value pairs in this RDD to the master as a Map. + * + * Warning: this doesn't return a multimap (so if you have multiple values to the same key, only + * one value per key is preserved in the map returned) */ def collectAsMap(): Map[K, V] = { val data = self.collect() diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 9a5cf13e3ba5..76418283486a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -880,6 +880,24 @@ abstract class RDD[T: ClassTag]( jobResult } + /** + * A version of {@link #aggregate()} that passes the TaskContext to the function that does + * aggregation for each partition. + */ + def aggregateWithContext[U: ClassTag](zeroValue: U)(seqOp: ((TaskContext, U), T) => U, combOp: (U, U) => U): U = { + // Clone the zero value since we will also be serializing it as part of tasks + var jobResult = Utils.clone(zeroValue, sc.env.closureSerializer.newInstance()) + //pad seqOp and combOp with taskContext to conform to aggregate's signature in TraversableOnce + val paddedSeqOp = (arg1: (TaskContext, U), item: T) => (arg1._1, seqOp(arg1, item)) + val paddedcombOp = (arg1 : (TaskContext, U), arg2: (TaskContext, U)) => (arg1._1, combOp(arg1._2, arg1._2)) + val cleanSeqOp = sc.clean(paddedSeqOp) + val cleanCombOp = sc.clean(paddedcombOp) + val aggregatePartition = (tc: TaskContext, it: Iterator[T]) => (it.aggregate(tc, zeroValue)(cleanSeqOp, cleanCombOp))._2 + val mergeResult = (index: Int, taskResult: U) => jobResult = combOp(jobResult, taskResult) + sc.runJob(this, aggregatePartition, mergeResult) + jobResult + } + /** * Return the number of elements in the RDD. */ diff --git a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala new file mode 100644 index 000000000000..fee92402810c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.random + +import org.apache.commons.math3.distribution.{PoissonDistribution, NormalDistribution} + +private[random] object PoissonBounds { + + val delta = 1e-4 / 3.0 + val phi = new NormalDistribution().cumulativeProbability(1.0 - delta) + + def getLambda1(s: Double): Double = { + var lb = math.max(0.0, s - math.sqrt(s / delta)) // Chebyshev's inequality + var ub = s + while (lb < ub - 1.0) { + val m = (lb + ub) / 2.0 + val poisson = new PoissonDistribution(m, 1e-15) + val y = poisson.inverseCumulativeProbability(1 - delta) + if (y > s) ub = m else lb = m + } + lb + } + + def getMinCount(lmbd: Double): Double = { + if(lmbd == 0) return 0 + val poisson = new PoissonDistribution(lmbd, 1e-15) + poisson.inverseCumulativeProbability(delta) + } + + def getLambda2(s: Double): Double = { + var lb = s + var ub = s + math.sqrt(s / delta) // Chebyshev's inequality + while (lb < ub - 1.0) { + val m = (lb + ub) / 2.0 + val poisson = new PoissonDistribution(m, 1e-15) + val y = poisson.inverseCumulativeProbability(delta) + if (y >= s) ub = m else lb = m + } + ub + } +} diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 1230565ea5b7..c208f84cb019 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.rdd +import java.util.Random + import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashSet import scala.util.Random @@ -70,6 +72,90 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { assert(valuesFor2.toList.sorted === List(1)) } + test("sampleByKey") { + def stratifier (fractionPositive: Double) = { + (x: Int) => if (x % 10 < (10 * fractionPositive).toInt) "1" else "0" + } + + // Without replacement validation + def takeSampleAndValidateBernoulli(stratifiedData: RDD[(String, Int)], samplingRate: Double, + seed: Long, n: Long) = { + val expectedSampleSize = stratifiedData.countByKey().mapValues(count => + math.ceil(count * samplingRate).toInt) + val sample = stratifiedData.sampleByKey(false, samplingRate, seed) + val sampleCounts = sample.countByKey() + val takeSample = sample.collect() + assert(sampleCounts.forall({case(k,v) => (expectedSampleSize.get(k).get == v)})) + assert(takeSample.size === takeSample.toSet.size) + assert(takeSample.forall(x => 1 <= x._2 && x._2 <= n), s"elements not in [1, $n]") + } + + // With replacement validation + def takeSampleAndValidatePoisson(stratifiedData: RDD[(String, Int)], samplingRate: Double, + seed: Long, n: Long) = { + val expectedSampleSize = stratifiedData.countByKey().mapValues(count => + math.ceil(count * samplingRate).toInt) + val sample = stratifiedData.sampleByKey(true, samplingRate, seed) + val sampleCounts = sample.countByKey() + val takeSample = sample.collect() + assert(sampleCounts.forall({case(k,v) => (expectedSampleSize.get(k).get == v)})) + val groupedByKey = takeSample.groupBy({case(k, v) => k}) + for ((key, entry) <- groupedByKey) { + if (expectedSampleSize.get(key).get >= 100 && samplingRate >= 0.1) { + // sample large enough for there to be repeats with high likelihood + assert(entry.toSet.size < expectedSampleSize.get(key).get) + } else { + assert(entry.toSet.size <= expectedSampleSize.get(key).get) + } + } + assert(takeSample.forall(x => 1 <= x._2 && x._2 <= n), s"elements not in [1, $n]") + } + + //vary RDD size + for (n <- List(100, 1000, 1000000)) { + val data = sc.parallelize(1 to n, 2) + val fractionPositive = 0.3 + val stratifiedData = data.keyBy(stratifier(fractionPositive)) + + val samplingRate = 0.1 + val seed = 1L + takeSampleAndValidateBernoulli(stratifiedData, samplingRate, seed, n) + takeSampleAndValidatePoisson(stratifiedData, samplingRate, seed, n) + } + + //vary fractionPositive + for (fractionPositive <- List(0.1, 0.3, 0.5, 0.7, 0.9)) { + val n = 100 + val data = sc.parallelize(1 to n, 2) + val stratifiedData = data.keyBy(stratifier(fractionPositive)) + + val samplingRate = 0.1 + val seed = 1L + takeSampleAndValidateBernoulli (stratifiedData, samplingRate, seed, n) + takeSampleAndValidatePoisson(stratifiedData, samplingRate, seed, n) + } + + //Use the same data for the rest of the tests + val fractionPositive = 0.3 + val n = 100 + val data = sc.parallelize(1 to n, 2) + val stratifiedData = data.keyBy(stratifier(fractionPositive)) + + //vary seed + for (seed <- 1 to 5) { + val samplingRate = 0.1 + takeSampleAndValidateBernoulli (stratifiedData, samplingRate, seed, n) + takeSampleAndValidatePoisson(stratifiedData, samplingRate, seed, n) + } + + //vary sampling rate + for (samplingRate <- List(0.01, 0.05, 0.1, 0.5)) { + val seed = 1L + takeSampleAndValidateBernoulli (stratifiedData, samplingRate, seed, n) + takeSampleAndValidatePoisson(stratifiedData, samplingRate, seed, n) + } + } + test("reduceByKey") { val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) val sums = pairs.reduceByKey(_+_).collect() @@ -130,7 +216,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { case(k, count) => assert(error(count, k) < relativeSD) } - val rnd = new Random() + val rnd = new scala.util.Random() // The expected count for key num would be num val randStacked = (1 to 100).flatMap { i => diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 08b3b93d6a31..70a5009f94b9 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -137,6 +137,38 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(result.toSet === Set(("a", 6), ("b", 2), ("c", 5))) } + test("aggregateWithContext") { + val data = Array(("a", 1), ("b", 2), ("a", 2), ("c", 5), ("a", 3)) + val numPartitions = 2 + val pairs = sc.makeRDD(data, numPartitions) + //determine the partitionId for each pair + type StringMap = HashMap[String, Int] + val partitions = pairs.collectPartitions() + val offSets = new StringMap + for (i <- 0 to numPartitions-1) { + partitions(i).foreach( {case (k, v) => offSets.put(k, offSets.getOrElse(k, 0) + i)}) + } + val emptyMap = new StringMap { + override def default(key: String): Int = 0 + } + val mergeElement: ((TaskContext, StringMap), (String, Int)) => StringMap = (arg1, pair) => { + val stringMap = arg1._2 + val tc = arg1._1 + stringMap(pair._1) += pair._2 + tc.partitionId + stringMap + } + val mergeMaps: (StringMap, StringMap) => StringMap = (map1, map2) => { + for ((key, value) <- map2) { + map1(key) += value + } + map1 + } + val result = pairs.aggregateWithContext(emptyMap)(mergeElement, mergeMaps) + val expected = Set(("a", 6), ("b", 2), ("c", 5)) + .map({case(k, v) => (k -> (offSets.getOrElse(k, 0) + v))}) + assert(result.toSet === expected) + } + test("basic caching") { val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() assert(rdd.collect().toList === List(1, 2, 3, 4)) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b082184fa1ed..105d432489a2 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -333,7 +333,7 @@ object SparkBuild extends Build { libraryDependencies ++= Seq( "com.google.guava" % "guava" % "14.0.1", "org.apache.commons" % "commons-lang3" % "3.3.2", - "org.apache.commons" % "commons-math3" % "3.3" % "test", + "org.apache.commons" % "commons-math3" % "3.3", "com.google.code.findbugs" % "jsr305" % "1.3.9", "log4j" % "log4j" % "1.2.17", "org.slf4j" % "slf4j-api" % slf4jVersion, From 1d413ce877a67379a0a74afefba071c018b0ca70 Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Mon, 9 Jun 2014 14:14:26 -0700 Subject: [PATCH 05/28] fixed checkstyle issues --- .../scala/org/apache/spark/rdd/PairRDDFunctions.scala | 8 ++++---- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 11 +++++++---- 2 files changed, 11 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 53219d1a94b7..5fa4ce751b54 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -266,10 +266,10 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) } val combOp = (r1: Result, r2: Result) => { - //take union of both key sets in case one partion doesn't contain all keys + // take union of both key sets in case one partion doesn't contain all keys val keyUnion = r1.resultMap.keys.toSet.union(r2.resultMap.keys.toSet) - //Use r2 to keep the combined result since r1 is usual empty + // Use r2 to keep the combined result since r1 is usual empty for (key <- keyUnion) { val entry1 = r1.resultMap.get(key) val entry2 = r2.resultMap.get(key) @@ -286,7 +286,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val zeroU = new Result(Map[K, Stratum]()) - //determine threshold for each stratum and resample + // determine threshold for each stratum and resample val finalResult = self.aggregateWithContext(zeroU)(seqOp, combOp).resultMap val thresholdByKey = new mutable.HashMap[K, Double]() for ((key, stratum) <- finalResult) { @@ -330,7 +330,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) // Bernoulli sampler self.mapPartitionsWithIndex((idx: Int, iter: Iterator[(K, V)]) => { val random = new RandomDataGenerator - random.reSeed(seed+idx) + random.reSeed(seed + idx) iter.filter(t => random.nextUniform(0.0, 1.0) < thresholdByKey.get(t._1).get) }, preservesPartitioning = true) } diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 76418283486a..41889c96dd28 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -884,15 +884,18 @@ abstract class RDD[T: ClassTag]( * A version of {@link #aggregate()} that passes the TaskContext to the function that does * aggregation for each partition. */ - def aggregateWithContext[U: ClassTag](zeroValue: U)(seqOp: ((TaskContext, U), T) => U, combOp: (U, U) => U): U = { + def aggregateWithContext[U: ClassTag](zeroValue: U)(seqOp: ((TaskContext, U), T) => U, + combOp: (U, U) => U): U = { // Clone the zero value since we will also be serializing it as part of tasks var jobResult = Utils.clone(zeroValue, sc.env.closureSerializer.newInstance()) - //pad seqOp and combOp with taskContext to conform to aggregate's signature in TraversableOnce + // pad seqOp and combOp with taskContext to conform to aggregate's signature in TraversableOnce val paddedSeqOp = (arg1: (TaskContext, U), item: T) => (arg1._1, seqOp(arg1, item)) - val paddedcombOp = (arg1 : (TaskContext, U), arg2: (TaskContext, U)) => (arg1._1, combOp(arg1._2, arg1._2)) + val paddedcombOp = (arg1 : (TaskContext, U), arg2: (TaskContext, U)) => + (arg1._1, combOp(arg1._2, arg1._2)) val cleanSeqOp = sc.clean(paddedSeqOp) val cleanCombOp = sc.clean(paddedcombOp) - val aggregatePartition = (tc: TaskContext, it: Iterator[T]) => (it.aggregate(tc, zeroValue)(cleanSeqOp, cleanCombOp))._2 + val aggregatePartition = (tc: TaskContext, it: Iterator[T]) => + (it.aggregate(tc, zeroValue)(cleanSeqOp, cleanCombOp))._2 val mergeResult = (index: Int, taskResult: U) => jobResult = combOp(jobResult, taskResult) sc.runJob(this, aggregatePartition, mergeResult) jobResult From 7e1a48182ebec54cd3a6a290b1dc27b928f57dba Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Mon, 9 Jun 2014 14:29:30 -0700 Subject: [PATCH 06/28] changed the permission on SamplingUtil --- .../main/scala/org/apache/spark/util/random/SamplingUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala index fee92402810c..9b3eea67c7c4 100644 --- a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala @@ -19,7 +19,7 @@ package org.apache.spark.util.random import org.apache.commons.math3.distribution.{PoissonDistribution, NormalDistribution} -private[random] object PoissonBounds { +private[spark] object PoissonBounds { val delta = 1e-4 / 3.0 val phi = new NormalDistribution().cumulativeProbability(1.0 - delta) From 46f6c8c86f3e1fdaf49b63796f1cd5bd6db79ec7 Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Mon, 9 Jun 2014 17:39:50 -0700 Subject: [PATCH 07/28] fixed the NPE caused by closures being cleaned before being passed into the aggregate function --- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 41889c96dd28..97ddab4e416c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1195,7 +1195,7 @@ abstract class RDD[T: ClassTag]( /** User code that created this RDD (e.g. `textFile`, `parallelize`). */ @transient private[spark] val creationSiteInfo = Utils.getCallSiteInfo - private[spark] def getCreationSite: String = creationSiteInfo.toString + private[spark] def getCreationSite: String = Option(creationSiteInfo).getOrElse("").toString private[spark] def elementClassTag: ClassTag[T] = classTag[T] From 50581fc8b08bd5f18cdf2288772c22f2549af0a5 Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Thu, 12 Jun 2014 14:36:37 -0700 Subject: [PATCH 08/28] added a TODO for logging in python --- python/pyspark/rdd.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 22cead99973d..38f353c1c29a 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -385,6 +385,7 @@ def takeSample(self, withReplacement, num, seed=None): # See: scala/spark/RDD.scala rand = Random(seed) while len(samples) < total: + #TODO add log warning when more than one iteration was run samples = self.sample(withReplacement, fraction, rand.randint(0, sys.maxint)).collect() sampler = RDDSampler(withReplacement, fraction, rand.randint(0, sys.maxint)) From 9e74ab505e5441eedfed5dbfbeac37566d3de1f0 Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Tue, 17 Jun 2014 15:24:22 -0700 Subject: [PATCH 09/28] Separated out most of the logic in sampleByKey into StratifiedSampler in util.random --- .../apache/spark/rdd/PairRDDFunctions.scala | 183 ++------------ .../spark/util/random/StratifiedSampler.scala | 236 ++++++++++++++++++ .../spark/rdd/PairRDDFunctionsSuite.scala | 80 ++++-- 3 files changed, 309 insertions(+), 190 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 32efa6cdf43e..079239ae526c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -49,7 +49,7 @@ import org.apache.spark.SparkContext._ import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.serializer.Serializer import org.apache.spark.util.Utils -import org.apache.spark.util.random.{PoissonBounds => PB} +import org.apache.spark.util.random.{Stratum, Result, StratifiedSampler, PoissonBounds => PB} /** * Extra functions available on RDDs of (key, value) pairs through an implicit conversion. @@ -210,177 +210,32 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) /** * Return a subset of this RDD sampled by key (via stratified sampling). - * We guarantee a sample size = math.ceil(fraction * S_i), where S_i is the size of the ith - * stratum. + * + * If exact set to true, we guarantee, with high probability, a sample size = + * math.ceil(fraction * S_i), where S_i is the size of the ith stratum (collection of entries + * that share the same key). When sampling without replacement, we need one additional pass over + * the RDD to guarantee sample size with a 99.99% confidence; when sampling with replacement, we + * need two additional passes over the RDD to guarantee sample size with a 99.99% confidence. * * @param withReplacement whether to sample with or without replacement - * @param fraction sampling rate + * @param fractionByKey function mapping key to sampling rate * @param seed seed for the random number generator + * @param exact whether sample size needs to be exactly math.ceil(fraction * size) per stratum * @return RDD containing the sampled subset */ def sampleByKey(withReplacement: Boolean, - fraction: Double, - seed: Long = Utils.random.nextLong): RDD[(K, V)]= { - - class Stratum(var numItems: Long = 0L, var numAccepted: Long = 0L) extends Serializable { - var waitList: ArrayBuffer[Double] = new ArrayBuffer[Double] - var q1: Option[Double] = None - var q2: Option[Double] = None - - def incrNumItems(by: Long = 1L) = numItems += by - - def incrNumAccepted(by: Long = 1L) = numAccepted += by - - def addToWaitList(elem: Double) = waitList += elem - - def addToWaitList(elems: ArrayBuffer[Double]) = waitList ++= elems - - override def toString() = { - "numItems: " + numItems + " numAccepted: " + numAccepted + " q1: " + q1 + " q2: " + q2 + - " waitListSize:" + waitList.size - } - } - - class Result(var resultMap: Map[K, Stratum], var cachedPartitionId: Option[Int] = None) - extends Serializable { - var rand: RandomDataGenerator = new RandomDataGenerator - - def getEntry(key: K, numItems: Long = 0L): Stratum = { - if (resultMap.get(key).isEmpty) { - resultMap += (key -> new Stratum(numItems)) - } - resultMap.get(key).get - } - - def getRand(partitionId: Int): RandomDataGenerator = { - if (cachedPartitionId.isEmpty || cachedPartitionId.get != partitionId) { - cachedPartitionId = Some(partitionId) - rand.reSeed(seed + partitionId) - } - rand - } - } - - // TODO implement the streaming version of sampling w/ replacement that doesn't require counts - // in order to save one pass over the RDD - val counts = if (withReplacement) Some(this.countByKey()) else None - - val seqOp = (U: (TaskContext, Result), item: (K, V)) => { - val delta = 5e-5 - val result = U._2 - val tc = U._1 - val rng = result.getRand(tc.partitionId) - val stratum = result.getEntry(item._1) - if (withReplacement) { - // compute q1 and q2 only if they haven't been computed already - // since they don't change from iteration to iteration. - // TODO change this to the streaming version - if (stratum.q1.isEmpty || stratum.q2.isEmpty) { - val n = counts.get(item._1) - val s = math.ceil(n * fraction).toLong - val lmbd1 = PB.getLambda1(s) - val minCount = PB.getMinCount(lmbd1) - val lmbd2 = if (lmbd1 == 0) PB.getLambda2(s) else PB.getLambda2(s - minCount) - val q1 = lmbd1 / n - val q2 = lmbd2 / n - stratum.q1 = Some(q1) - stratum.q2 = Some(q2) - } - val x1 = if (stratum.q1.get == 0) 0L else rng.nextPoisson(stratum.q1.get) - if (x1 > 0) { - stratum.incrNumAccepted(x1) - } - val x2 = rng.nextPoisson(stratum.q2.get).toInt - if (x2 > 0) { - stratum.addToWaitList(ArrayBuffer.fill(x2)(rng.nextUniform(0.0, 1.0))) - } - } else { - val g1 = - math.log(delta) / stratum.numItems - val g2 = (2.0 / 3.0) * g1 - val q1 = math.max(0, fraction + g2 - math.sqrt((g2 * g2 + 3 * g2 * fraction))) - val q2 = math.min(1, fraction + g1 + math.sqrt(g1 * g1 + 2 * g1 * fraction)) - - val x = rng.nextUniform(0.0, 1.0) - if (x < q1) { - stratum.incrNumAccepted() - } else if ( x < q2) { - stratum.addToWaitList(x) - } - stratum.q1 = Some(q1) - stratum.q2 = Some(q2) - } - stratum.incrNumItems() - result - } - - val combOp = (r1: Result, r2: Result) => { - // take union of both key sets in case one partion doesn't contain all keys - val keyUnion = r1.resultMap.keys.toSet.union(r2.resultMap.keys.toSet) - - // Use r2 to keep the combined result since r1 is usual empty - for (key <- keyUnion) { - val entry1 = r1.resultMap.get(key) - val entry2 = r2.resultMap.get(key) - if (entry2.isEmpty && entry1.isDefined) { - r2.resultMap += (key -> entry1.get) - } else if (entry1.isDefined && entry2.isDefined) { - entry2.get.addToWaitList(entry1.get.waitList) - entry2.get.incrNumAccepted(entry1.get.numAccepted) - entry2.get.incrNumItems(entry1.get.numItems) - } - } - r2 - } - - val zeroU = new Result(Map[K, Stratum]()) - - // determine threshold for each stratum and resample - val finalResult = self.aggregateWithContext(zeroU)(seqOp, combOp).resultMap - val thresholdByKey = new mutable.HashMap[K, Double]() - for ((key, stratum) <- finalResult) { - val s = math.ceil(stratum.numItems * fraction).toLong - breakable { - if (stratum.numAccepted > s) { - logWarning("Pre-accepted too many") - thresholdByKey += (key -> stratum.q1.get) - break - } - val numWaitListAccepted = (s - stratum.numAccepted).toInt - if (numWaitListAccepted >= stratum.waitList.size) { - logWarning("WaitList too short") - thresholdByKey += (key -> stratum.q2.get) - } else { - thresholdByKey += (key -> stratum.waitList.sorted.apply(numWaitListAccepted)) - } - } - } - + fractionByKey: K => Double, + seed: Long = Utils.random.nextLong, + exact: Boolean = true): RDD[(K, V)]= { if (withReplacement) { - // Poisson sampler - self.mapPartitionsWithIndex((idx: Int, iter: Iterator[(K, V)]) => { - val random = new RandomDataGenerator() - random.reSeed(seed + idx) - iter.flatMap { t => - val q1 = finalResult.get(t._1).get.q1.get - val q2 = finalResult.get(t._1).get.q2.get - val x1 = if (q1 == 0) 0L else random.nextPoisson(q1) - val x2 = random.nextPoisson(q2).toInt - val x = x1 + (0 until x2).filter(i => random.nextUniform(0.0, 1.0) < - thresholdByKey.get(t._1).get).size - if (x > 0) { - Iterator.fill(x.toInt)(t) - } else { - Iterator.empty - } - } - }, preservesPartitioning = true) + val counts = if (exact) Some(this.countByKey()) else None + val samplingFunc = + StratifiedSampler.getPoissonSamplingFunction(self, fractionByKey, exact, counts, seed) + self.mapPartitionsWithIndex(samplingFunc, preservesPartitioning = true) } else { - // Bernoulli sampler - self.mapPartitionsWithIndex((idx: Int, iter: Iterator[(K, V)]) => { - val random = new RandomDataGenerator - random.reSeed(seed + idx) - iter.filter(t => random.nextUniform(0.0, 1.0) < thresholdByKey.get(t._1).get) - }, preservesPartitioning = true) + val samplingFunc = + StratifiedSampler.getBernoulliSamplingFunction(self, fractionByKey, exact, seed) + self.mapPartitionsWithIndex(samplingFunc, preservesPartitioning = true) } } diff --git a/core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala b/core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala new file mode 100644 index 000000000000..5ddd78d71517 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala @@ -0,0 +1,236 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.random + +import scala.collection.mutable.ArrayBuffer +import scala.collection.{mutable, Map} +import org.apache.commons.math3.random.RandomDataGenerator +import org.apache.spark.{Logging, TaskContext} +import org.apache.spark.util.random.{PoissonBounds => PB} +import scala.Some +import org.apache.spark.rdd.RDD + +private[spark] object StratifiedSampler extends Logging { + def getSeqOp[K, V](withReplacement: Boolean, + fractionByKey: (K => Double), + counts: Option[Map[K, Long]]): ((TaskContext, Result[K]),(K, V)) => Result[K] = { + val delta = 5e-5 + (U: (TaskContext, Result[K]), item: (K, V)) => { + val result = U._2 + val tc = U._1 + val rng = result.getRand(tc.partitionId) + val fraction = fractionByKey(item._1) + val stratum = result.getEntry(item._1) + if (withReplacement) { + // compute q1 and q2 only if they haven't been computed already + // since they don't change from iteration to iteration. + // TODO change this to the streaming version + if (stratum.q1.isEmpty || stratum.q2.isEmpty) { + val n = counts.get(item._1) + val s = math.ceil(n * fraction).toLong + val lmbd1 = PB.getLambda1(s) + val minCount = PB.getMinCount(lmbd1) + val lmbd2 = if (lmbd1 == 0) PB.getLambda2(s) else PB.getLambda2(s - minCount) + val q1 = lmbd1 / n + val q2 = lmbd2 / n + stratum.q1 = Some(q1) + stratum.q2 = Some(q2) + } + val x1 = if (stratum.q1.get == 0) 0L else rng.nextPoisson(stratum.q1.get) + if (x1 > 0) { + stratum.incrNumAccepted(x1) + } + val x2 = rng.nextPoisson(stratum.q2.get).toInt + if (x2 > 0) { + stratum.addToWaitList(ArrayBuffer.fill(x2)(rng.nextUniform(0.0, 1.0))) + } + } else { + val g1 = - math.log(delta) / stratum.numItems + val g2 = (2.0 / 3.0) * g1 + val q1 = math.max(0, fraction + g2 - math.sqrt((g2 * g2 + 3 * g2 * fraction))) + val q2 = math.min(1, fraction + g1 + math.sqrt(g1 * g1 + 2 * g1 * fraction)) + + val x = rng.nextUniform(0.0, 1.0) + if (x < q1) { + stratum.incrNumAccepted() + } else if (x < q2) { + stratum.addToWaitList(x) + } + stratum.q1 = Some(q1) + stratum.q2 = Some(q2) + } + stratum.incrNumItems() + result + } + } + + def getCombOp[K](): (Result[K], Result[K]) => Result[K] = { + (r1: Result[K], r2: Result[K]) => { + // take union of both key sets in case one partition doesn't contain all keys + val keyUnion = r1.resultMap.keys.toSet.union(r2.resultMap.keys.toSet) + + // Use r2 to keep the combined result since r1 is usual empty + for (key <- keyUnion) { + val entry1 = r1.resultMap.get(key) + val entry2 = r2.resultMap.get(key) + if (entry2.isEmpty && entry1.isDefined) { + r2.resultMap += (key -> entry1.get) + } else if (entry1.isDefined && entry2.isDefined) { + entry2.get.addToWaitList(entry1.get.waitList) + entry2.get.incrNumAccepted(entry1.get.numAccepted) + entry2.get.incrNumItems(entry1.get.numItems) + } + } + r2 + } + } + + def computeThresholdByKey[K](finalResult: Map[K, Stratum], fractionByKey: (K => Double)): + (K => Double) = { + val thresholdByKey = new mutable.HashMap[K, Double]() + for ((key, stratum) <- finalResult) { + val fraction = fractionByKey(key) + val s = math.ceil(stratum.numItems * fraction).toLong + if (stratum.numAccepted > s) { + logWarning("Pre-accepted too many") + thresholdByKey += (key -> stratum.q1.get) + } else { + val numWaitListAccepted = (s - stratum.numAccepted).toInt + if (numWaitListAccepted >= stratum.waitList.size) { + logWarning("WaitList too short") + thresholdByKey += (key -> stratum.q2.get) + } else { + thresholdByKey += (key -> stratum.waitList.sorted.apply(numWaitListAccepted)) + } + } + } + thresholdByKey + } + + def computeThresholdByKey[K](finalResult: Map[K, String]): (K => String) = { + finalResult + } + + def getBernoulliSamplingFunction[K, V](rdd:RDD[(K, V)], + fractionByKey: K => Double, + exact: Boolean, + seed: Long): (Int, Iterator[(K, V)]) => Iterator[(K, V)] = { + var samplingRateByKey = fractionByKey + if (exact) { + // determine threshold for each stratum and resample + val seqOp = StratifiedSampler.getSeqOp[K,V](false, fractionByKey, None) + val combOp = StratifiedSampler.getCombOp[K]() + val zeroU = new Result[K](Map[K, Stratum](), seed = seed) + val finalResult = rdd.aggregateWithContext(zeroU)(seqOp, combOp).resultMap + samplingRateByKey = StratifiedSampler.computeThresholdByKey(finalResult, fractionByKey) + } + (idx: Int, iter: Iterator[(K, V)]) => { + val random = new RandomDataGenerator + random.reSeed(seed + idx) + iter.filter(t => random.nextUniform(0.0, 1.0) < samplingRateByKey(t._1)) + } + } + + def getPoissonSamplingFunction[K, V](rdd:RDD[(K, V)], + fractionByKey: K => Double, + exact: Boolean, + counts: Option[Map[K, Long]], + seed: Long): (Int, Iterator[(K, V)]) => Iterator[(K, V)] = { + // TODO implement the streaming version of sampling w/ replacement that doesn't require counts + if (exact) { + val seqOp = StratifiedSampler.getSeqOp[K,V](true, fractionByKey, counts) + val combOp = StratifiedSampler.getCombOp[K]() + val zeroU = new Result[K](Map[K, Stratum](), seed = seed) + val finalResult = rdd.aggregateWithContext(zeroU)(seqOp, combOp).resultMap + val thresholdByKey = StratifiedSampler.computeThresholdByKey(finalResult, fractionByKey) + (idx: Int, iter: Iterator[(K, V)]) => { + val random = new RandomDataGenerator() + random.reSeed(seed + idx) + iter.flatMap { t => + val q1 = finalResult.get(t._1).get.q1.getOrElse(0.0) + val q2 = finalResult.get(t._1).get.q2.getOrElse(0.0) + val x1 = if (q1 == 0) 0L else random.nextPoisson(q1) + val x2 = random.nextPoisson(q2).toInt + val x = x1 + (0 until x2).count(i => random.nextUniform(0.0, 1.0) < thresholdByKey(t._1)) + if (x > 0) { + Iterator.fill(x.toInt)(t) + } else { + Iterator.empty + } + } + } + } else { + (idx: Int, iter: Iterator[(K, V)]) => { + val random = new RandomDataGenerator() + random.reSeed(seed + idx) + iter.flatMap { t => + val count = random.nextPoisson(fractionByKey(t._1)).toInt + if (count > 0) { + Iterator.fill(count)(t) + } else { + Iterator.empty + } + } + } + } + } +} + +private[random] class Stratum(var numItems: Long = 0L, var numAccepted: Long = 0L) + extends Serializable { + + var waitList: ArrayBuffer[Double] = new ArrayBuffer[Double] + var q1: Option[Double] = None // upper bound for accepting item instantly + var q2: Option[Double] = None // upper bound for adding item to waitlist + + def incrNumItems(by: Long = 1L) = numItems += by + + def incrNumAccepted(by: Long = 1L) = numAccepted += by + + def addToWaitList(elem: Double) = waitList += elem + + def addToWaitList(elems: ArrayBuffer[Double]) = waitList ++= elems + + override def toString() = { + "numItems: " + numItems + " numAccepted: " + numAccepted + " q1: " + q1 + " q2: " + q2 + + " waitListSize:" + waitList.size + } +} + +private[random] class Result[K](var resultMap: Map[K, Stratum], + var cachedPartitionId: Option[Int] = None, + val seed: Long) + extends Serializable { + + var rand: RandomDataGenerator = new RandomDataGenerator + + def getEntry(key: K, numItems: Long = 0L): Stratum = { + if (!resultMap.contains(key)) { + resultMap += (key -> new Stratum(numItems)) + } + resultMap(key) + } + + def getRand(partitionId: Int): RandomDataGenerator = { + if (cachedPartitionId.isEmpty || cachedPartitionId.get != partitionId) { + cachedPartitionId = Some(partitionId) + rand.reSeed(seed + partitionId) + } + rand + } +} diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 0bdea9c9bfa5..586844006587 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark.rdd -import java.util.Random - import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashSet import scala.util.Random @@ -90,40 +88,74 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { (x: Int) => if (x % 10 < (10 * fractionPositive).toInt) "1" else "0" } + def checkSize(exact: Boolean, withReplacement: Boolean, + expected: Long, actual: Long, p: Double): Boolean = { + if (exact) { + return expected == actual + } + val stdev = if (withReplacement) math.sqrt(expected) else math.sqrt(expected * p * (1 - p)) + // Very forgiving margin since we're dealing with very small sample sizes most of the time + math.abs(actual - expected) <= 6 * stdev + } + // Without replacement validation - def takeSampleAndValidateBernoulli(stratifiedData: RDD[(String, Int)], samplingRate: Double, - seed: Long, n: Long) = { + def takeSampleAndValidateBernoulli(stratifiedData: RDD[(String, Int)], + exact: Boolean, + samplingRate: Double, + seed: Long, + n: Long) = { val expectedSampleSize = stratifiedData.countByKey().mapValues(count => math.ceil(count * samplingRate).toInt) - val sample = stratifiedData.sampleByKey(false, samplingRate, seed) + val fractionByKey = (_:String) => samplingRate + val sample = stratifiedData.sampleByKey(false, fractionByKey, seed, exact) val sampleCounts = sample.countByKey() val takeSample = sample.collect() - assert(sampleCounts.forall({case(k,v) => (expectedSampleSize.get(k).get == v)})) + assert(sampleCounts.forall({case(k,v) => + checkSize(exact, false, expectedSampleSize(k), v, samplingRate)})) assert(takeSample.size === takeSample.toSet.size) assert(takeSample.forall(x => 1 <= x._2 && x._2 <= n), s"elements not in [1, $n]") } // With replacement validation - def takeSampleAndValidatePoisson(stratifiedData: RDD[(String, Int)], samplingRate: Double, - seed: Long, n: Long) = { + def takeSampleAndValidatePoisson(stratifiedData: RDD[(String, Int)], + exact: Boolean, + samplingRate: Double, + seed: Long, + n: Long) = { val expectedSampleSize = stratifiedData.countByKey().mapValues(count => math.ceil(count * samplingRate).toInt) - val sample = stratifiedData.sampleByKey(true, samplingRate, seed) + val fractionByKey = (_:String) => samplingRate + val sample = stratifiedData.sampleByKey(true, fractionByKey, seed, exact) val sampleCounts = sample.countByKey() val takeSample = sample.collect() - assert(sampleCounts.forall({case(k,v) => (expectedSampleSize.get(k).get == v)})) + assert(sampleCounts.forall({case(k,v) => + checkSize(exact, true, expectedSampleSize(k), v, samplingRate)})) val groupedByKey = takeSample.groupBy({case(k, v) => k}) - for ((key, entry) <- groupedByKey) { - if (expectedSampleSize.get(key).get >= 100 && samplingRate >= 0.1) { + for ((key, v) <- groupedByKey) { + if (expectedSampleSize(key) >= 100 && samplingRate >= 0.1) { // sample large enough for there to be repeats with high likelihood - assert(entry.toSet.size < expectedSampleSize.get(key).get) + assert(v.toSet.size < expectedSampleSize(key)) } else { - assert(entry.toSet.size <= expectedSampleSize.get(key).get) + if (exact) { + assert(v.toSet.size <= expectedSampleSize(key)) + } else { + assert(checkSize(false, true, expectedSampleSize(key), v.toSet.size, samplingRate)) + } } } assert(takeSample.forall(x => 1 <= x._2 && x._2 <= n), s"elements not in [1, $n]") } + def checkAllCombos(stratifiedData: RDD[(String, Int)], samplingRate: Double, + seed: Long, n: Long) { + takeSampleAndValidateBernoulli(stratifiedData, true, samplingRate, seed, n) + takeSampleAndValidateBernoulli(stratifiedData, false, samplingRate, seed, n) + takeSampleAndValidatePoisson(stratifiedData, true, samplingRate, seed, n) + takeSampleAndValidatePoisson(stratifiedData, false, samplingRate, seed, n) + } + + val defaultSeed = 42L + //vary RDD size for (n <- List(100, 1000, 1000000)) { val data = sc.parallelize(1 to n, 2) @@ -131,9 +163,8 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { val stratifiedData = data.keyBy(stratifier(fractionPositive)) val samplingRate = 0.1 - val seed = 1L - takeSampleAndValidateBernoulli(stratifiedData, samplingRate, seed, n) - takeSampleAndValidatePoisson(stratifiedData, samplingRate, seed, n) + val seed = defaultSeed + checkAllCombos(stratifiedData, samplingRate, seed, n) } //vary fractionPositive @@ -143,9 +174,8 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { val stratifiedData = data.keyBy(stratifier(fractionPositive)) val samplingRate = 0.1 - val seed = 1L - takeSampleAndValidateBernoulli (stratifiedData, samplingRate, seed, n) - takeSampleAndValidatePoisson(stratifiedData, samplingRate, seed, n) + val seed = defaultSeed + checkAllCombos(stratifiedData, samplingRate, seed, n) } //Use the same data for the rest of the tests @@ -155,17 +185,15 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { val stratifiedData = data.keyBy(stratifier(fractionPositive)) //vary seed - for (seed <- 1 to 5) { + for (seed <- defaultSeed to defaultSeed + 5L) { val samplingRate = 0.1 - takeSampleAndValidateBernoulli (stratifiedData, samplingRate, seed, n) - takeSampleAndValidatePoisson(stratifiedData, samplingRate, seed, n) + checkAllCombos(stratifiedData, samplingRate, seed, n) } //vary sampling rate for (samplingRate <- List(0.01, 0.05, 0.1, 0.5)) { - val seed = 1L - takeSampleAndValidateBernoulli (stratifiedData, samplingRate, seed, n) - takeSampleAndValidatePoisson(stratifiedData, samplingRate, seed, n) + val seed = defaultSeed + checkAllCombos(stratifiedData, samplingRate, seed, n) } } From 0214a7659c62e4ff0f68f6e09cd7846547cd3bcb Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Tue, 17 Jun 2014 19:22:32 -0700 Subject: [PATCH 10/28] cleanUp MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Addressed reviewer comments and added better documentation of code. Added commons-math3 as a dependency of spark (okay’ed by Matei). “mvm clean install” compiled. Recovered files that were reverted by accident in the merge. TODOs: figure out API for sampleByKeyExact and update Java, Python, and the markdown file accordingly. --- core/pom.xml | 2 - .../main/scala/org/apache/spark/rdd/RDD.scala | 21 +++++++ .../spark/util/random/SamplingUtils.scala | 36 ++++++++++-- .../spark/util/random/StratifiedSampler.scala | 58 ++++++++++++++----- .../scala/org/apache/spark/rdd/RDDSuite.scala | 32 ++++++++++ 5 files changed, 129 insertions(+), 20 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index bd6767e03bb9..70b6674c83bd 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -70,8 +70,6 @@ org.apache.commons commons-math3 - 3.3 - test com.google.code.findbugs diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 1633b185861b..2b49241c579a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -874,6 +874,27 @@ abstract class RDD[T: ClassTag]( jobResult } + /** + * A version of {@link #aggregate()} that passes the TaskContext to the function that does + * aggregation for each partition. + */ + def aggregateWithContext[U: ClassTag](zeroValue: U)(seqOp: ((TaskContext, U), T) => U, + combOp: (U, U) => U): U = { + // Clone the zero value since we will also be serializing it as part of tasks + var jobResult = Utils.clone(zeroValue, sc.env.closureSerializer.newInstance()) + // pad seqOp and combOp with taskContext to conform to aggregate's signature in TraversableOnce + val paddedSeqOp = (arg1: (TaskContext, U), item: T) => (arg1._1, seqOp(arg1, item)) + val paddedcombOp = (arg1: (TaskContext, U), arg2: (TaskContext, U)) => + (arg1._1, combOp(arg1._2, arg1._2)) + val cleanSeqOp = sc.clean(paddedSeqOp) + val cleanCombOp = sc.clean(paddedcombOp) + val aggregatePartition = (tc: TaskContext, it: Iterator[T]) => + (it.aggregate(tc, zeroValue)(cleanSeqOp, cleanCombOp))._2 + val mergeResult = (index: Int, taskResult: U) => jobResult = combOp(jobResult, taskResult) + sc.runJob(this, aggregatePartition, mergeResult) + jobResult + } + /** * Return the number of elements in the RDD. */ diff --git a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala index fcfb6c97e893..24168fe2c6cf 100644 --- a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala @@ -17,7 +17,7 @@ package org.apache.spark.util.random -import org.apache.commons.math3.distribution.{PoissonDistribution, NormalDistribution} +import org.apache.commons.math3.distribution.PoissonDistribution private[spark] object SamplingUtils { @@ -43,7 +43,7 @@ private[spark] object SamplingUtils { * @return a sampling rate that guarantees sufficient sample size with 99.99% success rate */ def computeFractionForSampleSize(sampleSizeLowerBound: Int, total: Long, - withReplacement: Boolean): Double = { + withReplacement: Boolean): Double = { val fraction = sampleSizeLowerBound.toDouble / total if (withReplacement) { val numStDev = if (sampleSizeLowerBound < 12) 9 else 5 @@ -56,12 +56,29 @@ private[spark] object SamplingUtils { } } +/** + * Utility functions that help us determine bounds on adjusted sampling rate to guarantee exact + * sample sizes with high confidence when sampling with replacement. + * + * The algorithm for guaranteeing sample size instantly accepts items whose associated value drawn + * from Pois(s) is less than the lower bound and puts items whose value is between the lower and + * upper bound in a waitlist. The final sample is consisted of all items accepted on the fly and a + * portion of the waitlist needed to make the exact sample size. + */ private[spark] object PoissonBounds { val delta = 1e-4 / 3.0 - val phi = new NormalDistribution().cumulativeProbability(1.0 - delta) - def getLambda1(s: Double): Double = { + /** + * Compute the threshold for accepting items on the fly. The threshold value is a fairly small + * number, which means if the item has an associated value < threshold, it is highly likely to + * be in the final sample. Hence we accept items with values less than the returned value of this + * function instantly. + * + * @param s sample size + * @return threshold for accepting items on the fly + */ + def getLowerBound(s: Double): Double = { var lb = math.max(0.0, s - math.sqrt(s / delta)) // Chebyshev's inequality var ub = s while (lb < ub - 1.0) { @@ -79,7 +96,16 @@ private[spark] object PoissonBounds { poisson.inverseCumulativeProbability(delta) } - def getLambda2(s: Double): Double = { + /** + * Compute the threshold for waitlisting items. An item is waitlisted if its associated value is + * greater than the lower bound determined above but below the upper bound computed here. + * The value is computed such that we only need to keep log(s) items in the waitlist and still be + * able to guarantee sample size with high confidence. + * + * @param s sample size + * @return threshold for waitlisting the item + */ + def getUpperBound(s: Double): Double = { var lb = s var ub = s + math.sqrt(s / delta) // Chebyshev's inequality while (lb < ub - 1.0) { diff --git a/core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala b/core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala index 5ddd78d71517..3535fa8387af 100644 --- a/core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala +++ b/core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala @@ -26,6 +26,9 @@ import scala.Some import org.apache.spark.rdd.RDD private[spark] object StratifiedSampler extends Logging { + /** + * Returns the function used by aggregate to collect sampling statistics for each partition. + */ def getSeqOp[K, V](withReplacement: Boolean, fractionByKey: (K => Double), counts: Option[Map[K, Long]]): ((TaskContext, Result[K]),(K, V)) => Result[K] = { @@ -43,9 +46,9 @@ private[spark] object StratifiedSampler extends Logging { if (stratum.q1.isEmpty || stratum.q2.isEmpty) { val n = counts.get(item._1) val s = math.ceil(n * fraction).toLong - val lmbd1 = PB.getLambda1(s) + val lmbd1 = PB.getLowerBound(s) val minCount = PB.getMinCount(lmbd1) - val lmbd2 = if (lmbd1 == 0) PB.getLambda2(s) else PB.getLambda2(s - minCount) + val lmbd2 = if (lmbd1 == 0) PB.getUpperBound(s) else PB.getUpperBound(s - minCount) val q1 = lmbd1 / n val q2 = lmbd2 / n stratum.q1 = Some(q1) @@ -60,6 +63,8 @@ private[spark] object StratifiedSampler extends Logging { stratum.addToWaitList(ArrayBuffer.fill(x2)(rng.nextUniform(0.0, 1.0))) } } else { + // We use the streaming version of the algorithm for sampling without replacement. + // Hence, q1 and q2 change on every iteration. val g1 = - math.log(delta) / stratum.numItems val g2 = (2.0 / 3.0) * g1 val q1 = math.max(0, fraction + g2 - math.sqrt((g2 * g2 + 3 * g2 * fraction))) @@ -79,7 +84,11 @@ private[spark] object StratifiedSampler extends Logging { } } - def getCombOp[K](): (Result[K], Result[K]) => Result[K] = { + /** + * Returns the function used by aggregate to combine results from different partitions, as + * returned by seqOp. + */ + def getCombOp[K](): (Result[K], Result[K]) => Result[K] = { (r1: Result[K], r2: Result[K]) => { // take union of both key sets in case one partition doesn't contain all keys val keyUnion = r1.resultMap.keys.toSet.union(r2.resultMap.keys.toSet) @@ -100,6 +109,10 @@ private[spark] object StratifiedSampler extends Logging { } } + /** + * Given the result returned by the aggregate function, we need to determine the threshold used + * to accept items to generate the exact sample size. + */ def computeThresholdByKey[K](finalResult: Map[K, Stratum], fractionByKey: (K => Double)): (K => Double) = { val thresholdByKey = new mutable.HashMap[K, Double]() @@ -122,11 +135,15 @@ private[spark] object StratifiedSampler extends Logging { thresholdByKey } - def computeThresholdByKey[K](finalResult: Map[K, String]): (K => String) = { - finalResult - } - - def getBernoulliSamplingFunction[K, V](rdd:RDD[(K, V)], + /** + * Return the per partition sampling function used for sampling without replacement. + * + * When exact sample size is required, we make an additional pass over the RDD to determine the + * exact sampling rate that guarantees sample size with high confidence. + * + * The sampling function has a unique seed per partition. + */ + def getBernoulliSamplingFunction[K, V](rdd: RDD[(K, V)], fractionByKey: K => Double, exact: Boolean, seed: Long): (Int, Iterator[(K, V)]) => Iterator[(K, V)] = { @@ -146,6 +163,16 @@ private[spark] object StratifiedSampler extends Logging { } } + /** + * Return the per partition sampling function used for sampling with replacement. + * + * When exact sample size is required, we make two additional passed over the RDD to determine + * the exact sampling rate that guarantees sample size with high confidence. The first pass + * counts the number of items in each stratum (group of items with the same key) in the RDD, and + * the second pass uses the counts to determine exact sampling rates. + * + * The sampling function has a unique seed per partition. + */ def getPoissonSamplingFunction[K, V](rdd:RDD[(K, V)], fractionByKey: K => Double, exact: Boolean, @@ -191,6 +218,10 @@ private[spark] object StratifiedSampler extends Logging { } } +/** + * Object used by seqOp to keep track of the number of items accepted and items waitlisted per + * stratum, as well as the bounds for accepting and waitlisting items. + */ private[random] class Stratum(var numItems: Long = 0L, var numAccepted: Long = 0L) extends Serializable { @@ -205,13 +236,14 @@ private[random] class Stratum(var numItems: Long = 0L, var numAccepted: Long = 0 def addToWaitList(elem: Double) = waitList += elem def addToWaitList(elems: ArrayBuffer[Double]) = waitList ++= elems - - override def toString() = { - "numItems: " + numItems + " numAccepted: " + numAccepted + " q1: " + q1 + " q2: " + q2 + - " waitListSize:" + waitList.size - } } +/** + * Object used by seqOp and combOp to keep track of the sampling statistics for all strata. + * + * When used by seqOp for each partition, we also keep track of the partition ID in this object + * to make sure a single random number generator with a unique seed is used for each partition. + */ private[random] class Result[K](var resultMap: Map[K, Stratum], var cachedPartitionId: Option[Int] = None, val seed: Long) diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 0e5625b7645d..a3094a2fd626 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -141,6 +141,38 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(result.toSet === Set(("a", 6), ("b", 2), ("c", 5))) } + test("aggregateWithContext") { + val data = Array(("a", 1), ("b", 2), ("a", 2), ("c", 5), ("a", 3)) + val numPartitions = 2 + val pairs = sc.makeRDD(data, numPartitions) + //determine the partitionId for each pair + type StringMap = HashMap[String, Int] + val partitions = pairs.collectPartitions() + val offSets = new StringMap + for (i <- 0 to numPartitions - 1) { + partitions(i).foreach({ case (k, v) => offSets.put(k, offSets.getOrElse(k, 0) + i)}) + } + val emptyMap = new StringMap { + override def default(key: String): Int = 0 + } + val mergeElement: ((TaskContext, StringMap), (String, Int)) => StringMap = (arg1, pair) => { + val stringMap = arg1._2 + val tc = arg1._1 + stringMap(pair._1) += pair._2 + tc.partitionId + stringMap + } + val mergeMaps: (StringMap, StringMap) => StringMap = (map1, map2) => { + for ((key, value) <- map2) { + map1(key) += value + } + map1 + } + val result = pairs.aggregateWithContext(emptyMap)(mergeElement, mergeMaps) + val expected = Set(("a", 6), ("b", 2), ("c", 5)) + .map({ case (k, v) => (k -> (offSets.getOrElse(k, 0) + v))}) + assert(result.toSet === expected) + } + test("basic caching") { val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() assert(rdd.collect().toList === List(1, 2, 3, 4)) From 944a10cff3c218bafcb8b43e7e3f309cc644633e Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Wed, 18 Jun 2014 19:48:22 -0700 Subject: [PATCH 11/28] [SPARK-2145] Add lower bound on sampling rate to guarantee sampling performance --- .../main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 7 +++++++ core/src/main/scala/org/apache/spark/rdd/RDD.scala | 4 +++- 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index e0bc75e244ac..cb26ab6d1979 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -217,6 +217,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * the RDD to guarantee sample size with a 99.99% confidence; when sampling with replacement, we * need two additional passes over the RDD to guarantee sample size with a 99.99% confidence. * + * Note that if the sampling rate for any stratum is < 1e-10, we will throw an exception to + * avoid not being able to ever create the sample as an artifact of the RNG's quality. + * * @param withReplacement whether to sample with or without replacement * @param fractionByKey function mapping key to sampling rate * @param seed seed for the random number generator @@ -227,6 +230,10 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) fractionByKey: K => Double, seed: Long = Utils.random.nextLong, exact: Boolean = true): RDD[(K, V)]= { + + require(fractionByKey.asInstanceOf[Map[K, Double]].forall({case(k, v) => v >= 1e-10}), + "Unable to support sampling rates < 1e-10.") + if (withReplacement) { val counts = if (exact) Some(this.countByKey()) else None val samplingFunc = diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 2b49241c579a..d1000ae21e66 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -350,11 +350,13 @@ abstract class RDD[T: ClassTag]( /** * Return a sampled subset of this RDD. + * + * fraction < 1e-10 not supported. */ def sample(withReplacement: Boolean, fraction: Double, seed: Long = Utils.random.nextLong): RDD[T] = { - require(fraction >= 0.0, "Invalid fraction value: " + fraction) + require(fraction >= 1e-10, "Invalid fraction value: " + fraction) if (withReplacement) { new PartitionwiseSampledRDD[T, T](this, new PoissonSampler[T](fraction), seed) } else { From 1fe1cff58d99f336406f67f89f53fe6ab3bfde5e Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Thu, 19 Jun 2014 12:46:59 -0700 Subject: [PATCH 12/28] Changed fractionByKey to a map to enable arg check --- .../org/apache/spark/rdd/PairRDDFunctions.scala | 12 +++--------- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 3 +-- .../org/apache/spark/util/random/SamplingUtils.scala | 7 +++++-- 3 files changed, 9 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index cb26ab6d1979..1881f85d0ff5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -215,10 +215,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * math.ceil(fraction * S_i), where S_i is the size of the ith stratum (collection of entries * that share the same key). When sampling without replacement, we need one additional pass over * the RDD to guarantee sample size with a 99.99% confidence; when sampling with replacement, we - * need two additional passes over the RDD to guarantee sample size with a 99.99% confidence. - * - * Note that if the sampling rate for any stratum is < 1e-10, we will throw an exception to - * avoid not being able to ever create the sample as an artifact of the RNG's quality. + * need two additional passes. * * @param withReplacement whether to sample with or without replacement * @param fractionByKey function mapping key to sampling rate @@ -227,14 +224,11 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * @return RDD containing the sampled subset */ def sampleByKey(withReplacement: Boolean, - fractionByKey: K => Double, + fractionByKey: Map[K, Double], seed: Long = Utils.random.nextLong, exact: Boolean = true): RDD[(K, V)]= { - - require(fractionByKey.asInstanceOf[Map[K, Double]].forall({case(k, v) => v >= 1e-10}), - "Unable to support sampling rates < 1e-10.") - if (withReplacement) { + require(fractionByKey.forall({case(k, v) => v >= 0.0}), "Invalid sampling rates.") val counts = if (exact) Some(this.countByKey()) else None val samplingFunc = StratifiedSampler.getPoissonSamplingFunction(self, fractionByKey, exact, counts, seed) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index d1000ae21e66..9edc73d1c577 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -351,12 +351,11 @@ abstract class RDD[T: ClassTag]( /** * Return a sampled subset of this RDD. * - * fraction < 1e-10 not supported. */ def sample(withReplacement: Boolean, fraction: Double, seed: Long = Utils.random.nextLong): RDD[T] = { - require(fraction >= 1e-10, "Invalid fraction value: " + fraction) + require(fraction >= 0.0, "Invalid fraction value: " + fraction) if (withReplacement) { new PartitionwiseSampledRDD[T, T](this, new PoissonSampler[T](fraction), seed) } else { diff --git a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala index 24168fe2c6cf..19c00384ca60 100644 --- a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala @@ -37,6 +37,9 @@ private[spark] object SamplingUtils { * ~ Binomial(total, fraction) and our choice of q guarantees 1-delta, or 0.9999 success * rate, where success rate is defined the same as in sampling with replacement. * + * The smallest sampling rate supported is 1e-10 (in order to avoid running into the limit of the + * RNG's resolution). + * * @param sampleSizeLowerBound sample size * @param total size of RDD * @param withReplacement whether sampling with replacement @@ -47,11 +50,11 @@ private[spark] object SamplingUtils { val fraction = sampleSizeLowerBound.toDouble / total if (withReplacement) { val numStDev = if (sampleSizeLowerBound < 12) 9 else 5 - fraction + numStDev * math.sqrt(fraction / total) + math.max(1e-10, fraction + numStDev * math.sqrt(fraction / total)) } else { val delta = 1e-4 val gamma = - math.log(delta) / total - math.min(1, fraction + gamma + math.sqrt(gamma * gamma + 2 * gamma * fraction)) + math.min(1, math.max(1e-10, fraction + gamma + math.sqrt(gamma * gamma + 2 * gamma * fraction))) } } } From bd9dc6e08444ec83c5d6adc4c452b49d1ac2b154 Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Thu, 19 Jun 2014 13:00:47 -0700 Subject: [PATCH 13/28] unit bug and style violation fixed --- .../main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 2 +- .../scala/org/apache/spark/util/random/SamplingUtils.scala | 3 ++- .../scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala | 4 ++-- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 1881f85d0ff5..5c30c0a2f8cd 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -227,8 +227,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) fractionByKey: Map[K, Double], seed: Long = Utils.random.nextLong, exact: Boolean = true): RDD[(K, V)]= { + require(fractionByKey.forall({case(k, v) => v >= 0.0}), "Invalid sampling rates.") if (withReplacement) { - require(fractionByKey.forall({case(k, v) => v >= 0.0}), "Invalid sampling rates.") val counts = if (exact) Some(this.countByKey()) else None val samplingFunc = StratifiedSampler.getPoissonSamplingFunction(self, fractionByKey, exact, counts, seed) diff --git a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala index 19c00384ca60..0a4a3a7fc85f 100644 --- a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala @@ -54,7 +54,8 @@ private[spark] object SamplingUtils { } else { val delta = 1e-4 val gamma = - math.log(delta) / total - math.min(1, math.max(1e-10, fraction + gamma + math.sqrt(gamma * gamma + 2 * gamma * fraction))) + math.min(1, + math.max(1e-10, fraction + gamma + math.sqrt(gamma * gamma + 2 * gamma * fraction))) } } } diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 586844006587..4ac99a9dc682 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -106,7 +106,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { n: Long) = { val expectedSampleSize = stratifiedData.countByKey().mapValues(count => math.ceil(count * samplingRate).toInt) - val fractionByKey = (_:String) => samplingRate + val fractionByKey = Map("1" -> samplingRate, "0" -> samplingRate) val sample = stratifiedData.sampleByKey(false, fractionByKey, seed, exact) val sampleCounts = sample.countByKey() val takeSample = sample.collect() @@ -124,7 +124,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { n: Long) = { val expectedSampleSize = stratifiedData.countByKey().mapValues(count => math.ceil(count * samplingRate).toInt) - val fractionByKey = (_:String) => samplingRate + val fractionByKey = Map("1" -> samplingRate, "0" -> samplingRate) val sample = stratifiedData.sampleByKey(true, fractionByKey, seed, exact) val sampleCounts = sample.countByKey() val takeSample = sample.collect() From 4ad516b14559f06a32d65ef0ce2fa2d7526610bc Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Fri, 20 Jun 2014 11:38:09 -0700 Subject: [PATCH 14/28] remove unused imports from PairRDDFunctions --- .../main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 5c30c0a2f8cd..fe56fe7ba674 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -27,9 +27,7 @@ import scala.collection.Map import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag -import scala.util.control.Breaks._ -import org.apache.commons.math3.random.RandomDataGenerator import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.fs.FileSystem @@ -49,7 +47,7 @@ import org.apache.spark.SparkContext._ import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.serializer.Serializer import org.apache.spark.util.Utils -import org.apache.spark.util.random.{Stratum, Result, StratifiedSampler, PoissonBounds => PB} +import org.apache.spark.util.random.StratifiedSampler /** * Extra functions available on RDDs of (key, value) pairs through an implicit conversion. From 254e03c96e1f2aaa5baa9c3d384adeb117e0b7ab Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Thu, 3 Jul 2014 13:49:46 -0700 Subject: [PATCH 15/28] minor fixes and Java API. punting on python for now. moved aggregateWithContext out of RDD --- .../apache/spark/api/java/JavaPairRDD.scala | 34 +++++++++++++++- .../apache/spark/rdd/PairRDDFunctions.scala | 30 +++++++------- .../main/scala/org/apache/spark/rdd/RDD.scala | 21 ---------- .../spark/util/random/StratifiedSampler.scala | 40 ++++++++++++++++--- .../spark/rdd/PairRDDFunctionsSuite.scala | 8 ++-- .../scala/org/apache/spark/rdd/RDDSuite.scala | 32 --------------- 6 files changed, 85 insertions(+), 80 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 14fa9d8135af..e4aa46deb831 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.api.java -import java.util.{Comparator, List => JList} +import java.util.{Comparator, List => JList, Map => JMap} import java.lang.{Iterable => JIterable} import scala.collection.JavaConversions._ @@ -129,6 +129,38 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) def sample(withReplacement: Boolean, fraction: Double, seed: Long): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.sample(withReplacement, fraction, seed)) + /** + * Return a subset of this RDD sampled by key (via stratified sampling). + */ + def sampleByKey(withReplacement: Boolean, + fractions: JMap[K, Double], + exact: Boolean, + seed: Long): JavaPairRDD[K, V] = + new JavaPairRDD[K, V](rdd.sampleByKey(withReplacement, fractions, exact, seed)) + + + /** + * Return a subset of this RDD sampled by key (via stratified sampling). + */ + def sampleByKey(withReplacement: Boolean, + fractions: JMap[K, Double], + exact: Boolean): JavaPairRDD[K, V] = + sampleByKey(withReplacement, fractions, exact, Utils.random.nextLong) + + /** + * Return a subset of this RDD sampled by key (via stratified sampling). + */ + def sampleByKey(withReplacement: Boolean, + fractions: JMap[K, Double], + seed: Long): JavaPairRDD[K, V] = + sampleByKey(withReplacement, fractions, true, seed) + + /** + * Return a subset of this RDD sampled by key (via stratified sampling). + */ + def sampleByKey(withReplacement: Boolean, fractions: JMap[K, Double]): JavaPairRDD[K, V] = + sampleByKey(withReplacement, fractions, true, Utils.random.nextLong) + /** * Return the union of this RDD and another one. Any identical elements will appear multiple * times (use `.distinct()` to eliminate them). diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index fe56fe7ba674..3c563ff032d0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -19,12 +19,10 @@ package org.apache.spark.rdd import java.nio.ByteBuffer import java.text.SimpleDateFormat -import java.util.Date -import java.util.{HashMap => JHashMap} +import java.util.{Date, HashMap => JHashMap} +import scala.collection.{Map, mutable} import scala.collection.JavaConversions._ -import scala.collection.Map -import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag @@ -34,16 +32,14 @@ import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf, OutputFormat} -import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat, Job => NewAPIHadoopJob, +import org.apache.hadoop.mapreduce.{Job => NewAPIHadoopJob, OutputFormat => NewOutputFormat, RecordWriter => NewRecordWriter, SparkHadoopMapReduceUtil} -import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat} import org.apache.spark._ -import org.apache.spark.annotation.Experimental -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.SparkHadoopWriter import org.apache.spark.Partitioner.defaultPartitioner import org.apache.spark.SparkContext._ +import org.apache.spark.annotation.Experimental +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.serializer.Serializer import org.apache.spark.util.Utils @@ -216,24 +212,26 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * need two additional passes. * * @param withReplacement whether to sample with or without replacement - * @param fractionByKey function mapping key to sampling rate + * @param fractions map of specific keys to sampling rates * @param seed seed for the random number generator * @param exact whether sample size needs to be exactly math.ceil(fraction * size) per stratum * @return RDD containing the sampled subset */ def sampleByKey(withReplacement: Boolean, - fractionByKey: Map[K, Double], - seed: Long = Utils.random.nextLong, - exact: Boolean = true): RDD[(K, V)]= { - require(fractionByKey.forall({case(k, v) => v >= 0.0}), "Invalid sampling rates.") + fractions: Map[K, Double], + exact: Boolean = true, + seed: Long = Utils.random.nextLong): RDD[(K, V)]= { + + require(fractions.forall({case(k, v) => v >= 0.0}), "Invalid sampling rates.") + if (withReplacement) { val counts = if (exact) Some(this.countByKey()) else None val samplingFunc = - StratifiedSampler.getPoissonSamplingFunction(self, fractionByKey, exact, counts, seed) + StratifiedSampler.getPoissonSamplingFunction(self, fractions, exact, counts, seed) self.mapPartitionsWithIndex(samplingFunc, preservesPartitioning = true) } else { val samplingFunc = - StratifiedSampler.getBernoulliSamplingFunction(self, fractionByKey, exact, seed) + StratifiedSampler.getBernoulliSamplingFunction(self, fractions, exact, seed) self.mapPartitionsWithIndex(samplingFunc, preservesPartitioning = true) } } diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 9edc73d1c577..7a44cce27362 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -875,27 +875,6 @@ abstract class RDD[T: ClassTag]( jobResult } - /** - * A version of {@link #aggregate()} that passes the TaskContext to the function that does - * aggregation for each partition. - */ - def aggregateWithContext[U: ClassTag](zeroValue: U)(seqOp: ((TaskContext, U), T) => U, - combOp: (U, U) => U): U = { - // Clone the zero value since we will also be serializing it as part of tasks - var jobResult = Utils.clone(zeroValue, sc.env.closureSerializer.newInstance()) - // pad seqOp and combOp with taskContext to conform to aggregate's signature in TraversableOnce - val paddedSeqOp = (arg1: (TaskContext, U), item: T) => (arg1._1, seqOp(arg1, item)) - val paddedcombOp = (arg1: (TaskContext, U), arg2: (TaskContext, U)) => - (arg1._1, combOp(arg1._2, arg1._2)) - val cleanSeqOp = sc.clean(paddedSeqOp) - val cleanCombOp = sc.clean(paddedcombOp) - val aggregatePartition = (tc: TaskContext, it: Iterator[T]) => - (it.aggregate(tc, zeroValue)(cleanSeqOp, cleanCombOp))._2 - val mergeResult = (index: Int, taskResult: U) => jobResult = combOp(jobResult, taskResult) - sc.runJob(this, aggregatePartition, mergeResult) - jobResult - } - /** * Return the number of elements in the RDD. */ diff --git a/core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala b/core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala index 3535fa8387af..1dd586d752ef 100644 --- a/core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala +++ b/core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala @@ -17,15 +17,43 @@ package org.apache.spark.util.random +import scala.collection.{Map, mutable} import scala.collection.mutable.ArrayBuffer -import scala.collection.{mutable, Map} +import scala.reflect.ClassTag + import org.apache.commons.math3.random.RandomDataGenerator -import org.apache.spark.{Logging, TaskContext} -import org.apache.spark.util.random.{PoissonBounds => PB} -import scala.Some +import org.apache.spark.{Logging, SparkContext, TaskContext} import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils +import org.apache.spark.util.random.{PoissonBounds => PB} private[spark] object StratifiedSampler extends Logging { + + /** + * A version of {@link #aggregate()} that passes the TaskContext to the function that does + * aggregation for each partition. This function avoids creating an extra depth in the RDD + * lineage, as opposed to using mapPartitionsWithId, which results in slightly improved run time. + */ + def aggregateWithContext[U: ClassTag, T: ClassTag](zeroValue: U) + (rdd: RDD[T], + seqOp: ((TaskContext, U), T) => U, + combOp: (U, U) => U): U = { + val sc: SparkContext = rdd.sparkContext + // Clone the zero value since we will also be serializing it as part of tasks + var jobResult = Utils.clone(zeroValue, sc.env.closureSerializer.newInstance()) + // pad seqOp and combOp with taskContext to conform to aggregate's signature in TraversableOnce + val paddedSeqOp = (arg1: (TaskContext, U), item: T) => (arg1._1, seqOp(arg1, item)) + val paddedcombOp = (arg1: (TaskContext, U), arg2: (TaskContext, U)) => + (arg1._1, combOp(arg1._2, arg1._2)) + val cleanSeqOp = sc.clean(paddedSeqOp) + val cleanCombOp = sc.clean(paddedcombOp) + val aggregatePartition = (tc: TaskContext, it: Iterator[T]) => + (it.aggregate(tc, zeroValue)(cleanSeqOp, cleanCombOp))._2 + val mergeResult = (index: Int, taskResult: U) => jobResult = combOp(jobResult, taskResult) + sc.runJob(rdd, aggregatePartition, mergeResult) + jobResult + } + /** * Returns the function used by aggregate to collect sampling statistics for each partition. */ @@ -153,7 +181,7 @@ private[spark] object StratifiedSampler extends Logging { val seqOp = StratifiedSampler.getSeqOp[K,V](false, fractionByKey, None) val combOp = StratifiedSampler.getCombOp[K]() val zeroU = new Result[K](Map[K, Stratum](), seed = seed) - val finalResult = rdd.aggregateWithContext(zeroU)(seqOp, combOp).resultMap + val finalResult = aggregateWithContext(zeroU)(rdd, seqOp, combOp).resultMap samplingRateByKey = StratifiedSampler.computeThresholdByKey(finalResult, fractionByKey) } (idx: Int, iter: Iterator[(K, V)]) => { @@ -183,7 +211,7 @@ private[spark] object StratifiedSampler extends Logging { val seqOp = StratifiedSampler.getSeqOp[K,V](true, fractionByKey, counts) val combOp = StratifiedSampler.getCombOp[K]() val zeroU = new Result[K](Map[K, Stratum](), seed = seed) - val finalResult = rdd.aggregateWithContext(zeroU)(seqOp, combOp).resultMap + val finalResult = aggregateWithContext(zeroU)(rdd, seqOp, combOp).resultMap val thresholdByKey = StratifiedSampler.computeThresholdByKey(finalResult, fractionByKey) (idx: Int, iter: Iterator[(K, V)]) => { val random = new RandomDataGenerator() diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 4ac99a9dc682..06ea6cef6700 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -106,8 +106,8 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { n: Long) = { val expectedSampleSize = stratifiedData.countByKey().mapValues(count => math.ceil(count * samplingRate).toInt) - val fractionByKey = Map("1" -> samplingRate, "0" -> samplingRate) - val sample = stratifiedData.sampleByKey(false, fractionByKey, seed, exact) + val fractions = Map("1" -> samplingRate, "0" -> samplingRate) + val sample = stratifiedData.sampleByKey(false, fractions, exact, seed) val sampleCounts = sample.countByKey() val takeSample = sample.collect() assert(sampleCounts.forall({case(k,v) => @@ -124,8 +124,8 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { n: Long) = { val expectedSampleSize = stratifiedData.countByKey().mapValues(count => math.ceil(count * samplingRate).toInt) - val fractionByKey = Map("1" -> samplingRate, "0" -> samplingRate) - val sample = stratifiedData.sampleByKey(true, fractionByKey, seed, exact) + val fractions = Map("1" -> samplingRate, "0" -> samplingRate) + val sample = stratifiedData.sampleByKey(true, fractions, exact, seed) val sampleCounts = sample.countByKey() val takeSample = sample.collect() assert(sampleCounts.forall({case(k,v) => diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index a3094a2fd626..0e5625b7645d 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -141,38 +141,6 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(result.toSet === Set(("a", 6), ("b", 2), ("c", 5))) } - test("aggregateWithContext") { - val data = Array(("a", 1), ("b", 2), ("a", 2), ("c", 5), ("a", 3)) - val numPartitions = 2 - val pairs = sc.makeRDD(data, numPartitions) - //determine the partitionId for each pair - type StringMap = HashMap[String, Int] - val partitions = pairs.collectPartitions() - val offSets = new StringMap - for (i <- 0 to numPartitions - 1) { - partitions(i).foreach({ case (k, v) => offSets.put(k, offSets.getOrElse(k, 0) + i)}) - } - val emptyMap = new StringMap { - override def default(key: String): Int = 0 - } - val mergeElement: ((TaskContext, StringMap), (String, Int)) => StringMap = (arg1, pair) => { - val stringMap = arg1._2 - val tc = arg1._1 - stringMap(pair._1) += pair._2 + tc.partitionId - stringMap - } - val mergeMaps: (StringMap, StringMap) => StringMap = (map1, map2) => { - for ((key, value) <- map2) { - map1(key) += value - } - map1 - } - val result = pairs.aggregateWithContext(emptyMap)(mergeElement, mergeMaps) - val expected = Set(("a", 6), ("b", 2), ("c", 5)) - .map({ case (k, v) => (k -> (offSets.getOrElse(k, 0) + v))}) - assert(result.toSet === expected) - } - test("basic caching") { val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() assert(rdd.collect().toList === List(1, 2, 3, 4)) From ee9d260e5eb6ee2d6912e57491daeded1704248c Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Sun, 6 Jul 2014 16:20:39 -0700 Subject: [PATCH 16/28] addressed reviewer comments --- .../apache/spark/rdd/PairRDDFunctions.scala | 9 +- .../main/scala/org/apache/spark/rdd/RDD.scala | 1 - .../spark/util/random/SamplingUtils.scala | 7 +- .../spark/util/random/StratifiedSampler.scala | 191 +++++++++++------- 4 files changed, 122 insertions(+), 86 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 7762dc038073..02550d547bb3 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -212,18 +212,15 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) exact: Boolean = true, seed: Long = Utils.random.nextLong): RDD[(K, V)]= { - require(fractions.forall({case(k, v) => v >= 0.0}), "Invalid sampling rates.") + require(fractions.forall {case(k, v) => v >= 0.0}, "Invalid sampling rates.") - if (withReplacement) { + val samplingFunc = if (withReplacement) { val counts = if (exact) Some(this.countByKey()) else None - val samplingFunc = StratifiedSampler.getPoissonSamplingFunction(self, fractions, exact, counts, seed) - self.mapPartitionsWithIndex(samplingFunc, preservesPartitioning = true) } else { - val samplingFunc = StratifiedSampler.getBernoulliSamplingFunction(self, fractions, exact, seed) - self.mapPartitionsWithIndex(samplingFunc, preservesPartitioning = true) } + self.mapPartitionsWithIndex(samplingFunc, preservesPartitioning=true) } /** diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 5205eb6170a2..4e841bc992bf 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -350,7 +350,6 @@ abstract class RDD[T: ClassTag]( /** * Return a sampled subset of this RDD. - * */ def sample(withReplacement: Boolean, fraction: Double, diff --git a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala index 0a4a3a7fc85f..78109213aa34 100644 --- a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala @@ -72,6 +72,7 @@ private[spark] object SamplingUtils { private[spark] object PoissonBounds { val delta = 1e-4 / 3.0 + val epsilon = 1e-15 /** * Compute the threshold for accepting items on the fly. The threshold value is a fairly small @@ -87,7 +88,7 @@ private[spark] object PoissonBounds { var ub = s while (lb < ub - 1.0) { val m = (lb + ub) / 2.0 - val poisson = new PoissonDistribution(m, 1e-15) + val poisson = new PoissonDistribution(m, epsilon) val y = poisson.inverseCumulativeProbability(1 - delta) if (y > s) ub = m else lb = m } @@ -96,7 +97,7 @@ private[spark] object PoissonBounds { def getMinCount(lmbd: Double): Double = { if (lmbd == 0) return 0 - val poisson = new PoissonDistribution(lmbd, 1e-15) + val poisson = new PoissonDistribution(lmbd, epsilon) poisson.inverseCumulativeProbability(delta) } @@ -114,7 +115,7 @@ private[spark] object PoissonBounds { var ub = s + math.sqrt(s / delta) // Chebyshev's inequality while (lb < ub - 1.0) { val m = (lb + ub) / 2.0 - val poisson = new PoissonDistribution(m, 1e-15) + val poisson = new PoissonDistribution(m, epsilon) val y = poisson.inverseCumulativeProbability(delta) if (y >= s) ub = m else lb = m } diff --git a/core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala b/core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala index 1dd586d752ef..712ba0cd1226 100644 --- a/core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala +++ b/core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala @@ -17,22 +17,28 @@ package org.apache.spark.util.random -import scala.collection.{Map, mutable} -import scala.collection.mutable.ArrayBuffer +import scala.collection.Map +import scala.collection.mutable.{ArrayBuffer, HashMap, Map => MMap} import scala.reflect.ClassTag import org.apache.commons.math3.random.RandomDataGenerator import org.apache.spark.{Logging, SparkContext, TaskContext} import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils -import org.apache.spark.util.random.{PoissonBounds => PB} +/** + * Auxiliary functions and data structures for the sampleByKey method in PairRDDFunctions. + * + * For more theoretical background on the sampling technqiues used here, please refer to + * http://jmlr.org/proceedings/papers/v28/meng13a.html + */ private[spark] object StratifiedSampler extends Logging { /** * A version of {@link #aggregate()} that passes the TaskContext to the function that does * aggregation for each partition. This function avoids creating an extra depth in the RDD - * lineage, as opposed to using mapPartitionsWithId, which results in slightly improved run time. + * lineage, as opposed to using mapPartitionsWithIndex, which results in slightly improved + * run time. */ def aggregateWithContext[U: ClassTag, T: ClassTag](zeroValue: U) (rdd: RDD[T], @@ -43,10 +49,10 @@ private[spark] object StratifiedSampler extends Logging { var jobResult = Utils.clone(zeroValue, sc.env.closureSerializer.newInstance()) // pad seqOp and combOp with taskContext to conform to aggregate's signature in TraversableOnce val paddedSeqOp = (arg1: (TaskContext, U), item: T) => (arg1._1, seqOp(arg1, item)) - val paddedcombOp = (arg1: (TaskContext, U), arg2: (TaskContext, U)) => + val paddedCombOp = (arg1: (TaskContext, U), arg2: (TaskContext, U)) => (arg1._1, combOp(arg1._2, arg1._2)) val cleanSeqOp = sc.clean(paddedSeqOp) - val cleanCombOp = sc.clean(paddedcombOp) + val cleanCombOp = sc.clean(paddedCombOp) val aggregatePartition = (tc: TaskContext, it: Iterator[T]) => (it.aggregate(tc, zeroValue)(cleanSeqOp, cleanCombOp))._2 val mergeResult = (index: Int, taskResult: U) => jobResult = combOp(jobResult, taskResult) @@ -58,54 +64,55 @@ private[spark] object StratifiedSampler extends Logging { * Returns the function used by aggregate to collect sampling statistics for each partition. */ def getSeqOp[K, V](withReplacement: Boolean, - fractionByKey: (K => Double), - counts: Option[Map[K, Long]]): ((TaskContext, Result[K]),(K, V)) => Result[K] = { + fractions: Map[K, Double], + counts: Option[Map[K, Long]]): ((TaskContext, Result[K]), (K, V)) => Result[K] = { val delta = 5e-5 - (U: (TaskContext, Result[K]), item: (K, V)) => { - val result = U._2 - val tc = U._1 + (output: (TaskContext, Result[K]), item: (K, V)) => { + val result = output._2 + val tc = output._1 val rng = result.getRand(tc.partitionId) - val fraction = fractionByKey(item._1) + val fraction = fractions(item._1) val stratum = result.getEntry(item._1) if (withReplacement) { - // compute q1 and q2 only if they haven't been computed already + // compute acceptBound and waitListBound only if they haven't been computed already // since they don't change from iteration to iteration. // TODO change this to the streaming version - if (stratum.q1.isEmpty || stratum.q2.isEmpty) { + if (stratum.areBoundsEmpty) { val n = counts.get(item._1) - val s = math.ceil(n * fraction).toLong - val lmbd1 = PB.getLowerBound(s) - val minCount = PB.getMinCount(lmbd1) - val lmbd2 = if (lmbd1 == 0) PB.getUpperBound(s) else PB.getUpperBound(s - minCount) - val q1 = lmbd1 / n - val q2 = lmbd2 / n - stratum.q1 = Some(q1) - stratum.q2 = Some(q2) + val sampleSize = math.ceil(n * fraction).toLong + val lmbd1 = PoissonBounds.getLowerBound(sampleSize) + val minCount = PoissonBounds.getMinCount(lmbd1) + val lmbd2 = if (lmbd1 == 0) { + PoissonBounds.getUpperBound(sampleSize) + } else { + PoissonBounds.getUpperBound(sampleSize - minCount) + } + stratum.acceptBound = lmbd1 / n + stratum.waitListBound = lmbd2 / n } - val x1 = if (stratum.q1.get == 0) 0L else rng.nextPoisson(stratum.q1.get) + val x1 = if (stratum.acceptBound == 0.0) 0L else rng.nextPoisson(stratum.acceptBound) if (x1 > 0) { stratum.incrNumAccepted(x1) } - val x2 = rng.nextPoisson(stratum.q2.get).toInt + val x2 = rng.nextPoisson(stratum.waitListBound).toInt if (x2 > 0) { stratum.addToWaitList(ArrayBuffer.fill(x2)(rng.nextUniform(0.0, 1.0))) } } else { - // We use the streaming version of the algorithm for sampling without replacement. - // Hence, q1 and q2 change on every iteration. - val g1 = - math.log(delta) / stratum.numItems - val g2 = (2.0 / 3.0) * g1 - val q1 = math.max(0, fraction + g2 - math.sqrt((g2 * g2 + 3 * g2 * fraction))) - val q2 = math.min(1, fraction + g1 + math.sqrt(g1 * g1 + 2 * g1 * fraction)) + // We use the streaming version of the algorithm for sampling without replacement to avoid + // using an extra pass over the RDD for computing the count. + // Hence, acceptBound and waitListBound change on every iteration. + val g1 = - math.log(delta) / stratum.numItems // gamma1 + val g2 = (2.0 / 3.0) * g1 // gamma 2 + stratum.acceptBound = math.max(0, fraction + g2 - math.sqrt((g2 * g2 + 3 * g2 * fraction))) + stratum.waitListBound = math.min(1, fraction + g1 + math.sqrt(g1 * g1 + 2 * g1 * fraction)) val x = rng.nextUniform(0.0, 1.0) - if (x < q1) { + if (x < stratum.acceptBound) { stratum.incrNumAccepted() - } else if (x < q2) { + } else if (x < stratum.waitListBound) { stratum.addToWaitList(x) } - stratum.q1 = Some(q1) - stratum.q2 = Some(q2) } stratum.incrNumItems() result @@ -119,18 +126,15 @@ private[spark] object StratifiedSampler extends Logging { def getCombOp[K](): (Result[K], Result[K]) => Result[K] = { (r1: Result[K], r2: Result[K]) => { // take union of both key sets in case one partition doesn't contain all keys - val keyUnion = r1.resultMap.keys.toSet.union(r2.resultMap.keys.toSet) + val keyUnion = r1.resultMap.keySet.union(r2.resultMap.keySet) // Use r2 to keep the combined result since r1 is usual empty for (key <- keyUnion) { val entry1 = r1.resultMap.get(key) - val entry2 = r2.resultMap.get(key) - if (entry2.isEmpty && entry1.isDefined) { - r2.resultMap += (key -> entry1.get) - } else if (entry1.isDefined && entry2.isDefined) { - entry2.get.addToWaitList(entry1.get.waitList) - entry2.get.incrNumAccepted(entry1.get.numAccepted) - entry2.get.incrNumItems(entry1.get.numItems) + if (r2.resultMap.contains(key)) { + r2.resultMap(key).merge(entry1) + } else { + r2.addEntry(key, entry1) } } r2 @@ -138,23 +142,32 @@ private[spark] object StratifiedSampler extends Logging { } /** - * Given the result returned by the aggregate function, we need to determine the threshold used - * to accept items to generate the exact sample size. + * Given the result returned by the aggregate function, determine the threshold for accepting + * items to generate exact sample size. + * + * To do so, we compute sampleSize = math.ceil(size * samplingRate) for each stratum and compare + * it to the number of items that were accepted instantly and the number of items in the waitlist + * for that stratum. Most of the time, numAccepted <= sampleSize <= (numAccepted + numWaitlisted), + * which means we need to sort the elements in the waitlist by their associated values in order + * to find the value T s.t. |{elements in the stratum whose associated values <= T}| = sampleSize. + * Note that all elements in the waitlist have values >= bound for instant accept, so a T value + * in the waitlist range would allow all elements that were instantly accepted on the first pass + * to be included in the sample. */ - def computeThresholdByKey[K](finalResult: Map[K, Stratum], fractionByKey: (K => Double)): - (K => Double) = { - val thresholdByKey = new mutable.HashMap[K, Double]() + def computeThresholdByKey[K](finalResult: Map[K, Stratum], + fractions: Map[K, Double]): + Map[K, Double] = { + val thresholdByKey = new HashMap[K, Double]() for ((key, stratum) <- finalResult) { - val fraction = fractionByKey(key) - val s = math.ceil(stratum.numItems * fraction).toLong + val s = math.ceil(stratum.numItems * fractions(key)).toLong if (stratum.numAccepted > s) { logWarning("Pre-accepted too many") - thresholdByKey += (key -> stratum.q1.get) + thresholdByKey += (key -> stratum.acceptBound) } else { val numWaitListAccepted = (s - stratum.numAccepted).toInt if (numWaitListAccepted >= stratum.waitList.size) { logWarning("WaitList too short") - thresholdByKey += (key -> stratum.q2.get) + thresholdByKey += (key -> stratum.waitListBound) } else { thresholdByKey += (key -> stratum.waitList.sorted.apply(numWaitListAccepted)) } @@ -172,17 +185,17 @@ private[spark] object StratifiedSampler extends Logging { * The sampling function has a unique seed per partition. */ def getBernoulliSamplingFunction[K, V](rdd: RDD[(K, V)], - fractionByKey: K => Double, + fractions: Map[K, Double], exact: Boolean, seed: Long): (Int, Iterator[(K, V)]) => Iterator[(K, V)] = { - var samplingRateByKey = fractionByKey + var samplingRateByKey = fractions if (exact) { // determine threshold for each stratum and resample - val seqOp = StratifiedSampler.getSeqOp[K,V](false, fractionByKey, None) - val combOp = StratifiedSampler.getCombOp[K]() - val zeroU = new Result[K](Map[K, Stratum](), seed = seed) + val seqOp = getSeqOp[K, V](false, fractions, None) + val combOp = getCombOp[K]() + val zeroU = new Result[K](new HashMap[K, Stratum](), seed = seed) val finalResult = aggregateWithContext(zeroU)(rdd, seqOp, combOp).resultMap - samplingRateByKey = StratifiedSampler.computeThresholdByKey(finalResult, fractionByKey) + samplingRateByKey = computeThresholdByKey(finalResult, fractions) } (idx: Int, iter: Iterator[(K, V)]) => { val random = new RandomDataGenerator @@ -201,24 +214,24 @@ private[spark] object StratifiedSampler extends Logging { * * The sampling function has a unique seed per partition. */ - def getPoissonSamplingFunction[K, V](rdd:RDD[(K, V)], - fractionByKey: K => Double, + def getPoissonSamplingFunction[K, V](rdd: RDD[(K, V)], + fractions: Map[K, Double], exact: Boolean, counts: Option[Map[K, Long]], seed: Long): (Int, Iterator[(K, V)]) => Iterator[(K, V)] = { // TODO implement the streaming version of sampling w/ replacement that doesn't require counts if (exact) { - val seqOp = StratifiedSampler.getSeqOp[K,V](true, fractionByKey, counts) - val combOp = StratifiedSampler.getCombOp[K]() - val zeroU = new Result[K](Map[K, Stratum](), seed = seed) + val seqOp = getSeqOp[K, V](true, fractions, counts) + val combOp = getCombOp[K]() + val zeroU = new Result[K](new HashMap[K, Stratum](), seed=seed) val finalResult = aggregateWithContext(zeroU)(rdd, seqOp, combOp).resultMap - val thresholdByKey = StratifiedSampler.computeThresholdByKey(finalResult, fractionByKey) + val thresholdByKey = computeThresholdByKey(finalResult, fractions) (idx: Int, iter: Iterator[(K, V)]) => { val random = new RandomDataGenerator() random.reSeed(seed + idx) iter.flatMap { t => - val q1 = finalResult.get(t._1).get.q1.getOrElse(0.0) - val q2 = finalResult.get(t._1).get.q2.getOrElse(0.0) + val q1 = finalResult(t._1).acceptBound + val q2 = finalResult(t._1).waitListBound val x1 = if (q1 == 0) 0L else random.nextPoisson(q1) val x2 = random.nextPoisson(q2).toInt val x = x1 + (0 until x2).count(i => random.nextUniform(0.0, 1.0) < thresholdByKey(t._1)) @@ -234,7 +247,7 @@ private[spark] object StratifiedSampler extends Logging { val random = new RandomDataGenerator() random.reSeed(seed + idx) iter.flatMap { t => - val count = random.nextPoisson(fractionByKey(t._1)).toInt + val count = random.nextPoisson(fractions(t._1)).toInt if (count > 0) { Iterator.fill(count)(t) } else { @@ -250,20 +263,40 @@ private[spark] object StratifiedSampler extends Logging { * Object used by seqOp to keep track of the number of items accepted and items waitlisted per * stratum, as well as the bounds for accepting and waitlisting items. */ -private[random] class Stratum(var numItems: Long = 0L, var numAccepted: Long = 0L) +private class Stratum(var numItems: Long = 0L, var numAccepted: Long = 0L) extends Serializable { - var waitList: ArrayBuffer[Double] = new ArrayBuffer[Double] - var q1: Option[Double] = None // upper bound for accepting item instantly - var q2: Option[Double] = None // upper bound for adding item to waitlist + private val _waitList = new ArrayBuffer[Double] + private var _acceptBound: Option[Double] = None // upper bound for accepting item instantly + private var _waitListBound: Option[Double] = None // upper bound for adding item to waitlist def incrNumItems(by: Long = 1L) = numItems += by def incrNumAccepted(by: Long = 1L) = numAccepted += by - def addToWaitList(elem: Double) = waitList += elem + def addToWaitList(elem: Double) = _waitList += elem + + def addToWaitList(elems: ArrayBuffer[Double]) = _waitList ++= elems + + def waitList = _waitList + + def acceptBound = _acceptBound.getOrElse(0.0) - def addToWaitList(elems: ArrayBuffer[Double]) = waitList ++= elems + def acceptBound_= (value: Double) = _acceptBound = Some(value) + + def waitListBound = _waitListBound.getOrElse(0.0) + + def waitListBound_= (value: Double) = _waitListBound = Some(value) + + def areBoundsEmpty = _acceptBound.isEmpty || _waitListBound.isEmpty + + def merge(other: Option[Stratum]): Unit = { + if (other.isDefined) { + addToWaitList(other.get.waitList) + incrNumAccepted(other.get.numAccepted) + incrNumItems(other.get.numItems) + } + } } /** @@ -272,12 +305,12 @@ private[random] class Stratum(var numItems: Long = 0L, var numAccepted: Long = 0 * When used by seqOp for each partition, we also keep track of the partition ID in this object * to make sure a single random number generator with a unique seed is used for each partition. */ -private[random] class Result[K](var resultMap: Map[K, Stratum], - var cachedPartitionId: Option[Int] = None, - val seed: Long) +private[random] class Result[K](val resultMap: MMap[K, Stratum], + var cachedPartitionId: Option[Int] = None, + val seed: Long) extends Serializable { - var rand: RandomDataGenerator = new RandomDataGenerator + var rand = new RandomDataGenerator def getEntry(key: K, numItems: Long = 0L): Stratum = { if (!resultMap.contains(key)) { @@ -286,6 +319,12 @@ private[random] class Result[K](var resultMap: Map[K, Stratum], resultMap(key) } + def addEntry(key: K, entry: Option[Stratum]): Unit = { + if (entry.isDefined) { + resultMap += (key -> entry.get) + } + } + def getRand(partitionId: Int): RandomDataGenerator = { if (cachedPartitionId.isEmpty || cachedPartitionId.get != partitionId) { cachedPartitionId = Some(partitionId) From 9884a9f03b18f3170d90d168c246d78fc463028e Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Tue, 8 Jul 2014 11:58:39 -0700 Subject: [PATCH 17/28] style fix --- .../main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 4 ++-- .../org/apache/spark/util/random/StratifiedSampler.scala | 3 +-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 02550d547bb3..1abbd676de3d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -216,9 +216,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val samplingFunc = if (withReplacement) { val counts = if (exact) Some(this.countByKey()) else None - StratifiedSampler.getPoissonSamplingFunction(self, fractions, exact, counts, seed) + StratifiedSampler.getPoissonSamplingFunction(self, fractions, exact, counts, seed) } else { - StratifiedSampler.getBernoulliSamplingFunction(self, fractions, exact, seed) + StratifiedSampler.getBernoulliSamplingFunction(self, fractions, exact, seed) } self.mapPartitionsWithIndex(samplingFunc, preservesPartitioning=true) } diff --git a/core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala b/core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala index 712ba0cd1226..c9683e75e1da 100644 --- a/core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala +++ b/core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala @@ -155,8 +155,7 @@ private[spark] object StratifiedSampler extends Logging { * to be included in the sample. */ def computeThresholdByKey[K](finalResult: Map[K, Stratum], - fractions: Map[K, Double]): - Map[K, Double] = { + fractions: Map[K, Double]): Map[K, Double] = { val thresholdByKey = new HashMap[K, Double]() for ((key, stratum) <- finalResult) { val s = math.ceil(stratum.numItems * fractions(key)).toLong From 680b677bc5276e1499c59c7e24abfae7d85e5c7d Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Tue, 8 Jul 2014 16:49:25 -0700 Subject: [PATCH 18/28] use mapPartitionWithIndex instead also better readability and lots more comments. --- .../spark/util/random/StratifiedSampler.scala | 218 ++++++++---------- 1 file changed, 97 insertions(+), 121 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala b/core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala index c9683e75e1da..201d92c18b4c 100644 --- a/core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala +++ b/core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala @@ -19,45 +19,55 @@ package org.apache.spark.util.random import scala.collection.Map import scala.collection.mutable.{ArrayBuffer, HashMap, Map => MMap} -import scala.reflect.ClassTag import org.apache.commons.math3.random.RandomDataGenerator -import org.apache.spark.{Logging, SparkContext, TaskContext} +import org.apache.spark.Logging import org.apache.spark.rdd.RDD -import org.apache.spark.util.Utils /** * Auxiliary functions and data structures for the sampleByKey method in PairRDDFunctions. * - * For more theoretical background on the sampling technqiues used here, please refer to + * Essentially, when exact sample size is necessary, we make additional passes over the RDD to + * compute the exact threshold value to use for each stratum to guarantee exact sample size with + * high probability. This is achieved by maintaining a waitlist of size O(log(s)), where s is the + * desired sample size for each stratum. + * + * Like in simple random sampling, we generate a random value for each item from the + * uniform distribution [0.0, 1.0]. All items with values <= min(values of items in the waitlist) + * are accepted into the sample instantly. The threshold for instant accept is designed so that + * s - numAccepted = O(log(s)), where s is again the desired sample size. Thus, by maintaining a + * waitlist size = O(log(s)), we will be able to create a sample of the exact size s by adding + * a portion of the waitlist to the set of items that are instantly accepted. The exact threshold + * is computed by sorting the values in the waitlist and picking the value at (s - numAccepted). + * + * Note that since we use the same seed for the RNG when computing the thresholds and the actual + * sample, our computed thresholds are guaranteed to produce the desired sample size. + * + * For more theoretical background on the sampling techniques used here, please refer to * http://jmlr.org/proceedings/papers/v28/meng13a.html */ + private[spark] object StratifiedSampler extends Logging { /** - * A version of {@link #aggregate()} that passes the TaskContext to the function that does - * aggregation for each partition. This function avoids creating an extra depth in the RDD - * lineage, as opposed to using mapPartitionsWithIndex, which results in slightly improved - * run time. + * Count the number of items instantly accepted and generate the waitlist for each stratum. + * + * This is only invoked when exact sample size is required. */ - def aggregateWithContext[U: ClassTag, T: ClassTag](zeroValue: U) - (rdd: RDD[T], - seqOp: ((TaskContext, U), T) => U, - combOp: (U, U) => U): U = { - val sc: SparkContext = rdd.sparkContext - // Clone the zero value since we will also be serializing it as part of tasks - var jobResult = Utils.clone(zeroValue, sc.env.closureSerializer.newInstance()) - // pad seqOp and combOp with taskContext to conform to aggregate's signature in TraversableOnce - val paddedSeqOp = (arg1: (TaskContext, U), item: T) => (arg1._1, seqOp(arg1, item)) - val paddedCombOp = (arg1: (TaskContext, U), arg2: (TaskContext, U)) => - (arg1._1, combOp(arg1._2, arg1._2)) - val cleanSeqOp = sc.clean(paddedSeqOp) - val cleanCombOp = sc.clean(paddedCombOp) - val aggregatePartition = (tc: TaskContext, it: Iterator[T]) => - (it.aggregate(tc, zeroValue)(cleanSeqOp, cleanCombOp))._2 - val mergeResult = (index: Int, taskResult: U) => jobResult = combOp(jobResult, taskResult) - sc.runJob(rdd, aggregatePartition, mergeResult) - jobResult + def getCounts[K, V](rdd: RDD[(K, V)], + withReplacement: Boolean, + fractions: Map[K, Double], + counts: Option[Map[K, Long]], + seed: Long): MMap[K, Stratum] = { + val combOp = getCombOp[K] + val mappedPartitionRDD = rdd.mapPartitionsWithIndex({ case (partition, iter) => + val zeroU: MMap[K, Stratum] = new HashMap[K, Stratum]() + val rng = new RandomDataGenerator() + rng.reSeed(seed + partition) + val seqOp = getSeqOp(withReplacement, fractions, rng, counts) + Iterator(iter.aggregate(zeroU)(seqOp, combOp)) + }, preservesPartitioning=true) + mappedPartitionRDD.reduce(combOp) } /** @@ -65,20 +75,23 @@ private[spark] object StratifiedSampler extends Logging { */ def getSeqOp[K, V](withReplacement: Boolean, fractions: Map[K, Double], - counts: Option[Map[K, Long]]): ((TaskContext, Result[K]), (K, V)) => Result[K] = { + rng: RandomDataGenerator, + counts: Option[Map[K, Long]]): (MMap[K, Stratum], (K, V)) => MMap[K, Stratum] = { val delta = 5e-5 - (output: (TaskContext, Result[K]), item: (K, V)) => { - val result = output._2 - val tc = output._1 - val rng = result.getRand(tc.partitionId) - val fraction = fractions(item._1) - val stratum = result.getEntry(item._1) + (result: MMap[K, Stratum], item: (K, V)) => { + val key = item._1 + val fraction = fractions(key) + if (!result.contains(key)) { + result += (key -> new Stratum()) + } + val stratum = result(key) + if (withReplacement) { // compute acceptBound and waitListBound only if they haven't been computed already // since they don't change from iteration to iteration. // TODO change this to the streaming version if (stratum.areBoundsEmpty) { - val n = counts.get(item._1) + val n = counts.get(key) val sampleSize = math.ceil(n * fraction).toLong val lmbd1 = PoissonBounds.getLowerBound(sampleSize) val minCount = PoissonBounds.getMinCount(lmbd1) @@ -90,21 +103,22 @@ private[spark] object StratifiedSampler extends Logging { stratum.acceptBound = lmbd1 / n stratum.waitListBound = lmbd2 / n } - val x1 = if (stratum.acceptBound == 0.0) 0L else rng.nextPoisson(stratum.acceptBound) - if (x1 > 0) { - stratum.incrNumAccepted(x1) + val acceptBound = stratum.acceptBound + val copiesAccepted = if (acceptBound == 0.0) 0L else rng.nextPoisson(acceptBound) + if (copiesAccepted > 0) { + stratum.incrNumAccepted(copiesAccepted) } - val x2 = rng.nextPoisson(stratum.waitListBound).toInt - if (x2 > 0) { - stratum.addToWaitList(ArrayBuffer.fill(x2)(rng.nextUniform(0.0, 1.0))) + val copiesWaitlisted = rng.nextPoisson(stratum.waitListBound).toInt + if (copiesWaitlisted > 0) { + stratum.addToWaitList(ArrayBuffer.fill(copiesWaitlisted)(rng.nextUniform(0.0, 1.0))) } } else { // We use the streaming version of the algorithm for sampling without replacement to avoid // using an extra pass over the RDD for computing the count. // Hence, acceptBound and waitListBound change on every iteration. val g1 = - math.log(delta) / stratum.numItems // gamma1 - val g2 = (2.0 / 3.0) * g1 // gamma 2 - stratum.acceptBound = math.max(0, fraction + g2 - math.sqrt((g2 * g2 + 3 * g2 * fraction))) + val g2 = (2.0 / 3.0) * g1 // gamma 2 + stratum.acceptBound = math.max(0, fraction + g2 - math.sqrt(g2 * g2 + 3 * g2 * fraction)) stratum.waitListBound = math.min(1, fraction + g1 + math.sqrt(g1 * g1 + 2 * g1 * fraction)) val x = rng.nextUniform(0.0, 1.0) @@ -120,21 +134,20 @@ private[spark] object StratifiedSampler extends Logging { } /** - * Returns the function used by aggregate to combine results from different partitions, as - * returned by seqOp. + * Returns the function used combine results returned by seqOp from different partitions. */ - def getCombOp[K](): (Result[K], Result[K]) => Result[K] = { - (r1: Result[K], r2: Result[K]) => { + def getCombOp[K]: (MMap[K, Stratum], MMap[K, Stratum]) => MMap[K, Stratum] = { + (r1: MMap[K, Stratum], r2: MMap[K, Stratum]) => { // take union of both key sets in case one partition doesn't contain all keys - val keyUnion = r1.resultMap.keySet.union(r2.resultMap.keySet) - - // Use r2 to keep the combined result since r1 is usual empty - for (key <- keyUnion) { - val entry1 = r1.resultMap.get(key) - if (r2.resultMap.contains(key)) { - r2.resultMap(key).merge(entry1) + for (key <- r1.keySet.union(r2.keySet)) { + // Use r2 to keep the combined result since r1 is usual empty + val entry1 = r1.get(key) + if (r2.contains(key)) { + r2(key).merge(entry1) } else { - r2.addEntry(key, entry1) + if (entry1.isDefined) { + r2 += (key -> entry1.get) + } } } r2 @@ -142,8 +155,8 @@ private[spark] object StratifiedSampler extends Logging { } /** - * Given the result returned by the aggregate function, determine the threshold for accepting - * items to generate exact sample size. + * Given the result returned by getCounts, determine the threshold for accepting items to + * generate exact sample size. * * To do so, we compute sampleSize = math.ceil(size * samplingRate) for each stratum and compare * it to the number of items that were accepted instantly and the number of items in the waitlist @@ -158,12 +171,12 @@ private[spark] object StratifiedSampler extends Logging { fractions: Map[K, Double]): Map[K, Double] = { val thresholdByKey = new HashMap[K, Double]() for ((key, stratum) <- finalResult) { - val s = math.ceil(stratum.numItems * fractions(key)).toLong - if (stratum.numAccepted > s) { + val sampleSize = math.ceil(stratum.numItems * fractions(key)).toLong + if (stratum.numAccepted > sampleSize) { logWarning("Pre-accepted too many") thresholdByKey += (key -> stratum.acceptBound) } else { - val numWaitListAccepted = (s - stratum.numAccepted).toInt + val numWaitListAccepted = (sampleSize - stratum.numAccepted).toInt if (numWaitListAccepted >= stratum.waitList.size) { logWarning("WaitList too short") thresholdByKey += (key -> stratum.waitListBound) @@ -190,16 +203,13 @@ private[spark] object StratifiedSampler extends Logging { var samplingRateByKey = fractions if (exact) { // determine threshold for each stratum and resample - val seqOp = getSeqOp[K, V](false, fractions, None) - val combOp = getCombOp[K]() - val zeroU = new Result[K](new HashMap[K, Stratum](), seed = seed) - val finalResult = aggregateWithContext(zeroU)(rdd, seqOp, combOp).resultMap + val finalResult = getCounts(rdd, false, fractions, None, seed) samplingRateByKey = computeThresholdByKey(finalResult, fractions) } (idx: Int, iter: Iterator[(K, V)]) => { - val random = new RandomDataGenerator - random.reSeed(seed + idx) - iter.filter(t => random.nextUniform(0.0, 1.0) < samplingRateByKey(t._1)) + val rng = new RandomDataGenerator + rng.reSeed(seed + idx) + iter.filter(t => rng.nextUniform(0.0, 1.0) < samplingRateByKey(t._1)) } } @@ -220,22 +230,21 @@ private[spark] object StratifiedSampler extends Logging { seed: Long): (Int, Iterator[(K, V)]) => Iterator[(K, V)] = { // TODO implement the streaming version of sampling w/ replacement that doesn't require counts if (exact) { - val seqOp = getSeqOp[K, V](true, fractions, counts) - val combOp = getCombOp[K]() - val zeroU = new Result[K](new HashMap[K, Stratum](), seed=seed) - val finalResult = aggregateWithContext(zeroU)(rdd, seqOp, combOp).resultMap + val finalResult = getCounts(rdd, true, fractions, counts, seed) val thresholdByKey = computeThresholdByKey(finalResult, fractions) (idx: Int, iter: Iterator[(K, V)]) => { - val random = new RandomDataGenerator() - random.reSeed(seed + idx) - iter.flatMap { t => - val q1 = finalResult(t._1).acceptBound - val q2 = finalResult(t._1).waitListBound - val x1 = if (q1 == 0) 0L else random.nextPoisson(q1) - val x2 = random.nextPoisson(q2).toInt - val x = x1 + (0 until x2).count(i => random.nextUniform(0.0, 1.0) < thresholdByKey(t._1)) - if (x > 0) { - Iterator.fill(x.toInt)(t) + val rng = new RandomDataGenerator() + rng.reSeed(seed + idx) + iter.flatMap { item => + val key = item._1 + val q1 = finalResult(key).acceptBound + val q2 = finalResult(key).waitListBound + val copiesAccepted = if (q1 == 0) 0L else rng.nextPoisson(q1) + val copiesWailisted = rng.nextPoisson(q2).toInt + val copiesInSample = copiesAccepted + + (0 until copiesWailisted).count(i => rng.nextUniform(0.0, 1.0) < thresholdByKey(key)) + if (copiesInSample > 0) { + Iterator.fill(copiesInSample.toInt)(item) } else { Iterator.empty } @@ -243,12 +252,12 @@ private[spark] object StratifiedSampler extends Logging { } } else { (idx: Int, iter: Iterator[(K, V)]) => { - val random = new RandomDataGenerator() - random.reSeed(seed + idx) - iter.flatMap { t => - val count = random.nextPoisson(fractions(t._1)).toInt + val rng = new RandomDataGenerator() + rng.reSeed(seed + idx) + iter.flatMap { item => + val count = rng.nextPoisson(fractions(item._1)).toInt if (count > 0) { - Iterator.fill(count)(t) + Iterator.fill(count)(item) } else { Iterator.empty } @@ -261,8 +270,10 @@ private[spark] object StratifiedSampler extends Logging { /** * Object used by seqOp to keep track of the number of items accepted and items waitlisted per * stratum, as well as the bounds for accepting and waitlisting items. + * + * `[random]` here is necessary since it's in the return type signature of seqOp defined above */ -private class Stratum(var numItems: Long = 0L, var numAccepted: Long = 0L) +private[random] class Stratum(var numItems: Long = 0L, var numAccepted: Long = 0L) extends Serializable { private val _waitList = new ArrayBuffer[Double] @@ -297,38 +308,3 @@ private class Stratum(var numItems: Long = 0L, var numAccepted: Long = 0L) } } } - -/** - * Object used by seqOp and combOp to keep track of the sampling statistics for all strata. - * - * When used by seqOp for each partition, we also keep track of the partition ID in this object - * to make sure a single random number generator with a unique seed is used for each partition. - */ -private[random] class Result[K](val resultMap: MMap[K, Stratum], - var cachedPartitionId: Option[Int] = None, - val seed: Long) - extends Serializable { - - var rand = new RandomDataGenerator - - def getEntry(key: K, numItems: Long = 0L): Stratum = { - if (!resultMap.contains(key)) { - resultMap += (key -> new Stratum(numItems)) - } - resultMap(key) - } - - def addEntry(key: K, entry: Option[Stratum]): Unit = { - if (entry.isDefined) { - resultMap += (key -> entry.get) - } - } - - def getRand(partitionId: Int): RandomDataGenerator = { - if (cachedPartitionId.isEmpty || cachedPartitionId.get != partitionId) { - cachedPartitionId = Some(partitionId) - rand.reSeed(seed + partitionId) - } - rand - } -} From a10e68dd9c23b0fc1f25effd9ccd0ac3e7299206 Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Tue, 8 Jul 2014 18:09:14 -0700 Subject: [PATCH 19/28] style fix --- .../spark/util/random/SamplingUtils.scala | 9 +++-- .../spark/util/random/StratifiedSampler.scala | 33 ++++++++++--------- .../spark/rdd/PairRDDFunctionsSuite.scala | 23 ++++++++----- 3 files changed, 37 insertions(+), 28 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala index 78109213aa34..f697d88509f3 100644 --- a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala @@ -96,9 +96,12 @@ private[spark] object PoissonBounds { } def getMinCount(lmbd: Double): Double = { - if (lmbd == 0) return 0 - val poisson = new PoissonDistribution(lmbd, epsilon) - poisson.inverseCumulativeProbability(delta) + if (lmbd == 0) { + 0 + } else { + val poisson = new PoissonDistribution(lmbd, epsilon) + poisson.inverseCumulativeProbability(delta) + } } /** diff --git a/core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala b/core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala index 201d92c18b4c..28ea39352cce 100644 --- a/core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala +++ b/core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala @@ -116,10 +116,12 @@ private[spark] object StratifiedSampler extends Logging { // We use the streaming version of the algorithm for sampling without replacement to avoid // using an extra pass over the RDD for computing the count. // Hence, acceptBound and waitListBound change on every iteration. - val g1 = - math.log(delta) / stratum.numItems // gamma1 - val g2 = (2.0 / 3.0) * g1 // gamma 2 - stratum.acceptBound = math.max(0, fraction + g2 - math.sqrt(g2 * g2 + 3 * g2 * fraction)) - stratum.waitListBound = math.min(1, fraction + g1 + math.sqrt(g1 * g1 + 2 * g1 * fraction)) + val gamma1 = - math.log(delta) / stratum.numItems + val gamma2 = (2.0 / 3.0) * gamma1 + stratum.acceptBound = math.max(0, + fraction + gamma2 - math.sqrt(gamma2 * gamma2 + 3 * gamma2 * fraction)) + stratum.waitListBound = math.min(1, + fraction + gamma1 + math.sqrt(gamma1 * gamma1 + 2 * gamma1 * fraction)) val x = rng.nextUniform(0.0, 1.0) if (x < stratum.acceptBound) { @@ -137,20 +139,20 @@ private[spark] object StratifiedSampler extends Logging { * Returns the function used combine results returned by seqOp from different partitions. */ def getCombOp[K]: (MMap[K, Stratum], MMap[K, Stratum]) => MMap[K, Stratum] = { - (r1: MMap[K, Stratum], r2: MMap[K, Stratum]) => { + (result1: MMap[K, Stratum], result2: MMap[K, Stratum]) => { // take union of both key sets in case one partition doesn't contain all keys - for (key <- r1.keySet.union(r2.keySet)) { - // Use r2 to keep the combined result since r1 is usual empty - val entry1 = r1.get(key) - if (r2.contains(key)) { - r2(key).merge(entry1) + for (key <- result1.keySet.union(result2.keySet)) { + // Use result2 to keep the combined result since r1 is usual empty + val entry1 = result1.get(key) + if (result2.contains(key)) { + result2(key).merge(entry1) } else { if (entry1.isDefined) { - r2 += (key -> entry1.get) + result2 += (key -> entry1.get) } } } - r2 + result2 } } @@ -237,10 +239,9 @@ private[spark] object StratifiedSampler extends Logging { rng.reSeed(seed + idx) iter.flatMap { item => val key = item._1 - val q1 = finalResult(key).acceptBound - val q2 = finalResult(key).waitListBound - val copiesAccepted = if (q1 == 0) 0L else rng.nextPoisson(q1) - val copiesWailisted = rng.nextPoisson(q2).toInt + val acceptBound = finalResult(key).acceptBound + val copiesAccepted = if (acceptBound == 0) 0L else rng.nextPoisson(acceptBound) + val copiesWailisted = rng.nextPoisson(finalResult(key).waitListBound).toInt val copiesInSample = copiesAccepted + (0 until copiesWailisted).count(i => rng.nextUniform(0.0, 1.0) < thresholdByKey(key)) if (copiesInSample > 0) { diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 8b6e5199cde3..a86d02360a6b 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -88,8 +88,11 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { (x: Int) => if (x % 10 < (10 * fractionPositive).toInt) "1" else "0" } - def checkSize(exact: Boolean, withReplacement: Boolean, - expected: Long, actual: Long, p: Double): Boolean = { + def checkSize(exact: Boolean, + withReplacement: Boolean, + expected: Long, + actual: Long, + p: Double): Boolean = { if (exact) { return expected == actual } @@ -110,8 +113,8 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { val sample = stratifiedData.sampleByKey(false, fractions, exact, seed) val sampleCounts = sample.countByKey() val takeSample = sample.collect() - assert(sampleCounts.forall({case(k,v) => - checkSize(exact, false, expectedSampleSize(k), v, samplingRate)})) + assert(sampleCounts.forall {case(k,v) => + checkSize(exact, false, expectedSampleSize(k), v, samplingRate)}) assert(takeSample.size === takeSample.toSet.size) assert(takeSample.forall(x => 1 <= x._2 && x._2 <= n), s"elements not in [1, $n]") } @@ -128,9 +131,9 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { val sample = stratifiedData.sampleByKey(true, fractions, exact, seed) val sampleCounts = sample.countByKey() val takeSample = sample.collect() - assert(sampleCounts.forall({case(k,v) => - checkSize(exact, true, expectedSampleSize(k), v, samplingRate)})) - val groupedByKey = takeSample.groupBy({case(k, v) => k}) + assert(sampleCounts.forall {case(k,v) => + checkSize(exact, true, expectedSampleSize(k), v, samplingRate)}) + val groupedByKey = takeSample.groupBy {case(k, v) => k} for ((key, v) <- groupedByKey) { if (expectedSampleSize(key) >= 100 && samplingRate >= 0.1) { // sample large enough for there to be repeats with high likelihood @@ -146,8 +149,10 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { assert(takeSample.forall(x => 1 <= x._2 && x._2 <= n), s"elements not in [1, $n]") } - def checkAllCombos(stratifiedData: RDD[(String, Int)], samplingRate: Double, - seed: Long, n: Long) { + def checkAllCombos(stratifiedData: RDD[(String, Int)], + samplingRate: Double, + seed: Long, + n: Long) = { takeSampleAndValidateBernoulli(stratifiedData, true, samplingRate, seed, n) takeSampleAndValidateBernoulli(stratifiedData, false, samplingRate, seed, n) takeSampleAndValidatePoisson(stratifiedData, true, samplingRate, seed, n) From f4c21f324075bc5e7b8cba07a5c47d23fade542f Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Mon, 14 Jul 2014 18:34:10 -0700 Subject: [PATCH 20/28] Reviewer comments Added BernoulliBounds --- .../apache/spark/api/java/JavaPairRDD.scala | 41 +++++- .../apache/spark/rdd/PairRDDFunctions.scala | 30 ++-- .../spark/util/random/SamplingUtils.scala | 23 ++- ...er.scala => StratifiedSamplingUtils.scala} | 137 ++++++++---------- .../spark/rdd/PairRDDFunctionsSuite.scala | 28 ++-- 5 files changed, 152 insertions(+), 107 deletions(-) rename core/src/main/scala/org/apache/spark/util/random/{StratifiedSampler.scala => StratifiedSamplingUtils.scala} (70%) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 147acf8409c2..31bf8dced263 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -131,6 +131,15 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) /** * Return a subset of this RDD sampled by key (via stratified sampling). + * + * Create a sample of this RDD using variable sampling rates for different keys as specified by + * `fractions`, a key to sampling rate map. + * + * If `exact` is set to false, create the sample via simple random sampling, with one pass + * over the RDD, to produce a sample of size that's approximately equal to the sum of + * math.ceil(numItems * samplingRate) over all key values; otherwise, use additional passes over + * the RDD to create a sample size that's exactly equal to the sum of + * math.ceil(numItems * samplingRate) over all key values. */ def sampleByKey(withReplacement: Boolean, fractions: JMap[K, Double], @@ -138,9 +147,19 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) seed: Long): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.sampleByKey(withReplacement, fractions, exact, seed)) - /** * Return a subset of this RDD sampled by key (via stratified sampling). + * + * Create a sample of this RDD using variable sampling rates for different keys as specified by + * `fractions`, a key to sampling rate map. + * + * If `exact` is set to false, create the sample via simple random sampling, with one pass + * over the RDD, to produce a sample of size that's approximately equal to the sum of + * math.ceil(numItems * samplingRate) over all key values; otherwise, use additional passes over + * the RDD to create a sample size that's exactly equal to the sum of + * math.ceil(numItems * samplingRate) over all key values. + * + * Use Utils.random.nextLong as the default seed for the random number generator */ def sampleByKey(withReplacement: Boolean, fractions: JMap[K, Double], @@ -149,17 +168,33 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) /** * Return a subset of this RDD sampled by key (via stratified sampling). + * + * Create a sample of this RDD using variable sampling rates for different keys as specified by + * `fractions`, a key to sampling rate map. + * + * Produce a sample of size that's approximately equal to the sum of + * math.ceil(numItems * samplingRate) over all key values with one pass over the RDD via + * simple random sampling. */ def sampleByKey(withReplacement: Boolean, fractions: JMap[K, Double], seed: Long): JavaPairRDD[K, V] = - sampleByKey(withReplacement, fractions, true, seed) + sampleByKey(withReplacement, fractions, false, seed) /** * Return a subset of this RDD sampled by key (via stratified sampling). + * + * Create a sample of this RDD using variable sampling rates for different keys as specified by + * `fractions`, a key to sampling rate map. + * + * Produce a sample of size that's approximately equal to the sum of + * math.ceil(numItems * samplingRate) over all key values with one pass over the RDD via + * simple random sampling. + * + * Use Utils.random.nextLong as the default seed for the random number generator */ def sampleByKey(withReplacement: Boolean, fractions: JMap[K, Double]): JavaPairRDD[K, V] = - sampleByKey(withReplacement, fractions, true, Utils.random.nextLong) + sampleByKey(withReplacement, fractions, false, Utils.random.nextLong) /** * Return the union of this RDD and another one. Any identical elements will appear multiple diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 1abbd676de3d..f5153d31e7b7 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -43,7 +43,7 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.serializer.Serializer import org.apache.spark.util.Utils -import org.apache.spark.util.random.StratifiedSampler +import org.apache.spark.util.random.StratifiedSamplingUtils /** * Extra functions available on RDDs of (key, value) pairs through an implicit conversion. @@ -195,32 +195,36 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) /** * Return a subset of this RDD sampled by key (via stratified sampling). * - * If exact set to true, we guarantee, with high probability, a sample size = - * math.ceil(fraction * S_i), where S_i is the size of the ith stratum (collection of entries - * that share the same key). When sampling without replacement, we need one additional pass over - * the RDD to guarantee sample size with a 99.99% confidence; when sampling with replacement, we - * need two additional passes. + * Create a sample of this RDD using variable sampling rates for different keys as specified by + * `fractions`, a key to sampling rate map. + * + * If `exact` is set to false, create the sample via simple random sampling, with one pass + * over the RDD, to produce a sample of size that's approximately equal to the sum of + * math.ceil(numItems * samplingRate) over all key values; otherwise, use + * additional passes over the RDD to create a sample size that's exactly equal to the sum of + * math.ceil(numItems * samplingRate) over all key values with a 99.99% confidence. When sampling + * without replacement, we need one additional pass over the RDD to guarantee sample size; + * when sampling with replacement, we need two additional passes. * * @param withReplacement whether to sample with or without replacement * @param fractions map of specific keys to sampling rates * @param seed seed for the random number generator - * @param exact whether sample size needs to be exactly math.ceil(fraction * size) per stratum + * @param exact whether sample size needs to be exactly math.ceil(fraction * size) per key * @return RDD containing the sampled subset */ def sampleByKey(withReplacement: Boolean, fractions: Map[K, Double], - exact: Boolean = true, + exact: Boolean = false, seed: Long = Utils.random.nextLong): RDD[(K, V)]= { - require(fractions.forall {case(k, v) => v >= 0.0}, "Invalid sampling rates.") + require(fractions.values.forall(v => v >= 0.0), "Negative sampling rates.") val samplingFunc = if (withReplacement) { - val counts = if (exact) Some(this.countByKey()) else None - StratifiedSampler.getPoissonSamplingFunction(self, fractions, exact, counts, seed) + StratifiedSamplingUtils.getPoissonSamplingFunction(self, fractions, exact, seed) } else { - StratifiedSampler.getBernoulliSamplingFunction(self, fractions, exact, seed) + StratifiedSamplingUtils.getBernoulliSamplingFunction(self, fractions, exact, seed) } - self.mapPartitionsWithIndex(samplingFunc, preservesPartitioning=true) + self.mapPartitionsWithIndex(samplingFunc, preservesPartitioning = true) } /** diff --git a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala index f697d88509f3..95e521f573f2 100644 --- a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala @@ -52,10 +52,7 @@ private[spark] object SamplingUtils { val numStDev = if (sampleSizeLowerBound < 12) 9 else 5 math.max(1e-10, fraction + numStDev * math.sqrt(fraction / total)) } else { - val delta = 1e-4 - val gamma = - math.log(delta) / total - math.min(1, - math.max(1e-10, fraction + gamma + math.sqrt(gamma * gamma + 2 * gamma * fraction))) + BernoulliBounds.getLowerBound(1e-4, total, fraction) } } } @@ -125,3 +122,21 @@ private[spark] object PoissonBounds { ub } } + + +private[spark] object BernoulliBounds { + + val minSamplingRate = 1e-10 + + def getUpperBound(delta: Double, n: Long, fraction: Double): Double = { + val gamma = - math.log(delta) / n * (2.0 / 3.0) + math.max(minSamplingRate, + fraction + gamma - math.sqrt(gamma * gamma + 3 * gamma * fraction)) + } + + def getLowerBound(delta: Double, n: Long, fraction: Double): Double = { + val gamma = - math.log(delta) / n + math.min(1, + math.max(minSamplingRate, fraction + gamma + math.sqrt(gamma * gamma + 2 * gamma * fraction))) + } +} diff --git a/core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala b/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala similarity index 70% rename from core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala rename to core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala index 28ea39352cce..a9ac992f6925 100644 --- a/core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala +++ b/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala @@ -18,12 +18,17 @@ package org.apache.spark.util.random import scala.collection.Map -import scala.collection.mutable.{ArrayBuffer, HashMap, Map => MMap} +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer import org.apache.commons.math3.random.RandomDataGenerator + import org.apache.spark.Logging +import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD +import scala.reflect.ClassTag + /** * Auxiliary functions and data structures for the sampleByKey method in PairRDDFunctions. * @@ -35,8 +40,8 @@ import org.apache.spark.rdd.RDD * Like in simple random sampling, we generate a random value for each item from the * uniform distribution [0.0, 1.0]. All items with values <= min(values of items in the waitlist) * are accepted into the sample instantly. The threshold for instant accept is designed so that - * s - numAccepted = O(log(s)), where s is again the desired sample size. Thus, by maintaining a - * waitlist size = O(log(s)), we will be able to create a sample of the exact size s by adding + * s - numAccepted = O(sqrt(s)), where s is again the desired sample size. Thus, by maintaining a + * waitlist size = O(sqrt(s)), we will be able to create a sample of the exact size s by adding * a portion of the waitlist to the set of items that are instantly accepted. The exact threshold * is computed by sorting the values in the waitlist and picking the value at (s - numAccepted). * @@ -47,21 +52,21 @@ import org.apache.spark.rdd.RDD * http://jmlr.org/proceedings/papers/v28/meng13a.html */ -private[spark] object StratifiedSampler extends Logging { +private[spark] object StratifiedSamplingUtils extends Logging { /** * Count the number of items instantly accepted and generate the waitlist for each stratum. * * This is only invoked when exact sample size is required. */ - def getCounts[K, V](rdd: RDD[(K, V)], + def getAcceptanceResults[K, V](rdd: RDD[(K, V)], withReplacement: Boolean, fractions: Map[K, Double], counts: Option[Map[K, Long]], - seed: Long): MMap[K, Stratum] = { + seed: Long): mutable.Map[K, AcceptanceResult] = { val combOp = getCombOp[K] val mappedPartitionRDD = rdd.mapPartitionsWithIndex({ case (partition, iter) => - val zeroU: MMap[K, Stratum] = new HashMap[K, Stratum]() + val zeroU: mutable.Map[K, AcceptanceResult] = new mutable.HashMap[K, AcceptanceResult]() val rng = new RandomDataGenerator() rng.reSeed(seed + partition) val seqOp = getSeqOp(withReplacement, fractions, rng, counts) @@ -76,21 +81,22 @@ private[spark] object StratifiedSampler extends Logging { def getSeqOp[K, V](withReplacement: Boolean, fractions: Map[K, Double], rng: RandomDataGenerator, - counts: Option[Map[K, Long]]): (MMap[K, Stratum], (K, V)) => MMap[K, Stratum] = { + counts: Option[Map[K, Long]]): + (mutable.Map[K, AcceptanceResult], (K, V)) => mutable.Map[K, AcceptanceResult] = { val delta = 5e-5 - (result: MMap[K, Stratum], item: (K, V)) => { + (result: mutable.Map[K, AcceptanceResult], item: (K, V)) => { val key = item._1 val fraction = fractions(key) if (!result.contains(key)) { - result += (key -> new Stratum()) + result += (key -> new AcceptanceResult()) } - val stratum = result(key) + val acceptResult = result(key) if (withReplacement) { // compute acceptBound and waitListBound only if they haven't been computed already // since they don't change from iteration to iteration. // TODO change this to the streaming version - if (stratum.areBoundsEmpty) { + if (acceptResult.areBoundsEmpty) { val n = counts.get(key) val sampleSize = math.ceil(n * fraction).toLong val lmbd1 = PoissonBounds.getLowerBound(sampleSize) @@ -100,37 +106,35 @@ private[spark] object StratifiedSampler extends Logging { } else { PoissonBounds.getUpperBound(sampleSize - minCount) } - stratum.acceptBound = lmbd1 / n - stratum.waitListBound = lmbd2 / n + acceptResult.acceptBound = lmbd1 / n + acceptResult.waitListBound = lmbd2 / n } - val acceptBound = stratum.acceptBound + val acceptBound = acceptResult.acceptBound val copiesAccepted = if (acceptBound == 0.0) 0L else rng.nextPoisson(acceptBound) if (copiesAccepted > 0) { - stratum.incrNumAccepted(copiesAccepted) + acceptResult.numAccepted += copiesAccepted } - val copiesWaitlisted = rng.nextPoisson(stratum.waitListBound).toInt + val copiesWaitlisted = rng.nextPoisson(acceptResult.waitListBound).toInt if (copiesWaitlisted > 0) { - stratum.addToWaitList(ArrayBuffer.fill(copiesWaitlisted)(rng.nextUniform(0.0, 1.0))) + acceptResult.waitList ++= ArrayBuffer.fill(copiesWaitlisted)(rng.nextUniform(0.0, 1.0)) } } else { // We use the streaming version of the algorithm for sampling without replacement to avoid // using an extra pass over the RDD for computing the count. // Hence, acceptBound and waitListBound change on every iteration. - val gamma1 = - math.log(delta) / stratum.numItems - val gamma2 = (2.0 / 3.0) * gamma1 - stratum.acceptBound = math.max(0, - fraction + gamma2 - math.sqrt(gamma2 * gamma2 + 3 * gamma2 * fraction)) - stratum.waitListBound = math.min(1, - fraction + gamma1 + math.sqrt(gamma1 * gamma1 + 2 * gamma1 * fraction)) + acceptResult.acceptBound = + BernoulliBounds.getUpperBound(delta, acceptResult.numItems, fraction) + acceptResult.waitListBound = + BernoulliBounds.getLowerBound(delta, acceptResult.numItems, fraction) val x = rng.nextUniform(0.0, 1.0) - if (x < stratum.acceptBound) { - stratum.incrNumAccepted() - } else if (x < stratum.waitListBound) { - stratum.addToWaitList(x) + if (x < acceptResult.acceptBound) { + acceptResult.numAccepted += 1 + } else if (x < acceptResult.waitListBound) { + acceptResult.waitList += x } } - stratum.incrNumItems() + acceptResult.numItems += 1 result } } @@ -138,10 +142,11 @@ private[spark] object StratifiedSampler extends Logging { /** * Returns the function used combine results returned by seqOp from different partitions. */ - def getCombOp[K]: (MMap[K, Stratum], MMap[K, Stratum]) => MMap[K, Stratum] = { - (result1: MMap[K, Stratum], result2: MMap[K, Stratum]) => { + def getCombOp[K]: (mutable.Map[K, AcceptanceResult], mutable.Map[K, AcceptanceResult]) + => mutable.Map[K, AcceptanceResult] = { + (result1: mutable.Map[K, AcceptanceResult], result2: mutable.Map[K, AcceptanceResult]) => { // take union of both key sets in case one partition doesn't contain all keys - for (key <- result1.keySet.union(result2.keySet)) { + result1.keySet.union(result2.keySet).foreach { key => // Use result2 to keep the combined result since r1 is usual empty val entry1 = result1.get(key) if (result2.contains(key)) { @@ -169,21 +174,21 @@ private[spark] object StratifiedSampler extends Logging { * in the waitlist range would allow all elements that were instantly accepted on the first pass * to be included in the sample. */ - def computeThresholdByKey[K](finalResult: Map[K, Stratum], + def computeThresholdByKey[K](finalResult: Map[K, AcceptanceResult], fractions: Map[K, Double]): Map[K, Double] = { - val thresholdByKey = new HashMap[K, Double]() - for ((key, stratum) <- finalResult) { - val sampleSize = math.ceil(stratum.numItems * fractions(key)).toLong - if (stratum.numAccepted > sampleSize) { + val thresholdByKey = new mutable.HashMap[K, Double]() + for ((key, acceptResult) <- finalResult) { + val sampleSize = math.ceil(acceptResult.numItems * fractions(key)).toLong + if (acceptResult.numAccepted > sampleSize) { logWarning("Pre-accepted too many") - thresholdByKey += (key -> stratum.acceptBound) + thresholdByKey += (key -> acceptResult.acceptBound) } else { - val numWaitListAccepted = (sampleSize - stratum.numAccepted).toInt - if (numWaitListAccepted >= stratum.waitList.size) { + val numWaitListAccepted = (sampleSize - acceptResult.numAccepted).toInt + if (numWaitListAccepted >= acceptResult.waitList.size) { logWarning("WaitList too short") - thresholdByKey += (key -> stratum.waitListBound) + thresholdByKey += (key -> acceptResult.waitListBound) } else { - thresholdByKey += (key -> stratum.waitList.sorted.apply(numWaitListAccepted)) + thresholdByKey += (key -> acceptResult.waitList.sorted.apply(numWaitListAccepted)) } } } @@ -205,12 +210,14 @@ private[spark] object StratifiedSampler extends Logging { var samplingRateByKey = fractions if (exact) { // determine threshold for each stratum and resample - val finalResult = getCounts(rdd, false, fractions, None, seed) + val finalResult = getAcceptanceResults(rdd, false, fractions, None, seed) samplingRateByKey = computeThresholdByKey(finalResult, fractions) } (idx: Int, iter: Iterator[(K, V)]) => { val rng = new RandomDataGenerator rng.reSeed(seed + idx) + // Must use the same invoke pattern on the rng as in getSeqOp for without replacement + // in order to generate the same sequence of random numbers when creating the sample iter.filter(t => rng.nextUniform(0.0, 1.0) < samplingRateByKey(t._1)) } } @@ -225,14 +232,14 @@ private[spark] object StratifiedSampler extends Logging { * * The sampling function has a unique seed per partition. */ - def getPoissonSamplingFunction[K, V](rdd: RDD[(K, V)], + def getPoissonSamplingFunction[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)], fractions: Map[K, Double], exact: Boolean, - counts: Option[Map[K, Long]], seed: Long): (Int, Iterator[(K, V)]) => Iterator[(K, V)] = { // TODO implement the streaming version of sampling w/ replacement that doesn't require counts if (exact) { - val finalResult = getCounts(rdd, true, fractions, counts, seed) + val counts = Some(rdd.countByKey()) + val finalResult = getAcceptanceResults(rdd, true, fractions, counts, seed) val thresholdByKey = computeThresholdByKey(finalResult, fractions) (idx: Int, iter: Iterator[(K, V)]) => { val rng = new RandomDataGenerator() @@ -240,6 +247,8 @@ private[spark] object StratifiedSampler extends Logging { iter.flatMap { item => val key = item._1 val acceptBound = finalResult(key).acceptBound + // Must use the same invoke pattern on the rng as in getSeqOp for with replacement + // in order to generate the same sequence of random numbers when creating the sample val copiesAccepted = if (acceptBound == 0) 0L else rng.nextPoisson(acceptBound) val copiesWailisted = rng.nextPoisson(finalResult(key).waitListBound).toInt val copiesInSample = copiesAccepted + @@ -274,38 +283,20 @@ private[spark] object StratifiedSampler extends Logging { * * `[random]` here is necessary since it's in the return type signature of seqOp defined above */ -private[random] class Stratum(var numItems: Long = 0L, var numAccepted: Long = 0L) +private[random] class AcceptanceResult(var numItems: Long = 0L, var numAccepted: Long = 0L) extends Serializable { - private val _waitList = new ArrayBuffer[Double] - private var _acceptBound: Option[Double] = None // upper bound for accepting item instantly - private var _waitListBound: Option[Double] = None // upper bound for adding item to waitlist - - def incrNumItems(by: Long = 1L) = numItems += by - - def incrNumAccepted(by: Long = 1L) = numAccepted += by - - def addToWaitList(elem: Double) = _waitList += elem - - def addToWaitList(elems: ArrayBuffer[Double]) = _waitList ++= elems - - def waitList = _waitList - - def acceptBound = _acceptBound.getOrElse(0.0) - - def acceptBound_= (value: Double) = _acceptBound = Some(value) - - def waitListBound = _waitListBound.getOrElse(0.0) - - def waitListBound_= (value: Double) = _waitListBound = Some(value) + val waitList = new ArrayBuffer[Double] + var acceptBound: Double = Double.NaN // upper bound for accepting item instantly + var waitListBound: Double = Double.NaN // upper bound for adding item to waitlist - def areBoundsEmpty = _acceptBound.isEmpty || _waitListBound.isEmpty + def areBoundsEmpty = acceptBound.isNaN || waitListBound.isNaN - def merge(other: Option[Stratum]): Unit = { + def merge(other: Option[AcceptanceResult]): Unit = { if (other.isDefined) { - addToWaitList(other.get.waitList) - incrNumAccepted(other.get.numAccepted) - incrNumItems(other.get.numItems) + waitList ++= other.get.waitList + numAccepted += other.get.numAccepted + numItems += other.get.numItems } } } diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index a86d02360a6b..8d90ce3e92d1 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -107,16 +107,16 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { samplingRate: Double, seed: Long, n: Long) = { - val expectedSampleSize = stratifiedData.countByKey().mapValues(count => - math.ceil(count * samplingRate).toInt) + val expectedSampleSize = stratifiedData.countByKey() + .mapValues(count => math.ceil(count * samplingRate).toInt) val fractions = Map("1" -> samplingRate, "0" -> samplingRate) val sample = stratifiedData.sampleByKey(false, fractions, exact, seed) val sampleCounts = sample.countByKey() val takeSample = sample.collect() - assert(sampleCounts.forall {case(k,v) => - checkSize(exact, false, expectedSampleSize(k), v, samplingRate)}) + sampleCounts.foreach { case(k, v) => + assert(checkSize(exact, false, expectedSampleSize(k), v, samplingRate)) } assert(takeSample.size === takeSample.toSet.size) - assert(takeSample.forall(x => 1 <= x._2 && x._2 <= n), s"elements not in [1, $n]") + takeSample.foreach { x => assert(1 <= x._2 && x._2 <= n, s"elements not in [1, $n]") } } // With replacement validation @@ -131,9 +131,9 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { val sample = stratifiedData.sampleByKey(true, fractions, exact, seed) val sampleCounts = sample.countByKey() val takeSample = sample.collect() - assert(sampleCounts.forall {case(k,v) => - checkSize(exact, true, expectedSampleSize(k), v, samplingRate)}) - val groupedByKey = takeSample.groupBy {case(k, v) => k} + sampleCounts.foreach { case(k, v) => + assert(checkSize(exact, true, expectedSampleSize(k), v, samplingRate)) } + val groupedByKey = takeSample.groupBy(_._1) for ((key, v) <- groupedByKey) { if (expectedSampleSize(key) >= 100 && samplingRate >= 0.1) { // sample large enough for there to be repeats with high likelihood @@ -146,7 +146,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { } } } - assert(takeSample.forall(x => 1 <= x._2 && x._2 <= n), s"elements not in [1, $n]") + takeSample.foreach { x => assert(1 <= x._2 && x._2 <= n, s"elements not in [1, $n]") } } def checkAllCombos(stratifiedData: RDD[(String, Int)], @@ -161,7 +161,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { val defaultSeed = 42L - //vary RDD size + // vary RDD size for (n <- List(100, 1000, 1000000)) { val data = sc.parallelize(1 to n, 2) val fractionPositive = 0.3 @@ -172,7 +172,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { checkAllCombos(stratifiedData, samplingRate, seed, n) } - //vary fractionPositive + // vary fractionPositive for (fractionPositive <- List(0.1, 0.3, 0.5, 0.7, 0.9)) { val n = 100 val data = sc.parallelize(1 to n, 2) @@ -183,19 +183,19 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { checkAllCombos(stratifiedData, samplingRate, seed, n) } - //Use the same data for the rest of the tests + // Use the same data for the rest of the tests val fractionPositive = 0.3 val n = 100 val data = sc.parallelize(1 to n, 2) val stratifiedData = data.keyBy(stratifier(fractionPositive)) - //vary seed + // vary seed for (seed <- defaultSeed to defaultSeed + 5L) { val samplingRate = 0.1 checkAllCombos(stratifiedData, samplingRate, seed, n) } - //vary sampling rate + // vary sampling rate for (samplingRate <- List(0.01, 0.05, 0.1, 0.5)) { val seed = defaultSeed checkAllCombos(stratifiedData, samplingRate, seed, n) From b3013a44078f5aa382d986df41af0e4067147077 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 24 Jul 2014 22:21:51 -0700 Subject: [PATCH 21/28] move math3 back to test scope --- core/pom.xml | 2 ++ pom.xml | 1 + 2 files changed, 3 insertions(+) diff --git a/core/pom.xml b/core/pom.xml index 956414b35873..4ed920a750ff 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -79,6 +79,8 @@ org.apache.commons commons-math3 + 3.3 + test com.google.code.findbugs diff --git a/pom.xml b/pom.xml index fce7fd96d185..f163b0cd1d97 100644 --- a/pom.xml +++ b/pom.xml @@ -260,6 +260,7 @@ org.apache.commons commons-math3 3.3 + test com.google.code.findbugs From b2235297636495061461db14e7a9709afb378bc1 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 24 Jul 2014 23:09:34 -0700 Subject: [PATCH 22/28] use approx bounds for poisson fix poisson mean for waitlisting add unit tests for Java --- .../spark/util/random/SamplingUtils.scala | 81 +++++++------------ .../util/random/StratifiedSamplingUtils.scala | 51 +++++++----- .../java/org/apache/spark/JavaAPISuite.java | 37 +++++++++ 3 files changed, 98 insertions(+), 71 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala index 95e521f573f2..f46ef2d962e9 100644 --- a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala @@ -17,8 +17,6 @@ package org.apache.spark.util.random -import org.apache.commons.math3.distribution.PoissonDistribution - private[spark] object SamplingUtils { /** @@ -49,8 +47,7 @@ private[spark] object SamplingUtils { withReplacement: Boolean): Double = { val fraction = sampleSizeLowerBound.toDouble / total if (withReplacement) { - val numStDev = if (sampleSizeLowerBound < 12) 9 else 5 - math.max(1e-10, fraction + numStDev * math.sqrt(fraction / total)) + PoissonBounds.getUpperBound(sampleSizeLowerBound) } else { BernoulliBounds.getLowerBound(1e-4, total, fraction) } @@ -60,83 +57,61 @@ private[spark] object SamplingUtils { /** * Utility functions that help us determine bounds on adjusted sampling rate to guarantee exact * sample sizes with high confidence when sampling with replacement. - * - * The algorithm for guaranteeing sample size instantly accepts items whose associated value drawn - * from Pois(s) is less than the lower bound and puts items whose value is between the lower and - * upper bound in a waitlist. The final sample is consisted of all items accepted on the fly and a - * portion of the waitlist needed to make the exact sample size. */ private[spark] object PoissonBounds { - val delta = 1e-4 / 3.0 - val epsilon = 1e-15 - /** - * Compute the threshold for accepting items on the fly. The threshold value is a fairly small - * number, which means if the item has an associated value < threshold, it is highly likely to - * be in the final sample. Hence we accept items with values less than the returned value of this - * function instantly. - * - * @param s sample size - * @return threshold for accepting items on the fly + * Returns a lambda such that Pr[X > s] is very small, where X ~ Pois(lambda). */ def getLowerBound(s: Double): Double = { - var lb = math.max(0.0, s - math.sqrt(s / delta)) // Chebyshev's inequality - var ub = s - while (lb < ub - 1.0) { - val m = (lb + ub) / 2.0 - val poisson = new PoissonDistribution(m, epsilon) - val y = poisson.inverseCumulativeProbability(1 - delta) - if (y > s) ub = m else lb = m - } - lb - } - - def getMinCount(lmbd: Double): Double = { - if (lmbd == 0) { - 0 - } else { - val poisson = new PoissonDistribution(lmbd, epsilon) - poisson.inverseCumulativeProbability(delta) - } + math.max(s - numStd(s) * math.sqrt(s), 1e-15) } /** - * Compute the threshold for waitlisting items. An item is waitlisted if its associated value is - * greater than the lower bound determined above but below the upper bound computed here. - * The value is computed such that we only need to keep log(s) items in the waitlist and still be - * able to guarantee sample size with high confidence. + * Returns a lambda such that Pr[X < s] is very small, where X ~ Pois(lambda). * * @param s sample size - * @return threshold for waitlisting the item */ def getUpperBound(s: Double): Double = { - var lb = s - var ub = s + math.sqrt(s / delta) // Chebyshev's inequality - while (lb < ub - 1.0) { - val m = (lb + ub) / 2.0 - val poisson = new PoissonDistribution(m, epsilon) - val y = poisson.inverseCumulativeProbability(delta) - if (y >= s) ub = m else lb = m + math.max(s + numStd(s) * math.sqrt(s), 1e-10) + } + + private def numStd(s: Double): Double = { + // TODO: Make it tighter. + if (s < 6.0) { + 12.0 + } else if (s < 16.0) { + 9.0 + } else { + 6.0 } - ub } } - +/** + * Utility functions that help us determine bounds on adjusted sampling rate to guarantee exact + * sample size with high confidence when sampling without replacement. + */ private[spark] object BernoulliBounds { val minSamplingRate = 1e-10 + /** + * Returns a threshold such that if we apply Bernoulli sampling with that threshold, it is very + * unlikely to sample less than `fraction * n` items out of `n` items. + */ def getUpperBound(delta: Double, n: Long, fraction: Double): Double = { val gamma = - math.log(delta) / n * (2.0 / 3.0) math.max(minSamplingRate, fraction + gamma - math.sqrt(gamma * gamma + 3 * gamma * fraction)) } + /** + * Returns a threshold such that if we apply Bernoulli sampling with that threshold, it is very + * unlikely to sample more than `fraction * n` items out of `n` items. + */ def getLowerBound(delta: Double, n: Long, fraction: Double): Double = { val gamma = - math.log(delta) / n - math.min(1, - math.max(minSamplingRate, fraction + gamma + math.sqrt(gamma * gamma + 2 * gamma * fraction))) + math.min(1, fraction + gamma + math.sqrt(gamma * gamma + 2 * gamma * fraction)) } } diff --git a/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala b/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala index a9ac992f6925..7cb71daf693d 100644 --- a/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala @@ -17,12 +17,13 @@ package org.apache.spark.util.random +import cern.jet.random.Poisson +import cern.jet.random.engine.DRand + import scala.collection.Map import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import org.apache.commons.math3.random.RandomDataGenerator - import org.apache.spark.Logging import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD @@ -65,13 +66,13 @@ private[spark] object StratifiedSamplingUtils extends Logging { counts: Option[Map[K, Long]], seed: Long): mutable.Map[K, AcceptanceResult] = { val combOp = getCombOp[K] - val mappedPartitionRDD = rdd.mapPartitionsWithIndex({ case (partition, iter) => + val mappedPartitionRDD = rdd.mapPartitionsWithIndex { case (partition, iter) => val zeroU: mutable.Map[K, AcceptanceResult] = new mutable.HashMap[K, AcceptanceResult]() val rng = new RandomDataGenerator() rng.reSeed(seed + partition) val seqOp = getSeqOp(withReplacement, fractions, rng, counts) Iterator(iter.aggregate(zeroU)(seqOp, combOp)) - }, preservesPartitioning=true) + } mappedPartitionRDD.reduce(combOp) } @@ -100,23 +101,18 @@ private[spark] object StratifiedSamplingUtils extends Logging { val n = counts.get(key) val sampleSize = math.ceil(n * fraction).toLong val lmbd1 = PoissonBounds.getLowerBound(sampleSize) - val minCount = PoissonBounds.getMinCount(lmbd1) - val lmbd2 = if (lmbd1 == 0) { - PoissonBounds.getUpperBound(sampleSize) - } else { - PoissonBounds.getUpperBound(sampleSize - minCount) - } + val lmbd2 = PoissonBounds.getUpperBound(sampleSize) acceptResult.acceptBound = lmbd1 / n - acceptResult.waitListBound = lmbd2 / n + acceptResult.waitListBound = (lmbd2 - lmbd1) / n } val acceptBound = acceptResult.acceptBound val copiesAccepted = if (acceptBound == 0.0) 0L else rng.nextPoisson(acceptBound) if (copiesAccepted > 0) { acceptResult.numAccepted += copiesAccepted } - val copiesWaitlisted = rng.nextPoisson(acceptResult.waitListBound).toInt + val copiesWaitlisted = rng.nextPoisson(acceptResult.waitListBound) if (copiesWaitlisted > 0) { - acceptResult.waitList ++= ArrayBuffer.fill(copiesWaitlisted)(rng.nextUniform(0.0, 1.0)) + acceptResult.waitList ++= ArrayBuffer.fill(copiesWaitlisted)(rng.nextUniform()) } } else { // We use the streaming version of the algorithm for sampling without replacement to avoid @@ -127,7 +123,7 @@ private[spark] object StratifiedSamplingUtils extends Logging { acceptResult.waitListBound = BernoulliBounds.getLowerBound(delta, acceptResult.numItems, fraction) - val x = rng.nextUniform(0.0, 1.0) + val x = rng.nextUniform() if (x < acceptResult.acceptBound) { acceptResult.numAccepted += 1 } else if (x < acceptResult.waitListBound) { @@ -218,7 +214,7 @@ private[spark] object StratifiedSamplingUtils extends Logging { rng.reSeed(seed + idx) // Must use the same invoke pattern on the rng as in getSeqOp for without replacement // in order to generate the same sequence of random numbers when creating the sample - iter.filter(t => rng.nextUniform(0.0, 1.0) < samplingRateByKey(t._1)) + iter.filter(t => rng.nextUniform() < samplingRateByKey(t._1)) } } @@ -250,9 +246,9 @@ private[spark] object StratifiedSamplingUtils extends Logging { // Must use the same invoke pattern on the rng as in getSeqOp for with replacement // in order to generate the same sequence of random numbers when creating the sample val copiesAccepted = if (acceptBound == 0) 0L else rng.nextPoisson(acceptBound) - val copiesWailisted = rng.nextPoisson(finalResult(key).waitListBound).toInt + val copiesWailisted = rng.nextPoisson(finalResult(key).waitListBound) val copiesInSample = copiesAccepted + - (0 until copiesWailisted).count(i => rng.nextUniform(0.0, 1.0) < thresholdByKey(key)) + (0 until copiesWailisted).count(i => rng.nextUniform() < thresholdByKey(key)) if (copiesInSample > 0) { Iterator.fill(copiesInSample.toInt)(item) } else { @@ -265,7 +261,7 @@ private[spark] object StratifiedSamplingUtils extends Logging { val rng = new RandomDataGenerator() rng.reSeed(seed + idx) iter.flatMap { item => - val count = rng.nextPoisson(fractions(item._1)).toInt + val count = rng.nextPoisson(fractions(item._1)) if (count > 0) { Iterator.fill(count)(item) } else { @@ -275,6 +271,25 @@ private[spark] object StratifiedSamplingUtils extends Logging { } } } + + /** A random data generator that generates both uniform values and Poisson values. */ + private class RandomDataGenerator { + val uniform = new XORShiftRandom() + var poisson = new Poisson(1.0, new DRand) + + def reSeed(seed: Long) { + uniform.setSeed(seed) + poisson = new Poisson(1.0, new DRand(seed.toInt)) + } + + def nextPoisson(mean: Double): Int = { + poisson.nextInt(mean) + } + + def nextUniform(): Double = { + uniform.nextDouble() + } + } } /** diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index b2868b59ce6c..afd575aa6c63 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -29,6 +29,7 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.base.Optional; import com.google.common.base.Charsets; import com.google.common.io.Files; @@ -1203,4 +1204,40 @@ public Tuple2 call(Integer x) { pairRDD.collect(); // Works fine pairRDD.collectAsMap(); // Used to crash with ClassCastException } + + @Test + @SuppressWarnings("unchecked") + public void sampleByKey() { + JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8), 3); + JavaPairRDD rdd2 = rdd1.mapToPair( + new PairFunction() { + @Override + public Tuple2 call(Integer i) { + return new Tuple2(i % 2, 1); + } + }); + Map fractions = Maps.newHashMap(); + fractions.put(0, 0.5); + fractions.put(1, 1.0); + JavaPairRDD wr = rdd2.sampleByKey(true, fractions, 1L); + Map wrCounts = (Map) (Object) wr.countByKey(); + Assert.assertTrue(wrCounts.size() == 2); + Assert.assertTrue(wrCounts.get(0) > 0); + Assert.assertTrue(wrCounts.get(1) > 0); + JavaPairRDD wor = rdd2.sampleByKey(false, fractions, 1L); + Map worCounts = (Map) (Object) wor.countByKey(); + Assert.assertTrue(worCounts.size() == 2); + Assert.assertTrue(worCounts.get(0) > 0); + Assert.assertTrue(worCounts.get(1) > 0); + JavaPairRDD wrExact = rdd2.sampleByKey(true, fractions, true, 1L); + Map wrExactCounts = (Map) (Object) wrExact.countByKey(); + Assert.assertTrue(wrExactCounts.size() == 2); + Assert.assertTrue(wrExactCounts.get(0) == 2); + Assert.assertTrue(wrExactCounts.get(1) == 4); + JavaPairRDD worExact = rdd2.sampleByKey(false, fractions, true, 1L); + Map worExactCounts = (Map) (Object) worExact.countByKey(); + Assert.assertTrue(worExactCounts.size() == 2); + Assert.assertTrue(worExactCounts.get(0) == 2); + Assert.assertTrue(worExactCounts.get(1) == 4); + } } From 17a381b3d7be601e4319beca86169c9046b36f18 Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Mon, 28 Jul 2014 12:03:02 -0700 Subject: [PATCH 23/28] fixed a merge issue and a failed unit --- core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 1 + .../test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 026a569e7ec5..1af4e5f0b6d0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -43,6 +43,7 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.serializer.Serializer import org.apache.spark.util.Utils +import org.apache.spark.util.collection.CompactBuffer import org.apache.spark.util.random.StratifiedSamplingUtils /** diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 8d90ce3e92d1..ec2822c49f60 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -159,7 +159,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { takeSampleAndValidatePoisson(stratifiedData, false, samplingRate, seed, n) } - val defaultSeed = 42L + val defaultSeed = 1L // vary RDD size for (n <- List(100, 1000, 1000000)) { From eaf5771ca2712880577ccd48a696ff13f33bb90a Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Mon, 28 Jul 2014 15:11:02 -0700 Subject: [PATCH 24/28] bug fixes. --- .../spark/util/random/SamplingUtils.scala | 20 +++++++++---------- .../util/random/StratifiedSamplingUtils.scala | 13 ++++++------ .../spark/rdd/PairRDDFunctionsSuite.scala | 9 +++------ 3 files changed, 19 insertions(+), 23 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala index 467fb84dd42e..586e5d927bac 100644 --- a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala @@ -91,11 +91,11 @@ private[spark] object SamplingUtils { */ def computeFractionForSampleSize(sampleSizeLowerBound: Int, total: Long, withReplacement: Boolean): Double = { - val fraction = sampleSizeLowerBound.toDouble / total if (withReplacement) { - PoissonBounds.getUpperBound(sampleSizeLowerBound) + PoissonBounds.getUpperBound(sampleSizeLowerBound) / total } else { - BernoulliBounds.getLowerBound(1e-4, total, fraction) + val fraction = sampleSizeLowerBound.toDouble / total + BinomialBounds.getUpperBound(1e-4, total, fraction) } } } @@ -138,25 +138,25 @@ private[spark] object PoissonBounds { * Utility functions that help us determine bounds on adjusted sampling rate to guarantee exact * sample size with high confidence when sampling without replacement. */ -private[spark] object BernoulliBounds { +private[spark] object BinomialBounds { val minSamplingRate = 1e-10 /** - * Returns a threshold such that if we apply Bernoulli sampling with that threshold, it is very - * unlikely to sample less than `fraction * n` items out of `n` items. + * Returns a threshold `p` such that if we conduct n Bernoulli trials with success rate = `p`, + * it is very unlikely to have more than `fraction * n` successes. */ - def getUpperBound(delta: Double, n: Long, fraction: Double): Double = { + def getLowerBound(delta: Double, n: Long, fraction: Double): Double = { val gamma = - math.log(delta) / n * (2.0 / 3.0) math.max(minSamplingRate, fraction + gamma - math.sqrt(gamma * gamma + 3 * gamma * fraction)) } /** - * Returns a threshold such that if we apply Bernoulli sampling with that threshold, it is very - * unlikely to sample more than `fraction * n` items out of `n` items. + * Returns a threshold `p` such that if we conduct n Bernoulli trials with success rate = `p`, + * it is very unlikely to have less than `fraction * n` successes. */ - def getLowerBound(delta: Double, n: Long, fraction: Double): Double = { + def getUpperBound(delta: Double, n: Long, fraction: Double): Double = { val gamma = - math.log(delta) / n math.min(1, fraction + gamma + math.sqrt(gamma * gamma + 2 * gamma * fraction)) } diff --git a/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala b/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala index 7cb71daf693d..8f95d7c6b799 100644 --- a/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala @@ -17,19 +17,18 @@ package org.apache.spark.util.random -import cern.jet.random.Poisson -import cern.jet.random.engine.DRand - import scala.collection.Map import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import scala.reflect.ClassTag + +import cern.jet.random.Poisson +import cern.jet.random.engine.DRand import org.apache.spark.Logging import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD -import scala.reflect.ClassTag - /** * Auxiliary functions and data structures for the sampleByKey method in PairRDDFunctions. * @@ -119,9 +118,9 @@ private[spark] object StratifiedSamplingUtils extends Logging { // using an extra pass over the RDD for computing the count. // Hence, acceptBound and waitListBound change on every iteration. acceptResult.acceptBound = - BernoulliBounds.getUpperBound(delta, acceptResult.numItems, fraction) + BinomialBounds.getLowerBound(delta, acceptResult.numItems, fraction) acceptResult.waitListBound = - BernoulliBounds.getLowerBound(delta, acceptResult.numItems, fraction) + BinomialBounds.getUpperBound(delta, acceptResult.numItems, fraction) val x = rng.nextUniform() if (x < acceptResult.acceptBound) { diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index ec2822c49f60..4f49d4a1d4d3 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -168,8 +168,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { val stratifiedData = data.keyBy(stratifier(fractionPositive)) val samplingRate = 0.1 - val seed = defaultSeed - checkAllCombos(stratifiedData, samplingRate, seed, n) + checkAllCombos(stratifiedData, samplingRate, defaultSeed, n) } // vary fractionPositive @@ -179,8 +178,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { val stratifiedData = data.keyBy(stratifier(fractionPositive)) val samplingRate = 0.1 - val seed = defaultSeed - checkAllCombos(stratifiedData, samplingRate, seed, n) + checkAllCombos(stratifiedData, samplingRate, defaultSeed, n) } // Use the same data for the rest of the tests @@ -197,8 +195,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { // vary sampling rate for (samplingRate <- List(0.01, 0.05, 0.1, 0.5)) { - val seed = defaultSeed - checkAllCombos(stratifiedData, samplingRate, seed, n) + checkAllCombos(stratifiedData, samplingRate, defaultSeed, n) } } From 245439ef18393d814821f4c1d445aee8cc774f09 Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Tue, 29 Jul 2014 11:50:32 -0700 Subject: [PATCH 25/28] moved minSamplingRate to getUpperBound --- .../scala/org/apache/spark/util/random/SamplingUtils.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala index 586e5d927bac..c9a864ae6277 100644 --- a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala @@ -148,8 +148,7 @@ private[spark] object BinomialBounds { */ def getLowerBound(delta: Double, n: Long, fraction: Double): Double = { val gamma = - math.log(delta) / n * (2.0 / 3.0) - math.max(minSamplingRate, - fraction + gamma - math.sqrt(gamma * gamma + 3 * gamma * fraction)) + fraction + gamma - math.sqrt(gamma * gamma + 3 * gamma * fraction) } /** @@ -158,6 +157,7 @@ private[spark] object BinomialBounds { */ def getUpperBound(delta: Double, n: Long, fraction: Double): Double = { val gamma = - math.log(delta) / n - math.min(1, fraction + gamma + math.sqrt(gamma * gamma + 2 * gamma * fraction)) + math.min(1, + math.max(minSamplingRate, fraction + gamma + math.sqrt(gamma * gamma + 2 * gamma * fraction))) } } From 555a3f9e5b8e4191b6dc6e7b71b0357a9cc4bafc Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Fri, 8 Aug 2014 23:24:52 -0700 Subject: [PATCH 26/28] separate out sampleByKeyExact as its own API --- .../apache/spark/api/java/JavaPairRDD.scala | 66 +++--- .../apache/spark/rdd/PairRDDFunctions.scala | 52 +++-- .../java/org/apache/spark/JavaAPISuite.java | 20 +- .../spark/rdd/PairRDDFunctionsSuite.scala | 204 +++++++++++------- 4 files changed, 216 insertions(+), 126 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 76d4193e96ae..ad954f5944c1 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -133,68 +133,64 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * Return a subset of this RDD sampled by key (via stratified sampling). * * Create a sample of this RDD using variable sampling rates for different keys as specified by - * `fractions`, a key to sampling rate map. - * - * If `exact` is set to false, create the sample via simple random sampling, with one pass - * over the RDD, to produce a sample of size that's approximately equal to the sum of - * math.ceil(numItems * samplingRate) over all key values; otherwise, use additional passes over - * the RDD to create a sample size that's exactly equal to the sum of + * `fractions`, a key to sampling rate map, via simple random sampling with one pass over the + * RDD, to produce a sample of size that's approximately equal to the sum of * math.ceil(numItems * samplingRate) over all key values. */ def sampleByKey(withReplacement: Boolean, fractions: JMap[K, Double], - exact: Boolean, seed: Long): JavaPairRDD[K, V] = - new JavaPairRDD[K, V](rdd.sampleByKey(withReplacement, fractions, exact, seed)) + new JavaPairRDD[K, V](rdd.sampleByKey(withReplacement, fractions, seed)) /** * Return a subset of this RDD sampled by key (via stratified sampling). * * Create a sample of this RDD using variable sampling rates for different keys as specified by - * `fractions`, a key to sampling rate map. - * - * If `exact` is set to false, create the sample via simple random sampling, with one pass - * over the RDD, to produce a sample of size that's approximately equal to the sum of - * math.ceil(numItems * samplingRate) over all key values; otherwise, use additional passes over - * the RDD to create a sample size that's exactly equal to the sum of + * `fractions`, a key to sampling rate map, via simple random sampling with one pass over the + * RDD, to produce a sample of size that's approximately equal to the sum of * math.ceil(numItems * samplingRate) over all key values. * - * Use Utils.random.nextLong as the default seed for the random number generator + * Use Utils.random.nextLong as the default seed for the random number generator. */ def sampleByKey(withReplacement: Boolean, - fractions: JMap[K, Double], - exact: Boolean): JavaPairRDD[K, V] = - sampleByKey(withReplacement, fractions, exact, Utils.random.nextLong) + fractions: JMap[K, Double]): JavaPairRDD[K, V] = + sampleByKey(withReplacement, fractions, Utils.random.nextLong) /** - * Return a subset of this RDD sampled by key (via stratified sampling). + * ::Experimental:: * - * Create a sample of this RDD using variable sampling rates for different keys as specified by - * `fractions`, a key to sampling rate map. + * Return a subset of this RDD sampled by key (via stratified sampling) containing exactly + * math.ceil(numItems * samplingRate) for each stratum (group of pairs with the same key). * - * Produce a sample of size that's approximately equal to the sum of - * math.ceil(numItems * samplingRate) over all key values with one pass over the RDD via - * simple random sampling. + * This method differs from [[sampleByKey]] in that we make additional passes over the RDD to + * create a sample size that's exactly equal to the sum of math.ceil(numItems * samplingRate) + * over all key values with a 99.99% confidence. When sampling without replacement, we need one + * additional pass over the RDD to guarantee sample size; when sampling with replacement, we need + * two additional passes. */ - def sampleByKey(withReplacement: Boolean, + @Experimental + def sampleByKeyExact(withReplacement: Boolean, fractions: JMap[K, Double], seed: Long): JavaPairRDD[K, V] = - sampleByKey(withReplacement, fractions, false, seed) + new JavaPairRDD[K, V](rdd.sampleByKeyExact(withReplacement, fractions, seed)) /** - * Return a subset of this RDD sampled by key (via stratified sampling). + * ::Experimental:: * - * Create a sample of this RDD using variable sampling rates for different keys as specified by - * `fractions`, a key to sampling rate map. + * Return a subset of this RDD sampled by key (via stratified sampling) containing exactly + * math.ceil(numItems * samplingRate) for each stratum (group of pairs with the same key). * - * Produce a sample of size that's approximately equal to the sum of - * math.ceil(numItems * samplingRate) over all key values with one pass over the RDD via - * simple random sampling. + * This method differs from [[sampleByKey]] in that we make additional passes over the RDD to + * create a sample size that's exactly equal to the sum of math.ceil(numItems * samplingRate) + * over all key values with a 99.99% confidence. When sampling without replacement, we need one + * additional pass over the RDD to guarantee sample size; when sampling with replacement, we need + * two additional passes. * - * Use Utils.random.nextLong as the default seed for the random number generator + * Use Utils.random.nextLong as the default seed for the random number generator. */ - def sampleByKey(withReplacement: Boolean, fractions: JMap[K, Double]): JavaPairRDD[K, V] = - sampleByKey(withReplacement, fractions, false, Utils.random.nextLong) + @Experimental + def sampleByKeyExact(withReplacement: Boolean, fractions: JMap[K, Double]): JavaPairRDD[K, V] = + sampleByKeyExact(withReplacement, fractions, Utils.random.nextLong) /** * Return the union of this RDD and another one. Any identical elements will appear multiple diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 93af50c0a9cd..4fc1224cbbfc 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -197,33 +197,57 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * Return a subset of this RDD sampled by key (via stratified sampling). * * Create a sample of this RDD using variable sampling rates for different keys as specified by - * `fractions`, a key to sampling rate map. - * - * If `exact` is set to false, create the sample via simple random sampling, with one pass - * over the RDD, to produce a sample of size that's approximately equal to the sum of - * math.ceil(numItems * samplingRate) over all key values; otherwise, use - * additional passes over the RDD to create a sample size that's exactly equal to the sum of - * math.ceil(numItems * samplingRate) over all key values with a 99.99% confidence. When sampling - * without replacement, we need one additional pass over the RDD to guarantee sample size; - * when sampling with replacement, we need two additional passes. + * `fractions`, a key to sampling rate map, via simple random sampling with one pass over the + * RDD, to produce a sample of size that's approximately equal to the sum of + * math.ceil(numItems * samplingRate) over all key values. * * @param withReplacement whether to sample with or without replacement * @param fractions map of specific keys to sampling rates * @param seed seed for the random number generator - * @param exact whether sample size needs to be exactly math.ceil(fraction * size) per key * @return RDD containing the sampled subset */ def sampleByKey(withReplacement: Boolean, fractions: Map[K, Double], - exact: Boolean = false, - seed: Long = Utils.random.nextLong): RDD[(K, V)]= { + seed: Long = Utils.random.nextLong): RDD[(K, V)] = { + + require(fractions.values.forall(v => v >= 0.0), "Negative sampling rates.") + + val samplingFunc = if (withReplacement) { + StratifiedSamplingUtils.getPoissonSamplingFunction(self, fractions, false, seed) + } else { + StratifiedSamplingUtils.getBernoulliSamplingFunction(self, fractions, false, seed) + } + self.mapPartitionsWithIndex(samplingFunc, preservesPartitioning = true) + } + + /** + * ::Experimental:: + * + * Return a subset of this RDD sampled by key (via stratified sampling) containing exactly + * math.ceil(numItems * samplingRate) for each stratum (group of pairs with the same key). + * + * This method differs from [[sampleByKey]] in that we make additional passes over the RDD to + * create a sample size that's exactly equal to the sum of math.ceil(numItems * samplingRate) + * over all key values with a 99.99% confidence. When sampling without replacement, we need one + * additional pass over the RDD to guarantee sample size; when sampling with replacement, we need + * two additional passes. + * + * @param withReplacement whether to sample with or without replacement + * @param fractions map of specific keys to sampling rates + * @param seed seed for the random number generator + * @return RDD containing the sampled subset + */ + @Experimental + def sampleByKeyExact(withReplacement: Boolean, + fractions: Map[K, Double], + seed: Long = Utils.random.nextLong): RDD[(K, V)] = { require(fractions.values.forall(v => v >= 0.0), "Negative sampling rates.") val samplingFunc = if (withReplacement) { - StratifiedSamplingUtils.getPoissonSamplingFunction(self, fractions, exact, seed) + StratifiedSamplingUtils.getPoissonSamplingFunction(self, fractions, true, seed) } else { - StratifiedSamplingUtils.getBernoulliSamplingFunction(self, fractions, exact, seed) + StratifiedSamplingUtils.getBernoulliSamplingFunction(self, fractions, true, seed) } self.mapPartitionsWithIndex(samplingFunc, preservesPartitioning = true) } diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 56150caa5d6b..a1449c090e36 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -1239,12 +1239,28 @@ public Tuple2 call(Integer i) { Assert.assertTrue(worCounts.size() == 2); Assert.assertTrue(worCounts.get(0) > 0); Assert.assertTrue(worCounts.get(1) > 0); - JavaPairRDD wrExact = rdd2.sampleByKey(true, fractions, true, 1L); + } + + @Test + @SuppressWarnings("unchecked") + public void sampleByKeyExact() { + JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8), 3); + JavaPairRDD rdd2 = rdd1.mapToPair( + new PairFunction() { + @Override + public Tuple2 call(Integer i) { + return new Tuple2(i % 2, 1); + } + }); + Map fractions = Maps.newHashMap(); + fractions.put(0, 0.5); + fractions.put(1, 1.0); + JavaPairRDD wrExact = rdd2.sampleByKeyExact(true, fractions, 1L); Map wrExactCounts = (Map) (Object) wrExact.countByKey(); Assert.assertTrue(wrExactCounts.size() == 2); Assert.assertTrue(wrExactCounts.get(0) == 2); Assert.assertTrue(wrExactCounts.get(1) == 4); - JavaPairRDD worExact = rdd2.sampleByKey(false, fractions, true, 1L); + JavaPairRDD worExact = rdd2.sampleByKeyExact(false, fractions, 1L); Map worExactCounts = (Map) (Object) worExact.countByKey(); Assert.assertTrue(worExactCounts.size() == 2); Assert.assertTrue(worExactCounts.get(0) == 2); diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 4f49d4a1d4d3..4b15162eb8ba 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -84,118 +84,81 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { } test("sampleByKey") { - def stratifier (fractionPositive: Double) = { - (x: Int) => if (x % 10 < (10 * fractionPositive).toInt) "1" else "0" - } - def checkSize(exact: Boolean, - withReplacement: Boolean, - expected: Long, - actual: Long, - p: Double): Boolean = { - if (exact) { - return expected == actual - } - val stdev = if (withReplacement) math.sqrt(expected) else math.sqrt(expected * p * (1 - p)) - // Very forgiving margin since we're dealing with very small sample sizes most of the time - math.abs(actual - expected) <= 6 * stdev + val defaultSeed = 1L + + // vary RDD size + for (n <- List(100, 1000, 1000000)) { + val data = sc.parallelize(1 to n, 2) + val fractionPositive = 0.3 + val stratifiedData = data.keyBy(StratifiedAuxiliary.stratifier(fractionPositive)) + val samplingRate = 0.1 + StratifiedAuxiliary.testSample(stratifiedData, samplingRate, defaultSeed, n) } - // Without replacement validation - def takeSampleAndValidateBernoulli(stratifiedData: RDD[(String, Int)], - exact: Boolean, - samplingRate: Double, - seed: Long, - n: Long) = { - val expectedSampleSize = stratifiedData.countByKey() - .mapValues(count => math.ceil(count * samplingRate).toInt) - val fractions = Map("1" -> samplingRate, "0" -> samplingRate) - val sample = stratifiedData.sampleByKey(false, fractions, exact, seed) - val sampleCounts = sample.countByKey() - val takeSample = sample.collect() - sampleCounts.foreach { case(k, v) => - assert(checkSize(exact, false, expectedSampleSize(k), v, samplingRate)) } - assert(takeSample.size === takeSample.toSet.size) - takeSample.foreach { x => assert(1 <= x._2 && x._2 <= n, s"elements not in [1, $n]") } + // vary fractionPositive + for (fractionPositive <- List(0.1, 0.3, 0.5, 0.7, 0.9)) { + val n = 100 + val data = sc.parallelize(1 to n, 2) + val stratifiedData = data.keyBy(StratifiedAuxiliary.stratifier(fractionPositive)) + val samplingRate = 0.1 + StratifiedAuxiliary.testSample(stratifiedData, samplingRate, defaultSeed, n) } - // With replacement validation - def takeSampleAndValidatePoisson(stratifiedData: RDD[(String, Int)], - exact: Boolean, - samplingRate: Double, - seed: Long, - n: Long) = { - val expectedSampleSize = stratifiedData.countByKey().mapValues(count => - math.ceil(count * samplingRate).toInt) - val fractions = Map("1" -> samplingRate, "0" -> samplingRate) - val sample = stratifiedData.sampleByKey(true, fractions, exact, seed) - val sampleCounts = sample.countByKey() - val takeSample = sample.collect() - sampleCounts.foreach { case(k, v) => - assert(checkSize(exact, true, expectedSampleSize(k), v, samplingRate)) } - val groupedByKey = takeSample.groupBy(_._1) - for ((key, v) <- groupedByKey) { - if (expectedSampleSize(key) >= 100 && samplingRate >= 0.1) { - // sample large enough for there to be repeats with high likelihood - assert(v.toSet.size < expectedSampleSize(key)) - } else { - if (exact) { - assert(v.toSet.size <= expectedSampleSize(key)) - } else { - assert(checkSize(false, true, expectedSampleSize(key), v.toSet.size, samplingRate)) - } - } - } - takeSample.foreach { x => assert(1 <= x._2 && x._2 <= n, s"elements not in [1, $n]") } + // Use the same data for the rest of the tests + val fractionPositive = 0.3 + val n = 100 + val data = sc.parallelize(1 to n, 2) + val stratifiedData = data.keyBy(StratifiedAuxiliary.stratifier(fractionPositive)) + + // vary seed + for (seed <- defaultSeed to defaultSeed + 5L) { + val samplingRate = 0.1 + StratifiedAuxiliary.testSample(stratifiedData, samplingRate, seed, n) } - def checkAllCombos(stratifiedData: RDD[(String, Int)], - samplingRate: Double, - seed: Long, - n: Long) = { - takeSampleAndValidateBernoulli(stratifiedData, true, samplingRate, seed, n) - takeSampleAndValidateBernoulli(stratifiedData, false, samplingRate, seed, n) - takeSampleAndValidatePoisson(stratifiedData, true, samplingRate, seed, n) - takeSampleAndValidatePoisson(stratifiedData, false, samplingRate, seed, n) + // vary sampling rate + for (samplingRate <- List(0.01, 0.05, 0.1, 0.5)) { + StratifiedAuxiliary.testSample(stratifiedData, samplingRate, defaultSeed, n) } + } + test("sampleByKeyExact") { val defaultSeed = 1L // vary RDD size for (n <- List(100, 1000, 1000000)) { val data = sc.parallelize(1 to n, 2) val fractionPositive = 0.3 - val stratifiedData = data.keyBy(stratifier(fractionPositive)) - + val stratifiedData = data.keyBy(StratifiedAuxiliary.stratifier(fractionPositive)) val samplingRate = 0.1 - checkAllCombos(stratifiedData, samplingRate, defaultSeed, n) + StratifiedAuxiliary.testSampleExact(stratifiedData, samplingRate, defaultSeed, n) } // vary fractionPositive for (fractionPositive <- List(0.1, 0.3, 0.5, 0.7, 0.9)) { val n = 100 val data = sc.parallelize(1 to n, 2) - val stratifiedData = data.keyBy(stratifier(fractionPositive)) - + val stratifiedData = data.keyBy(StratifiedAuxiliary.stratifier(fractionPositive)) val samplingRate = 0.1 - checkAllCombos(stratifiedData, samplingRate, defaultSeed, n) + StratifiedAuxiliary.testSampleExact(stratifiedData, samplingRate, defaultSeed, n) } // Use the same data for the rest of the tests val fractionPositive = 0.3 val n = 100 val data = sc.parallelize(1 to n, 2) - val stratifiedData = data.keyBy(stratifier(fractionPositive)) + val stratifiedData = data.keyBy(StratifiedAuxiliary.stratifier(fractionPositive)) // vary seed for (seed <- defaultSeed to defaultSeed + 5L) { val samplingRate = 0.1 - checkAllCombos(stratifiedData, samplingRate, seed, n) + StratifiedAuxiliary.testSampleExact(stratifiedData, samplingRate, seed, n) } // vary sampling rate for (samplingRate <- List(0.01, 0.05, 0.1, 0.5)) { - checkAllCombos(stratifiedData, samplingRate, defaultSeed, n) + StratifiedAuxiliary.testSampleExact(stratifiedData, samplingRate, defaultSeed, n) } } @@ -556,6 +519,97 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { intercept[IllegalArgumentException] {shuffled.lookup(-1)} } + private object StratifiedAuxiliary { + def stratifier (fractionPositive: Double) = { + (x: Int) => if (x % 10 < (10 * fractionPositive).toInt) "1" else "0" + } + + def checkSize(exact: Boolean, + withReplacement: Boolean, + expected: Long, + actual: Long, + p: Double): Boolean = { + if (exact) { + return expected == actual + } + val stdev = if (withReplacement) math.sqrt(expected) else math.sqrt(expected * p * (1 - p)) + // Very forgiving margin since we're dealing with very small sample sizes most of the time + math.abs(actual - expected) <= 6 * stdev + } + + def testSampleExact(stratifiedData: RDD[(String, Int)], + samplingRate: Double, + seed: Long, + n: Long) = { + testBernoulli(stratifiedData, true, samplingRate, seed, n) + testPoisson(stratifiedData, true, samplingRate, seed, n) + } + + def testSample(stratifiedData: RDD[(String, Int)], + samplingRate: Double, + seed: Long, + n: Long) = { + testBernoulli(stratifiedData, false, samplingRate, seed, n) + testPoisson(stratifiedData, false, samplingRate, seed, n) + } + + // Without replacement validation + def testBernoulli(stratifiedData: RDD[(String, Int)], + exact: Boolean, + samplingRate: Double, + seed: Long, + n: Long) = { + val expectedSampleSize = stratifiedData.countByKey() + .mapValues(count => math.ceil(count * samplingRate).toInt) + val fractions = Map("1" -> samplingRate, "0" -> samplingRate) + val sample = if (exact) { + stratifiedData.sampleByKeyExact(false, fractions, seed) + } else { + stratifiedData.sampleByKey(false, fractions, seed) + } + val sampleCounts = sample.countByKey() + val takeSample = sample.collect() + sampleCounts.foreach { case(k, v) => + assert(checkSize(exact, false, expectedSampleSize(k), v, samplingRate)) } + assert(takeSample.size === takeSample.toSet.size) + takeSample.foreach { x => assert(1 <= x._2 && x._2 <= n, s"elements not in [1, $n]") } + } + + // With replacement validation + def testPoisson(stratifiedData: RDD[(String, Int)], + exact: Boolean, + samplingRate: Double, + seed: Long, + n: Long) = { + val expectedSampleSize = stratifiedData.countByKey().mapValues(count => + math.ceil(count * samplingRate).toInt) + val fractions = Map("1" -> samplingRate, "0" -> samplingRate) + val sample = if (exact) { + stratifiedData.sampleByKeyExact(true, fractions, seed) + } else { + stratifiedData.sampleByKey(true, fractions, seed) + } + val sampleCounts = sample.countByKey() + val takeSample = sample.collect() + sampleCounts.foreach { case (k, v) => + assert(checkSize(exact, true, expectedSampleSize(k), v, samplingRate)) } + val groupedByKey = takeSample.groupBy(_._1) + for ((key, v) <- groupedByKey) { + if (expectedSampleSize(key) >= 100 && samplingRate >= 0.1) { + // sample large enough for there to be repeats with high likelihood + assert(v.toSet.size < expectedSampleSize(key)) + } else { + if (exact) { + assert(v.toSet.size <= expectedSampleSize(key)) + } else { + assert(checkSize(false, true, expectedSampleSize(key), v.toSet.size, samplingRate)) + } + } + } + takeSample.foreach { x => assert(1 <= x._2 && x._2 <= n, s"elements not in [1, $n]") } + } + } + } /* From 2948aae4ffe097616c9dde1e58a1d5bedc1c1bf7 Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Fri, 8 Aug 2014 23:32:41 -0700 Subject: [PATCH 27/28] remove unrelated changes --- docs/mllib-stats.md | 9 --------- pom.xml | 6 ------ 2 files changed, 15 deletions(-) delete mode 100644 docs/mllib-stats.md diff --git a/docs/mllib-stats.md b/docs/mllib-stats.md deleted file mode 100644 index 83caaa3bab0b..000000000000 --- a/docs/mllib-stats.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -layout: global -title: Linear Methods - MLlib -displayTitle: MLlib - Statistical Functions ---- - -* Table of contents -{:toc} - diff --git a/pom.xml b/pom.xml index ca4efe76060a..920912353fe9 100644 --- a/pom.xml +++ b/pom.xml @@ -277,12 +277,6 @@ 3.3 test - - org.apache.commons - commons-math3 - 3.3 - test - com.google.code.findbugs jsr305 From 0ad97b2b0b1c8088207028aa043bebad3a5bfbec Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Sun, 10 Aug 2014 15:50:24 -0700 Subject: [PATCH 28/28] reviewer comments. --- .../org/apache/spark/api/java/JavaPairRDD.scala | 2 -- .../org/apache/spark/rdd/PairRDDFunctions.scala | 1 - .../src/test/java/org/apache/spark/JavaAPISuite.java | 12 ++++++------ .../org/apache/spark/rdd/PairRDDFunctionsSuite.scala | 5 +++-- 4 files changed, 9 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index ad954f5944c1..feeb6c02caa7 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -158,7 +158,6 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) /** * ::Experimental:: - * * Return a subset of this RDD sampled by key (via stratified sampling) containing exactly * math.ceil(numItems * samplingRate) for each stratum (group of pairs with the same key). * @@ -176,7 +175,6 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) /** * ::Experimental:: - * * Return a subset of this RDD sampled by key (via stratified sampling) containing exactly * math.ceil(numItems * samplingRate) for each stratum (group of pairs with the same key). * diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 4fc1224cbbfc..108deb82e6ad 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -222,7 +222,6 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) /** * ::Experimental:: - * * Return a subset of this RDD sampled by key (via stratified sampling) containing exactly * math.ceil(numItems * samplingRate) for each stratum (group of pairs with the same key). * diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index a1449c090e36..e1c13de04a0b 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -1246,12 +1246,12 @@ public Tuple2 call(Integer i) { public void sampleByKeyExact() { JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8), 3); JavaPairRDD rdd2 = rdd1.mapToPair( - new PairFunction() { - @Override - public Tuple2 call(Integer i) { - return new Tuple2(i % 2, 1); - } - }); + new PairFunction() { + @Override + public Tuple2 call(Integer i) { + return new Tuple2(i % 2, 1); + } + }); Map fractions = Maps.newHashMap(); fractions.put(0, 0.5); fractions.put(1, 1.0); diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 4b15162eb8ba..63d3ddb4af98 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -592,7 +592,8 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { val sampleCounts = sample.countByKey() val takeSample = sample.collect() sampleCounts.foreach { case (k, v) => - assert(checkSize(exact, true, expectedSampleSize(k), v, samplingRate)) } + assert(checkSize(exact, true, expectedSampleSize(k), v, samplingRate)) + } val groupedByKey = takeSample.groupBy(_._1) for ((key, v) <- groupedByKey) { if (expectedSampleSize(key) >= 100 && samplingRate >= 0.1) { @@ -606,7 +607,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { } } } - takeSample.foreach { x => assert(1 <= x._2 && x._2 <= n, s"elements not in [1, $n]") } + takeSample.foreach(x => assert(1 <= x._2 && x._2 <= n, s"elements not in [1, $n]")) } }