Skip to content

Commit

Permalink
SPARK-4022 [CORE] [MLLIB] Replace colt dependency (LGPL) with commons…
Browse files Browse the repository at this point in the history
…-math

This change replaces usages of colt with commons-math3 equivalents, and makes some minor necessary adjustments to related code and tests to match.

Author: Sean Owen <[email protected]>

Closes apache#2928 from srowen/SPARK-4022 and squashes the following commits:

61a232f [Sean Owen] Fix failure due to different sampling in JavaAPISuite.sample()
16d66b8 [Sean Owen] Simplify seeding with call to reseedRandomGenerator
a1a78e0 [Sean Owen] Use Well19937c
31c7641 [Sean Owen] Fix Python Poisson test by choosing a different seed; about 88% of seeds should work but 1 didn't, it seems
5c9c67f [Sean Owen] Additional test fixes from review
d8f88e0 [Sean Owen] Replace colt with commons-math3. Some tests do not pass yet.
  • Loading branch information
srowen authored and mengxr committed Oct 27, 2014
1 parent 1d7bcc8 commit bfa614b
Show file tree
Hide file tree
Showing 23 changed files with 175 additions and 181 deletions.
12 changes: 0 additions & 12 deletions LICENSE
Original file line number Diff line number Diff line change
Expand Up @@ -712,18 +712,6 @@ THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE,
EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.

========================================================================
For colt:
========================================================================

Copyright (c) 1999 CERN - European Organization for Nuclear Research.
Permission to use, copy, modify, distribute and sell this software and its documentation for any purpose is hereby granted without fee, provided that the above copyright notice appear in all copies and that both that copyright notice and this permission notice appear in supporting documentation. CERN makes no representations about the suitability of this software for any purpose. It is provided "as is" without expressed or implied warranty.

Packages hep.aida.*

Written by Pavel Binko, Dino Ferrero Merlino, Wolfgang Hoschek, Tony Johnson, Andreas Pfeiffer, and others. Check the FreeHEP home page for more info. Permission to use and/or redistribute this work is granted under the terms of the LGPL License, with the exception that any usage related to military applications is expressly forbidden. The software and documentation made available under the terms of this license are provided with no warranty.


========================================================================
For SnapTree:
========================================================================
Expand Down
4 changes: 4 additions & 0 deletions assembly/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -146,6 +146,10 @@
<exclude>com/google/common/base/Present*</exclude>
</excludes>
</relocation>
<relocation>
<pattern>org.apache.commons.math3</pattern>
<shadedPattern>org.spark-project.commons.math3</shadedPattern>
</relocation>
</relocations>
<transformers>
<transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
Expand Down
6 changes: 0 additions & 6 deletions core/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -85,8 +85,6 @@
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-math3</artifactId>
<version>3.3</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>com.google.code.findbugs</groupId>
Expand Down Expand Up @@ -162,10 +160,6 @@
<artifactId>json4s-jackson_${scala.binary.version}</artifactId>
<version>3.2.10</version>
</dependency>
<dependency>
<groupId>colt</groupId>
<artifactId>colt</artifactId>
</dependency>
<dependency>
<groupId>org.apache.mesos</groupId>
<artifactId>mesos</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.spark.partial

import cern.jet.stat.Probability
import org.apache.commons.math3.distribution.NormalDistribution

