You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by sr...@apache.org on 2017/02/01 13:26:26 UTC

spark git commit: [SPARK-19402][DOCS] Support LaTex inline formula correctly and fix warnings in Scala/Java APIs generation

Repository: spark
Updated Branches:
  refs/heads/master 9ac05225e -> f1a1f2607


[SPARK-19402][DOCS] Support LaTex inline formula correctly and fix warnings in Scala/Java APIs generation

## What changes were proposed in this pull request?

This PR proposes three things as below:

- Support LaTex inline-formula, `\( ... \)` in Scala API documentation
  It seems currently,

  ```
  \( ... \)
  ```

  are rendered as they are, for example,

  <img width="345" alt="2017-01-30 10 01 13" src="https://cloud.githubusercontent.com/assets/6477701/22423960/ab37d54a-e737-11e6-9196-4f6229c0189c.png">

  It seems mistakenly more backslashes were added.

- Fix warnings Scaladoc/Javadoc generation
  This PR fixes t two types of warnings as below:

  ```
  [warn] .../spark/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala:335: Could not find any member to link for "UnsupportedOperationException".
  [warn]   /**
  [warn]   ^
  ```

  ```
  [warn] .../spark/sql/core/src/main/scala/org/apache/spark/sql/internal/VariableSubstitution.scala:24: Variable var undefined in comment for class VariableSubstitution in class VariableSubstitution
  [warn]  * `${var}`, `${system:var}` and `${env:var}`.
  [warn]      ^
  ```

- Fix Javadoc8 break
  ```
  [error] .../spark/mllib/target/java/org/apache/spark/ml/PredictionModel.java:7: error: reference not found
  [error]  *                       E.g., {link VectorUDT} for vector features.
  [error]                                       ^
  [error] .../spark/mllib/target/java/org/apache/spark/ml/PredictorParams.java:12: error: reference not found
  [error]    *                          E.g., {link VectorUDT} for vector features.
  [error]                                            ^
  [error] .../spark/mllib/target/java/org/apache/spark/ml/Predictor.java:10: error: reference not found
  [error]  *                       E.g., {link VectorUDT} for vector features.
  [error]                                       ^
  [error] .../spark/sql/hive/target/java/org/apache/spark/sql/hive/HiveAnalysis.java:5: error: reference not found
  [error]  * Note that, this rule must be run after {link PreprocessTableInsertion}.
  [error]                                                  ^
  ```

## How was this patch tested?

Manually via `sbt unidoc` and `jeykil build`.

Author: hyukjinkwon <gu...@gmail.com>

Closes #16741 from HyukjinKwon/warn-and-break.


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

Branch: refs/heads/master
Commit: f1a1f2607d44416a5d2f37ca1746cc7e881895b9
Parents: 9ac0522
Author: hyukjinkwon <gu...@gmail.com>
Authored: Wed Feb 1 13:26:16 2017 +0000
Committer: Sean Owen <so...@cloudera.com>
Committed: Wed Feb 1 13:26:16 2017 +0000

----------------------------------------------------------------------
 .../main/scala/org/apache/spark/SparkConf.scala | 12 +++++------
 .../org/apache/spark/rdd/PairRDDFunctions.scala |  4 ++--
 .../apache/spark/scheduler/SparkListener.scala  |  2 +-
 .../org/apache/spark/scheduler/package.scala    |  4 ++--
 .../apache/spark/storage/TopologyMapper.scala   |  2 +-
 docs/js/api-docs.js                             |  2 +-
 .../org/apache/spark/ml/linalg/Vectors.scala    |  2 +-
 .../scala/org/apache/spark/ml/Predictor.scala   | 16 +++++++-------
 .../apache/spark/ml/attribute/attributes.scala  |  4 ++--
 .../org/apache/spark/ml/attribute/package.scala |  2 +-
 .../spark/ml/classification/Classifier.scala    | 12 +++++------
 .../MultilayerPerceptronClassifier.scala        |  4 ++--
 .../ProbabilisticClassifier.scala               | 10 +++++----
 .../apache/spark/ml/feature/MinMaxScaler.scala  |  2 +-
 .../org/apache/spark/ml/feature/package.scala   |  6 +++---
 .../org/apache/spark/ml/param/params.scala      | 18 +++++++++-------
 .../apache/spark/ml/regression/Regressor.scala  |  2 +-
 .../org/apache/spark/ml/util/ReadWrite.scala    | 16 +++++++-------
 .../org/apache/spark/mllib/clustering/LDA.scala |  6 +++---
 .../spark/mllib/fpm/AssociationRules.scala      |  2 +-
 .../org/apache/spark/mllib/fpm/FPGrowth.scala   |  2 +-
 .../org/apache/spark/mllib/linalg/Vectors.scala |  2 +-
 .../spark/mllib/optimization/Gradient.scala     | 21 ++++++++++---------
 .../spark/mllib/stat/test/StreamingTest.scala   |  2 +-
 .../mllib/tree/model/treeEnsembleModels.scala   |  2 +-
 .../main/scala/org/apache/spark/sql/Row.scala   |  6 +++---
 .../spark/sql/types/AbstractDataType.scala      |  4 ++--
 .../org/apache/spark/sql/types/ArrayType.scala  |  2 +-
 .../org/apache/spark/sql/DataFrameReader.scala  |  6 +++---
 .../spark/sql/DataFrameStatFunctions.scala      |  2 +-
 .../scala/org/apache/spark/sql/Dataset.scala    |  6 +++---
 .../scala/org/apache/spark/sql/SQLContext.scala | 22 ++++++++++----------
 .../org/apache/spark/sql/SparkSession.scala     |  4 ++--
 .../scala/org/apache/spark/sql/functions.scala  | 16 +++++++-------
 .../apache/spark/sql/sources/interfaces.scala   |  2 +-
 .../spark/sql/streaming/DataStreamReader.scala  |  2 +-
 .../spark/sql/streaming/StreamingQuery.scala    |  2 +-
 .../sql/streaming/StreamingQueryListener.scala  |  2 +-
 .../apache/spark/sql/hive/HiveInspectors.scala  |  2 +-
 .../apache/spark/sql/hive/HiveStrategies.scala  |  2 +-
 .../hive/execution/InsertIntoHiveTable.scala    |  4 ++--
 .../org/apache/spark/streaming/State.scala      |  2 +-
 42 files changed, 124 insertions(+), 119 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/core/src/main/scala/org/apache/spark/SparkConf.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index 308a1ed..fe912e6 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -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 java.util.NoSuchElementException
