Repository: spark
Updated Branches:
  refs/heads/master 0019005a2 -> 6c00c069e


[SPARK-3249][DOC] Fix links in ScalaDoc that cause warning messages in `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 <gurwls...@gmail.com>

Closes #16604 from HyukjinKwon/SPARK-3249.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/6c00c069
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/6c00c069
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/6c00c069

Branch: refs/heads/master
Commit: 6c00c069e3c3f5904abd122cea1d56683031cca0
Parents: 0019005
Author: hyukjinkwon <gurwls...@gmail.com>
Authored: Tue Jan 17 12:28:15 2017 +0000
Committer: Sean Owen <so...@cloudera.com>
Committed: Tue Jan 17 12:28:15 2017 +0000

----------------------------------------------------------------------
 .../scala/org/apache/spark/Accumulator.scala    |  3 +-
 .../org/apache/spark/api/java/JavaPairRDD.scala |  4 +-
 .../org/apache/spark/api/java/JavaRDDLike.scala |  4 +-
 .../org/apache/spark/rdd/PairRDDFunctions.scala |  6 +-
 .../main/scala/org/apache/spark/rdd/RDD.scala   |  2 +-
 .../scala/org/apache/spark/graphx/Graph.scala   |  2 +-
 .../org/apache/spark/graphx/GraphOps.scala      |  4 +-
 .../scala/org/apache/spark/ml/Pipeline.scala    | 10 +--
 .../org/apache/spark/ml/param/params.scala      |  2 +-
 .../spark/mllib/clustering/LDAModel.scala       | 12 ++-
 .../spark/mllib/fpm/AssociationRules.scala      |  4 +-
 .../org/apache/spark/mllib/fpm/FPGrowth.scala   |  4 +-
 .../mllib/optimization/GradientDescent.scala    |  2 +-
 .../apache/spark/mllib/random/RandomRDDs.scala  | 82 ++++++++++----------
 .../apache/spark/mllib/tree/DecisionTree.scala  | 20 ++---
 .../spark/mllib/tree/GradientBoostedTrees.scala |  2 +-
 .../apache/spark/mllib/tree/RandomForest.scala  |  4 +-
 .../org/apache/spark/mllib/util/MLUtils.scala   |  2 +-
 .../org/apache/spark/sql/SQLImplicits.scala     |  2 +-
 .../scala/org/apache/spark/sql/functions.scala  |  8 +-
 20 files changed, 93 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/6c00c069/core/src/main/scala/org/apache/spark/Accumulator.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/Accumulator.scala 
b/core/src/main/scala/org/apache/spark/Accumulator.scala
index 7bea636..9d5fbef 100644
--- a/core/src/main/scala/org/apache/spark/Accumulator.scala
+++ b/core/src/main/scala/org/apache/spark/Accumulator.scala
@@ -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.

http://git-wip-us.apache.org/repos/asf/spark/blob/6c00c069/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala
----------------------------------------------------------------------
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 766aea2..9544475 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
@@ -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
@@ -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

http://git-wip-us.apache.org/repos/asf/spark/blob/6c00c069/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
----------------------------------------------------------------------
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 eda16d9..91ae100 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
@@ -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)
 
@@ -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,

http://git-wip-us.apache.org/repos/asf/spark/blob/6c00c069/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala 
b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
index ec12b99..d7bfdba 100644
--- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
@@ -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,
@@ -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,
@@ -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,

http://git-wip-us.apache.org/repos/asf/spark/blob/6c00c069/core/src/main/scala/org/apache/spark/rdd/RDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala 
b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
index 374abcc..a7e01f3 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
@@ -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 {
 

http://git-wip-us.apache.org/repos/asf/spark/blob/6c00c069/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala 
b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
index c55a588..b3a3420 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
@@ -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.

http://git-wip-us.apache.org/repos/asf/spark/blob/6c00c069/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala 
b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
index 9090730..475bccf 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
@@ -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)
@@ -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)

http://git-wip-us.apache.org/repos/asf/spark/blob/6c00c069/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala 
b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala
index 08e9cb9..b76dc5f 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala
@@ -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.
@@ -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.

http://git-wip-us.apache.org/repos/asf/spark/blob/6c00c069/mllib/src/main/scala/org/apache/spark/ml/param/params.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala 
b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala
index 9adb0fa..ab0620c 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala
@@ -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)

http://git-wip-us.apache.org/repos/asf/spark/blob/6c00c069/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala
----------------------------------------------------------------------
diff --git 
a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala 
b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala
index ae33698..7fd722a 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala
@@ -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 = {
@@ -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)]])
@@ -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)
@@ -414,7 +418,7 @@ class LocalLDAModel private[spark] (
   }
 
   /**
-   * Java-friendly version of [[topicDistributions]]
+   * Java-friendly version of `topicDistributions`
    */
   @Since("1.4.1")
   def topicDistributions(

http://git-wip-us.apache.org/repos/asf/spark/blob/6c00c069/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala
----------------------------------------------------------------------
diff --git 
a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala 
b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala
index dca0314..85a90fa 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala
@@ -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]

http://git-wip-us.apache.org/repos/asf/spark/blob/6c00c069/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala 
b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala
index e3cf0d4..635da00 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala
@@ -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]

http://git-wip-us.apache.org/repos/asf/spark/blob/6c00c069/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala
----------------------------------------------------------------------
diff --git 
a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala
 
b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala
index 8979707..07a67a9 100644
--- 
a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala
+++ 
b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala
@@ -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)],

http://git-wip-us.apache.org/repos/asf/spark/blob/6c00c069/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala
----------------------------------------------------------------------
diff --git 
a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala 
b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala
index 85d4d7f..258b176 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala
@@ -57,7 +57,7 @@ object RandomRDDs {
   }
 
   /**
-   * Java-friendly version of [[RandomRDDs#uniformRDD]].
+   * Java-friendly version of `RandomRDDs.uniformRDD`.
    */
   @Since("1.1.0")
   def uniformJavaRDD(
@@ -69,7 +69,7 @@ object RandomRDDs {
   }
 
   /**
-   * [[RandomRDDs#uniformJavaRDD]] with the default seed.
+   * `RandomRDDs.uniformJavaRDD` with the default seed.
    */
   @Since("1.1.0")
   def uniformJavaRDD(jsc: JavaSparkContext, size: Long, numPartitions: Int): 
JavaDoubleRDD = {
@@ -77,7 +77,7 @@ object RandomRDDs {
   }
 
   /**
-   * [[RandomRDDs#uniformJavaRDD]] with the default number of partitions and 
the default seed.
+   * `RandomRDDs.uniformJavaRDD` with the default number of partitions and the 
default seed.
    */
   @Since("1.1.0")
   def uniformJavaRDD(jsc: JavaSparkContext, size: Long): JavaDoubleRDD = {
@@ -107,7 +107,7 @@ object RandomRDDs {
   }
 
   /**
-   * Java-friendly version of [[RandomRDDs#normalRDD]].
+   * Java-friendly version of `RandomRDDs.normalRDD`.
    */
   @Since("1.1.0")
   def normalJavaRDD(
@@ -119,7 +119,7 @@ object RandomRDDs {
   }
 
   /**
-   * [[RandomRDDs#normalJavaRDD]] with the default seed.
+   * `RandomRDDs.normalJavaRDD` with the default seed.
    */
   @Since("1.1.0")
   def normalJavaRDD(jsc: JavaSparkContext, size: Long, numPartitions: Int): 
JavaDoubleRDD = {
@@ -127,7 +127,7 @@ object RandomRDDs {
   }
 
   /**
-   * [[RandomRDDs#normalJavaRDD]] with the default number of partitions and 
the default seed.
+   * `RandomRDDs.normalJavaRDD` with the default number of partitions and the 
default seed.
    */
   @Since("1.1.0")
   def normalJavaRDD(jsc: JavaSparkContext, size: Long): JavaDoubleRDD = {
@@ -157,7 +157,7 @@ object RandomRDDs {
   }
 
   /**
-   * Java-friendly version of [[RandomRDDs#poissonRDD]].
+   * Java-friendly version of `RandomRDDs.poissonRDD`.
    */
   @Since("1.1.0")
   def poissonJavaRDD(
@@ -170,7 +170,7 @@ object RandomRDDs {
   }
 
   /**
-   * [[RandomRDDs#poissonJavaRDD]] with the default seed.
+   * `RandomRDDs.poissonJavaRDD` with the default seed.
    */
   @Since("1.1.0")
   def poissonJavaRDD(
@@ -182,7 +182,7 @@ object RandomRDDs {
   }
 
   /**
-   * [[RandomRDDs#poissonJavaRDD]] with the default number of partitions and 
the default seed.
+   * `RandomRDDs.poissonJavaRDD` with the default number of partitions and the 
default seed.
    */
   @Since("1.1.0")
   def poissonJavaRDD(jsc: JavaSparkContext, mean: Double, size: Long): 
JavaDoubleRDD = {
@@ -212,7 +212,7 @@ object RandomRDDs {
   }
 
   /**
-   * Java-friendly version of [[RandomRDDs#exponentialRDD]].
+   * Java-friendly version of `RandomRDDs.exponentialRDD`.
    */
   @Since("1.3.0")
   def exponentialJavaRDD(
@@ -225,7 +225,7 @@ object RandomRDDs {
   }
 
   /**
-   * [[RandomRDDs#exponentialJavaRDD]] with the default seed.
+   * `RandomRDDs.exponentialJavaRDD` with the default seed.
    */
   @Since("1.3.0")
   def exponentialJavaRDD(
@@ -237,7 +237,7 @@ object RandomRDDs {
   }
 
   /**
-   * [[RandomRDDs#exponentialJavaRDD]] with the default number of partitions 
and the default seed.
+   * `RandomRDDs.exponentialJavaRDD` with the default number of partitions and 
the default seed.
    */
   @Since("1.3.0")
   def exponentialJavaRDD(jsc: JavaSparkContext, mean: Double, size: Long): 
JavaDoubleRDD = {
@@ -269,7 +269,7 @@ object RandomRDDs {
   }
 
   /**
-   * Java-friendly version of [[RandomRDDs#gammaRDD]].
+   * Java-friendly version of `RandomRDDs.gammaRDD`.
    */
   @Since("1.3.0")
   def gammaJavaRDD(
@@ -283,7 +283,7 @@ object RandomRDDs {
   }
 
   /**
-   * [[RandomRDDs#gammaJavaRDD]] with the default seed.
+   * `RandomRDDs.gammaJavaRDD` with the default seed.
    */
   @Since("1.3.0")
   def gammaJavaRDD(
@@ -296,7 +296,7 @@ object RandomRDDs {
   }
 
   /**
-   * [[RandomRDDs#gammaJavaRDD]] with the default number of partitions and the 
default seed.
+   * `RandomRDDs.gammaJavaRDD` with the default number of partitions and the 
default seed.
    */
   @Since("1.3.0")
   def gammaJavaRDD(
@@ -332,7 +332,7 @@ object RandomRDDs {
   }
 
   /**
-   * Java-friendly version of [[RandomRDDs#logNormalRDD]].
+   * Java-friendly version of `RandomRDDs.logNormalRDD`.
    */
   @Since("1.3.0")
   def logNormalJavaRDD(
@@ -346,7 +346,7 @@ object RandomRDDs {
   }
 
   /**
-   * [[RandomRDDs#logNormalJavaRDD]] with the default seed.
+   * `RandomRDDs.logNormalJavaRDD` with the default seed.
    */
   @Since("1.3.0")
   def logNormalJavaRDD(
@@ -359,7 +359,7 @@ object RandomRDDs {
   }
 
   /**
-   * [[RandomRDDs#logNormalJavaRDD]] with the default number of partitions and 
the default seed.
+   * `RandomRDDs.logNormalJavaRDD` with the default number of partitions and 
the default seed.
    */
   @Since("1.3.0")
   def logNormalJavaRDD(
@@ -419,7 +419,7 @@ object RandomRDDs {
 
   /**
    * :: DeveloperApi ::
-   * [[RandomRDDs#randomJavaRDD]] with the default seed.
+   * `RandomRDDs.randomJavaRDD` with the default seed.
    */
   @DeveloperApi
   @Since("1.6.0")
@@ -433,7 +433,7 @@ object RandomRDDs {
 
   /**
    * :: DeveloperApi ::
-   * [[RandomRDDs#randomJavaRDD]] with the default seed & numPartitions
+   * `RandomRDDs.randomJavaRDD` with the default seed & numPartitions
    */
   @DeveloperApi
   @Since("1.6.0")
@@ -469,7 +469,7 @@ object RandomRDDs {
   }
 
   /**
-   * Java-friendly version of [[RandomRDDs#uniformVectorRDD]].
+   * Java-friendly version of `RandomRDDs.uniformVectorRDD`.
    */
   @Since("1.1.0")
   def uniformJavaVectorRDD(
@@ -482,7 +482,7 @@ object RandomRDDs {
   }
 
   /**
-   * [[RandomRDDs#uniformJavaVectorRDD]] with the default seed.
+   * `RandomRDDs.uniformJavaVectorRDD` with the default seed.
    */
   @Since("1.1.0")
   def uniformJavaVectorRDD(
@@ -494,7 +494,7 @@ object RandomRDDs {
   }
 
   /**
-   * [[RandomRDDs#uniformJavaVectorRDD]] with the default number of partitions 
and the default seed.
+   * `RandomRDDs.uniformJavaVectorRDD` with the default number of partitions 
and the default seed.
    */
   @Since("1.1.0")
   def uniformJavaVectorRDD(
@@ -527,7 +527,7 @@ object RandomRDDs {
   }
 
   /**
-   * Java-friendly version of [[RandomRDDs#normalVectorRDD]].
+   * Java-friendly version of `RandomRDDs.normalVectorRDD`.
    */
   @Since("1.1.0")
   def normalJavaVectorRDD(
@@ -540,7 +540,7 @@ object RandomRDDs {
   }
 
   /**
-   * [[RandomRDDs#normalJavaVectorRDD]] with the default seed.
+   * `RandomRDDs.normalJavaVectorRDD` with the default seed.
    */
   @Since("1.1.0")
   def normalJavaVectorRDD(
@@ -552,7 +552,7 @@ object RandomRDDs {
   }
 
   /**
-   * [[RandomRDDs#normalJavaVectorRDD]] with the default number of partitions 
and the default seed.
+   * `RandomRDDs.normalJavaVectorRDD` with the default number of partitions 
and the default seed.
    */
   @Since("1.1.0")
   def normalJavaVectorRDD(
@@ -590,7 +590,7 @@ object RandomRDDs {
   }
 
   /**
-   * Java-friendly version of [[RandomRDDs#logNormalVectorRDD]].
+   * Java-friendly version of `RandomRDDs.logNormalVectorRDD`.
    */
   @Since("1.3.0")
   def logNormalJavaVectorRDD(
@@ -605,7 +605,7 @@ object RandomRDDs {
   }
 
   /**
-   * [[RandomRDDs#logNormalJavaVectorRDD]] with the default seed.
+   * `RandomRDDs.logNormalJavaVectorRDD` with the default seed.
    */
   @Since("1.3.0")
   def logNormalJavaVectorRDD(
@@ -619,7 +619,7 @@ object RandomRDDs {
   }
 
   /**
-   * [[RandomRDDs#logNormalJavaVectorRDD]] with the default number of 
partitions and
+   * `RandomRDDs.logNormalJavaVectorRDD` with the default number of partitions 
and
    * the default seed.
    */
   @Since("1.3.0")
@@ -657,7 +657,7 @@ object RandomRDDs {
   }
 
   /**
-   * Java-friendly version of [[RandomRDDs#poissonVectorRDD]].
+   * Java-friendly version of `RandomRDDs.poissonVectorRDD`.
    */
   @Since("1.1.0")
   def poissonJavaVectorRDD(
@@ -671,7 +671,7 @@ object RandomRDDs {
   }
 
   /**
-   * [[RandomRDDs#poissonJavaVectorRDD]] with the default seed.
+   * `RandomRDDs.poissonJavaVectorRDD` with the default seed.
    */
   @Since("1.1.0")
   def poissonJavaVectorRDD(
@@ -684,7 +684,7 @@ object RandomRDDs {
   }
 
   /**
-   * [[RandomRDDs#poissonJavaVectorRDD]] with the default number of partitions 
and the default seed.
+   * `RandomRDDs.poissonJavaVectorRDD` with the default number of partitions 
and the default seed.
    */
   @Since("1.1.0")
   def poissonJavaVectorRDD(
@@ -721,7 +721,7 @@ object RandomRDDs {
   }
 
   /**
-   * Java-friendly version of [[RandomRDDs#exponentialVectorRDD]].
+   * Java-friendly version of `RandomRDDs.exponentialVectorRDD`.
    */
   @Since("1.3.0")
   def exponentialJavaVectorRDD(
@@ -735,7 +735,7 @@ object RandomRDDs {
   }
 
   /**
-   * [[RandomRDDs#exponentialJavaVectorRDD]] with the default seed.
+   * `RandomRDDs.exponentialJavaVectorRDD` with the default seed.
    */
   @Since("1.3.0")
   def exponentialJavaVectorRDD(
@@ -748,7 +748,7 @@ object RandomRDDs {
   }
 
   /**
-   * [[RandomRDDs#exponentialJavaVectorRDD]] with the default number of 
partitions
+   * `RandomRDDs.exponentialJavaVectorRDD` with the default number of 
partitions
    * and the default seed.
    */
   @Since("1.3.0")
@@ -788,7 +788,7 @@ object RandomRDDs {
   }
 
   /**
-   * Java-friendly version of [[RandomRDDs#gammaVectorRDD]].
+   * Java-friendly version of `RandomRDDs.gammaVectorRDD`.
    */
   @Since("1.3.0")
   def gammaJavaVectorRDD(
@@ -803,7 +803,7 @@ object RandomRDDs {
   }
 
   /**
-   * [[RandomRDDs#gammaJavaVectorRDD]] with the default seed.
+   * `RandomRDDs.gammaJavaVectorRDD` with the default seed.
    */
   @Since("1.3.0")
   def gammaJavaVectorRDD(
@@ -817,7 +817,7 @@ object RandomRDDs {
   }
 
   /**
-   * [[RandomRDDs#gammaJavaVectorRDD]] with the default number of partitions 
and the default seed.
+   * `RandomRDDs.gammaJavaVectorRDD` with the default number of partitions and 
the default seed.
    */
   @Since("1.3.0")
   def gammaJavaVectorRDD(
@@ -857,7 +857,7 @@ object RandomRDDs {
 
   /**
    * :: DeveloperApi ::
-   * Java-friendly version of [[RandomRDDs#randomVectorRDD]].
+   * Java-friendly version of `RandomRDDs.randomVectorRDD`.
    */
   @DeveloperApi
   @Since("1.6.0")
@@ -873,7 +873,7 @@ object RandomRDDs {
 
   /**
    * :: DeveloperApi ::
-   * [[RandomRDDs#randomJavaVectorRDD]] with the default seed.
+   * `RandomRDDs.randomJavaVectorRDD` with the default seed.
    */
   @DeveloperApi
   @Since("1.6.0")
@@ -888,7 +888,7 @@ object RandomRDDs {
 
   /**
    * :: DeveloperApi ::
-   * [[RandomRDDs#randomJavaVectorRDD]] with the default number of partitions 
and the default seed.
+   * `RandomRDDs.randomJavaVectorRDD` with the default number of partitions 
and the default seed.
    */
   @DeveloperApi
   @Since("1.6.0")

http://git-wip-us.apache.org/repos/asf/spark/blob/6c00c069/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala
----------------------------------------------------------------------
diff --git 
a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala 
b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala
index 499c807..e5aece7 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala
@@ -83,8 +83,8 @@ object DecisionTree extends Serializable with Logging {
    *                 categorical), depth of the tree, quantile calculation 
strategy, etc.
    * @return DecisionTreeModel that can be used for prediction.
    *
-   * @note Using [[org.apache.spark.mllib.tree.DecisionTree$#trainClassifier]]
-   * and [[org.apache.spark.mllib.tree.DecisionTree$#trainRegressor]]
+   * @note Using `org.apache.spark.mllib.tree.DecisionTree.trainClassifier`
+   * and `org.apache.spark.mllib.tree.DecisionTree.trainRegressor`
    * is recommended to clearly separate classification and regression.
    */
   @Since("1.0.0")
@@ -105,8 +105,8 @@ object DecisionTree extends Serializable with Logging {
    *                 1 internal node + 2 leaf nodes).
    * @return DecisionTreeModel that can be used for prediction.
    *
-   * @note Using [[org.apache.spark.mllib.tree.DecisionTree$#trainClassifier]]
-   * and [[org.apache.spark.mllib.tree.DecisionTree$#trainRegressor]]
+   * @note Using `org.apache.spark.mllib.tree.DecisionTree.trainClassifier`
+   * and `org.apache.spark.mllib.tree.DecisionTree.trainRegressor`
    * is recommended to clearly separate classification and regression.
    */
   @Since("1.0.0")
@@ -133,8 +133,8 @@ object DecisionTree extends Serializable with Logging {
    * @param numClasses Number of classes for classification. Default value of 
2.
    * @return DecisionTreeModel that can be used for prediction.
    *
-   * @note Using [[org.apache.spark.mllib.tree.DecisionTree$#trainClassifier]]
-   * and [[org.apache.spark.mllib.tree.DecisionTree$#trainRegressor]]
+   * @note Using `org.apache.spark.mllib.tree.DecisionTree.trainClassifier`
+   * and `org.apache.spark.mllib.tree.DecisionTree.trainRegressor`
    * is recommended to clearly separate classification and regression.
    */
   @Since("1.2.0")
@@ -167,8 +167,8 @@ object DecisionTree extends Serializable with Logging {
    *                                indexed from 0: {0, 1, ..., k-1}.
    * @return DecisionTreeModel that can be used for prediction.
    *
-   * @note Using [[org.apache.spark.mllib.tree.DecisionTree$#trainClassifier]]
-   * and [[org.apache.spark.mllib.tree.DecisionTree$#trainRegressor]]
+   * @note Using `org.apache.spark.mllib.tree.DecisionTree.trainClassifier`
+   * and `org.apache.spark.mllib.tree.DecisionTree.trainRegressor`
    * is recommended to clearly separate classification and regression.
    */
   @Since("1.0.0")
@@ -218,7 +218,7 @@ object DecisionTree extends Serializable with Logging {
   }
 
   /**
-   * Java-friendly API for 
[[org.apache.spark.mllib.tree.DecisionTree$#trainClassifier]]
+   * Java-friendly API for 
`org.apache.spark.mllib.tree.DecisionTree.trainClassifier`
    */
   @Since("1.1.0")
   def trainClassifier(
@@ -262,7 +262,7 @@ object DecisionTree extends Serializable with Logging {
   }
 
   /**
-   * Java-friendly API for 
[[org.apache.spark.mllib.tree.DecisionTree$#trainRegressor]]
+   * Java-friendly API for 
`org.apache.spark.mllib.tree.DecisionTree.trainRegressor`
    */
   @Since("1.1.0")
   def trainRegressor(

http://git-wip-us.apache.org/repos/asf/spark/blob/6c00c069/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala
----------------------------------------------------------------------
diff --git 
a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala 
b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala
index 3e85678..df2c1b0 100644
--- 
a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala
+++ 
b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala
@@ -136,7 +136,7 @@ object GradientBoostedTrees extends Logging {
   }
 
   /**
-   * Java-friendly API for 
[[org.apache.spark.mllib.tree.GradientBoostedTrees$#train]]
+   * Java-friendly API for 
`org.apache.spark.mllib.tree.GradientBoostedTrees.train`
    */
   @Since("1.2.0")
   def train(

http://git-wip-us.apache.org/repos/asf/spark/blob/6c00c069/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala
----------------------------------------------------------------------
diff --git 
a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala 
b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala
index 1f6cb08..d1331a5 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala
@@ -172,7 +172,7 @@ object RandomForest extends Serializable with Logging {
   }
 
   /**
-   * Java-friendly API for 
[[org.apache.spark.mllib.tree.RandomForest$#trainClassifier]]
+   * Java-friendly API for 
`org.apache.spark.mllib.tree.RandomForest.trainClassifier`
    */
   @Since("1.2.0")
   def trainClassifier(
@@ -259,7 +259,7 @@ object RandomForest extends Serializable with Logging {
   }
 
   /**
-   * Java-friendly API for 
[[org.apache.spark.mllib.tree.RandomForest$#trainRegressor]]
+   * Java-friendly API for 
`org.apache.spark.mllib.tree.RandomForest.trainRegressor`
    */
   @Since("1.2.0")
   def trainRegressor(

http://git-wip-us.apache.org/repos/asf/spark/blob/6c00c069/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala 
b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala
index 6bb3271..de66c7c 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala
@@ -149,7 +149,7 @@ object MLUtils extends Logging {
    * Save labeled data in LIBSVM format.
    * @param data an RDD of LabeledPoint to be saved
    * @param dir directory to save the data
-   * @see [[org.apache.spark.mllib.util.MLUtils#loadLibSVMFile]]
+   * @see `org.apache.spark.mllib.util.MLUtils.loadLibSVMFile`
    */
   @Since("1.0.0")
   def saveAsLibSVMFile(data: RDD[LabeledPoint], dir: String) {

http://git-wip-us.apache.org/repos/asf/spark/blob/6c00c069/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala
index 2caf723..375df64 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala
@@ -259,7 +259,7 @@ abstract class SQLImplicits extends LowPrioritySQLImplicits 
{
  * Conflicting implicits are placed here to disambiguate resolution.
  *
  * Reasons for including specific implicits:
- * newProductEncoder - to disambiguate for [[List]]s which are both [[Seq]] 
and [[Product]]
+ * newProductEncoder - to disambiguate for `List`s which are both `Seq` and 
`Product`
  */
 trait LowPrioritySQLImplicits {
   /** @since 1.6.0 */

http://git-wip-us.apache.org/repos/asf/spark/blob/6c00c069/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
index 9a080fd..cabe1f4 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
@@ -633,7 +633,7 @@ object functions {
   def skewness(columnName: String): Column = skewness(Column(columnName))
 
   /**
-   * Aggregate function: alias for [[stddev_samp]].
+   * Aggregate function: alias for `stddev_samp`.
    *
    * @group agg_funcs
    * @since 1.6.0
@@ -641,7 +641,7 @@ object functions {
   def stddev(e: Column): Column = withAggregateFunction { StddevSamp(e.expr) }
 
   /**
-   * Aggregate function: alias for [[stddev_samp]].
+   * Aggregate function: alias for `stddev_samp`.
    *
    * @group agg_funcs
    * @since 1.6.0
@@ -717,7 +717,7 @@ object functions {
   def sumDistinct(columnName: String): Column = sumDistinct(Column(columnName))
 
   /**
-   * Aggregate function: alias for [[var_samp]].
+   * Aggregate function: alias for `var_samp`.
    *
    * @group agg_funcs
    * @since 1.6.0
@@ -725,7 +725,7 @@ object functions {
   def variance(e: Column): Column = withAggregateFunction { 
VarianceSamp(e.expr) }
 
   /**
-   * Aggregate function: alias for [[var_samp]].
+   * Aggregate function: alias for `var_samp`.
    *
    * @group agg_funcs
    * @since 1.6.0


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to