aboutsummaryrefslogtreecommitdiff
path: root/core/src/main/scala/spark/partial/GroupedSumEvaluator.scala
blob: ae2b63f7cb7dea855481e6454d0e2e729ceb84a2 (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
/*
 * Licensed to the Apache Software Foundation (ASF) under one or more
 * contributor license agreements.  See the NOTICE file distributed with
 * this work for additional information regarding copyright ownership.
 * The ASF licenses this file to You under the Apache License, Version 2.0
 * (the "License"); you may not use this file except in compliance with
 * the License.  You may obtain a copy of the License at
 *
 *    http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing, software
 * distributed under the License is distributed on an "AS IS" BASIS,
 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */

package spark.partial

import java.util.{HashMap => JHashMap}
import java.util.{Map => JMap}

import scala.collection.mutable.HashMap
import scala.collection.Map
import scala.collection.JavaConversions.mapAsScalaMap

import spark.util.StatCounter

/**
 * An ApproximateEvaluator for sums by key. Returns a map of key to confidence interval.
 */
private[spark] class GroupedSumEvaluator[T](totalOutputs: Int, confidence: Double)
  extends ApproximateEvaluator[JHashMap[T, StatCounter], Map[T, BoundedDouble]] {

  var outputsMerged = 0
  var sums = new JHashMap[T, StatCounter]   // Sum of counts for each key

  override def merge(outputId: Int, taskResult: JHashMap[T, StatCounter]) {
    outputsMerged += 1
    val iter = taskResult.entrySet.iterator()
    while (iter.hasNext) {
      val entry = iter.next()
      val old = sums.get(entry.getKey)
      if (old != null) {
        old.merge(entry.getValue)
      } else {
        sums.put(entry.getKey, entry.getValue)
      }
    }
  }

  override def currentResult(): Map[T, BoundedDouble] = {
    if (outputsMerged == totalOutputs) {
      val result = new JHashMap[T, BoundedDouble](sums.size)
      val iter = sums.entrySet.iterator()
      while (iter.hasNext) {
        val entry = iter.next()
        val sum = entry.getValue.sum
        result(entry.getKey) = new BoundedDouble(sum, 1.0, sum, sum)
      }
      result
    } else if (outputsMerged == 0) {
      new HashMap[T, BoundedDouble]
    } else {
      val p = outputsMerged.toDouble / totalOutputs
      val studentTCacher = new StudentTCacher(confidence)
      val result = new JHashMap[T, BoundedDouble](sums.size)
      val iter = sums.entrySet.iterator()
      while (iter.hasNext) {
        val entry = iter.next()
        val counter = entry.getValue
        val meanEstimate = counter.mean
        val meanVar = counter.sampleVariance / counter.count
        val countEstimate = (counter.count + 1 - p) / p
        val countVar = (counter.count + 1) * (1 - p) / (p * p)
        val sumEstimate = meanEstimate * countEstimate
        val sumVar = (meanEstimate * meanEstimate * countVar) +
                     (countEstimate * countEstimate * meanVar) +
                     (meanVar * countVar)
        val sumStdev = math.sqrt(sumVar)
        val confFactor = studentTCacher.get(counter.count)
        val low = sumEstimate - confFactor * sumStdev
        val high = sumEstimate + confFactor * sumStdev
        result(entry.getKey) = new BoundedDouble(sumEstimate, confidence, low, high)
      }
      result
    }
  }
}