From 14419775202e6eef1f0e1f0c74c7be9030aca73d Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Thu, 29 May 2014 15:22:14 -0700 Subject: [PATCH 1/4] 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 bab50f5ce2888..6cb58dbd291c4 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 aa03e9276fb34..2fdf45a0c8b8e 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 4dc8ada00a3e8..e53103755b279 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 e686068f7a99a..5bdcb9bef6d62 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 7bf9f135fd340..01d6eef32be63 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 8ef1e91f609fb..a6b6c26a49395 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 07578b8d937fc..b400404ad97c7 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 2/4] 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 6cb58dbd291c4..2b9f750e07d97 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 2fdf45a0c8b8e..9a5cf13e3ba52 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 e53103755b279..247f10173f1e9 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 5bdcb9bef6d62..08b3b93d6a31f 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 01d6eef32be63..64c8cd3c7810a 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 a6b6c26a49395..7314168fa8b6e 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 8f84f193e6ebc6f77fd31f56cd29b517a150e6b2 Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Thu, 29 May 2014 19:19:56 -0700 Subject: [PATCH 3/4] [SPARK-1970] unit test in XORShiftRandomSuite Since we're adding commons-math3 as a test dependency to spark core, updating the test for randomness in XORShiftRandomSuite to use an actual chi square test as opposed to hardcoding everything. --- .../util/random/XORShiftRandomSuite.scala | 49 +++++++------------ 1 file changed, 18 insertions(+), 31 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala index 0865c6386f7cd..3225d096a0f81 100644 --- a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala @@ -20,6 +20,8 @@ package org.apache.spark.util.random import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers +import org.apache.commons.math3.stat.inference.ChiSquareTest + import org.apache.spark.util.Utils.times import scala.language.reflectiveCalls @@ -33,45 +35,30 @@ class XORShiftRandomSuite extends FunSuite with ShouldMatchers { } /* - * This test is based on a chi-squared test for randomness. The values are hard-coded - * so as not to create Spark's dependency on apache.commons.math3 just to call one - * method for calculating the exact p-value for a given number of random numbers - * and bins. In case one would want to move to a full-fledged test based on - * apache.commons.math3, the relevant class is here: - * org.apache.commons.math3.stat.inference.ChiSquareTest + * This test is based on a chi-squared test for randomness. */ test ("XORShift generates valid random numbers") { val f = fixture - val numBins = 10 - // create 10 bins - val bins = Array.fill(numBins)(0) + val numBins = 10 // create 10 bins + val numRows = 5 // create 5 rows + val bins = Array.ofDim[Long](numRows, numBins) - // populate bins based on modulus of the random number - times(f.hundMil) {bins(math.abs(f.xorRand.nextInt) % 10) += 1} + // populate bins based on modulus of the random number for each row + for (r <- 0 to numRows-1) { + times(f.hundMil) {bins(r)(math.abs(f.xorRand.nextInt) % numBins) += 1} + } - /* since the seed is deterministic, until the algorithm is changed, we know the result will be - * exactly this: Array(10004908, 9993136, 9994600, 10000744, 10000091, 10002474, 10002272, - * 10000790, 10002286, 9998699), so the test will never fail at the prespecified (5%) - * significance level. However, should the RNG implementation change, the test should still - * pass at the same significance level. The chi-squared test done in R gave the following - * results: - * > chisq.test(c(10004908, 9993136, 9994600, 10000744, 10000091, 10002474, 10002272, - * 10000790, 10002286, 9998699)) - * Chi-squared test for given probabilities - * data: c(10004908, 9993136, 9994600, 10000744, 10000091, 10002474, 10002272, 10000790, - * 10002286, 9998699) - * X-squared = 11.975, df = 9, p-value = 0.2147 - * Note that the p-value was ~0.22. The test will fail if alpha < 0.05, which for 100 million - * random numbers - * and 10 bins will happen at X-squared of ~16.9196. So, the test will fail if X-squared - * is greater than or equal to that number. + /* + * Perform the chi square test on the 5 rows of randomly generated numbers evenly divided into + * 10 bins. chiSquareTest returns true iff the null hypothesis (that the classifications + * represented by the counts in the columns of the input 2-way table are independent of the + * rows) can be rejected with 100 * (1 - alpha) percent confidence, where alpha is prespeficied + * as 0.05 */ - val binSize = f.hundMil/numBins - val xSquared = bins.map(x => math.pow((binSize - x), 2)/binSize).sum - xSquared should be < (16.9196) - + val chiTest = new ChiSquareTest + assert(chiTest.chiSquareTest(bins, 0.05) === false) } test ("XORShift with zero seed") { From da0e8919afb5f3d335840a3dcae398ecf6921b27 Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Thu, 12 Jun 2014 20:10:38 -0700 Subject: [PATCH 4/4] remove math3 from common pom --- pom.xml | 5 ----- 1 file changed, 5 deletions(-) diff --git a/pom.xml b/pom.xml index ee0ddcc1d845c..0d46bb4114f73 100644 --- a/pom.xml +++ b/pom.xml @@ -256,11 +256,6 @@ commons-codec 1.5 - - org.apache.commons - commons-math3 - 3.3 - com.google.code.findbugs jsr305