Skip to content

Commit

Permalink
[SPARK-3249][DOC] Fix links in ScalaDoc that cause warning messages i…
Browse files Browse the repository at this point in the history
…n `sbt/sbt unidoc`

## What changes were proposed in this pull request?

This PR proposes to fix ambiguous link warnings by simply making them as code blocks for both javadoc and scaladoc.

```
[warn] .../spark/core/src/main/scala/org/apache/spark/Accumulator.scala:20: The link target "SparkContext#accumulator" is ambiguous. Several members fit the target:
[warn] .../spark/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala:281: The link target "runMiniBatchSGD" is ambiguous. Several members fit the target:
[warn] .../spark/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala:83: The link target "run" is ambiguous. Several members fit the target:
...
```

This PR also fixes javadoc8 break as below:

```
[error] .../spark/sql/core/target/java/org/apache/spark/sql/LowPrioritySQLImplicits.java:7: error: reference not found
[error]  * newProductEncoder - to disambiguate for {link List}s which are both {link Seq} and {link Product}
[error]                                                   ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/LowPrioritySQLImplicits.java:7: error: reference not found
[error]  * newProductEncoder - to disambiguate for {link List}s which are both {link Seq} and {link Product}
[error]                                                                                ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/LowPrioritySQLImplicits.java:7: error: reference not found
[error]  * newProductEncoder - to disambiguate for {link List}s which are both {link Seq} and {link Product}
[error]                                                                                                ^
[info] 3 errors
```

## How was this patch tested?

Manually via `sbt unidoc > output.txt` and the checked it via `cat output.txt | grep ambiguous`

and `sbt unidoc | grep error`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16604 from HyukjinKwon/SPARK-3249.
  • Loading branch information
