Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-3359][DOCS] Make javadoc8 working for unidoc/genjavadoc compatibility in Java API documentation #16013

Closed
wants to merge 17 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion core/src/main/scala/org/apache/spark/Accumulator.scala
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ package org.apache.spark
*
* An accumulator is created from an initial value `v` by calling
* [[SparkContext#accumulator SparkContext.accumulator]].
* Tasks running on the cluster can then add to it using the [[Accumulable#+= +=]] operator.
* Tasks running on the cluster can then add to it using the `+=` operator.
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I just decided to keep original format rather than trying to make this pretty.

The original was as below:

  • Scala
    2016-11-26 12 46 33

  • Java
    2016-11-26 12 46 46

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

After this PR it still prints the same.

  • Scala

    2016-11-26 12 51 56
  • Java

    2016-11-26 12 51 38

* However, they cannot read its value. Only the driver program can read the accumulator's value,
* using its [[#value]] method.
*
Expand Down
12 changes: 6 additions & 6 deletions core/src/main/scala/org/apache/spark/SparkConf.scala
Original file line number Diff line number Diff line change
Expand Up @@ -262,7 +262,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria
/**
* Get a time parameter as seconds; throws a NoSuchElementException if it's not set. If no
* suffix is provided then seconds are assumed.
* @throws NoSuchElementException
* @throws java.util.NoSuchElementException
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is interesting. Using @throws NoSuchElementException complains as below:

[error]   location: class VectorIndexerModel
[error] .../java/org/apache/spark/SparkConf.java:226: error: reference not found
[error]    * @throws NoSuchElementException
[error]      ^

*/
def getTimeAsSeconds(key: String): Long = {
Utils.timeStringAsSeconds(get(key))
Expand All @@ -279,7 +279,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria
/**
* Get a time parameter as milliseconds; throws a NoSuchElementException if it's not set. If no
* suffix is provided then milliseconds are assumed.
* @throws NoSuchElementException
* @throws java.util.NoSuchElementException
*/
def getTimeAsMs(key: String): Long = {
Utils.timeStringAsMs(get(key))
Expand All @@ -296,7 +296,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria
/**
* Get a size parameter as bytes; throws a NoSuchElementException if it's not set. If no
* suffix is provided then bytes are assumed.
* @throws NoSuchElementException
* @throws java.util.NoSuchElementException
*/
def getSizeAsBytes(key: String): Long = {
Utils.byteStringAsBytes(get(key))
Expand All @@ -320,7 +320,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria
/**
* Get a size parameter as Kibibytes; throws a NoSuchElementException if it's not set. If no
* suffix is provided then Kibibytes are assumed.
* @throws NoSuchElementException
* @throws java.util.NoSuchElementException
*/
def getSizeAsKb(key: String): Long = {
Utils.byteStringAsKb(get(key))
Expand All @@ -337,7 +337,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria
/**
* Get a size parameter as Mebibytes; throws a NoSuchElementException if it's not set. If no
* suffix is provided then Mebibytes are assumed.
* @throws NoSuchElementException
* @throws java.util.NoSuchElementException
*/
def getSizeAsMb(key: String): Long = {
Utils.byteStringAsMb(get(key))
Expand All @@ -354,7 +354,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria
/**
* Get a size parameter as Gibibytes; throws a NoSuchElementException if it's not set. If no
* suffix is provided then Gibibytes are assumed.
* @throws NoSuchElementException
* @throws java.util.NoSuchElementException
*/
def getSizeAsGb(key: String): Long = {
Utils.byteStringAsGb(get(key))
Expand Down
14 changes: 7 additions & 7 deletions core/src/main/scala/org/apache/spark/SparkContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -645,7 +645,7 @@ class SparkContext(config: SparkConf) extends Logging {

/**
* Get a local property set in this thread, or null if it is missing. See
* [[org.apache.spark.SparkContext.setLocalProperty]].
* `org.apache.spark.SparkContext.setLocalProperty`.
*/
def getLocalProperty(key: String): String =
Option(localProperties.get).map(_.getProperty(key)).orNull
Expand All @@ -663,7 +663,7 @@ class SparkContext(config: SparkConf) extends Logging {
* Application programmers can use this method to group all those jobs together and give a
* group description. Once set, the Spark web UI will associate such jobs with this group.
*
* The application can also use [[org.apache.spark.SparkContext.cancelJobGroup]] to cancel all
* The application can also use `org.apache.spark.SparkContext.cancelJobGroup` to cancel all
* running jobs in this group. For example,
* {{{
* // In the main thread:
Expand Down Expand Up @@ -1384,7 +1384,7 @@ class SparkContext(config: SparkConf) extends Logging {
}

/**
* Create and register a [[CollectionAccumulator]], which starts with empty list and accumulates
* Create and register a `CollectionAccumulator`, which starts with empty list and accumulates
* inputs by adding them into the list.
*/
def collectionAccumulator[T]: CollectionAccumulator[T] = {
Expand All @@ -1394,7 +1394,7 @@ class SparkContext(config: SparkConf) extends Logging {
}

/**
* Create and register a [[CollectionAccumulator]], which starts with empty list and accumulates
* Create and register a `CollectionAccumulator`, which starts with empty list and accumulates
* inputs by adding them into the list.
*/
def collectionAccumulator[T](name: String): CollectionAccumulator[T] = {
Expand Down Expand Up @@ -2043,7 +2043,7 @@ class SparkContext(config: SparkConf) extends Logging {
}

/**
* Cancel active jobs for the specified group. See [[org.apache.spark.SparkContext.setJobGroup]]
* Cancel active jobs for the specified group. See `org.apache.spark.SparkContext.setJobGroup`
* for more information.
*/
def cancelJobGroup(groupId: String) {
Expand All @@ -2061,7 +2061,7 @@ class SparkContext(config: SparkConf) extends Logging {
* Cancel a given job if it's scheduled or running.
*
* @param jobId the job ID to cancel
* @throws InterruptedException if the cancel message cannot be sent
* @note Throws `InterruptedException` if the cancel message cannot be sent
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hm, InterruptedException is in java.lang so I am surprised it isn't found. Does it help if you write @throws java.lang.InterruptedException? that's better if it works

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure, I will try!

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hm.. interesting. I haven't looked into this deeper but it seems it fails anyway.

[error] .../java/org/apache/spark/SparkContext.java:1150: error: exception not thrown: java.lang.InterruptedException
[error]    * @throws java.lang.InterruptedException if the cancel message cannot be sent
[error]              ^

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hm, so it's just complaining that it's documented as a checked exception but can't be thrown according to the byte code. It has a point there, but I am also kind of surprised it's an error. OK Leave it the way you have it as it seems like it's the only way that works.

*/
def cancelJob(jobId: Int) {
dagScheduler.cancelJob(jobId)
Expand All @@ -2071,7 +2071,7 @@ class SparkContext(config: SparkConf) extends Logging {
* Cancel a given stage and all jobs associated with it.
*
* @param stageId the stage ID to cancel
* @throws InterruptedException if the cancel message cannot be sent
* @note Throws `InterruptedException` if the cancel message cannot be sent
*/
def cancelStage(stageId: Int) {
dagScheduler.cancelStage(stageId)
Expand Down
4 changes: 2 additions & 2 deletions core/src/main/scala/org/apache/spark/TaskContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -164,7 +164,7 @@ abstract class TaskContext extends Serializable {

/**
* Get a local property set upstream in the driver, or null if it is missing. See also
* [[org.apache.spark.SparkContext.setLocalProperty]].
* `org.apache.spark.SparkContext.setLocalProperty`.
*/
def getLocalProperty(key: String): String

Expand All @@ -174,7 +174,7 @@ abstract class TaskContext extends Serializable {
/**
* ::DeveloperApi::
* Returns all metrics sources with the given name which are associated with the instance
* which runs the task. For more information see [[org.apache.spark.metrics.MetricsSystem!]].
* which runs the task. For more information see `org.apache.spark.metrics.MetricsSystem`.
*/
@DeveloperApi
def getMetricsSources(sourceName: String): Seq[Source]
Expand Down
2 changes: 1 addition & 1 deletion core/src/main/scala/org/apache/spark/TaskEndReason.scala
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,7 @@ sealed trait TaskFailedReason extends TaskEndReason {

/**
* :: DeveloperApi ::
* A [[org.apache.spark.scheduler.ShuffleMapTask]] that completed successfully earlier, but we
* A `org.apache.spark.scheduler.ShuffleMapTask` that completed successfully earlier, but we
* lost the executor before the stage completed. This means Spark needs to reschedule the task
* to be re-executed on a different executor.
*/
Expand Down
2 changes: 1 addition & 1 deletion core/src/main/scala/org/apache/spark/TestUtils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -186,7 +186,7 @@ private[spark] object TestUtils {


/**
* A [[SparkListener]] that detects whether spills have occurred in Spark jobs.
* A `SparkListener` that detects whether spills have occurred in Spark jobs.
*/
private class SpillListener extends SparkListener {
private val stageIdToTaskMetrics = new mutable.HashMap[Int, ArrayBuffer[TaskMetrics]]
Expand Down
8 changes: 5 additions & 3 deletions core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala
Original file line number Diff line number Diff line change
Expand Up @@ -103,7 +103,8 @@ class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T])
* @param withReplacement can elements be sampled multiple times (replaced when sampled out)
* @param fraction expected size of the sample as a fraction of this RDD's size
* without replacement: probability that each element is chosen; fraction must be [0, 1]
* with replacement: expected number of times each element is chosen; fraction must be >= 0
* with replacement: expected number of times each element is chosen; fraction must be greater
* than or equal to 0
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I can work around this to {@code >=} if this looks too verbose.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Prose is fine too, but, was this an error? is the point that the escape itself gets escaped? I might have missed this is your various comments.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oh, yes. Exactly. It was not an error but it was printed as they are (> as >). I had a hard time to figure this out in #16013 (comment)

I haven't looked into this super deeper as the output is incorrect anyway but I suspect this replacement in genjavadoc.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In short, the only workarounds to print them I found were inlined tags such as {@code ...} and {{{...}}} block with newlines.

*
* @note This is NOT guaranteed to provide exactly the fraction of the count
* of the given `RDD`.
Expand All @@ -117,7 +118,8 @@ class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T])
* @param withReplacement can elements be sampled multiple times (replaced when sampled out)
* @param fraction expected size of the sample as a fraction of this RDD's size
* without replacement: probability that each element is chosen; fraction must be [0, 1]
* with replacement: expected number of times each element is chosen; fraction must be >= 0
* with replacement: expected number of times each element is chosen; fraction must be greater
* than or equal to 0
* @param seed seed for the random number generator
*
* @note This is NOT guaranteed to provide exactly the fraction of the count
Expand Down Expand Up @@ -167,7 +169,7 @@ class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T])
* Return an RDD with the elements from `this` that are not in `other`.
*
* Uses `this` partitioner/partition size, because even if `other` is huge, the resulting
* RDD will be <= us.
* RDD will be less than or equal to us.
*/
def subtract(other: JavaRDD[T]): JavaRDD[T] = wrapRDD(rdd.subtract(other))

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -152,10 +152,10 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable {

/**
* Compute a histogram using the provided buckets. The buckets are all open
* to the right except for the last which is closed
* to the right except for the last which is closed.
* e.g. for the array
* [1, 10, 20, 50] the buckets are [1, 10) [10, 20) [20, 50]
* e.g 1<=x<10 , 10<=x<20, 20<=x<=50
* e.g {@code <=x<10, 10<=x<20, 20<=x<=50}
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  • Java
    2016-11-28 3 03 44

  • Scala
    2016-11-28 3 03 31

* And on the input of 1 and 50 we would have a histogram of 1, 0, 1
*
* @note If your histogram is evenly spaced (e.g. [0, 10, 20, 30]) this can be switched
Expand Down
2 changes: 1 addition & 1 deletion core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
Original file line number Diff line number Diff line change
Expand Up @@ -96,7 +96,7 @@ private[spark] class HadoopPartition(rddId: Int, override val index: Int, s: Inp
* @param minPartitions Minimum number of HadoopRDD partitions (Hadoop Splits) to generate.
*
* @note Instantiating this class directly is not recommended, please use
* [[org.apache.spark.SparkContext.hadoopRDD()]]
* `org.apache.spark.SparkContext.hadoopRDD()`
*/
@DeveloperApi
class HadoopRDD[K, V](
Expand Down
15 changes: 12 additions & 3 deletions core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,10 @@ private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) e
* The RDD takes care of closing the connection.
* @param sql the text of the query.
* The query must contain two ? placeholders for parameters used to partition the results.
* E.g. "select title, author from books where ? <= id and id <= ?"
* For example,
* {{{
* select title, author from books where ? <= id and id <= ?
* }}}
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  • Java
    2016-11-27 4 30 08

  • Scala
    2016-11-27 4 31 55

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  • Java
    2016-11-28 2 21 59

  • Scala
    2016-11-28 2 22 34

* @param lowerBound the minimum value of the first placeholder
* @param upperBound the maximum value of the second placeholder
* The lower and upper bounds are inclusive.
Expand Down Expand Up @@ -151,7 +154,10 @@ object JdbcRDD {
* The RDD takes care of closing the connection.
* @param sql the text of the query.
* The query must contain two ? placeholders for parameters used to partition the results.
* E.g. "select title, author from books where ? <= id and id <= ?"
* For example,
* {{{
* select title, author from books where ? <= id and id <= ?
* }}}
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  • Java
    2016-11-27 4 29 07

  • Scala
    2016-11-27 4 31 51

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Same comment on the rest, ideally, can we do this?

For example: select title ... <= ?

or at best

For example:

select title ... <= ?

* @param lowerBound the minimum value of the first placeholder
* @param upperBound the maximum value of the second placeholder
* The lower and upper bounds are inclusive.
Expand Down Expand Up @@ -191,7 +197,10 @@ object JdbcRDD {
* The RDD takes care of closing the connection.
* @param sql the text of the query.
* The query must contain two ? placeholders for parameters used to partition the results.
* E.g. "select title, author from books where ? <= id and id <= ?"
* For example,
* {{{
* select title, author from books where ? <= id and id <= ?
* }}}
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  • Java
    2016-11-27 4 30 08

  • Scala
    2016-11-27 4 30 17

* @param lowerBound the minimum value of the first placeholder
* @param upperBound the maximum value of the second placeholder
* The lower and upper bounds are inclusive.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,7 @@ private[spark] class NewHadoopPartition(
* @param valueClass Class of the value associated with the inputFormatClass.
*
* @note Instantiating this class directly is not recommended, please use
* [[org.apache.spark.SparkContext.newAPIHadoopRDD()]]
* `org.apache.spark.SparkContext.newAPIHadoopRDD()`
*/
@DeveloperApi
class NewHadoopRDD[K, V](
Expand Down
20 changes: 10 additions & 10 deletions core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
Original file line number Diff line number Diff line change
Expand Up @@ -399,9 +399,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available
* <a href="http://dx.doi.org/10.1145/2452376.2452456">here</a>.
*
* The relative accuracy is approximately `1.054 / sqrt(2^p)`. Setting a nonzero `sp > p`
* would trigger sparse representation of registers, which may reduce the memory consumption
* and increase accuracy when the cardinality is small.
* The relative accuracy is approximately `1.054 / sqrt(2^p)`. Setting a nonzero (`sp` is
* greater than `p`) would trigger sparse representation of registers, which may reduce the
* memory consumption and increase accuracy when the cardinality is small.
*
* @param p The precision value for the normal set.
* `p` must be a value between 4 and `sp` if `sp` is not zero (32 max).
Expand Down Expand Up @@ -492,8 +492,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* each time the resulting RDD is evaluated.
*
* @note This operation may be very expensive. If you are grouping in order to perform an
* aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]]
* or [[PairRDDFunctions.reduceByKey]] will provide much better performance.
* aggregation (such as a sum or average) over each key, using `PairRDDFunctions.aggregateByKey`
* or `PairRDDFunctions.reduceByKey` will provide much better performance.
*
* @note As currently implemented, groupByKey must be able to hold all the key-value pairs for any
* key in memory. If a key has too many values, it can result in an [[OutOfMemoryError]].
Expand All @@ -516,8 +516,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* each group is not guaranteed, and may even differ each time the resulting RDD is evaluated.
*
* @note This operation may be very expensive. If you are grouping in order to perform an
* aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]]
* or [[PairRDDFunctions.reduceByKey]] will provide much better performance.
* aggregation (such as a sum or average) over each key, using `PairRDDFunctions.aggregateByKey`
* or `PairRDDFunctions.reduceByKey` will provide much better performance.
*
* @note As currently implemented, groupByKey must be able to hold all the key-value pairs for any
* key in memory. If a key has too many values, it can result in an [[OutOfMemoryError]].
Expand Down Expand Up @@ -637,8 +637,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* evaluated.
*
* @note This operation may be very expensive. If you are grouping in order to perform an
* aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]]
* or [[PairRDDFunctions.reduceByKey]] will provide much better performance.
* aggregation (such as a sum or average) over each key, using `PairRDDFunctions.aggregateByKey`
* or `PairRDDFunctions.reduceByKey` will provide much better performance.
*/
def groupByKey(): RDD[(K, Iterable[V])] = self.withScope {
groupByKey(defaultPartitioner(self))
Expand Down Expand Up @@ -908,7 +908,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* Return an RDD with the pairs from `this` whose keys are not in `other`.
*
* Uses `this` partitioner/partition size, because even if `other` is huge, the resulting
* RDD will be <= us.
* RDD will be less than or equal to us.
*/
def subtractByKey[W: ClassTag](other: RDD[(K, W)]): RDD[(K, V)] = self.withScope {
subtractByKey(other, self.partitioner.getOrElse(new HashPartitioner(self.partitions.length)))
Expand Down
24 changes: 16 additions & 8 deletions core/src/main/scala/org/apache/spark/rdd/RDD.scala
Original file line number Diff line number Diff line change
Expand Up @@ -469,7 +469,8 @@ abstract class RDD[T: ClassTag](
* @param withReplacement can elements be sampled multiple times (replaced when sampled out)
* @param fraction expected size of the sample as a fraction of this RDD's size
* without replacement: probability that each element is chosen; fraction must be [0, 1]
* with replacement: expected number of times each element is chosen; fraction must be &gt;= 0
* with replacement: expected number of times each element is chosen; fraction must be greater
* than or equal to 0
* @param seed seed for the random number generator
*
* @note This is NOT guaranteed to provide exactly the fraction of the count
Expand Down Expand Up @@ -750,8 +751,10 @@ abstract class RDD[T: ClassTag](
* print line function (like out.println()) as the 2nd parameter.
* An example of pipe the RDD data of groupBy() in a streaming way,
* instead of constructing a huge String to concat all the elements:
* def printRDDElement(record:(String, Seq[String]), f:String=&gt;Unit) =
* for (e &lt;- record._2) {f(e)}
* {{{
* def printRDDElement(record:(String, Seq[String]), f:String=>Unit) =
* for (e <- record._2) {f(e)}
* }}}
Copy link
Member Author

@HyukjinKwon HyukjinKwon Nov 26, 2016

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  • Scala
    2016-11-28 12 47 08

  • Java
    2016-11-28 12 46 45

* @param separateWorkingDir Use separate working directories for each task.
* @param bufferSize Buffer size for the stdin writer for the piped process.
* @param encoding Char encoding used for interacting (via stdin, stdout and stderr) with
Expand Down Expand Up @@ -1184,8 +1187,13 @@ abstract class RDD[T: ClassTag](
*
* @note This method should only be used if the resulting map is expected to be small, as
* the whole thing is loaded into the driver's memory.
* To handle very large results, consider using rdd.map(x =&gt; (x, 1L)).reduceByKey(_ + _), which
* returns an RDD[T, Long] instead of a map.
* To handle very large results, consider using
*
* {{{
* rdd.map(x => (x, 1L)).reduceByKey(_ + _)
* }}}
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  • Scala

    2016-11-27 3 13 26
  • Java

    2016-11-27 3 13 08

*
* , which returns an RDD[T, Long] instead of a map.
*/
def countByValue()(implicit ord: Ordering[T] = null): Map[T, Long] = withScope {
map(value => (value, null)).countByKey()
Expand Down Expand Up @@ -1223,9 +1231,9 @@ abstract class RDD[T: ClassTag](
* Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available
* <a href="http://dx.doi.org/10.1145/2452376.2452456">here</a>.
*
* The relative accuracy is approximately `1.054 / sqrt(2^p)`. Setting a nonzero `sp &gt; p`
* would trigger sparse representation of registers, which may reduce the memory consumption
* and increase accuracy when the cardinality is small.
* The relative accuracy is approximately `1.054 / sqrt(2^p)`. Setting a nonzero (`sp` is greater
* than `p`) would trigger sparse representation of registers, which may reduce the memory
* consumption and increase accuracy when the cardinality is small.
*
* @param p The precision value for the normal set.
* `p` must be a value between 4 and `sp` if `sp` is not zero (32 max).
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,8 @@ import org.apache.spark.Partition

/**
* Enumeration to manage state transitions of an RDD through checkpointing
* [ Initialized --> checkpointing in progress --> checkpointed ].
*
* [ Initialized --{@literal >} checkpointing in progress --{@literal >} checkpointed ]
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  • Java
    2016-11-28 3 41 11

Scaladoc not found.

*/
private[spark] object CheckpointState extends Enumeration {
type CheckpointState = Value
Expand Down
Loading