+   * @throws java.util.NoSuchElementException If the time parameter is not set
    */
   def getTimeAsSeconds(key: String): Long = {
     Utils.timeStringAsSeconds(get(key))
@@ -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 java.util.NoSuchElementException
+   * @throws java.util.NoSuchElementException If the time parameter is not set
    */
   def getTimeAsMs(key: String): Long = {
     Utils.timeStringAsMs(get(key))
@@ -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 java.util.NoSuchElementException
+   * @throws java.util.NoSuchElementException If the size parameter is not set
    */
   def getSizeAsBytes(key: String): Long = {
     Utils.byteStringAsBytes(get(key))
@@ -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 java.util.NoSuchElementException
+   * @throws java.util.NoSuchElementException If the size parameter is not set
    */
   def getSizeAsKb(key: String): Long = {
     Utils.byteStringAsKb(get(key))
@@ -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 java.util.NoSuchElementException
+   * @throws java.util.NoSuchElementException If the size parameter is not set
    */
   def getSizeAsMb(key: String): Long = {
     Utils.byteStringAsMb(get(key))
@@ -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 java.util.NoSuchElementException
+   * @throws java.util.NoSuchElementException If the size parameter is not set
    */
   def getSizeAsGb(key: String): Long = {
     Utils.byteStringAsGb(get(key))

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/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 d7bfdba..41093bd 100644
--- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
@@ -496,7 +496,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
    * 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]].
+   * key in memory. If a key has too many values, it can result in an `OutOfMemoryError`.
    */
   def groupByKey(partitioner: Partitioner): RDD[(K, Iterable[V])] = self.withScope {
     // groupByKey shouldn't use map side combine because map side combine does not
@@ -520,7 +520,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
    * 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]].
+   * key in memory. If a key has too many values, it can result in an `OutOfMemoryError`.
    */
   def groupByKey(numPartitions: Int): RDD[(K, Iterable[V])] = self.withScope {
     groupByKey(new HashPartitioner(numPartitions))

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
index 1b12af7..4331add 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
@@ -294,7 +294,7 @@ private[spark] trait SparkListenerInterface {
 
 /**
  * :: DeveloperApi ::
- * A default implementation for [[SparkListenerInterface]] that has no-op implementations for
+ * A default implementation for `SparkListenerInterface` that has no-op implementations for
  * all callbacks.
  *
  * Note that this is an internal interface which might change in different Spark releases.

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/core/src/main/scala/org/apache/spark/scheduler/package.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/package.scala b/core/src/main/scala/org/apache/spark/scheduler/package.scala
index f0dbfc2..4847c41 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/package.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/package.scala
@@ -18,7 +18,7 @@
 package org.apache.spark
 
 /**
- * Spark's scheduling components. This includes the [[org.apache.spark.scheduler.DAGScheduler]] and
- * lower level [[org.apache.spark.scheduler.TaskScheduler]].
+ * Spark's scheduling components. This includes the `org.apache.spark.scheduler.DAGScheduler` and
+ * lower level `org.apache.spark.scheduler.TaskScheduler`.
  */
 package object scheduler

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/core/src/main/scala/org/apache/spark/storage/TopologyMapper.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/storage/TopologyMapper.scala b/core/src/main/scala/org/apache/spark/storage/TopologyMapper.scala
index a0f0fde..a150a8e 100644
--- a/core/src/main/scala/org/apache/spark/storage/TopologyMapper.scala
+++ b/core/src/main/scala/org/apache/spark/storage/TopologyMapper.scala
@@ -60,7 +60,7 @@ class DefaultTopologyMapper(conf: SparkConf) extends TopologyMapper(conf) with L
 
 /**
  * A simple file based topology mapper. This expects topology information provided as a
- * [[java.util.Properties]] file. The name of the file is obtained from SparkConf property
+ * `java.util.Properties` file. The name of the file is obtained from SparkConf property
  * `spark.storage.replication.topologyFile`. To use this topology mapper, set the
  * `spark.storage.replication.topologyMapper` property to
  * [[org.apache.spark.storage.FileBasedTopologyMapper]]

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/docs/js/api-docs.js
----------------------------------------------------------------------
diff --git a/docs/js/api-docs.js b/docs/js/api-docs.js
index 96c63cc..13514e1 100644
--- a/docs/js/api-docs.js
+++ b/docs/js/api-docs.js
@@ -50,7 +50,7 @@ $(document).ready(function() {
     MathJax.Hub.Config({
       displayAlign: "left",
       tex2jax: {
-        inlineMath: [ ["$", "$"], ["\\\\(","\\\\)"] ],
+        inlineMath: [ ["$", "$"], ["\\(","\\)"] ],
         displayMath: [ ["$$","$$"], ["\\[", "\\]"] ],
         processEscapes: true,
         skipTags: ['script', 'noscript', 'style', 'textarea', 'pre', 'a']

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala
----------------------------------------------------------------------
diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala
index 22e4ec6..8e166ba 100644
--- a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala
+++ b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala
@@ -169,7 +169,7 @@ sealed trait Vector extends Serializable {
 /**
  * Factory methods for [[org.apache.spark.ml.linalg.Vector]].
  * We don't use the name `Vector` because Scala imports
- * [[scala.collection.immutable.Vector]] by default.
+ * `scala.collection.immutable.Vector` by default.
  */
 @Since("2.0.0")
 object Vectors {

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala
index 215f9d8..08b0cb9 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala
@@ -40,7 +40,7 @@ private[ml] trait PredictorParams extends Params
    * @param schema input schema
    * @param fitting whether this is in fitting
    * @param featuresDataType  SQL DataType for FeaturesType.
-   *                          E.g., [[VectorUDT]] for vector features.
+   *                          E.g., `VectorUDT` for vector features.
    * @return output schema
    */
   protected def validateAndTransformSchema(
@@ -72,7 +72,7 @@ private[ml] trait PredictorParams extends Params
  * in `fit()`.
  *
  * @tparam FeaturesType  Type of features.
- *                       E.g., [[VectorUDT]] for vector features.
+ *                       E.g., `VectorUDT` for vector features.
  * @tparam Learner  Specialization of this class.  If you subclass this type, use this type
  *                  parameter to specify the concrete type.
  * @tparam M  Specialization of [[PredictionModel]].  If you subclass this type, use this type
@@ -122,7 +122,7 @@ abstract class Predictor[
 
   /**
    * Train a model using the given dataset and parameters.
-   * Developers can implement this instead of [[fit()]] to avoid dealing with schema validation
+   * Developers can implement this instead of `fit()` to avoid dealing with schema validation
    * and copying parameters into the model.
    *
    * @param dataset  Training dataset
@@ -133,7 +133,7 @@ abstract class Predictor[
   /**
    * Returns the SQL DataType corresponding to the FeaturesType type parameter.
    *
-   * This is used by [[validateAndTransformSchema()]].
+   * This is used by `validateAndTransformSchema()`.
    * This workaround is needed since SQL has different APIs for Scala and Java.
    *
    * The default value is VectorUDT, but it may be overridden if FeaturesType is not Vector.
@@ -160,7 +160,7 @@ abstract class Predictor[
  * Abstraction for a model for prediction tasks (regression and classification).
  *
  * @tparam FeaturesType  Type of features.
- *                       E.g., [[VectorUDT]] for vector features.
+ *                       E.g., `VectorUDT` for vector features.
  * @tparam M  Specialization of [[PredictionModel]].  If you subclass this type, use this type
  *            parameter to specify the concrete type for the corresponding model.
  */
@@ -181,7 +181,7 @@ abstract class PredictionModel[FeaturesType, M <: PredictionModel[FeaturesType,
   /**
    * Returns the SQL DataType corresponding to the FeaturesType type parameter.
    *
-   * This is used by [[validateAndTransformSchema()]].
+   * This is used by `validateAndTransformSchema()`.
    * This workaround is needed since SQL has different APIs for Scala and Java.
    *
    * The default value is VectorUDT, but it may be overridden if FeaturesType is not Vector.
@@ -197,7 +197,7 @@ abstract class PredictionModel[FeaturesType, M <: PredictionModel[FeaturesType,
    * the predictions as a new column [[predictionCol]].
    *
    * @param dataset input dataset
-   * @return transformed dataset with [[predictionCol]] of type [[Double]]
+   * @return transformed dataset with [[predictionCol]] of type `Double`
    */
   override def transform(dataset: Dataset[_]): DataFrame = {
     transformSchema(dataset.schema, logging = true)
@@ -219,7 +219,7 @@ abstract class PredictionModel[FeaturesType, M <: PredictionModel[FeaturesType,
 
   /**
    * Predict label for the given features.
-   * This internal method is used to implement [[transform()]] and output [[predictionCol]].
+   * This internal method is used to implement `transform()` and output [[predictionCol]].
    */
   protected def predict(features: FeaturesType): Double
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala b/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala
index 7fbfee7..1cd2b1a 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala
@@ -126,7 +126,7 @@ private[attribute] trait AttributeFactory {
   private[attribute] def fromMetadata(metadata: Metadata): Attribute
 
   /**
-   * Creates an [[Attribute]] from a [[StructField]] instance, optionally preserving name.
+   * Creates an [[Attribute]] from a `StructField` instance, optionally preserving name.
    */
   private[ml] def decodeStructField(field: StructField, preserveName: Boolean): Attribute = {
     require(field.dataType.isInstanceOf[NumericType])
@@ -145,7 +145,7 @@ private[attribute] trait AttributeFactory {
   }
 
   /**
-   * Creates an [[Attribute]] from a [[StructField]] instance.
+   * Creates an [[Attribute]] from a `StructField` instance.
    */
   def fromStructField(field: StructField): Attribute = decodeStructField(field, false)
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/mllib/src/main/scala/org/apache/spark/ml/attribute/package.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/attribute/package.scala b/mllib/src/main/scala/org/apache/spark/ml/attribute/package.scala
index f696405..25ce028 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/attribute/package.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/package.scala
@@ -23,7 +23,7 @@ import org.apache.spark.sql.DataFrame
 /**
  * ==ML attributes==
  *
- * The ML pipeline API uses [[DataFrame]]s as ML datasets.
+ * The ML pipeline API uses `DataFrame`s as ML datasets.
  * Each dataset consists of typed columns, e.g., string, double, vector, etc.
  * However, knowing only the column type may not be sufficient to handle the data properly.
  * For instance, a double column with values 0.0, 1.0, 2.0, ... may represent some label indices,

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala
index 8fd6b70..d8608d8 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala
@@ -71,7 +71,7 @@ abstract class Classifier[
    * and put it in an RDD with strong types.
    *
    * @param dataset  DataFrame with columns for labels ([[org.apache.spark.sql.types.NumericType]])
-   *                 and features ([[Vector]]).
+   *                 and features (`Vector`).
    * @param numClasses  Number of classes label can take.  Labels must be integers in the range
    *                    [0, numClasses).
    * @throws SparkException  if any label is not an integer >= 0
@@ -94,7 +94,7 @@ abstract class Classifier[
    * by finding the maximum label value.
    *
    * Label validation (ensuring all labels are integers >= 0) needs to be handled elsewhere,
-   * such as in [[extractLabeledPoints()]].
+   * such as in `extractLabeledPoints()`.
    *
    * @param dataset  Dataset which contains a column [[labelCol]]
    * @param maxNumClasses  Maximum number of classes allowed when inferred from data.  If numClasses
@@ -150,7 +150,7 @@ abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[Featur
   /**
    * Transforms dataset by reading from [[featuresCol]], and appending new columns as specified by
    * parameters:
-   *  - predicted labels as [[predictionCol]] of type [[Double]]
+   *  - predicted labels as [[predictionCol]] of type `Double`
    *  - raw predictions (confidences) as [[rawPredictionCol]] of type `Vector`.
    *
    * @param dataset input dataset
@@ -192,10 +192,10 @@ abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[Featur
 
   /**
    * Predict label for the given features.
-   * This internal method is used to implement [[transform()]] and output [[predictionCol]].
+   * This internal method is used to implement `transform()` and output [[predictionCol]].
    *
    * This default implementation for classification predicts the index of the maximum value
-   * from [[predictRaw()]].
+   * from `predictRaw()`.
    */
   override protected def predict(features: FeaturesType): Double = {
     raw2prediction(predictRaw(features))
@@ -205,7 +205,7 @@ abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[Featur
    * Raw prediction for each possible label.
    * The meaning of a "raw" prediction may vary between algorithms, but it intuitively gives
    * a measure of confidence in each possible label (where larger = more confident).
-   * This internal method is used to implement [[transform()]] and output [[rawPredictionCol]].
+   * This internal method is used to implement `transform()` and output [[rawPredictionCol]].
    *
    * @return  vector where element i is the raw prediction for label i.
    *          This raw prediction may be any real number, where a larger value indicates greater

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala
index 93cc1e6..95c1337 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala
@@ -225,7 +225,7 @@ class MultilayerPerceptronClassifier @Since("1.5.0") (
 
   /**
    * Train a model using the given dataset and parameters.
-   * Developers can implement this instead of [[fit()]] to avoid dealing with schema validation
+   * Developers can implement this instead of `fit()` to avoid dealing with schema validation
    * and copying parameters into the model.
    *
    * @param dataset Training dataset
@@ -321,7 +321,7 @@ class MultilayerPerceptronClassificationModel private[ml] (
 
   /**
    * Predict label for the given features.
-   * This internal method is used to implement [[transform()]] and output [[predictionCol]].
+   * This internal method is used to implement `transform()` and output [[predictionCol]].
    */
   override protected def predict(features: Vector): Double = {
     LabelConverter.decodeLabel(mlpModel.predict(features))

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala
index e89da6f..ef08134 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala
@@ -93,7 +93,7 @@ abstract class ProbabilisticClassificationModel[
   /**
    * Transforms dataset by reading from [[featuresCol]], and appending new columns as specified by
    * parameters:
-   *  - predicted labels as [[predictionCol]] of type [[Double]]
+   *  - predicted labels as [[predictionCol]] of type `Double`
    *  - raw predictions (confidences) as [[rawPredictionCol]] of type `Vector`
    *  - probability of each class as [[probabilityCol]] of type `Vector`.
    *
@@ -158,13 +158,15 @@ abstract class ProbabilisticClassificationModel[
    * doing the computation in-place.
    * These predictions are also called class conditional probabilities.
    *
-   * This internal method is used to implement [[transform()]] and output [[probabilityCol]].
+   * This internal method is used to implement `transform()` and output [[probabilityCol]].
    *
    * @return Estimated class conditional probabilities (modified input vector)
    */
   protected def raw2probabilityInPlace(rawPrediction: Vector): Vector
 
-  /** Non-in-place version of [[raw2probabilityInPlace()]] */
+  /**
+   * Non-in-place version of `raw2probabilityInPlace()`
+   */
   protected def raw2probability(rawPrediction: Vector): Vector = {
     val probs = rawPrediction.copy
     raw2probabilityInPlace(probs)
@@ -182,7 +184,7 @@ abstract class ProbabilisticClassificationModel[
    * Predict the probability of each class given the features.
    * These predictions are also called class conditional probabilities.
    *
-   * This internal method is used to implement [[transform()]] and output [[probabilityCol]].
+   * This internal method is used to implement `transform()` and output [[probabilityCol]].
    *
    * @return Estimated class conditional probabilities
    */

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala
index 19978c9..f648dec 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala
@@ -84,7 +84,7 @@ private[feature] trait MinMaxScalerParams extends Params with HasInputCol with H
  *    $$
  * </blockquote>
  *
- * For the case $E_{max} == E_{min}$, $Rescaled(e_i) = 0.5 * (max + min)$.
+ * For the case \(E_{max} == E_{min}\), \(Rescaled(e_i) = 0.5 * (max + min)\).
  *
  * @note Since zero values will probably be transformed to non-zero values, output of the
  * transformer will be DenseVector even for sparse input.

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/mllib/src/main/scala/org/apache/spark/ml/feature/package.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/package.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/package.scala
index 5dd648a..d75a6dc 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/package.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/package.scala
@@ -25,13 +25,13 @@ import org.apache.spark.sql.DataFrame
  *
  * The `ml.feature` package provides common feature transformers that help convert raw data or
  * features into more suitable forms for model fitting.
- * Most feature transformers are implemented as [[Transformer]]s, which transform one [[DataFrame]]
+ * Most feature transformers are implemented as [[Transformer]]s, which transform one `DataFrame`
  * into another, e.g., [[HashingTF]].
  * Some feature transformers are implemented as [[Estimator]]s, because the transformation requires
  * some aggregated information of the dataset, e.g., document frequencies in [[IDF]].
- * For those feature transformers, calling [[Estimator!.fit]] is required to obtain the model first,
+ * For those feature transformers, calling `Estimator.fit` is required to obtain the model first,
  * e.g., [[IDFModel]], in order to apply transformation.
- * The transformation is usually done by appending new columns to the input [[DataFrame]], so all
+ * The transformation is usually done by appending new columns to the input `DataFrame`, so all
  * input columns are carried over.
  *
  * We try to make each transformer minimal, so it becomes flexible to assemble feature

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/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 ab0620c..12ad800 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
@@ -446,7 +446,7 @@ class StringArrayParam(parent: Params, name: String, doc: String, isValid: Array
   def this(parent: Params, name: String, doc: String) =
     this(parent, name, doc, ParamValidators.alwaysTrue)
 
-  /** Creates a param pair with a [[java.util.List]] of values (for Java and Python). */
+  /** Creates a param pair with a `java.util.List` of values (for Java and Python). */
   def w(value: java.util.List[String]): ParamPair[Array[String]] = w(value.asScala.toArray)
 
   override def jsonEncode(value: Array[String]): String = {
@@ -471,7 +471,7 @@ class DoubleArrayParam(parent: Params, name: String, doc: String, isValid: Array
   def this(parent: Params, name: String, doc: String) =
     this(parent, name, doc, ParamValidators.alwaysTrue)
 
-  /** Creates a param pair with a [[java.util.List]] of values (for Java and Python). */
+  /** Creates a param pair with a `java.util.List` of values (for Java and Python). */
   def w(value: java.util.List[java.lang.Double]): ParamPair[Array[Double]] =
     w(value.asScala.map(_.asInstanceOf[Double]).toArray)
 
@@ -501,7 +501,7 @@ class IntArrayParam(parent: Params, name: String, doc: String, isValid: Array[In
   def this(parent: Params, name: String, doc: String) =
     this(parent, name, doc, ParamValidators.alwaysTrue)
 
-  /** Creates a param pair with a [[java.util.List]] of values (for Java and Python). */
+  /** Creates a param pair with a `java.util.List` of values (for Java and Python). */
   def w(value: java.util.List[java.lang.Integer]): ParamPair[Array[Int]] =
     w(value.asScala.map(_.asInstanceOf[Int]).toArray)
 
@@ -652,7 +652,9 @@ trait Params extends Identifiable with Serializable {
       throw new NoSuchElementException(s"Failed to find a default value for ${param.name}"))
   }
 
-  /** An alias for [[getOrDefault()]]. */
+  /**
+   * An alias for `getOrDefault()`.
+   */
   protected final def $[T](param: Param[T]): T = getOrDefault(param)
 
   /**
@@ -749,14 +751,14 @@ trait Params extends Identifiable with Serializable {
    * Copies param values from this instance to another instance for params shared by them.
    *
    * This handles default Params and explicitly set Params separately.
-   * Default Params are copied from and to [[defaultParamMap]], and explicitly set Params are
-   * copied from and to [[paramMap]].
+   * Default Params are copied from and to `defaultParamMap`, and explicitly set Params are
+   * copied from and to `paramMap`.
    * Warning: This implicitly assumes that this [[Params]] instance and the target instance
    *          share the same set of default Params.
    *
    * @param to the target instance, which should work with the same set of default Params as this
    *           source instance
-   * @param extra extra params to be copied to the target's [[paramMap]]
+   * @param extra extra params to be copied to the target's `paramMap`
    * @return the target instance with param values copied
    */
   protected def copyValues[T <: Params](to: T, extra: ParamMap = ParamMap.empty): T = {
@@ -822,7 +824,7 @@ final class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any])
     this
   }
 
-  /** Put param pairs with a [[java.util.List]] of values for Python. */
+  /** Put param pairs with a `java.util.List` of values for Python. */
   private[ml] def put(paramPairs: JList[ParamPair[_]]): this.type = {
     put(paramPairs.asScala: _*)
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala
index be35657..c0a1683 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala
@@ -40,7 +40,7 @@ private[spark] abstract class Regressor[
 /**
  * :: DeveloperApi ::
  *
- * Model produced by a [[Regressor]].
+ * Model produced by a `Regressor`.
  *
  * @tparam FeaturesType  Type of input features.  E.g., [[org.apache.spark.mllib.linalg.Vector]]
  * @tparam M  Concrete Model type.

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala b/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala
index c0e3801..09bddcd 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala
@@ -37,7 +37,7 @@ import org.apache.spark.sql.{SparkSession, SQLContext}
 import org.apache.spark.util.Utils
 
 /**
- * Trait for [[MLWriter]] and [[MLReader]].
+ * Trait for `MLWriter` and `MLReader`.
  */
 private[util] sealed trait BaseReadWrite {
   private var optionSparkSession: Option[SparkSession] = None
@@ -112,7 +112,7 @@ abstract class MLWriter extends BaseReadWrite with Logging {
   }
 
   /**
-   * [[save()]] handles overwriting and then calls this method.  Subclasses should override this
+   * `save()` handles overwriting and then calls this method.  Subclasses should override this
    * method to implement the actual saving of the instance.
    */
   @Since("1.6.0")
@@ -135,13 +135,13 @@ abstract class MLWriter extends BaseReadWrite with Logging {
 }
 
 /**
- * Trait for classes that provide [[MLWriter]].
+ * Trait for classes that provide `MLWriter`.
  */
 @Since("1.6.0")
 trait MLWritable {
 
   /**
-   * Returns an [[MLWriter]] instance for this ML instance.
+   * Returns an `MLWriter` instance for this ML instance.
    */
   @Since("1.6.0")
   def write: MLWriter
@@ -193,7 +193,7 @@ abstract class MLReader[T] extends BaseReadWrite {
 }
 
 /**
- * Trait for objects that provide [[MLReader]].
+ * Trait for objects that provide `MLReader`.
  *
  * @tparam T ML instance type
  */
@@ -201,7 +201,7 @@ abstract class MLReader[T] extends BaseReadWrite {
 trait MLReadable[T] {
 
   /**
-   * Returns an [[MLReader]] instance for this class.
+   * Returns an `MLReader` instance for this class.
    */
   @Since("1.6.0")
   def read: MLReader[T]
@@ -235,7 +235,7 @@ trait DefaultParamsReadable[T] extends MLReadable[T] {
 }
 
 /**
- * Default [[MLWriter]] implementation for transformers and estimators that contain basic
+ * Default `MLWriter` implementation for transformers and estimators that contain basic
  * (json4s-serializable) params and no data. This will not handle more complex params or types with
  * data (e.g., models with coefficients).
  *
@@ -309,7 +309,7 @@ private[ml] object DefaultParamsWriter {
 }
 
 /**
- * Default [[MLReader]] implementation for transformers and estimators that contain basic
+ * Default `MLReader` implementation for transformers and estimators that contain basic
  * (json4s-serializable) params and no data. This will not handle more complex params or types with
  * data (e.g., models with coefficients).
  *

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala
index 4cb9200..6c5f529 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala
@@ -91,7 +91,7 @@ class LDA private (
    * distributions over topics ("theta").
    *
    * This method assumes the Dirichlet distribution is symmetric and can be described by a single
-   * [[Double]] parameter. It should fail if docConcentration is asymmetric.
+   * `Double` parameter. It should fail if docConcentration is asymmetric.
    */
   @Since("1.3.0")
   def getDocConcentration: Double = {
@@ -113,7 +113,7 @@ class LDA private (
    *
    * If set to a singleton vector Vector(-1), then docConcentration is set automatically. If set to
    * singleton vector Vector(t) where t != -1, then t is replicated to a vector of length k during
-   * `LDAOptimizer.initialize()`. Otherwise, the [[docConcentration]] vector must be length k.
+   * `LDAOptimizer.initialize()`. Otherwise, the `docConcentration` vector must be length k.
    * (default = Vector(-1) = automatic)
    *
    * Optimizer-specific parameter settings:
@@ -137,7 +137,7 @@ class LDA private (
   }
 
   /**
-   * Replicates a [[Double]] docConcentration to create a symmetric prior.
+   * Replicates a `Double` docConcentration to create a symmetric prior.
    */
   @Since("1.3.0")
   def setDocConcentration(docConcentration: Double): this.type = {

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/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 85a90fa..acb83ac 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
@@ -54,7 +54,7 @@ class AssociationRules private[fpm] (
   }
 
   /**
-   * Computes the association rules with confidence above [[minConfidence]].
+   * Computes the association rules with confidence above `minConfidence`.
    * @param freqItemsets frequent itemset model obtained from [[FPGrowth]]
    * @return a `Set[Rule[Item]]` containing the association rules.
    *

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/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 635da00..f6b1143 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
@@ -52,7 +52,7 @@ class FPGrowthModel[Item: ClassTag] @Since("1.3.0") (
     @Since("1.3.0") val freqItemsets: RDD[FreqItemset[Item]])
   extends Saveable with Serializable {
   /**
-   * Generates association rules for the [[Item]]s in [[freqItemsets]].
+   * Generates association rules for the `Item`s in [[freqItemsets]].
    * @param confidence minimal confidence of the rules produced
    */
   @Since("1.5.0")

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala
index 63ea9d3..723addc 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala
@@ -273,7 +273,7 @@ class VectorUDT extends UserDefinedType[Vector] {
 /**
  * Factory methods for [[org.apache.spark.mllib.linalg.Vector]].
  * We don't use the name `Vector` because Scala imports
- * [[scala.collection.immutable.Vector]] by default.
+ * `scala.collection.immutable.Vector` by default.
  */
 @Since("1.0.0")
 object Vectors {

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala
index 0efce3c..88c7324 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala
@@ -78,7 +78,7 @@ abstract class Gradient extends Serializable {
  *
  * for K classes multiclass classification problem.
  *
- * The model weights $w = (w_1, w_2, ..., w_{K-1})^T$ becomes a matrix which has dimension of
+ * The model weights \(w = (w_1, w_2, ..., w_{K-1})^T\) becomes a matrix which has dimension of
  * (K-1) * (N+1) if the intercepts are added. If the intercepts are not added, the dimension
  * will be (K-1) * N.
  *
@@ -93,9 +93,9 @@ abstract class Gradient extends Serializable {
  *    $$
  * </blockquote>
  *
- * where $\alpha(i) = 1$ if $i \ne 0$, and
- *       $\alpha(i) = 0$ if $i == 0$,
- *       $margins_i = x w_i$.
+ * where $\alpha(i) = 1$ if \(i \ne 0\), and
+ *       $\alpha(i) = 0$ if \(i == 0\),
+ *       \(margins_i = x w_i\).
  *
  * For optimization, we have to calculate the first derivative of the loss function, and
  * a simple calculation shows that
@@ -110,18 +110,19 @@ abstract class Gradient extends Serializable {
  *    $$
  * </blockquote>
  *
- * where $\delta_{i, j} = 1$ if $i == j$,
- *       $\delta_{i, j} = 0$ if $i != j$, and
+ * where $\delta_{i, j} = 1$ if \(i == j\),
+ *       $\delta_{i, j} = 0$ if \(i != j\), and
  *       multiplier =
  *         $\exp(margins_i) / (1 + \sum_k^{K-1} \exp(margins_i)) - (1-\alpha(y)\delta_{y, i+1})$
  *
  * If any of margins is larger than 709.78, the numerical computation of multiplier and loss
  * function will be suffered from arithmetic overflow. This issue occurs when there are outliers
  * in data which are far away from hyperplane, and this will cause the failing of training once
- * infinity / infinity is introduced. Note that this is only a concern when max(margins) &gt; 0.
+ * infinity / infinity is introduced. Note that this is only a concern when max(margins)
+ * {@literal >} 0.
  *
- * Fortunately, when max(margins) = maxMargin &gt; 0, the loss function and the multiplier can be
- * easily rewritten into the following equivalent numerically stable formula.
+ * Fortunately, when max(margins) = maxMargin {@literal >} 0, the loss function and the multiplier
+ * can be easily rewritten into the following equivalent numerically stable formula.
  *
  * <blockquote>
  *    $$
@@ -133,7 +134,7 @@ abstract class Gradient extends Serializable {
  *    \end{align}
  *    $$
  * </blockquote>
-
+ *
  * where sum = $\exp(-maxMargin) + \sum_i^{K-1}\exp(margins_i - maxMargin) - 1$.
  *
  * Note that each term, $(margins_i - maxMargin)$ in $\exp$ is smaller than zero; as a result,

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTest.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTest.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTest.scala
index d680237..551ea35 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTest.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTest.scala
@@ -54,7 +54,7 @@ case class BinarySample @Since("1.6.0") (
  * cumulative processing, using all batches seen so far.
  *
  * Different tests may be used for assessing statistical significance depending on assumptions
- * satisfied by data. For more details, see [[StreamingTestMethod]]. The `testMethod` specifies
+ * satisfied by data. For more details, see `StreamingTestMethod`. The `testMethod` specifies
  * which test will be used.
  *
  * Use a builder pattern to construct a streaming test in an application, for example:

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala
index 2999507..fc1d412 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala
@@ -341,7 +341,7 @@ private[tree] sealed class TreeEnsembleModel(
   def predict(features: RDD[Vector]): RDD[Double] = features.map(x => predict(x))
 
   /**
-   * Java-friendly version of [[org.apache.spark.mllib.tree.model.TreeEnsembleModel#predict]].
+   * Java-friendly version of `org.apache.spark.mllib.tree.model.TreeEnsembleModel.predict`.
    */
   def predict(features: JavaRDD[Vector]): JavaRDD[java.lang.Double] = {
     predict(features.rdd).toJavaRDD().asInstanceOf[JavaRDD[java.lang.Double]]

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala
index c362104..180c2d1 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala
@@ -48,7 +48,7 @@ object Row {
   def apply(values: Any*): Row = new GenericRow(values.toArray)
 
   /**
-   * This method can be used to construct a [[Row]] from a [[Seq]] of values.
+   * This method can be used to construct a [[Row]] from a `Seq` of values.
    */
   def fromSeq(values: Seq[Any]): Row = new GenericRow(values.toArray)
 
@@ -283,7 +283,7 @@ trait Row extends Serializable {
   def getSeq[T](i: Int): Seq[T] = getAs[Seq[T]](i)
 
   /**
-   * Returns the value at position i of array type as [[java.util.List]].
+   * Returns the value at position i of array type as `java.util.List`.
    *
    * @throws ClassCastException when data type does not match.
    */
@@ -298,7 +298,7 @@ trait Row extends Serializable {
   def getMap[K, V](i: Int): scala.collection.Map[K, V] = getAs[Map[K, V]](i)
 
   /**
-   * Returns the value at position i of array type as a [[java.util.Map]].
+   * Returns the value at position i of array type as a `java.util.Map`.
    *
    * @throws ClassCastException when data type does not match.
    */

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala
index da5775b..1d54ff5 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala
@@ -80,7 +80,7 @@ private[sql] object TypeCollection {
 
   /**
    * Types that can be ordered/compared. In the long run we should probably make this a trait
-   * that can be mixed into each data type, and perhaps create an [[AbstractDataType]].
+   * that can be mixed into each data type, and perhaps create an `AbstractDataType`.
    */
   // TODO: Should we consolidate this with RowOrdering.isOrderable?
   val Ordered = TypeCollection(
@@ -106,7 +106,7 @@ private[sql] object TypeCollection {
 
 
 /**
- * An [[AbstractDataType]] that matches any concrete data types.
+ * An `AbstractDataType` that matches any concrete data types.
  */
 protected[sql] object AnyDataType extends AbstractDataType {
 

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala
index 98efba1..38c4048 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala
@@ -49,7 +49,7 @@ object ArrayType extends AbstractDataType {
  * The data type for collections of multiple values.
  * Internally these are represented as columns that contain a ``scala.collection.Seq``.
  *
- * Please use [[DataTypes.createArrayType()]] to create a specific instance.
+ * Please use `DataTypes.createArrayType()` to create a specific instance.
  *
  * An [[ArrayType]] object comprises two fields, `elementType: [[DataType]]` and
  * `containsNull: Boolean`. The field of `elementType` is used to specify the type of

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
index fe34d59..a787d5a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
@@ -310,7 +310,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging {
    * Lines text format or newline-delimited JSON</a>) and returns the result as
    * a `DataFrame`.
    *
-   * Unless the schema is specified using [[schema]] function, this function goes through the
+   * Unless the schema is specified using `schema` function, this function goes through the
    * input once to determine the input schema.
    *
    * @param jsonRDD input RDD with one JSON object per record
@@ -322,7 +322,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging {
    * Loads an `RDD[String]` storing JSON objects (<a href="http://jsonlines.org/">JSON Lines
    * text format or newline-delimited JSON</a>) and returns the result as a `DataFrame`.
    *
-   * Unless the schema is specified using [[schema]] function, this function goes through the
+   * Unless the schema is specified using `schema` function, this function goes through the
    * input once to determine the input schema.
    *
    * @param jsonRDD input RDD with one JSON object per record
@@ -365,7 +365,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging {
    *
    * This function will go through the input once to determine the input schema if `inferSchema`
    * is enabled. To avoid going through the entire data once, disable `inferSchema` option or
-   * specify the schema explicitly using [[schema]].
+   * specify the schema explicitly using `schema`.
    *
    * You can set the following CSV-specific options to deal with CSV files:
    * <ul>

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala
index 89c3a74..7294532 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala
@@ -152,7 +152,7 @@ final class DataFrameStatFunctions private[sql](df: DataFrame) {
    * The number of distinct values for each column should be less than 1e4. At most 1e6 non-zero
    * pair frequencies will be returned.
    * The first column of each row will be the distinct values of `col1` and the column names will
-   * be the distinct values of `col2`. The name of the first column will be `$col1_$col2`. Counts
+   * be the distinct values of `col2`. The name of the first column will be `col1_col2`. Counts
    * will be returned as `Long`s. Pairs that have no occurrences will have zero as their counts.
    * Null elements will be replaced by "null", and back ticks will be dropped from elements if they
    * exist.

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
index 391c34f..ce6e8be 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
@@ -2520,7 +2520,7 @@ class Dataset[T] private[sql](
   def unpersist(): this.type = unpersist(blocking = false)
 
   /**
-   * Represents the content of the Dataset as an `RDD` of [[T]].
+   * Represents the content of the Dataset as an `RDD` of `T`.
    *
    * @group basic
    * @since 1.6.0
@@ -2534,14 +2534,14 @@ class Dataset[T] private[sql](
   }
 
   /**
-   * Returns the content of the Dataset as a `JavaRDD` of [[T]]s.
+   * Returns the content of the Dataset as a `JavaRDD` of `T`s.
    * @group basic
    * @since 1.6.0
    */
   def toJavaRDD: JavaRDD[T] = rdd.toJavaRDD()
 
   /**
-   * Returns the content of the Dataset as a `JavaRDD` of [[T]]s.
+   * Returns the content of the Dataset as a `JavaRDD` of `T`s.
    * @group basic
    * @since 1.6.0
    */

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index 1a7fd68..ea465e2 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -298,7 +298,7 @@ class SQLContext private[sql](val sparkSession: SparkSession)
   }
 
   /**
-   * Convert a [[BaseRelation]] created for external data sources into a `DataFrame`.
+   * Convert a `BaseRelation` created for external data sources into a `DataFrame`.
    *
    * @group dataframes
    * @since 1.3.0
@@ -309,7 +309,7 @@ class SQLContext private[sql](val sparkSession: SparkSession)
 
   /**
    * :: DeveloperApi ::
-   * Creates a `DataFrame` from an [[RDD]] containing [[Row]]s using the given schema.
+   * Creates a `DataFrame` from an `RDD` containing [[Row]]s using the given schema.
    * It is important to make sure that the structure of every [[Row]] of the provided RDD matches
    * the provided schema. Otherwise, there will be runtime exception.
    * Example:
@@ -406,7 +406,7 @@ class SQLContext private[sql](val sparkSession: SparkSession)
 
   /**
    * :: Experimental ::
-   * Creates a [[Dataset]] from a [[java.util.List]] of a given type. This method requires an
+   * Creates a [[Dataset]] from a `java.util.List` of a given type. This method requires an
    * encoder (to convert a JVM object of type `T` to and from the internal Spark SQL representation)
    * that is generally created automatically through implicits from a `SparkSession`, or can be
    * created explicitly by calling static methods on [[Encoders]].
@@ -438,7 +438,7 @@ class SQLContext private[sql](val sparkSession: SparkSession)
 
   /**
    * :: DeveloperApi ::
-   * Creates a `DataFrame` from a [[JavaRDD]] containing [[Row]]s using the given schema.
+   * Creates a `DataFrame` from a `JavaRDD` containing [[Row]]s using the given schema.
    * It is important to make sure that the structure of every [[Row]] of the provided RDD matches
    * the provided schema. Otherwise, there will be runtime exception.
    *
@@ -453,7 +453,7 @@ class SQLContext private[sql](val sparkSession: SparkSession)
 
   /**
    * :: DeveloperApi ::
-   * Creates a `DataFrame` from a [[java.util.List]] containing [[Row]]s using the given schema.
+   * Creates a `DataFrame` from a `java.util.List` containing [[Row]]s using the given schema.
    * It is important to make sure that the structure of every [[Row]] of the provided List matches
    * the provided schema. Otherwise, there will be runtime exception.
    *
@@ -518,7 +518,7 @@ class SQLContext private[sql](val sparkSession: SparkSession)
 
   /**
    * :: Experimental ::
-   * Returns a [[DataStreamReader]] that can be used to read streaming data in as a `DataFrame`.
+   * Returns a `DataStreamReader` that can be used to read streaming data in as a `DataFrame`.
    * {{{
    *   sparkSession.readStream.parquet("/path/to/directory/of/parquet/files")
    *   sparkSession.readStream.schema(schema).json("/path/to/directory/of/json/files")
@@ -638,7 +638,7 @@ class SQLContext private[sql](val sparkSession: SparkSession)
 
   /**
    * :: Experimental ::
-   * Creates a `DataFrame` with a single [[LongType]] column named `id`, containing elements
+   * Creates a `DataFrame` with a single `LongType` column named `id`, containing elements
    * in a range from 0 to `end` (exclusive) with step value 1.
    *
    * @since 1.4.1
@@ -650,7 +650,7 @@ class SQLContext private[sql](val sparkSession: SparkSession)
 
   /**
    * :: Experimental ::
-   * Creates a `DataFrame` with a single [[LongType]] column named `id`, containing elements
+   * Creates a `DataFrame` with a single `LongType` column named `id`, containing elements
    * in a range from `start` to `end` (exclusive) with step value 1.
    *
    * @since 1.4.0
@@ -662,7 +662,7 @@ class SQLContext private[sql](val sparkSession: SparkSession)
 
   /**
    * :: Experimental ::
-   * Creates a `DataFrame` with a single [[LongType]] column named `id`, containing elements
+   * Creates a `DataFrame` with a single `LongType` column named `id`, containing elements
    * in a range from `start` to `end` (exclusive) with a step value.
    *
    * @since 2.0.0
@@ -676,7 +676,7 @@ class SQLContext private[sql](val sparkSession: SparkSession)
 
   /**
    * :: Experimental ::
-   * Creates a `DataFrame` with a single [[LongType]] column named `id`, containing elements
+   * Creates a `DataFrame` with a single `LongType` column named `id`, containing elements
    * in an range from `start` to `end` (exclusive) with an step value, with partition number
    * specified.
    *
@@ -733,7 +733,7 @@ class SQLContext private[sql](val sparkSession: SparkSession)
   }
 
   /**
-   * Returns a [[StreamingQueryManager]] that allows managing all the
+   * Returns a `StreamingQueryManager` that allows managing all the
    * [[org.apache.spark.sql.streaming.StreamingQuery StreamingQueries]] active on `this` context.
    *
    * @since 2.0.0

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
index f3dde48..e1fdb2f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
@@ -323,7 +323,7 @@ class SparkSession private(
 
   /**
    * :: DeveloperApi ::
-   * Creates a `DataFrame` from a [[java.util.List]] containing [[Row]]s using the given schema.
+   * Creates a `DataFrame` from a `java.util.List` containing [[Row]]s using the given schema.
    * It is important to make sure that the structure of every [[Row]] of the provided List matches
    * the provided schema. Otherwise, there will be runtime exception.
    *
@@ -448,7 +448,7 @@ class SparkSession private(
 
   /**
    * :: Experimental ::
-   * Creates a [[Dataset]] from a [[java.util.List]] of a given type. This method requires an
+   * Creates a [[Dataset]] from a `java.util.List` of a given type. This method requires an
    * encoder (to convert a JVM object of type `T` to and from the internal Spark SQL representation)
    * that is generally created automatically through implicits from a `SparkSession`, or can be
    * created explicitly by calling static methods on [[Encoders]].

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/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 c86ae5b..5e27484 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
@@ -1200,8 +1200,8 @@ object functions {
    * Creates a new struct column.
    * If the input column is a column in a `DataFrame`, or a derived column expression
    * that is named (i.e. aliased), its name would be remained as the StructField's name,
-   * otherwise, the newly generated StructField's name would be auto generated as col${index + 1},
-   * i.e. col1, col2, col3, ...
+   * otherwise, the newly generated StructField's name would be auto generated as
+   * `col` with a suffix `index + 1`, i.e. col1, col2, col3, ...
    *
    * @group normal_funcs
    * @since 1.4.0
@@ -2482,7 +2482,7 @@ object functions {
    * format given by the second argument.
    *
    * A pattern could be for instance `dd.MM.yyyy` and could return a string like '18.03.1993'. All
-   * pattern letters of [[java.text.SimpleDateFormat]] can be used.
+   * pattern letters of `java.text.SimpleDateFormat` can be used.
    *
    * @note Use when ever possible specialized functions like [[year]]. These benefit from a
    * specialized implementation.
@@ -2732,14 +2732,14 @@ object functions {
    * @param timeColumn The column or the expression to use as the timestamp for windowing by time.
    *                   The time column must be of TimestampType.
    * @param windowDuration A string specifying the width of the window, e.g. `10 minutes`,
-   *                       `1 second`. Check [[org.apache.spark.unsafe.types.CalendarInterval]] for
+   *                       `1 second`. Check `org.apache.spark.unsafe.types.CalendarInterval` for
    *                       valid duration identifiers. Note that the duration is a fixed length of
    *                       time, and does not vary over time according to a calendar. For example,
    *                       `1 day` always means 86,400,000 milliseconds, not a calendar day.
    * @param slideDuration A string specifying the sliding interval of the window, e.g. `1 minute`.
    *                      A new window will be generated every `slideDuration`. Must be less than
    *                      or equal to the `windowDuration`. Check
-   *                      [[org.apache.spark.unsafe.types.CalendarInterval]] for valid duration
+   *                      `org.apache.spark.unsafe.types.CalendarInterval` for valid duration
    *                      identifiers. This duration is likewise absolute, and does not vary
     *                     according to a calendar.
    * @param startTime The offset with respect to 1970-01-01 00:00:00 UTC with which to start
@@ -2790,14 +2790,14 @@ object functions {
    * @param timeColumn The column or the expression to use as the timestamp for windowing by time.
    *                   The time column must be of TimestampType.
    * @param windowDuration A string specifying the width of the window, e.g. `10 minutes`,
-   *                       `1 second`. Check [[org.apache.spark.unsafe.types.CalendarInterval]] for
+   *                       `1 second`. Check `org.apache.spark.unsafe.types.CalendarInterval` for
    *                       valid duration identifiers. Note that the duration is a fixed length of
    *                       time, and does not vary over time according to a calendar. For example,
    *                       `1 day` always means 86,400,000 milliseconds, not a calendar day.
    * @param slideDuration A string specifying the sliding interval of the window, e.g. `1 minute`.
    *                      A new window will be generated every `slideDuration`. Must be less than
    *                      or equal to the `windowDuration`. Check
-   *                      [[org.apache.spark.unsafe.types.CalendarInterval]] for valid duration
+   *                      `org.apache.spark.unsafe.types.CalendarInterval` for valid duration
    *                      identifiers. This duration is likewise absolute, and does not vary
    *                     according to a calendar.
    *
@@ -2837,7 +2837,7 @@ object functions {
    * @param timeColumn The column or the expression to use as the timestamp for windowing by time.
    *                   The time column must be of TimestampType.
    * @param windowDuration A string specifying the width of the window, e.g. `10 minutes`,
-   *                       `1 second`. Check [[org.apache.spark.unsafe.types.CalendarInterval]] for
+   *                       `1 second`. Check `org.apache.spark.unsafe.types.CalendarInterval` for
    *                       valid duration identifiers.
    *
    * @group datetime_funcs

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala
index f288ad6..ff8b15b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala
@@ -305,7 +305,7 @@ trait InsertableRelation {
  * ::Experimental::
  * An interface for experimenting with a more direct connection to the query planner.  Compared to
  * [[PrunedFilteredScan]], this operator receives the raw expressions from the
- * [[org.apache.spark.sql.catalyst.plans.logical.LogicalPlan]].  Unlike the other APIs this
+ * `org.apache.spark.sql.catalyst.plans.logical.LogicalPlan`.  Unlike the other APIs this
  * interface is NOT designed to be binary compatible across releases and thus should only be used
  * for experimentation.
  *

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala
index 6d2cede..b7ffb3c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala
@@ -192,7 +192,7 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo
    *
    * This function will go through the input once to determine the input schema if `inferSchema`
    * is enabled. To avoid going through the entire data once, disable `inferSchema` option or
-   * specify the schema explicitly using [[schema]].
+   * specify the schema explicitly using `schema`.
    *
    * You can set the following CSV-specific options to deal with CSV files:
    * <ul>

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala
index 9c00259..12a1bb1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala
@@ -137,7 +137,7 @@ trait StreamingQuery {
    * Blocks until all available data in the source has been processed and committed to the sink.
    * This method is intended for testing. Note that in the case of continually arriving data, this
    * method may block forever. Additionally, this method is only guaranteed to block until data that
-   * has been synchronously appended data to a [[org.apache.spark.sql.execution.streaming.Source]]
+   * has been synchronously appended data to a `org.apache.spark.sql.execution.streaming.Source`
    * prior to invocation. (i.e. `getOffset` must immediately reflect the addition).
    * @since 2.0.0
    */

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala
index 6b871b1..c376913 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala
@@ -52,7 +52,7 @@ abstract class StreamingQueryListener {
    * @note This method is asynchronous. The status in [[StreamingQuery]] will always be
    *       latest no matter when this method is called. Therefore, the status of [[StreamingQuery]]
    *       may be changed before/when you process the event. E.g., you may find [[StreamingQuery]]
-   *       is terminated when you are processing [[QueryProgressEvent]].
+   *       is terminated when you are processing `QueryProgressEvent`.
    * @since 2.0.0
    */
   def onQueryProgress(event: QueryProgressEvent): Unit

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
index 52aa108..21b729b 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
@@ -772,7 +772,7 @@ private[hive] trait HiveInspectors {
 
   /**
    * Map the catalyst expression to ObjectInspector, however,
-   * if the expression is [[Literal]] or foldable, a constant writable object inspector returns;
+   * if the expression is `Literal` or foldable, a constant writable object inspector returns;
    * Otherwise, we always get the object inspector according to its data type(in catalyst)
    * @param expr Catalyst expression to be mapped
    * @return Hive java objectinspector (recursively).

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
index badccae..39be417 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
@@ -80,7 +80,7 @@ class DetermineHiveSerde(conf: SQLConf) extends Rule[LogicalPlan] {
 /**
  * Replaces generic operations with specific variants that are designed to work with Hive.
  *
- * Note that, this rule must be run after [[PreprocessTableInsertion]].
+ * Note that, this rule must be run after `PreprocessTableInsertion`.
  */
 class HiveAnalysis(session: SparkSession) extends Rule[LogicalPlan] {
   override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
index ce418ae..3e654d8 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
@@ -50,10 +50,10 @@ import org.apache.spark.SparkException
  * blame Reynold for this! He was just moving code around!
  *
  * In the future we should converge the write path for Hive with the normal data source write path,
- * as defined in [[org.apache.spark.sql.execution.datasources.FileFormatWriter]].
+ * as defined in `org.apache.spark.sql.execution.datasources.FileFormatWriter`.
  *
  * @param table the logical plan representing the table. In the future this should be a
- *              [[org.apache.spark.sql.catalyst.catalog.CatalogTable]] once we converge Hive tables
+ *              `org.apache.spark.sql.catalyst.catalog.CatalogTable` once we converge Hive tables
  *              and data source tables.
  * @param partition a map from the partition key to the partition value (optional). If the partition
  *                  value is optional, dynamic partition insert will be performed.

http://git-wip-us.apache.org/repos/asf/spark/blob/f1a1f260/streaming/src/main/scala/org/apache/spark/streaming/State.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/State.scala b/streaming/src/main/scala/org/apache/spark/streaming/State.scala
index 23cf48e..734c6ef 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/State.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/State.scala
@@ -120,7 +120,7 @@ sealed abstract class State[S] {
   def isTimingOut(): Boolean
 
   /**
-   * Get the state as a [[scala.Option]]. It will be `Some(state)` if it exists, otherwise `None`.
+   * Get the state as a `scala.Option`. It will be `Some(state)` if it exists, otherwise `None`.
    */
   @inline final def getOption(): Option[S] = if (exists) Some(get()) else None
 


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