Skip to content

Commit bfe7b52

Browse files
committed
Adding experimental for approximate counts
1 parent 8d0c873 commit bfe7b52

File tree

5 files changed

+46
-11
lines changed

5 files changed

+46
-11
lines changed

core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala

+16-3
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@ import scala.reflect.ClassTag
2323

2424
import org.apache.spark.Partitioner
2525
import org.apache.spark.SparkContext.doubleRDDToDoubleRDDFunctions
26+
import org.apache.spark.annotations.Experimental
2627
import org.apache.spark.api.java.function.{Function => JFunction}
2728
import org.apache.spark.partial.{BoundedDouble, PartialResult}
2829
import org.apache.spark.rdd.RDD
@@ -184,14 +185,26 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[JDouble, Ja
184185
def meanApprox(timeout: Long, confidence: JDouble): PartialResult[BoundedDouble] =
185186
srdd.meanApprox(timeout, confidence)
186187

187-
/** (Experimental) Approximate operation to return the mean within a timeout. */
188+
/**
189+
* :: Experimental ::
190+
* Approximate operation to return the mean within a timeout.
191+
*/
192+
@Experimental
188193
def meanApprox(timeout: Long): PartialResult[BoundedDouble] = srdd.meanApprox(timeout)
189194

190-
/** (Experimental) Approximate operation to return the sum within a timeout. */
195+
/**
196+
* :: Experimental ::
197+
* Approximate operation to return the sum within a timeout.
198+
*/
199+
@Experimental
191200
def sumApprox(timeout: Long, confidence: JDouble): PartialResult[BoundedDouble] =
192201
srdd.sumApprox(timeout, confidence)
193202

194-
/** (Experimental) Approximate operation to return the sum within a timeout. */
203+
/**
204+
* :: Experimental ::
205+
* Approximate operation to return the sum within a timeout.
206+
*/
207+
@Experimental
195208
def sumApprox(timeout: Long): PartialResult[BoundedDouble] = srdd.sumApprox(timeout)
196209

197210
/**

core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala

+8-3
Original file line numberDiff line numberDiff line change
@@ -26,11 +26,12 @@ import com.google.common.base.Optional
2626
import org.apache.hadoop.conf.Configuration
2727
import org.apache.hadoop.io.compress.CompressionCodec
2828
import org.apache.hadoop.mapred.{JobConf, OutputFormat}
29-
import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat, Job}
29+
import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat}
3030

3131
import org.apache.spark.{HashPartitioner, Partitioner}
3232
import org.apache.spark.Partitioner._
3333
import org.apache.spark.SparkContext.rddToPairRDDFunctions
34+
import org.apache.spark.annotations.Experimental
3435
import org.apache.spark.api.java.JavaSparkContext.fakeClassTag
3536
import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2, PairFunction}
3637
import org.apache.spark.partial.{BoundedDouble, PartialResult}
@@ -200,16 +201,20 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])
200201
def countByKey(): java.util.Map[K, Long] = mapAsJavaMap(rdd.countByKey())
201202

202203
/**
203-
* (Experimental) Approximate version of countByKey that can return a partial result if it does
204+
* :: Experimental ::
205+
* Approximate version of countByKey that can return a partial result if it does
204206
* not finish within a timeout.
205207
*/
208+
@Experimental
206209
def countByKeyApprox(timeout: Long): PartialResult[java.util.Map[K, BoundedDouble]] =
207210
rdd.countByKeyApprox(timeout).map(mapAsJavaMap)
208211

209212
/**
210-
* (Experimental) Approximate version of countByKey that can return a partial result if it does
213+
* :: Experimental ::
214+
* Approximate version of countByKey that can return a partial result if it does
211215
* not finish within a timeout.
212216
*/
217+
@Experimental
213218
def countByKeyApprox(timeout: Long, confidence: Double = 0.95)
214219
: PartialResult[java.util.Map[K, BoundedDouble]] =
215220
rdd.countByKeyApprox(timeout, confidence).map(mapAsJavaMap)

core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala

+7-2
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@ import com.google.common.base.Optional
2626
import org.apache.hadoop.io.compress.CompressionCodec
2727

2828
import org.apache.spark.{Partition, SparkContext, TaskContext}
29+
import org.apache.spark.annotations.Experimental
2930
import org.apache.spark.api.java.JavaPairRDD._
3031
import org.apache.spark.api.java.JavaSparkContext.fakeClassTag
3132
import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2, _}
@@ -331,16 +332,20 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
331332
def count(): Long = rdd.count()
332333