/**
* An ApproximateEvaluator for counts.
Expand Down Expand Up @@ -46,7 +46,8 @@ private[spark] class CountEvaluator(totalOutputs: Int, confidence: Double)
val mean = (sum + 1 - p) / p
val variance = (sum + 1) * (1 - p) / (p * p)
val stdev = math.sqrt(variance)
val confFactor = Probability.normalInverse(1 - (1 - confidence) / 2)
val confFactor = new NormalDistribution().
inverseCumulativeProbability(1 - (1 - confidence) / 2)
val low = mean - confFactor * stdev
val high = mean + confFactor * stdev
new BoundedDouble(mean, confidence, low, high)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import scala.collection.Map
import scala.collection.mutable.HashMap
import scala.reflect.ClassTag

import cern.jet.stat.Probability
import org.apache.commons.math3.distribution.NormalDistribution

import org.apache.spark.util.collection.OpenHashMap

Expand Down Expand Up @@ -55,7 +55,8 @@ private[spark] class GroupedCountEvaluator[T : ClassTag](totalOutputs: Int, conf
new HashMap[T, BoundedDouble]
} else {
val p = outputsMerged.toDouble / totalOutputs
val confFactor = Probability.normalInverse(1 - (1 - confidence) / 2)
val confFactor = new NormalDistribution().
inverseCumulativeProbability(1 - (1 - confidence) / 2)
val result = new JHashMap[T, BoundedDouble](sums.size)
sums.foreach { case (key, sum) =>
val mean = (sum + 1 - p) / p
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.spark.partial

import cern.jet.stat.Probability
import org.apache.commons.math3.distribution.{NormalDistribution, TDistribution}

import org.apache.spark.util.StatCounter

Expand Down Expand Up @@ -45,9 +45,10 @@ private[spark] class MeanEvaluator(totalOutputs: Int, confidence: Double)
val stdev = math.sqrt(counter.sampleVariance / counter.count)
val confFactor = {
if (counter.count > 100) {
Probability.normalInverse(1 - (1 - confidence) / 2)
new NormalDistribution().inverseCumulativeProbability(1 - (1 - confidence) / 2)
} else {
Probability.studentTInverse(1 - confidence, (counter.count - 1).toInt)
val degreesOfFreedom = (counter.count - 1).toInt
new TDistribution(degreesOfFreedom).inverseCumulativeProbability(1 - (1 - confidence) / 2)
}
}
val low = mean - confFactor * stdev
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,16 +17,18 @@

package org.apache.spark.partial

import cern.jet.stat.Probability
import org.apache.commons.math3.distribution.{TDistribution, NormalDistribution}

/**
* A utility class for caching Student's T distribution values for a given confidence level
* and various sample sizes. This is used by the MeanEvaluator to efficiently calculate
* confidence intervals for many keys.
*/
private[spark] class StudentTCacher(confidence: Double) {

val NORMAL_APPROX_SAMPLE_SIZE = 100 // For samples bigger than this, use Gaussian approximation
val normalApprox = Probability.normalInverse(1 - (1 - confidence) / 2)

val normalApprox = new NormalDistribution().inverseCumulativeProbability(1 - (1 - confidence) / 2)
val cache = Array.fill[Double](NORMAL_APPROX_SAMPLE_SIZE)(-1.0)

def get(sampleSize: Long): Double = {
Expand All @@ -35,7 +37,8 @@ private[spark] class StudentTCacher(confidence: Double) {
} else {
val size = sampleSize.toInt
if (cache(size) < 0) {
cache(size) = Probability.studentTInverse(1 - confidence, size - 1)
val tDist = new TDistribution(size - 1)
cache(size) = tDist.inverseCumulativeProbability(1 - (1 - confidence) / 2)
}
cache(size)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.spark.partial

import cern.jet.stat.Probability
import org.apache.commons.math3.distribution.{TDistribution, NormalDistribution}

import org.apache.spark.util.StatCounter

Expand Down Expand Up @@ -55,9 +55,10 @@ private[spark] class SumEvaluator(totalOutputs: Int, confidence: Double)
val sumStdev = math.sqrt(sumVar)
val confFactor = {
if (counter.count > 100) {
Probability.normalInverse(1 - (1 - confidence) / 2)
new NormalDistribution().inverseCumulativeProbability(1 - (1 - confidence) / 2)
} else {
Probability.studentTInverse(1 - confidence, (counter.count - 1).toInt)
val degreesOfFreedom = (counter.count - 1).toInt
new TDistribution(degreesOfFreedom).inverseCumulativeProbability(1 - (1 - confidence) / 2)
}
}
val low = sumEstimate - confFactor * sumStdev
Expand Down
9 changes: 5 additions & 4 deletions core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala
Original file line number Diff line number Diff line change
Expand Up @@ -21,8 +21,7 @@ import java.util.Random

import scala.reflect.ClassTag

import cern.jet.random.Poisson
import cern.jet.random.engine.DRand
import org.apache.commons.math3.distribution.PoissonDistribution

import org.apache.spark.{Partition, TaskContext}

Expand Down Expand Up @@ -53,9 +52,11 @@ private[spark] class SampledRDD[T: ClassTag](
if (withReplacement) {
// For large datasets, the expected number of occurrences of each element in a sample with
// replacement is Poisson(frac). We use that to get a count for each element.
val poisson = new Poisson(frac, new DRand(split.seed))
val poisson = new PoissonDistribution(frac)
poisson.reseedRandomGenerator(split.seed)

firstParent[T].iterator(split.prev, context).flatMap { element =>
val count = poisson.nextInt()
val count = poisson.sample()
if (count == 0) {
Iterator.empty // Avoid object allocation when we return 0 items, which is quite often
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,7 @@ package org.apache.spark.util.random

import java.util.Random

import cern.jet.random.Poisson
import cern.jet.random.engine.DRand
import org.apache.commons.math3.distribution.PoissonDistribution

import org.apache.spark.annotation.DeveloperApi

Expand Down Expand Up @@ -87,15 +86,16 @@ class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false)
@DeveloperApi
class PoissonSampler[T](mean: Double) extends RandomSampler[T, T] {

private[random] var rng = new Poisson(mean, new DRand)
private[random] var rng = new PoissonDistribution(mean)

override def setSeed(seed: Long) {
rng = new Poisson(mean, new DRand(seed.toInt))
rng = new PoissonDistribution(mean)
rng.reseedRandomGenerator(seed)
}

override def sample(items: Iterator[T]): Iterator[T] = {
items.flatMap { item =>
val count = rng.nextInt()
val count = rng.sample()
if (count == 0) {
Iterator.empty
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,8 +22,7 @@ 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.commons.math3.distribution.PoissonDistribution

import org.apache.spark.Logging
import org.apache.spark.SparkContext._
Expand Down Expand Up @@ -209,7 +208,7 @@ private[spark] object StratifiedSamplingUtils extends Logging {
samplingRateByKey = computeThresholdByKey(finalResult, fractions)
}
(idx: Int, iter: Iterator[(K, V)]) => {
val rng = new RandomDataGenerator
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
Expand Down Expand Up @@ -245,9 +244,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)
val copiesWaitlisted = rng.nextPoisson(finalResult(key).waitListBound)
val copiesInSample = copiesAccepted +
(0 until copiesWailisted).count(i => rng.nextUniform() < thresholdByKey(key))
(0 until copiesWaitlisted).count(i => rng.nextUniform() < thresholdByKey(key))
if (copiesInSample > 0) {
Iterator.fill(copiesInSample.toInt)(item)
} else {
Expand All @@ -261,10 +260,10 @@ private[spark] object StratifiedSamplingUtils extends Logging {
rng.reSeed(seed + idx)
iter.flatMap { item =>
val count = rng.nextPoisson(fractions(item._1))
if (count > 0) {
Iterator.fill(count)(item)
} else {
if (count == 0) {
Iterator.empty
} else {
Iterator.fill(count)(item)
}
}
}
Expand All @@ -274,15 +273,24 @@ 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)
// commons-math3 doesn't have a method to generate Poisson from an arbitrary mean;
// maintain a cache of Poisson(m) distributions for various m
val poissonCache = mutable.Map[Double, PoissonDistribution]()
var poissonSeed = 0L

def reSeed(seed: Long) {
def reSeed(seed: Long): Unit = {
uniform.setSeed(seed)
poisson = new Poisson(1.0, new DRand(seed.toInt))
poissonSeed = seed
poissonCache.clear()
}

def nextPoisson(mean: Double): Int = {
poisson.nextInt(mean)
val poisson = poissonCache.getOrElseUpdate(mean, {
val newPoisson = new PoissonDistribution(mean)
newPoisson.reseedRandomGenerator(poissonSeed)
newPoisson
})
poisson.sample()
}

def nextUniform(): Double = {
Expand Down
2 changes: 1 addition & 1 deletion core/src/test/java/org/apache/spark/JavaAPISuite.java
Original file line number Diff line number Diff line change
Expand Up @@ -142,7 +142,7 @@ public void sample() {
JavaRDD<Integer> rdd = sc.parallelize(ints);
JavaRDD<Integer> sample20 = rdd.sample(true, 0.2, 11);
// expected 2 but of course result varies randomly a bit
Assert.assertEquals(3, sample20.count());
Assert.assertEquals(1, sample20.count());
JavaRDD<Integer> sample20NoReplacement = rdd.sample(false, 0.2, 11);
Assert.assertEquals(2, sample20NoReplacement.count());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,8 @@ package org.apache.spark.util.random

import java.util.Random

import cern.jet.random.Poisson
import org.apache.commons.math3.distribution.PoissonDistribution

import org.scalatest.{BeforeAndAfter, FunSuite}
import org.scalatest.mock.EasyMockSugar

Expand All @@ -28,11 +29,11 @@ class RandomSamplerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar
val a = List(1, 2, 3, 4, 5, 6, 7, 8, 9)

var random: Random = _
var poisson: Poisson = _
var poisson: PoissonDistribution = _

before {
random = mock[Random]
poisson = mock[Poisson]
poisson = mock[PoissonDistribution]
}

test("BernoulliSamplerWithRange") {
Expand Down Expand Up @@ -101,7 +102,7 @@ class RandomSamplerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar
test("PoissonSampler") {
expecting {
for(x <- Seq(0, 1, 2, 0, 1, 1, 0, 0, 0)) {
poisson.nextInt().andReturn(x)
poisson.sample().andReturn(x)
}
}
whenExecuting(poisson) {
Expand Down
8 changes: 8 additions & 0 deletions examples/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -156,6 +156,10 @@
<artifactId>algebird-core_${scala.binary.version}</artifactId>
<version>0.1.11</version>
</dependency>
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-math3</artifactId>
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
<artifactId>scalatest_${scala.binary.version}</artifactId>
Expand Down Expand Up @@ -268,6 +272,10 @@
<exclude>com.google.common.base.Optional**</exclude>
</excludes>
</relocation>
<relocation>
<pattern>org.apache.commons.math3</pattern>
<shadedPattern>org.spark-project.commons.math3</shadedPattern>
</relocation>
</relocations>
<transformers>
<transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
Expand Down
Loading

0 comments on commit bfa614b

Please sign in to comment.