aboutsummaryrefslogtreecommitdiff
path: root/core/src/main/scala/org/apache/spark/api
diff options
context:
space:
mode:
Diffstat (limited to 'core/src/main/scala/org/apache/spark/api')
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala51
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala12
2 files changed, 37 insertions, 26 deletions
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala
index 4c8f9ed6fb..7dcfbf741c 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala
@@ -672,38 +672,47 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])
/**
* Return approximate number of distinct values for each key in this RDD.
- * The accuracy of approximation can be controlled through the relative standard deviation
- * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in
- * more accurate counts but increase the memory footprint and vise versa. Uses the provided
- * Partitioner to partition the output RDD.
+ *
+ * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice:
+ * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available
+ * <a href="http://dx.doi.org/10.1145/2452376.2452456">here</a>.
+ *
+ * @param relativeSD Relative accuracy. Smaller values create counters that require more space.
+ * It must be greater than 0.000017.
+ * @param partitioner partitioner of the resulting RDD.
*/
- def countApproxDistinctByKey(relativeSD: Double, partitioner: Partitioner): JavaRDD[(K, Long)] = {
- rdd.countApproxDistinctByKey(relativeSD, partitioner)
+ def countApproxDistinctByKey(relativeSD: Double, partitioner: Partitioner): JavaPairRDD[K, Long] =
+ {
+ fromRDD(rdd.countApproxDistinctByKey(relativeSD, partitioner))
}
/**
- * Return approximate number of distinct values for each key this RDD.
- * The accuracy of approximation can be controlled through the relative standard deviation
- * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in
- * more accurate counts but increase the memory footprint and vise versa. The default value of
- * relativeSD is 0.05. Hash-partitions the output RDD using the existing partitioner/parallelism
- * level.
+ * Return approximate number of distinct values for each key in this RDD.
+ *
+ * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice:
+ * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available
+ * <a href="http://dx.doi.org/10.1145/2452376.2452456">here</a>.
+ *
+ * @param relativeSD Relative accuracy. Smaller values create counters that require more space.
+ * It must be greater than 0.000017.
+ * @param numPartitions number of partitions of the resulting RDD.
*/
- def countApproxDistinctByKey(relativeSD: Double = 0.05): JavaRDD[(K, Long)] = {
- rdd.countApproxDistinctByKey(relativeSD)
+ def countApproxDistinctByKey(relativeSD: Double, numPartitions: Int): JavaPairRDD[K, Long] = {
+ fromRDD(rdd.countApproxDistinctByKey(relativeSD, numPartitions))
}
-
/**
* Return approximate number of distinct values for each key in this RDD.
- * The accuracy of approximation can be controlled through the relative standard deviation
- * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in
- * more accurate counts but increase the memory footprint and vise versa. HashPartitions the
- * output RDD into numPartitions.
*
+ * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice:
+ * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available
+ * <a href="http://dx.doi.org/10.1145/2452376.2452456">here</a>.
+ *
+ * @param relativeSD Relative accuracy. Smaller values create counters that require more space.
+ * It must be greater than 0.000017.
*/
- def countApproxDistinctByKey(relativeSD: Double, numPartitions: Int): JavaRDD[(K, Long)] = {
- rdd.countApproxDistinctByKey(relativeSD, numPartitions)
+ def countApproxDistinctByKey(relativeSD: Double): JavaPairRDD[K, Long] = {
+ fromRDD(rdd.countApproxDistinctByKey(relativeSD))
}
/** Assign a name to this RDD */
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
index 619bfd75be..330569a8d8 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
@@ -560,12 +560,14 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
/**
* Return approximate number of distinct elements in the RDD.
*
- * The accuracy of approximation can be controlled through the relative standard deviation
- * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in
- * more accurate counts but increase the memory footprint and vise versa. The default value of
- * relativeSD is 0.05.
+ * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice:
+ * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available
+ * <a href="http://dx.doi.org/10.1145/2452376.2452456">here</a>.
+ *
+ * @param relativeSD Relative accuracy. Smaller values create counters that require more space.
+ * It must be greater than 0.000017.
*/
- def countApproxDistinct(relativeSD: Double = 0.05): Long = rdd.countApproxDistinct(relativeSD)
+ def countApproxDistinct(relativeSD: Double): Long = rdd.countApproxDistinct(relativeSD)
def name(): String = rdd.name