From cd7bf7fc45cd5f685c165b71383f09edef35184f Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Thu, 9 Oct 2014 11:40:32 +0200 Subject: [PATCH 01/66] Parameterize CMS with type K This change parameterizes the type K that is used to identify the elements to be counted. Previously, the CMS implementation hardcoded `Long` as the type. In other words, this change turns the hardcoded CMS[K=Long] into a parameterized CMS[K]. --- .../com/twitter/algebird/CountMinSketch.scala | 464 +++++++++++------- .../com/twitter/algebird/SketchMap.scala | 5 +- .../twitter/algebird/CountMinSketchTest.scala | 49 +- .../summer/HeavyHittersCachingSummer.scala | 7 +- 4 files changed, 318 insertions(+), 207 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index b87058e74..2e67ce9f0 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -61,70 +61,100 @@ import scala.collection.immutable.SortedSet * are often of the form: "with probability p >= 1 - delta, the estimate is close to the truth by * some factor depending on eps." * - * @eps A parameter that bounds the error of each query estimate. - * @delta A bound on the probability that a query estimate does not lie within some small interval + * Implicit conversions for commonly used types for `K` such as [[Long]] and [[BigInt]]: + * {{{ + * import com.twitter.algebird.CMSHasherImplicits._ + * }}} + * + * @param eps A parameter that bounds the error of each query estimate. + * @param delta A bound on the probability that a query estimate does not lie within some small interval * (an interval that depends on eps) around the truth. - * @seed A seed to initialize the random number generator used to create the pairwise independent + * @param seed A seed to initialize the random number generator used to create the pairwise independent * hash functions. - * @heavyHittersPct A threshold for finding heavy hitters, i.e., elements that appear at least + * @param heavyHittersPct A threshold for finding heavy hitters, i.e., elements that appear at least * (heavyHittersPct * totalCount) times in the stream. + * @tparam K The type used to identify the elements to be counted. For example, if you want to count the occurrence of + * user names, you could map each username to a unique numeric ID expressed as a `Long`, and then count the + * occurrences of those `Long`s with a CMS of type `K=Long`. Note that this mapping between the elements of + * your problem domain and their identifiers used for counting via CMS should be bijective. + * We require [[Ordering]] and [[CMSHasher]] context bounds for `K`, see [[CMSHasherImplicits]] for available + * implicits that can be imported. + * Which type K should you pick in practice? For domains that have less than `2^64` unique elements, you'd + * typically use [[Long]]. For larger domains you can try [[BigInt]], for example. */ -class CountMinSketchMonoid(eps: Double, delta: Double, seed: Int, - heavyHittersPct: Double = 0.01) extends Monoid[CMS] { +class CountMinSketchMonoid[K: Ordering: CMSHasher](eps: Double, + delta: Double, + seed: Int, + heavyHittersPct: Double = 0.01) extends Monoid[CMS[K]] { assert(0 < eps && eps < 1, "eps must lie in (0, 1)") assert(0 < delta && delta < 1, "delta must lie in (0, 1)") assert(0 < heavyHittersPct && heavyHittersPct < 1, "heavyHittersPct must lie in (0, 1)") - // Typically, we would use d pairwise independent hash functions of the form + // Typically, we would use d pair-wise independent hash functions of the form // // h_i(x) = a_i * x + b_i (mod p) // - // But for this particular application, setting b_i does not matter - // (since all it does is shift the results of a particular hash), - // so we omit it and simply use hash functions of the form + // But for this particular application, setting b_i does not matter (since all it does is shift the results of a + // particular hash), so we omit it (by setting b_i to 0) and simply use hash functions of the form // // h_i(x) = a_i * x (mod p) - val hashes: Seq[CMSHash] = { + // + val hashes: Seq[CMSHash[K]] = { val r = new scala.util.Random(seed) val numHashes = CMS.depth(delta) val numCounters = CMS.width(eps) - (0 to (numHashes - 1)).map { _ => CMSHash(r.nextInt, 0, numCounters) } + (0 to (numHashes - 1)).map { _ => CMSHash[K](r.nextInt(), 0, numCounters) } } val params = CMSParams(hashes, eps, delta, heavyHittersPct) - val zero: CMS = CMSZero(params) + val zero: CMS[K] = CMSZero[K](params) /** * We assume the Count-Min sketches on the left and right use the same hash functions. */ - def plus(left: CMS, right: CMS): CMS = left ++ right + def plus(left: CMS[K], right: CMS[K]): CMS[K] = left ++ right /** * Create a Count-Min sketch out of a single item or data stream. */ - def create(item: Long): CMS = CMSItem(item, params) - def create(data: Seq[Long]): CMS = { + def create(item: K): CMS[K] = CMSItem[K](item, params) + + def create(data: Seq[K]): CMS[K] = { data.foldLeft(zero) { case (acc, x) => plus(acc, create(x)) } } + } object CMS { - def monoid(eps: Double, delta: Double, seed: Int, heavyHittersPct: Double = 0.01): CountMinSketchMonoid = - new CountMinSketchMonoid(eps, delta, seed, heavyHittersPct) - def monoid(depth: Int, width: Int, seed: Int, heavyHittersPct: Double): CountMinSketchMonoid = - new CountMinSketchMonoid(CMS.eps(width), CMS.delta(depth), seed, heavyHittersPct) - - def aggregator(eps: Double, delta: Double, seed: Int, heavyHittersPct: Double = 0.01): CountMinSketchAggregator = { - val monoid = new CountMinSketchMonoid(eps, delta, seed, heavyHittersPct) - new CountMinSketchAggregator(monoid) + def monoid[K: Ordering: CMSHasher](eps: Double, + delta: Double, + seed: Int, + heavyHittersPct: Double = 0.01): CountMinSketchMonoid[K] = + new CountMinSketchMonoid[K](eps, delta, seed, heavyHittersPct) + + def monoid[K: Ordering: CMSHasher](depth: Int, + width: Int, + seed: Int, + heavyHittersPct: Double): CountMinSketchMonoid[K] = + new CountMinSketchMonoid[K](CMS.eps(width), CMS.delta(depth), seed, heavyHittersPct) + + def aggregator[K: Ordering: CMSHasher](eps: Double, + delta: Double, + seed: Int, + heavyHittersPct: Double = 0.01): CountMinSketchAggregator[K] = { + val monoid = new CountMinSketchMonoid[K](eps, delta, seed, heavyHittersPct) + new CountMinSketchAggregator[K](monoid) } - def aggregator(depth: Int, width: Int, seed: Int, heavyHittersPct: Double): CountMinSketchAggregator = { - val monoid = new CountMinSketchMonoid(CMS.eps(width), CMS.delta(depth), seed, heavyHittersPct) - new CountMinSketchAggregator(monoid) + def aggregator[K: Ordering: CMSHasher](depth: Int, + width: Int, + seed: Int, + heavyHittersPct: Double): CountMinSketchAggregator[K] = { + val monoid = new CountMinSketchMonoid[K](CMS.eps(width), CMS.delta(depth), seed, heavyHittersPct) + new CountMinSketchAggregator[K](monoid) } /** @@ -133,26 +163,32 @@ object CMS { * width = ceil(e / eps) */ def eps(width: Int) = scala.math.exp(1.0) / width + def delta(depth: Int) = 1.0 / scala.math.exp(depth) + def depth(delta: Double) = scala.math.ceil(scala.math.log(1.0 / delta)).toInt + def width(eps: Double) = scala.math.ceil(scala.math.exp(1) / eps).toInt + } /** * The actual Count-Min sketch data structure. */ -sealed abstract class CMS extends java.io.Serializable { +sealed abstract class CMS[K] extends java.io.Serializable { // Parameters used to bound confidence in error estimates. def eps: Double + def delta: Double // Number of hash functions. def depth: Int = CMS.depth(delta) + // Number of counters per hash function. def width: Int = CMS.width(eps) - def ++(other: CMS): CMS + def ++(other: CMS[K]): CMS[K] /** * Returns an estimate of the total number of times this item has been seen @@ -162,7 +198,7 @@ sealed abstract class CMS extends java.io.Serializable { * With probability p >= 1 - delta, it also holds that * estimatedFrequency <= trueFrequency + eps * totalCount. */ - def frequency(item: Long): Approximate[Long] + def frequency(item: K): Approximate[Long] /** * Returns an estimate of the inner product against another data stream. @@ -177,7 +213,7 @@ sealed abstract class CMS extends java.io.Serializable { * With probability p >= 1 - delta, it also holds that * estimatedInnerProduct <= actualInnerProduct + eps * thisTotalCount * otherTotalCount */ - def innerProduct(other: CMS): Approximate[Long] + def innerProduct(other: CMS[K]): Approximate[Long] /** * Finds all heavy hitters, i.e., elements in the stream that appear at least @@ -192,129 +228,145 @@ sealed abstract class CMS extends java.io.Serializable { * time requires tracking at most 100 items. */ def heavyHittersPct: Double - def heavyHitters: Set[Long] - // Total number of elements seen in the data stream so far. + def heavyHitters: Set[K] + + /** + * Total number of elements seen in the data stream so far. + */ def totalCount: Long - // The first frequency moment is the total number of elements in the stream. + + /** + * The first frequency moment is the total number of elements in the stream. + */ def f1: Long = totalCount - // The second frequency moment is \sum a_i^2, where a_i is the count of the ith element. + + /** + * The second frequency moment is `\sum a_i^2`, where a_i is the count of the ith element. + */ def f2: Approximate[Long] = innerProduct(this) + } /** - * Used for initialization. + * Zero element. Used for initialization. */ -case class CMSZero(params: CMSParams) extends CMS { +case class CMSZero[K: Ordering](params: CMSParams[K]) extends CMS[K] { + def eps: Double = params.eps + def delta: Double = params.delta + def heavyHittersPct: Double = params.heavyHittersPct - def totalCount = 0L - def ++(other: CMS) = other - def frequency(item: Long) = Approximate.exact(0L) - def innerProduct(other: CMS) = Approximate.exact(0L) - def heavyHitters = Set[Long]() + def totalCount: Long = 0L + + def ++(other: CMS[K]): CMS[K] = other + + def frequency(item: K): Approximate[Long] = Approximate.exact(0L) + + def innerProduct(other: CMS[K]): Approximate[Long] = Approximate.exact(0L) + + def heavyHitters: Set[K] = Set[K]() + } /** - * Used for holding a single element, to avoid repeatedly adding elements from - * sparse counts tables. + * Used for holding a single element, to avoid repeatedly adding elements from sparse counts tables. */ -case class CMSItem(item: Long, params: CMSParams) extends CMS { +case class CMSItem[K: Ordering](item: K, params: CMSParams[K]) extends CMS[K] { def eps: Double = params.eps + def delta: Double = params.delta + def heavyHittersPct: Double = params.heavyHittersPct - def totalCount = 1L + def totalCount: Long = 1L - def ++(other: CMS): CMS = { + def ++(other: CMS[K]): CMS[K] = { other match { - case other: CMSZero => this - case other: CMSItem => CMSInstance(params) + item + other.item - case other: CMSInstance => other + item + case other: CMSZero[_] => this + case other: CMSItem[K] => CMSInstance[K](params) + item + other.item + case other: CMSInstance[K] => other + item } } - def frequency(x: Long) = if (item == x) Approximate.exact(1L) else Approximate.exact(0L) + def frequency(x: K): Approximate[Long] = if (item == x) Approximate.exact(1L) else Approximate.exact(0L) - def innerProduct(other: CMS): Approximate[Long] = other.frequency(item) + def innerProduct(other: CMS[K]): Approximate[Long] = other.frequency(item) - def heavyHitters = Set(item) + def heavyHitters: Set[K] = Set(item) } /** * The general Count-Min sketch structure, used for holding any number of elements. */ -case class CMSInstance(countsTable: CMSCountsTable, totalCount: Long, - hhs: HeavyHitters, params: CMSParams) extends CMS { +case class CMSInstance[K: Ordering](countsTable: CMSInstance.CMSCountsTable[K], + totalCount: Long, + hhs: CMSInstance.HeavyHitters[K], + params: CMSParams[K]) extends CMS[K] { def eps: Double = params.eps + def delta: Double = params.delta + def heavyHittersPct: Double = params.heavyHittersPct - def ++(other: CMS): CMS = { + def ++(other: CMS[K]): CMS[K] = { other match { - case other: CMSZero => this - case other: CMSItem => this + other.item - case other: CMSInstance => { + case other: CMSZero[_] => this + case other: CMSItem[K] => this + other.item + case other: CMSInstance[K] => val newTotalCount = totalCount + other.totalCount val newHhs = (hhs ++ other.hhs).dropCountsBelow(params.heavyHittersPct * newTotalCount) - CMSInstance(countsTable ++ other.countsTable, newTotalCount, newHhs, params) - } + CMSInstance[K](countsTable ++ other.countsTable, newTotalCount, newHhs, params) } } private def makeApprox(est: Long): Approximate[Long] = { - if (est == 0L) { - Approximate.exact(0L) - } else { + if (est == 0L) Approximate.exact(0L) + else { val lower = math.max(0L, est - (eps * totalCount).toLong) Approximate(lower, est, est, 1 - delta) } } - def frequency(item: Long): Approximate[Long] = { - val estimates = countsTable.counts.zipWithIndex.map { - case (row, i) => - row(params.hashes(i)(item)) - } + def frequency(item: K): Approximate[Long] = { + val estimates = countsTable.counts.zipWithIndex.map { case (row, i) => row(params.hashes(i)(item)) } makeApprox(estimates.min) } /** - * Let X be a CMS, and let count_X[j, k] denote the value in X's 2-dimensional count table at row j and - * column k. - * Then the Count-Min sketch estimate of the inner product between A and B is the minimum inner product - * between their rows: + * Let X be a CMS, and let count_X[j, k] denote the value in X's 2-dimensional count table at row j and column k. + * Then the Count-Min sketch estimate of the inner product between A and B is the minimum inner product between their + * rows: * estimatedInnerProduct = min_j (\sum_k count_A[j, k] * count_B[j, k]) */ - def innerProduct(other: CMS): Approximate[Long] = { + def innerProduct(other: CMS[K]): Approximate[Long] = { other match { - case other: CMSInstance => { + case other: CMSInstance[_] => assert((other.depth, other.width) == (depth, width), "Tables must have the same dimensions.") def innerProductAtDepth(d: Int) = (0 to (width - 1)).map { w => countsTable.getCount(d, w) * other.countsTable.getCount(d, w) }.sum - val est = (0 to (depth - 1)).map { innerProductAtDepth(_) }.min + val est = (0 to (depth - 1)).map { innerProductAtDepth }.min Approximate(est - (eps * totalCount * other.totalCount).toLong, est, est, 1 - delta) - } case _ => other.innerProduct(this) } } - def heavyHitters: Set[Long] = hhs.items + def heavyHitters: Set[K] = hhs.items /** * Updates the sketch with a new element from the data stream. */ - def +(item: Long): CMSInstance = this + (item, 1L) - def +(item: Long, count: Long): CMSInstance = { - if (count < 0) { - throw new Exception("Negative counts not implemented") - } else { + def +(item: K): CMSInstance[K] = this + (item, 1L) + + def +(item: K, count: Long): CMSInstance[K] = { + require(count >= 0, "count must be >= 0 (negative counts not implemented") + if (count != 0L) { val newHhs = updateHeavyHitters(item, count) val newCountsTable = (0 to (depth - 1)).foldLeft(countsTable) { @@ -322,16 +374,14 @@ case class CMSInstance(countsTable: CMSCountsTable, totalCount: Long, val pos = (row, params.hashes(row)(item)) table + (pos, count) } - - CMSInstance(newCountsTable, totalCount + count, newHhs, params) - } + CMSInstance[K](newCountsTable, totalCount + count, newHhs, params) + } else this } /** - * Updates the data structure of heavy hitters when a new item (with associated count) - * enters the stream. + * Updates the data structure of heavy hitters when a new item (with associated count) enters the stream. */ - private def updateHeavyHitters(item: Long, count: Long): HeavyHitters = { + private def updateHeavyHitters(item: K, count: Long): CMSInstance.HeavyHitters[K] = { val oldItemCount = frequency(item).estimate val newItemCount = oldItemCount + count val newTotalCount = totalCount + count @@ -339,139 +389,193 @@ case class CMSInstance(countsTable: CMSCountsTable, totalCount: Long, // If the new item is a heavy hitter, add it, and remove any previous instances. val newHhs = if (newItemCount >= heavyHittersPct * newTotalCount) { - hhs - HeavyHitter(item, oldItemCount) + HeavyHitter(item, newItemCount) - } else { - hhs - } + hhs - CMSInstance.HeavyHitter[K](item, oldItemCount) + CMSInstance.HeavyHitter[K](item, newItemCount) + } else hhs // Remove any items below the new heavy hitter threshold. newHhs.dropCountsBelow(heavyHittersPct * newTotalCount) } -} -object CMSInstance { - // Initializes a CMSInstance with all zeroes. - def apply(params: CMSParams): CMSInstance = { - val countsTable = CMSCountsTable(CMS.depth(params.delta), CMS.width(params.eps)) - CMSInstance(countsTable, 0, HeavyHitters(), params) - } } -/** - * The Count-Min sketch uses pairwise independent hash functions drawn from - * a universal hashing family of the form - * - * h(x) = [a * x + b (mod p)] (mod m) - */ -case class CMSHash(a: Int, b: Int, width: Int) { - - val PRIME_MODULUS = (1L << 31) - 1 +object CMSInstance { /** - * Returns a * x + b (mod p) (mod width) + * Initializes a CMSInstance with all zeroes. */ - def apply(x: Long): Int = { - val unmodded = a * x + b - - // Apparently a super fast way of computing x mod 2^p-1 - // See page 149 of - // http://www.cs.princeton.edu/courses/archive/fall09/cos521/Handouts/universalclasses.pdf - // after Proposition 7. - val modded = (unmodded + (unmodded >> 32)) & PRIME_MODULUS - - // Modulo-ing integers is apparently twice as fast as - // modulo-ing Longs. - modded.toInt % width + def apply[K: Ordering](params: CMSParams[K]): CMSInstance[K] = { + val countsTable = CMSCountsTable[K](CMS.depth(params.delta), CMS.width(params.eps)) + implicit val heavyHitterOrdering = HeavyHitter.ordering[K] + CMSInstance[K](countsTable, 0, HeavyHitters[K](SortedSet[HeavyHitter[K]]()), params) } -} -/** - * The 2-dimensional table of counters used in the Count-Min sketch. - * Each row corresponds to a particular hash function. - * TODO: implement a dense matrix type, and use it here - */ -case class CMSCountsTable(counts: Vector[Vector[Long]]) { - assert(depth > 0, "Table must have at least 1 row.") - assert(width > 0, "Table must have at least 1 column.") + /** + * The 2-dimensional table of counters used in the Count-Min sketch. + * Each row corresponds to a particular hash function. + * TODO: implement a dense matrix type, and use it here + */ + case class CMSCountsTable[K](counts: Vector[Vector[Long]]) { + assert(depth > 0, "Table must have at least 1 row.") + assert(width > 0, "Table must have at least 1 column.") - def depth: Int = counts.size - def width: Int = counts(0).size + def depth: Int = counts.size - def getCount(pos: (Int, Int)): Long = { - val (row, col) = pos + def width: Int = counts(0).size - assert(row < depth && col < width, "Position must be within the bounds of this table.") + def getCount(pos: (Int, Int)): Long = { + val (row, col) = pos - counts(row)(col) - } + assert(row < depth && col < width, "Position must be within the bounds of this table.") - /** - * Updates the count of a single cell in the table. - */ - def +(pos: (Int, Int), count: Long): CMSCountsTable = { - val (row, col) = pos - val currCount = getCount(pos) - val newCounts = counts.updated(row, counts(row).updated(col, currCount + count)) + counts(row)(col) + } - CMSCountsTable(newCounts) - } + /** + * Updates the count of a single cell in the table. + */ + def +(pos: (Int, Int), count: Long): CMSCountsTable[K] = { + val (row, col) = pos + val currCount = getCount(pos) + val newCounts = counts.updated(row, counts(row).updated(col, currCount + count)) + CMSCountsTable[K](newCounts) + } - /** - * Adds another counts table to this one, through elementwise addition. - */ - def ++(other: CMSCountsTable): CMSCountsTable = { - assert((depth, width) == (other.depth, other.width), "Tables must have the same dimensions.") - val iil = Monoid.plus[IndexedSeq[IndexedSeq[Long]]](counts, other.counts) - def toVector[V](is: IndexedSeq[V]): Vector[V] = { - is match { + /** + * Adds another counts table to this one, through element-wise addition. + */ + def ++(other: CMSCountsTable[K]): CMSCountsTable[K] = { + assert((depth, width) == (other.depth, other.width), "Tables must have the same dimensions.") + val iil: IndexedSeq[IndexedSeq[Long]] = Monoid.plus[IndexedSeq[IndexedSeq[Long]]](counts, other.counts) + def toVector[V](is: IndexedSeq[V]): Vector[V] = is match { case v: Vector[_] => v case _ => Vector(is: _*) } + CMSCountsTable[K](toVector(iil.map { toVector })) } - CMSCountsTable(toVector(iil.map { toVector(_) })) } -} -object CMSCountsTable { - // Creates a new CMSCountsTable with counts initialized to all zeroes. - def apply(depth: Int, width: Int): CMSCountsTable = CMSCountsTable(Vector.fill[Long](depth, width)(0L)) + object CMSCountsTable { + // Creates a new CMSCountsTable with counts initialized to all zeroes. + def apply[K: Ordering](depth: Int, width: Int): CMSCountsTable[K] = + CMSCountsTable[K](Vector.fill[Long](depth, width)(0L)) + } + + /** + * Containers for holding heavy hitter items and their associated counts. + */ + case class HeavyHitters[K: Ordering](hhs: SortedSet[HeavyHitter[K]]) { + + def -(hh: HeavyHitter[K]): HeavyHitters[K] = HeavyHitters[K](hhs - hh) + + def +(hh: HeavyHitter[K]): HeavyHitters[K] = HeavyHitters[K](hhs + hh) + + def ++(other: HeavyHitters[K]): HeavyHitters[K] = HeavyHitters[K](hhs ++ other.hhs) + + def items: Set[K] = hhs.map { _.item } + + def dropCountsBelow(minCount: Double): HeavyHitters[K] = HeavyHitters[K](hhs.dropWhile { _.count < minCount }) + } + + case class HeavyHitter[K: Ordering](item: K, count: Long) + + object HeavyHitter { + + def ordering[K: Ordering]: Ordering[HeavyHitter[K]] = Ordering.by { hh: HeavyHitter[K] => (hh.count, hh.item) } + + } + } /** * Convenience class for holding constant parameters of a Count-Min sketch. */ -case class CMSParams(hashes: Seq[CMSHash], eps: Double, delta: Double, heavyHittersPct: Double) +case class CMSParams[K](hashes: Seq[CMSHash[K]], eps: Double, delta: Double, heavyHittersPct: Double) /** - * Containers for holding heavy hitter items and their associated counts. + * An Aggregator for the CountMinSketch. Can be created using `CMS.aggregator`. */ -case class HeavyHitters( - hhs: SortedSet[HeavyHitter] = SortedSet[HeavyHitter]()(HeavyHitter.ordering)) { +case class CountMinSketchAggregator[K](cmsMonoid: CountMinSketchMonoid[K]) + extends MonoidAggregator[K, CMS[K], CMS[K]] { - def -(hh: HeavyHitter) = HeavyHitters(hhs - hh) - def +(hh: HeavyHitter) = HeavyHitters(hhs + hh) + val monoid = cmsMonoid - def ++(other: HeavyHitters) = HeavyHitters(hhs ++ other.hhs) + def prepare(value: K): CMS[K] = monoid.create(value) - def items: Set[Long] = hhs.map { _.item } + def present(cms: CMS[K]): CMS[K] = cms - def dropCountsBelow(minCount: Double): HeavyHitters = { - HeavyHitters(hhs.dropWhile { _.count < minCount }) - } } -case class HeavyHitter(item: Long, count: Long) -object HeavyHitter { - val ordering = Ordering.by { hh: HeavyHitter => (hh.count, hh.item) } +trait CMSHasher[K] { + + val PRIME_MODULUS = (1L << 31) - 1 + + /** + * Returns `a * x + b (mod p) (mod width)`. + */ + def hash(a: Int, b: Int, width: Int)(x: K): Int + } /** - * An Aggregator for the CountMinSketch. - * Can be created using CMS.aggregator + * The Count-Min sketch uses `d` pair-wise independent hash functions drawn from a universal hashing family of the form: + * + * `h(x) = [a * x + b (mod p)] (mod m)` */ -case class CountMinSketchAggregator(cmsMonoid: CountMinSketchMonoid) extends MonoidAggregator[Long, CMS, CMS] { - val monoid = cmsMonoid +case class CMSHash[K: CMSHasher](a: Int, b: Int, width: Int) { + + /** + * Returns `a * x + b (mod p) (mod width)`. + */ + def apply(x: K): Int = implicitly[CMSHasher[K]].hash(a, b, width)(x) - def prepare(value: Long) = monoid.create(value) - def present(cms: CMS) = cms } + +object CMSHasherImplicits { + + implicit object CMSHasherLong extends CMSHasher[Long] { + + def hash(a: Int, b: Int, width: Int)(x: Long) = { + val unModded: Long = (x * a) + b + // Apparently a super fast way of computing x mod 2^p-1 + // See page 149 of http://www.cs.princeton.edu/courses/archive/fall09/cos521/Handouts/universalclasses.pdf + // after Proposition 7. + val modded: Long = (unModded + (unModded >> 32)) & PRIME_MODULUS + // Modulo-ing integers is apparently twice as fast as modulo-ing Longs. + modded.toInt % width + } + + } + + implicit object CMSHasherByte extends CMSHasher[Byte] { + + def hash(a: Int, b: Int, width: Int)(x: Byte) = CMSHasherInt.hash(a, b, width)(x) + + } + + implicit object CMSHasherShort extends CMSHasher[Short] { + + def hash(a: Int, b: Int, width: Int)(x: Short) = CMSHasherInt.hash(a, b, width)(x) + + } + + implicit object CMSHasherInt extends CMSHasher[Int] { + + def hash(a: Int, b: Int, width: Int)(x: Int) = { + val unModded: Int = (x * a) + b + val modded: Long = (unModded + (unModded >> 32)) & PRIME_MODULUS + modded.toInt % width + } + + } + + implicit object CMSHasherBigInt extends CMSHasher[BigInt] { + + def hash(a: Int, b: Int, width: Int)(x: BigInt) = { + val unModded: BigInt = (x * a) + b + val modded: BigInt = (unModded + (unModded >> 32)) & PRIME_MODULUS + modded.toInt % width + } + + } + +} \ No newline at end of file diff --git a/algebird-core/src/main/scala/com/twitter/algebird/SketchMap.scala b/algebird-core/src/main/scala/com/twitter/algebird/SketchMap.scala index 827644c5c..5fb1aad3a 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/SketchMap.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/SketchMap.scala @@ -17,6 +17,7 @@ limitations under the License. package com.twitter.algebird import scala.collection.breakOut +import com.twitter.algebird.CMSHasherImplicits._ import com.twitter.algebird.matrix.AdaptiveMatrix /** @@ -28,7 +29,7 @@ import com.twitter.algebird.matrix.AdaptiveMatrix /** * Hashes an arbitrary key type to one that the Sketch Map can use. */ -case class SketchMapHash[K](hasher: CMSHash, seed: Int)(implicit serialization: K => Array[Byte]) { +case class SketchMapHash[K](hasher: CMSHash[Long], seed: Int)(implicit serialization: K => Array[Byte]) { def apply(obj: K): Int = { val (first, second) = MurmurHash128(seed)(serialization(obj)) hasher(first ^ second) @@ -137,7 +138,7 @@ case class SketchMapParams[K](seed: Int, width: Int, depth: Int, heavyHittersCou val numHashes = depth val numCounters = width (0 to (numHashes - 1)).map { _ => - val smhash: SketchMapHash[K] = SketchMapHash(CMSHash(r.nextInt, 0, numCounters), seed)(serialization) + val smhash: SketchMapHash[K] = SketchMapHash(CMSHash[Long](r.nextInt, 0, numCounters), seed)(serialization) new (K => Int) { override def apply(k: K) = smhash(k) } } } diff --git a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala index be8b859ef..1ec7b595d 100644 --- a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala +++ b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala @@ -1,45 +1,48 @@ package com.twitter.algebird -import org.scalatest._ - -import org.scalatest.{ DiagrammedAssertions, PropSpec, Matchers, WordSpec } +import org.scalatest.{ PropSpec, Matchers, WordSpec } import org.scalatest.prop.PropertyChecks import org.scalacheck.{ Gen, Arbitrary } class CountMinSketchLaws extends PropSpec with PropertyChecks with Matchers { import BaseProperties._ + import CMSHasherImplicits._ + + type KEY = Long val DELTA = 1E-8 val EPS = 0.005 val SEED = 1 - implicit val cmsMonoid = new CountMinSketchMonoid(EPS, DELTA, SEED) + implicit val cmsMonoid = new CountMinSketchMonoid[KEY](EPS, DELTA, SEED) implicit val cmsGen = Arbitrary { - for (v <- Gen.choose(0, 10000)) yield (cmsMonoid.create(v)) + for (v <- Gen.choose(0, 10000)) yield cmsMonoid.create(v) } property("CountMinSketch is a Monoid") { - monoidLaws[CMS] + monoidLaws[CMS[KEY]] } } class CountMinSketchTest extends WordSpec with Matchers { + import CMSHasherImplicits._ + + type KEY = Long // To highlight which type we're testing; actual test data is still hardcoded to Long, e.g. `1L`. + val DELTA = 1E-10 val EPS = 0.001 val SEED = 1 - val CMS_MONOID = new CountMinSketchMonoid(EPS, DELTA, SEED) + val CMS_MONOID = new CountMinSketchMonoid[KEY](EPS, DELTA, SEED) val RAND = new scala.util.Random /** * Returns the exact frequency of {x} in {data}. */ - def exactFrequency(data: Seq[Long], x: Long): Long = { - data.filter { _ == x }.size - } + def exactFrequency(data: Seq[Long], x: Long): Long = data.count(_ == x) /** * Returns the exact inner product between two data streams, when the streams @@ -84,8 +87,8 @@ class CountMinSketchTest extends WordSpec with Matchers { val approx = cms.frequency(x).estimate val maxError = approx - cms.frequency(x).min - assert(approx >= (exact)) - assert((approx - exact) <= (maxError)) + assert(approx >= exact) + assert((approx - exact) <= maxError) } } @@ -154,13 +157,13 @@ class CountMinSketchTest extends WordSpec with Matchers { val data = data1 ++ data2 ++ data3 // Find elements that appear at least 20% of the time. - val cms = (new CountMinSketchMonoid(EPS, DELTA, SEED, 0.2)).create(data) + val cms = new CountMinSketchMonoid[KEY](EPS, DELTA, SEED, 0.2).create(data) val trueHhs = exactHeavyHitters(data, cms.heavyHittersPct) val estimatedHhs = cms.heavyHitters // All true heavy hitters must be claimed as heavy hitters. - assert((trueHhs.intersect(estimatedHhs) == trueHhs) == true) + assert(trueHhs.intersect(estimatedHhs) == trueHhs) // It should be very unlikely that any element with count less than // (heavyHittersPct - eps) * totalCount is claimed as a heavy hitter. @@ -170,7 +173,7 @@ class CountMinSketchTest extends WordSpec with Matchers { } "drop old heavy hitters when new heavy hitters replace them" in { - val monoid = new CountMinSketchMonoid(EPS, DELTA, SEED, 0.3) + val monoid = new CountMinSketchMonoid[KEY](EPS, DELTA, SEED, 0.3) val cms1 = monoid.create(Seq(1L, 2L, 2L)) assert(cms1.heavyHitters == Set(1L, 2L)) @@ -186,10 +189,10 @@ class CountMinSketchTest extends WordSpec with Matchers { "exactly compute heavy hitters in a small stream" in { val data1 = Seq(1L, 2L, 2L, 3L, 3L, 3L, 4L, 4L, 4L, 4L, 5L, 5L, 5L, 5L, 5L) - val cms1 = (new CountMinSketchMonoid(EPS, DELTA, SEED, 0.01)).create(data1) - val cms2 = (new CountMinSketchMonoid(EPS, DELTA, SEED, 0.1)).create(data1) - val cms3 = (new CountMinSketchMonoid(EPS, DELTA, SEED, 0.3)).create(data1) - val cms4 = (new CountMinSketchMonoid(EPS, DELTA, SEED, 0.9)).create(data1) + val cms1 = new CountMinSketchMonoid[KEY](EPS, DELTA, SEED, 0.01).create(data1) + val cms2 = new CountMinSketchMonoid[KEY](EPS, DELTA, SEED, 0.1).create(data1) + val cms3 = new CountMinSketchMonoid[KEY](EPS, DELTA, SEED, 0.3).create(data1) + val cms4 = new CountMinSketchMonoid[KEY](EPS, DELTA, SEED, 0.9).create(data1) assert(cms1.heavyHitters == Set(1L, 2L, 3L, 4L, 5L)) assert(cms2.heavyHitters == Set(2L, 3L, 4L, 5L)) assert(cms3.heavyHitters == Set(5L)) @@ -198,10 +201,10 @@ class CountMinSketchTest extends WordSpec with Matchers { "work as an Aggregator" in { val data1 = Seq(1L, 2L, 2L, 3L, 3L, 3L, 4L, 4L, 4L, 4L, 5L, 5L, 5L, 5L, 5L) - val cms1 = CMS.aggregator(EPS, DELTA, SEED, 0.01).apply(data1) - val cms2 = CMS.aggregator(EPS, DELTA, SEED, 0.1).apply(data1) - val cms3 = CMS.aggregator(EPS, DELTA, SEED, 0.3).apply(data1) - val cms4 = CMS.aggregator(EPS, DELTA, SEED, 0.9).apply(data1) + val cms1 = CMS.aggregator[KEY](EPS, DELTA, SEED, 0.01).apply(data1) + val cms2 = CMS.aggregator[KEY](EPS, DELTA, SEED, 0.1).apply(data1) + val cms3 = CMS.aggregator[KEY](EPS, DELTA, SEED, 0.3).apply(data1) + val cms4 = CMS.aggregator[KEY](EPS, DELTA, SEED, 0.9).apply(data1) assert(cms1.heavyHitters == Set(1L, 2L, 3L, 4L, 5L)) assert(cms2.heavyHitters == Set(2L, 3L, 4L, 5L)) assert(cms3.heavyHitters == Set(5L)) diff --git a/algebird-util/src/main/scala/com/twitter/algebird/util/summer/HeavyHittersCachingSummer.scala b/algebird-util/src/main/scala/com/twitter/algebird/util/summer/HeavyHittersCachingSummer.scala index 24063eb90..90d28941c 100644 --- a/algebird-util/src/main/scala/com/twitter/algebird/util/summer/HeavyHittersCachingSummer.scala +++ b/algebird-util/src/main/scala/com/twitter/algebird/util/summer/HeavyHittersCachingSummer.scala @@ -42,6 +42,9 @@ case class RollOverFrequency(toLong: Long) case class HeavyHittersPercent(toFloat: Float) class ApproxHHTracker(hhPct: HeavyHittersPercent, updateFreq: UpdateFrequency, roFreq: RollOverFrequency) { + + import CMSHasherImplicits._ + private[this] final val WIDTH = 1000 private[this] final val DEPTH = 4 private[this] final val hh = new java.util.HashMap[Int, Long]() @@ -52,9 +55,9 @@ class ApproxHHTracker(hhPct: HeavyHittersPercent, updateFreq: UpdateFrequency, r private[this] final val rollOverFrequency = roFreq.toLong private[this] final var countsTable = Array.fill(WIDTH * DEPTH)(0L) - private[this] final val hashes: IndexedSeq[CMSHash] = { + private[this] final val hashes: IndexedSeq[CMSHash[Long]] = { val r = new scala.util.Random(5) - (0 until DEPTH).map { _ => CMSHash(r.nextInt, 0, WIDTH) } + (0 until DEPTH).map { _ => CMSHash[Long](r.nextInt, 0, WIDTH) } }.toIndexedSeq @inline From 90fe5e138eba59e27963e1b094a197b3fc1c145e Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Thu, 9 Oct 2014 12:34:02 +0200 Subject: [PATCH 02/66] Use require instead of assert to enforce contract "If a static checker finds a violation of assert it considers it an error in the code, while when require is violated it assumes the caller is at fault." This commit changes assert to require in those places where it is used to validate input parameters. See stackoverflow.com/questions/8002835 --- .../com/twitter/algebird/CountMinSketch.scala | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index 2e67ce9f0..6738f7f7c 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -87,9 +87,9 @@ class CountMinSketchMonoid[K: Ordering: CMSHasher](eps: Double, seed: Int, heavyHittersPct: Double = 0.01) extends Monoid[CMS[K]] { - assert(0 < eps && eps < 1, "eps must lie in (0, 1)") - assert(0 < delta && delta < 1, "delta must lie in (0, 1)") - assert(0 < heavyHittersPct && heavyHittersPct < 1, "heavyHittersPct must lie in (0, 1)") + require(0 < eps && eps < 1, "eps must lie in (0, 1)") + require(0 < delta && delta < 1, "delta must lie in (0, 1)") + require(0 < heavyHittersPct && heavyHittersPct < 1, "heavyHittersPct must lie in (0, 1)") // Typically, we would use d pair-wise independent hash functions of the form // @@ -345,7 +345,7 @@ case class CMSInstance[K: Ordering](countsTable: CMSInstance.CMSCountsTable[K], def innerProduct(other: CMS[K]): Approximate[Long] = { other match { case other: CMSInstance[_] => - assert((other.depth, other.width) == (depth, width), "Tables must have the same dimensions.") + require((other.depth, other.width) == (depth, width), "Tables must have the same dimensions.") def innerProductAtDepth(d: Int) = (0 to (width - 1)).map { w => countsTable.getCount(d, w) * other.countsTable.getCount(d, w) @@ -415,8 +415,8 @@ object CMSInstance { * TODO: implement a dense matrix type, and use it here */ case class CMSCountsTable[K](counts: Vector[Vector[Long]]) { - assert(depth > 0, "Table must have at least 1 row.") - assert(width > 0, "Table must have at least 1 column.") + require(depth > 0, "Table must have at least 1 row.") + require(width > 0, "Table must have at least 1 column.") def depth: Int = counts.size @@ -424,9 +424,7 @@ object CMSInstance { def getCount(pos: (Int, Int)): Long = { val (row, col) = pos - - assert(row < depth && col < width, "Position must be within the bounds of this table.") - + require(row < depth && col < width, "Position must be within the bounds of this table.") counts(row)(col) } @@ -444,7 +442,7 @@ object CMSInstance { * Adds another counts table to this one, through element-wise addition. */ def ++(other: CMSCountsTable[K]): CMSCountsTable[K] = { - assert((depth, width) == (other.depth, other.width), "Tables must have the same dimensions.") + require((depth, width) == (other.depth, other.width), "Tables must have the same dimensions.") val iil: IndexedSeq[IndexedSeq[Long]] = Monoid.plus[IndexedSeq[IndexedSeq[Long]]](counts, other.counts) def toVector[V](is: IndexedSeq[V]): Vector[V] = is match { case v: Vector[_] => v From c770d79e2df2be6110c93357165b1c497696ca6d Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Thu, 9 Oct 2014 13:22:42 +0200 Subject: [PATCH 03/66] Test monoid laws of CMS for Byte, Short, Int, Long, BigInt --- .../twitter/algebird/CountMinSketchTest.scala | 39 +++++++++++++++---- 1 file changed, 32 insertions(+), 7 deletions(-) diff --git a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala index 1ec7b595d..308067c4d 100644 --- a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala +++ b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala @@ -8,20 +8,45 @@ class CountMinSketchLaws extends PropSpec with PropertyChecks with Matchers { import BaseProperties._ import CMSHasherImplicits._ - type KEY = Long - val DELTA = 1E-8 val EPS = 0.005 val SEED = 1 - implicit val cmsMonoid = new CountMinSketchMonoid[KEY](EPS, DELTA, SEED) - implicit val cmsGen = + private def createArbitrary[K : Numeric](cmsMonoid: CountMinSketchMonoid[K]): Arbitrary[CMS[K]] = { + val k = implicitly[Numeric[K]] Arbitrary { - for (v <- Gen.choose(0, 10000)) yield cmsMonoid.create(v) + for (v <- Gen.choose(0, 10000)) yield cmsMonoid.create(k.fromInt(v)) } + } + + property("CountMinSketch[Byte] is a Monoid") { + implicit val cmsMonoid = new CountMinSketchMonoid[Byte](EPS, DELTA, SEED) + implicit val cmsGen = createArbitrary[Byte](cmsMonoid) + monoidLaws[CMS[Byte]] + } + + property("CountMinSketch[Short] is a Monoid") { + implicit val cmsMonoid = new CountMinSketchMonoid[Short](EPS, DELTA, SEED) + implicit val cmsGen = createArbitrary[Short](cmsMonoid) + monoidLaws[CMS[Short]] + } + + property("CountMinSketch[Int] is a Monoid") { + implicit val cmsMonoid = new CountMinSketchMonoid[Int](EPS, DELTA, SEED) + implicit val cmsGen = createArbitrary[Int](cmsMonoid) + monoidLaws[CMS[Int]] + } + + property("CountMinSketch[Long] is a Monoid") { + implicit val cmsMonoid = new CountMinSketchMonoid[Long](EPS, DELTA, SEED) + implicit val cmsGen = createArbitrary[Long](cmsMonoid) + monoidLaws[CMS[Long]] + } - property("CountMinSketch is a Monoid") { - monoidLaws[CMS[KEY]] + property("CountMinSketch[BigInt] is a Monoid") { + implicit val cmsMonoid = new CountMinSketchMonoid[BigInt](EPS, DELTA, SEED) + implicit val cmsGen = createArbitrary[BigInt](cmsMonoid) + monoidLaws[CMS[BigInt]] } } From 89acbfc6bb02cf7e9e3ba21ae72d60fc710f0009 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Thu, 9 Oct 2014 14:10:29 +0200 Subject: [PATCH 04/66] CMS: use a type alias to test K=Long in test spec --- .../twitter/algebird/CountMinSketchTest.scala | 101 ++++++++++-------- 1 file changed, 57 insertions(+), 44 deletions(-) diff --git a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala index 308067c4d..d6a59775d 100644 --- a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala +++ b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala @@ -6,7 +6,7 @@ import org.scalacheck.{ Gen, Arbitrary } class CountMinSketchLaws extends PropSpec with PropertyChecks with Matchers { import BaseProperties._ - import CMSHasherImplicits._ + import CMSHasherImplicits._ // required, although e.g. IntelliJ IDEA may flag it as unused import val DELTA = 1E-8 val EPS = 0.005 @@ -55,7 +55,7 @@ class CountMinSketchTest extends WordSpec with Matchers { import CMSHasherImplicits._ - type KEY = Long // To highlight which type we're testing; actual test data is still hardcoded to Long, e.g. `1L`. + type KEY = Long val DELTA = 1E-10 val EPS = 0.001 @@ -67,13 +67,13 @@ class CountMinSketchTest extends WordSpec with Matchers { /** * Returns the exact frequency of {x} in {data}. */ - def exactFrequency(data: Seq[Long], x: Long): Long = data.count(_ == x) + def exactFrequency(data: Seq[KEY], x: KEY): Long = data.count(_ == x) /** * Returns the exact inner product between two data streams, when the streams * are viewed as count vectors. */ - def exactInnerProduct(data1: Seq[Long], data2: Seq[Long]): Long = { + def exactInnerProduct(data1: Seq[KEY], data2: Seq[KEY]): Long = { val counts1 = data1.groupBy(x => x).mapValues(_.size) val counts2 = data2.groupBy(x => x).mapValues(_.size) @@ -83,7 +83,7 @@ class CountMinSketchTest extends WordSpec with Matchers { /** * Returns the elements in {data} that appear at least heavyHittersPct * data.size times. */ - def exactHeavyHitters(data: Seq[Long], heavyHittersPct: Double): Set[Long] = { + def exactHeavyHitters(data: Seq[KEY], heavyHittersPct: Double): Set[KEY] = { val counts = data.groupBy(x => x).mapValues(_.size) val totalCount = counts.values.sum counts.filter { _._2 >= heavyHittersPct * totalCount }.keys.toSet @@ -94,7 +94,7 @@ class CountMinSketchTest extends WordSpec with Matchers { "count total number of elements in a stream" in { val totalCount = 1243 val range = 234 - val data = (0 to (totalCount - 1)).map { _ => RAND.nextInt(range).toLong } + val data = (0 to (totalCount - 1)).map { _ => RAND.nextInt(range) }.toK[KEY] val cms = CMS_MONOID.create(data) assert(cms.totalCount == totalCount) @@ -103,11 +103,11 @@ class CountMinSketchTest extends WordSpec with Matchers { "estimate frequencies" in { val totalCount = 5678 val range = 897 - val data = (0 to (totalCount - 1)).map { _ => RAND.nextInt(range).toLong } + val data = (0 to (totalCount - 1)).map { _ => RAND.nextInt(range) }.toK[KEY] val cms = CMS_MONOID.create(data) (0 to 100).foreach { _ => - val x = RAND.nextInt(range).toLong + val x = RAND.nextInt(range).toK[KEY] val exact = exactFrequency(data, x) val approx = cms.frequency(x).estimate val maxError = approx - cms.frequency(x).min @@ -126,19 +126,19 @@ class CountMinSketchTest extends WordSpec with Matchers { assert(cms.frequency(1).estimate == 1) assert(cms.frequency(2).estimate == 2) - val three = CMS_MONOID.create(Seq(1L, 1L, 1L)) - assert(three.frequency(1L).estimate == 3) - val four = CMS_MONOID.create(Seq(1L, 1L, 1L, 1L)) - assert(four.frequency(1L).estimate == 4) + val three = CMS_MONOID.create(Seq(1, 1, 1).toK[KEY]) + assert(three.frequency(1).estimate == 3) + val four = CMS_MONOID.create(Seq(1, 1, 1, 1).toK[KEY]) + assert(four.frequency(1).estimate == 4) val cms2 = CMS_MONOID.plus(four, three) - assert(cms2.frequency(1L).estimate == 7) + assert(cms2.frequency(1).estimate == 7) } "estimate inner products" in { val totalCount = 5234 val range = 1390 - val data1 = (0 to (totalCount - 1)).map { _ => RAND.nextInt(range).toLong } - val data2 = (0 to (totalCount - 1)).map { _ => RAND.nextInt(range).toLong } + val data1 = (0 to (totalCount - 1)).map { _ => RAND.nextInt(range) }.toK[KEY] + val data2 = (0 to (totalCount - 1)).map { _ => RAND.nextInt(range) }.toK[KEY] val cms1 = CMS_MONOID.create(data1) val cms2 = CMS_MONOID.create(data1) @@ -154,31 +154,31 @@ class CountMinSketchTest extends WordSpec with Matchers { "exactly compute inner product of small streams" in { // Nothing in common. - val a1 = List(1L, 2L, 3L) - val a2 = List(4L, 5L, 6L) + val a1 = List(1, 2, 3).toK[KEY] + val a2 = List(4, 5, 6).toK[KEY] assert(CMS_MONOID.create(a1).innerProduct(CMS_MONOID.create(a2)).estimate == 0) // One element in common. - val b1 = List(1L, 2L, 3L) - val b2 = List(3L, 5L, 6L) + val b1 = List(1, 2, 3).toK[KEY] + val b2 = List(3, 5, 6).toK[KEY] assert(CMS_MONOID.create(b1).innerProduct(CMS_MONOID.create(b2)).estimate == 1) // Multiple, non-repeating elements in common. - val c1 = List(1L, 2L, 3L) - val c2 = List(3L, 2L, 6L) + val c1 = List(1, 2, 3).toK[KEY] + val c2 = List(3, 2, 6).toK[KEY] assert(CMS_MONOID.create(c1).innerProduct(CMS_MONOID.create(c2)).estimate == 2) // Multiple, repeating elements in common. - val d1 = List(1L, 2L, 2L, 3L, 3L) - val d2 = List(2L, 3L, 3L, 6L) + val d1 = List(1, 2, 2, 3, 3).toK[KEY] + val d2 = List(2, 3, 3, 6).toK[KEY] assert(CMS_MONOID.create(d1).innerProduct(CMS_MONOID.create(d2)).estimate == 6) } "estimate heavy hitters" in { // Simple way of making some elements appear much more often than others. - val data1 = (1 to 3000).map { _ => RAND.nextInt(3).toLong } - val data2 = (1 to 3000).map { _ => RAND.nextInt(10).toLong } - val data3 = (1 to 1450).map { _ => -1L } // element close to being a 20% heavy hitter + val data1 = (1 to 3000).map { _ => RAND.nextInt(3) }.toK[KEY] + val data2 = (1 to 3000).map { _ => RAND.nextInt(10) }.toK[KEY] + val data3 = (1 to 1450).map { _ => -1 }.toK[KEY] // element close to being a 20% heavy hitter val data = data1 ++ data2 ++ data3 // Find elements that appear at least 20% of the time. @@ -199,41 +199,54 @@ class CountMinSketchTest extends WordSpec with Matchers { "drop old heavy hitters when new heavy hitters replace them" in { val monoid = new CountMinSketchMonoid[KEY](EPS, DELTA, SEED, 0.3) - val cms1 = monoid.create(Seq(1L, 2L, 2L)) - assert(cms1.heavyHitters == Set(1L, 2L)) + val cms1 = monoid.create(Seq(1, 2, 2).toK[KEY]) + assert(cms1.heavyHitters == Set(1, 2)) - val cms2 = cms1 ++ monoid.create(2L) - assert(cms2.heavyHitters == Set(2L)) + val cms2 = cms1 ++ monoid.create(2) + assert(cms2.heavyHitters == Set(2)) - val cms3 = cms2 ++ monoid.create(1L) - assert(cms3.heavyHitters == Set(1L, 2L)) + val cms3 = cms2 ++ monoid.create(1) + assert(cms3.heavyHitters == Set(1, 2)) - val cms4 = cms3 ++ monoid.create(Seq(0L, 0L, 0L, 0L, 0L, 0L)) - assert(cms4.heavyHitters == Set(0L)) + val cms4 = cms3 ++ monoid.create(Seq(0, 0, 0, 0, 0, 0).toK[KEY]) + assert(cms4.heavyHitters == Set(0)) } "exactly compute heavy hitters in a small stream" in { - val data1 = Seq(1L, 2L, 2L, 3L, 3L, 3L, 4L, 4L, 4L, 4L, 5L, 5L, 5L, 5L, 5L) + val data1 = Seq(1, 2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5, 5, 5, 5).toK[KEY] val cms1 = new CountMinSketchMonoid[KEY](EPS, DELTA, SEED, 0.01).create(data1) val cms2 = new CountMinSketchMonoid[KEY](EPS, DELTA, SEED, 0.1).create(data1) val cms3 = new CountMinSketchMonoid[KEY](EPS, DELTA, SEED, 0.3).create(data1) val cms4 = new CountMinSketchMonoid[KEY](EPS, DELTA, SEED, 0.9).create(data1) - assert(cms1.heavyHitters == Set(1L, 2L, 3L, 4L, 5L)) - assert(cms2.heavyHitters == Set(2L, 3L, 4L, 5L)) - assert(cms3.heavyHitters == Set(5L)) - assert(cms4.heavyHitters == Set[Long]()) + assert(cms1.heavyHitters == Set(1, 2, 3, 4, 5)) + assert(cms2.heavyHitters == Set(2, 3, 4, 5)) + assert(cms3.heavyHitters == Set(5)) + assert(cms4.heavyHitters == Set[KEY]()) } "work as an Aggregator" in { - val data1 = Seq(1L, 2L, 2L, 3L, 3L, 3L, 4L, 4L, 4L, 4L, 5L, 5L, 5L, 5L, 5L) + val data1 = Seq(1, 2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5, 5, 5, 5).toK[KEY] val cms1 = CMS.aggregator[KEY](EPS, DELTA, SEED, 0.01).apply(data1) val cms2 = CMS.aggregator[KEY](EPS, DELTA, SEED, 0.1).apply(data1) val cms3 = CMS.aggregator[KEY](EPS, DELTA, SEED, 0.3).apply(data1) val cms4 = CMS.aggregator[KEY](EPS, DELTA, SEED, 0.9).apply(data1) - assert(cms1.heavyHitters == Set(1L, 2L, 3L, 4L, 5L)) - assert(cms2.heavyHitters == Set(2L, 3L, 4L, 5L)) - assert(cms3.heavyHitters == Set(5L)) - assert(cms4.heavyHitters == Set[Long]()) + assert(cms1.heavyHitters == Set(1, 2, 3, 4, 5)) + assert(cms2.heavyHitters == Set(2, 3, 4, 5)) + assert(cms3.heavyHitters == Set(5)) + assert(cms4.heavyHitters == Set[KEY]()) } } + + implicit class IntCast(x: Int) { + def toK[K : Numeric]: K = implicitly[Numeric[K]].fromInt(x) + } + + implicit class SeqCast(xs: Seq[Int]) { + def toK[K : Numeric]: Seq[K] = xs map { _.toK[K] } + } + + implicit class SetCast(xs: Set[Int]) { + def toK[K : Numeric]: Set[K] = xs map { _.toK[K] } + } + } From 59329188e23c14f69f831209a5f1bf4e7d25324f Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Thu, 9 Oct 2014 14:24:17 +0200 Subject: [PATCH 05/66] Use idiomatic ScalaTest matching style instead of asserts --- .../twitter/algebird/CountMinSketchTest.scala | 64 ++++++++++--------- 1 file changed, 34 insertions(+), 30 deletions(-) diff --git a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala index d6a59775d..5281b91e4 100644 --- a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala +++ b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala @@ -97,7 +97,7 @@ class CountMinSketchTest extends WordSpec with Matchers { val data = (0 to (totalCount - 1)).map { _ => RAND.nextInt(range) }.toK[KEY] val cms = CMS_MONOID.create(data) - assert(cms.totalCount == totalCount) + cms.totalCount should be (totalCount) } "estimate frequencies" in { @@ -110,10 +110,12 @@ class CountMinSketchTest extends WordSpec with Matchers { val x = RAND.nextInt(range).toK[KEY] val exact = exactFrequency(data, x) val approx = cms.frequency(x).estimate + val estimationError = approx - exact val maxError = approx - cms.frequency(x).min + val beWithinTolerance = be >= 0L and be <= maxError - assert(approx >= exact) - assert((approx - exact) <= maxError) + approx should be >= exact + estimationError should beWithinTolerance } } @@ -122,16 +124,16 @@ class CountMinSketchTest extends WordSpec with Matchers { val two = CMS_MONOID.create(2) val cms = CMS_MONOID.plus(CMS_MONOID.plus(one, two), two) - assert(cms.frequency(0).estimate == 0) - assert(cms.frequency(1).estimate == 1) - assert(cms.frequency(2).estimate == 2) + cms.frequency(0).estimate should be (0) + cms.frequency(1).estimate should be (1) + cms.frequency(2).estimate should be (2) val three = CMS_MONOID.create(Seq(1, 1, 1).toK[KEY]) - assert(three.frequency(1).estimate == 3) + three.frequency(1).estimate should be (3) val four = CMS_MONOID.create(Seq(1, 1, 1, 1).toK[KEY]) - assert(four.frequency(1).estimate == 4) + four.frequency(1).estimate should be (4) val cms2 = CMS_MONOID.plus(four, three) - assert(cms2.frequency(1).estimate == 7) + cms2.frequency(1).estimate should be (7) } "estimate inner products" in { @@ -145,33 +147,35 @@ class CountMinSketchTest extends WordSpec with Matchers { val approxA = cms1.innerProduct(cms2) val approx = approxA.estimate val exact = exactInnerProduct(data1, data2) + val estimationError = approx - exact val maxError = approx - approxA.min + val beWithinTolerance = be >= 0L and be <= maxError - assert(approx == cms2.innerProduct(cms1).estimate) - assert(approx >= exact) - assert((approx - exact) <= maxError) + approx should be(cms2.innerProduct(cms1).estimate) + approx should be >= exact + estimationError should beWithinTolerance } "exactly compute inner product of small streams" in { // Nothing in common. val a1 = List(1, 2, 3).toK[KEY] val a2 = List(4, 5, 6).toK[KEY] - assert(CMS_MONOID.create(a1).innerProduct(CMS_MONOID.create(a2)).estimate == 0) + CMS_MONOID.create(a1).innerProduct(CMS_MONOID.create(a2)).estimate should be (0) // One element in common. val b1 = List(1, 2, 3).toK[KEY] val b2 = List(3, 5, 6).toK[KEY] - assert(CMS_MONOID.create(b1).innerProduct(CMS_MONOID.create(b2)).estimate == 1) + CMS_MONOID.create(b1).innerProduct(CMS_MONOID.create(b2)).estimate should be (1) // Multiple, non-repeating elements in common. val c1 = List(1, 2, 3).toK[KEY] val c2 = List(3, 2, 6).toK[KEY] - assert(CMS_MONOID.create(c1).innerProduct(CMS_MONOID.create(c2)).estimate == 2) + CMS_MONOID.create(c1).innerProduct(CMS_MONOID.create(c2)).estimate should be (2) // Multiple, repeating elements in common. val d1 = List(1, 2, 2, 3, 3).toK[KEY] val d2 = List(2, 3, 3, 6).toK[KEY] - assert(CMS_MONOID.create(d1).innerProduct(CMS_MONOID.create(d2)).estimate == 6) + CMS_MONOID.create(d1).innerProduct(CMS_MONOID.create(d2)).estimate should be (6) } "estimate heavy hitters" in { @@ -188,28 +192,28 @@ class CountMinSketchTest extends WordSpec with Matchers { val estimatedHhs = cms.heavyHitters // All true heavy hitters must be claimed as heavy hitters. - assert(trueHhs.intersect(estimatedHhs) == trueHhs) + trueHhs.intersect(estimatedHhs) should be (trueHhs) // It should be very unlikely that any element with count less than // (heavyHittersPct - eps) * totalCount is claimed as a heavy hitter. val minHhCount = (cms.heavyHittersPct - cms.eps) * cms.totalCount val infrequent = data.groupBy{ x => x }.mapValues{ _.size }.filter{ _._2 < minHhCount }.keys.toSet - assert(infrequent.intersect(estimatedHhs).size == 0) + infrequent.intersect(estimatedHhs) should be ('empty) } "drop old heavy hitters when new heavy hitters replace them" in { val monoid = new CountMinSketchMonoid[KEY](EPS, DELTA, SEED, 0.3) val cms1 = monoid.create(Seq(1, 2, 2).toK[KEY]) - assert(cms1.heavyHitters == Set(1, 2)) + cms1.heavyHitters should be (Set(1,2)) val cms2 = cms1 ++ monoid.create(2) - assert(cms2.heavyHitters == Set(2)) + cms2.heavyHitters should be (Set(2)) val cms3 = cms2 ++ monoid.create(1) - assert(cms3.heavyHitters == Set(1, 2)) + cms3.heavyHitters should be (Set(1, 2)) val cms4 = cms3 ++ monoid.create(Seq(0, 0, 0, 0, 0, 0).toK[KEY]) - assert(cms4.heavyHitters == Set(0)) + cms4.heavyHitters should be (Set(0)) } "exactly compute heavy hitters in a small stream" in { @@ -218,10 +222,10 @@ class CountMinSketchTest extends WordSpec with Matchers { val cms2 = new CountMinSketchMonoid[KEY](EPS, DELTA, SEED, 0.1).create(data1) val cms3 = new CountMinSketchMonoid[KEY](EPS, DELTA, SEED, 0.3).create(data1) val cms4 = new CountMinSketchMonoid[KEY](EPS, DELTA, SEED, 0.9).create(data1) - assert(cms1.heavyHitters == Set(1, 2, 3, 4, 5)) - assert(cms2.heavyHitters == Set(2, 3, 4, 5)) - assert(cms3.heavyHitters == Set(5)) - assert(cms4.heavyHitters == Set[KEY]()) + cms1.heavyHitters should be (Set(1, 2, 3, 4, 5)) + cms2.heavyHitters should be (Set(2, 3, 4, 5)) + cms3.heavyHitters should be (Set(5)) + cms4.heavyHitters should be (Set[KEY]()) } "work as an Aggregator" in { @@ -230,10 +234,10 @@ class CountMinSketchTest extends WordSpec with Matchers { val cms2 = CMS.aggregator[KEY](EPS, DELTA, SEED, 0.1).apply(data1) val cms3 = CMS.aggregator[KEY](EPS, DELTA, SEED, 0.3).apply(data1) val cms4 = CMS.aggregator[KEY](EPS, DELTA, SEED, 0.9).apply(data1) - assert(cms1.heavyHitters == Set(1, 2, 3, 4, 5)) - assert(cms2.heavyHitters == Set(2, 3, 4, 5)) - assert(cms3.heavyHitters == Set(5)) - assert(cms4.heavyHitters == Set[KEY]()) + cms1.heavyHitters should be (Set(1, 2, 3, 4, 5)) + cms2.heavyHitters should be (Set(2, 3, 4, 5)) + cms3.heavyHitters should be (Set(5)) + cms4.heavyHitters should be (Set[KEY]()) } } From bfd5aba4c484e677f088157d912cd414784ccc17 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Thu, 9 Oct 2014 17:44:08 +0200 Subject: [PATCH 06/66] CMS: test K in {Short, Int, Long, BigInt} --- .../twitter/algebird/CountMinSketchTest.scala | 118 +++++++++--------- 1 file changed, 60 insertions(+), 58 deletions(-) diff --git a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala index 5281b91e4..085041191 100644 --- a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala +++ b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala @@ -4,15 +4,16 @@ import org.scalatest.{ PropSpec, Matchers, WordSpec } import org.scalatest.prop.PropertyChecks import org.scalacheck.{ Gen, Arbitrary } +import CMSHasherImplicits._ // required, although e.g. IntelliJ IDEA may flag it as unused import + class CountMinSketchLaws extends PropSpec with PropertyChecks with Matchers { import BaseProperties._ - import CMSHasherImplicits._ // required, although e.g. IntelliJ IDEA may flag it as unused import val DELTA = 1E-8 val EPS = 0.005 val SEED = 1 - private def createArbitrary[K : Numeric](cmsMonoid: CountMinSketchMonoid[K]): Arbitrary[CMS[K]] = { + private def createArbitrary[K: Numeric](cmsMonoid: CountMinSketchMonoid[K]): Arbitrary[CMS[K]] = { val k = implicitly[Numeric[K]] Arbitrary { for (v <- Gen.choose(0, 10000)) yield cmsMonoid.create(k.fromInt(v)) @@ -51,29 +52,30 @@ class CountMinSketchLaws extends PropSpec with PropertyChecks with Matchers { } -class CountMinSketchTest extends WordSpec with Matchers { - - import CMSHasherImplicits._ +class CountMinSketchShortTest extends CountMinSketchTest[Short] +class CountMinSketchIntTest extends CountMinSketchTest[Int] +class CountMinSketchLongTest extends CountMinSketchTest[Long] +class CountMinSketchBigIntTest extends CountMinSketchTest[BigInt] - type KEY = Long +abstract class CountMinSketchTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Matchers { val DELTA = 1E-10 val EPS = 0.001 val SEED = 1 - val CMS_MONOID = new CountMinSketchMonoid[KEY](EPS, DELTA, SEED) + val CMS_MONOID = new CountMinSketchMonoid[K](EPS, DELTA, SEED) val RAND = new scala.util.Random /** * Returns the exact frequency of {x} in {data}. */ - def exactFrequency(data: Seq[KEY], x: KEY): Long = data.count(_ == x) + def exactFrequency(data: Seq[K], x: K): Long = data.count(_ == x) /** * Returns the exact inner product between two data streams, when the streams * are viewed as count vectors. */ - def exactInnerProduct(data1: Seq[KEY], data2: Seq[KEY]): Long = { + def exactInnerProduct(data1: Seq[K], data2: Seq[K]): Long = { val counts1 = data1.groupBy(x => x).mapValues(_.size) val counts2 = data2.groupBy(x => x).mapValues(_.size) @@ -83,7 +85,7 @@ class CountMinSketchTest extends WordSpec with Matchers { /** * Returns the elements in {data} that appear at least heavyHittersPct * data.size times. */ - def exactHeavyHitters(data: Seq[KEY], heavyHittersPct: Double): Set[KEY] = { + def exactHeavyHitters(data: Seq[K], heavyHittersPct: Double): Set[K] = { val counts = data.groupBy(x => x).mapValues(_.size) val totalCount = counts.values.sum counts.filter { _._2 >= heavyHittersPct * totalCount }.keys.toSet @@ -94,7 +96,7 @@ class CountMinSketchTest extends WordSpec with Matchers { "count total number of elements in a stream" in { val totalCount = 1243 val range = 234 - val data = (0 to (totalCount - 1)).map { _ => RAND.nextInt(range) }.toK[KEY] + val data = (0 to (totalCount - 1)).map { _ => RAND.nextInt(range) }.toK[K] val cms = CMS_MONOID.create(data) cms.totalCount should be (totalCount) @@ -103,11 +105,11 @@ class CountMinSketchTest extends WordSpec with Matchers { "estimate frequencies" in { val totalCount = 5678 val range = 897 - val data = (0 to (totalCount - 1)).map { _ => RAND.nextInt(range) }.toK[KEY] + val data = (0 to (totalCount - 1)).map { _ => RAND.nextInt(range) }.toK[K] val cms = CMS_MONOID.create(data) (0 to 100).foreach { _ => - val x = RAND.nextInt(range).toK[KEY] + val x = RAND.nextInt(range).toK[K] val exact = exactFrequency(data, x) val approx = cms.frequency(x).estimate val estimationError = approx - exact @@ -120,27 +122,27 @@ class CountMinSketchTest extends WordSpec with Matchers { } "exactly compute frequencies in a small stream" in { - val one = CMS_MONOID.create(1) - val two = CMS_MONOID.create(2) + val one = CMS_MONOID.create(1.toK[K]) + val two = CMS_MONOID.create(2.toK[K]) val cms = CMS_MONOID.plus(CMS_MONOID.plus(one, two), two) - cms.frequency(0).estimate should be (0) - cms.frequency(1).estimate should be (1) - cms.frequency(2).estimate should be (2) + cms.frequency(0.toK[K]).estimate should be (0) + cms.frequency(1.toK[K]).estimate should be (1) + cms.frequency(2.toK[K]).estimate should be (2) - val three = CMS_MONOID.create(Seq(1, 1, 1).toK[KEY]) - three.frequency(1).estimate should be (3) - val four = CMS_MONOID.create(Seq(1, 1, 1, 1).toK[KEY]) - four.frequency(1).estimate should be (4) + val three = CMS_MONOID.create(Seq(1, 1, 1).toK[K]) + three.frequency(1.toK[K]).estimate should be (3) + val four = CMS_MONOID.create(Seq(1, 1, 1, 1).toK[K]) + four.frequency(1.toK[K]).estimate should be (4) val cms2 = CMS_MONOID.plus(four, three) - cms2.frequency(1).estimate should be (7) + cms2.frequency(1.toK[K]).estimate should be (7) } "estimate inner products" in { val totalCount = 5234 val range = 1390 - val data1 = (0 to (totalCount - 1)).map { _ => RAND.nextInt(range) }.toK[KEY] - val data2 = (0 to (totalCount - 1)).map { _ => RAND.nextInt(range) }.toK[KEY] + val data1 = (0 to (totalCount - 1)).map { _ => RAND.nextInt(range) }.toK[K] + val data2 = (0 to (totalCount - 1)).map { _ => RAND.nextInt(range) }.toK[K] val cms1 = CMS_MONOID.create(data1) val cms2 = CMS_MONOID.create(data1) @@ -158,35 +160,35 @@ class CountMinSketchTest extends WordSpec with Matchers { "exactly compute inner product of small streams" in { // Nothing in common. - val a1 = List(1, 2, 3).toK[KEY] - val a2 = List(4, 5, 6).toK[KEY] + val a1 = List(1, 2, 3).toK[K] + val a2 = List(4, 5, 6).toK[K] CMS_MONOID.create(a1).innerProduct(CMS_MONOID.create(a2)).estimate should be (0) // One element in common. - val b1 = List(1, 2, 3).toK[KEY] - val b2 = List(3, 5, 6).toK[KEY] + val b1 = List(1, 2, 3).toK[K] + val b2 = List(3, 5, 6).toK[K] CMS_MONOID.create(b1).innerProduct(CMS_MONOID.create(b2)).estimate should be (1) // Multiple, non-repeating elements in common. - val c1 = List(1, 2, 3).toK[KEY] - val c2 = List(3, 2, 6).toK[KEY] + val c1 = List(1, 2, 3).toK[K] + val c2 = List(3, 2, 6).toK[K] CMS_MONOID.create(c1).innerProduct(CMS_MONOID.create(c2)).estimate should be (2) // Multiple, repeating elements in common. - val d1 = List(1, 2, 2, 3, 3).toK[KEY] - val d2 = List(2, 3, 3, 6).toK[KEY] + val d1 = List(1, 2, 2, 3, 3).toK[K] + val d2 = List(2, 3, 3, 6).toK[K] CMS_MONOID.create(d1).innerProduct(CMS_MONOID.create(d2)).estimate should be (6) } "estimate heavy hitters" in { // Simple way of making some elements appear much more often than others. - val data1 = (1 to 3000).map { _ => RAND.nextInt(3) }.toK[KEY] - val data2 = (1 to 3000).map { _ => RAND.nextInt(10) }.toK[KEY] - val data3 = (1 to 1450).map { _ => -1 }.toK[KEY] // element close to being a 20% heavy hitter + val data1 = (1 to 3000).map { _ => RAND.nextInt(3) }.toK[K] + val data2 = (1 to 3000).map { _ => RAND.nextInt(10) }.toK[K] + val data3 = (1 to 1450).map { _ => -1 }.toK[K] // element close to being a 20% heavy hitter val data = data1 ++ data2 ++ data3 // Find elements that appear at least 20% of the time. - val cms = new CountMinSketchMonoid[KEY](EPS, DELTA, SEED, 0.2).create(data) + val cms = new CountMinSketchMonoid[K](EPS, DELTA, SEED, 0.2).create(data) val trueHhs = exactHeavyHitters(data, cms.heavyHittersPct) val estimatedHhs = cms.heavyHitters @@ -202,55 +204,55 @@ class CountMinSketchTest extends WordSpec with Matchers { } "drop old heavy hitters when new heavy hitters replace them" in { - val monoid = new CountMinSketchMonoid[KEY](EPS, DELTA, SEED, 0.3) - val cms1 = monoid.create(Seq(1, 2, 2).toK[KEY]) - cms1.heavyHitters should be (Set(1,2)) + val monoid = new CountMinSketchMonoid[K](EPS, DELTA, SEED, 0.3) + val cms1 = monoid.create(Seq(1, 2, 2).toK[K]) + cms1.heavyHitters should be (Set(1, 2)) - val cms2 = cms1 ++ monoid.create(2) + val cms2 = cms1 ++ monoid.create(2.toK[K]) cms2.heavyHitters should be (Set(2)) - val cms3 = cms2 ++ monoid.create(1) + val cms3 = cms2 ++ monoid.create(1.toK[K]) cms3.heavyHitters should be (Set(1, 2)) - val cms4 = cms3 ++ monoid.create(Seq(0, 0, 0, 0, 0, 0).toK[KEY]) + val cms4 = cms3 ++ monoid.create(Seq(0, 0, 0, 0, 0, 0).toK[K]) cms4.heavyHitters should be (Set(0)) } "exactly compute heavy hitters in a small stream" in { - val data1 = Seq(1, 2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5, 5, 5, 5).toK[KEY] - val cms1 = new CountMinSketchMonoid[KEY](EPS, DELTA, SEED, 0.01).create(data1) - val cms2 = new CountMinSketchMonoid[KEY](EPS, DELTA, SEED, 0.1).create(data1) - val cms3 = new CountMinSketchMonoid[KEY](EPS, DELTA, SEED, 0.3).create(data1) - val cms4 = new CountMinSketchMonoid[KEY](EPS, DELTA, SEED, 0.9).create(data1) + val data1 = Seq(1, 2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5, 5, 5, 5).toK[K] + val cms1 = new CountMinSketchMonoid[K](EPS, DELTA, SEED, 0.01).create(data1) + val cms2 = new CountMinSketchMonoid[K](EPS, DELTA, SEED, 0.1).create(data1) + val cms3 = new CountMinSketchMonoid[K](EPS, DELTA, SEED, 0.3).create(data1) + val cms4 = new CountMinSketchMonoid[K](EPS, DELTA, SEED, 0.9).create(data1) cms1.heavyHitters should be (Set(1, 2, 3, 4, 5)) cms2.heavyHitters should be (Set(2, 3, 4, 5)) cms3.heavyHitters should be (Set(5)) - cms4.heavyHitters should be (Set[KEY]()) + cms4.heavyHitters should be (Set[K]()) } "work as an Aggregator" in { - val data1 = Seq(1, 2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5, 5, 5, 5).toK[KEY] - val cms1 = CMS.aggregator[KEY](EPS, DELTA, SEED, 0.01).apply(data1) - val cms2 = CMS.aggregator[KEY](EPS, DELTA, SEED, 0.1).apply(data1) - val cms3 = CMS.aggregator[KEY](EPS, DELTA, SEED, 0.3).apply(data1) - val cms4 = CMS.aggregator[KEY](EPS, DELTA, SEED, 0.9).apply(data1) + val data1 = Seq(1, 2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5, 5, 5, 5).toK[K] + val cms1 = CMS.aggregator[K](EPS, DELTA, SEED, 0.01).apply(data1) + val cms2 = CMS.aggregator[K](EPS, DELTA, SEED, 0.1).apply(data1) + val cms3 = CMS.aggregator[K](EPS, DELTA, SEED, 0.3).apply(data1) + val cms4 = CMS.aggregator[K](EPS, DELTA, SEED, 0.9).apply(data1) cms1.heavyHitters should be (Set(1, 2, 3, 4, 5)) cms2.heavyHitters should be (Set(2, 3, 4, 5)) cms3.heavyHitters should be (Set(5)) - cms4.heavyHitters should be (Set[KEY]()) + cms4.heavyHitters should be (Set[K]()) } } implicit class IntCast(x: Int) { - def toK[K : Numeric]: K = implicitly[Numeric[K]].fromInt(x) + def toK[K: Numeric]: K = implicitly[Numeric[K]].fromInt(x) } implicit class SeqCast(xs: Seq[Int]) { - def toK[K : Numeric]: Seq[K] = xs map { _.toK[K] } + def toK[K: Numeric]: Seq[K] = xs map { _.toK[K] } } implicit class SetCast(xs: Set[Int]) { - def toK[K : Numeric]: Set[K] = xs map { _.toK[K] } + def toK[K: Numeric]: Set[K] = xs map { _.toK[K] } } } From f3865930ff0044a76377bd4f2aa680e6df98cc7e Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Thu, 9 Oct 2014 17:56:56 +0200 Subject: [PATCH 07/66] CMS: remove implicit for Byte --- .../main/scala/com/twitter/algebird/CountMinSketch.scala | 6 ------ .../scala/com/twitter/algebird/CountMinSketchTest.scala | 6 ------ 2 files changed, 12 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index 6738f7f7c..905027fe7 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -544,12 +544,6 @@ object CMSHasherImplicits { } - implicit object CMSHasherByte extends CMSHasher[Byte] { - - def hash(a: Int, b: Int, width: Int)(x: Byte) = CMSHasherInt.hash(a, b, width)(x) - - } - implicit object CMSHasherShort extends CMSHasher[Short] { def hash(a: Int, b: Int, width: Int)(x: Short) = CMSHasherInt.hash(a, b, width)(x) diff --git a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala index 085041191..465e11567 100644 --- a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala +++ b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala @@ -20,12 +20,6 @@ class CountMinSketchLaws extends PropSpec with PropertyChecks with Matchers { } } - property("CountMinSketch[Byte] is a Monoid") { - implicit val cmsMonoid = new CountMinSketchMonoid[Byte](EPS, DELTA, SEED) - implicit val cmsGen = createArbitrary[Byte](cmsMonoid) - monoidLaws[CMS[Byte]] - } - property("CountMinSketch[Short] is a Monoid") { implicit val cmsMonoid = new CountMinSketchMonoid[Short](EPS, DELTA, SEED) implicit val cmsGen = createArbitrary[Short](cmsMonoid) From 43f5101cae31b9240585dfbadb5d28f950091231 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Fri, 10 Oct 2014 09:03:10 +0200 Subject: [PATCH 08/66] Reformat class parameters of CMSInstance --- .../main/scala/com/twitter/algebird/CountMinSketch.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index 905027fe7..566950e96 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -302,9 +302,9 @@ case class CMSItem[K: Ordering](item: K, params: CMSParams[K]) extends CMS[K] { * The general Count-Min sketch structure, used for holding any number of elements. */ case class CMSInstance[K: Ordering](countsTable: CMSInstance.CMSCountsTable[K], - totalCount: Long, - hhs: CMSInstance.HeavyHitters[K], - params: CMSParams[K]) extends CMS[K] { + totalCount: Long, + hhs: CMSInstance.HeavyHitters[K], + params: CMSParams[K]) extends CMS[K] { def eps: Double = params.eps From d9118178edf4fe0221b32800f42daaceeddf16a5 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Fri, 10 Oct 2014 15:39:38 +0200 Subject: [PATCH 09/66] CMS: split responsibility of counting/freq estimation and heavy hitters tracking. --- .../com/twitter/algebird/CountMinSketch.scala | 596 ++++++++++++------ .../com/twitter/algebird/SketchMap.scala | 6 +- .../twitter/algebird/CountMinSketchTest.scala | 59 +- .../summer/HeavyHittersCachingSummer.scala | 6 +- 4 files changed, 425 insertions(+), 242 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index 566950e96..0d6e25431 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -53,6 +53,13 @@ import scala.collection.immutable.SortedSet * @author Edwin Chen */ +case class CmsParams[K](hashes: Seq[CmsHash[K]], eps: Double, delta: Double) { + + require(0 < eps && eps < 1, "eps must lie in (0, 1)") + require(0 < delta && delta < 1, "delta must lie in (0, 1)") + +} + /** * Monoid for adding Count-Min sketches. * @@ -63,7 +70,7 @@ import scala.collection.immutable.SortedSet * * Implicit conversions for commonly used types for `K` such as [[Long]] and [[BigInt]]: * {{{ - * import com.twitter.algebird.CMSHasherImplicits._ + * import com.twitter.algebird.CmsHasherImplicits._ * }}} * * @param eps A parameter that bounds the error of each query estimate. @@ -71,124 +78,65 @@ import scala.collection.immutable.SortedSet * (an interval that depends on eps) around the truth. * @param seed A seed to initialize the random number generator used to create the pairwise independent * hash functions. - * @param heavyHittersPct A threshold for finding heavy hitters, i.e., elements that appear at least - * (heavyHittersPct * totalCount) times in the stream. * @tparam K The type used to identify the elements to be counted. For example, if you want to count the occurrence of * user names, you could map each username to a unique numeric ID expressed as a `Long`, and then count the * occurrences of those `Long`s with a CMS of type `K=Long`. Note that this mapping between the elements of * your problem domain and their identifiers used for counting via CMS should be bijective. - * We require [[Ordering]] and [[CMSHasher]] context bounds for `K`, see [[CMSHasherImplicits]] for available + * We require [[Ordering]] and [[CmsHasher]] context bounds for `K`, see [[CmsHasherImplicits]] for available * implicits that can be imported. * Which type K should you pick in practice? For domains that have less than `2^64` unique elements, you'd * typically use [[Long]]. For larger domains you can try [[BigInt]], for example. */ -class CountMinSketchMonoid[K: Ordering: CMSHasher](eps: Double, - delta: Double, - seed: Int, - heavyHittersPct: Double = 0.01) extends Monoid[CMS[K]] { +class CmsMonoid[K: Ordering: CmsHasher](eps: Double, delta: Double, seed: Int) extends Monoid[Cms[K]] { - require(0 < eps && eps < 1, "eps must lie in (0, 1)") - require(0 < delta && delta < 1, "delta must lie in (0, 1)") - require(0 < heavyHittersPct && heavyHittersPct < 1, "heavyHittersPct must lie in (0, 1)") - - // Typically, we would use d pair-wise independent hash functions of the form - // - // h_i(x) = a_i * x + b_i (mod p) - // - // But for this particular application, setting b_i does not matter (since all it does is shift the results of a - // particular hash), so we omit it (by setting b_i to 0) and simply use hash functions of the form - // - // h_i(x) = a_i * x (mod p) - // - val hashes: Seq[CMSHash[K]] = { - val r = new scala.util.Random(seed) - val numHashes = CMS.depth(delta) - val numCounters = CMS.width(eps) - (0 to (numHashes - 1)).map { _ => CMSHash[K](r.nextInt(), 0, numCounters) } + val params = { + val hashes: Seq[CmsHash[K]] = CmsFunctions.generateHashes(eps, delta, seed) + CmsParams(hashes, eps, delta) } - val params = CMSParams(hashes, eps, delta, heavyHittersPct) - - val zero: CMS[K] = CMSZero[K](params) + val zero: Cms[K] = CmsZero[K](params) /** - * We assume the Count-Min sketches on the left and right use the same hash functions. + * We assume the sketches on the left and right use the same hash functions. */ - def plus(left: CMS[K], right: CMS[K]): CMS[K] = left ++ right + def plus(left: Cms[K], right: Cms[K]): Cms[K] = left ++ right /** - * Create a Count-Min sketch out of a single item or data stream. + * Create a sketch out of a single item or data stream. */ - def create(item: K): CMS[K] = CMSItem[K](item, params) + def create(item: K): Cms[K] = CmsItem[K](item, params) - def create(data: Seq[K]): CMS[K] = { + def create(data: Seq[K]): Cms[K] = { data.foldLeft(zero) { case (acc, x) => plus(acc, create(x)) } } } -object CMS { - - def monoid[K: Ordering: CMSHasher](eps: Double, - delta: Double, - seed: Int, - heavyHittersPct: Double = 0.01): CountMinSketchMonoid[K] = - new CountMinSketchMonoid[K](eps, delta, seed, heavyHittersPct) +trait CmsCounting[K, C[_]] { - def monoid[K: Ordering: CMSHasher](depth: Int, - width: Int, - seed: Int, - heavyHittersPct: Double): CountMinSketchMonoid[K] = - new CountMinSketchMonoid[K](CMS.eps(width), CMS.delta(depth), seed, heavyHittersPct) - - def aggregator[K: Ordering: CMSHasher](eps: Double, - delta: Double, - seed: Int, - heavyHittersPct: Double = 0.01): CountMinSketchAggregator[K] = { - val monoid = new CountMinSketchMonoid[K](eps, delta, seed, heavyHittersPct) - new CountMinSketchAggregator[K](monoid) - } + // Parameters used to bound confidence in error estimates. + def eps: Double - def aggregator[K: Ordering: CMSHasher](depth: Int, - width: Int, - seed: Int, - heavyHittersPct: Double): CountMinSketchAggregator[K] = { - val monoid = new CountMinSketchMonoid[K](CMS.eps(width), CMS.delta(depth), seed, heavyHittersPct) - new CountMinSketchAggregator[K](monoid) - } + def delta: Double /** - * Functions to translate between (eps, delta) and (depth, width). The translation is: - * depth = ceil(ln 1/delta) - * width = ceil(e / eps) + * Number of hash functions. */ - def eps(width: Int) = scala.math.exp(1.0) / width - - def delta(depth: Int) = 1.0 / scala.math.exp(depth) - - def depth(delta: Double) = scala.math.ceil(scala.math.log(1.0 / delta)).toInt - - def width(eps: Double) = scala.math.ceil(scala.math.exp(1) / eps).toInt - -} + def depth: Int = CmsFunctions.depth(delta) -/** - * The actual Count-Min sketch data structure. - */ -sealed abstract class CMS[K] extends java.io.Serializable { - - // Parameters used to bound confidence in error estimates. - def eps: Double - - def delta: Double + /** + * Number of counters per hash function. + */ + def width: Int = CmsFunctions.width(eps) - // Number of hash functions. - def depth: Int = CMS.depth(delta) + def ++(other: C[K]): C[K] - // Number of counters per hash function. - def width: Int = CMS.width(eps) + /** + * Updates the sketch with a new element from the data stream. + */ + def +(item: K): C[K] = this + (item, 1L) - def ++(other: CMS[K]): CMS[K] + def +(item: K, count: Long): C[K] /** * Returns an estimate of the total number of times this item has been seen @@ -213,7 +161,26 @@ sealed abstract class CMS[K] extends java.io.Serializable { * With probability p >= 1 - delta, it also holds that * estimatedInnerProduct <= actualInnerProduct + eps * thisTotalCount * otherTotalCount */ - def innerProduct(other: CMS[K]): Approximate[Long] + def innerProduct(other: C[K]): Approximate[Long] + + /** + * Total number of elements seen in the data stream so far. + */ + def totalCount: Long + + /** + * The first frequency moment is the total number of elements in the stream. + */ + def f1: Long = totalCount + + /** + * The second frequency moment is `\sum a_i^2`, where a_i is the count of the ith element. + */ + def f2: Approximate[Long] + +} + +trait CmsHeavyHitters[K] { /** * Finds all heavy hitters, i.e., elements in the stream that appear at least @@ -231,95 +198,136 @@ sealed abstract class CMS[K] extends java.io.Serializable { def heavyHitters: Set[K] - /** - * Total number of elements seen in the data stream so far. - */ - def totalCount: Long +} - /** - * The first frequency moment is the total number of elements in the stream. - */ - def f1: Long = totalCount +/** + * Functions to translate between (eps, delta) and (depth, width). The translation is: + * depth = ceil(ln 1/delta) + * width = ceil(e / eps) + */ +object CmsFunctions { + + def eps(width: Int): Double = scala.math.exp(1.0) / width + + def delta(depth: Int): Double = 1.0 / scala.math.exp(depth) + + def depth(delta: Double): Int = scala.math.ceil(scala.math.log(1.0 / delta)).toInt + + def width(eps: Double): Int = scala.math.ceil(scala.math.exp(1) / eps).toInt + + // Typically, we would use d pair-wise independent hash functions of the form + // + // h_i(x) = a_i * x + b_i (mod p) + // + // But for this particular application, setting b_i does not matter (since all it does is shift the results of a + // particular hash), so we omit it (by setting b_i to 0) and simply use hash functions of the form + // + // h_i(x) = a_i * x (mod p) + // + def generateHashes[K: CmsHasher](eps: Double, delta: Double, seed: Int): Seq[CmsHash[K]] = { + val r = new scala.util.Random(seed) + val numHashes = depth(delta) + val numCounters = width(eps) + (0 to (numHashes - 1)).map { _ => CmsHash[K](r.nextInt(), 0, numCounters) } + } + +} + +object Cms { + + def monoid[K: Ordering: CmsHasher](eps: Double, delta: Double, seed: Int): CmsMonoid[K] = + new CmsMonoid[K](eps, delta, seed) + + def monoid[K: Ordering: CmsHasher](depth: Int, width: Int, seed: Int): CmsMonoid[K] = + monoid(CmsFunctions.eps(width), CmsFunctions.delta(depth), seed) + + def aggregator[K: Ordering: CmsHasher](eps: Double, delta: Double, seed: Int): CmsAggregator[K] = + new CmsAggregator[K](monoid(eps, delta, seed)) + + def aggregator[K: Ordering: CmsHasher](depth: Int, width: Int, seed: Int): CmsAggregator[K] = + aggregator(CmsFunctions.eps(width), CmsFunctions.delta(depth), seed) /** - * The second frequency moment is `\sum a_i^2`, where a_i is the count of the ith element. + * Returns a fresh, zeroed CMS instance. */ - def f2: Approximate[Long] = innerProduct(this) + def apply[K: Ordering: CmsHasher](eps: Double, delta: Double, seed: Int): Cms[K] = { + val params = { + val hashes: Seq[CmsHash[K]] = CmsFunctions.generateHashes(eps, delta, seed) + CmsParams(hashes, eps, delta) + } + CmsInstance[K](params) + } } /** - * Zero element. Used for initialization. + * The actual Count-Min sketch data structure. This data structure only allows for counting and frequency estimation. */ -case class CMSZero[K: Ordering](params: CMSParams[K]) extends CMS[K] { +// TODO: Maybe we need to require Ordering[K] only here in the base class? +sealed abstract class Cms[K: Ordering](params: CmsParams[K]) extends java.io.Serializable with CmsCounting[K, Cms] { - def eps: Double = params.eps + override val eps: Double = params.eps - def delta: Double = params.delta + override val delta: Double = params.delta - def heavyHittersPct: Double = params.heavyHittersPct + override def f2: Approximate[Long] = innerProduct(this) - def totalCount: Long = 0L +} + +/** + * Zero element. Used for initialization. + */ +case class CmsZero[K: Ordering](params: CmsParams[K]) extends Cms[K](params) { + + override val totalCount: Long = 0L - def ++(other: CMS[K]): CMS[K] = other + override def +(item: K, count: Long): Cms[K] = CmsInstance[K](params) + (item, count) - def frequency(item: K): Approximate[Long] = Approximate.exact(0L) + override def ++(other: Cms[K]): Cms[K] = other - def innerProduct(other: CMS[K]): Approximate[Long] = Approximate.exact(0L) + override def frequency(item: K): Approximate[Long] = Approximate.exact(0L) - def heavyHitters: Set[K] = Set[K]() + override def innerProduct(other: Cms[K]): Approximate[Long] = Approximate.exact(0L) } /** * Used for holding a single element, to avoid repeatedly adding elements from sparse counts tables. */ -case class CMSItem[K: Ordering](item: K, params: CMSParams[K]) extends CMS[K] { - def eps: Double = params.eps +case class CmsItem[K: Ordering](item: K, params: CmsParams[K]) extends Cms[K](params) { - def delta: Double = params.delta + override val totalCount: Long = 1L - def heavyHittersPct: Double = params.heavyHittersPct + override def +(x: K, count: Long): Cms[K] = CmsInstance[K](params) + item + (x, count) - def totalCount: Long = 1L - - def ++(other: CMS[K]): CMS[K] = { + override def ++(other: Cms[K]): Cms[K] = { other match { - case other: CMSZero[_] => this - case other: CMSItem[K] => CMSInstance[K](params) + item + other.item - case other: CMSInstance[K] => other + item + case other: CmsZero[_] => this + case other: CmsItem[K] => CmsInstance[K](params) + item + other.item + case other: CmsInstance[K] => other + item } } - def frequency(x: K): Approximate[Long] = if (item == x) Approximate.exact(1L) else Approximate.exact(0L) + override def frequency(x: K): Approximate[Long] = if (item == x) Approximate.exact(1L) else Approximate.exact(0L) - def innerProduct(other: CMS[K]): Approximate[Long] = other.frequency(item) + override def innerProduct(other: Cms[K]): Approximate[Long] = other.frequency(item) - def heavyHitters: Set[K] = Set(item) } /** * The general Count-Min sketch structure, used for holding any number of elements. */ -case class CMSInstance[K: Ordering](countsTable: CMSInstance.CMSCountsTable[K], - totalCount: Long, - hhs: CMSInstance.HeavyHitters[K], - params: CMSParams[K]) extends CMS[K] { - - def eps: Double = params.eps +case class CmsInstance[K: Ordering](countsTable: CmsInstance.CmsCountsTable[K], + override val totalCount: Long, + params: CmsParams[K]) extends Cms[K](params) { - def delta: Double = params.delta - - def heavyHittersPct: Double = params.heavyHittersPct - - def ++(other: CMS[K]): CMS[K] = { + def ++(other: Cms[K]): Cms[K] = { other match { - case other: CMSZero[_] => this - case other: CMSItem[K] => this + other.item - case other: CMSInstance[K] => + case other: CmsZero[_] => this + case other: CmsItem[K] => this + other.item + case other: CmsInstance[K] => val newTotalCount = totalCount + other.totalCount - val newHhs = (hhs ++ other.hhs).dropCountsBelow(params.heavyHittersPct * newTotalCount) - CMSInstance[K](countsTable ++ other.countsTable, newTotalCount, newHhs, params) + CmsInstance[K](countsTable ++ other.countsTable, newTotalCount, params) } } @@ -342,9 +350,9 @@ case class CMSInstance[K: Ordering](countsTable: CMSInstance.CMSCountsTable[K], * rows: * estimatedInnerProduct = min_j (\sum_k count_A[j, k] * count_B[j, k]) */ - def innerProduct(other: CMS[K]): Approximate[Long] = { + def innerProduct(other: Cms[K]): Approximate[Long] = { other match { - case other: CMSInstance[_] => + case other: CmsInstance[_] => require((other.depth, other.width) == (depth, width), "Tables must have the same dimensions.") def innerProductAtDepth(d: Int) = (0 to (width - 1)).map { w => @@ -357,56 +365,29 @@ case class CMSInstance[K: Ordering](countsTable: CMSInstance.CMSCountsTable[K], } } - def heavyHitters: Set[K] = hhs.items - - /** - * Updates the sketch with a new element from the data stream. - */ - def +(item: K): CMSInstance[K] = this + (item, 1L) - - def +(item: K, count: Long): CMSInstance[K] = { + override def +(item: K, count: Long): CmsInstance[K] = { require(count >= 0, "count must be >= 0 (negative counts not implemented") if (count != 0L) { - val newHhs = updateHeavyHitters(item, count) val newCountsTable = (0 to (depth - 1)).foldLeft(countsTable) { case (table, row) => val pos = (row, params.hashes(row)(item)) table + (pos, count) } - CMSInstance[K](newCountsTable, totalCount + count, newHhs, params) + CmsInstance[K](newCountsTable, totalCount + count, params) } else this } - /** - * Updates the data structure of heavy hitters when a new item (with associated count) enters the stream. - */ - private def updateHeavyHitters(item: K, count: Long): CMSInstance.HeavyHitters[K] = { - val oldItemCount = frequency(item).estimate - val newItemCount = oldItemCount + count - val newTotalCount = totalCount + count - - // If the new item is a heavy hitter, add it, and remove any previous instances. - val newHhs = - if (newItemCount >= heavyHittersPct * newTotalCount) { - hhs - CMSInstance.HeavyHitter[K](item, oldItemCount) + CMSInstance.HeavyHitter[K](item, newItemCount) - } else hhs - - // Remove any items below the new heavy hitter threshold. - newHhs.dropCountsBelow(heavyHittersPct * newTotalCount) - } - } -object CMSInstance { +object CmsInstance { /** - * Initializes a CMSInstance with all zeroes. + * Initializes a CmsInstance with all zeroes, i.e. nothing has been counted yet. */ - def apply[K: Ordering](params: CMSParams[K]): CMSInstance[K] = { - val countsTable = CMSCountsTable[K](CMS.depth(params.delta), CMS.width(params.eps)) - implicit val heavyHitterOrdering = HeavyHitter.ordering[K] - CMSInstance[K](countsTable, 0, HeavyHitters[K](SortedSet[HeavyHitter[K]]()), params) + def apply[K: Ordering](params: CmsParams[K]): CmsInstance[K] = { + val countsTable = CmsCountsTable[K](CmsFunctions.depth(params.delta), CmsFunctions.width(params.eps)) + CmsInstance[K](countsTable, 0, params) } /** @@ -414,7 +395,7 @@ object CMSInstance { * Each row corresponds to a particular hash function. * TODO: implement a dense matrix type, and use it here */ - case class CMSCountsTable[K](counts: Vector[Vector[Long]]) { + case class CmsCountsTable[K](counts: Vector[Vector[Long]]) { require(depth > 0, "Table must have at least 1 row.") require(width > 0, "Table must have at least 1 column.") @@ -431,79 +412,278 @@ object CMSInstance { /** * Updates the count of a single cell in the table. */ - def +(pos: (Int, Int), count: Long): CMSCountsTable[K] = { + def +(pos: (Int, Int), count: Long): CmsCountsTable[K] = { val (row, col) = pos val currCount = getCount(pos) val newCounts = counts.updated(row, counts(row).updated(col, currCount + count)) - CMSCountsTable[K](newCounts) + CmsCountsTable[K](newCounts) } /** * Adds another counts table to this one, through element-wise addition. */ - def ++(other: CMSCountsTable[K]): CMSCountsTable[K] = { + def ++(other: CmsCountsTable[K]): CmsCountsTable[K] = { require((depth, width) == (other.depth, other.width), "Tables must have the same dimensions.") val iil: IndexedSeq[IndexedSeq[Long]] = Monoid.plus[IndexedSeq[IndexedSeq[Long]]](counts, other.counts) def toVector[V](is: IndexedSeq[V]): Vector[V] = is match { case v: Vector[_] => v case _ => Vector(is: _*) } - CMSCountsTable[K](toVector(iil.map { toVector })) + CmsCountsTable[K](toVector(iil.map { toVector })) } } - object CMSCountsTable { - // Creates a new CMSCountsTable with counts initialized to all zeroes. - def apply[K: Ordering](depth: Int, width: Int): CMSCountsTable[K] = - CMSCountsTable[K](Vector.fill[Long](depth, width)(0L)) + object CmsCountsTable { + /** + * Creates a new CmsCountsTable with counts initialized to all zeroes. + */ + def apply[K: Ordering](depth: Int, width: Int): CmsCountsTable[K] = + CmsCountsTable[K](Vector.fill[Long](depth, width)(0L)) } +} + +case class TopPctCmsParams(heavyHittersPct: Double) { + + require(0 < heavyHittersPct && heavyHittersPct < 1, "heavyHittersPct must lie in (0, 1)") + +} + +/** + * Monoid for adding Count-Min sketches. + * + * eps and delta are parameters that bound the error of each query estimate. For example, errors in + * answering queries (e.g., how often has element x appeared in the stream described by the sketch?) + * are often of the form: "with probability p >= 1 - delta, the estimate is close to the truth by + * some factor depending on eps." + * + * Implicit conversions for commonly used types for `K` such as [[Long]] and [[BigInt]]: + * {{{ + * import com.twitter.algebird.CMSHasherImplicits._ + * }}} + * + * @param cms A counting-only CMS instance, which is used for the counting performed by this class. + * @param heavyHittersPct A threshold for finding heavy hitters, i.e., elements that appear at least + * (heavyHittersPct * totalCount) times in the stream. + * @tparam K The type used to identify the elements to be counted. For example, if you want to count the occurrence of + * user names, you could map each username to a unique numeric ID expressed as a `Long`, and then count the + * occurrences of those `Long`s with a CMS of type `K=Long`. Note that this mapping between the elements of + * your problem domain and their identifiers used for counting via CMS should be bijective. + * We require [[Ordering]] and [[CmsHasher]] context bounds for `K`, see [[CmsHasherImplicits]] for available + * implicits that can be imported. + * Which type K should you pick in practice? For domains that have less than `2^64` unique elements, you'd + * typically use [[Long]]. For larger domains you can try [[BigInt]], for example. + */ +class TopPctCmsMonoid[K: Ordering](cms: Cms[K], heavyHittersPct: Double = 0.01) extends Monoid[TopPctCms[K]] { + + val conf: TopPctCmsParams = TopPctCmsParams(heavyHittersPct) + + val zero: TopPctCms[K] = TopPctCmsZero[K](cms, conf) + + /** + * We assume the sketches on the left and right use the same hash functions. + */ + def plus(left: TopPctCms[K], right: TopPctCms[K]): TopPctCms[K] = left ++ right + /** - * Containers for holding heavy hitter items and their associated counts. + * Create a sketch out of a single item or data stream. */ - case class HeavyHitters[K: Ordering](hhs: SortedSet[HeavyHitter[K]]) { + def create(item: K): TopPctCms[K] = TopPctCmsItem[K](item, cms, conf) + + def create(data: Seq[K]): TopPctCms[K] = { + data.foldLeft(zero) { case (acc, x) => plus(acc, create(x)) } + } + +} + +object TopPctCms { + + def monoid[K: Ordering: CmsHasher](eps: Double, + delta: Double, + seed: Int, + heavyHittersPct: Double): TopPctCmsMonoid[K] = + new TopPctCmsMonoid[K](Cms(eps, delta, seed), heavyHittersPct) - def -(hh: HeavyHitter[K]): HeavyHitters[K] = HeavyHitters[K](hhs - hh) + def monoid[K: Ordering: CmsHasher](depth: Int, + width: Int, + seed: Int, + heavyHittersPct: Double): TopPctCmsMonoid[K] = + monoid(CmsFunctions.eps(width), CmsFunctions.delta(depth), seed, heavyHittersPct) - def +(hh: HeavyHitter[K]): HeavyHitters[K] = HeavyHitters[K](hhs + hh) + def aggregator[K: Ordering: CmsHasher](eps: Double, + delta: Double, + seed: Int, + heavyHittersPct: Double): TopPctCmsAggregator[K] = + new TopPctCmsAggregator[K](monoid(eps, delta, seed, heavyHittersPct)) - def ++(other: HeavyHitters[K]): HeavyHitters[K] = HeavyHitters[K](hhs ++ other.hhs) + def aggregator[K: Ordering: CmsHasher](depth: Int, + width: Int, + seed: Int, + heavyHittersPct: Double): TopPctCmsAggregator[K] = + aggregator(CmsFunctions.eps(width), CmsFunctions.delta(depth), seed, heavyHittersPct) + +} - def items: Set[K] = hhs.map { _.item } +sealed abstract class TopPctCms[K: Ordering](val cms: Cms[K], conf: TopPctCmsParams) + extends java.io.Serializable with CmsCounting[K, TopPctCms] with CmsHeavyHitters[K] { + + override val eps: Double = cms.eps + + override val delta: Double = cms.delta + + override val totalCount: Long = cms.totalCount + + override val heavyHittersPct = conf.heavyHittersPct + + override def frequency(item: K): Approximate[Long] = cms.frequency(item) + + override def innerProduct(other: TopPctCms[K]): Approximate[Long] = cms.innerProduct(other.cms) + + def f2: Approximate[Long] = innerProduct(this) - def dropCountsBelow(minCount: Double): HeavyHitters[K] = HeavyHitters[K](hhs.dropWhile { _.count < minCount }) +} + +/** + * Zero element. Used for initialization. + */ +case class TopPctCmsZero[K: Ordering](override val cms: Cms[K], conf: TopPctCmsParams) extends TopPctCms[K](cms, conf) { + + override val heavyHitters: Set[K] = Set.empty[K] + + override def +(item: K, count: Long): TopPctCms[K] = TopPctCmsInstance(cms, conf) + (item, count) + + override def ++(other: TopPctCms[K]): TopPctCms[K] = other + +} + +/** + * Used for holding a single element, to avoid repeatedly adding elements from sparse counts tables. + */ +// TODO: Do we still need this, given that TopPctCmsItem does not count iself? +case class TopPctCmsItem[K: Ordering](item: K, + override val cms: Cms[K], + conf: TopPctCmsParams) extends TopPctCms[K](cms, conf) { + + override val heavyHitters: Set[K] = Set(item) + + override def +(x: K, count: Long): TopPctCms[K] = TopPctCmsInstance(cms, conf) + item + (x, count) + + override def ++(other: TopPctCms[K]): TopPctCms[K] = { + other match { + case other: TopPctCmsZero[_] => this + case other: TopPctCmsItem[K] => TopPctCmsInstance[K](cms, conf) + item + other.item + case other: TopPctCmsInstance[K] => other + item + } + } + +} + +object TopPctCmsInstance { + + def apply[K: Ordering](cms: Cms[K], conf: TopPctCmsParams): TopPctCmsInstance[K] = { + implicit val heavyHitterOrdering = HeavyHitter.ordering[K] + TopPctCmsInstance[K](cms, HeavyHitters[K](SortedSet[HeavyHitter[K]]()), conf) + } + +} + +case class TopPctCmsInstance[K: Ordering](override val cms: Cms[K], hhs: HeavyHitters[K], conf: TopPctCmsParams) + extends TopPctCms[K](cms, conf) { + + override def heavyHitters: Set[K] = hhs.items + + override def +(item: K, count: Long): TopPctCmsInstance[K] = { + require(count >= 0, "count must be >= 0 (negative counts not implemented") + if (count != 0L) { + val newHhs = updateHeavyHitters(item, count) + val newCms = cms + (item, count) + TopPctCmsInstance[K](newCms, newHhs, conf) + } else this } - case class HeavyHitter[K: Ordering](item: K, count: Long) + /** + * Updates the data structure of heavy hitters when a new item (with associated count) enters the stream. + */ + private def updateHeavyHitters(item: K, count: Long): HeavyHitters[K] = { + val oldItemCount = frequency(item).estimate + val newItemCount = oldItemCount + count + val newTotalCount = totalCount + count - object HeavyHitter { + // If the new item is a heavy hitter, add it, and remove any previous instances. + val newHhs = + if (newItemCount >= conf.heavyHittersPct * newTotalCount) { + hhs - HeavyHitter[K](item, oldItemCount) + HeavyHitter[K](item, newItemCount) + } else hhs - def ordering[K: Ordering]: Ordering[HeavyHitter[K]] = Ordering.by { hh: HeavyHitter[K] => (hh.count, hh.item) } + // Remove any items below the new heavy hitter threshold. + newHhs.dropCountsBelow(conf.heavyHittersPct * newTotalCount) + } + override def ++(other: TopPctCms[K]): TopPctCms[K] = { + other match { + case other: TopPctCmsZero[_] => this + case other: TopPctCmsItem[K] => this + other.item + case other: TopPctCmsInstance[K] => + val newTotalCount = totalCount + other.totalCount + val newHhs = (hhs ++ other.hhs).dropCountsBelow(conf.heavyHittersPct * newTotalCount) + TopPctCmsInstance(cms ++ other.cms, newHhs, conf) + } } } /** - * Convenience class for holding constant parameters of a Count-Min sketch. + * Containers for holding heavy hitter items and their associated counts. */ -case class CMSParams[K](hashes: Seq[CMSHash[K]], eps: Double, delta: Double, heavyHittersPct: Double) +case class HeavyHitters[K: Ordering](hhs: SortedSet[HeavyHitter[K]]) { + + def -(hh: HeavyHitter[K]): HeavyHitters[K] = HeavyHitters[K](hhs - hh) + + def +(hh: HeavyHitter[K]): HeavyHitters[K] = HeavyHitters[K](hhs + hh) + + def ++(other: HeavyHitters[K]): HeavyHitters[K] = HeavyHitters[K](hhs ++ other.hhs) + + def items: Set[K] = hhs.map { _.item } + + def dropCountsBelow(minCount: Double): HeavyHitters[K] = HeavyHitters[K](hhs.dropWhile { _.count < minCount }) +} + +case class HeavyHitter[K: Ordering](item: K, count: Long) + +object HeavyHitter { + + def ordering[K: Ordering]: Ordering[HeavyHitter[K]] = Ordering.by { hh: HeavyHitter[K] => (hh.count, hh.item) } + +} /** - * An Aggregator for the CountMinSketch. Can be created using `CMS.aggregator`. + * An Aggregator for the Count-Min Sketch. Can be created using [[Cms.aggregator]]. */ -case class CountMinSketchAggregator[K](cmsMonoid: CountMinSketchMonoid[K]) - extends MonoidAggregator[K, CMS[K], CMS[K]] { +case class CmsAggregator[K](cmsMonoid: CmsMonoid[K]) extends MonoidAggregator[K, Cms[K], Cms[K]] { val monoid = cmsMonoid - def prepare(value: K): CMS[K] = monoid.create(value) + def prepare(value: K): Cms[K] = monoid.create(value) + + def present(cms: Cms[K]): Cms[K] = cms + +} + +/** + * An Aggregator for the Top Percentage Count-Min Sketch. Can be created using [[TopPctCms.aggregator]]. + */ +case class TopPctCmsAggregator[K](topPctCmsMonoid: TopPctCmsMonoid[K]) + extends MonoidAggregator[K, TopPctCms[K], TopPctCms[K]] { + + val monoid = topPctCmsMonoid + + def prepare(value: K): TopPctCms[K] = monoid.create(value) - def present(cms: CMS[K]): CMS[K] = cms + def present(cms: TopPctCms[K]): TopPctCms[K] = cms } -trait CMSHasher[K] { +trait CmsHasher[K] { val PRIME_MODULUS = (1L << 31) - 1 @@ -519,18 +699,18 @@ trait CMSHasher[K] { * * `h(x) = [a * x + b (mod p)] (mod m)` */ -case class CMSHash[K: CMSHasher](a: Int, b: Int, width: Int) { +case class CmsHash[K: CmsHasher](a: Int, b: Int, width: Int) { /** * Returns `a * x + b (mod p) (mod width)`. */ - def apply(x: K): Int = implicitly[CMSHasher[K]].hash(a, b, width)(x) + def apply(x: K): Int = implicitly[CmsHasher[K]].hash(a, b, width)(x) } -object CMSHasherImplicits { +object CmsHasherImplicits { - implicit object CMSHasherLong extends CMSHasher[Long] { + implicit object CmsHasherLong extends CmsHasher[Long] { def hash(a: Int, b: Int, width: Int)(x: Long) = { val unModded: Long = (x * a) + b @@ -544,13 +724,13 @@ object CMSHasherImplicits { } - implicit object CMSHasherShort extends CMSHasher[Short] { + implicit object CmsHasherShort extends CmsHasher[Short] { - def hash(a: Int, b: Int, width: Int)(x: Short) = CMSHasherInt.hash(a, b, width)(x) + def hash(a: Int, b: Int, width: Int)(x: Short) = CmsHasherInt.hash(a, b, width)(x) } - implicit object CMSHasherInt extends CMSHasher[Int] { + implicit object CmsHasherInt extends CmsHasher[Int] { def hash(a: Int, b: Int, width: Int)(x: Int) = { val unModded: Int = (x * a) + b @@ -560,7 +740,7 @@ object CMSHasherImplicits { } - implicit object CMSHasherBigInt extends CMSHasher[BigInt] { + implicit object CmsHasherBigInt extends CmsHasher[BigInt] { def hash(a: Int, b: Int, width: Int)(x: BigInt) = { val unModded: BigInt = (x * a) + b diff --git a/algebird-core/src/main/scala/com/twitter/algebird/SketchMap.scala b/algebird-core/src/main/scala/com/twitter/algebird/SketchMap.scala index 5fb1aad3a..9912e7fc6 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/SketchMap.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/SketchMap.scala @@ -17,7 +17,7 @@ limitations under the License. package com.twitter.algebird import scala.collection.breakOut -import com.twitter.algebird.CMSHasherImplicits._ +import com.twitter.algebird.CmsHasherImplicits._ import com.twitter.algebird.matrix.AdaptiveMatrix /** @@ -29,7 +29,7 @@ import com.twitter.algebird.matrix.AdaptiveMatrix /** * Hashes an arbitrary key type to one that the Sketch Map can use. */ -case class SketchMapHash[K](hasher: CMSHash[Long], seed: Int)(implicit serialization: K => Array[Byte]) { +case class SketchMapHash[K](hasher: CmsHash[Long], seed: Int)(implicit serialization: K => Array[Byte]) { def apply(obj: K): Int = { val (first, second) = MurmurHash128(seed)(serialization(obj)) hasher(first ^ second) @@ -138,7 +138,7 @@ case class SketchMapParams[K](seed: Int, width: Int, depth: Int, heavyHittersCou val numHashes = depth val numCounters = width (0 to (numHashes - 1)).map { _ => - val smhash: SketchMapHash[K] = SketchMapHash(CMSHash[Long](r.nextInt, 0, numCounters), seed)(serialization) + val smhash: SketchMapHash[K] = SketchMapHash(CmsHash[Long](r.nextInt, 0, numCounters), seed)(serialization) new (K => Int) { override def apply(k: K) = smhash(k) } } } diff --git a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala index 465e11567..540c3ba0a 100644 --- a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala +++ b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala @@ -4,16 +4,16 @@ import org.scalatest.{ PropSpec, Matchers, WordSpec } import org.scalatest.prop.PropertyChecks import org.scalacheck.{ Gen, Arbitrary } -import CMSHasherImplicits._ // required, although e.g. IntelliJ IDEA may flag it as unused import +import CmsHasherImplicits._ // required, although e.g. IntelliJ IDEA may flag it as unused import -class CountMinSketchLaws extends PropSpec with PropertyChecks with Matchers { +class CmsLaws extends PropSpec with PropertyChecks with Matchers { import BaseProperties._ val DELTA = 1E-8 val EPS = 0.005 val SEED = 1 - private def createArbitrary[K: Numeric](cmsMonoid: CountMinSketchMonoid[K]): Arbitrary[CMS[K]] = { + private def createArbitrary[K: Numeric](cmsMonoid: CmsMonoid[K]): Arbitrary[Cms[K]] = { val k = implicitly[Numeric[K]] Arbitrary { for (v <- Gen.choose(0, 10000)) yield cmsMonoid.create(k.fromInt(v)) @@ -21,43 +21,43 @@ class CountMinSketchLaws extends PropSpec with PropertyChecks with Matchers { } property("CountMinSketch[Short] is a Monoid") { - implicit val cmsMonoid = new CountMinSketchMonoid[Short](EPS, DELTA, SEED) + implicit val cmsMonoid = Cms.monoid[Short](EPS, DELTA, SEED) implicit val cmsGen = createArbitrary[Short](cmsMonoid) - monoidLaws[CMS[Short]] + monoidLaws[Cms[Short]] } property("CountMinSketch[Int] is a Monoid") { - implicit val cmsMonoid = new CountMinSketchMonoid[Int](EPS, DELTA, SEED) + implicit val cmsMonoid = Cms.monoid[Int](EPS, DELTA, SEED) implicit val cmsGen = createArbitrary[Int](cmsMonoid) - monoidLaws[CMS[Int]] + monoidLaws[Cms[Int]] } property("CountMinSketch[Long] is a Monoid") { - implicit val cmsMonoid = new CountMinSketchMonoid[Long](EPS, DELTA, SEED) + implicit val cmsMonoid = Cms.monoid[Long](EPS, DELTA, SEED) implicit val cmsGen = createArbitrary[Long](cmsMonoid) - monoidLaws[CMS[Long]] + monoidLaws[Cms[Long]] } property("CountMinSketch[BigInt] is a Monoid") { - implicit val cmsMonoid = new CountMinSketchMonoid[BigInt](EPS, DELTA, SEED) + implicit val cmsMonoid = Cms.monoid[BigInt](EPS, DELTA, SEED) implicit val cmsGen = createArbitrary[BigInt](cmsMonoid) - monoidLaws[CMS[BigInt]] + monoidLaws[Cms[BigInt]] } } -class CountMinSketchShortTest extends CountMinSketchTest[Short] -class CountMinSketchIntTest extends CountMinSketchTest[Int] -class CountMinSketchLongTest extends CountMinSketchTest[Long] -class CountMinSketchBigIntTest extends CountMinSketchTest[BigInt] +class CmsShortTest extends CmsTest[Short] +class CmsIntTest extends CmsTest[Int] +class CmsLongTest extends CmsTest[Long] +class CmsBigIntTest extends CmsTest[BigInt] -abstract class CountMinSketchTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Matchers { +abstract class CmsTest[K: Ordering: CmsHasher: Numeric] extends WordSpec with Matchers { val DELTA = 1E-10 val EPS = 0.001 val SEED = 1 - val CMS_MONOID = new CountMinSketchMonoid[K](EPS, DELTA, SEED) + val CMS_MONOID = Cms.monoid[K](EPS, DELTA, SEED) val RAND = new scala.util.Random /** @@ -182,7 +182,7 @@ abstract class CountMinSketchTest[K: Ordering: CMSHasher: Numeric] extends WordS val data = data1 ++ data2 ++ data3 // Find elements that appear at least 20% of the time. - val cms = new CountMinSketchMonoid[K](EPS, DELTA, SEED, 0.2).create(data) + val cms = TopPctCms.monoid[K](EPS, DELTA, SEED, 0.2).create(data) val trueHhs = exactHeavyHitters(data, cms.heavyHittersPct) val estimatedHhs = cms.heavyHitters @@ -198,7 +198,7 @@ abstract class CountMinSketchTest[K: Ordering: CMSHasher: Numeric] extends WordS } "drop old heavy hitters when new heavy hitters replace them" in { - val monoid = new CountMinSketchMonoid[K](EPS, DELTA, SEED, 0.3) + val monoid = TopPctCms.monoid[K](EPS, DELTA, SEED, 0.3) val cms1 = monoid.create(Seq(1, 2, 2).toK[K]) cms1.heavyHitters should be (Set(1, 2)) @@ -214,27 +214,30 @@ abstract class CountMinSketchTest[K: Ordering: CMSHasher: Numeric] extends WordS "exactly compute heavy hitters in a small stream" in { val data1 = Seq(1, 2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5, 5, 5, 5).toK[K] - val cms1 = new CountMinSketchMonoid[K](EPS, DELTA, SEED, 0.01).create(data1) - val cms2 = new CountMinSketchMonoid[K](EPS, DELTA, SEED, 0.1).create(data1) - val cms3 = new CountMinSketchMonoid[K](EPS, DELTA, SEED, 0.3).create(data1) - val cms4 = new CountMinSketchMonoid[K](EPS, DELTA, SEED, 0.9).create(data1) + val cms1 = TopPctCms.monoid[K](EPS, DELTA, SEED, 0.01).create(data1) + val cms2 = TopPctCms.monoid[K](EPS, DELTA, SEED, 0.1).create(data1) + val cms3 = TopPctCms.monoid[K](EPS, DELTA, SEED, 0.3).create(data1) + val cms4 = TopPctCms.monoid[K](EPS, DELTA, SEED, 0.9).create(data1) cms1.heavyHitters should be (Set(1, 2, 3, 4, 5)) cms2.heavyHitters should be (Set(2, 3, 4, 5)) cms3.heavyHitters should be (Set(5)) cms4.heavyHitters should be (Set[K]()) } + // TODO: test aggregator for Cms, not only for TopPctCms + "work as an Aggregator" in { val data1 = Seq(1, 2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5, 5, 5, 5).toK[K] - val cms1 = CMS.aggregator[K](EPS, DELTA, SEED, 0.01).apply(data1) - val cms2 = CMS.aggregator[K](EPS, DELTA, SEED, 0.1).apply(data1) - val cms3 = CMS.aggregator[K](EPS, DELTA, SEED, 0.3).apply(data1) - val cms4 = CMS.aggregator[K](EPS, DELTA, SEED, 0.9).apply(data1) + val cms1 = TopPctCms.aggregator[K](EPS, DELTA, SEED, 0.01).apply(data1) + val cms2 = TopPctCms.aggregator[K](EPS, DELTA, SEED, 0.1).apply(data1) + val cms3 = TopPctCms.aggregator[K](EPS, DELTA, SEED, 0.3).apply(data1) + val cms4 = TopPctCms.aggregator[K](EPS, DELTA, SEED, 0.9).apply(data1) cms1.heavyHitters should be (Set(1, 2, 3, 4, 5)) cms2.heavyHitters should be (Set(2, 3, 4, 5)) cms3.heavyHitters should be (Set(5)) cms4.heavyHitters should be (Set[K]()) } + } implicit class IntCast(x: Int) { @@ -249,4 +252,4 @@ abstract class CountMinSketchTest[K: Ordering: CMSHasher: Numeric] extends WordS def toK[K: Numeric]: Set[K] = xs map { _.toK[K] } } -} +} \ No newline at end of file diff --git a/algebird-util/src/main/scala/com/twitter/algebird/util/summer/HeavyHittersCachingSummer.scala b/algebird-util/src/main/scala/com/twitter/algebird/util/summer/HeavyHittersCachingSummer.scala index 90d28941c..640e56d96 100644 --- a/algebird-util/src/main/scala/com/twitter/algebird/util/summer/HeavyHittersCachingSummer.scala +++ b/algebird-util/src/main/scala/com/twitter/algebird/util/summer/HeavyHittersCachingSummer.scala @@ -43,7 +43,7 @@ case class HeavyHittersPercent(toFloat: Float) class ApproxHHTracker(hhPct: HeavyHittersPercent, updateFreq: UpdateFrequency, roFreq: RollOverFrequency) { - import CMSHasherImplicits._ + import CmsHasherImplicits._ private[this] final val WIDTH = 1000 private[this] final val DEPTH = 4 @@ -55,9 +55,9 @@ class ApproxHHTracker(hhPct: HeavyHittersPercent, updateFreq: UpdateFrequency, r private[this] final val rollOverFrequency = roFreq.toLong private[this] final var countsTable = Array.fill(WIDTH * DEPTH)(0L) - private[this] final val hashes: IndexedSeq[CMSHash[Long]] = { + private[this] final val hashes: IndexedSeq[CmsHash[Long]] = { val r = new scala.util.Random(5) - (0 until DEPTH).map { _ => CMSHash[Long](r.nextInt, 0, WIDTH) } + (0 until DEPTH).map { _ => CmsHash[Long](r.nextInt, 0, WIDTH) } }.toIndexedSeq @inline From 7288f8ea65dd928fe447c7ae02ed0ad37a4ecf8a Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Sat, 11 Oct 2014 13:11:24 +0200 Subject: [PATCH 10/66] Improve documentation --- .../com/twitter/algebird/CountMinSketch.scala | 119 ++++++++++++------ 1 file changed, 82 insertions(+), 37 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index 0d6e25431..78dce3785 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -53,6 +53,16 @@ import scala.collection.immutable.SortedSet * @author Edwin Chen */ +/** + * Configuration paramaters for [[Cms]]. + * + * @param hashes Pair-wise independent hashes functions. We need `N=depth` such functions (`depth` can be derived from + * `delta`). + * @param eps One-sided error bound on the error of each point query, i.e. frequency estimate. + * @param delta A bound on the probability that a query estimate does not lie within some small interval + * (an interval that depends on `eps`) around the truth. + * @tparam K + */ case class CmsParams[K](hashes: Seq[CmsHash[K]], eps: Double, delta: Double) { require(0 < eps && eps < 1, "eps must lie in (0, 1)") @@ -73,9 +83,9 @@ case class CmsParams[K](hashes: Seq[CmsHash[K]], eps: Double, delta: Double) { * import com.twitter.algebird.CmsHasherImplicits._ * }}} * - * @param eps A parameter that bounds the error of each query estimate. + * @param eps One-sided error bound on the error of each point query, i.e. frequency estimate. * @param delta A bound on the probability that a query estimate does not lie within some small interval - * (an interval that depends on eps) around the truth. + * (an interval that depends on `eps`) around the truth. * @param seed A seed to initialize the random number generator used to create the pairwise independent * hash functions. * @tparam K The type used to identify the elements to be counted. For example, if you want to count the occurrence of @@ -102,49 +112,60 @@ class CmsMonoid[K: Ordering: CmsHasher](eps: Double, delta: Double, seed: Int) e def plus(left: Cms[K], right: Cms[K]): Cms[K] = left ++ right /** - * Create a sketch out of a single item or data stream. + * Create a sketch out of a single item. */ def create(item: K): Cms[K] = CmsItem[K](item, params) - def create(data: Seq[K]): Cms[K] = { - data.foldLeft(zero) { case (acc, x) => plus(acc, create(x)) } - } + /** + * Create a sketch out of multiple items. + */ + def create(data: Seq[K]): Cms[K] = data.foldLeft(zero) { case (acc, x) => plus(acc, create(x)) } } trait CmsCounting[K, C[_]] { - // Parameters used to bound confidence in error estimates. + /** + * Returns the one-sided error bound on the error of each point query, i.e. frequency estimate. + */ def eps: Double + /** + * Returns the bound on the probability that a query estimate does NOT lie within some small interval (an interval + * that depends on `eps`) around the truth. + */ def delta: Double /** - * Number of hash functions. + * Number of hash functions (also: number of rows in the counting table). This number is derived from `delta`. */ def depth: Int = CmsFunctions.depth(delta) /** - * Number of counters per hash function. + * Number of counters per hash function (also: number of columns in the counting table). This number is derived from + * `eps`. */ def width: Int = CmsFunctions.width(eps) def ++(other: C[K]): C[K] /** - * Updates the sketch with a new element from the data stream. + * Counts the item and returns the result as a new sketch. */ def +(item: K): C[K] = this + (item, 1L) + /** + * Counts the item `count` times and returns the result as a new sketch. + */ def +(item: K, count: Long): C[K] /** * Returns an estimate of the total number of times this item has been seen * in the stream so far. This estimate is an upper bound. * - * It is always true that trueFrequency <= estimatedFrequency. - * With probability p >= 1 - delta, it also holds that - * estimatedFrequency <= trueFrequency + eps * totalCount. + * It is always true that `estimatedFrequency >= trueFrequency`. + * With probability `p >= 1 - delta`, it also holds that + * `estimatedFrequency <= trueFrequency + eps * totalCount`. */ def frequency(item: K): Approximate[Long] @@ -153,18 +174,18 @@ trait CmsCounting[K, C[_]] { * * In other words, let a_i denote the number of times element i has been seen in * the data stream summarized by this CMS, and let b_i denote the same for the other CMS. - * Then this returns an estimate of = \sum a_i b_i + * Then this returns an estimate of ` = \sum a_i b_i`. * - * Note: this can also be viewed as the join size between two relations. + * Note: This can also be viewed as the join size between two relations. * * It is always true that actualInnerProduct <= estimatedInnerProduct. - * With probability p >= 1 - delta, it also holds that - * estimatedInnerProduct <= actualInnerProduct + eps * thisTotalCount * otherTotalCount + * With probability `p >= 1 - delta`, it also holds that + * `estimatedInnerProduct <= actualInnerProduct + eps * thisTotalCount * otherTotalCount`. */ def innerProduct(other: C[K]): Approximate[Long] /** - * Total number of elements seen in the data stream so far. + * Total number of elements counted (i.e. seen in the data stream) so far. */ def totalCount: Long @@ -174,7 +195,7 @@ trait CmsCounting[K, C[_]] { def f1: Long = totalCount /** - * The second frequency moment is `\sum a_i^2`, where a_i is the count of the ith element. + * The second frequency moment is `\sum a_i^2`, where `a_i` is the count of the i-th element. */ def f2: Approximate[Long] @@ -184,47 +205,70 @@ trait CmsHeavyHitters[K] { /** * Finds all heavy hitters, i.e., elements in the stream that appear at least - * (heavyHittersPct * totalCount) times. + * `(heavyHittersPct * totalCount)` times. * - * Every item that appears at least (heavyHittersPct * totalCount) times is output, - * and with probability p >= 1 - delta, no item whose count is less than - * (heavyHittersPct - eps) * totalCount is output. + * Every item that appears at least `(heavyHittersPct * totalCount)` times is output, + * and with probability `p >= 1 - delta`, no item whose count is less than + * `(heavyHittersPct - eps) * totalCount` is output. * - * Note that the set of heavy hitters contains at most 1 / heavyHittersPct + * Note that the set of heavy hitters contains at most `1 / heavyHittersPct` * elements, so keeping track of all elements that appear more than (say) 1% of the * time requires tracking at most 100 items. */ def heavyHittersPct: Double + /** + * Returns the descendingly sorted list of heavy hitters (e.g. the heaviest hitter is the first element). + */ def heavyHitters: Set[K] } /** - * Functions to translate between (eps, delta) and (depth, width). The translation is: - * depth = ceil(ln 1/delta) - * width = ceil(e / eps) + * Helper functions to generate or to translate between various `Cms` parameters. */ object CmsFunctions { + /** + * Translates from `width` to `eps`. + */ def eps(width: Int): Double = scala.math.exp(1.0) / width + /** + * Translates from `depth` to `delta`. + */ def delta(depth: Int): Double = 1.0 / scala.math.exp(depth) + /** + * Translates from `delta` to `depth`. + */ def depth(delta: Double): Int = scala.math.ceil(scala.math.log(1.0 / delta)).toInt + /** + * Translates from `eps` to `width`. + */ def width(eps: Double): Int = scala.math.ceil(scala.math.exp(1) / eps).toInt - // Typically, we would use d pair-wise independent hash functions of the form - // - // h_i(x) = a_i * x + b_i (mod p) - // - // But for this particular application, setting b_i does not matter (since all it does is shift the results of a - // particular hash), so we omit it (by setting b_i to 0) and simply use hash functions of the form - // - // h_i(x) = a_i * x (mod p) - // + /** + * Generates `N=depth` pair-wise independent hash functions. + * + * @param eps One-sided error bound on the error of each point query, i.e. frequency estimate. + * @param delta Error bound on the probability that a query estimate does NOT lie within some small interval around + * the truth. + * @param seed Seed for the random number generator. + * @tparam K The type used to identify the elements to be counted. + * @return The generated hash functions. + */ def generateHashes[K: CmsHasher](eps: Double, delta: Double, seed: Int): Seq[CmsHash[K]] = { + // Typically, we would use d -- aka depth -- pair-wise independent hash functions of the form + // + // h_i(x) = a_i * x + b_i (mod p) + // + // But for this particular application, setting b_i does not matter (since all it does is shift the results of a + // particular hash), so we omit it (by setting b_i to 0) and simply use hash functions of the form + // + // h_i(x) = a_i * x (mod p) + // val r = new scala.util.Random(seed) val numHashes = depth(delta) val numCounters = width(eps) @@ -375,7 +419,8 @@ case class CmsInstance[K: Ordering](countsTable: CmsInstance.CmsCountsTable[K], table + (pos, count) } CmsInstance[K](newCountsTable, totalCount + count, params) - } else this + } + else this } } From 49f275fdd264e92dba18b3a273737200cdb74b59 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Sat, 11 Oct 2014 13:17:31 +0200 Subject: [PATCH 11/66] Reorder classes and improve their docs --- .../com/twitter/algebird/CountMinSketch.scala | 145 +++++++++--------- 1 file changed, 75 insertions(+), 70 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index 78dce3785..3f4034f64 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -53,23 +53,6 @@ import scala.collection.immutable.SortedSet * @author Edwin Chen */ -/** - * Configuration paramaters for [[Cms]]. - * - * @param hashes Pair-wise independent hashes functions. We need `N=depth` such functions (`depth` can be derived from - * `delta`). - * @param eps One-sided error bound on the error of each point query, i.e. frequency estimate. - * @param delta A bound on the probability that a query estimate does not lie within some small interval - * (an interval that depends on `eps`) around the truth. - * @tparam K - */ -case class CmsParams[K](hashes: Seq[CmsHash[K]], eps: Double, delta: Double) { - - require(0 < eps && eps < 1, "eps must lie in (0, 1)") - require(0 < delta && delta < 1, "delta must lie in (0, 1)") - -} - /** * Monoid for adding Count-Min sketches. * @@ -123,6 +106,81 @@ class CmsMonoid[K: Ordering: CmsHasher](eps: Double, delta: Double, seed: Int) e } +/** + * Configuration paramaters for [[Cms]]. + * + * @param hashes Pair-wise independent hashes functions. We need `N=depth` such functions (`depth` can be derived from + * `delta`). + * @param eps One-sided error bound on the error of each point query, i.e. frequency estimate. + * @param delta A bound on the probability that a query estimate does not lie within some small interval + * (an interval that depends on `eps`) around the truth. + * @tparam K + */ +case class CmsParams[K](hashes: Seq[CmsHash[K]], eps: Double, delta: Double) { + + require(0 < eps && eps < 1, "eps must lie in (0, 1)") + require(0 < delta && delta < 1, "delta must lie in (0, 1)") + +} + +/** + * Helper functions to generate or to translate between various CMS parameters (cf. [[CmsParams]]). + */ +object CmsFunctions { + + /** + * Translates from `width` to `eps`. + */ + def eps(width: Int): Double = scala.math.exp(1.0) / width + + /** + * Translates from `depth` to `delta`. + */ + def delta(depth: Int): Double = 1.0 / scala.math.exp(depth) + + /** + * Translates from `delta` to `depth`. + */ + def depth(delta: Double): Int = scala.math.ceil(scala.math.log(1.0 / delta)).toInt + + /** + * Translates from `eps` to `width`. + */ + def width(eps: Double): Int = scala.math.ceil(scala.math.exp(1) / eps).toInt + + /** + * Generates `N=depth` pair-wise independent hash functions. + * + * @param eps One-sided error bound on the error of each point query, i.e. frequency estimate. + * @param delta Error bound on the probability that a query estimate does NOT lie within some small interval around + * the truth. + * @param seed Seed for the random number generator. + * @tparam K The type used to identify the elements to be counted. + * @return The generated hash functions. + */ + def generateHashes[K: CmsHasher](eps: Double, delta: Double, seed: Int): Seq[CmsHash[K]] = { + // Typically, we would use d -- aka depth -- pair-wise independent hash functions of the form + // + // h_i(x) = a_i * x + b_i (mod p) + // + // But for this particular application, setting b_i does not matter (since all it does is shift the results of a + // particular hash), so we omit it (by setting b_i to 0) and simply use hash functions of the form + // + // h_i(x) = a_i * x (mod p) + // + val r = new scala.util.Random(seed) + val numHashes = depth(delta) + val numCounters = width(eps) + (0 to (numHashes - 1)).map { _ => CmsHash[K](r.nextInt(), 0, numCounters) } + } + +} + +/** + * + * @tparam K The type used to identify the elements to be counted. + * @tparam C The type of the CMS variant. + */ trait CmsCounting[K, C[_]] { /** @@ -224,59 +282,6 @@ trait CmsHeavyHitters[K] { } -/** - * Helper functions to generate or to translate between various `Cms` parameters. - */ -object CmsFunctions { - - /** - * Translates from `width` to `eps`. - */ - def eps(width: Int): Double = scala.math.exp(1.0) / width - - /** - * Translates from `depth` to `delta`. - */ - def delta(depth: Int): Double = 1.0 / scala.math.exp(depth) - - /** - * Translates from `delta` to `depth`. - */ - def depth(delta: Double): Int = scala.math.ceil(scala.math.log(1.0 / delta)).toInt - - /** - * Translates from `eps` to `width`. - */ - def width(eps: Double): Int = scala.math.ceil(scala.math.exp(1) / eps).toInt - - /** - * Generates `N=depth` pair-wise independent hash functions. - * - * @param eps One-sided error bound on the error of each point query, i.e. frequency estimate. - * @param delta Error bound on the probability that a query estimate does NOT lie within some small interval around - * the truth. - * @param seed Seed for the random number generator. - * @tparam K The type used to identify the elements to be counted. - * @return The generated hash functions. - */ - def generateHashes[K: CmsHasher](eps: Double, delta: Double, seed: Int): Seq[CmsHash[K]] = { - // Typically, we would use d -- aka depth -- pair-wise independent hash functions of the form - // - // h_i(x) = a_i * x + b_i (mod p) - // - // But for this particular application, setting b_i does not matter (since all it does is shift the results of a - // particular hash), so we omit it (by setting b_i to 0) and simply use hash functions of the form - // - // h_i(x) = a_i * x (mod p) - // - val r = new scala.util.Random(seed) - val numHashes = depth(delta) - val numCounters = width(eps) - (0 to (numHashes - 1)).map { _ => CmsHash[K](r.nextInt(), 0, numCounters) } - } - -} - object Cms { def monoid[K: Ordering: CmsHasher](eps: Double, delta: Double, seed: Int): CmsMonoid[K] = From adb85b06db9f9c79b746c1fbed60f8db00e8ffc6 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Sat, 11 Oct 2014 13:31:47 +0200 Subject: [PATCH 12/66] Improve documentation --- .../com/twitter/algebird/CountMinSketch.scala | 43 +++++++++++++++---- 1 file changed, 34 insertions(+), 9 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index 3f4034f64..03dff33ae 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -107,7 +107,7 @@ class CmsMonoid[K: Ordering: CmsHasher](eps: Double, delta: Double, seed: Int) e } /** - * Configuration paramaters for [[Cms]]. + * Configuration parameters for [[Cms]]. * * @param hashes Pair-wise independent hashes functions. We need `N=depth` such functions (`depth` can be derived from * `delta`). @@ -177,9 +177,13 @@ object CmsFunctions { } /** + * A trait for CMS implementations that can count elements in a data stream and that can answer point queries (i.e. + * frequency estimates) for these elements. + * + * Known implementations: [[Cms]], [[TopPctCms]]. * * @tparam K The type used to identify the elements to be counted. - * @tparam C The type of the CMS variant. + * @tparam C The type of the actual CMS that implements this trait. */ trait CmsCounting[K, C[_]] { @@ -205,6 +209,9 @@ trait CmsCounting[K, C[_]] { */ def width: Int = CmsFunctions.width(eps) + /** + * Returns a new sketch that is the combination of this sketch and the other sketch. + */ def ++(other: C[K]): C[K] /** @@ -259,6 +266,13 @@ trait CmsCounting[K, C[_]] { } +/** + * A trait for CMS implementations that can track heavy hitters in a data stream. + * + * Known implementations: [[TopPctCms]]. + * + * @tparam K The type used to identify the elements to be counted. + */ trait CmsHeavyHitters[K] { /** @@ -310,9 +324,8 @@ object Cms { } /** - * The actual Count-Min sketch data structure. This data structure only allows for counting and frequency estimation. + * A Count-Min sketch data structure that allows for counting and frequency estimation of elements in a data stream. */ -// TODO: Maybe we need to require Ordering[K] only here in the base class? sealed abstract class Cms[K: Ordering](params: CmsParams[K]) extends java.io.Serializable with CmsCounting[K, Cms] { override val eps: Double = params.eps @@ -536,10 +549,13 @@ class TopPctCmsMonoid[K: Ordering](cms: Cms[K], heavyHittersPct: Double = 0.01) def plus(left: TopPctCms[K], right: TopPctCms[K]): TopPctCms[K] = left ++ right /** - * Create a sketch out of a single item or data stream. + * Create a sketch out of a single item. */ def create(item: K): TopPctCms[K] = TopPctCmsItem[K](item, cms, conf) + /** + * Create a sketch out of multiple items. + */ def create(data: Seq[K]): TopPctCms[K] = { data.foldLeft(zero) { case (acc, x) => plus(acc, create(x)) } } @@ -574,6 +590,12 @@ object TopPctCms { } +/** + * A Count-Min sketch data structure that allows for (a) counting and frequency estimation of elements in a data stream + * and (b) tracking the heavy hitters among these elements. + * + * @tparam K The type used to identify the elements to be counted. + */ sealed abstract class TopPctCms[K: Ordering](val cms: Cms[K], conf: TopPctCmsParams) extends java.io.Serializable with CmsCounting[K, TopPctCms] with CmsHeavyHitters[K] { @@ -609,7 +631,6 @@ case class TopPctCmsZero[K: Ordering](override val cms: Cms[K], conf: TopPctCmsP /** * Used for holding a single element, to avoid repeatedly adding elements from sparse counts tables. */ -// TODO: Do we still need this, given that TopPctCmsItem does not count iself? case class TopPctCmsItem[K: Ordering](item: K, override val cms: Cms[K], conf: TopPctCmsParams) extends TopPctCms[K](cms, conf) { @@ -707,7 +728,7 @@ object HeavyHitter { } /** - * An Aggregator for the Count-Min Sketch. Can be created using [[Cms.aggregator]]. + * An Aggregator for [[Cms]]. Can be created using [[Cms.aggregator]]. */ case class CmsAggregator[K](cmsMonoid: CmsMonoid[K]) extends MonoidAggregator[K, Cms[K], Cms[K]] { @@ -720,7 +741,7 @@ case class CmsAggregator[K](cmsMonoid: CmsMonoid[K]) extends MonoidAggregator[K, } /** - * An Aggregator for the Top Percentage Count-Min Sketch. Can be created using [[TopPctCms.aggregator]]. + * An Aggregator for [[TopPctCms]]. Can be created using [[TopPctCms.aggregator]]. */ case class TopPctCmsAggregator[K](topPctCmsMonoid: TopPctCmsMonoid[K]) extends MonoidAggregator[K, TopPctCms[K], TopPctCms[K]] { @@ -745,7 +766,8 @@ trait CmsHasher[K] { } /** - * The Count-Min sketch uses `d` pair-wise independent hash functions drawn from a universal hashing family of the form: + * The Count-Min sketch uses `d` (aka `depth`) pair-wise independent hash functions drawn from a universal hashing + * family of the form: * * `h(x) = [a * x + b (mod p)] (mod m)` */ @@ -758,6 +780,9 @@ case class CmsHash[K: CmsHasher](a: Int, b: Int, width: Int) { } +/** + * Implicits that enable CMS-hashing for common data types such as [[Long]] and [[BigInt]]. + */ object CmsHasherImplicits { implicit object CmsHasherLong extends CmsHasher[Long] { From 278213deec463a20f771ebc2a2d20f9cbafbcf6f Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Sat, 11 Oct 2014 13:36:56 +0200 Subject: [PATCH 13/66] Require at least d (aka depth) hash functions --- .../src/main/scala/com/twitter/algebird/CountMinSketch.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index 03dff33ae..f9610673f 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -120,6 +120,7 @@ case class CmsParams[K](hashes: Seq[CmsHash[K]], eps: Double, delta: Double) { require(0 < eps && eps < 1, "eps must lie in (0, 1)") require(0 < delta && delta < 1, "delta must lie in (0, 1)") + require(hashes.size >= CmsFunctions.depth(delta), s"we require at least ${CmsFunctions.depth(delta)} hash functions") } From 316c11034d67b170e06045d527f57184eaceeb17 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Sat, 11 Oct 2014 13:38:29 +0200 Subject: [PATCH 14/66] Move TODO out of scaladoc --- .../src/main/scala/com/twitter/algebird/CountMinSketch.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index f9610673f..bcf6d9cf5 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -457,8 +457,8 @@ object CmsInstance { /** * The 2-dimensional table of counters used in the Count-Min sketch. * Each row corresponds to a particular hash function. - * TODO: implement a dense matrix type, and use it here */ + // TODO: implement a dense matrix type, and use it here case class CmsCountsTable[K](counts: Vector[Vector[Long]]) { require(depth > 0, "Table must have at least 1 row.") require(width > 0, "Table must have at least 1 column.") From 0fd84f9ecfb037b17d9d6ed9c1460b9afa899397 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Sat, 11 Oct 2014 13:39:16 +0200 Subject: [PATCH 15/66] Move implicits to top of class --- .../twitter/algebird/CountMinSketchTest.scala | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala index 540c3ba0a..f45381493 100644 --- a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala +++ b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala @@ -60,6 +60,18 @@ abstract class CmsTest[K: Ordering: CmsHasher: Numeric] extends WordSpec with Ma val CMS_MONOID = Cms.monoid[K](EPS, DELTA, SEED) val RAND = new scala.util.Random + implicit class IntCast(x: Int) { + def toK[K: Numeric]: K = implicitly[Numeric[K]].fromInt(x) + } + + implicit class SeqCast(xs: Seq[Int]) { + def toK[K: Numeric]: Seq[K] = xs map { _.toK[K] } + } + + implicit class SetCast(xs: Set[Int]) { + def toK[K: Numeric]: Set[K] = xs map { _.toK[K] } + } + /** * Returns the exact frequency of {x} in {data}. */ @@ -240,16 +252,4 @@ abstract class CmsTest[K: Ordering: CmsHasher: Numeric] extends WordSpec with Ma } - implicit class IntCast(x: Int) { - def toK[K: Numeric]: K = implicitly[Numeric[K]].fromInt(x) - } - - implicit class SeqCast(xs: Seq[Int]) { - def toK[K: Numeric]: Seq[K] = xs map { _.toK[K] } - } - - implicit class SetCast(xs: Set[Int]) { - def toK[K: Numeric]: Set[K] = xs map { _.toK[K] } - } - } \ No newline at end of file From fc7ea4f70e3d508a335d632a1af488909c729e2e Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Sat, 11 Oct 2014 13:40:51 +0200 Subject: [PATCH 16/66] Do not shadow type parameter K of test suite --- .../main/scala/com/twitter/algebird/CountMinSketch.scala | 3 +-- .../scala/com/twitter/algebird/CountMinSketchTest.scala | 6 +++--- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index bcf6d9cf5..3f8e786ff 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -438,8 +438,7 @@ case class CmsInstance[K: Ordering](countsTable: CmsInstance.CmsCountsTable[K], table + (pos, count) } CmsInstance[K](newCountsTable, totalCount + count, params) - } - else this + } else this } } diff --git a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala index f45381493..4f474aeb2 100644 --- a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala +++ b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala @@ -61,15 +61,15 @@ abstract class CmsTest[K: Ordering: CmsHasher: Numeric] extends WordSpec with Ma val RAND = new scala.util.Random implicit class IntCast(x: Int) { - def toK[K: Numeric]: K = implicitly[Numeric[K]].fromInt(x) + def toK[A: Numeric]: A = implicitly[Numeric[A]].fromInt(x) } implicit class SeqCast(xs: Seq[Int]) { - def toK[K: Numeric]: Seq[K] = xs map { _.toK[K] } + def toK[A: Numeric]: Seq[A] = xs map { _.toK[A] } } implicit class SetCast(xs: Set[Int]) { - def toK[K: Numeric]: Set[K] = xs map { _.toK[K] } + def toK[A: Numeric]: Set[A] = xs map { _.toK[A] } } /** From ec997b28474c2ba17e7d6bbdbe467895a031d1b5 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Sat, 11 Oct 2014 13:48:01 +0200 Subject: [PATCH 17/66] Add monoid laws test for Top-% CMS --- .../twitter/algebird/CountMinSketchTest.scala | 41 +++++++++++++++++++ 1 file changed, 41 insertions(+) diff --git a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala index 4f474aeb2..ef30d1882 100644 --- a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala +++ b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala @@ -46,6 +46,47 @@ class CmsLaws extends PropSpec with PropertyChecks with Matchers { } +class TopPctCmsLaws extends PropSpec with PropertyChecks with Matchers { + import BaseProperties._ + + val DELTA = 1E-8 + val EPS = 0.005 + val SEED = 1 + val HEAVY_HITTERS_PCT = 0.1 + + private def createArbitrary[K: Numeric](cmsMonoid: TopPctCmsMonoid[K]): Arbitrary[TopPctCms[K]] = { + val k = implicitly[Numeric[K]] + Arbitrary { + for (v <- Gen.choose(0, 10000)) yield cmsMonoid.create(k.fromInt(v)) + } + } + + property("TopPctCms[Short] is a Monoid") { + implicit val cmsMonoid = TopPctCms.monoid[Short](EPS, DELTA, SEED, HEAVY_HITTERS_PCT) + implicit val cmsGen = createArbitrary[Short](cmsMonoid) + monoidLaws[TopPctCms[Short]] + } + + property("TopPctCms[Int] is a Monoid") { + implicit val cmsMonoid = TopPctCms.monoid[Int](EPS, DELTA, SEED, HEAVY_HITTERS_PCT) + implicit val cmsGen = createArbitrary[Int](cmsMonoid) + monoidLaws[TopPctCms[Int]] + } + + property("TopPctCms[Long] is a Monoid") { + implicit val cmsMonoid = TopPctCms.monoid[Long](EPS, DELTA, SEED, HEAVY_HITTERS_PCT) + implicit val cmsGen = createArbitrary[Long](cmsMonoid) + monoidLaws[TopPctCms[Long]] + } + + property("TopPctCms[BigInt] is a Monoid") { + implicit val cmsMonoid = TopPctCms.monoid[BigInt](EPS, DELTA, SEED, HEAVY_HITTERS_PCT) + implicit val cmsGen = createArbitrary[BigInt](cmsMonoid) + monoidLaws[TopPctCms[BigInt]] + } + +} + class CmsShortTest extends CmsTest[Short] class CmsIntTest extends CmsTest[Int] class CmsLongTest extends CmsTest[Long] From 83b51339fcf32f5951bf50393489a3f4d7f8b695 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Sat, 11 Oct 2014 13:48:59 +0200 Subject: [PATCH 18/66] Add default value (0.01) for heavyHittersPct --- .../main/scala/com/twitter/algebird/CountMinSketch.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index 3f8e786ff..2d8b01dac 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -567,25 +567,25 @@ object TopPctCms { def monoid[K: Ordering: CmsHasher](eps: Double, delta: Double, seed: Int, - heavyHittersPct: Double): TopPctCmsMonoid[K] = + heavyHittersPct: Double = 0.01): TopPctCmsMonoid[K] = new TopPctCmsMonoid[K](Cms(eps, delta, seed), heavyHittersPct) def monoid[K: Ordering: CmsHasher](depth: Int, width: Int, seed: Int, - heavyHittersPct: Double): TopPctCmsMonoid[K] = + heavyHittersPct: Double = 0.01): TopPctCmsMonoid[K] = monoid(CmsFunctions.eps(width), CmsFunctions.delta(depth), seed, heavyHittersPct) def aggregator[K: Ordering: CmsHasher](eps: Double, delta: Double, seed: Int, - heavyHittersPct: Double): TopPctCmsAggregator[K] = + heavyHittersPct: Double= 0.01): TopPctCmsAggregator[K] = new TopPctCmsAggregator[K](monoid(eps, delta, seed, heavyHittersPct)) def aggregator[K: Ordering: CmsHasher](depth: Int, width: Int, seed: Int, - heavyHittersPct: Double): TopPctCmsAggregator[K] = + heavyHittersPct: Double = 0.01): TopPctCmsAggregator[K] = aggregator(CmsFunctions.eps(width), CmsFunctions.delta(depth), seed, heavyHittersPct) } From 791239da198655ef36e4bad3fdc8be8b532fa27a Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Sat, 11 Oct 2014 13:55:35 +0200 Subject: [PATCH 19/66] Uppercase Cms to CMS throughout all CMS-related classes --- .../com/twitter/algebird/CountMinSketch.scala | 278 +++++++++--------- .../com/twitter/algebird/SketchMap.scala | 6 +- .../twitter/algebird/CountMinSketchTest.scala | 70 ++--- .../summer/HeavyHittersCachingSummer.scala | 6 +- 4 files changed, 180 insertions(+), 180 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index 2d8b01dac..38382c3ab 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -63,7 +63,7 @@ import scala.collection.immutable.SortedSet * * Implicit conversions for commonly used types for `K` such as [[Long]] and [[BigInt]]: * {{{ - * import com.twitter.algebird.CmsHasherImplicits._ + * import com.twitter.algebird.CMSHasherImplicits._ * }}} * * @param eps One-sided error bound on the error of each point query, i.e. frequency estimate. @@ -75,39 +75,39 @@ import scala.collection.immutable.SortedSet * user names, you could map each username to a unique numeric ID expressed as a `Long`, and then count the * occurrences of those `Long`s with a CMS of type `K=Long`. Note that this mapping between the elements of * your problem domain and their identifiers used for counting via CMS should be bijective. - * We require [[Ordering]] and [[CmsHasher]] context bounds for `K`, see [[CmsHasherImplicits]] for available + * We require [[Ordering]] and [[CMSHasher]] context bounds for `K`, see [[CMSHasherImplicits]] for available * implicits that can be imported. * Which type K should you pick in practice? For domains that have less than `2^64` unique elements, you'd * typically use [[Long]]. For larger domains you can try [[BigInt]], for example. */ -class CmsMonoid[K: Ordering: CmsHasher](eps: Double, delta: Double, seed: Int) extends Monoid[Cms[K]] { +class CMSMonoid[K: Ordering: CMSHasher](eps: Double, delta: Double, seed: Int) extends Monoid[CMS[K]] { val params = { - val hashes: Seq[CmsHash[K]] = CmsFunctions.generateHashes(eps, delta, seed) - CmsParams(hashes, eps, delta) + val hashes: Seq[CMSHash[K]] = CMSFunctions.generateHashes(eps, delta, seed) + CMSParams(hashes, eps, delta) } - val zero: Cms[K] = CmsZero[K](params) + val zero: CMS[K] = CMSZero[K](params) /** * We assume the sketches on the left and right use the same hash functions. */ - def plus(left: Cms[K], right: Cms[K]): Cms[K] = left ++ right + def plus(left: CMS[K], right: CMS[K]): CMS[K] = left ++ right /** * Create a sketch out of a single item. */ - def create(item: K): Cms[K] = CmsItem[K](item, params) + def create(item: K): CMS[K] = CMSItem[K](item, params) /** * Create a sketch out of multiple items. */ - def create(data: Seq[K]): Cms[K] = data.foldLeft(zero) { case (acc, x) => plus(acc, create(x)) } + def create(data: Seq[K]): CMS[K] = data.foldLeft(zero) { case (acc, x) => plus(acc, create(x)) } } /** - * Configuration parameters for [[Cms]]. + * Configuration parameters for [[CMS]]. * * @param hashes Pair-wise independent hashes functions. We need `N=depth` such functions (`depth` can be derived from * `delta`). @@ -116,18 +116,18 @@ class CmsMonoid[K: Ordering: CmsHasher](eps: Double, delta: Double, seed: Int) e * (an interval that depends on `eps`) around the truth. * @tparam K */ -case class CmsParams[K](hashes: Seq[CmsHash[K]], eps: Double, delta: Double) { +case class CMSParams[K](hashes: Seq[CMSHash[K]], eps: Double, delta: Double) { require(0 < eps && eps < 1, "eps must lie in (0, 1)") require(0 < delta && delta < 1, "delta must lie in (0, 1)") - require(hashes.size >= CmsFunctions.depth(delta), s"we require at least ${CmsFunctions.depth(delta)} hash functions") + require(hashes.size >= CMSFunctions.depth(delta), s"we require at least ${CMSFunctions.depth(delta)} hash functions") } /** - * Helper functions to generate or to translate between various CMS parameters (cf. [[CmsParams]]). + * Helper functions to generate or to translate between various CMS parameters (cf. [[CMSParams]]). */ -object CmsFunctions { +object CMSFunctions { /** * Translates from `width` to `eps`. @@ -159,7 +159,7 @@ object CmsFunctions { * @tparam K The type used to identify the elements to be counted. * @return The generated hash functions. */ - def generateHashes[K: CmsHasher](eps: Double, delta: Double, seed: Int): Seq[CmsHash[K]] = { + def generateHashes[K: CMSHasher](eps: Double, delta: Double, seed: Int): Seq[CMSHash[K]] = { // Typically, we would use d -- aka depth -- pair-wise independent hash functions of the form // // h_i(x) = a_i * x + b_i (mod p) @@ -172,7 +172,7 @@ object CmsFunctions { val r = new scala.util.Random(seed) val numHashes = depth(delta) val numCounters = width(eps) - (0 to (numHashes - 1)).map { _ => CmsHash[K](r.nextInt(), 0, numCounters) } + (0 to (numHashes - 1)).map { _ => CMSHash[K](r.nextInt(), 0, numCounters) } } } @@ -181,12 +181,12 @@ object CmsFunctions { * A trait for CMS implementations that can count elements in a data stream and that can answer point queries (i.e. * frequency estimates) for these elements. * - * Known implementations: [[Cms]], [[TopPctCms]]. + * Known implementations: [[CMS]], [[TopPctCMS]]. * * @tparam K The type used to identify the elements to be counted. * @tparam C The type of the actual CMS that implements this trait. */ -trait CmsCounting[K, C[_]] { +trait CMSCounting[K, C[_]] { /** * Returns the one-sided error bound on the error of each point query, i.e. frequency estimate. @@ -202,13 +202,13 @@ trait CmsCounting[K, C[_]] { /** * Number of hash functions (also: number of rows in the counting table). This number is derived from `delta`. */ - def depth: Int = CmsFunctions.depth(delta) + def depth: Int = CMSFunctions.depth(delta) /** * Number of counters per hash function (also: number of columns in the counting table). This number is derived from * `eps`. */ - def width: Int = CmsFunctions.width(eps) + def width: Int = CMSFunctions.width(eps) /** * Returns a new sketch that is the combination of this sketch and the other sketch. @@ -270,11 +270,11 @@ trait CmsCounting[K, C[_]] { /** * A trait for CMS implementations that can track heavy hitters in a data stream. * - * Known implementations: [[TopPctCms]]. + * Known implementations: [[TopPctCMS]]. * * @tparam K The type used to identify the elements to be counted. */ -trait CmsHeavyHitters[K] { +trait CMSHeavyHitters[K] { /** * Finds all heavy hitters, i.e., elements in the stream that appear at least @@ -297,29 +297,29 @@ trait CmsHeavyHitters[K] { } -object Cms { +object CMS { - def monoid[K: Ordering: CmsHasher](eps: Double, delta: Double, seed: Int): CmsMonoid[K] = - new CmsMonoid[K](eps, delta, seed) + def monoid[K: Ordering: CMSHasher](eps: Double, delta: Double, seed: Int): CMSMonoid[K] = + new CMSMonoid[K](eps, delta, seed) - def monoid[K: Ordering: CmsHasher](depth: Int, width: Int, seed: Int): CmsMonoid[K] = - monoid(CmsFunctions.eps(width), CmsFunctions.delta(depth), seed) + def monoid[K: Ordering: CMSHasher](depth: Int, width: Int, seed: Int): CMSMonoid[K] = + monoid(CMSFunctions.eps(width), CMSFunctions.delta(depth), seed) - def aggregator[K: Ordering: CmsHasher](eps: Double, delta: Double, seed: Int): CmsAggregator[K] = - new CmsAggregator[K](monoid(eps, delta, seed)) + def aggregator[K: Ordering: CMSHasher](eps: Double, delta: Double, seed: Int): CMSAggregator[K] = + new CMSAggregator[K](monoid(eps, delta, seed)) - def aggregator[K: Ordering: CmsHasher](depth: Int, width: Int, seed: Int): CmsAggregator[K] = - aggregator(CmsFunctions.eps(width), CmsFunctions.delta(depth), seed) + def aggregator[K: Ordering: CMSHasher](depth: Int, width: Int, seed: Int): CMSAggregator[K] = + aggregator(CMSFunctions.eps(width), CMSFunctions.delta(depth), seed) /** * Returns a fresh, zeroed CMS instance. */ - def apply[K: Ordering: CmsHasher](eps: Double, delta: Double, seed: Int): Cms[K] = { + def apply[K: Ordering: CMSHasher](eps: Double, delta: Double, seed: Int): CMS[K] = { val params = { - val hashes: Seq[CmsHash[K]] = CmsFunctions.generateHashes(eps, delta, seed) - CmsParams(hashes, eps, delta) + val hashes: Seq[CMSHash[K]] = CMSFunctions.generateHashes(eps, delta, seed) + CMSParams(hashes, eps, delta) } - CmsInstance[K](params) + CMSInstance[K](params) } } @@ -327,7 +327,7 @@ object Cms { /** * A Count-Min sketch data structure that allows for counting and frequency estimation of elements in a data stream. */ -sealed abstract class Cms[K: Ordering](params: CmsParams[K]) extends java.io.Serializable with CmsCounting[K, Cms] { +sealed abstract class CMS[K: Ordering](params: CMSParams[K]) extends java.io.Serializable with CMSCounting[K, CMS] { override val eps: Double = params.eps @@ -340,57 +340,57 @@ sealed abstract class Cms[K: Ordering](params: CmsParams[K]) extends java.io.Ser /** * Zero element. Used for initialization. */ -case class CmsZero[K: Ordering](params: CmsParams[K]) extends Cms[K](params) { +case class CMSZero[K: Ordering](params: CMSParams[K]) extends CMS[K](params) { override val totalCount: Long = 0L - override def +(item: K, count: Long): Cms[K] = CmsInstance[K](params) + (item, count) + override def +(item: K, count: Long): CMS[K] = CMSInstance[K](params) + (item, count) - override def ++(other: Cms[K]): Cms[K] = other + override def ++(other: CMS[K]): CMS[K] = other override def frequency(item: K): Approximate[Long] = Approximate.exact(0L) - override def innerProduct(other: Cms[K]): Approximate[Long] = Approximate.exact(0L) + override def innerProduct(other: CMS[K]): Approximate[Long] = Approximate.exact(0L) } /** * Used for holding a single element, to avoid repeatedly adding elements from sparse counts tables. */ -case class CmsItem[K: Ordering](item: K, params: CmsParams[K]) extends Cms[K](params) { +case class CMSItem[K: Ordering](item: K, params: CMSParams[K]) extends CMS[K](params) { override val totalCount: Long = 1L - override def +(x: K, count: Long): Cms[K] = CmsInstance[K](params) + item + (x, count) + override def +(x: K, count: Long): CMS[K] = CMSInstance[K](params) + item + (x, count) - override def ++(other: Cms[K]): Cms[K] = { + override def ++(other: CMS[K]): CMS[K] = { other match { - case other: CmsZero[_] => this - case other: CmsItem[K] => CmsInstance[K](params) + item + other.item - case other: CmsInstance[K] => other + item + case other: CMSZero[_] => this + case other: CMSItem[K] => CMSInstance[K](params) + item + other.item + case other: CMSInstance[K] => other + item } } override def frequency(x: K): Approximate[Long] = if (item == x) Approximate.exact(1L) else Approximate.exact(0L) - override def innerProduct(other: Cms[K]): Approximate[Long] = other.frequency(item) + override def innerProduct(other: CMS[K]): Approximate[Long] = other.frequency(item) } /** * The general Count-Min sketch structure, used for holding any number of elements. */ -case class CmsInstance[K: Ordering](countsTable: CmsInstance.CmsCountsTable[K], +case class CMSInstance[K: Ordering](countsTable: CMSInstance.CountsTable[K], override val totalCount: Long, - params: CmsParams[K]) extends Cms[K](params) { + params: CMSParams[K]) extends CMS[K](params) { - def ++(other: Cms[K]): Cms[K] = { + def ++(other: CMS[K]): CMS[K] = { other match { - case other: CmsZero[_] => this - case other: CmsItem[K] => this + other.item - case other: CmsInstance[K] => + case other: CMSZero[_] => this + case other: CMSItem[K] => this + other.item + case other: CMSInstance[K] => val newTotalCount = totalCount + other.totalCount - CmsInstance[K](countsTable ++ other.countsTable, newTotalCount, params) + CMSInstance[K](countsTable ++ other.countsTable, newTotalCount, params) } } @@ -413,9 +413,9 @@ case class CmsInstance[K: Ordering](countsTable: CmsInstance.CmsCountsTable[K], * rows: * estimatedInnerProduct = min_j (\sum_k count_A[j, k] * count_B[j, k]) */ - def innerProduct(other: Cms[K]): Approximate[Long] = { + def innerProduct(other: CMS[K]): Approximate[Long] = { other match { - case other: CmsInstance[_] => + case other: CMSInstance[_] => require((other.depth, other.width) == (depth, width), "Tables must have the same dimensions.") def innerProductAtDepth(d: Int) = (0 to (width - 1)).map { w => @@ -428,7 +428,7 @@ case class CmsInstance[K: Ordering](countsTable: CmsInstance.CmsCountsTable[K], } } - override def +(item: K, count: Long): CmsInstance[K] = { + override def +(item: K, count: Long): CMSInstance[K] = { require(count >= 0, "count must be >= 0 (negative counts not implemented") if (count != 0L) { val newCountsTable = @@ -437,20 +437,20 @@ case class CmsInstance[K: Ordering](countsTable: CmsInstance.CmsCountsTable[K], val pos = (row, params.hashes(row)(item)) table + (pos, count) } - CmsInstance[K](newCountsTable, totalCount + count, params) + CMSInstance[K](newCountsTable, totalCount + count, params) } else this } } -object CmsInstance { +object CMSInstance { /** - * Initializes a CmsInstance with all zeroes, i.e. nothing has been counted yet. + * Initializes a [[CMSInstance]] with all zeroes, i.e. nothing has been counted yet. */ - def apply[K: Ordering](params: CmsParams[K]): CmsInstance[K] = { - val countsTable = CmsCountsTable[K](CmsFunctions.depth(params.delta), CmsFunctions.width(params.eps)) - CmsInstance[K](countsTable, 0, params) + def apply[K: Ordering](params: CMSParams[K]): CMSInstance[K] = { + val countsTable = CountsTable[K](CMSFunctions.depth(params.delta), CMSFunctions.width(params.eps)) + CMSInstance[K](countsTable, 0, params) } /** @@ -458,7 +458,7 @@ object CmsInstance { * Each row corresponds to a particular hash function. */ // TODO: implement a dense matrix type, and use it here - case class CmsCountsTable[K](counts: Vector[Vector[Long]]) { + case class CountsTable[K](counts: Vector[Vector[Long]]) { require(depth > 0, "Table must have at least 1 row.") require(width > 0, "Table must have at least 1 column.") @@ -475,38 +475,38 @@ object CmsInstance { /** * Updates the count of a single cell in the table. */ - def +(pos: (Int, Int), count: Long): CmsCountsTable[K] = { + def +(pos: (Int, Int), count: Long): CountsTable[K] = { val (row, col) = pos val currCount = getCount(pos) val newCounts = counts.updated(row, counts(row).updated(col, currCount + count)) - CmsCountsTable[K](newCounts) + CountsTable[K](newCounts) } /** * Adds another counts table to this one, through element-wise addition. */ - def ++(other: CmsCountsTable[K]): CmsCountsTable[K] = { + def ++(other: CountsTable[K]): CountsTable[K] = { require((depth, width) == (other.depth, other.width), "Tables must have the same dimensions.") val iil: IndexedSeq[IndexedSeq[Long]] = Monoid.plus[IndexedSeq[IndexedSeq[Long]]](counts, other.counts) def toVector[V](is: IndexedSeq[V]): Vector[V] = is match { case v: Vector[_] => v case _ => Vector(is: _*) } - CmsCountsTable[K](toVector(iil.map { toVector })) + CountsTable[K](toVector(iil.map { toVector })) } } - object CmsCountsTable { + object CountsTable { /** - * Creates a new CmsCountsTable with counts initialized to all zeroes. + * Creates a new [[CountsTable]] with counts initialized to all zeroes. */ - def apply[K: Ordering](depth: Int, width: Int): CmsCountsTable[K] = - CmsCountsTable[K](Vector.fill[Long](depth, width)(0L)) + def apply[K: Ordering](depth: Int, width: Int): CountsTable[K] = + CountsTable[K](Vector.fill[Long](depth, width)(0L)) } } -case class TopPctCmsParams(heavyHittersPct: Double) { +case class TopPctCMSParams(heavyHittersPct: Double) { require(0 < heavyHittersPct && heavyHittersPct < 1, "heavyHittersPct must lie in (0, 1)") @@ -532,61 +532,61 @@ case class TopPctCmsParams(heavyHittersPct: Double) { * user names, you could map each username to a unique numeric ID expressed as a `Long`, and then count the * occurrences of those `Long`s with a CMS of type `K=Long`. Note that this mapping between the elements of * your problem domain and their identifiers used for counting via CMS should be bijective. - * We require [[Ordering]] and [[CmsHasher]] context bounds for `K`, see [[CmsHasherImplicits]] for available + * We require [[Ordering]] and [[CMSHasher]] context bounds for `K`, see [[CMSHasherImplicits]] for available * implicits that can be imported. * Which type K should you pick in practice? For domains that have less than `2^64` unique elements, you'd * typically use [[Long]]. For larger domains you can try [[BigInt]], for example. */ -class TopPctCmsMonoid[K: Ordering](cms: Cms[K], heavyHittersPct: Double = 0.01) extends Monoid[TopPctCms[K]] { +class TopPctCMSMonoid[K: Ordering](cms: CMS[K], heavyHittersPct: Double = 0.01) extends Monoid[TopPctCMS[K]] { - val conf: TopPctCmsParams = TopPctCmsParams(heavyHittersPct) + val conf: TopPctCMSParams = TopPctCMSParams(heavyHittersPct) - val zero: TopPctCms[K] = TopPctCmsZero[K](cms, conf) + val zero: TopPctCMS[K] = TopPctCMSZero[K](cms, conf) /** * We assume the sketches on the left and right use the same hash functions. */ - def plus(left: TopPctCms[K], right: TopPctCms[K]): TopPctCms[K] = left ++ right + def plus(left: TopPctCMS[K], right: TopPctCMS[K]): TopPctCMS[K] = left ++ right /** * Create a sketch out of a single item. */ - def create(item: K): TopPctCms[K] = TopPctCmsItem[K](item, cms, conf) + def create(item: K): TopPctCMS[K] = TopPctCMSItem[K](item, cms, conf) /** * Create a sketch out of multiple items. */ - def create(data: Seq[K]): TopPctCms[K] = { + def create(data: Seq[K]): TopPctCMS[K] = { data.foldLeft(zero) { case (acc, x) => plus(acc, create(x)) } } } -object TopPctCms { +object TopPctCMS { - def monoid[K: Ordering: CmsHasher](eps: Double, + def monoid[K: Ordering: CMSHasher](eps: Double, delta: Double, seed: Int, - heavyHittersPct: Double = 0.01): TopPctCmsMonoid[K] = - new TopPctCmsMonoid[K](Cms(eps, delta, seed), heavyHittersPct) + heavyHittersPct: Double = 0.01): TopPctCMSMonoid[K] = + new TopPctCMSMonoid[K](CMS(eps, delta, seed), heavyHittersPct) - def monoid[K: Ordering: CmsHasher](depth: Int, + def monoid[K: Ordering: CMSHasher](depth: Int, width: Int, seed: Int, - heavyHittersPct: Double = 0.01): TopPctCmsMonoid[K] = - monoid(CmsFunctions.eps(width), CmsFunctions.delta(depth), seed, heavyHittersPct) + heavyHittersPct: Double = 0.01): TopPctCMSMonoid[K] = + monoid(CMSFunctions.eps(width), CMSFunctions.delta(depth), seed, heavyHittersPct) - def aggregator[K: Ordering: CmsHasher](eps: Double, + def aggregator[K: Ordering: CMSHasher](eps: Double, delta: Double, seed: Int, - heavyHittersPct: Double= 0.01): TopPctCmsAggregator[K] = - new TopPctCmsAggregator[K](monoid(eps, delta, seed, heavyHittersPct)) + heavyHittersPct: Double= 0.01): TopPctCMSAggregator[K] = + new TopPctCMSAggregator[K](monoid(eps, delta, seed, heavyHittersPct)) - def aggregator[K: Ordering: CmsHasher](depth: Int, + def aggregator[K: Ordering: CMSHasher](depth: Int, width: Int, seed: Int, - heavyHittersPct: Double = 0.01): TopPctCmsAggregator[K] = - aggregator(CmsFunctions.eps(width), CmsFunctions.delta(depth), seed, heavyHittersPct) + heavyHittersPct: Double = 0.01): TopPctCMSAggregator[K] = + aggregator(CMSFunctions.eps(width), CMSFunctions.delta(depth), seed, heavyHittersPct) } @@ -596,8 +596,8 @@ object TopPctCms { * * @tparam K The type used to identify the elements to be counted. */ -sealed abstract class TopPctCms[K: Ordering](val cms: Cms[K], conf: TopPctCmsParams) - extends java.io.Serializable with CmsCounting[K, TopPctCms] with CmsHeavyHitters[K] { +sealed abstract class TopPctCMS[K: Ordering](val cms: CMS[K], conf: TopPctCMSParams) + extends java.io.Serializable with CMSCounting[K, TopPctCMS] with CMSHeavyHitters[K] { override val eps: Double = cms.eps @@ -609,7 +609,7 @@ sealed abstract class TopPctCms[K: Ordering](val cms: Cms[K], conf: TopPctCmsPar override def frequency(item: K): Approximate[Long] = cms.frequency(item) - override def innerProduct(other: TopPctCms[K]): Approximate[Long] = cms.innerProduct(other.cms) + override def innerProduct(other: TopPctCMS[K]): Approximate[Long] = cms.innerProduct(other.cms) def f2: Approximate[Long] = innerProduct(this) @@ -618,57 +618,57 @@ sealed abstract class TopPctCms[K: Ordering](val cms: Cms[K], conf: TopPctCmsPar /** * Zero element. Used for initialization. */ -case class TopPctCmsZero[K: Ordering](override val cms: Cms[K], conf: TopPctCmsParams) extends TopPctCms[K](cms, conf) { +case class TopPctCMSZero[K: Ordering](override val cms: CMS[K], conf: TopPctCMSParams) extends TopPctCMS[K](cms, conf) { override val heavyHitters: Set[K] = Set.empty[K] - override def +(item: K, count: Long): TopPctCms[K] = TopPctCmsInstance(cms, conf) + (item, count) + override def +(item: K, count: Long): TopPctCMS[K] = TopPctCMSInstance(cms, conf) + (item, count) - override def ++(other: TopPctCms[K]): TopPctCms[K] = other + override def ++(other: TopPctCMS[K]): TopPctCMS[K] = other } /** * Used for holding a single element, to avoid repeatedly adding elements from sparse counts tables. */ -case class TopPctCmsItem[K: Ordering](item: K, - override val cms: Cms[K], - conf: TopPctCmsParams) extends TopPctCms[K](cms, conf) { +case class TopPctCMSItem[K: Ordering](item: K, + override val cms: CMS[K], + conf: TopPctCMSParams) extends TopPctCMS[K](cms, conf) { override val heavyHitters: Set[K] = Set(item) - override def +(x: K, count: Long): TopPctCms[K] = TopPctCmsInstance(cms, conf) + item + (x, count) + override def +(x: K, count: Long): TopPctCMS[K] = TopPctCMSInstance(cms, conf) + item + (x, count) - override def ++(other: TopPctCms[K]): TopPctCms[K] = { + override def ++(other: TopPctCMS[K]): TopPctCMS[K] = { other match { - case other: TopPctCmsZero[_] => this - case other: TopPctCmsItem[K] => TopPctCmsInstance[K](cms, conf) + item + other.item - case other: TopPctCmsInstance[K] => other + item + case other: TopPctCMSZero[_] => this + case other: TopPctCMSItem[K] => TopPctCMSInstance[K](cms, conf) + item + other.item + case other: TopPctCMSInstance[K] => other + item } } } -object TopPctCmsInstance { +object TopPctCMSInstance { - def apply[K: Ordering](cms: Cms[K], conf: TopPctCmsParams): TopPctCmsInstance[K] = { + def apply[K: Ordering](cms: CMS[K], conf: TopPctCMSParams): TopPctCMSInstance[K] = { implicit val heavyHitterOrdering = HeavyHitter.ordering[K] - TopPctCmsInstance[K](cms, HeavyHitters[K](SortedSet[HeavyHitter[K]]()), conf) + TopPctCMSInstance[K](cms, HeavyHitters[K](SortedSet[HeavyHitter[K]]()), conf) } } -case class TopPctCmsInstance[K: Ordering](override val cms: Cms[K], hhs: HeavyHitters[K], conf: TopPctCmsParams) - extends TopPctCms[K](cms, conf) { +case class TopPctCMSInstance[K: Ordering](override val cms: CMS[K], hhs: HeavyHitters[K], conf: TopPctCMSParams) + extends TopPctCMS[K](cms, conf) { override def heavyHitters: Set[K] = hhs.items - override def +(item: K, count: Long): TopPctCmsInstance[K] = { + override def +(item: K, count: Long): TopPctCMSInstance[K] = { require(count >= 0, "count must be >= 0 (negative counts not implemented") if (count != 0L) { val newHhs = updateHeavyHitters(item, count) - val newCms = cms + (item, count) - TopPctCmsInstance[K](newCms, newHhs, conf) + val newCMS = cms + (item, count) + TopPctCMSInstance[K](newCMS, newHhs, conf) } else this } @@ -690,14 +690,14 @@ case class TopPctCmsInstance[K: Ordering](override val cms: Cms[K], hhs: HeavyHi newHhs.dropCountsBelow(conf.heavyHittersPct * newTotalCount) } - override def ++(other: TopPctCms[K]): TopPctCms[K] = { + override def ++(other: TopPctCMS[K]): TopPctCMS[K] = { other match { - case other: TopPctCmsZero[_] => this - case other: TopPctCmsItem[K] => this + other.item - case other: TopPctCmsInstance[K] => + case other: TopPctCMSZero[_] => this + case other: TopPctCMSItem[K] => this + other.item + case other: TopPctCMSInstance[K] => val newTotalCount = totalCount + other.totalCount val newHhs = (hhs ++ other.hhs).dropCountsBelow(conf.heavyHittersPct * newTotalCount) - TopPctCmsInstance(cms ++ other.cms, newHhs, conf) + TopPctCMSInstance(cms ++ other.cms, newHhs, conf) } } @@ -728,33 +728,33 @@ object HeavyHitter { } /** - * An Aggregator for [[Cms]]. Can be created using [[Cms.aggregator]]. + * An Aggregator for [[CMS]]. Can be created using [[CMS.aggregator]]. */ -case class CmsAggregator[K](cmsMonoid: CmsMonoid[K]) extends MonoidAggregator[K, Cms[K], Cms[K]] { +case class CMSAggregator[K](cmsMonoid: CMSMonoid[K]) extends MonoidAggregator[K, CMS[K], CMS[K]] { val monoid = cmsMonoid - def prepare(value: K): Cms[K] = monoid.create(value) + def prepare(value: K): CMS[K] = monoid.create(value) - def present(cms: Cms[K]): Cms[K] = cms + def present(cms: CMS[K]): CMS[K] = cms } /** - * An Aggregator for [[TopPctCms]]. Can be created using [[TopPctCms.aggregator]]. + * An Aggregator for [[TopPctCMS]]. Can be created using [[TopPctCMS.aggregator]]. */ -case class TopPctCmsAggregator[K](topPctCmsMonoid: TopPctCmsMonoid[K]) - extends MonoidAggregator[K, TopPctCms[K], TopPctCms[K]] { +case class TopPctCMSAggregator[K](cmsMonoid: TopPctCMSMonoid[K]) + extends MonoidAggregator[K, TopPctCMS[K], TopPctCMS[K]] { - val monoid = topPctCmsMonoid + val monoid = cmsMonoid - def prepare(value: K): TopPctCms[K] = monoid.create(value) + def prepare(value: K): TopPctCMS[K] = monoid.create(value) - def present(cms: TopPctCms[K]): TopPctCms[K] = cms + def present(cms: TopPctCMS[K]): TopPctCMS[K] = cms } -trait CmsHasher[K] { +trait CMSHasher[K] { val PRIME_MODULUS = (1L << 31) - 1 @@ -771,21 +771,21 @@ trait CmsHasher[K] { * * `h(x) = [a * x + b (mod p)] (mod m)` */ -case class CmsHash[K: CmsHasher](a: Int, b: Int, width: Int) { +case class CMSHash[K: CMSHasher](a: Int, b: Int, width: Int) { /** * Returns `a * x + b (mod p) (mod width)`. */ - def apply(x: K): Int = implicitly[CmsHasher[K]].hash(a, b, width)(x) + def apply(x: K): Int = implicitly[CMSHasher[K]].hash(a, b, width)(x) } /** * Implicits that enable CMS-hashing for common data types such as [[Long]] and [[BigInt]]. */ -object CmsHasherImplicits { +object CMSHasherImplicits { - implicit object CmsHasherLong extends CmsHasher[Long] { + implicit object CMSHasherLong extends CMSHasher[Long] { def hash(a: Int, b: Int, width: Int)(x: Long) = { val unModded: Long = (x * a) + b @@ -799,13 +799,13 @@ object CmsHasherImplicits { } - implicit object CmsHasherShort extends CmsHasher[Short] { + implicit object CMSHasherShort extends CMSHasher[Short] { - def hash(a: Int, b: Int, width: Int)(x: Short) = CmsHasherInt.hash(a, b, width)(x) + def hash(a: Int, b: Int, width: Int)(x: Short) = CMSHasherInt.hash(a, b, width)(x) } - implicit object CmsHasherInt extends CmsHasher[Int] { + implicit object CMSHasherInt extends CMSHasher[Int] { def hash(a: Int, b: Int, width: Int)(x: Int) = { val unModded: Int = (x * a) + b @@ -815,7 +815,7 @@ object CmsHasherImplicits { } - implicit object CmsHasherBigInt extends CmsHasher[BigInt] { + implicit object CMSHasherBigInt extends CMSHasher[BigInt] { def hash(a: Int, b: Int, width: Int)(x: BigInt) = { val unModded: BigInt = (x * a) + b diff --git a/algebird-core/src/main/scala/com/twitter/algebird/SketchMap.scala b/algebird-core/src/main/scala/com/twitter/algebird/SketchMap.scala index 9912e7fc6..5fb1aad3a 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/SketchMap.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/SketchMap.scala @@ -17,7 +17,7 @@ limitations under the License. package com.twitter.algebird import scala.collection.breakOut -import com.twitter.algebird.CmsHasherImplicits._ +import com.twitter.algebird.CMSHasherImplicits._ import com.twitter.algebird.matrix.AdaptiveMatrix /** @@ -29,7 +29,7 @@ import com.twitter.algebird.matrix.AdaptiveMatrix /** * Hashes an arbitrary key type to one that the Sketch Map can use. */ -case class SketchMapHash[K](hasher: CmsHash[Long], seed: Int)(implicit serialization: K => Array[Byte]) { +case class SketchMapHash[K](hasher: CMSHash[Long], seed: Int)(implicit serialization: K => Array[Byte]) { def apply(obj: K): Int = { val (first, second) = MurmurHash128(seed)(serialization(obj)) hasher(first ^ second) @@ -138,7 +138,7 @@ case class SketchMapParams[K](seed: Int, width: Int, depth: Int, heavyHittersCou val numHashes = depth val numCounters = width (0 to (numHashes - 1)).map { _ => - val smhash: SketchMapHash[K] = SketchMapHash(CmsHash[Long](r.nextInt, 0, numCounters), seed)(serialization) + val smhash: SketchMapHash[K] = SketchMapHash(CMSHash[Long](r.nextInt, 0, numCounters), seed)(serialization) new (K => Int) { override def apply(k: K) = smhash(k) } } } diff --git a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala index ef30d1882..cb328d565 100644 --- a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala +++ b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala @@ -4,7 +4,7 @@ import org.scalatest.{ PropSpec, Matchers, WordSpec } import org.scalatest.prop.PropertyChecks import org.scalacheck.{ Gen, Arbitrary } -import CmsHasherImplicits._ // required, although e.g. IntelliJ IDEA may flag it as unused import +import CMSHasherImplicits._ // required, although e.g. IntelliJ IDEA may flag it as unused import class CmsLaws extends PropSpec with PropertyChecks with Matchers { import BaseProperties._ @@ -13,7 +13,7 @@ class CmsLaws extends PropSpec with PropertyChecks with Matchers { val EPS = 0.005 val SEED = 1 - private def createArbitrary[K: Numeric](cmsMonoid: CmsMonoid[K]): Arbitrary[Cms[K]] = { + private def createArbitrary[K: Numeric](cmsMonoid: CMSMonoid[K]): Arbitrary[CMS[K]] = { val k = implicitly[Numeric[K]] Arbitrary { for (v <- Gen.choose(0, 10000)) yield cmsMonoid.create(k.fromInt(v)) @@ -21,27 +21,27 @@ class CmsLaws extends PropSpec with PropertyChecks with Matchers { } property("CountMinSketch[Short] is a Monoid") { - implicit val cmsMonoid = Cms.monoid[Short](EPS, DELTA, SEED) + implicit val cmsMonoid = CMS.monoid[Short](EPS, DELTA, SEED) implicit val cmsGen = createArbitrary[Short](cmsMonoid) - monoidLaws[Cms[Short]] + monoidLaws[CMS[Short]] } property("CountMinSketch[Int] is a Monoid") { - implicit val cmsMonoid = Cms.monoid[Int](EPS, DELTA, SEED) + implicit val cmsMonoid = CMS.monoid[Int](EPS, DELTA, SEED) implicit val cmsGen = createArbitrary[Int](cmsMonoid) - monoidLaws[Cms[Int]] + monoidLaws[CMS[Int]] } property("CountMinSketch[Long] is a Monoid") { - implicit val cmsMonoid = Cms.monoid[Long](EPS, DELTA, SEED) + implicit val cmsMonoid = CMS.monoid[Long](EPS, DELTA, SEED) implicit val cmsGen = createArbitrary[Long](cmsMonoid) - monoidLaws[Cms[Long]] + monoidLaws[CMS[Long]] } property("CountMinSketch[BigInt] is a Monoid") { - implicit val cmsMonoid = Cms.monoid[BigInt](EPS, DELTA, SEED) + implicit val cmsMonoid = CMS.monoid[BigInt](EPS, DELTA, SEED) implicit val cmsGen = createArbitrary[BigInt](cmsMonoid) - monoidLaws[Cms[BigInt]] + monoidLaws[CMS[BigInt]] } } @@ -54,7 +54,7 @@ class TopPctCmsLaws extends PropSpec with PropertyChecks with Matchers { val SEED = 1 val HEAVY_HITTERS_PCT = 0.1 - private def createArbitrary[K: Numeric](cmsMonoid: TopPctCmsMonoid[K]): Arbitrary[TopPctCms[K]] = { + private def createArbitrary[K: Numeric](cmsMonoid: TopPctCMSMonoid[K]): Arbitrary[TopPctCMS[K]] = { val k = implicitly[Numeric[K]] Arbitrary { for (v <- Gen.choose(0, 10000)) yield cmsMonoid.create(k.fromInt(v)) @@ -62,43 +62,43 @@ class TopPctCmsLaws extends PropSpec with PropertyChecks with Matchers { } property("TopPctCms[Short] is a Monoid") { - implicit val cmsMonoid = TopPctCms.monoid[Short](EPS, DELTA, SEED, HEAVY_HITTERS_PCT) + implicit val cmsMonoid = TopPctCMS.monoid[Short](EPS, DELTA, SEED, HEAVY_HITTERS_PCT) implicit val cmsGen = createArbitrary[Short](cmsMonoid) - monoidLaws[TopPctCms[Short]] + monoidLaws[TopPctCMS[Short]] } property("TopPctCms[Int] is a Monoid") { - implicit val cmsMonoid = TopPctCms.monoid[Int](EPS, DELTA, SEED, HEAVY_HITTERS_PCT) + implicit val cmsMonoid = TopPctCMS.monoid[Int](EPS, DELTA, SEED, HEAVY_HITTERS_PCT) implicit val cmsGen = createArbitrary[Int](cmsMonoid) - monoidLaws[TopPctCms[Int]] + monoidLaws[TopPctCMS[Int]] } property("TopPctCms[Long] is a Monoid") { - implicit val cmsMonoid = TopPctCms.monoid[Long](EPS, DELTA, SEED, HEAVY_HITTERS_PCT) + implicit val cmsMonoid = TopPctCMS.monoid[Long](EPS, DELTA, SEED, HEAVY_HITTERS_PCT) implicit val cmsGen = createArbitrary[Long](cmsMonoid) - monoidLaws[TopPctCms[Long]] + monoidLaws[TopPctCMS[Long]] } property("TopPctCms[BigInt] is a Monoid") { - implicit val cmsMonoid = TopPctCms.monoid[BigInt](EPS, DELTA, SEED, HEAVY_HITTERS_PCT) + implicit val cmsMonoid = TopPctCMS.monoid[BigInt](EPS, DELTA, SEED, HEAVY_HITTERS_PCT) implicit val cmsGen = createArbitrary[BigInt](cmsMonoid) - monoidLaws[TopPctCms[BigInt]] + monoidLaws[TopPctCMS[BigInt]] } } -class CmsShortTest extends CmsTest[Short] -class CmsIntTest extends CmsTest[Int] -class CmsLongTest extends CmsTest[Long] -class CmsBigIntTest extends CmsTest[BigInt] +class CMSShortTest extends CMSTest[Short] +class CMSIntTest extends CMSTest[Int] +class CMSLongTest extends CMSTest[Long] +class CMSBigIntTest extends CMSTest[BigInt] -abstract class CmsTest[K: Ordering: CmsHasher: Numeric] extends WordSpec with Matchers { +abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Matchers { val DELTA = 1E-10 val EPS = 0.001 val SEED = 1 - val CMS_MONOID = Cms.monoid[K](EPS, DELTA, SEED) + val CMS_MONOID = CMS.monoid[K](EPS, DELTA, SEED) val RAND = new scala.util.Random implicit class IntCast(x: Int) { @@ -235,7 +235,7 @@ abstract class CmsTest[K: Ordering: CmsHasher: Numeric] extends WordSpec with Ma val data = data1 ++ data2 ++ data3 // Find elements that appear at least 20% of the time. - val cms = TopPctCms.monoid[K](EPS, DELTA, SEED, 0.2).create(data) + val cms = TopPctCMS.monoid[K](EPS, DELTA, SEED, 0.2).create(data) val trueHhs = exactHeavyHitters(data, cms.heavyHittersPct) val estimatedHhs = cms.heavyHitters @@ -251,7 +251,7 @@ abstract class CmsTest[K: Ordering: CmsHasher: Numeric] extends WordSpec with Ma } "drop old heavy hitters when new heavy hitters replace them" in { - val monoid = TopPctCms.monoid[K](EPS, DELTA, SEED, 0.3) + val monoid = TopPctCMS.monoid[K](EPS, DELTA, SEED, 0.3) val cms1 = monoid.create(Seq(1, 2, 2).toK[K]) cms1.heavyHitters should be (Set(1, 2)) @@ -267,10 +267,10 @@ abstract class CmsTest[K: Ordering: CmsHasher: Numeric] extends WordSpec with Ma "exactly compute heavy hitters in a small stream" in { val data1 = Seq(1, 2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5, 5, 5, 5).toK[K] - val cms1 = TopPctCms.monoid[K](EPS, DELTA, SEED, 0.01).create(data1) - val cms2 = TopPctCms.monoid[K](EPS, DELTA, SEED, 0.1).create(data1) - val cms3 = TopPctCms.monoid[K](EPS, DELTA, SEED, 0.3).create(data1) - val cms4 = TopPctCms.monoid[K](EPS, DELTA, SEED, 0.9).create(data1) + val cms1 = TopPctCMS.monoid[K](EPS, DELTA, SEED, 0.01).create(data1) + val cms2 = TopPctCMS.monoid[K](EPS, DELTA, SEED, 0.1).create(data1) + val cms3 = TopPctCMS.monoid[K](EPS, DELTA, SEED, 0.3).create(data1) + val cms4 = TopPctCMS.monoid[K](EPS, DELTA, SEED, 0.9).create(data1) cms1.heavyHitters should be (Set(1, 2, 3, 4, 5)) cms2.heavyHitters should be (Set(2, 3, 4, 5)) cms3.heavyHitters should be (Set(5)) @@ -281,10 +281,10 @@ abstract class CmsTest[K: Ordering: CmsHasher: Numeric] extends WordSpec with Ma "work as an Aggregator" in { val data1 = Seq(1, 2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5, 5, 5, 5).toK[K] - val cms1 = TopPctCms.aggregator[K](EPS, DELTA, SEED, 0.01).apply(data1) - val cms2 = TopPctCms.aggregator[K](EPS, DELTA, SEED, 0.1).apply(data1) - val cms3 = TopPctCms.aggregator[K](EPS, DELTA, SEED, 0.3).apply(data1) - val cms4 = TopPctCms.aggregator[K](EPS, DELTA, SEED, 0.9).apply(data1) + val cms1 = TopPctCMS.aggregator[K](EPS, DELTA, SEED, 0.01).apply(data1) + val cms2 = TopPctCMS.aggregator[K](EPS, DELTA, SEED, 0.1).apply(data1) + val cms3 = TopPctCMS.aggregator[K](EPS, DELTA, SEED, 0.3).apply(data1) + val cms4 = TopPctCMS.aggregator[K](EPS, DELTA, SEED, 0.9).apply(data1) cms1.heavyHitters should be (Set(1, 2, 3, 4, 5)) cms2.heavyHitters should be (Set(2, 3, 4, 5)) cms3.heavyHitters should be (Set(5)) diff --git a/algebird-util/src/main/scala/com/twitter/algebird/util/summer/HeavyHittersCachingSummer.scala b/algebird-util/src/main/scala/com/twitter/algebird/util/summer/HeavyHittersCachingSummer.scala index 640e56d96..90d28941c 100644 --- a/algebird-util/src/main/scala/com/twitter/algebird/util/summer/HeavyHittersCachingSummer.scala +++ b/algebird-util/src/main/scala/com/twitter/algebird/util/summer/HeavyHittersCachingSummer.scala @@ -43,7 +43,7 @@ case class HeavyHittersPercent(toFloat: Float) class ApproxHHTracker(hhPct: HeavyHittersPercent, updateFreq: UpdateFrequency, roFreq: RollOverFrequency) { - import CmsHasherImplicits._ + import CMSHasherImplicits._ private[this] final val WIDTH = 1000 private[this] final val DEPTH = 4 @@ -55,9 +55,9 @@ class ApproxHHTracker(hhPct: HeavyHittersPercent, updateFreq: UpdateFrequency, r private[this] final val rollOverFrequency = roFreq.toLong private[this] final var countsTable = Array.fill(WIDTH * DEPTH)(0L) - private[this] final val hashes: IndexedSeq[CmsHash[Long]] = { + private[this] final val hashes: IndexedSeq[CMSHash[Long]] = { val r = new scala.util.Random(5) - (0 until DEPTH).map { _ => CmsHash[Long](r.nextInt, 0, WIDTH) } + (0 until DEPTH).map { _ => CMSHash[Long](r.nextInt, 0, WIDTH) } }.toIndexedSeq @inline From 12f4ff04fdb102427787739dcea527ad0f62bfce Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Sat, 11 Oct 2014 14:04:36 +0200 Subject: [PATCH 20/66] Revert 83b5133: Add default value (0.01) for heavyHittersPct Providing default values leads to scalac errors, example:: method aggregator$default$4:[K]=> Double and method aggregator$default$4:[K]=> Double have same type --- .../main/scala/com/twitter/algebird/CountMinSketch.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index 38382c3ab..d2b415a65 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -567,25 +567,25 @@ object TopPctCMS { def monoid[K: Ordering: CMSHasher](eps: Double, delta: Double, seed: Int, - heavyHittersPct: Double = 0.01): TopPctCMSMonoid[K] = + heavyHittersPct: Double): TopPctCMSMonoid[K] = new TopPctCMSMonoid[K](CMS(eps, delta, seed), heavyHittersPct) def monoid[K: Ordering: CMSHasher](depth: Int, width: Int, seed: Int, - heavyHittersPct: Double = 0.01): TopPctCMSMonoid[K] = + heavyHittersPct: Double): TopPctCMSMonoid[K] = monoid(CMSFunctions.eps(width), CMSFunctions.delta(depth), seed, heavyHittersPct) def aggregator[K: Ordering: CMSHasher](eps: Double, delta: Double, seed: Int, - heavyHittersPct: Double= 0.01): TopPctCMSAggregator[K] = + heavyHittersPct: Double): TopPctCMSAggregator[K] = new TopPctCMSAggregator[K](monoid(eps, delta, seed, heavyHittersPct)) def aggregator[K: Ordering: CMSHasher](depth: Int, width: Int, seed: Int, - heavyHittersPct: Double = 0.01): TopPctCMSAggregator[K] = + heavyHittersPct: Double): TopPctCMSAggregator[K] = aggregator(CMSFunctions.eps(width), CMSFunctions.delta(depth), seed, heavyHittersPct) } From be6c213bfe1a8b195fac548e863c4b299565f7b5 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Sat, 11 Oct 2014 14:06:32 +0200 Subject: [PATCH 21/66] Move scaladoc to correct class --- .../scala/com/twitter/algebird/CountMinSketch.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index d2b415a65..0ebef5446 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -754,6 +754,12 @@ case class TopPctCMSAggregator[K](cmsMonoid: TopPctCMSMonoid[K]) } +/** + * The Count-Min sketch uses `d` (aka `depth`) pair-wise independent hash functions drawn from a universal hashing + * family of the form: + * + * `h(x) = [a * x + b (mod p)] (mod m)` + */ trait CMSHasher[K] { val PRIME_MODULUS = (1L << 31) - 1 @@ -765,12 +771,6 @@ trait CMSHasher[K] { } -/** - * The Count-Min sketch uses `d` (aka `depth`) pair-wise independent hash functions drawn from a universal hashing - * family of the form: - * - * `h(x) = [a * x + b (mod p)] (mod m)` - */ case class CMSHash[K: CMSHasher](a: Int, b: Int, width: Int) { /** From c02d771ae1e658333ae3e1a3e03875e14793cc2c Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Sat, 11 Oct 2014 14:07:43 +0200 Subject: [PATCH 22/66] Reorder code --- .../scala/com/twitter/algebird/CountMinSketch.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index 0ebef5446..5e48affe3 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -506,12 +506,6 @@ object CMSInstance { } -case class TopPctCMSParams(heavyHittersPct: Double) { - - require(0 < heavyHittersPct && heavyHittersPct < 1, "heavyHittersPct must lie in (0, 1)") - -} - /** * Monoid for adding Count-Min sketches. * @@ -562,6 +556,12 @@ class TopPctCMSMonoid[K: Ordering](cms: CMS[K], heavyHittersPct: Double = 0.01) } +case class TopPctCMSParams(heavyHittersPct: Double) { + + require(0 < heavyHittersPct && heavyHittersPct < 1, "heavyHittersPct must lie in (0, 1)") + +} + object TopPctCMS { def monoid[K: Ordering: CMSHasher](eps: Double, From ae0a96341668abbe57738037228e2ed5f8495d51 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Sat, 11 Oct 2014 14:09:41 +0200 Subject: [PATCH 23/66] Improve docs --- .../com/twitter/algebird/CountMinSketch.scala | 40 +++++++++---------- 1 file changed, 20 insertions(+), 20 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index 5e48affe3..651df447e 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -22,26 +22,31 @@ import scala.collection.immutable.SortedSet * A Count-Min sketch is a probabilistic data structure used for summarizing * streams of data in sub-linear space. * - * It works as follows. Let (eps, delta) be two parameters that describe the - * confidence in our error estimates, and let d = ceil(ln 1/delta) - * and w = ceil(e / eps). Then: + * It works as follows. Let `(eps, delta)` be two parameters that describe the + * confidence in our error estimates, and let `d = ceil(ln 1/delta)` + * and `w = ceil(e / eps)`. * - * - Take d pairwise independent hash functions h_i, each of which maps - * onto the domain [0, w - 1]. - * - Create a 2-dimensional table of counts, with d rows and w columns, + * Note: Throughout the code `d` and `w` are called `depth` and `width`, + * respectively. + * + * Then: + * + * - Take `d` pairwise independent hash functions `h_i`, each of which maps + * onto the domain `[0, w - 1]`. + * - Create a 2-dimensional table of counts, with `d` rows and `w` columns, * initialized with all zeroes. * - When a new element x arrives in the stream, update the table of counts - * by setting counts[i, h_i[x]] += 1, for each 1 <= i <= d. + * by setting `counts[i, h_i[x]] += 1`, for each `1 <= i <= d`. * - (Note the rough similarity to a Bloom filter.) * * As an example application, suppose you want to estimate the number of - * times an element x has appeared in a data stream so far. + * times an element `x` has appeared in a data stream so far. * The Count-Min sketch estimate of this frequency is * * min_i { counts[i, h_i[x]] } * - * With probability at least 1 - delta, this estimate is within eps * N - * of the true frequency (i.e., true frequency <= estimate <= true frequency + eps * N), + * With probability at least `1 - delta`, this estimate is within `eps * N` + * of the true frequency (i.e., `true frequency <= estimate <= true frequency + eps * N`), * where N is the total size of the stream so far. * * See http://www.eecs.harvard.edu/~michaelm/CS222/countmin.pdf for technical details, @@ -54,10 +59,10 @@ import scala.collection.immutable.SortedSet */ /** - * Monoid for adding Count-Min sketches. + * Monoid for adding [[CMS]] sketches. * - * eps and delta are parameters that bound the error of each query estimate. For example, errors in - * answering queries (e.g., how often has element x appeared in the stream described by the sketch?) + * `eps` and `delta` are parameters that bound the error of each query estimate. For example, errors in + * answering point queries (e.g., how often has element x appeared in the stream described by the sketch?) * are often of the form: "with probability p >= 1 - delta, the estimate is close to the truth by * some factor depending on eps." * @@ -507,19 +512,14 @@ object CMSInstance { } /** - * Monoid for adding Count-Min sketches. - * - * eps and delta are parameters that bound the error of each query estimate. For example, errors in - * answering queries (e.g., how often has element x appeared in the stream described by the sketch?) - * are often of the form: "with probability p >= 1 - delta, the estimate is close to the truth by - * some factor depending on eps." + * Monoid for adding [[TopPctCMS]] sketches. * * Implicit conversions for commonly used types for `K` such as [[Long]] and [[BigInt]]: * {{{ * import com.twitter.algebird.CMSHasherImplicits._ * }}} * - * @param cms A counting-only CMS instance, which is used for the counting performed by this class. + * @param cms A [[CMS]] instance, which is used for the counting and the frequency estimation performed by this class. * @param heavyHittersPct A threshold for finding heavy hitters, i.e., elements that appear at least * (heavyHittersPct * totalCount) times in the stream. * @tparam K The type used to identify the elements to be counted. For example, if you want to count the occurrence of From 13121c9a7b6e08ee5c00ab3e29c52df97d253aff Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Sun, 12 Oct 2014 14:38:04 +0200 Subject: [PATCH 24/66] Add caliper benchmark for Count-Min Sketch --- .../algebird/caliper/CMSBenchmark.scala | 86 +++++++++++++++++++ 1 file changed, 86 insertions(+) create mode 100644 algebird-caliper/src/test/scala/com/twitter/algebird/caliper/CMSBenchmark.scala diff --git a/algebird-caliper/src/test/scala/com/twitter/algebird/caliper/CMSBenchmark.scala b/algebird-caliper/src/test/scala/com/twitter/algebird/caliper/CMSBenchmark.scala new file mode 100644 index 000000000..1df76d096 --- /dev/null +++ b/algebird-caliper/src/test/scala/com/twitter/algebird/caliper/CMSBenchmark.scala @@ -0,0 +1,86 @@ +package com.twitter.algebird.caliper + +import com.google.caliper.{ Param, SimpleBenchmark } +import com.twitter.algebird.{ TopPctCMS, CMSHasherImplicits, TopPctCMSMonoid } + +/** + * Benchmarks the Count-Min sketch implementation in Algebird. + * + * We benchmark different `K` types as well as different input data streams. + */ +// Once we can convince cappi (https://github.com/softprops/capp) -- the sbt plugin we use to run +// caliper benchmarks -- to work with the latest caliper 1.0-beta-1, we would: +// - Let `CMSBenchmark` extend `Benchmark` (instead of `SimpleBenchmark`) +// - Annotate `timePlus` with `@MacroBenchmark`. +class CMSBenchmark extends SimpleBenchmark { + + @Param(Array("0.1", "0.005")) + val eps: Double = 0.0 + + @Param(Array("0.0000001" /* 1E-8 */ )) + val delta: Double = 0.0 + + @Param(Array("0.2")) + val heavyHittersPct: Double = 0.0 + + @Param(Array("100")) + val operations: Int = 0 // Number of operations per benchmark repetition (cf. `reps`) + + @Param(Array("2048")) + val maxBits: Int = 0 + + var random: scala.util.Random = _ + var cmsLongMonoid: TopPctCMSMonoid[Long] = _ + var cmsBigIntMonoid: TopPctCMSMonoid[BigInt] = _ + + override def setUp { + // Required import of implicit values (e.g. for BigInt- or Long-backed CMS instances) + import CMSHasherImplicits._ + + cmsLongMonoid = { + val seed = 1 + TopPctCMS.monoid[Long](eps, delta, seed, heavyHittersPct) + } + + cmsBigIntMonoid = { + val seed = 1 + TopPctCMS.monoid[BigInt](eps, delta, seed, heavyHittersPct) + } + + random = new scala.util.Random + } + + // Case A (K=Long): We count the first hundred integers, i.e. [1, 100] + def timePlusOfFirstHundredIntegersWithLongCms(reps: Int): Int = { + var dummy = 0 + while (dummy < reps) { + (1 to operations).view.foldLeft(cmsLongMonoid.zero)((l, r) => { l ++ cmsLongMonoid.create(r) }) + dummy += 1 + } + dummy + } + + // Case B.1 (K=BigInt): We count the first hundred integers, i.e. [1, 100] + def timePlusOfFirstHundredIntegersWithBigIntCms(reps: Int): Int = { + var dummy = 0 + while (dummy < reps) { + (1 to operations).view.foldLeft(cmsBigIntMonoid.zero)((l, r) => { l ++ cmsBigIntMonoid.create(r) }) + dummy += 1 + } + dummy + } + + // Case B.2 (K=BigInt): We draw numbers randomly from a 2^maxBits address space + def timePlusOfRandom2048BitNumbersWithBigIntCms(reps: Int): Int = { + var dummy = 0 + while (dummy < reps) { + (1 to operations).view.foldLeft(cmsBigIntMonoid.zero)((l, r) => { + val n = scala.math.BigInt(maxBits, random) + l ++ cmsBigIntMonoid.create(n) + }) + dummy += 1 + } + dummy + } + +} \ No newline at end of file From c22218a8dff90dedbd57d6c081f334d0b68caafb Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Sun, 12 Oct 2014 14:10:41 +0200 Subject: [PATCH 25/66] Add README for algebird-caliper --- algebird-caliper/README.md | 25 +++++++++++++++++++++++++ 1 file changed, 25 insertions(+) create mode 100644 algebird-caliper/README.md diff --git a/algebird-caliper/README.md b/algebird-caliper/README.md new file mode 100644 index 000000000..695213bda --- /dev/null +++ b/algebird-caliper/README.md @@ -0,0 +1,25 @@ +[Caliper](https://code.google.com/p/caliper/)-based Benchmarks for Algebird data structures. + +# Usage + +Run the following commands from the top-level Algebird directory: + + $ ./sbt # <<< enter sbt REPL + > project algebird-caliper + +Now you can run the following commands from within the sbt REPL: + + # List available benchmarks + > show cappi::benchmarks + + # Run a particular benchmark + > cappi::benchmarkOnly com.twitter.algebird.caliper.HLLBenchmark + + # Debug a particular benchmark (shows e.g. number of repetitions that will be run) + > cappi::benchmarkOnly --debug com.twitter.algebird.caliper.HLLBenchmark + + # Run all benchmarks (apparently this is broken, see https://github.com/softprops/cappi/issues/1) + > cappi::benchmarks + +You can find further details in the [cappi](https://github.com/softprops/cappi) documentation, which is the sbt plugin +we use to run the caliper benchmarks. From 093991c61c755e69e3af6ee1f02fae2676da890b Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Sun, 12 Oct 2014 14:41:05 +0200 Subject: [PATCH 26/66] Add example output to algebird-caliper README --- algebird-caliper/README.md | 27 +++++++++++++++++++++++++++ 1 file changed, 27 insertions(+) diff --git a/algebird-caliper/README.md b/algebird-caliper/README.md index 695213bda..60d41aa79 100644 --- a/algebird-caliper/README.md +++ b/algebird-caliper/README.md @@ -23,3 +23,30 @@ Now you can run the following commands from within the sbt REPL: You can find further details in the [cappi](https://github.com/softprops/cappi) documentation, which is the sbt plugin we use to run the caliper benchmarks. + +Example output for [CMSBenchmark](src/test/scala/com/twitter/algebird/caliper/CMSBenchmark.scala): + + > cappi::benchmarkOnly com.twitter.algebird.caliper.CMSBenchmark + [info] Running com.google.caliper.Runner com.twitter.algebird.caliper.CMSBenchmark + [info] 0% Scenario{vm=java, trial=0, benchmark=PlusOfFirstHundredIntegersWithLongCms, delta=0.0000001, eps=0.1, heavyHittersPct=0.2, maxBits=2048, operations=100} 292576.31 ns; σ=1271.12 ns @ 3 trials + [info] 17% Scenario{vm=java, trial=0, benchmark=PlusOfFirstHundredIntegersWithBigIntCms, delta=0.0000001, eps=0.1, heavyHittersPct=0.2, maxBits=2048, operations=100} 830195.29 ns; σ=7349.10 ns @ 3 trials + [info] 33% Scenario{vm=java, trial=0, benchmark=PlusOfRandom2048BitNumbersWithBigIntCms, delta=0.0000001, eps=0.1, heavyHittersPct=0.2, maxBits=2048, operations=100} 3362751.81 ns; σ=104683.16 ns @ 10 trials + [info] 50% Scenario{vm=java, trial=0, benchmark=PlusOfFirstHundredIntegersWithLongCms, delta=0.0000001, eps=0.005, heavyHittersPct=0.2, maxBits=2048, operations=100} 384133.61 ns; σ=41211.47 ns @ 10 trials + [info] 67% Scenario{vm=java, trial=0, benchmark=PlusOfFirstHundredIntegersWithBigIntCms, delta=0.0000001, eps=0.005, heavyHittersPct=0.2, maxBits=2048, operations=100} 1018308.55 ns; σ=43285.12 ns @ 10 trials + [info] 83% Scenario{vm=java, trial=0, benchmark=PlusOfRandom2048BitNumbersWithBigIntCms, delta=0.0000001, eps=0.005, heavyHittersPct=0.2, maxBits=2048, operations=100} 3610991.09 ns; σ=195033.95 ns @ 10 trials + [info] + [info] benchmark eps us linear runtime + [info] PlusOfFirstHundredIntegersWithLongCms 0.1 293 == + [info] PlusOfFirstHundredIntegersWithLongCms 0.005 384 === + [info] PlusOfFirstHundredIntegersWithBigIntCms 0.1 830 ====== + [info] PlusOfFirstHundredIntegersWithBigIntCms 0.005 1018 ======== + [info] PlusOfRandom2048BitNumbersWithBigIntCms 0.1 3363 =========================== + [info] PlusOfRandom2048BitNumbersWithBigIntCms 0.005 3611 ============================== + [info] + [info] vm: java + [info] trial: 0 + [info] delta: 0.0000001 + [info] heavyHittersPct: 0.2 + [info] maxBits: 2048 + [info] operations: 100 + [success] Total time: 74 s, completed Oct 12, 2014 2:36:04 PM From 1e065e2ade3805568009d09f57291b2b7e30fb00 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Sun, 12 Oct 2014 15:52:47 +0200 Subject: [PATCH 27/66] Rename conf to params to be consistent across CMS implementations --- .../com/twitter/algebird/CountMinSketch.scala | 41 ++++++++++--------- 1 file changed, 22 insertions(+), 19 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index 651df447e..2995c8580 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -533,9 +533,9 @@ object CMSInstance { */ class TopPctCMSMonoid[K: Ordering](cms: CMS[K], heavyHittersPct: Double = 0.01) extends Monoid[TopPctCMS[K]] { - val conf: TopPctCMSParams = TopPctCMSParams(heavyHittersPct) + val params: TopPctCMSParams = TopPctCMSParams(heavyHittersPct) - val zero: TopPctCMS[K] = TopPctCMSZero[K](cms, conf) + val zero: TopPctCMS[K] = TopPctCMSZero[K](cms, params) /** * We assume the sketches on the left and right use the same hash functions. @@ -545,7 +545,7 @@ class TopPctCMSMonoid[K: Ordering](cms: CMS[K], heavyHittersPct: Double = 0.01) /** * Create a sketch out of a single item. */ - def create(item: K): TopPctCMS[K] = TopPctCMSItem[K](item, cms, conf) + def create(item: K): TopPctCMS[K] = TopPctCMSItem[K](item, cms, params) /** * Create a sketch out of multiple items. @@ -596,7 +596,7 @@ object TopPctCMS { * * @tparam K The type used to identify the elements to be counted. */ -sealed abstract class TopPctCMS[K: Ordering](val cms: CMS[K], conf: TopPctCMSParams) +sealed abstract class TopPctCMS[K: Ordering](val cms: CMS[K], params: TopPctCMSParams) extends java.io.Serializable with CMSCounting[K, TopPctCMS] with CMSHeavyHitters[K] { override val eps: Double = cms.eps @@ -605,7 +605,7 @@ sealed abstract class TopPctCMS[K: Ordering](val cms: CMS[K], conf: TopPctCMSPar override val totalCount: Long = cms.totalCount - override val heavyHittersPct = conf.heavyHittersPct + override val heavyHittersPct = params.heavyHittersPct override def frequency(item: K): Approximate[Long] = cms.frequency(item) @@ -618,11 +618,12 @@ sealed abstract class TopPctCMS[K: Ordering](val cms: CMS[K], conf: TopPctCMSPar /** * Zero element. Used for initialization. */ -case class TopPctCMSZero[K: Ordering](override val cms: CMS[K], conf: TopPctCMSParams) extends TopPctCMS[K](cms, conf) { +case class TopPctCMSZero[K: Ordering](override val cms: CMS[K], + params: TopPctCMSParams) extends TopPctCMS[K](cms, params) { override val heavyHitters: Set[K] = Set.empty[K] - override def +(item: K, count: Long): TopPctCMS[K] = TopPctCMSInstance(cms, conf) + (item, count) + override def +(item: K, count: Long): TopPctCMS[K] = TopPctCMSInstance(cms, params) + (item, count) override def ++(other: TopPctCMS[K]): TopPctCMS[K] = other @@ -633,16 +634,16 @@ case class TopPctCMSZero[K: Ordering](override val cms: CMS[K], conf: TopPctCMSP */ case class TopPctCMSItem[K: Ordering](item: K, override val cms: CMS[K], - conf: TopPctCMSParams) extends TopPctCMS[K](cms, conf) { + params: TopPctCMSParams) extends TopPctCMS[K](cms, params) { override val heavyHitters: Set[K] = Set(item) - override def +(x: K, count: Long): TopPctCMS[K] = TopPctCMSInstance(cms, conf) + item + (x, count) + override def +(x: K, count: Long): TopPctCMS[K] = TopPctCMSInstance(cms, params) + item + (x, count) override def ++(other: TopPctCMS[K]): TopPctCMS[K] = { other match { case other: TopPctCMSZero[_] => this - case other: TopPctCMSItem[K] => TopPctCMSInstance[K](cms, conf) + item + other.item + case other: TopPctCMSItem[K] => TopPctCMSInstance[K](cms, params) + item + other.item case other: TopPctCMSInstance[K] => other + item } } @@ -651,15 +652,17 @@ case class TopPctCMSItem[K: Ordering](item: K, object TopPctCMSInstance { - def apply[K: Ordering](cms: CMS[K], conf: TopPctCMSParams): TopPctCMSInstance[K] = { + def apply[K: Ordering](cms: CMS[K], params: TopPctCMSParams): TopPctCMSInstance[K] = { implicit val heavyHitterOrdering = HeavyHitter.ordering[K] - TopPctCMSInstance[K](cms, HeavyHitters[K](SortedSet[HeavyHitter[K]]()), conf) + TopPctCMSInstance[K](cms, HeavyHitters[K](SortedSet[HeavyHitter[K]]()), params) } } -case class TopPctCMSInstance[K: Ordering](override val cms: CMS[K], hhs: HeavyHitters[K], conf: TopPctCMSParams) - extends TopPctCMS[K](cms, conf) { +case class TopPctCMSInstance[K: Ordering](override val cms: CMS[K], + hhs: HeavyHitters[K], + params: TopPctCMSParams) + extends TopPctCMS[K](cms, params) { override def heavyHitters: Set[K] = hhs.items @@ -668,7 +671,7 @@ case class TopPctCMSInstance[K: Ordering](override val cms: CMS[K], hhs: HeavyHi if (count != 0L) { val newHhs = updateHeavyHitters(item, count) val newCMS = cms + (item, count) - TopPctCMSInstance[K](newCMS, newHhs, conf) + TopPctCMSInstance[K](newCMS, newHhs, params) } else this } @@ -682,12 +685,12 @@ case class TopPctCMSInstance[K: Ordering](override val cms: CMS[K], hhs: HeavyHi // If the new item is a heavy hitter, add it, and remove any previous instances. val newHhs = - if (newItemCount >= conf.heavyHittersPct * newTotalCount) { + if (newItemCount >= params.heavyHittersPct * newTotalCount) { hhs - HeavyHitter[K](item, oldItemCount) + HeavyHitter[K](item, newItemCount) } else hhs // Remove any items below the new heavy hitter threshold. - newHhs.dropCountsBelow(conf.heavyHittersPct * newTotalCount) + newHhs.dropCountsBelow(params.heavyHittersPct * newTotalCount) } override def ++(other: TopPctCMS[K]): TopPctCMS[K] = { @@ -696,8 +699,8 @@ case class TopPctCMSInstance[K: Ordering](override val cms: CMS[K], hhs: HeavyHi case other: TopPctCMSItem[K] => this + other.item case other: TopPctCMSInstance[K] => val newTotalCount = totalCount + other.totalCount - val newHhs = (hhs ++ other.hhs).dropCountsBelow(conf.heavyHittersPct * newTotalCount) - TopPctCMSInstance(cms ++ other.cms, newHhs, conf) + val newHhs = (hhs ++ other.hhs).dropCountsBelow(params.heavyHittersPct * newTotalCount) + TopPctCMSInstance(cms ++ other.cms, newHhs, params) } } From 8dea74395ebcfa3553bd2aa21631ce63a2fbffdf Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Sun, 12 Oct 2014 20:56:48 +0200 Subject: [PATCH 28/66] Assert that computed hashes are not negative Hashes must be >= 0 to prevent IndexOutOfBoundsException when accessing rows in the counting table. --- .../main/scala/com/twitter/algebird/CountMinSketch.scala | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index 2995c8580..73b27b65b 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -813,7 +813,9 @@ object CMSHasherImplicits { def hash(a: Int, b: Int, width: Int)(x: Int) = { val unModded: Int = (x * a) + b val modded: Long = (unModded + (unModded >> 32)) & PRIME_MODULUS - modded.toInt % width + val h = modded.toInt % width + assert(h >= 0, "hash must not be negative") + h } } @@ -823,7 +825,9 @@ object CMSHasherImplicits { def hash(a: Int, b: Int, width: Int)(x: BigInt) = { val unModded: BigInt = (x * a) + b val modded: BigInt = (unModded + (unModded >> 32)) & PRIME_MODULUS - modded.toInt % width + val h = modded.toInt % width + assert(h >= 0, "hash must not be negative") + h } } From e54a2713e5b08902d1715960d14a0f59f1d8dee4 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Mon, 13 Oct 2014 10:03:22 +0200 Subject: [PATCH 29/66] Improve scaladoc of CMS and TopPctCMS --- .../com/twitter/algebird/CountMinSketch.scala | 64 +++++++++++++++++++ 1 file changed, 64 insertions(+) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index 73b27b65b..755488c00 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -331,6 +331,37 @@ object CMS { /** * A Count-Min sketch data structure that allows for counting and frequency estimation of elements in a data stream. + * + * Tip: If you also need to track heavy hitters ("Top N" problems), take a look at [[TopPctCMS]]. + * + * =Usage= + * + * This example demonstrates how to count [[Long]] elements with [[CMS]], i.e. `K=Long`. + * + * Note that the actual counting is always performed with a [[Long]], regardless of your choice of `K`. That is, + * the counting table behind the scenes is backed by [[Long]] values (at least in the current implementation), and thus + * the returned frequency estimates are always instances of `Approximate[Long]`. + * + * @example {{{ + * // Implicits that enabling CMS-hashing of `Long` values. + * import com.twitter.algebird.CMSHasherImplicits._ + * + * // Create a monoid for a CMS that can count `Long` elements. + * val cmsMonoid: CMSMonoid[Long] = { + * val eps = 0.001 + * val delta = 1E-10 + * val seed = 1 + * CMS.monoid[Long](eps, delta, seed) + * } + * + * // Create a CMS instance that has counted the element `1L`. + * val cms: CMS[Long] = cmsMonoid.create(1L) + * + * // Estimate frequency of `1L` + * val estimate: Approximate[Long] = cms.frequency(1L) + * }}} + * + * @tparam K The type used to identify the elements to be counted. */ sealed abstract class CMS[K: Ordering](params: CMSParams[K]) extends java.io.Serializable with CMSCounting[K, CMS] { @@ -594,6 +625,39 @@ object TopPctCMS { * A Count-Min sketch data structure that allows for (a) counting and frequency estimation of elements in a data stream * and (b) tracking the heavy hitters among these elements. * + * Tip: If you do not need to track heavy hitters, take a look at [[CMS]], which is more efficient in this case. + * + * =Usage= + * + * This example demonstrates how to count `Long` elements with [[TopPctCMS]], i.e. `K=Long`. + * + * Note that the actual counting is always performed with a [[Long]], regardless of your choice of `K`. That is, + * the counting table behind the scenes is backed by [[Long]] values (at least in the current implementation), and thus + * the returned frequency estimates are always instances of `Approximate[Long]`. + * + * @example {{{ + * // Implicits that enabling CMS-hashing of `Long` values. + * import com.twitter.algebird.CMSHasherImplicits._ + * + * // Create a monoid for a CMS that can count `Long` elements. + * val topPctCMSMonoid: TopPctCMSMonoid[Long] = { + * val eps = 0.001 + * val delta = 1E-10 + * val seed = 1 + * val heavyHittersPct = 0.1 + * TopPctCMS.monoid[Long](eps, delta, seed, heavyHittersPct) + * } + * + * // Create a TopPctCMS instance that has counted the element `1L`. + * val topPctCMS: TopPctCMS[Long] = topPctCMSMonoid.create(1L) + * + * // Estimate frequency of `1L` + * val estimate: Approximate[Long] = topPctCMS.frequency(1L) + * + * // What are the heavy hitters so far? + * val heavyHitters: Set[Long] = topPctCMS.heavyHitters + * }}} + * * @tparam K The type used to identify the elements to be counted. */ sealed abstract class TopPctCMS[K: Ordering](val cms: CMS[K], params: TopPctCMSParams) From d1cc35bb421175bf26553ddba0cf958c1db7f94d Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Mon, 13 Oct 2014 10:18:43 +0200 Subject: [PATCH 30/66] Add aggregator test for CMS[K], split test cases by CMS trait --- .../twitter/algebird/CountMinSketchTest.scala | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala index cb328d565..6f27a1203 100644 --- a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala +++ b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala @@ -138,7 +138,7 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma counts.filter { _._2 >= heavyHittersPct * totalCount }.keys.toSet } - "CountMinSketch" should { + "A Count-Min sketch implementing CMSCounting" should { "count total number of elements in a stream" in { val totalCount = 1243 @@ -227,6 +227,20 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma CMS_MONOID.create(d1).innerProduct(CMS_MONOID.create(d2)).estimate should be (6) } + "work as an Aggregator" in { + val data1 = Seq(1, 2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5, 5, 5, 5).toK[K] + val cms = CMS.aggregator[K](EPS, DELTA, SEED).apply(data1) + cms.frequency(1.toK[K]).estimate should be(1L) + cms.frequency(2.toK[K]).estimate should be(2L) + cms.frequency(3.toK[K]).estimate should be(3L) + cms.frequency(4.toK[K]).estimate should be(4L) + cms.frequency(5.toK[K]).estimate should be(5L) + } + + } + + "A Count-Min sketch implementing CMSHeavyHitters" should { + "estimate heavy hitters" in { // Simple way of making some elements appear much more often than others. val data1 = (1 to 3000).map { _ => RAND.nextInt(3) }.toK[K] @@ -277,8 +291,6 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma cms4.heavyHitters should be (Set[K]()) } - // TODO: test aggregator for Cms, not only for TopPctCms - "work as an Aggregator" in { val data1 = Seq(1, 2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5, 5, 5, 5).toK[K] val cms1 = TopPctCMS.aggregator[K](EPS, DELTA, SEED, 0.01).apply(data1) From 10af35f1303a7225aad7f9bf3a4c8fb841550f75 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Mon, 13 Oct 2014 10:33:08 +0200 Subject: [PATCH 31/66] Use TopPctCMS for testing CMSCounting functionality We reason that because TopPctCMS[K] encapsulates CMS[K] and uses it for all its counting/querying functionality (like an adapter) we can test CMS[K] indirectly through testing TopPctCMS[K]. --- .../twitter/algebird/CountMinSketchTest.scala | 37 +++++++++++-------- 1 file changed, 22 insertions(+), 15 deletions(-) diff --git a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala index 6f27a1203..346c076c9 100644 --- a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala +++ b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala @@ -98,7 +98,14 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma val EPS = 0.001 val SEED = 1 - val CMS_MONOID = CMS.monoid[K](EPS, DELTA, SEED) + // We use TopPctCMS for testing CMSCounting functionality. We argue that because TopPctCMS[K] encapsulates CMS[K] + // and uses it for all its counting/querying functionality (like an adapter) we can test CMS[K] indirectly through + // testing TopPctCMS[K]. + val COUNTING_CMS_MONOID = { + val ANY_HEAVY_HITTERS_PCT = 0.1 // heavy hitters functionality is not relevant for the tests using this monoid + TopPctCMS.monoid[K](EPS, DELTA, SEED, ANY_HEAVY_HITTERS_PCT) + } + val RAND = new scala.util.Random implicit class IntCast(x: Int) { @@ -144,7 +151,7 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma val totalCount = 1243 val range = 234 val data = (0 to (totalCount - 1)).map { _ => RAND.nextInt(range) }.toK[K] - val cms = CMS_MONOID.create(data) + val cms = COUNTING_CMS_MONOID.create(data) cms.totalCount should be (totalCount) } @@ -153,7 +160,7 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma val totalCount = 5678 val range = 897 val data = (0 to (totalCount - 1)).map { _ => RAND.nextInt(range) }.toK[K] - val cms = CMS_MONOID.create(data) + val cms = COUNTING_CMS_MONOID.create(data) (0 to 100).foreach { _ => val x = RAND.nextInt(range).toK[K] @@ -169,19 +176,19 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma } "exactly compute frequencies in a small stream" in { - val one = CMS_MONOID.create(1.toK[K]) - val two = CMS_MONOID.create(2.toK[K]) - val cms = CMS_MONOID.plus(CMS_MONOID.plus(one, two), two) + val one = COUNTING_CMS_MONOID.create(1.toK[K]) + val two = COUNTING_CMS_MONOID.create(2.toK[K]) + val cms = COUNTING_CMS_MONOID.plus(COUNTING_CMS_MONOID.plus(one, two), two) cms.frequency(0.toK[K]).estimate should be (0) cms.frequency(1.toK[K]).estimate should be (1) cms.frequency(2.toK[K]).estimate should be (2) - val three = CMS_MONOID.create(Seq(1, 1, 1).toK[K]) + val three = COUNTING_CMS_MONOID.create(Seq(1, 1, 1).toK[K]) three.frequency(1.toK[K]).estimate should be (3) - val four = CMS_MONOID.create(Seq(1, 1, 1, 1).toK[K]) + val four = COUNTING_CMS_MONOID.create(Seq(1, 1, 1, 1).toK[K]) four.frequency(1.toK[K]).estimate should be (4) - val cms2 = CMS_MONOID.plus(four, three) + val cms2 = COUNTING_CMS_MONOID.plus(four, three) cms2.frequency(1.toK[K]).estimate should be (7) } @@ -190,8 +197,8 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma val range = 1390 val data1 = (0 to (totalCount - 1)).map { _ => RAND.nextInt(range) }.toK[K] val data2 = (0 to (totalCount - 1)).map { _ => RAND.nextInt(range) }.toK[K] - val cms1 = CMS_MONOID.create(data1) - val cms2 = CMS_MONOID.create(data1) + val cms1 = COUNTING_CMS_MONOID.create(data1) + val cms2 = COUNTING_CMS_MONOID.create(data1) val approxA = cms1.innerProduct(cms2) val approx = approxA.estimate @@ -209,22 +216,22 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma // Nothing in common. val a1 = List(1, 2, 3).toK[K] val a2 = List(4, 5, 6).toK[K] - CMS_MONOID.create(a1).innerProduct(CMS_MONOID.create(a2)).estimate should be (0) + COUNTING_CMS_MONOID.create(a1).innerProduct(COUNTING_CMS_MONOID.create(a2)).estimate should be (0) // One element in common. val b1 = List(1, 2, 3).toK[K] val b2 = List(3, 5, 6).toK[K] - CMS_MONOID.create(b1).innerProduct(CMS_MONOID.create(b2)).estimate should be (1) + COUNTING_CMS_MONOID.create(b1).innerProduct(COUNTING_CMS_MONOID.create(b2)).estimate should be (1) // Multiple, non-repeating elements in common. val c1 = List(1, 2, 3).toK[K] val c2 = List(3, 2, 6).toK[K] - CMS_MONOID.create(c1).innerProduct(CMS_MONOID.create(c2)).estimate should be (2) + COUNTING_CMS_MONOID.create(c1).innerProduct(COUNTING_CMS_MONOID.create(c2)).estimate should be (2) // Multiple, repeating elements in common. val d1 = List(1, 2, 2, 3, 3).toK[K] val d2 = List(2, 3, 3, 6).toK[K] - CMS_MONOID.create(d1).innerProduct(CMS_MONOID.create(d2)).estimate should be (6) + COUNTING_CMS_MONOID.create(d1).innerProduct(COUNTING_CMS_MONOID.create(d2)).estimate should be (6) } "work as an Aggregator" in { From e61fad4dd17dfd7764bcb531fbe4d5d33f648deb Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Mon, 13 Oct 2014 10:38:02 +0200 Subject: [PATCH 32/66] Also test CMSCounting-based aggregator functionality of TopPctCMS --- .../com/twitter/algebird/CountMinSketchTest.scala | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala index 346c076c9..30950c14c 100644 --- a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala +++ b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala @@ -236,12 +236,23 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma "work as an Aggregator" in { val data1 = Seq(1, 2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5, 5, 5, 5).toK[K] + val cms = CMS.aggregator[K](EPS, DELTA, SEED).apply(data1) cms.frequency(1.toK[K]).estimate should be(1L) cms.frequency(2.toK[K]).estimate should be(2L) cms.frequency(3.toK[K]).estimate should be(3L) cms.frequency(4.toK[K]).estimate should be(4L) cms.frequency(5.toK[K]).estimate should be(5L) + + val topPctCMS = { + val anyHeavyHittersPct = 0.1 // exact setting not relevant for this test + TopPctCMS.aggregator[K](EPS, DELTA, SEED, anyHeavyHittersPct).apply(data1) + } + topPctCMS.frequency(1.toK[K]).estimate should be(1L) + topPctCMS.frequency(2.toK[K]).estimate should be(2L) + topPctCMS.frequency(3.toK[K]).estimate should be(3L) + topPctCMS.frequency(4.toK[K]).estimate should be(4L) + topPctCMS.frequency(5.toK[K]).estimate should be(5L) } } From ddd298530398faac1d4c516f65b4fcb41d267312 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Mon, 13 Oct 2014 10:45:54 +0200 Subject: [PATCH 33/66] Add reference to Spire's numerical data types as possible choices for K --- .../src/main/scala/com/twitter/algebird/CountMinSketch.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index 755488c00..9fce21ca6 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -83,7 +83,9 @@ import scala.collection.immutable.SortedSet * We require [[Ordering]] and [[CMSHasher]] context bounds for `K`, see [[CMSHasherImplicits]] for available * implicits that can be imported. * Which type K should you pick in practice? For domains that have less than `2^64` unique elements, you'd - * typically use [[Long]]. For larger domains you can try [[BigInt]], for example. + * typically use [[Long]]. For larger domains you can try [[BigInt]], for example. Other possibilities + * include Spire's `SafeLong` and `Numerical` data types (https://github.com/non/spire), though Algebird does + * not include the required implicits for CMS-hashing (cf. [[CMSHasherImplicits]]. */ class CMSMonoid[K: Ordering: CMSHasher](eps: Double, delta: Double, seed: Int) extends Monoid[CMS[K]] { From dd5eaae0f32213f3e2e40a57766625f17c790bbc Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Tue, 14 Oct 2014 09:22:25 +0200 Subject: [PATCH 34/66] heavyHittersPct should not be part of the trait/contact of CMSHeavyHitters It is up to the implementation how the semantics of heavy hitters should be defined (e.g. top % vs. top N). --- .../com/twitter/algebird/CountMinSketch.scala | 32 ++++++++++--------- 1 file changed, 17 insertions(+), 15 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index 9fce21ca6..0fbf97596 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -277,26 +277,16 @@ trait CMSCounting[K, C[_]] { /** * A trait for CMS implementations that can track heavy hitters in a data stream. * + * It is up to the implementation how the semantics of tracking heavy hitters are defined. For instance, one + * implementation could track the "top %" heavy hitters whereas another implementation could track the "top N" heavy + * hitters. + * * Known implementations: [[TopPctCMS]]. * * @tparam K The type used to identify the elements to be counted. */ trait CMSHeavyHitters[K] { - /** - * Finds all heavy hitters, i.e., elements in the stream that appear at least - * `(heavyHittersPct * totalCount)` times. - * - * Every item that appears at least `(heavyHittersPct * totalCount)` times is output, - * and with probability `p >= 1 - delta`, no item whose count is less than - * `(heavyHittersPct - eps) * totalCount` is output. - * - * Note that the set of heavy hitters contains at most `1 / heavyHittersPct` - * elements, so keeping track of all elements that appear more than (say) 1% of the - * time requires tracking at most 100 items. - */ - def heavyHittersPct: Double - /** * Returns the descendingly sorted list of heavy hitters (e.g. the heaviest hitter is the first element). */ @@ -671,7 +661,19 @@ sealed abstract class TopPctCMS[K: Ordering](val cms: CMS[K], params: TopPctCMSP override val totalCount: Long = cms.totalCount - override val heavyHittersPct = params.heavyHittersPct + /** + * Finds all heavy hitters, i.e., elements in the stream that appear at least + * `(heavyHittersPct * totalCount)` times. + * + * Every item that appears at least `(heavyHittersPct * totalCount)` times is output, + * and with probability `p >= 1 - delta`, no item whose count is less than + * `(heavyHittersPct - eps) * totalCount` is output. + * + * Note that the set of heavy hitters contains at most `1 / heavyHittersPct` + * elements, so keeping track of all elements that appear more than (say) 1% of the + * time requires tracking at most 100 items. + */ + val heavyHittersPct: Double = params.heavyHittersPct override def frequency(item: K): Approximate[Long] = cms.frequency(item) From 9162b200ce0d2f75089134cc73aab162dde9a1cd Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Tue, 14 Oct 2014 09:37:43 +0200 Subject: [PATCH 35/66] Clarify the meaning and effect of heavyHittersPct --- .../scala/com/twitter/algebird/CountMinSketch.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index 0fbf97596..902c22f07 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -665,13 +665,13 @@ sealed abstract class TopPctCMS[K: Ordering](val cms: CMS[K], params: TopPctCMSP * Finds all heavy hitters, i.e., elements in the stream that appear at least * `(heavyHittersPct * totalCount)` times. * - * Every item that appears at least `(heavyHittersPct * totalCount)` times is output, - * and with probability `p >= 1 - delta`, no item whose count is less than - * `(heavyHittersPct - eps) * totalCount` is output. + * Every item that appears at least `(heavyHittersPct * totalCount)` times is output, and with probability + * `p >= 1 - delta`, no item whose count is less than `(heavyHittersPct - eps) * totalCount` is output. * - * Note that the set of heavy hitters contains at most `1 / heavyHittersPct` - * elements, so keeping track of all elements that appear more than (say) 1% of the - * time requires tracking at most 100 items. + * This also means that this parameter is an upper bound on the number of heavy hitters that will be tracked: the set + * of heavy hitters contains at most `1 / heavyHittersPct` elements. For example, if `heavyHittersPct=0.01` (or + * 0.25), then at most `1 / 0.01 = 100` items (or `1 / 0.25 = 4` items) will be tracked/returned as heavy hitters. + * This parameter can thus control the memory footprint required for tracking heavy hitters. */ val heavyHittersPct: Double = params.heavyHittersPct From a76b645974e087f05fd1339b52c5ca5d6ac3bf60 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Tue, 14 Oct 2014 11:00:40 +0200 Subject: [PATCH 36/66] Simplify purging heavy hitters when building a new TopPctCMSInstance --- .../main/scala/com/twitter/algebird/CountMinSketch.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index 902c22f07..3e6c841c1 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -766,9 +766,9 @@ case class TopPctCMSInstance[K: Ordering](override val cms: CMS[K], case other: TopPctCMSZero[_] => this case other: TopPctCMSItem[K] => this + other.item case other: TopPctCMSInstance[K] => - val newTotalCount = totalCount + other.totalCount - val newHhs = (hhs ++ other.hhs).dropCountsBelow(params.heavyHittersPct * newTotalCount) - TopPctCMSInstance(cms ++ other.cms, newHhs, params) + val newCms = cms ++ other.cms + val newHhs = (hhs ++ other.hhs).dropCountsBelow(params.heavyHittersPct * newCms.totalCount) + TopPctCMSInstance(newCms, newHhs, params) } } From 47d12ac45dc6137525c002d72b5981ecf0444a0f Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Tue, 14 Oct 2014 12:05:41 +0200 Subject: [PATCH 37/66] Decouple logic from computing/pruning heavy hitters from "Top" CMS functionality We can inject the heavy hitter related logic into a single TopCMS implementation, and thereby achieve top-% as well as top-N CMS functionality as needed. --- .../algebird/caliper/CMSBenchmark.scala | 6 +- .../com/twitter/algebird/CountMinSketch.scala | 282 +++++++++++++----- .../twitter/algebird/CountMinSketchTest.scala | 164 +++++++--- 3 files changed, 333 insertions(+), 119 deletions(-) diff --git a/algebird-caliper/src/test/scala/com/twitter/algebird/caliper/CMSBenchmark.scala b/algebird-caliper/src/test/scala/com/twitter/algebird/caliper/CMSBenchmark.scala index 1df76d096..d42c95cc7 100644 --- a/algebird-caliper/src/test/scala/com/twitter/algebird/caliper/CMSBenchmark.scala +++ b/algebird-caliper/src/test/scala/com/twitter/algebird/caliper/CMSBenchmark.scala @@ -1,7 +1,7 @@ package com.twitter.algebird.caliper import com.google.caliper.{ Param, SimpleBenchmark } -import com.twitter.algebird.{ TopPctCMS, CMSHasherImplicits, TopPctCMSMonoid } +import com.twitter.algebird.{ TopCMS, CMSHasherImplicits, TopPctCMSMonoid } /** * Benchmarks the Count-Min sketch implementation in Algebird. @@ -39,12 +39,12 @@ class CMSBenchmark extends SimpleBenchmark { cmsLongMonoid = { val seed = 1 - TopPctCMS.monoid[Long](eps, delta, seed, heavyHittersPct) + TopCMS.monoid[Long](eps, delta, seed, heavyHittersPct) } cmsBigIntMonoid = { val seed = 1 - TopPctCMS.monoid[BigInt](eps, delta, seed, heavyHittersPct) + TopCMS.monoid[BigInt](eps, delta, seed, heavyHittersPct) } random = new scala.util.Random diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index 3e6c841c1..dbafbebf8 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -188,7 +188,7 @@ object CMSFunctions { * A trait for CMS implementations that can count elements in a data stream and that can answer point queries (i.e. * frequency estimates) for these elements. * - * Known implementations: [[CMS]], [[TopPctCMS]]. + * Known implementations: [[CMS]], [[TopCMS]]. * * @tparam K The type used to identify the elements to be counted. * @tparam C The type of the actual CMS that implements this trait. @@ -281,15 +281,16 @@ trait CMSCounting[K, C[_]] { * implementation could track the "top %" heavy hitters whereas another implementation could track the "top N" heavy * hitters. * - * Known implementations: [[TopPctCMS]]. + * Known implementations: [[TopCMS]]. * * @tparam K The type used to identify the elements to be counted. */ trait CMSHeavyHitters[K] { /** - * Returns the descendingly sorted list of heavy hitters (e.g. the heaviest hitter is the first element). + * Returns the set of heavy hitters. */ + // TODO: Should we change the contract to return a descendingly sorted list of heavy hitters (heaviest = first)? def heavyHitters: Set[K] } @@ -324,7 +325,7 @@ object CMS { /** * A Count-Min sketch data structure that allows for counting and frequency estimation of elements in a data stream. * - * Tip: If you also need to track heavy hitters ("Top N" problems), take a look at [[TopPctCMS]]. + * Tip: If you also need to track heavy hitters ("Top N" problems), take a look at [[TopCMS]]. * * =Usage= * @@ -535,7 +536,7 @@ object CMSInstance { } /** - * Monoid for adding [[TopPctCMS]] sketches. + * Monoid for adding [[TopCMS]] sketches. * * Implicit conversions for commonly used types for `K` such as [[Long]] and [[BigInt]]: * {{{ @@ -554,37 +555,83 @@ object CMSInstance { * Which type K should you pick in practice? For domains that have less than `2^64` unique elements, you'd * typically use [[Long]]. For larger domains you can try [[BigInt]], for example. */ -class TopPctCMSMonoid[K: Ordering](cms: CMS[K], heavyHittersPct: Double = 0.01) extends Monoid[TopPctCMS[K]] { +class TopPctCMSMonoid[K: Ordering](cms: CMS[K], heavyHittersPct: Double = 0.01) extends Monoid[TopCMS[K]] { - val params: TopPctCMSParams = TopPctCMSParams(heavyHittersPct) + val params: TopCMSParams[K] = { + val logic = new TopPctHeavyHittersLogic[K](heavyHittersPct) + TopCMSParams[K](logic) + } - val zero: TopPctCMS[K] = TopPctCMSZero[K](cms, params) + val zero: TopCMS[K] = TopCMSZero[K](cms, params) /** * We assume the sketches on the left and right use the same hash functions. */ - def plus(left: TopPctCMS[K], right: TopPctCMS[K]): TopPctCMS[K] = left ++ right + def plus(left: TopCMS[K], right: TopCMS[K]): TopCMS[K] = left ++ right /** * Create a sketch out of a single item. */ - def create(item: K): TopPctCMS[K] = TopPctCMSItem[K](item, cms, params) + def create(item: K): TopCMS[K] = TopCMSItem[K](item, cms, params) /** * Create a sketch out of multiple items. */ - def create(data: Seq[K]): TopPctCMS[K] = { + def create(data: Seq[K]): TopCMS[K] = { data.foldLeft(zero) { case (acc, x) => plus(acc, create(x)) } } } -case class TopPctCMSParams(heavyHittersPct: Double) { +/** + * Monoid for adding [[TopCMS]] sketches. + * + * Implicit conversions for commonly used types for `K` such as [[Long]] and [[BigInt]]: + * {{{ + * import com.twitter.algebird.CMSHasherImplicits._ + * }}} + * + * @param cms A [[CMS]] instance, which is used for the counting and the frequency estimation performed by this class. + * @param heavyHittersN The maximum number of heavy hitters to track. + * @tparam K The type used to identify the elements to be counted. For example, if you want to count the occurrence of + * user names, you could map each username to a unique numeric ID expressed as a `Long`, and then count the + * occurrences of those `Long`s with a CMS of type `K=Long`. Note that this mapping between the elements of + * your problem domain and their identifiers used for counting via CMS should be bijective. + * We require [[Ordering]] and [[CMSHasher]] context bounds for `K`, see [[CMSHasherImplicits]] for available + * implicits that can be imported. + * Which type K should you pick in practice? For domains that have less than `2^64` unique elements, you'd + * typically use [[Long]]. For larger domains you can try [[BigInt]], for example. + */ +class TopNCMSMonoid[K: Ordering](cms: CMS[K], heavyHittersN: Int = 100) extends Monoid[TopCMS[K]] { - require(0 < heavyHittersPct && heavyHittersPct < 1, "heavyHittersPct must lie in (0, 1)") + val params: TopCMSParams[K] = { + val logic = new TopNHeavyHittersLogic[K](heavyHittersN) + TopCMSParams[K](logic) + } + + val zero: TopCMS[K] = TopCMSZero[K](cms, params) + + /** + * We assume the sketches on the left and right use the same hash functions. + */ + def plus(left: TopCMS[K], right: TopCMS[K]): TopCMS[K] = left ++ right + + /** + * Create a sketch out of a single item. + */ + def create(item: K): TopCMS[K] = TopCMSItem[K](item, cms, params) + + /** + * Create a sketch out of multiple items. + */ + def create(data: Seq[K]): TopCMS[K] = { + data.foldLeft(zero) { case (acc, x) => plus(acc, create(x)) } + } } +case class TopCMSParams[K: Ordering](logic: HeavyHittersLogic[K]) + object TopPctCMS { def monoid[K: Ordering: CMSHasher](eps: Double, @@ -613,6 +660,34 @@ object TopPctCMS { } +object TopNCMS { + + def monoid[K: Ordering: CMSHasher](eps: Double, + delta: Double, + seed: Int, + heavyHittersN: Int): TopNCMSMonoid[K] = + new TopNCMSMonoid[K](CMS(eps, delta, seed), heavyHittersN) + + def monoid[K: Ordering: CMSHasher](depth: Int, + width: Int, + seed: Int, + heavyHittersN: Int): TopNCMSMonoid[K] = + monoid(CMSFunctions.eps(width), CMSFunctions.delta(depth), seed, heavyHittersN) + + def aggregator[K: Ordering: CMSHasher](eps: Double, + delta: Double, + seed: Int, + heavyHittersN: Int): TopNCMSAggregator[K] = + new TopNCMSAggregator[K](monoid(eps, delta, seed, heavyHittersN)) + + def aggregator[K: Ordering: CMSHasher](depth: Int, + width: Int, + seed: Int, + heavyHittersN: Int): TopNCMSAggregator[K] = + aggregator(CMSFunctions.eps(width), CMSFunctions.delta(depth), seed, heavyHittersN) + +} + /** * A Count-Min sketch data structure that allows for (a) counting and frequency estimation of elements in a data stream * and (b) tracking the heavy hitters among these elements. @@ -621,7 +696,7 @@ object TopPctCMS { * * =Usage= * - * This example demonstrates how to count `Long` elements with [[TopPctCMS]], i.e. `K=Long`. + * This example demonstrates how to count `Long` elements with [[TopCMS]], i.e. `K=Long`. * * Note that the actual counting is always performed with a [[Long]], regardless of your choice of `K`. That is, * the counting table behind the scenes is backed by [[Long]] values (at least in the current implementation), and thus @@ -652,8 +727,8 @@ object TopPctCMS { * * @tparam K The type used to identify the elements to be counted. */ -sealed abstract class TopPctCMS[K: Ordering](val cms: CMS[K], params: TopPctCMSParams) - extends java.io.Serializable with CMSCounting[K, TopPctCMS] with CMSHeavyHitters[K] { +sealed abstract class TopCMS[K: Ordering](val cms: CMS[K], params: TopCMSParams[K]) + extends java.io.Serializable with CMSCounting[K, TopCMS] with CMSHeavyHitters[K] { override val eps: Double = cms.eps @@ -661,23 +736,9 @@ sealed abstract class TopPctCMS[K: Ordering](val cms: CMS[K], params: TopPctCMSP override val totalCount: Long = cms.totalCount - /** - * Finds all heavy hitters, i.e., elements in the stream that appear at least - * `(heavyHittersPct * totalCount)` times. - * - * Every item that appears at least `(heavyHittersPct * totalCount)` times is output, and with probability - * `p >= 1 - delta`, no item whose count is less than `(heavyHittersPct - eps) * totalCount` is output. - * - * This also means that this parameter is an upper bound on the number of heavy hitters that will be tracked: the set - * of heavy hitters contains at most `1 / heavyHittersPct` elements. For example, if `heavyHittersPct=0.01` (or - * 0.25), then at most `1 / 0.01 = 100` items (or `1 / 0.25 = 4` items) will be tracked/returned as heavy hitters. - * This parameter can thus control the memory footprint required for tracking heavy hitters. - */ - val heavyHittersPct: Double = params.heavyHittersPct - override def frequency(item: K): Approximate[Long] = cms.frequency(item) - override def innerProduct(other: TopPctCMS[K]): Approximate[Long] = cms.innerProduct(other.cms) + override def innerProduct(other: TopCMS[K]): Approximate[Long] = cms.innerProduct(other.cms) def f2: Approximate[Long] = innerProduct(this) @@ -686,90 +747,121 @@ sealed abstract class TopPctCMS[K: Ordering](val cms: CMS[K], params: TopPctCMSP /** * Zero element. Used for initialization. */ -case class TopPctCMSZero[K: Ordering](override val cms: CMS[K], - params: TopPctCMSParams) extends TopPctCMS[K](cms, params) { +case class TopCMSZero[K: Ordering](override val cms: CMS[K], params: TopCMSParams[K]) + extends TopCMS[K](cms, params) { override val heavyHitters: Set[K] = Set.empty[K] - override def +(item: K, count: Long): TopPctCMS[K] = TopPctCMSInstance(cms, params) + (item, count) + override def +(item: K, count: Long): TopCMS[K] = TopCMSInstance(cms, params) + (item, count) - override def ++(other: TopPctCMS[K]): TopPctCMS[K] = other + override def ++(other: TopCMS[K]): TopCMS[K] = other } /** * Used for holding a single element, to avoid repeatedly adding elements from sparse counts tables. */ -case class TopPctCMSItem[K: Ordering](item: K, - override val cms: CMS[K], - params: TopPctCMSParams) extends TopPctCMS[K](cms, params) { +case class TopCMSItem[K: Ordering](item: K, override val cms: CMS[K], params: TopCMSParams[K]) + extends TopCMS[K](cms, params) { override val heavyHitters: Set[K] = Set(item) - override def +(x: K, count: Long): TopPctCMS[K] = TopPctCMSInstance(cms, params) + item + (x, count) + override def +(x: K, count: Long): TopCMS[K] = TopCMSInstance(cms, params) + item + (x, count) - override def ++(other: TopPctCMS[K]): TopPctCMS[K] = { + override def ++(other: TopCMS[K]): TopCMS[K] = { other match { - case other: TopPctCMSZero[_] => this - case other: TopPctCMSItem[K] => TopPctCMSInstance[K](cms, params) + item + other.item - case other: TopPctCMSInstance[K] => other + item + case other: TopCMSZero[_] => this + case other: TopCMSItem[K] => TopCMSInstance[K](cms, params) + item + other.item + case other: TopCMSInstance[K] => other + item } } } -object TopPctCMSInstance { +object TopCMSInstance { - def apply[K: Ordering](cms: CMS[K], params: TopPctCMSParams): TopPctCMSInstance[K] = { - implicit val heavyHitterOrdering = HeavyHitter.ordering[K] - TopPctCMSInstance[K](cms, HeavyHitters[K](SortedSet[HeavyHitter[K]]()), params) + def apply[K: Ordering](cms: CMS[K], params: TopCMSParams[K]): TopCMSInstance[K] = { + TopCMSInstance[K](cms, HeavyHitters.empty[K], params) } } -case class TopPctCMSInstance[K: Ordering](override val cms: CMS[K], - hhs: HeavyHitters[K], - params: TopPctCMSParams) - extends TopPctCMS[K](cms, params) { +case class TopCMSInstance[K: Ordering](override val cms: CMS[K], hhs: HeavyHitters[K], params: TopCMSParams[K]) + extends TopCMS[K](cms, params) { override def heavyHitters: Set[K] = hhs.items - override def +(item: K, count: Long): TopPctCMSInstance[K] = { + override def +(item: K, count: Long): TopCMSInstance[K] = { require(count >= 0, "count must be >= 0 (negative counts not implemented") if (count != 0L) { - val newHhs = updateHeavyHitters(item, count) - val newCMS = cms + (item, count) - TopPctCMSInstance[K](newCMS, newHhs, params) + val newCms = cms + (item, count) + val newHhs = params.logic.updateHeavyHitters(cms, newCms)(hhs, item, count) + TopCMSInstance[K](newCms, newHhs, params) } else this } - /** - * Updates the data structure of heavy hitters when a new item (with associated count) enters the stream. - */ - private def updateHeavyHitters(item: K, count: Long): HeavyHitters[K] = { - val oldItemCount = frequency(item).estimate + override def ++(other: TopCMS[K]): TopCMS[K] = { + other match { + case other: TopCMSZero[_] => this + case other: TopCMSItem[K] => this + other.item + case other: TopCMSInstance[K] => + val newCms = cms ++ other.cms + val newHhs = params.logic.updateHeavyHitters(newCms)(hhs, other.hhs) + TopCMSInstance(newCms, newHhs, params) + } + } + +} + +abstract class HeavyHittersLogic[K: Ordering] { + + def updateHeavyHitters(oldCms: CMS[K], newCms: CMS[K])(hhs: HeavyHitters[K], item: K, count: Long): HeavyHitters[K] = { + val oldItemCount = oldCms.frequency(item).estimate + val oldHh = HeavyHitter[K](item, oldItemCount) val newItemCount = oldItemCount + count - val newTotalCount = totalCount + count + val newHh = HeavyHitter[K](item, newItemCount) + purgeHeavyHitters(newCms)(hhs - oldHh + newHh) + } + + def updateHeavyHitters(cms: CMS[K])(left: HeavyHitters[K], right: HeavyHitters[K]) = { + val allItems = left.items ++ right.items + val hitters = allItems.map { case i => HeavyHitter[K](i, cms.frequency(i).estimate) } + val newHhs = HeavyHitters.from(hitters) + purgeHeavyHitters(cms)(newHhs) + } + + def purgeHeavyHitters(cms: CMS[K])(hhs: HeavyHitters[K]): HeavyHitters[K] + +} + +/** + * Finds all heavy hitters, i.e., elements in the stream that appear at least `(heavyHittersPct * totalCount)` times. + * + * Every item that appears at least `(heavyHittersPct * totalCount)` times is output, and with probability + * `p >= 1 - delta`, no item whose count is less than `(heavyHittersPct - eps) * totalCount` is output. + * + * This also means that this parameter is an upper bound on the number of heavy hitters that will be tracked: the set + * of heavy hitters contains at most `1 / heavyHittersPct` elements. For example, if `heavyHittersPct=0.01` (or + * 0.25), then at most `1 / 0.01 = 100` items (or `1 / 0.25 = 4` items) will be tracked/returned as heavy hitters. + * This parameter can thus control the memory footprint required for tracking heavy hitters. + */ +case class TopPctHeavyHittersLogic[K: Ordering](heavyHittersPct: Double) extends HeavyHittersLogic[K] { - // If the new item is a heavy hitter, add it, and remove any previous instances. - val newHhs = - if (newItemCount >= params.heavyHittersPct * newTotalCount) { - hhs - HeavyHitter[K](item, oldItemCount) + HeavyHitter[K](item, newItemCount) - } else hhs + require(0 < heavyHittersPct && heavyHittersPct < 1, "heavyHittersPct must lie in (0, 1)") - // Remove any items below the new heavy hitter threshold. - newHhs.dropCountsBelow(params.heavyHittersPct * newTotalCount) + override def purgeHeavyHitters(cms: CMS[K])(hitters: HeavyHitters[K]): HeavyHitters[K] = { + val minCount = heavyHittersPct * cms.totalCount + HeavyHitters[K](hitters.hhs.dropWhile { _.count < minCount }) } - override def ++(other: TopPctCMS[K]): TopPctCMS[K] = { - other match { - case other: TopPctCMSZero[_] => this - case other: TopPctCMSItem[K] => this + other.item - case other: TopPctCMSInstance[K] => - val newCms = cms ++ other.cms - val newHhs = (hhs ++ other.hhs).dropCountsBelow(params.heavyHittersPct * newCms.totalCount) - TopPctCMSInstance(newCms, newHhs, params) - } +} + +case class TopNHeavyHittersLogic[K: Ordering](heavyHittersN: Int) extends HeavyHittersLogic[K] { + + require(heavyHittersN > 0, "heavyHittersN must be > 0") + + override def purgeHeavyHitters(cms: CMS[K])(hitters: HeavyHitters[K]): HeavyHitters[K] = { + HeavyHitters[K](hitters.hhs.takeRight(heavyHittersN)) } } @@ -787,7 +879,21 @@ case class HeavyHitters[K: Ordering](hhs: SortedSet[HeavyHitter[K]]) { def items: Set[K] = hhs.map { _.item } - def dropCountsBelow(minCount: Double): HeavyHitters[K] = HeavyHitters[K](hhs.dropWhile { _.count < minCount }) +} + +object HeavyHitters { + + def empty[K: Ordering]: HeavyHitters[K] = HeavyHitters(emptyHhs) + + private def emptyHhs[K: Ordering]: SortedSet[HeavyHitter[K]] = { + implicit val heavyHitterOrdering = HeavyHitter.ordering[K] + SortedSet[HeavyHitter[K]]() + } + + def from[K: Ordering](hhs: Set[HeavyHitter[K]]): HeavyHitters[K] = { + HeavyHitters(hhs.foldLeft(emptyHhs)(_ + _)) + } + } case class HeavyHitter[K: Ordering](item: K, count: Long) @@ -815,13 +921,27 @@ case class CMSAggregator[K](cmsMonoid: CMSMonoid[K]) extends MonoidAggregator[K, * An Aggregator for [[TopPctCMS]]. Can be created using [[TopPctCMS.aggregator]]. */ case class TopPctCMSAggregator[K](cmsMonoid: TopPctCMSMonoid[K]) - extends MonoidAggregator[K, TopPctCMS[K], TopPctCMS[K]] { + extends MonoidAggregator[K, TopCMS[K], TopCMS[K]] { + + val monoid = cmsMonoid + + def prepare(value: K): TopCMS[K] = monoid.create(value) + + def present(cms: TopCMS[K]): TopCMS[K] = cms + +} + +/** + * An Aggregator for [[TopNCMS]]. Can be created using [[TopNCMS.aggregator]]. + */ +case class TopNCMSAggregator[K](cmsMonoid: TopNCMSMonoid[K]) + extends MonoidAggregator[K, TopCMS[K], TopCMS[K]] { val monoid = cmsMonoid - def prepare(value: K): TopPctCMS[K] = monoid.create(value) + def prepare(value: K): TopCMS[K] = monoid.create(value) - def present(cms: TopPctCMS[K]): TopPctCMS[K] = cms + def present(cms: TopCMS[K]): TopCMS[K] = cms } diff --git a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala index 30950c14c..79d9f1dd5 100644 --- a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala +++ b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala @@ -4,7 +4,9 @@ import org.scalatest.{ PropSpec, Matchers, WordSpec } import org.scalatest.prop.PropertyChecks import org.scalacheck.{ Gen, Arbitrary } -import CMSHasherImplicits._ // required, although e.g. IntelliJ IDEA may flag it as unused import +import CMSHasherImplicits._ + +// required, although e.g. IntelliJ IDEA may flag it as unused import class CmsLaws extends PropSpec with PropertyChecks with Matchers { import BaseProperties._ @@ -54,7 +56,7 @@ class TopPctCmsLaws extends PropSpec with PropertyChecks with Matchers { val SEED = 1 val HEAVY_HITTERS_PCT = 0.1 - private def createArbitrary[K: Numeric](cmsMonoid: TopPctCMSMonoid[K]): Arbitrary[TopPctCMS[K]] = { + private def createArbitrary[K: Numeric](cmsMonoid: TopPctCMSMonoid[K]): Arbitrary[TopCMS[K]] = { val k = implicitly[Numeric[K]] Arbitrary { for (v <- Gen.choose(0, 10000)) yield cmsMonoid.create(k.fromInt(v)) @@ -64,25 +66,25 @@ class TopPctCmsLaws extends PropSpec with PropertyChecks with Matchers { property("TopPctCms[Short] is a Monoid") { implicit val cmsMonoid = TopPctCMS.monoid[Short](EPS, DELTA, SEED, HEAVY_HITTERS_PCT) implicit val cmsGen = createArbitrary[Short](cmsMonoid) - monoidLaws[TopPctCMS[Short]] + monoidLaws[TopCMS[Short]] } property("TopPctCms[Int] is a Monoid") { implicit val cmsMonoid = TopPctCMS.monoid[Int](EPS, DELTA, SEED, HEAVY_HITTERS_PCT) implicit val cmsGen = createArbitrary[Int](cmsMonoid) - monoidLaws[TopPctCMS[Int]] + monoidLaws[TopCMS[Int]] } property("TopPctCms[Long] is a Monoid") { implicit val cmsMonoid = TopPctCMS.monoid[Long](EPS, DELTA, SEED, HEAVY_HITTERS_PCT) implicit val cmsGen = createArbitrary[Long](cmsMonoid) - monoidLaws[TopPctCMS[Long]] + monoidLaws[TopCMS[Long]] } property("TopPctCms[BigInt] is a Monoid") { implicit val cmsMonoid = TopPctCMS.monoid[BigInt](EPS, DELTA, SEED, HEAVY_HITTERS_PCT) implicit val cmsGen = createArbitrary[BigInt](cmsMonoid) - monoidLaws[TopPctCMS[BigInt]] + monoidLaws[TopCMS[BigInt]] } } @@ -153,7 +155,7 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma val data = (0 to (totalCount - 1)).map { _ => RAND.nextInt(range) }.toK[K] val cms = COUNTING_CMS_MONOID.create(data) - cms.totalCount should be (totalCount) + cms.totalCount should be(totalCount) } "estimate frequencies" in { @@ -180,16 +182,16 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma val two = COUNTING_CMS_MONOID.create(2.toK[K]) val cms = COUNTING_CMS_MONOID.plus(COUNTING_CMS_MONOID.plus(one, two), two) - cms.frequency(0.toK[K]).estimate should be (0) - cms.frequency(1.toK[K]).estimate should be (1) - cms.frequency(2.toK[K]).estimate should be (2) + cms.frequency(0.toK[K]).estimate should be(0) + cms.frequency(1.toK[K]).estimate should be(1) + cms.frequency(2.toK[K]).estimate should be(2) val three = COUNTING_CMS_MONOID.create(Seq(1, 1, 1).toK[K]) - three.frequency(1.toK[K]).estimate should be (3) + three.frequency(1.toK[K]).estimate should be(3) val four = COUNTING_CMS_MONOID.create(Seq(1, 1, 1, 1).toK[K]) - four.frequency(1.toK[K]).estimate should be (4) + four.frequency(1.toK[K]).estimate should be(4) val cms2 = COUNTING_CMS_MONOID.plus(four, three) - cms2.frequency(1.toK[K]).estimate should be (7) + cms2.frequency(1.toK[K]).estimate should be(7) } "estimate inner products" in { @@ -216,22 +218,22 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma // Nothing in common. val a1 = List(1, 2, 3).toK[K] val a2 = List(4, 5, 6).toK[K] - COUNTING_CMS_MONOID.create(a1).innerProduct(COUNTING_CMS_MONOID.create(a2)).estimate should be (0) + COUNTING_CMS_MONOID.create(a1).innerProduct(COUNTING_CMS_MONOID.create(a2)).estimate should be(0) // One element in common. val b1 = List(1, 2, 3).toK[K] val b2 = List(3, 5, 6).toK[K] - COUNTING_CMS_MONOID.create(b1).innerProduct(COUNTING_CMS_MONOID.create(b2)).estimate should be (1) + COUNTING_CMS_MONOID.create(b1).innerProduct(COUNTING_CMS_MONOID.create(b2)).estimate should be(1) // Multiple, non-repeating elements in common. val c1 = List(1, 2, 3).toK[K] val c2 = List(3, 2, 6).toK[K] - COUNTING_CMS_MONOID.create(c1).innerProduct(COUNTING_CMS_MONOID.create(c2)).estimate should be (2) + COUNTING_CMS_MONOID.create(c1).innerProduct(COUNTING_CMS_MONOID.create(c2)).estimate should be(2) // Multiple, repeating elements in common. val d1 = List(1, 2, 2, 3, 3).toK[K] val d2 = List(2, 3, 3, 6).toK[K] - COUNTING_CMS_MONOID.create(d1).innerProduct(COUNTING_CMS_MONOID.create(d2)).estimate should be (6) + COUNTING_CMS_MONOID.create(d1).innerProduct(COUNTING_CMS_MONOID.create(d2)).estimate should be(6) } "work as an Aggregator" in { @@ -257,7 +259,7 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma } - "A Count-Min sketch implementing CMSHeavyHitters" should { + "A Top-% Count-Min sketch implementing CMSHeavyHitters" should { "estimate heavy hitters" in { // Simple way of making some elements appear much more often than others. @@ -267,34 +269,35 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma val data = data1 ++ data2 ++ data3 // Find elements that appear at least 20% of the time. + val heavyHittersPct = 0.2 val cms = TopPctCMS.monoid[K](EPS, DELTA, SEED, 0.2).create(data) - val trueHhs = exactHeavyHitters(data, cms.heavyHittersPct) + val trueHhs = exactHeavyHitters(data, heavyHittersPct) val estimatedHhs = cms.heavyHitters // All true heavy hitters must be claimed as heavy hitters. - trueHhs.intersect(estimatedHhs) should be (trueHhs) + trueHhs.intersect(estimatedHhs) should be(trueHhs) // It should be very unlikely that any element with count less than // (heavyHittersPct - eps) * totalCount is claimed as a heavy hitter. - val minHhCount = (cms.heavyHittersPct - cms.eps) * cms.totalCount - val infrequent = data.groupBy{ x => x }.mapValues{ _.size }.filter{ _._2 < minHhCount }.keys.toSet - infrequent.intersect(estimatedHhs) should be ('empty) + val minHhCount = (heavyHittersPct - cms.eps) * cms.totalCount + val infrequent = data.groupBy { x => x }.mapValues { _.size }.filter { _._2 < minHhCount }.keys.toSet + infrequent.intersect(estimatedHhs) should be('empty) } "drop old heavy hitters when new heavy hitters replace them" in { val monoid = TopPctCMS.monoid[K](EPS, DELTA, SEED, 0.3) val cms1 = monoid.create(Seq(1, 2, 2).toK[K]) - cms1.heavyHitters should be (Set(1, 2)) + cms1.heavyHitters should be(Set(1, 2)) val cms2 = cms1 ++ monoid.create(2.toK[K]) - cms2.heavyHitters should be (Set(2)) + cms2.heavyHitters should be(Set(2)) val cms3 = cms2 ++ monoid.create(1.toK[K]) - cms3.heavyHitters should be (Set(1, 2)) + cms3.heavyHitters should be(Set(1, 2)) val cms4 = cms3 ++ monoid.create(Seq(0, 0, 0, 0, 0, 0).toK[K]) - cms4.heavyHitters should be (Set(0)) + cms4.heavyHitters should be(Set(0)) } "exactly compute heavy hitters in a small stream" in { @@ -303,10 +306,47 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma val cms2 = TopPctCMS.monoid[K](EPS, DELTA, SEED, 0.1).create(data1) val cms3 = TopPctCMS.monoid[K](EPS, DELTA, SEED, 0.3).create(data1) val cms4 = TopPctCMS.monoid[K](EPS, DELTA, SEED, 0.9).create(data1) - cms1.heavyHitters should be (Set(1, 2, 3, 4, 5)) - cms2.heavyHitters should be (Set(2, 3, 4, 5)) - cms3.heavyHitters should be (Set(5)) - cms4.heavyHitters should be (Set[K]()) + cms1.heavyHitters should be(Set(1, 2, 3, 4, 5)) + cms2.heavyHitters should be(Set(2, 3, 4, 5)) + cms3.heavyHitters should be(Set(5)) + cms4.heavyHitters should be(Set[K]()) + } + + "compute heavy hitters correctly (regression test of GH-353)" in { + val monoid = TopPctCMS.monoid(EPS, DELTA, SEED, 0.1) + + val data1 = Seq(1, 1, 1, 2, 2, 3).toK[K] + val data2 = Seq(3, 4, 4, 4, 5, 5).toK[K] + val data3 = Seq(3, 6, 6, 6, 7, 7).toK[K] + val data4 = Seq(3, 8, 8, 8, 9, 9).toK[K] + val singleData = data1 ++ data2 ++ data3 ++ data4 + + /* + Data sets from above shown in tabular view + + Item 1 2 3 4 total (= singleData) + ---------------------------------------- + A (1) 3 - - - 3 + B (2) 2 - - - 2 + C (3) 1 1 1 1 4 <<< C is global top 1 heavy hitter + D (4) - 3 - - 3 + E (5) - 2 - - 2 + F (6) - - 3 - 3 + G (7) - - 2 - 2 + H (8) - - - 3 3 + I (9) - - - 2 2 + + */ + + val cms1 = monoid.create(data1) + val cms2 = monoid.create(data2) + val cms3 = monoid.create(data3) + val cms4 = monoid.create(data4) + val aggregated = cms1 ++ cms2 ++ cms3 ++ cms4 + + val single = monoid.create(singleData) + aggregated.heavyHitters should be(single.heavyHitters) + aggregated.heavyHitters contains(3.toK[K]) // C=3 is global top 1 heavy hitter } "work as an Aggregator" in { @@ -315,10 +355,64 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma val cms2 = TopPctCMS.aggregator[K](EPS, DELTA, SEED, 0.1).apply(data1) val cms3 = TopPctCMS.aggregator[K](EPS, DELTA, SEED, 0.3).apply(data1) val cms4 = TopPctCMS.aggregator[K](EPS, DELTA, SEED, 0.9).apply(data1) - cms1.heavyHitters should be (Set(1, 2, 3, 4, 5)) - cms2.heavyHitters should be (Set(2, 3, 4, 5)) - cms3.heavyHitters should be (Set(5)) - cms4.heavyHitters should be (Set[K]()) + cms1.heavyHitters should be(Set(1, 2, 3, 4, 5)) + cms2.heavyHitters should be(Set(2, 3, 4, 5)) + cms3.heavyHitters should be(Set(5)) + cms4.heavyHitters should be(Set[K]()) + } + + } + + "A Top-N Count-Min sketch implementing CMSHeavyHitters" should { + + // Note: As we learned in https://github.com/twitter/algebird/issues/353 + // a top-N CMS cannot merge heavy hitters correctly all the time. + // This is because merging top-N based heavy hitters is not an associative + // operation. + // + // We still keep the heavy hitter related tests below in the spec, + // but be aware that in general you cannot trust the heavy hitters + // functionality in top-N CMS implementations. + + "drop old heavy hitters when new heavy hitters replace them" in { + val heavyHittersN = 2 + val monoid = TopNCMS.monoid[K](EPS, DELTA, SEED, heavyHittersN) + val cms1 = monoid.create(Seq(1, 2, 2).toK[K]) // 1x 1, 2x 2 + cms1.heavyHitters should be(Set(1, 2)) + val cms2 = cms1 ++ monoid.create(Seq(3, 3, 3).toK[K]) // 1x 1, 2x 2, 3x 3 + cms2.heavyHitters should be(Set(2, 3)) + val cms3 = cms2 ++ monoid.create(Seq(1, 1, 1).toK[K]) // 4x 1, 2x 2, 3x 3 + cms3.heavyHitters should be(Set(3, 1)) + val cms4 = cms3 ++ monoid.create(Seq(6, 6, 6, 6, 6, 6).toK[K]) // 4x 1, 2x 2, 3x 3, 6x 6 + cms4.heavyHitters should be(Set(1, 6)) + } + + "exactly compute heavy hitters in a small stream" in { + val data1 = Seq(1, 2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5, 5, 5, 5).toK[K] + val cms1 = TopNCMS.monoid[K](EPS, DELTA, SEED, 5).create(data1) + val cms2 = TopNCMS.monoid[K](EPS, DELTA, SEED, 4).create(data1) + val cms3 = TopNCMS.monoid[K](EPS, DELTA, SEED, 3).create(data1) + val cms4 = TopNCMS.monoid[K](EPS, DELTA, SEED, 2).create(data1) + val cms5 = TopNCMS.monoid[K](EPS, DELTA, SEED, 1).create(data1) + cms1.heavyHitters should be(Set(1, 2, 3, 4, 5)) + cms2.heavyHitters should be(Set(2, 3, 4, 5)) + cms3.heavyHitters should be(Set(3, 4, 5)) + cms4.heavyHitters should be(Set(4, 5)) + cms5.heavyHitters should be(Set(5)) + } + + "work as an Aggregator" in { + val data1 = Seq(1, 2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5, 5, 5, 5).toK[K] + val cms1 = TopNCMS.aggregator[K](EPS, DELTA, SEED, 5).apply(data1) + val cms2 = TopNCMS.aggregator[K](EPS, DELTA, SEED, 4).apply(data1) + val cms3 = TopNCMS.aggregator[K](EPS, DELTA, SEED, 3).apply(data1) + val cms4 = TopNCMS.aggregator[K](EPS, DELTA, SEED, 2).apply(data1) + val cms5 = TopNCMS.aggregator[K](EPS, DELTA, SEED, 1).apply(data1) + cms1.heavyHitters should be(Set(1, 2, 3, 4, 5)) + cms2.heavyHitters should be(Set(2, 3, 4, 5)) + cms3.heavyHitters should be(Set(3, 4, 5)) + cms4.heavyHitters should be(Set(4, 5)) + cms5.heavyHitters should be(Set(5)) } } From ba19158950c4e9447e5786b4be1cf4004f29fc1d Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Wed, 15 Oct 2014 14:51:35 +0200 Subject: [PATCH 38/66] Restructure code --- .../com/twitter/algebird/CountMinSketch.scala | 316 +++++++++--------- 1 file changed, 158 insertions(+), 158 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index dbafbebf8..3b582a17c 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -113,6 +113,19 @@ class CMSMonoid[K: Ordering: CMSHasher](eps: Double, delta: Double, seed: Int) e } +/** + * An Aggregator for [[CMS]]. Can be created using [[CMS.aggregator]]. + */ +case class CMSAggregator[K](cmsMonoid: CMSMonoid[K]) extends MonoidAggregator[K, CMS[K], CMS[K]] { + + val monoid = cmsMonoid + + def prepare(value: K): CMS[K] = monoid.create(value) + + def present(cms: CMS[K]): CMS[K] = cms + +} + /** * Configuration parameters for [[CMS]]. * @@ -535,159 +548,8 @@ object CMSInstance { } -/** - * Monoid for adding [[TopCMS]] sketches. - * - * Implicit conversions for commonly used types for `K` such as [[Long]] and [[BigInt]]: - * {{{ - * import com.twitter.algebird.CMSHasherImplicits._ - * }}} - * - * @param cms A [[CMS]] instance, which is used for the counting and the frequency estimation performed by this class. - * @param heavyHittersPct A threshold for finding heavy hitters, i.e., elements that appear at least - * (heavyHittersPct * totalCount) times in the stream. - * @tparam K The type used to identify the elements to be counted. For example, if you want to count the occurrence of - * user names, you could map each username to a unique numeric ID expressed as a `Long`, and then count the - * occurrences of those `Long`s with a CMS of type `K=Long`. Note that this mapping between the elements of - * your problem domain and their identifiers used for counting via CMS should be bijective. - * We require [[Ordering]] and [[CMSHasher]] context bounds for `K`, see [[CMSHasherImplicits]] for available - * implicits that can be imported. - * Which type K should you pick in practice? For domains that have less than `2^64` unique elements, you'd - * typically use [[Long]]. For larger domains you can try [[BigInt]], for example. - */ -class TopPctCMSMonoid[K: Ordering](cms: CMS[K], heavyHittersPct: Double = 0.01) extends Monoid[TopCMS[K]] { - - val params: TopCMSParams[K] = { - val logic = new TopPctHeavyHittersLogic[K](heavyHittersPct) - TopCMSParams[K](logic) - } - - val zero: TopCMS[K] = TopCMSZero[K](cms, params) - - /** - * We assume the sketches on the left and right use the same hash functions. - */ - def plus(left: TopCMS[K], right: TopCMS[K]): TopCMS[K] = left ++ right - - /** - * Create a sketch out of a single item. - */ - def create(item: K): TopCMS[K] = TopCMSItem[K](item, cms, params) - - /** - * Create a sketch out of multiple items. - */ - def create(data: Seq[K]): TopCMS[K] = { - data.foldLeft(zero) { case (acc, x) => plus(acc, create(x)) } - } - -} - -/** - * Monoid for adding [[TopCMS]] sketches. - * - * Implicit conversions for commonly used types for `K` such as [[Long]] and [[BigInt]]: - * {{{ - * import com.twitter.algebird.CMSHasherImplicits._ - * }}} - * - * @param cms A [[CMS]] instance, which is used for the counting and the frequency estimation performed by this class. - * @param heavyHittersN The maximum number of heavy hitters to track. - * @tparam K The type used to identify the elements to be counted. For example, if you want to count the occurrence of - * user names, you could map each username to a unique numeric ID expressed as a `Long`, and then count the - * occurrences of those `Long`s with a CMS of type `K=Long`. Note that this mapping between the elements of - * your problem domain and their identifiers used for counting via CMS should be bijective. - * We require [[Ordering]] and [[CMSHasher]] context bounds for `K`, see [[CMSHasherImplicits]] for available - * implicits that can be imported. - * Which type K should you pick in practice? For domains that have less than `2^64` unique elements, you'd - * typically use [[Long]]. For larger domains you can try [[BigInt]], for example. - */ -class TopNCMSMonoid[K: Ordering](cms: CMS[K], heavyHittersN: Int = 100) extends Monoid[TopCMS[K]] { - - val params: TopCMSParams[K] = { - val logic = new TopNHeavyHittersLogic[K](heavyHittersN) - TopCMSParams[K](logic) - } - - val zero: TopCMS[K] = TopCMSZero[K](cms, params) - - /** - * We assume the sketches on the left and right use the same hash functions. - */ - def plus(left: TopCMS[K], right: TopCMS[K]): TopCMS[K] = left ++ right - - /** - * Create a sketch out of a single item. - */ - def create(item: K): TopCMS[K] = TopCMSItem[K](item, cms, params) - - /** - * Create a sketch out of multiple items. - */ - def create(data: Seq[K]): TopCMS[K] = { - data.foldLeft(zero) { case (acc, x) => plus(acc, create(x)) } - } - -} - case class TopCMSParams[K: Ordering](logic: HeavyHittersLogic[K]) -object TopPctCMS { - - def monoid[K: Ordering: CMSHasher](eps: Double, - delta: Double, - seed: Int, - heavyHittersPct: Double): TopPctCMSMonoid[K] = - new TopPctCMSMonoid[K](CMS(eps, delta, seed), heavyHittersPct) - - def monoid[K: Ordering: CMSHasher](depth: Int, - width: Int, - seed: Int, - heavyHittersPct: Double): TopPctCMSMonoid[K] = - monoid(CMSFunctions.eps(width), CMSFunctions.delta(depth), seed, heavyHittersPct) - - def aggregator[K: Ordering: CMSHasher](eps: Double, - delta: Double, - seed: Int, - heavyHittersPct: Double): TopPctCMSAggregator[K] = - new TopPctCMSAggregator[K](monoid(eps, delta, seed, heavyHittersPct)) - - def aggregator[K: Ordering: CMSHasher](depth: Int, - width: Int, - seed: Int, - heavyHittersPct: Double): TopPctCMSAggregator[K] = - aggregator(CMSFunctions.eps(width), CMSFunctions.delta(depth), seed, heavyHittersPct) - -} - -object TopNCMS { - - def monoid[K: Ordering: CMSHasher](eps: Double, - delta: Double, - seed: Int, - heavyHittersN: Int): TopNCMSMonoid[K] = - new TopNCMSMonoid[K](CMS(eps, delta, seed), heavyHittersN) - - def monoid[K: Ordering: CMSHasher](depth: Int, - width: Int, - seed: Int, - heavyHittersN: Int): TopNCMSMonoid[K] = - monoid(CMSFunctions.eps(width), CMSFunctions.delta(depth), seed, heavyHittersN) - - def aggregator[K: Ordering: CMSHasher](eps: Double, - delta: Double, - seed: Int, - heavyHittersN: Int): TopNCMSAggregator[K] = - new TopNCMSAggregator[K](monoid(eps, delta, seed, heavyHittersN)) - - def aggregator[K: Ordering: CMSHasher](depth: Int, - width: Int, - seed: Int, - heavyHittersN: Int): TopNCMSAggregator[K] = - aggregator(CMSFunctions.eps(width), CMSFunctions.delta(depth), seed, heavyHittersN) - -} - /** * A Count-Min sketch data structure that allows for (a) counting and frequency estimation of elements in a data stream * and (b) tracking the heavy hitters among these elements. @@ -905,15 +767,78 @@ object HeavyHitter { } /** - * An Aggregator for [[CMS]]. Can be created using [[CMS.aggregator]]. + * Monoid for Top-% based [[TopCMS]] sketches. + * + * Implicit conversions for commonly used types for `K` such as [[Long]] and [[BigInt]]: + * {{{ + * import com.twitter.algebird.CMSHasherImplicits._ + * }}} + * + * @param cms A [[CMS]] instance, which is used for the counting and the frequency estimation performed by this class. + * @param heavyHittersPct A threshold for finding heavy hitters, i.e., elements that appear at least + * (heavyHittersPct * totalCount) times in the stream. + * @tparam K The type used to identify the elements to be counted. For example, if you want to count the occurrence of + * user names, you could map each username to a unique numeric ID expressed as a `Long`, and then count the + * occurrences of those `Long`s with a CMS of type `K=Long`. Note that this mapping between the elements of + * your problem domain and their identifiers used for counting via CMS should be bijective. + * We require [[Ordering]] and [[CMSHasher]] context bounds for `K`, see [[CMSHasherImplicits]] for available + * implicits that can be imported. + * Which type K should you pick in practice? For domains that have less than `2^64` unique elements, you'd + * typically use [[Long]]. For larger domains you can try [[BigInt]], for example. */ -case class CMSAggregator[K](cmsMonoid: CMSMonoid[K]) extends MonoidAggregator[K, CMS[K], CMS[K]] { +class TopPctCMSMonoid[K: Ordering](cms: CMS[K], heavyHittersPct: Double = 0.01) extends Monoid[TopCMS[K]] { - val monoid = cmsMonoid + val params: TopCMSParams[K] = { + val logic = new TopPctHeavyHittersLogic[K](heavyHittersPct) + TopCMSParams[K](logic) + } - def prepare(value: K): CMS[K] = monoid.create(value) + val zero: TopCMS[K] = TopCMSZero[K](cms, params) - def present(cms: CMS[K]): CMS[K] = cms + /** + * We assume the sketches on the left and right use the same hash functions. + */ + def plus(left: TopCMS[K], right: TopCMS[K]): TopCMS[K] = left ++ right + + /** + * Create a sketch out of a single item. + */ + def create(item: K): TopCMS[K] = TopCMSItem[K](item, cms, params) + + /** + * Create a sketch out of multiple items. + */ + def create(data: Seq[K]): TopCMS[K] = { + data.foldLeft(zero) { case (acc, x) => plus(acc, create(x)) } + } + +} + +object TopPctCMS { + + def monoid[K: Ordering: CMSHasher](eps: Double, + delta: Double, + seed: Int, + heavyHittersPct: Double): TopPctCMSMonoid[K] = + new TopPctCMSMonoid[K](CMS(eps, delta, seed), heavyHittersPct) + + def monoid[K: Ordering: CMSHasher](depth: Int, + width: Int, + seed: Int, + heavyHittersPct: Double): TopPctCMSMonoid[K] = + monoid(CMSFunctions.eps(width), CMSFunctions.delta(depth), seed, heavyHittersPct) + + def aggregator[K: Ordering: CMSHasher](eps: Double, + delta: Double, + seed: Int, + heavyHittersPct: Double): TopPctCMSAggregator[K] = + new TopPctCMSAggregator[K](monoid(eps, delta, seed, heavyHittersPct)) + + def aggregator[K: Ordering: CMSHasher](depth: Int, + width: Int, + seed: Int, + heavyHittersPct: Double): TopPctCMSAggregator[K] = + aggregator(CMSFunctions.eps(width), CMSFunctions.delta(depth), seed, heavyHittersPct) } @@ -921,7 +846,7 @@ case class CMSAggregator[K](cmsMonoid: CMSMonoid[K]) extends MonoidAggregator[K, * An Aggregator for [[TopPctCMS]]. Can be created using [[TopPctCMS.aggregator]]. */ case class TopPctCMSAggregator[K](cmsMonoid: TopPctCMSMonoid[K]) - extends MonoidAggregator[K, TopCMS[K], TopCMS[K]] { + extends MonoidAggregator[K, TopCMS[K], TopCMS[K]] { val monoid = cmsMonoid @@ -931,11 +856,86 @@ case class TopPctCMSAggregator[K](cmsMonoid: TopPctCMSMonoid[K]) } +/** + * Monoid for Top-N based [[TopCMS]] sketches. + * + * Implicit conversions for commonly used types for `K` such as [[Long]] and [[BigInt]]: + * {{{ + * import com.twitter.algebird.CMSHasherImplicits._ + * }}} + * + * @param cms A [[CMS]] instance, which is used for the counting and the frequency estimation performed by this class. + * @param heavyHittersN The maximum number of heavy hitters to track. + * @tparam K The type used to identify the elements to be counted. For example, if you want to count the occurrence of + * user names, you could map each username to a unique numeric ID expressed as a `Long`, and then count the + * occurrences of those `Long`s with a CMS of type `K=Long`. Note that this mapping between the elements of + * your problem domain and their identifiers used for counting via CMS should be bijective. + * We require [[Ordering]] and [[CMSHasher]] context bounds for `K`, see [[CMSHasherImplicits]] for available + * implicits that can be imported. + * Which type K should you pick in practice? For domains that have less than `2^64` unique elements, you'd + * typically use [[Long]]. For larger domains you can try [[BigInt]], for example. + */ +class TopNCMSMonoid[K: Ordering](cms: CMS[K], heavyHittersN: Int = 100) extends Monoid[TopCMS[K]] { + + val params: TopCMSParams[K] = { + val logic = new TopNHeavyHittersLogic[K](heavyHittersN) + TopCMSParams[K](logic) + } + + val zero: TopCMS[K] = TopCMSZero[K](cms, params) + + /** + * We assume the sketches on the left and right use the same hash functions. + */ + def plus(left: TopCMS[K], right: TopCMS[K]): TopCMS[K] = left ++ right + + /** + * Create a sketch out of a single item. + */ + def create(item: K): TopCMS[K] = TopCMSItem[K](item, cms, params) + + /** + * Create a sketch out of multiple items. + */ + def create(data: Seq[K]): TopCMS[K] = { + data.foldLeft(zero) { case (acc, x) => plus(acc, create(x)) } + } + +} + +object TopNCMS { + + def monoid[K: Ordering: CMSHasher](eps: Double, + delta: Double, + seed: Int, + heavyHittersN: Int): TopNCMSMonoid[K] = + new TopNCMSMonoid[K](CMS(eps, delta, seed), heavyHittersN) + + def monoid[K: Ordering: CMSHasher](depth: Int, + width: Int, + seed: Int, + heavyHittersN: Int): TopNCMSMonoid[K] = + monoid(CMSFunctions.eps(width), CMSFunctions.delta(depth), seed, heavyHittersN) + + def aggregator[K: Ordering: CMSHasher](eps: Double, + delta: Double, + seed: Int, + heavyHittersN: Int): TopNCMSAggregator[K] = + new TopNCMSAggregator[K](monoid(eps, delta, seed, heavyHittersN)) + + def aggregator[K: Ordering: CMSHasher](depth: Int, + width: Int, + seed: Int, + heavyHittersN: Int): TopNCMSAggregator[K] = + aggregator(CMSFunctions.eps(width), CMSFunctions.delta(depth), seed, heavyHittersN) + +} + /** * An Aggregator for [[TopNCMS]]. Can be created using [[TopNCMS.aggregator]]. */ case class TopNCMSAggregator[K](cmsMonoid: TopNCMSMonoid[K]) - extends MonoidAggregator[K, TopCMS[K], TopCMS[K]] { + extends MonoidAggregator[K, TopCMS[K], TopCMS[K]] { val monoid = cmsMonoid From ec1e4f87a6bc9b21278b8fb84e4cb083b34534bb Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Wed, 15 Oct 2014 14:51:55 +0200 Subject: [PATCH 39/66] Explicitly cast to Vector[V] --- .../src/main/scala/com/twitter/algebird/CountMinSketch.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index 3b582a17c..c97fc5b25 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -531,7 +531,7 @@ object CMSInstance { require((depth, width) == (other.depth, other.width), "Tables must have the same dimensions.") val iil: IndexedSeq[IndexedSeq[Long]] = Monoid.plus[IndexedSeq[IndexedSeq[Long]]](counts, other.counts) def toVector[V](is: IndexedSeq[V]): Vector[V] = is match { - case v: Vector[_] => v + case v: Vector[_] => v.asInstanceOf[Vector[V]] case _ => Vector(is: _*) } CountsTable[K](toVector(iil.map { toVector })) From cf7a1509708c3841df8d5d243a6ed64885b4ea04 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Wed, 15 Oct 2014 14:54:10 +0200 Subject: [PATCH 40/66] Document K type --- .../src/main/scala/com/twitter/algebird/CountMinSketch.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index c97fc5b25..2504d4b9e 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -134,7 +134,7 @@ case class CMSAggregator[K](cmsMonoid: CMSMonoid[K]) extends MonoidAggregator[K, * @param eps One-sided error bound on the error of each point query, i.e. frequency estimate. * @param delta A bound on the probability that a query estimate does not lie within some small interval * (an interval that depends on `eps`) around the truth. - * @tparam K + * @tparam K The type used to identify the elements to be counted. */ case class CMSParams[K](hashes: Seq[CMSHash[K]], eps: Double, delta: Double) { From 3cf86f2b67ad8f9ef7f514891dc39f12e4751417 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Wed, 15 Oct 2014 14:57:39 +0200 Subject: [PATCH 41/66] Minor code reformatting (no functional changes) --- .../com/twitter/algebird/CountMinSketch.scala | 36 ++++++++----------- 1 file changed, 15 insertions(+), 21 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index 2504d4b9e..0752d82ee 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -539,11 +539,13 @@ object CMSInstance { } object CountsTable { + /** * Creates a new [[CountsTable]] with counts initialized to all zeroes. */ def apply[K: Ordering](depth: Int, width: Int): CountsTable[K] = CountsTable[K](Vector.fill[Long](depth, width)(0L)) + } } @@ -630,12 +632,10 @@ case class TopCMSItem[K: Ordering](item: K, override val cms: CMS[K], params: To override def +(x: K, count: Long): TopCMS[K] = TopCMSInstance(cms, params) + item + (x, count) - override def ++(other: TopCMS[K]): TopCMS[K] = { - other match { - case other: TopCMSZero[_] => this - case other: TopCMSItem[K] => TopCMSInstance[K](cms, params) + item + other.item - case other: TopCMSInstance[K] => other + item - } + override def ++(other: TopCMS[K]): TopCMS[K] = other match { + case other: TopCMSZero[_] => this + case other: TopCMSItem[K] => TopCMSInstance[K](cms, params) + item + other.item + case other: TopCMSInstance[K] => other + item } } @@ -662,15 +662,13 @@ case class TopCMSInstance[K: Ordering](override val cms: CMS[K], hhs: HeavyHitte } else this } - override def ++(other: TopCMS[K]): TopCMS[K] = { - other match { - case other: TopCMSZero[_] => this - case other: TopCMSItem[K] => this + other.item - case other: TopCMSInstance[K] => - val newCms = cms ++ other.cms - val newHhs = params.logic.updateHeavyHitters(newCms)(hhs, other.hhs) - TopCMSInstance(newCms, newHhs, params) - } + override def ++(other: TopCMS[K]): TopCMS[K] = other match { + case other: TopCMSZero[_] => this + case other: TopCMSItem[K] => this + other.item + case other: TopCMSInstance[K] => + val newCms = cms ++ other.cms + val newHhs = params.logic.updateHeavyHitters(newCms)(hhs, other.hhs) + TopCMSInstance(newCms, newHhs, params) } } @@ -752,9 +750,7 @@ object HeavyHitters { SortedSet[HeavyHitter[K]]() } - def from[K: Ordering](hhs: Set[HeavyHitter[K]]): HeavyHitters[K] = { - HeavyHitters(hhs.foldLeft(emptyHhs)(_ + _)) - } + def from[K: Ordering](hhs: Set[HeavyHitter[K]]): HeavyHitters[K] = HeavyHitters(hhs.foldLeft(emptyHhs)(_ + _)) } @@ -897,9 +893,7 @@ class TopNCMSMonoid[K: Ordering](cms: CMS[K], heavyHittersN: Int = 100) extends /** * Create a sketch out of multiple items. */ - def create(data: Seq[K]): TopCMS[K] = { - data.foldLeft(zero) { case (acc, x) => plus(acc, create(x)) } - } + def create(data: Seq[K]): TopCMS[K] = data.foldLeft(zero) { case (acc, x) => plus(acc, create(x)) } } From c3f36787727b38961eef34c23b62cd56e2552f7b Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Wed, 15 Oct 2014 15:00:30 +0200 Subject: [PATCH 42/66] Add TODO to re-consider returning a sorted list for HeavyHitters#items --- .../src/main/scala/com/twitter/algebird/CountMinSketch.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index 0752d82ee..7ffbf08a6 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -737,6 +737,7 @@ case class HeavyHitters[K: Ordering](hhs: SortedSet[HeavyHitter[K]]) { def ++(other: HeavyHitters[K]): HeavyHitters[K] = HeavyHitters[K](hhs ++ other.hhs) + // TODO: Should we return a descendingly sorted list here? def items: Set[K] = hhs.map { _.item } } From 3273d0bfed3fbe1358e32a984d56b9fe7368c052 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Wed, 15 Oct 2014 15:11:34 +0200 Subject: [PATCH 43/66] Include heavy hitters logic in CMSHeavyHitters trait --- .../com/twitter/algebird/CountMinSketch.scala | 25 ++++++++++++++----- 1 file changed, 19 insertions(+), 6 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index 7ffbf08a6..c5ed9c4b2 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -300,6 +300,11 @@ trait CMSCounting[K, C[_]] { */ trait CMSHeavyHitters[K] { + /** + * The pluggable logic with which heavy hitters are being tracked. + */ + def heavyHittersLogic: HeavyHittersLogic[K] + /** * Returns the set of heavy hitters. */ @@ -606,6 +611,11 @@ sealed abstract class TopCMS[K: Ordering](val cms: CMS[K], params: TopCMSParams[ def f2: Approximate[Long] = innerProduct(this) + /** + * The pluggable logic with which heavy hitters are being tracked. + */ + override def heavyHittersLogic: HeavyHittersLogic[K] = params.logic + } /** @@ -657,7 +667,7 @@ case class TopCMSInstance[K: Ordering](override val cms: CMS[K], hhs: HeavyHitte require(count >= 0, "count must be >= 0 (negative counts not implemented") if (count != 0L) { val newCms = cms + (item, count) - val newHhs = params.logic.updateHeavyHitters(cms, newCms)(hhs, item, count) + val newHhs = heavyHittersLogic.updateHeavyHitters(cms, newCms)(hhs, item, count) TopCMSInstance[K](newCms, newHhs, params) } else this } @@ -667,12 +677,15 @@ case class TopCMSInstance[K: Ordering](override val cms: CMS[K], hhs: HeavyHitte case other: TopCMSItem[K] => this + other.item case other: TopCMSInstance[K] => val newCms = cms ++ other.cms - val newHhs = params.logic.updateHeavyHitters(newCms)(hhs, other.hhs) + val newHhs = heavyHittersLogic.updateHeavyHitters(newCms)(hhs, other.hhs) TopCMSInstance(newCms, newHhs, params) } } +/** + * Controls how a CMS that implements [[CMSHeavyHitters]] tracks heavy hitters. + */ abstract class HeavyHittersLogic[K: Ordering] { def updateHeavyHitters(oldCms: CMS[K], newCms: CMS[K])(hhs: HeavyHitters[K], item: K, count: Long): HeavyHitters[K] = { @@ -705,7 +718,7 @@ abstract class HeavyHittersLogic[K: Ordering] { * 0.25), then at most `1 / 0.01 = 100` items (or `1 / 0.25 = 4` items) will be tracked/returned as heavy hitters. * This parameter can thus control the memory footprint required for tracking heavy hitters. */ -case class TopPctHeavyHittersLogic[K: Ordering](heavyHittersPct: Double) extends HeavyHittersLogic[K] { +case class TopPctLogic[K: Ordering](heavyHittersPct: Double) extends HeavyHittersLogic[K] { require(0 < heavyHittersPct && heavyHittersPct < 1, "heavyHittersPct must lie in (0, 1)") @@ -716,7 +729,7 @@ case class TopPctHeavyHittersLogic[K: Ordering](heavyHittersPct: Double) extends } -case class TopNHeavyHittersLogic[K: Ordering](heavyHittersN: Int) extends HeavyHittersLogic[K] { +case class TopNLogic[K: Ordering](heavyHittersN: Int) extends HeavyHittersLogic[K] { require(heavyHittersN > 0, "heavyHittersN must be > 0") @@ -786,7 +799,7 @@ object HeavyHitter { class TopPctCMSMonoid[K: Ordering](cms: CMS[K], heavyHittersPct: Double = 0.01) extends Monoid[TopCMS[K]] { val params: TopCMSParams[K] = { - val logic = new TopPctHeavyHittersLogic[K](heavyHittersPct) + val logic = new TopPctLogic[K](heavyHittersPct) TopCMSParams[K](logic) } @@ -875,7 +888,7 @@ case class TopPctCMSAggregator[K](cmsMonoid: TopPctCMSMonoid[K]) class TopNCMSMonoid[K: Ordering](cms: CMS[K], heavyHittersN: Int = 100) extends Monoid[TopCMS[K]] { val params: TopCMSParams[K] = { - val logic = new TopNHeavyHittersLogic[K](heavyHittersN) + val logic = new TopNLogic[K](heavyHittersN) TopCMSParams[K](logic) } From f2a1e539b3e0c276ff150d9888c317d0202c22ac Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Wed, 15 Oct 2014 15:17:54 +0200 Subject: [PATCH 44/66] CMSBenchmark: Use correct CMS classes This commit fixes a bug that breaks the algebird-caliper code. The bug was introduced by our refactoring of the CMS functionality in algebird-core. --- .../scala/com/twitter/algebird/caliper/CMSBenchmark.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/algebird-caliper/src/test/scala/com/twitter/algebird/caliper/CMSBenchmark.scala b/algebird-caliper/src/test/scala/com/twitter/algebird/caliper/CMSBenchmark.scala index d42c95cc7..34e2ab83c 100644 --- a/algebird-caliper/src/test/scala/com/twitter/algebird/caliper/CMSBenchmark.scala +++ b/algebird-caliper/src/test/scala/com/twitter/algebird/caliper/CMSBenchmark.scala @@ -1,7 +1,7 @@ package com.twitter.algebird.caliper import com.google.caliper.{ Param, SimpleBenchmark } -import com.twitter.algebird.{ TopCMS, CMSHasherImplicits, TopPctCMSMonoid } +import com.twitter.algebird.{ TopPctCMS, TopCMS, CMSHasherImplicits, TopPctCMSMonoid } /** * Benchmarks the Count-Min sketch implementation in Algebird. @@ -39,12 +39,12 @@ class CMSBenchmark extends SimpleBenchmark { cmsLongMonoid = { val seed = 1 - TopCMS.monoid[Long](eps, delta, seed, heavyHittersPct) + TopPctCMS.monoid[Long](eps, delta, seed, heavyHittersPct) } cmsBigIntMonoid = { val seed = 1 - TopCMS.monoid[BigInt](eps, delta, seed, heavyHittersPct) + TopPctCMS.monoid[BigInt](eps, delta, seed, heavyHittersPct) } random = new scala.util.Random From 317eb3207cf48634166e649c8e3fee09ce48c4f0 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Thu, 16 Oct 2014 09:59:12 +0200 Subject: [PATCH 45/66] Document reasoning behind our use of implicit conversions --- .../test/scala/com/twitter/algebird/CountMinSketchTest.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala index 79d9f1dd5..4390899c0 100644 --- a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala +++ b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala @@ -110,6 +110,9 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma val RAND = new scala.util.Random + // Convenience methods to convert from `Int` to the actual `K` type, and we prefer these conversions to be explicit + // (cf. JavaConverters vs. JavaConversions). We use the name `toK` to clarify the intent and to prevent name conflicts + // with the existing `to[Col]` method in Scala. implicit class IntCast(x: Int) { def toK[A: Numeric]: A = implicitly[Numeric[A]].fromInt(x) } From f4479e42847ef5e55767507360d1ce518acf8fe7 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Thu, 16 Oct 2014 09:59:50 +0200 Subject: [PATCH 46/66] Remove unneeded comment --- .../test/scala/com/twitter/algebird/CountMinSketchTest.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala index 4390899c0..478267401 100644 --- a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala +++ b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala @@ -6,8 +6,6 @@ import org.scalacheck.{ Gen, Arbitrary } import CMSHasherImplicits._ -// required, although e.g. IntelliJ IDEA may flag it as unused import - class CmsLaws extends PropSpec with PropertyChecks with Matchers { import BaseProperties._ From 5daf4542334cca317b05e32ba0884439c0f5b937 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Thu, 16 Oct 2014 10:05:13 +0200 Subject: [PATCH 47/66] Clarify why we keep the top-N CMS tests in the spec --- .../com/twitter/algebird/CountMinSketchTest.scala | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala index 478267401..febd986ae 100644 --- a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala +++ b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala @@ -366,14 +366,13 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma "A Top-N Count-Min sketch implementing CMSHeavyHitters" should { - // Note: As we learned in https://github.com/twitter/algebird/issues/353 - // a top-N CMS cannot merge heavy hitters correctly all the time. - // This is because merging top-N based heavy hitters is not an associative + // Note: As described in https://github.com/twitter/algebird/issues/353, a top-N CMS is, in general, not able to + // merge heavy hitters correctly. This is because merging top-N based heavy hitters is not an associative // operation. // - // We still keep the heavy hitter related tests below in the spec, - // but be aware that in general you cannot trust the heavy hitters - // functionality in top-N CMS implementations. + // We still keep the heavy hitter related tests (which work because of "suitable" characteristics of the input data) + // below in the spec, but be aware that, in general, merging top-N CMS instances is an unsafe operation, where + // "unsafe" means it may lead to biased results. "drop old heavy hitters when new heavy hitters replace them" in { val heavyHittersN = 2 From 10423030c597a65682d3ea74a5f0de8a7104e867 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Thu, 16 Oct 2014 10:34:19 +0200 Subject: [PATCH 48/66] Add warning about unsafe merge operation for top-N CMS --- .../com/twitter/algebird/CountMinSketch.scala | 33 ++++++++++++++++++- 1 file changed, 32 insertions(+), 1 deletion(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index c5ed9c4b2..b71e48e29 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -733,6 +733,15 @@ case class TopNLogic[K: Ordering](heavyHittersN: Int) extends HeavyHittersLogic[ require(heavyHittersN > 0, "heavyHittersN must be > 0") + /** + * '''Warning:''' top-N computations are not associative. The effect is that a top-N CMS has an ordering bias (with + * regard to heavy hitters) when merging instances. This means merging heavy hitters across CMS instances may lead to + * incorrect, biased results: the outcome is biased by the order in which CMS instances / heavy hitters are being + * merged, with the rule of thumb being that the earlier a set of heavy hitters is being merged, the more likely is + * the end result biased towards these heavy hitters. + * + * @see Discussion in [[https://github.com/twitter/algebird/issues/353 Algebird issue 353]] + */ override def purgeHeavyHitters(cms: CMS[K])(hitters: HeavyHitters[K]): HeavyHitters[K] = { HeavyHitters[K](hitters.hhs.takeRight(heavyHittersN)) } @@ -867,7 +876,29 @@ case class TopPctCMSAggregator[K](cmsMonoid: TopPctCMSMonoid[K]) } /** - * Monoid for Top-N based [[TopCMS]] sketches. + * Monoid for top-N based [[TopCMS]] sketches. '''Use with care! (see warning below)''' + * + * =Warning: Unsafe merge operation= + * + * Top-N computations are not associative. The effect is that a top-N CMS has an ordering bias (with regard to heavy + * hitters) when ''merging'' CMS instances (e.g. via `++`). This means merging heavy hitters across CMS instances may + * lead to incorrect, biased results: the outcome is biased by the order in which CMS instances / heavy hitters are + * being merged, with the rule of thumb being that the earlier a set of heavy hitters is being merged, the more likely + * is the end result biased towards these heavy hitters. + * + * The warning above only applies when ''merging'' CMS instances. That is, a given top-N CMS instance will correctly + * compute its own heavy hitters. + * + * See the discussion in [[https://github.com/twitter/algebird/issues/353 Algebird issue 353]] for further details. + * + * =Alternatives= + * + * The following, alternative data structures may be better picks than a top-N based CMS: + * + * - [[TopPctCMS]]: Has safe merge semantics for its instances including heavy hitters. + * - [[SpaceSaver]]: Has the same ordering bias than a top-N CMS, but at least it provides bounds on the bias. + * + * =Usage= * * Implicit conversions for commonly used types for `K` such as [[Long]] and [[BigInt]]: * {{{ From 680c1e97c99a216d7fb961c880dfc2b90ceb6212 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Thu, 16 Oct 2014 10:35:37 +0200 Subject: [PATCH 49/66] Clarify semantics of updateHeavyHitters --- .../src/main/scala/com/twitter/algebird/CountMinSketch.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index b71e48e29..7fc790620 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -697,9 +697,8 @@ abstract class HeavyHittersLogic[K: Ordering] { } def updateHeavyHitters(cms: CMS[K])(left: HeavyHitters[K], right: HeavyHitters[K]) = { - val allItems = left.items ++ right.items - val hitters = allItems.map { case i => HeavyHitter[K](i, cms.frequency(i).estimate) } - val newHhs = HeavyHitters.from(hitters) + val candidates = (left.items ++ right.items).map { case i => HeavyHitter[K](i, cms.frequency(i).estimate) } + val newHhs = HeavyHitters.from(candidates) purgeHeavyHitters(cms)(newHhs) } From 855f54fd7a564a49c4f8edf0e9b0cbbc280af683 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Thu, 16 Oct 2014 10:36:05 +0200 Subject: [PATCH 50/66] Remove TODOs --- .../src/main/scala/com/twitter/algebird/CountMinSketch.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index 7fc790620..eb9900570 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -308,7 +308,6 @@ trait CMSHeavyHitters[K] { /** * Returns the set of heavy hitters. */ - // TODO: Should we change the contract to return a descendingly sorted list of heavy hitters (heaviest = first)? def heavyHitters: Set[K] } @@ -758,7 +757,6 @@ case class HeavyHitters[K: Ordering](hhs: SortedSet[HeavyHitter[K]]) { def ++(other: HeavyHitters[K]): HeavyHitters[K] = HeavyHitters[K](hhs ++ other.hhs) - // TODO: Should we return a descendingly sorted list here? def items: Set[K] = hhs.map { _.item } } From c4a578f8f9a6a5dfde377c28069479e8b055b703 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Thu, 16 Oct 2014 10:39:52 +0200 Subject: [PATCH 51/66] Update scaladoc of TopCMS to match latest code --- .../scala/com/twitter/algebird/CountMinSketch.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index eb9900570..a2718cd7c 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -560,6 +560,8 @@ case class TopCMSParams[K: Ordering](logic: HeavyHittersLogic[K]) * A Count-Min sketch data structure that allows for (a) counting and frequency estimation of elements in a data stream * and (b) tracking the heavy hitters among these elements. * + * The logic of how heavy hitters are computed is pluggable, see [[HeavyHittersLogic]]. + * * Tip: If you do not need to track heavy hitters, take a look at [[CMS]], which is more efficient in this case. * * =Usage= @@ -583,14 +585,14 @@ case class TopCMSParams[K: Ordering](logic: HeavyHittersLogic[K]) * TopPctCMS.monoid[Long](eps, delta, seed, heavyHittersPct) * } * - * // Create a TopPctCMS instance that has counted the element `1L`. - * val topPctCMS: TopPctCMS[Long] = topPctCMSMonoid.create(1L) + * // Create a TopCMS instance that has counted the element `1L`. + * val topCMS: TopCMS[Long] = topPctCMSMonoid.create(1L) * * // Estimate frequency of `1L` - * val estimate: Approximate[Long] = topPctCMS.frequency(1L) + * val estimate: Approximate[Long] = topCMS.frequency(1L) * * // What are the heavy hitters so far? - * val heavyHitters: Set[Long] = topPctCMS.heavyHitters + * val heavyHitters: Set[Long] = topCMS.heavyHitters * }}} * * @tparam K The type used to identify the elements to be counted. From d2bcb72ebefdd20deb987ac8265babc023a0bc5b Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Thu, 16 Oct 2014 10:46:25 +0200 Subject: [PATCH 52/66] Clarify scaladoc comment of heavyHittersLogic --- .../src/main/scala/com/twitter/algebird/CountMinSketch.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index a2718cd7c..686196fe1 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -301,7 +301,7 @@ trait CMSCounting[K, C[_]] { trait CMSHeavyHitters[K] { /** - * The pluggable logic with which heavy hitters are being tracked. + * The pluggable logic of how heavy hitters are being tracked. */ def heavyHittersLogic: HeavyHittersLogic[K] From 840f180e421d2ce1499104706fbd2d40524d2678 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Thu, 16 Oct 2014 10:52:27 +0200 Subject: [PATCH 53/66] Improve docs --- .../com/twitter/algebird/CountMinSketch.scala | 20 ++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index 686196fe1..4e87d21df 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -729,19 +729,21 @@ case class TopPctLogic[K: Ordering](heavyHittersPct: Double) extends HeavyHitter } +/** + * Tracks the top N heavy hitters, where `N` is defined by `heavyHittersN`. + * + * '''Warning:''' top-N computations are not associative. The effect is that a top-N CMS has an ordering bias (with + * regard to heavy hitters) when merging instances. This means merging heavy hitters across CMS instances may lead to + * incorrect, biased results: the outcome is biased by the order in which CMS instances / heavy hitters are being + * merged, with the rule of thumb being that the earlier a set of heavy hitters is being merged, the more likely is + * the end result biased towards these heavy hitters. + * + * @see Discussion in [[https://github.com/twitter/algebird/issues/353 Algebird issue 353]] + */ case class TopNLogic[K: Ordering](heavyHittersN: Int) extends HeavyHittersLogic[K] { require(heavyHittersN > 0, "heavyHittersN must be > 0") - /** - * '''Warning:''' top-N computations are not associative. The effect is that a top-N CMS has an ordering bias (with - * regard to heavy hitters) when merging instances. This means merging heavy hitters across CMS instances may lead to - * incorrect, biased results: the outcome is biased by the order in which CMS instances / heavy hitters are being - * merged, with the rule of thumb being that the earlier a set of heavy hitters is being merged, the more likely is - * the end result biased towards these heavy hitters. - * - * @see Discussion in [[https://github.com/twitter/algebird/issues/353 Algebird issue 353]] - */ override def purgeHeavyHitters(cms: CMS[K])(hitters: HeavyHitters[K]): HeavyHitters[K] = { HeavyHitters[K](hitters.hhs.takeRight(heavyHittersN)) } From 989f709ddba2371a4e19468741d084755e044a9c Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Thu, 16 Oct 2014 11:14:36 +0200 Subject: [PATCH 54/66] Require the same hash functions for sketches when merging them --- .../com/twitter/algebird/CountMinSketch.scala | 87 +++++++++++-------- .../twitter/algebird/CountMinSketchTest.scala | 2 +- 2 files changed, 52 insertions(+), 37 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index 4e87d21df..58e4950bf 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -97,9 +97,14 @@ class CMSMonoid[K: Ordering: CMSHasher](eps: Double, delta: Double, seed: Int) e val zero: CMS[K] = CMSZero[K](params) /** - * We assume the sketches on the left and right use the same hash functions. + * Combines the two sketches. + * + * The sketches must use the same hash functions. */ - def plus(left: CMS[K], right: CMS[K]): CMS[K] = left ++ right + def plus(left: CMS[K], right: CMS[K]): CMS[K] = { + require(left.params.hashes == right.params.hashes, "The sketches must use the same hash functions.") + left ++ right + } /** * Create a sketch out of a single item. @@ -373,7 +378,7 @@ object CMS { * * @tparam K The type used to identify the elements to be counted. */ -sealed abstract class CMS[K: Ordering](params: CMSParams[K]) extends java.io.Serializable with CMSCounting[K, CMS] { +sealed abstract class CMS[K: Ordering](val params: CMSParams[K]) extends java.io.Serializable with CMSCounting[K, CMS] { override val eps: Double = params.eps @@ -386,7 +391,7 @@ sealed abstract class CMS[K: Ordering](params: CMSParams[K]) extends java.io.Ser /** * Zero element. Used for initialization. */ -case class CMSZero[K: Ordering](params: CMSParams[K]) extends CMS[K](params) { +case class CMSZero[K: Ordering](override val params: CMSParams[K]) extends CMS[K](params) { override val totalCount: Long = 0L @@ -403,7 +408,7 @@ case class CMSZero[K: Ordering](params: CMSParams[K]) extends CMS[K](params) { /** * Used for holding a single element, to avoid repeatedly adding elements from sparse counts tables. */ -case class CMSItem[K: Ordering](item: K, params: CMSParams[K]) extends CMS[K](params) { +case class CMSItem[K: Ordering](item: K, override val params: CMSParams[K]) extends CMS[K](params) { override val totalCount: Long = 1L @@ -428,7 +433,7 @@ case class CMSItem[K: Ordering](item: K, params: CMSParams[K]) extends CMS[K](pa */ case class CMSInstance[K: Ordering](countsTable: CMSInstance.CountsTable[K], override val totalCount: Long, - params: CMSParams[K]) extends CMS[K](params) { + override val params: CMSParams[K]) extends CMS[K](params) { def ++(other: CMS[K]): CMS[K] = { other match { @@ -816,9 +821,14 @@ class TopPctCMSMonoid[K: Ordering](cms: CMS[K], heavyHittersPct: Double = 0.01) val zero: TopCMS[K] = TopCMSZero[K](cms, params) /** - * We assume the sketches on the left and right use the same hash functions. + * Combines the two sketches. + * + * The sketches must use the same hash functions. */ - def plus(left: TopCMS[K], right: TopCMS[K]): TopCMS[K] = left ++ right + def plus(left: TopCMS[K], right: TopCMS[K]): TopCMS[K] = { + require(left.cms.params.hashes == right.cms.params.hashes, "The sketches must use the same hash functions.") + left ++ right + } /** * Create a sketch out of a single item. @@ -837,27 +847,27 @@ class TopPctCMSMonoid[K: Ordering](cms: CMS[K], heavyHittersPct: Double = 0.01) object TopPctCMS { def monoid[K: Ordering: CMSHasher](eps: Double, - delta: Double, - seed: Int, - heavyHittersPct: Double): TopPctCMSMonoid[K] = + delta: Double, + seed: Int, + heavyHittersPct: Double): TopPctCMSMonoid[K] = new TopPctCMSMonoid[K](CMS(eps, delta, seed), heavyHittersPct) def monoid[K: Ordering: CMSHasher](depth: Int, - width: Int, - seed: Int, - heavyHittersPct: Double): TopPctCMSMonoid[K] = + width: Int, + seed: Int, + heavyHittersPct: Double): TopPctCMSMonoid[K] = monoid(CMSFunctions.eps(width), CMSFunctions.delta(depth), seed, heavyHittersPct) def aggregator[K: Ordering: CMSHasher](eps: Double, - delta: Double, - seed: Int, - heavyHittersPct: Double): TopPctCMSAggregator[K] = + delta: Double, + seed: Int, + heavyHittersPct: Double): TopPctCMSAggregator[K] = new TopPctCMSAggregator[K](monoid(eps, delta, seed, heavyHittersPct)) def aggregator[K: Ordering: CMSHasher](depth: Int, - width: Int, - seed: Int, - heavyHittersPct: Double): TopPctCMSAggregator[K] = + width: Int, + seed: Int, + heavyHittersPct: Double): TopPctCMSAggregator[K] = aggregator(CMSFunctions.eps(width), CMSFunctions.delta(depth), seed, heavyHittersPct) } @@ -866,7 +876,7 @@ object TopPctCMS { * An Aggregator for [[TopPctCMS]]. Can be created using [[TopPctCMS.aggregator]]. */ case class TopPctCMSAggregator[K](cmsMonoid: TopPctCMSMonoid[K]) - extends MonoidAggregator[K, TopCMS[K], TopCMS[K]] { + extends MonoidAggregator[K, TopCMS[K], TopCMS[K]] { val monoid = cmsMonoid @@ -927,9 +937,14 @@ class TopNCMSMonoid[K: Ordering](cms: CMS[K], heavyHittersN: Int = 100) extends val zero: TopCMS[K] = TopCMSZero[K](cms, params) /** - * We assume the sketches on the left and right use the same hash functions. + * Combines the two sketches. + * + * The sketches must use the same hash functions. */ - def plus(left: TopCMS[K], right: TopCMS[K]): TopCMS[K] = left ++ right + def plus(left: TopCMS[K], right: TopCMS[K]): TopCMS[K] = { + require(left.cms.params.hashes == right.cms.params.hashes, "The sketches must use the same hash functions.") + left ++ right + } /** * Create a sketch out of a single item. @@ -946,27 +961,27 @@ class TopNCMSMonoid[K: Ordering](cms: CMS[K], heavyHittersN: Int = 100) extends object TopNCMS { def monoid[K: Ordering: CMSHasher](eps: Double, - delta: Double, - seed: Int, - heavyHittersN: Int): TopNCMSMonoid[K] = + delta: Double, + seed: Int, + heavyHittersN: Int): TopNCMSMonoid[K] = new TopNCMSMonoid[K](CMS(eps, delta, seed), heavyHittersN) def monoid[K: Ordering: CMSHasher](depth: Int, - width: Int, - seed: Int, - heavyHittersN: Int): TopNCMSMonoid[K] = + width: Int, + seed: Int, + heavyHittersN: Int): TopNCMSMonoid[K] = monoid(CMSFunctions.eps(width), CMSFunctions.delta(depth), seed, heavyHittersN) def aggregator[K: Ordering: CMSHasher](eps: Double, - delta: Double, - seed: Int, - heavyHittersN: Int): TopNCMSAggregator[K] = + delta: Double, + seed: Int, + heavyHittersN: Int): TopNCMSAggregator[K] = new TopNCMSAggregator[K](monoid(eps, delta, seed, heavyHittersN)) def aggregator[K: Ordering: CMSHasher](depth: Int, - width: Int, - seed: Int, - heavyHittersN: Int): TopNCMSAggregator[K] = + width: Int, + seed: Int, + heavyHittersN: Int): TopNCMSAggregator[K] = aggregator(CMSFunctions.eps(width), CMSFunctions.delta(depth), seed, heavyHittersN) } @@ -975,7 +990,7 @@ object TopNCMS { * An Aggregator for [[TopNCMS]]. Can be created using [[TopNCMS.aggregator]]. */ case class TopNCMSAggregator[K](cmsMonoid: TopNCMSMonoid[K]) - extends MonoidAggregator[K, TopCMS[K], TopCMS[K]] { + extends MonoidAggregator[K, TopCMS[K], TopCMS[K]] { val monoid = cmsMonoid diff --git a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala index febd986ae..56952ff20 100644 --- a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala +++ b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala @@ -347,7 +347,7 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma val single = monoid.create(singleData) aggregated.heavyHitters should be(single.heavyHitters) - aggregated.heavyHitters contains(3.toK[K]) // C=3 is global top 1 heavy hitter + aggregated.heavyHitters contains (3.toK[K]) // C=3 is global top 1 heavy hitter } "work as an Aggregator" in { From 15563d3c1596496680d37bf5c30188e6376cedd9 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Thu, 16 Oct 2014 11:20:18 +0200 Subject: [PATCH 55/66] Use scaladoc wording recommended by Scala style guide --- .../com/twitter/algebird/CountMinSketch.scala | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index 58e4950bf..e7ca3e462 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -107,12 +107,12 @@ class CMSMonoid[K: Ordering: CMSHasher](eps: Double, delta: Double, seed: Int) e } /** - * Create a sketch out of a single item. + * Creates a sketch out of a single item. */ def create(item: K): CMS[K] = CMSItem[K](item, params) /** - * Create a sketch out of multiple items. + * Creates a sketch out of multiple items. */ def create(data: Seq[K]): CMS[K] = data.foldLeft(zero) { case (acc, x) => plus(acc, create(x)) } @@ -361,7 +361,7 @@ object CMS { * // Implicits that enabling CMS-hashing of `Long` values. * import com.twitter.algebird.CMSHasherImplicits._ * - * // Create a monoid for a CMS that can count `Long` elements. + * // Creates a monoid for a CMS that can count `Long` elements. * val cmsMonoid: CMSMonoid[Long] = { * val eps = 0.001 * val delta = 1E-10 @@ -369,10 +369,10 @@ object CMS { * CMS.monoid[Long](eps, delta, seed) * } * - * // Create a CMS instance that has counted the element `1L`. + * // Creates a CMS instance that has counted the element `1L`. * val cms: CMS[Long] = cmsMonoid.create(1L) * - * // Estimate frequency of `1L` + * // Estimates the frequency of `1L` * val estimate: Approximate[Long] = cms.frequency(1L) * }}} * @@ -581,7 +581,7 @@ case class TopCMSParams[K: Ordering](logic: HeavyHittersLogic[K]) * // Implicits that enabling CMS-hashing of `Long` values. * import com.twitter.algebird.CMSHasherImplicits._ * - * // Create a monoid for a CMS that can count `Long` elements. + * // Creates a monoid for a CMS that can count `Long` elements. * val topPctCMSMonoid: TopPctCMSMonoid[Long] = { * val eps = 0.001 * val delta = 1E-10 @@ -590,10 +590,10 @@ case class TopCMSParams[K: Ordering](logic: HeavyHittersLogic[K]) * TopPctCMS.monoid[Long](eps, delta, seed, heavyHittersPct) * } * - * // Create a TopCMS instance that has counted the element `1L`. + * // Creates a TopCMS instance that has counted the element `1L`. * val topCMS: TopCMS[Long] = topPctCMSMonoid.create(1L) * - * // Estimate frequency of `1L` + * // Estimates the frequency of `1L` * val estimate: Approximate[Long] = topCMS.frequency(1L) * * // What are the heavy hitters so far? @@ -831,12 +831,12 @@ class TopPctCMSMonoid[K: Ordering](cms: CMS[K], heavyHittersPct: Double = 0.01) } /** - * Create a sketch out of a single item. + * Creates a sketch out of a single item. */ def create(item: K): TopCMS[K] = TopCMSItem[K](item, cms, params) /** - * Create a sketch out of multiple items. + * Creates a sketch out of multiple items. */ def create(data: Seq[K]): TopCMS[K] = { data.foldLeft(zero) { case (acc, x) => plus(acc, create(x)) } @@ -947,12 +947,12 @@ class TopNCMSMonoid[K: Ordering](cms: CMS[K], heavyHittersN: Int = 100) extends } /** - * Create a sketch out of a single item. + * Creates a sketch out of a single item. */ def create(item: K): TopCMS[K] = TopCMSItem[K](item, cms, params) /** - * Create a sketch out of multiple items. + * Creates a sketch out of multiple items. */ def create(data: Seq[K]): TopCMS[K] = data.foldLeft(zero) { case (acc, x) => plus(acc, create(x)) } From 3a1441de3b528a43824ba54c96245cc8f6ccb153 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Thu, 16 Oct 2014 11:30:48 +0200 Subject: [PATCH 56/66] Update top-N CMS tests to clarify top-N limitations --- .../twitter/algebird/CountMinSketchTest.scala | 31 ++++++++++--------- 1 file changed, 17 insertions(+), 14 deletions(-) diff --git a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala index 56952ff20..b5ad0437c 100644 --- a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala +++ b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala @@ -369,35 +369,38 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma // Note: As described in https://github.com/twitter/algebird/issues/353, a top-N CMS is, in general, not able to // merge heavy hitters correctly. This is because merging top-N based heavy hitters is not an associative // operation. - // - // We still keep the heavy hitter related tests (which work because of "suitable" characteristics of the input data) - // below in the spec, but be aware that, in general, merging top-N CMS instances is an unsafe operation, where - // "unsafe" means it may lead to biased results. - "drop old heavy hitters when new heavy hitters replace them" in { + // This test involves merging of top-N CMS instances, which is not an associative operation. This means that the + // success or failure of this test depends on the merging order and/or the test data characteristics. + "drop old heavy hitters when new heavy hitters replace them (positive test case)" in { val heavyHittersN = 2 val monoid = TopNCMS.monoid[K](EPS, DELTA, SEED, heavyHittersN) - val cms1 = monoid.create(Seq(1, 2, 2).toK[K]) // 1x 1, 2x 2 + val cms1 = monoid.create(Seq(1, 2, 2).toK[K]) cms1.heavyHitters should be(Set(1, 2)) - val cms2 = cms1 ++ monoid.create(Seq(3, 3, 3).toK[K]) // 1x 1, 2x 2, 3x 3 + val cms2 = cms1 ++ monoid.create(Seq(3, 3, 3).toK[K]) cms2.heavyHitters should be(Set(2, 3)) - val cms3 = cms2 ++ monoid.create(Seq(1, 1, 1).toK[K]) // 4x 1, 2x 2, 3x 3 + val cms3 = cms2 ++ monoid.create(Seq(1, 1, 1).toK[K]) cms3.heavyHitters should be(Set(3, 1)) - val cms4 = cms3 ++ monoid.create(Seq(6, 6, 6, 6, 6, 6).toK[K]) // 4x 1, 2x 2, 3x 3, 6x 6 + val cms4 = cms3 ++ monoid.create(Seq(6, 6, 6, 6, 6, 6).toK[K]) cms4.heavyHitters should be(Set(1, 6)) } - "exactly compute heavy hitters in a small stream" in { + "exactly compute its own heavy hitters in a small stream" in { val data1 = Seq(1, 2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5, 5, 5, 5).toK[K] + val cms1 = TopNCMS.monoid[K](EPS, DELTA, SEED, 5).create(data1) - val cms2 = TopNCMS.monoid[K](EPS, DELTA, SEED, 4).create(data1) - val cms3 = TopNCMS.monoid[K](EPS, DELTA, SEED, 3).create(data1) - val cms4 = TopNCMS.monoid[K](EPS, DELTA, SEED, 2).create(data1) - val cms5 = TopNCMS.monoid[K](EPS, DELTA, SEED, 1).create(data1) cms1.heavyHitters should be(Set(1, 2, 3, 4, 5)) + + val cms2 = TopNCMS.monoid[K](EPS, DELTA, SEED, 4).create(data1) cms2.heavyHitters should be(Set(2, 3, 4, 5)) + + val cms3 = TopNCMS.monoid[K](EPS, DELTA, SEED, 3).create(data1) cms3.heavyHitters should be(Set(3, 4, 5)) + + val cms4 = TopNCMS.monoid[K](EPS, DELTA, SEED, 2).create(data1) cms4.heavyHitters should be(Set(4, 5)) + + val cms5 = TopNCMS.monoid[K](EPS, DELTA, SEED, 1).create(data1) cms5.heavyHitters should be(Set(5)) } From 47416ea7acd81b477efc3da280aa36afe7ef3a89 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Thu, 16 Oct 2014 11:31:54 +0200 Subject: [PATCH 57/66] Clarify test case of top-% CMS --- .../com/twitter/algebird/CountMinSketchTest.scala | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala index b5ad0437c..8f5759878 100644 --- a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala +++ b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala @@ -301,15 +301,19 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma cms4.heavyHitters should be(Set(0)) } - "exactly compute heavy hitters in a small stream" in { + "exactly compute its own heavy hitters in a small stream" in { val data1 = Seq(1, 2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5, 5, 5, 5).toK[K] + val cms1 = TopPctCMS.monoid[K](EPS, DELTA, SEED, 0.01).create(data1) - val cms2 = TopPctCMS.monoid[K](EPS, DELTA, SEED, 0.1).create(data1) - val cms3 = TopPctCMS.monoid[K](EPS, DELTA, SEED, 0.3).create(data1) - val cms4 = TopPctCMS.monoid[K](EPS, DELTA, SEED, 0.9).create(data1) cms1.heavyHitters should be(Set(1, 2, 3, 4, 5)) + + val cms2 = TopPctCMS.monoid[K](EPS, DELTA, SEED, 0.1).create(data1) cms2.heavyHitters should be(Set(2, 3, 4, 5)) + + val cms3 = TopPctCMS.monoid[K](EPS, DELTA, SEED, 0.3).create(data1) cms3.heavyHitters should be(Set(5)) + + val cms4 = TopPctCMS.monoid[K](EPS, DELTA, SEED, 0.9).create(data1) cms4.heavyHitters should be(Set[K]()) } From bd5c279aff9973782b9809b4bb784fe86d90ce9f Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Thu, 16 Oct 2014 11:33:48 +0200 Subject: [PATCH 58/66] Clarify merge tests --- .../test/scala/com/twitter/algebird/CountMinSketchTest.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala index 8f5759878..2042e6b57 100644 --- a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala +++ b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala @@ -286,7 +286,7 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma infrequent.intersect(estimatedHhs) should be('empty) } - "drop old heavy hitters when new heavy hitters replace them" in { + ", when adding CMS instances, drop old heavy hitters when new heavy hitters replace them" in { val monoid = TopPctCMS.monoid[K](EPS, DELTA, SEED, 0.3) val cms1 = monoid.create(Seq(1, 2, 2).toK[K]) cms1.heavyHitters should be(Set(1, 2)) @@ -376,7 +376,7 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma // This test involves merging of top-N CMS instances, which is not an associative operation. This means that the // success or failure of this test depends on the merging order and/or the test data characteristics. - "drop old heavy hitters when new heavy hitters replace them (positive test case)" in { + ", when adding CMS instances, drop old heavy hitters when new heavy hitters replace them (positive test case)" in { val heavyHittersN = 2 val monoid = TopNCMS.monoid[K](EPS, DELTA, SEED, heavyHittersN) val cms1 = monoid.create(Seq(1, 2, 2).toK[K]) From 90490f3d137c903d23122376b8fed55fe497cd48 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Thu, 16 Oct 2014 12:12:16 +0200 Subject: [PATCH 59/66] Improve spec of CMS, e.g. new tests added --- .../twitter/algebird/CountMinSketchTest.scala | 182 +++++++++++++++--- 1 file changed, 157 insertions(+), 25 deletions(-) diff --git a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala index 2042e6b57..4b6b5f123 100644 --- a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala +++ b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala @@ -237,7 +237,7 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma COUNTING_CMS_MONOID.create(d1).innerProduct(COUNTING_CMS_MONOID.create(d2)).estimate should be(6) } - "work as an Aggregator" in { + "work as an Aggregator when created from a single, small stream" in { val data1 = Seq(1, 2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5, 5, 5, 5).toK[K] val cms = CMS.aggregator[K](EPS, DELTA, SEED).apply(data1) @@ -256,6 +256,16 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma topPctCMS.frequency(3.toK[K]).estimate should be(3L) topPctCMS.frequency(4.toK[K]).estimate should be(4L) topPctCMS.frequency(5.toK[K]).estimate should be(5L) + + val topNCMS = { + val anyHeavyHittersN = 1 // exact setting not relevant for this test + TopNCMS.aggregator[K](EPS, DELTA, SEED, anyHeavyHittersN).apply(data1) + } + topNCMS.frequency(1.toK[K]).estimate should be(1L) + topNCMS.frequency(2.toK[K]).estimate should be(2L) + topNCMS.frequency(3.toK[K]).estimate should be(3L) + topNCMS.frequency(4.toK[K]).estimate should be(4L) + topNCMS.frequency(5.toK[K]).estimate should be(5L) } } @@ -286,7 +296,7 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma infrequent.intersect(estimatedHhs) should be('empty) } - ", when adding CMS instances, drop old heavy hitters when new heavy hitters replace them" in { + "(when adding CMS instances) drop old heavy hitters when new heavy hitters replace them" in { val monoid = TopPctCMS.monoid[K](EPS, DELTA, SEED, 0.3) val cms1 = monoid.create(Seq(1, 2, 2).toK[K]) cms1.heavyHitters should be(Set(1, 2)) @@ -301,23 +311,24 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma cms4.heavyHitters should be(Set(0)) } - "exactly compute its own heavy hitters in a small stream" in { - val data1 = Seq(1, 2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5, 5, 5, 5).toK[K] - - val cms1 = TopPctCMS.monoid[K](EPS, DELTA, SEED, 0.01).create(data1) - cms1.heavyHitters should be(Set(1, 2, 3, 4, 5)) + "(when adding individual items) drop old heavy hitters when new heavy hitters replace them" in { + val monoid = TopPctCMS.monoid[K](EPS, DELTA, SEED, 0.3) + val cms1 = monoid.create(Seq(1, 2, 2).toK[K]) + cms1.heavyHitters should be(Set(1, 2)) - val cms2 = TopPctCMS.monoid[K](EPS, DELTA, SEED, 0.1).create(data1) - cms2.heavyHitters should be(Set(2, 3, 4, 5)) + val cms2 = cms1 + 2.toK[K] + cms2.heavyHitters should be(Set(2)) - val cms3 = TopPctCMS.monoid[K](EPS, DELTA, SEED, 0.3).create(data1) - cms3.heavyHitters should be(Set(5)) + val cms3 = cms2 + 1.toK[K] + cms3.heavyHitters should be(Set(1, 2)) - val cms4 = TopPctCMS.monoid[K](EPS, DELTA, SEED, 0.9).create(data1) - cms4.heavyHitters should be(Set[K]()) + val heaviest = 0.toK[K] + val cms4 = cms3 + heaviest + heaviest + heaviest + heaviest + heaviest + heaviest + cms4.heavyHitters should be(Set(heaviest)) } - "compute heavy hitters correctly (regression test of GH-353)" in { + "(when adding CMS instances) merge heavy hitters correctly [GH-353 regression test]" in { + // See https://github.com/twitter/algebird/issues/353 val monoid = TopPctCMS.monoid(EPS, DELTA, SEED, 0.1) val data1 = Seq(1, 1, 1, 2, 2, 3).toK[K] @@ -354,15 +365,35 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma aggregated.heavyHitters contains (3.toK[K]) // C=3 is global top 1 heavy hitter } - "work as an Aggregator" in { + "exactly compute heavy hitters when created from single, small stream" in { + val data1 = Seq(1, 2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5, 5, 5, 5).toK[K] + + val cms1 = TopPctCMS.monoid[K](EPS, DELTA, SEED, 0.01).create(data1) + cms1.heavyHitters should be(Set(1, 2, 3, 4, 5)) + + val cms2 = TopPctCMS.monoid[K](EPS, DELTA, SEED, 0.1).create(data1) + cms2.heavyHitters should be(Set(2, 3, 4, 5)) + + val cms3 = TopPctCMS.monoid[K](EPS, DELTA, SEED, 0.3).create(data1) + cms3.heavyHitters should be(Set(5)) + + val cms4 = TopPctCMS.monoid[K](EPS, DELTA, SEED, 0.9).create(data1) + cms4.heavyHitters should be(Set[K]()) + } + + "work as an Aggregator when created from a single, small stream" in { val data1 = Seq(1, 2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5, 5, 5, 5).toK[K] + val cms1 = TopPctCMS.aggregator[K](EPS, DELTA, SEED, 0.01).apply(data1) - val cms2 = TopPctCMS.aggregator[K](EPS, DELTA, SEED, 0.1).apply(data1) - val cms3 = TopPctCMS.aggregator[K](EPS, DELTA, SEED, 0.3).apply(data1) - val cms4 = TopPctCMS.aggregator[K](EPS, DELTA, SEED, 0.9).apply(data1) cms1.heavyHitters should be(Set(1, 2, 3, 4, 5)) + + val cms2 = TopPctCMS.aggregator[K](EPS, DELTA, SEED, 0.1).apply(data1) cms2.heavyHitters should be(Set(2, 3, 4, 5)) + + val cms3 = TopPctCMS.aggregator[K](EPS, DELTA, SEED, 0.3).apply(data1) cms3.heavyHitters should be(Set(5)) + + val cms4 = TopPctCMS.aggregator[K](EPS, DELTA, SEED, 0.9).apply(data1) cms4.heavyHitters should be(Set[K]()) } @@ -376,7 +407,7 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma // This test involves merging of top-N CMS instances, which is not an associative operation. This means that the // success or failure of this test depends on the merging order and/or the test data characteristics. - ", when adding CMS instances, drop old heavy hitters when new heavy hitters replace them (positive test case)" in { + "(when adding CMS instances) drop old heavy hitters when new heavy hitters replace them, if merge order matches data" in { val heavyHittersN = 2 val monoid = TopNCMS.monoid[K](EPS, DELTA, SEED, heavyHittersN) val cms1 = monoid.create(Seq(1, 2, 2).toK[K]) @@ -389,7 +420,103 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma cms4.heavyHitters should be(Set(1, 6)) } - "exactly compute its own heavy hitters in a small stream" in { + "(when adding individual items) drop old heavy hitters when new heavy hitters replace them" in { + val monoid = TopPctCMS.monoid[K](EPS, DELTA, SEED, 0.3) + val cms1 = monoid.create(Seq(1, 2, 2).toK[K]) + cms1.heavyHitters should be(Set(1, 2)) + + val cms2 = cms1 + 2.toK[K] + cms2.heavyHitters should be(Set(2)) + + val cms3 = cms2 + 1.toK[K] + cms3.heavyHitters should be(Set(1, 2)) + + val heaviest = 0.toK[K] + val cms4 = cms3 + heaviest + heaviest + heaviest + heaviest + heaviest + heaviest + cms4.heavyHitters should be(Set(heaviest)) + } + + // This test documents the order bias of top-N CMS, i.e. it's a negative test case. + // See https://github.com/twitter/algebird/issues/353 + "(when adding CMS instances) generally FAIL to merge heavy hitters correctly due to order bias" in { + val topN = 2 + val monoid = TopNCMS.monoid(EPS, DELTA, SEED, topN) + + val data1 = Seq(1, 1, 1, 2, 2, 3).toK[K] + val data2 = Seq(3, 4, 4, 4, 5, 5).toK[K] + val data3 = Seq(3, 6, 6, 6, 7, 7).toK[K] + val data4 = Seq(3, 8, 8, 8, 9, 9).toK[K] + val singleData = data1 ++ data2 ++ data3 ++ data4 + + /* + Data sets from above shown in tabular view + + Item 1 2 3 4 total (= singleData) + ---------------------------------------- + A (1) 3 - - - 3 + B (2) 2 - - - 2 + C (3) 1 1 1 1 4 <<< C is global top 1 heavy hitter + D (4) - 3 - - 3 + E (5) - 2 - - 2 + F (6) - - 3 - 3 + G (7) - - 2 - 2 + H (8) - - - 3 3 + I (9) - - - 2 2 + + */ + + val cms1 = monoid.create(data1) + val cms2 = monoid.create(data2) + val cms3 = monoid.create(data3) + val cms4 = monoid.create(data4) + val aggregated = cms1 ++ cms2 ++ cms3 ++ cms4 + + val single = monoid.create(singleData) + aggregated.heavyHitters shouldNot be(single.heavyHitters) + aggregated.heavyHitters shouldNot contain(3.toK[K]) // C=3 is global top 1 heavy hitter + } + + // Compared to adding top-N CMS instances, which is generally unsafe because of order bias (see test cases above), + // adding individual items to a top-N CMS is a safe operation. + // See https://github.com/twitter/algebird/issues/353 + "(when adding individual items) merge heavy hitters correctly [GH-353 regression test]" in { + val topN = 2 + val monoid = TopNCMS.monoid(EPS, DELTA, SEED, topN) + + val data1 = Seq(1, 1, 1, 2, 2, 3).toK[K] + val data2 = Seq(3, 4, 4, 4, 5, 5).toK[K] + val data3 = Seq(3, 6, 6, 6, 7, 7).toK[K] + val data4 = Seq(3, 8, 8, 8, 9, 9).toK[K] + val singleData = data1 ++ data2 ++ data3 ++ data4 + + /* + Data sets from above shown in tabular view + + Item 1 2 3 4 total (= singleData) + ---------------------------------------- + A (1) 3 - - - 3 + B (2) 2 - - - 2 + C (3) 1 1 1 1 4 <<< C is global top 1 heavy hitter + D (4) - 3 - - 3 + E (5) - 2 - - 2 + F (6) - - 3 - 3 + G (7) - - 2 - 2 + H (8) - - - 3 3 + I (9) - - - 2 2 + + */ + + val cms1 = monoid.create(data1) + val cms2 = cms1 + 3.toK[K] + 4.toK[K] + 4.toK[K] + 4.toK[K] + 5.toK[K] + 5.toK[K] // effectively "++ data2" + val cms3 = cms2 + 3.toK[K] + 6.toK[K] + 6.toK[K] + 6.toK[K] + 7.toK[K] + 7.toK[K] // "++ data3" + val aggregated = cms3 + 3.toK[K] + 8.toK[K] + 8.toK[K] + 8.toK[K] + 9.toK[K] + 9.toK[K] // "++ data4" + + val single = monoid.create(singleData) + aggregated.heavyHitters should be(single.heavyHitters) + aggregated.heavyHitters should contain(3.toK[K]) // C=3 is global top 1 heavy hitter + } + + "exactly compute heavy hitters when created from single, small stream" in { val data1 = Seq(1, 2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5, 5, 5, 5).toK[K] val cms1 = TopNCMS.monoid[K](EPS, DELTA, SEED, 5).create(data1) @@ -408,17 +535,22 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma cms5.heavyHitters should be(Set(5)) } - "work as an Aggregator" in { + "work as an Aggregator when created from a single, small stream" in { val data1 = Seq(1, 2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5, 5, 5, 5).toK[K] + val cms1 = TopNCMS.aggregator[K](EPS, DELTA, SEED, 5).apply(data1) - val cms2 = TopNCMS.aggregator[K](EPS, DELTA, SEED, 4).apply(data1) - val cms3 = TopNCMS.aggregator[K](EPS, DELTA, SEED, 3).apply(data1) - val cms4 = TopNCMS.aggregator[K](EPS, DELTA, SEED, 2).apply(data1) - val cms5 = TopNCMS.aggregator[K](EPS, DELTA, SEED, 1).apply(data1) cms1.heavyHitters should be(Set(1, 2, 3, 4, 5)) + + val cms2 = TopNCMS.aggregator[K](EPS, DELTA, SEED, 4).apply(data1) cms2.heavyHitters should be(Set(2, 3, 4, 5)) + + val cms3 = TopNCMS.aggregator[K](EPS, DELTA, SEED, 3).apply(data1) cms3.heavyHitters should be(Set(3, 4, 5)) + + val cms4 = TopNCMS.aggregator[K](EPS, DELTA, SEED, 2).apply(data1) cms4.heavyHitters should be(Set(4, 5)) + + val cms5 = TopNCMS.aggregator[K](EPS, DELTA, SEED, 1).apply(data1) cms5.heavyHitters should be(Set(5)) } From e5d444145233ca4644a1667999f9c9620a5af69b Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Thu, 16 Oct 2014 12:18:25 +0200 Subject: [PATCH 60/66] Clarify when warning on top-N CMS merges actually apply --- .../scala/com/twitter/algebird/CountMinSketch.scala | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index e7ca3e462..73370f79a 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -889,7 +889,7 @@ case class TopPctCMSAggregator[K](cmsMonoid: TopPctCMSMonoid[K]) /** * Monoid for top-N based [[TopCMS]] sketches. '''Use with care! (see warning below)''' * - * =Warning: Unsafe merge operation= + * =Warning: Adding top-N CMS instances (`++`) is an unsafe operation= * * Top-N computations are not associative. The effect is that a top-N CMS has an ordering bias (with regard to heavy * hitters) when ''merging'' CMS instances (e.g. via `++`). This means merging heavy hitters across CMS instances may @@ -897,14 +897,17 @@ case class TopPctCMSAggregator[K](cmsMonoid: TopPctCMSMonoid[K]) * being merged, with the rule of thumb being that the earlier a set of heavy hitters is being merged, the more likely * is the end result biased towards these heavy hitters. * - * The warning above only applies when ''merging'' CMS instances. That is, a given top-N CMS instance will correctly - * compute its own heavy hitters. + * The warning above only applies when ''adding CMS instances'' (think: `cms1 ++ cms2`). In comparison, heavy hitters + * are correctly computed when: + * + * - a top-N CMS instance is created from a single data stream, i.e. `Seq[K]` + * - items are added/counted individually, i.e. `cms + item` or `cms + (item, count)`. * * See the discussion in [[https://github.com/twitter/algebird/issues/353 Algebird issue 353]] for further details. * * =Alternatives= * - * The following, alternative data structures may be better picks than a top-N based CMS: + * The following, alternative data structures may be better picks than a top-N based CMS given the warning above: * * - [[TopPctCMS]]: Has safe merge semantics for its instances including heavy hitters. * - [[SpaceSaver]]: Has the same ordering bias than a top-N CMS, but at least it provides bounds on the bias. From a57bb74fae6a06a1daf8447f69eca42db8150d0a Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Tue, 18 Nov 2014 13:42:53 +0100 Subject: [PATCH 61/66] Use iterator to prevent unneeded allocation --- .../scala/com/twitter/algebird/CountMinSketch.scala | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index 73370f79a..8dce44c8a 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -453,10 +453,8 @@ case class CMSInstance[K: Ordering](countsTable: CMSInstance.CountsTable[K], } } - def frequency(item: K): Approximate[Long] = { - val estimates = countsTable.counts.zipWithIndex.map { case (row, i) => row(params.hashes(i)(item)) } - makeApprox(estimates.min) - } + def frequency(item: K): Approximate[Long] = + makeApprox(countsTable.counts.iterator.zip(params.hashes.iterator).map { case (row, hash) => row(hash(item)) }.min) /** * Let X be a CMS, and let count_X[j, k] denote the value in X's 2-dimensional count table at row j and column k. @@ -469,11 +467,11 @@ case class CMSInstance[K: Ordering](countsTable: CMSInstance.CountsTable[K], case other: CMSInstance[_] => require((other.depth, other.width) == (depth, width), "Tables must have the same dimensions.") - def innerProductAtDepth(d: Int) = (0 to (width - 1)).map { w => + def innerProductAtDepth(d: Int) = (0 to (width - 1)).iterator.map { w => countsTable.getCount(d, w) * other.countsTable.getCount(d, w) }.sum - val est = (0 to (depth - 1)).map { innerProductAtDepth }.min + val est = (0 to (depth - 1)).iterator.map { innerProductAtDepth }.min Approximate(est - (eps * totalCount * other.totalCount).toLong, est, est, 1 - delta) case _ => other.innerProduct(this) } From 40f673ca1e15917558d32b08b918c454b386a956 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Tue, 18 Nov 2014 14:03:12 +0100 Subject: [PATCH 62/66] Remove unneeded Ordering context bound from HeavyHitter case class --- .../src/main/scala/com/twitter/algebird/CountMinSketch.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index 8dce44c8a..ad8b4d326 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -781,7 +781,7 @@ object HeavyHitters { } -case class HeavyHitter[K: Ordering](item: K, count: Long) +case class HeavyHitter[K](item: K, count: Long) object HeavyHitter { From cd17745069d4fd8ff2fcf4745edb62131b4d23f4 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Tue, 18 Nov 2014 16:28:47 +0100 Subject: [PATCH 63/66] Fix typo in test description --- .../test/scala/com/twitter/algebird/CountMinSketchTest.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala index 4b6b5f123..cf944e2fa 100644 --- a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala +++ b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala @@ -365,7 +365,7 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma aggregated.heavyHitters contains (3.toK[K]) // C=3 is global top 1 heavy hitter } - "exactly compute heavy hitters when created from single, small stream" in { + "exactly compute heavy hitters when created from a single, small stream" in { val data1 = Seq(1, 2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5, 5, 5, 5).toK[K] val cms1 = TopPctCMS.monoid[K](EPS, DELTA, SEED, 0.01).create(data1) @@ -516,7 +516,7 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma aggregated.heavyHitters should contain(3.toK[K]) // C=3 is global top 1 heavy hitter } - "exactly compute heavy hitters when created from single, small stream" in { + "exactly compute heavy hitters when created a from single, small stream" in { val data1 = Seq(1, 2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5, 5, 5, 5).toK[K] val cms1 = TopNCMS.monoid[K](EPS, DELTA, SEED, 5).create(data1) From bec16fe435b751f9a6248ab4718f1024b4342150 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Wed, 19 Nov 2014 11:21:47 +0100 Subject: [PATCH 64/66] Add com.twitter.algebird.legacy for old CMS and CountMinSketchMonoid --- .../legacy/CountMinSketchMonoid.scala | 28 +++++++++++++ .../com/twitter/algebird/legacy/package.scala | 42 +++++++++++++++++++ .../twitter/algebird/CountMinSketchTest.scala | 39 +++++++++++++++++ 3 files changed, 109 insertions(+) create mode 100644 algebird-core/src/main/scala/com/twitter/algebird/legacy/CountMinSketchMonoid.scala create mode 100644 algebird-core/src/main/scala/com/twitter/algebird/legacy/package.scala diff --git a/algebird-core/src/main/scala/com/twitter/algebird/legacy/CountMinSketchMonoid.scala b/algebird-core/src/main/scala/com/twitter/algebird/legacy/CountMinSketchMonoid.scala new file mode 100644 index 000000000..bba251e34 --- /dev/null +++ b/algebird-core/src/main/scala/com/twitter/algebird/legacy/CountMinSketchMonoid.scala @@ -0,0 +1,28 @@ +package com.twitter.algebird.legacy + +import com.twitter.algebird.TopPctCMS + +/** + * Creates instances of type `CountMinSketchMonoid` (which is a type alias for the legacy class of the same name in + * Algebird versions < 0.8.1). + * + * =Deprecated since 0.8.1= + * + * Only use this object when transitioning legacy code. Newer code should use [[TopPctCMS.monoid()]]: + * + * {{{ + * import com.twitter.algebird.CMSHasherImplicits._ + * val cmsMonoid = TopPctCMS.monoid[Long](...) + * }}} + */ +object CountMinSketchMonoid { + + import com.twitter.algebird.CMSHasherImplicits._ + + @deprecated( + "You should use TopPctCMS.monoid[Long]() instead of legacy.CountMinSketchMonoid, and import CMSHasherImplicits._", + since = "0.8.1") + def apply(eps: Double, delta: Double, seed: Int, heavyHittersPct: Double = 0.01): CountMinSketchMonoid = + TopPctCMS.monoid[Long](eps, delta, seed, heavyHittersPct) + +} \ No newline at end of file diff --git a/algebird-core/src/main/scala/com/twitter/algebird/legacy/package.scala b/algebird-core/src/main/scala/com/twitter/algebird/legacy/package.scala new file mode 100644 index 000000000..a99bb220d --- /dev/null +++ b/algebird-core/src/main/scala/com/twitter/algebird/legacy/package.scala @@ -0,0 +1,42 @@ +package com.twitter.algebird + +package object legacy { + + /** + * For legacy code (Algebird versions < 0.8.1) that uses CMS we provide this functionally equivalent CMS type, which + * is a CMS with [[Long]]-based keys. + * + * This type is an alias for `TopCMS[Long]` (see [[TopCMS]]). + * + * =Usage= + * + * You typically create instances of this type via [[CountMinSketchMonoid]]. + */ + @deprecated("You should use TopCMS[Long] instead of legacy.CMS", since = "0.8.1") + type CMS = TopCMS[Long] + + /** + * For legacy code (Algebird versions < 0.8.1) that uses CMS we provide this functionally equivalent CMS monoid, which + * allows you to work with (e.g. create) top-% CMS instances with [[Long]]-based keys. + * + * =Usage= + * + * You typically create instances of this type via `CountMinSketchMonoid.apply()`, see example below. + * + * =Example= + * + * {{{ + * import com.twitter.algebird.legacy.CountMinSketchMonoid + * + * // Pre-0.8.1 you would have used `new CountMinSketchMonoid(EPS, DELTA, SEED)` (note the `new`). + * val cmsMonoid: CountMinSketchMonoid = CountMinSketchMonoid(EPS, DELTA, SEED) + * }}} + * + * =Implementation details= + * + * This type is an alias for `TopPctCMSMonoid[Long]` (see [[TopPctCMSMonoid]]). + */ + @deprecated("You should use TopPctCMSMonoid[Long] instead of legacy.CountMinSketchMonoid", since = "0.8.1") + type CountMinSketchMonoid = TopPctCMSMonoid[Long] + +} \ No newline at end of file diff --git a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala index cf944e2fa..875615299 100644 --- a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala +++ b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala @@ -556,4 +556,43 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma } +} + +/** + * This spec verifies that we provide legacy types for the CMS and CountMinSketchMonoid classes we had in Algebird + * versions < 0.8.1. Note that this spec is not meant to verify their actual functionality. + */ +class LegacyCMSSpec extends WordSpec with Matchers { + + import legacy.CountMinSketchMonoid + + val DELTA = 1E-10 + val EPS = 0.001 + val SEED = 1 + + val CMS_MONOID: CountMinSketchMonoid = CountMinSketchMonoid(EPS, DELTA, SEED) + + "The legacy package" should { + + "provide a legacy type for the CMS implementation in Algebird versions < 0.8.1" in { + val cms: legacy.CMS = CMS_MONOID.create(Seq(0L, 0L)) + cms.frequency(0L).estimate should be (2) + cms.heavyHitters should be(Set(0L)) + } + + "provide a legacy type for the CMS monoid implementation in Algebird versions < 0.8.1" in { + val cmsMonoid: CountMinSketchMonoid = { + val eps = 0.001 + val delta = 1E-5 + val seed = 1 + val heavyHittersPct = 0.1 + CountMinSketchMonoid(eps, delta, seed, heavyHittersPct) + } + val cms = cmsMonoid.create(Seq(0L, 0L)) + cms.frequency(0L).estimate should be (2) + cms.heavyHitters should be(Set(0L)) + } + + } + } \ No newline at end of file From 9c02f1a2070e4675a1d871dfb59e5a0598391322 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Wed, 19 Nov 2014 13:38:05 +0100 Subject: [PATCH 65/66] Add tests to verify roundtripping depth/delta and width/eps --- .../twitter/algebird/CountMinSketchTest.scala | 40 +++++++++++++++++++ 1 file changed, 40 insertions(+) diff --git a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala index 875615299..170f870f2 100644 --- a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala +++ b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala @@ -558,6 +558,46 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma } +class CMSFunctionsSpec extends PropSpec with PropertyChecks with Matchers { + + property("roundtrips width->eps->width for common width values") { + forAll(Table("i", 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 20, 30, 40, 50, 60, 70, 80, 90, 100)) { (i: Int) => + CMSFunctions.width(CMSFunctions.eps(i)) should be(i) + } + } + + // Negative test case to document a precision error forsuch as: + // scala> val width = 39 + // scala> scala.math.exp(1) / CMSFunctions.eps(width) + // res171: Double = 39.00000000000001 <<< should be 39.0 + property("fail to roundtrip width->eps->width for widths that expose type precision limits") { + forAll(Table(("i", "expI"), (39, 40), (86, 87), (238, 239))) { (i: Int, expI: Int) => + i shouldNot be(expI) // assert correctness of test data + CMSFunctions.width(CMSFunctions.eps(i)) should be(expI) + } + } + + property("roundtrips depth->delta->depth for common depth values") { + // For all i > 709 this test break because of precision limits: For all i > 709 will return 0.0, which is not the + // mathematically correct value but rather the asymptote of delta. + val maxI = 709 + forAll(Gen.choose(0, maxI)) { (i: Int) => + CMSFunctions.depth(CMSFunctions.delta(i)) should be(i) + } + } + + // Negative test case to document a precision error that is exposed by all depths > 709. + // For all i > 709, CMSFunctions.delta(i) will return 0.0, which is not the mathematically correct value but rather + // the asymptote of the delta function. + property("fail to roundtrip depth->delta->depth for depths that expose type precision limits") { + val maxI = 709 + forAll(Gen.choose(maxI + 1, 10000)) { (i: Int) => + CMSFunctions.depth(CMSFunctions.delta(i)) should be(Int.MaxValue) + } + } + +} + /** * This spec verifies that we provide legacy types for the CMS and CountMinSketchMonoid classes we had in Algebird * versions < 0.8.1. Note that this spec is not meant to verify their actual functionality. From ba498d6bc4a58e1a1ad480ebecf379a2a6e5134d Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Wed, 19 Nov 2014 14:05:19 +0100 Subject: [PATCH 66/66] Improve width() by truncating decimal places to eliminate precision errors --- .../com/twitter/algebird/CountMinSketch.scala | 13 ++++++++++++- .../twitter/algebird/CountMinSketchTest.scala | 17 ++++------------- 2 files changed, 16 insertions(+), 14 deletions(-) diff --git a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala index ad8b4d326..af3658c45 100644 --- a/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala +++ b/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala @@ -172,7 +172,18 @@ object CMSFunctions { /** * Translates from `eps` to `width`. */ - def width(eps: Double): Int = scala.math.ceil(scala.math.exp(1) / eps).toInt + def width(eps: Double): Int = scala.math.ceil(truncatePrecisionError(scala.math.exp(1) / eps)).toInt + + // Eliminates precision errors such as the following: + // + // scala> val width = 39 + // scala> scala.math.exp(1) / CMSFunctions.eps(width) + // res171: Double = 39.00000000000001 <<< should be 39.0 + // + // Because of the actual types on which CMSFunctions operates (i.e. Int and Double), the maximum number of decimal + // places should be 6. + private def truncatePrecisionError(i: Double, decimalPlaces: Int = 6) = + BigDecimal(i).setScale(decimalPlaces, BigDecimal.RoundingMode.HALF_UP).toDouble /** * Generates `N=depth` pair-wise independent hash functions. diff --git a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala index 170f870f2..5381d738d 100644 --- a/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala +++ b/algebird-test/src/test/scala/com/twitter/algebird/CountMinSketchTest.scala @@ -561,19 +561,10 @@ abstract class CMSTest[K: Ordering: CMSHasher: Numeric] extends WordSpec with Ma class CMSFunctionsSpec extends PropSpec with PropertyChecks with Matchers { property("roundtrips width->eps->width for common width values") { - forAll(Table("i", 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 20, 30, 40, 50, 60, 70, 80, 90, 100)) { (i: Int) => - CMSFunctions.width(CMSFunctions.eps(i)) should be(i) - } - } - - // Negative test case to document a precision error forsuch as: - // scala> val width = 39 - // scala> scala.math.exp(1) / CMSFunctions.eps(width) - // res171: Double = 39.00000000000001 <<< should be 39.0 - property("fail to roundtrip width->eps->width for widths that expose type precision limits") { - forAll(Table(("i", "expI"), (39, 40), (86, 87), (238, 239))) { (i: Int, expI: Int) => - i shouldNot be(expI) // assert correctness of test data - CMSFunctions.width(CMSFunctions.eps(i)) should be(expI) + forAll { (i: Int) => + whenever(i > 0) { + CMSFunctions.width(CMSFunctions.eps(i)) should be(i) + } } }