Github user falaki commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1025#discussion_r14544795
  
    --- Diff: 
core/src/main/scala/org/apache/spark/util/random/StratifiedSampler.scala ---
    @@ -0,0 +1,296 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.random
    +
    +import scala.collection.{Map, mutable}
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +import org.apache.commons.math3.random.RandomDataGenerator
    +import org.apache.spark.{Logging, SparkContext, TaskContext}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.util.Utils
    +import org.apache.spark.util.random.{PoissonBounds => PB}
    +
    +private[spark] object StratifiedSampler extends Logging {
    +
    +  /**
    +   * A version of {@link #aggregate()} that passes the TaskContext to the 
function that does
    +   * aggregation for each partition. This function avoids creating an 
extra depth in the RDD
    +   * lineage, as opposed to using mapPartitionsWithId, which results in 
slightly improved run time.
    +   */
    +  def aggregateWithContext[U: ClassTag, T: ClassTag](zeroValue: U)
    +      (rdd: RDD[T],
    +       seqOp: ((TaskContext, U), T) => U,
    +       combOp: (U, U) => U): U = {
    +    val sc: SparkContext = rdd.sparkContext
    +    // Clone the zero value since we will also be serializing it as part 
of tasks
    +    var jobResult = Utils.clone(zeroValue, 
sc.env.closureSerializer.newInstance())
    +    // pad seqOp and combOp with taskContext to conform to aggregate's 
signature in TraversableOnce
    +    val paddedSeqOp = (arg1: (TaskContext, U), item: T) => (arg1._1, 
seqOp(arg1, item))
    +    val paddedcombOp = (arg1: (TaskContext, U), arg2: (TaskContext, U)) =>
    +      (arg1._1, combOp(arg1._2, arg1._2))
    +    val cleanSeqOp = sc.clean(paddedSeqOp)
    +    val cleanCombOp = sc.clean(paddedcombOp)
    +    val aggregatePartition = (tc: TaskContext, it: Iterator[T]) =>
    +      (it.aggregate(tc, zeroValue)(cleanSeqOp, cleanCombOp))._2
    +    val mergeResult = (index: Int, taskResult: U) => jobResult = 
combOp(jobResult, taskResult)
    +    sc.runJob(rdd, aggregatePartition, mergeResult)
    +    jobResult
    +  }
    +
    +  /**
    +   * Returns the function used by aggregate to collect sampling statistics 
for each partition.
    +   */
    +  def getSeqOp[K, V](withReplacement: Boolean,
    +      fractionByKey: (K => Double),
    +      counts: Option[Map[K, Long]]): ((TaskContext, Result[K]),(K, V)) => 
Result[K] = {
    +    val delta = 5e-5
    +    (U: (TaskContext, Result[K]), item: (K, V)) => {
    +      val result = U._2
    +      val tc = U._1
    +      val rng = result.getRand(tc.partitionId)
    +      val fraction = fractionByKey(item._1)
    +      val stratum = result.getEntry(item._1)
    +      if (withReplacement) {
    +        // compute q1 and q2 only if they haven't been computed already
    +        // since they don't change from iteration to iteration.
    +        // TODO change this to the streaming version
    +        if (stratum.q1.isEmpty || stratum.q2.isEmpty) {
    +          val n = counts.get(item._1)
    +          val s = math.ceil(n * fraction).toLong
    +          val lmbd1 = PB.getLowerBound(s)
    +          val minCount = PB.getMinCount(lmbd1)
    +          val lmbd2 = if (lmbd1 == 0) PB.getUpperBound(s) else 
PB.getUpperBound(s - minCount)
    +          val q1 = lmbd1 / n
    +          val q2 = lmbd2 / n
    +          stratum.q1 = Some(q1)
    +          stratum.q2 = Some(q2)
    +        }
    +        val x1 = if (stratum.q1.get == 0) 0L else 
rng.nextPoisson(stratum.q1.get)
    +        if (x1 > 0) {
    +          stratum.incrNumAccepted(x1)
    +        }
    +        val x2 = rng.nextPoisson(stratum.q2.get).toInt
    +        if (x2 > 0) {
    +          stratum.addToWaitList(ArrayBuffer.fill(x2)(rng.nextUniform(0.0, 
1.0)))
    +        }
    +      } else {
    +        // We use the streaming version of the algorithm for sampling 
without replacement.
    +        // Hence, q1 and q2 change on every iteration.
    +        val g1 = - math.log(delta) / stratum.numItems
    +        val g2 = (2.0 / 3.0) * g1
    +        val q1 = math.max(0, fraction + g2 - math.sqrt((g2 * g2 + 3 * g2 * 
fraction)))
    +        val q2 = math.min(1, fraction + g1 + math.sqrt(g1 * g1 + 2 * g1 * 
fraction))
    +
    +        val x = rng.nextUniform(0.0, 1.0)
    +        if (x < q1) {
    +          stratum.incrNumAccepted()
    +        } else if (x < q2) {
    +          stratum.addToWaitList(x)
    +        }
    +        stratum.q1 = Some(q1)
    +        stratum.q2 = Some(q2)
    +      }
    +      stratum.incrNumItems()
    +      result
    +    }
    +  }
    +
    +  /**
    +   * Returns the function used by aggregate to combine results from 
different partitions, as
    +   * returned by seqOp.
    +   */
    +  def getCombOp[K](): (Result[K], Result[K]) => Result[K] = {
    +    (r1: Result[K], r2: Result[K]) => {
    +      // take union of both key sets in case one partition doesn't contain 
all keys
    +      val keyUnion = r1.resultMap.keys.toSet.union(r2.resultMap.keys.toSet)
    +
    +      // Use r2 to keep the combined result since r1 is usual empty
    +      for (key <- keyUnion) {
    +        val entry1 = r1.resultMap.get(key)
    +        val entry2 = r2.resultMap.get(key)
    +        if (entry2.isEmpty && entry1.isDefined) {
    +          r2.resultMap += (key -> entry1.get)
    +        } else if (entry1.isDefined && entry2.isDefined) {
    +          entry2.get.addToWaitList(entry1.get.waitList)
    +          entry2.get.incrNumAccepted(entry1.get.numAccepted)
    +          entry2.get.incrNumItems(entry1.get.numItems)
    +        }
    +      }
    +      r2
    +    }
    +  }
    +
    +  /**
    +   * Given the result returned by the aggregate function, we need to 
determine the threshold used
    +   * to accept items to generate the exact sample size.
    +   */
    +  def computeThresholdByKey[K](finalResult: Map[K, Stratum], 
fractionByKey: (K => Double)):
    +    (K => Double) = {
    +    val thresholdByKey = new mutable.HashMap[K, Double]()
    +    for ((key, stratum) <- finalResult) {
    +      val fraction = fractionByKey(key)
    +      val s = math.ceil(stratum.numItems * fraction).toLong
    +      if (stratum.numAccepted > s) {
    +        logWarning("Pre-accepted too many")
    +        thresholdByKey += (key -> stratum.q1.get)
    +      } else {
    +        val numWaitListAccepted = (s - stratum.numAccepted).toInt
    +        if (numWaitListAccepted >= stratum.waitList.size) {
    +          logWarning("WaitList too short")
    +          thresholdByKey += (key -> stratum.q2.get)
    +        } else {
    +          thresholdByKey += (key -> 
stratum.waitList.sorted.apply(numWaitListAccepted))
    +        }
    +      }
    +    }
    +    thresholdByKey
    +  }
    +
    +  /**
    +   * Return the per partition sampling function used for sampling without 
replacement.
    +   *
    +   * When exact sample size is required, we make an additional pass over 
the RDD to determine the
    +   * exact sampling rate that guarantees sample size with high confidence.
    +   *
    +   * The sampling function has a unique seed per partition.
    +   */
    +  def getBernoulliSamplingFunction[K, V](rdd: RDD[(K,  V)],
    +      fractionByKey: K => Double,
    +      exact: Boolean,
    +      seed: Long): (Int, Iterator[(K, V)]) => Iterator[(K, V)] = {
    +    var samplingRateByKey = fractionByKey
    +    if (exact) {
    +      // determine threshold for each stratum and resample
    +      val seqOp = StratifiedSampler.getSeqOp[K,V](false, fractionByKey, 
None)
    +      val combOp = StratifiedSampler.getCombOp[K]()
    +      val zeroU = new Result[K](Map[K, Stratum](), seed = seed)
    +      val finalResult = aggregateWithContext(zeroU)(rdd, seqOp, 
combOp).resultMap
    +      samplingRateByKey = 
StratifiedSampler.computeThresholdByKey(finalResult, fractionByKey)
    +    }
    +    (idx: Int, iter: Iterator[(K, V)]) => {
    +      val random = new RandomDataGenerator
    +      random.reSeed(seed + idx)
    +      iter.filter(t => random.nextUniform(0.0, 1.0) < 
samplingRateByKey(t._1))
    +    }
    +  }
    +
    +  /**
    +   * Return the per partition sampling function used for sampling with 
replacement.
    +   *
    +   * When exact sample size is required, we make two additional passed 
over the RDD to determine
    +   * the exact sampling rate that guarantees sample size with high 
confidence. The first pass
    +   * counts the number of items in each stratum (group of items with the 
same key) in the RDD, and
    +   * the second pass uses the counts to determine exact sampling rates.
    +   *
    +   * The sampling function has a unique seed per partition.
    +   */
    +  def getPoissonSamplingFunction[K, V](rdd:RDD[(K,  V)],
    +      fractionByKey: K => Double,
    +      exact: Boolean,
    +      counts: Option[Map[K, Long]],
    +      seed: Long): (Int, Iterator[(K, V)]) => Iterator[(K, V)] = {
    +    // TODO implement the streaming version of sampling w/ replacement 
that doesn't require counts
    +    if (exact) {
    +      val seqOp = StratifiedSampler.getSeqOp[K,V](true, fractionByKey, 
counts)
    --- End diff --
    
    Similarly remove ```StratifiedSampler``` object name from these lines


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

Reply via email to