333334
/**
334-
* (Experimental) Approximate version of count() that returns a potentially incomplete result
335+
* :: Experimental ::
336+
* Approximate version of count() that returns a potentially incomplete result
335337
* within a timeout, even if not all tasks have finished.
336338
*/
339+
@Experimental
337340
def countApprox(timeout: Long, confidence: Double): PartialResult[BoundedDouble] =
338341
rdd.countApprox(timeout, confidence)
339342

340343
/**
341-
* (Experimental) Approximate version of count() that returns a potentially incomplete result
344+
* :: Experimental ::
345+
* Approximate version of count() that returns a potentially incomplete result
342346
* within a timeout, even if not all tasks have finished.
343347
*/
348+
@Experimental
344349
def countApprox(timeout: Long): PartialResult[BoundedDouble] =
345350
rdd.countApprox(timeout)
346351

core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala

+11-2
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717

1818
package org.apache.spark.rdd
1919

20+
import org.apache.spark.annotations.Experimental
2021
import org.apache.spark.{TaskContext, Logging}
2122
import org.apache.spark.partial.BoundedDouble
2223
import org.apache.spark.partial.MeanEvaluator
@@ -63,14 +64,22 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable {
6364
*/
6465
def sampleVariance(): Double = stats().sampleVariance
6566

66-
/** (Experimental) Approximate operation to return the mean within a timeout. */
67+
/**
68+
* :: Experimental ::
69+
* Approximate operation to return the mean within a timeout.
70+
*/
71+
@Experimental
6772
def meanApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = {
6873
val processPartition = (ctx: TaskContext, ns: Iterator[Double]) => StatCounter(ns)
6974
val evaluator = new MeanEvaluator(self.partitions.size, confidence)
7075
self.context.runApproximateJob(self, processPartition, evaluator, timeout)
7176
}
7277

73-
/** (Experimental) Approximate operation to return the sum within a timeout. */
78+
/**
79+
* :: Experimental ::
80+
* Approximate operation to return the sum within a timeout.
81+
*/
82+
@Experimental
7483
def sumApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = {
7584
val processPartition = (ctx: TaskContext, ns: Iterator[Double]) => StatCounter(ns)
7685
val evaluator = new SumEvaluator(self.partitions.size, confidence)

core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala

+4-1
Original file line numberDiff line numberDiff line change
@@ -39,6 +39,7 @@ RecordWriter => NewRecordWriter, SparkHadoopMapReduceUtil}
3939
import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat}
4040

4141
import org.apache.spark._
42+
import org.apache.spark.annotations.Experimental
4243
import org.apache.spark.deploy.SparkHadoopUtil
4344
import org.apache.spark.SparkHadoopWriter
4445
import org.apache.spark.Partitioner.defaultPartitioner
@@ -201,9 +202,11 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)])
201202
def countByKey(): Map[K, Long] = self.map(_._1).countByValue()
202203

203204
/**
204-
* (Experimental) Approximate version of countByKey that can return a partial result if it does
205+
* :: Experimental ::
206+
* Approximate version of countByKey that can return a partial result if it does
205207
* not finish within a timeout.
206208
*/
209+
@Experimental
207210
def countByKeyApprox(timeout: Long, confidence: Double = 0.95)
208211
: PartialResult[Map[K, BoundedDouble]] = {
209212
self.map(_._1).countByValueApprox(timeout, confidence)

0 commit comments

Comments
 (0)