HyukjinKwon authored and srowen committed Jan 17, 2017
1 parent 0019005 commit 6c00c06
Show file tree
Hide file tree
Showing 20 changed files with 93 additions and 86 deletions.
3 changes: 1 addition & 2 deletions core/src/main/scala/org/apache/spark/Accumulator.scala
Original file line number Diff line number Diff line change
Expand Up @@ -24,8 +24,7 @@ package org.apache.spark
* They can be used to implement counters (as in MapReduce) or sums. Spark natively supports
* accumulators of numeric value types, and programmers can add support for new types.
*
* An accumulator is created from an initial value `v` by calling
* [[SparkContext#accumulator SparkContext.accumulator]].
* An accumulator is created from an initial value `v` by calling `SparkContext.accumulator`.
* Tasks running on the cluster can then add to it using the `+=` operator.
* However, they cannot read its value. Only the driver program can read the accumulator's value,
* using its [[#value]] method.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -166,7 +166,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])
* Return a subset of this RDD sampled by key (via stratified sampling) containing exactly
* math.ceil(numItems * samplingRate) for each stratum (group of pairs with the same key).
*
* This method differs from [[sampleByKey]] in that we make additional passes over the RDD to
* This method differs from `sampleByKey` in that we make additional passes over the RDD to
* create a sample size that's exactly equal to the sum of math.ceil(numItems * samplingRate)
* over all key values with a 99.99% confidence. When sampling without replacement, we need one
* additional pass over the RDD to guarantee sample size; when sampling with replacement, we need
Expand All @@ -184,7 +184,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])
* Return a subset of this RDD sampled by key (via stratified sampling) containing exactly
* math.ceil(numItems * samplingRate) for each stratum (group of pairs with the same key).
*
* This method differs from [[sampleByKey]] in that we make additional passes over the RDD to
* This method differs from `sampleByKey` in that we make additional passes over the RDD to
* create a sample size that's exactly equal to the sum of math.ceil(numItems * samplingRate)
* over all key values with a 99.99% confidence. When sampling without replacement, we need one
* additional pass over the RDD to guarantee sample size; when sampling with replacement, we need
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -393,7 +393,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
def treeReduce(f: JFunction2[T, T, T], depth: Int): T = rdd.treeReduce(f, depth)

/**
* [[org.apache.spark.api.java.JavaRDDLike#treeReduce]] with suggested depth 2.
* `org.apache.spark.api.java.JavaRDDLike.treeReduce` with suggested depth 2.
*/
def treeReduce(f: JFunction2[T, T, T]): T = treeReduce(f, 2)

Expand Down Expand Up @@ -440,7 +440,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
}

/**
* [[org.apache.spark.api.java.JavaRDDLike#treeAggregate]] with suggested depth 2.
* `org.apache.spark.api.java.JavaRDDLike.treeAggregate` with suggested depth 2.
*/
def treeAggregate[U](
zeroValue: U,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -109,7 +109,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* functions. This method is here for backward compatibility. It does not provide combiner
* classtag information to the shuffle.
*
* @see [[combineByKeyWithClassTag]]
* @see `combineByKeyWithClassTag`
*/
def combineByKey[C](
createCombiner: V => C,
Expand All @@ -127,7 +127,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* This method is here for backward compatibility. It does not provide combiner
* classtag information to the shuffle.
*
* @see [[combineByKeyWithClassTag]]
* @see `combineByKeyWithClassTag`
*/
def combineByKey[C](
createCombiner: V => C,
Expand Down Expand Up @@ -608,7 +608,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* existing partitioner/parallelism level. This method is here for backward compatibility. It
* does not provide combiner classtag information to the shuffle.
*
* @see [[combineByKeyWithClassTag]]
* @see `combineByKeyWithClassTag`
*/
def combineByKey[C](
createCombiner: V => C,
Expand Down
2 changes: 1 addition & 1 deletion core/src/main/scala/org/apache/spark/rdd/RDD.scala
Original file line number Diff line number Diff line change
Expand Up @@ -1841,7 +1841,7 @@ abstract class RDD[T: ClassTag](
* Defines implicit functions that provide extra functionalities on RDDs of specific types.
*
* For example, [[RDD.rddToPairRDDFunctions]] converts an RDD into a [[PairRDDFunctions]] for
* key-value-pair RDDs, and enabling extra functionalities such as [[PairRDDFunctions.reduceByKey]].
* key-value-pair RDDs, and enabling extra functionalities such as `PairRDDFunctions.reduceByKey`.
*/
object RDD {

Expand Down
2 changes: 1 addition & 1 deletion graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
Original file line number Diff line number Diff line change
Expand Up @@ -331,7 +331,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab

/**
* Merges multiple edges between two vertices into a single edge. For correct results, the graph
* must have been partitioned using [[partitionBy]].
* must have been partitioned using `partitionBy`.
*
* @param merge the user-supplied commutative associative function to merge edge attributes
* for duplicate edges.
Expand Down
4 changes: 2 additions & 2 deletions graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
Original file line number Diff line number Diff line change
Expand Up @@ -428,7 +428,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali
* Compute the connected component membership of each vertex and return a graph with the vertex
* value containing the lowest vertex id in the connected component containing that vertex.
*
* @see [[org.apache.spark.graphx.lib.ConnectedComponents$#run]]
* @see `org.apache.spark.graphx.lib.ConnectedComponents.run`
*/
def connectedComponents(): Graph[VertexId, ED] = {
ConnectedComponents.run(graph)
Expand All @@ -438,7 +438,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali
* Compute the connected component membership of each vertex and return a graph with the vertex
* value containing the lowest vertex id in the connected component containing that vertex.
*
* @see [[org.apache.spark.graphx.lib.ConnectedComponents$#run]]
* @see `org.apache.spark.graphx.lib.ConnectedComponents.run`
*/
def connectedComponents(maxIterations: Int): Graph[VertexId, ED] = {
ConnectedComponents.run(graph, maxIterations)
Expand Down
10 changes: 5 additions & 5 deletions mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala
Original file line number Diff line number Diff line change
Expand Up @@ -83,11 +83,11 @@ abstract class PipelineStage extends Params with Logging {

/**
* A simple pipeline, which acts as an estimator. A Pipeline consists of a sequence of stages, each
* of which is either an [[Estimator]] or a [[Transformer]]. When [[Pipeline#fit]] is called, the
* stages are executed in order. If a stage is an [[Estimator]], its [[Estimator#fit]] method will
* of which is either an [[Estimator]] or a [[Transformer]]. When `Pipeline.fit` is called, the
* stages are executed in order. If a stage is an [[Estimator]], its `Estimator.fit` method will
* be called on the input dataset to fit a model. Then the model, which is a transformer, will be
* used to transform the dataset as the input to the next stage. If a stage is a [[Transformer]],
* its [[Transformer#transform]] method will be called to produce the dataset for the next stage.
* its `Transformer.transform` method will be called to produce the dataset for the next stage.
* The fitted model from a [[Pipeline]] is a [[PipelineModel]], which consists of fitted models and
* transformers, corresponding to the pipeline stages. If there are no stages, the pipeline acts as
* an identity transformer.
Expand Down Expand Up @@ -121,9 +121,9 @@ class Pipeline @Since("1.4.0") (

/**
* Fits the pipeline to the input dataset with additional parameters. If a stage is an
* [[Estimator]], its [[Estimator#fit]] method will be called on the input dataset to fit a model.
* [[Estimator]], its `Estimator.fit` method will be called on the input dataset to fit a model.
* Then the model, which is a transformer, will be used to transform the dataset as the input to
* the next stage. If a stage is a [[Transformer]], its [[Transformer#transform]] method will be
* the next stage. If a stage is a [[Transformer]], its `Transformer.transform` method will be
* called to produce the dataset for the next stage. The fitted model from a [[Pipeline]] is an
* [[PipelineModel]], which consists of fitted models and transformers, corresponding to the
* pipeline stages. If there are no stages, the output model acts as an identity transformer.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -728,7 +728,7 @@ trait Params extends Identifiable with Serializable {
}

/**
* [[extractParamMap]] with no extra values.
* `extractParamMap` with no extra values.
*/
final def extractParamMap(): ParamMap = {
extractParamMap(ParamMap.empty)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -237,7 +237,7 @@ class LocalLDAModel private[spark] (
vocabSize)

/**
* Java-friendly version of [[logLikelihood]]
* Java-friendly version of `logLikelihood`
*/
@Since("1.5.0")
def logLikelihood(documents: JavaPairRDD[java.lang.Long, Vector]): Double = {
Expand All @@ -259,7 +259,9 @@ class LocalLDAModel private[spark] (
-logLikelihood(documents) / corpusTokenCount
}

/** Java-friendly version of [[logPerplexity]] */
/**
* Java-friendly version of `logPerplexity`
*/
@Since("1.5.0")
def logPerplexity(documents: JavaPairRDD[java.lang.Long, Vector]): Double = {
logPerplexity(documents.rdd.asInstanceOf[RDD[(Long, Vector)]])
Expand Down Expand Up @@ -365,7 +367,9 @@ class LocalLDAModel private[spark] (
}
}

/** Get a method usable as a UDF for [[topicDistributions()]] */
/**
* Get a method usable as a UDF for `topicDistributions()`
*/
private[spark] def getTopicDistributionMethod(sc: SparkContext): Vector => Vector = {
val expElogbeta = exp(LDAUtils.dirichletExpectation(topicsMatrix.asBreeze.toDenseMatrix.t).t)
val expElogbetaBc = sc.broadcast(expElogbeta)
Expand Down Expand Up @@ -414,7 +418,7 @@ class LocalLDAModel private[spark] (
}

/**
* Java-friendly version of [[topicDistributions]]
* Java-friendly version of `topicDistributions`
*/
@Since("1.4.1")
def topicDistributions(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -80,7 +80,9 @@ class AssociationRules private[fpm] (
}.filter(_.confidence >= minConfidence)
}

/** Java-friendly version of [[run]]. */
/**
* Java-friendly version of `run`.
*/
@Since("1.5.0")
def run[Item](freqItemsets: JavaRDD[FreqItemset[Item]]): JavaRDD[Rule[Item]] = {
val tag = fakeClassTag[Item]
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -218,7 +218,9 @@ class FPGrowth private (
new FPGrowthModel(freqItemsets)
}

/** Java-friendly version of [[run]]. */
/**
* Java-friendly version of `run`.
*/
@Since("1.3.0")
def run[Item, Basket <: JavaIterable[Item]](data: JavaRDD[Basket]): FPGrowthModel[Item] = {
implicit val tag = fakeClassTag[Item]
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -279,7 +279,7 @@ object GradientDescent extends Logging {
}

/**
* Alias of [[runMiniBatchSGD]] with convergenceTol set to default value of 0.001.
* Alias of `runMiniBatchSGD` with convergenceTol set to default value of 0.001.
*/
def runMiniBatchSGD(
data: RDD[(Double, Vector)],
Expand Down

0 comments on commit 6c00c06

Please sign in to comment.