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 2016/11/29 09:41:46 UTC

[4/4] spark git commit: [SPARK-3359][DOCS] Make javadoc8 working for unidoc/genjavadoc compatibility in Java API documentation

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

## What changes were proposed in this pull request?

This PR make `sbt unidoc` complete with Java 8.

This PR roughly includes several fixes as below:

- Fix unrecognisable class and method links in javadoc by changing it from `[[..]]` to `` `...` ``

  ```diff
  - * A column that will be computed based on the data in a [[DataFrame]].
  + * A column that will be computed based on the data in a `DataFrame`.
  ```

- Fix throws annotations so that they are recognisable in javadoc

- Fix URL links to `<a href="http..."></a>`.

  ```diff
  - * [[http://en.wikipedia.org/wiki/Decision_tree_learning Decision tree]] model for regression.
  + * <a href="http://en.wikipedia.org/wiki/Decision_tree_learning">
  + * Decision tree (Wikipedia)</a> model for regression.
  ```

  ```diff
  -   * see http://en.wikipedia.org/wiki/Receiver_operating_characteristic
  +   * see <a href="http://en.wikipedia.org/wiki/Receiver_operating_characteristic">
  +   * Receiver operating characteristic (Wikipedia)</a>
  ```

- Fix < to > to

  - `greater than`/`greater than or equal to` or `less than`/`less than or equal to` where applicable.

  - Wrap it with `{{{...}}}` to print them in javadoc or use `{code ...}` or `{literal ..}`. Please refer https://github.com/apache/spark/pull/16013#discussion_r89665558

- Fix `</p>` complaint

## How was this patch tested?

Manually tested by `jekyll build` with Java 7 and 8

```
java version "1.7.0_80"
Java(TM) SE Runtime Environment (build 1.7.0_80-b15)
Java HotSpot(TM) 64-Bit Server VM (build 24.80-b11, mixed mode)
```

```
java version "1.8.0_45"
Java(TM) SE Runtime Environment (build 1.8.0_45-b14)
Java HotSpot(TM) 64-Bit Server VM (build 25.45-b02, mixed mode)
```

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

Closes #16013 from HyukjinKwon/SPARK-3359-errors-more.


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

Branch: refs/heads/master
Commit: f830bb9170f6b853565d9dd30ca7418b93a54fe3
Parents: 7d5cb3a
Author: hyukjinkwon <gu...@gmail.com>
Authored: Tue Nov 29 09:41:32 2016 +0000
Committer: Sean Owen <so...@cloudera.com>
Committed: Tue Nov 29 09:41:32 2016 +0000

----------------------------------------------------------------------
 .../scala/org/apache/spark/Accumulator.scala    |  2 +-
 .../main/scala/org/apache/spark/SparkConf.scala | 12 ++--
 .../scala/org/apache/spark/SparkContext.scala   | 14 ++---
 .../scala/org/apache/spark/TaskContext.scala    |  4 +-
 .../scala/org/apache/spark/TaskEndReason.scala  |  2 +-
 .../main/scala/org/apache/spark/TestUtils.scala |  2 +-
 .../org/apache/spark/api/java/JavaRDD.scala     |  8 ++-
 .../apache/spark/rdd/DoubleRDDFunctions.scala   |  4 +-
 .../scala/org/apache/spark/rdd/HadoopRDD.scala  |  2 +-
 .../scala/org/apache/spark/rdd/JdbcRDD.scala    | 15 ++++-
 .../org/apache/spark/rdd/NewHadoopRDD.scala     |  2 +-
 .../org/apache/spark/rdd/PairRDDFunctions.scala | 20 +++----
 .../main/scala/org/apache/spark/rdd/RDD.scala   | 24 +++++---
 .../apache/spark/rdd/RDDCheckpointData.scala    |  3 +-
 .../org/apache/spark/rdd/coalesce-public.scala  |  4 +-
 .../spark/rpc/netty/RpcEndpointVerifier.scala   |  4 +-
 .../spark/scheduler/InputFormatInfo.scala       |  2 +-
 .../org/apache/spark/scheduler/ResultTask.scala |  2 +-
 .../apache/spark/scheduler/ShuffleMapTask.scala |  2 +-
 .../scala/org/apache/spark/scheduler/Task.scala |  2 +-
 .../spark/scheduler/TaskDescription.scala       |  2 +-
 .../spark/storage/BlockManagerMessages.scala    |  2 +-
 .../storage/ShuffleBlockFetcherIterator.scala   |  4 +-
 .../scala/org/apache/spark/ui/UIUtils.scala     |  2 +-
 .../scala/org/apache/spark/util/Utils.scala     |  7 ++-
 .../spark/util/random/SamplingUtils.scala       | 18 +++---
 .../util/random/StratifiedSamplingUtils.scala   | 33 +++++++----
 .../flume/FlumePollingInputDStream.scala        |  2 +-
 .../spark/streaming/kafka/KafkaCluster.scala    | 20 +++++--
 .../streaming/kafka/KafkaInputDStream.scala     |  2 +-
 .../spark/streaming/kafka/KafkaUtils.scala      | 18 +++---
 .../spark/streaming/kafka/OffsetRange.scala     |  2 +-
 .../org/apache/spark/graphx/GraphLoader.scala   |  2 +-
 .../spark/graphx/impl/VertexPartitionBase.scala |  2 +-
 .../graphx/impl/VertexPartitionBaseOps.scala    |  2 +-
 .../apache/spark/graphx/lib/TriangleCount.scala |  2 +-
 .../ml/classification/LogisticRegression.scala  | 15 ++---
 .../spark/ml/clustering/BisectingKMeans.scala   |  4 +-
 .../spark/ml/clustering/GaussianMixture.scala   |  2 +-
 .../org/apache/spark/ml/clustering/LDA.scala    | 10 ++--
 .../apache/spark/ml/feature/Bucketizer.scala    |  2 +-
 .../spark/ml/feature/CountVectorizer.scala      |  9 +--
 .../org/apache/spark/ml/feature/HashingTF.scala |  2 +-
 .../org/apache/spark/ml/feature/NGram.scala     |  2 +-
 .../apache/spark/ml/feature/Normalizer.scala    |  2 +-
 .../apache/spark/ml/feature/OneHotEncoder.scala |  4 +-
 .../scala/org/apache/spark/ml/feature/PCA.scala |  4 +-
 .../spark/ml/feature/PolynomialExpansion.scala  |  3 +-
 .../spark/ml/feature/QuantileDiscretizer.scala  |  6 +-
 .../spark/ml/feature/SQLTransformer.scala       |  8 ++-
 .../spark/ml/feature/StopWordsRemover.scala     |  2 +-
 .../apache/spark/ml/feature/StringIndexer.scala |  8 +--
 .../org/apache/spark/ml/feature/Tokenizer.scala |  2 +-
 .../apache/spark/ml/feature/VectorIndexer.scala |  9 +--
 .../apache/spark/ml/feature/VectorSlicer.scala  |  4 +-
 .../apache/spark/ml/feature/package-info.java   |  4 +-
 .../org/apache/spark/ml/param/params.scala      | 45 ++++++++-------
 .../apache/spark/ml/recommendation/ALS.scala    |  6 +-
 .../ml/regression/AFTSurvivalRegression.scala   |  2 +-
 .../ml/regression/DecisionTreeRegressor.scala   |  3 +-
 .../spark/ml/regression/GBTRegressor.scala      |  2 +-
 .../GeneralizedLinearRegression.scala           | 22 +++----
 .../ml/regression/IsotonicRegression.scala      |  4 +-
 .../spark/ml/regression/LinearRegression.scala  | 43 +++++++-------
 .../ml/regression/RandomForestRegressor.scala   |  2 +-
 .../ml/tree/impl/DecisionTreeMetadata.scala     |  2 +-
 .../apache/spark/ml/util/MetadataUtils.scala    |  2 +-
 .../org/apache/spark/ml/util/ReadWrite.scala    |  8 +--
 .../apache/spark/mllib/classification/SVM.scala |  2 +-
 .../mllib/clustering/BisectingKMeans.scala      | 14 ++---
 .../mllib/clustering/GaussianMixture.scala      |  2 +-
 .../org/apache/spark/mllib/clustering/LDA.scala | 24 ++++----
 .../spark/mllib/clustering/LDAModel.scala       |  4 +-
 .../spark/mllib/clustering/LDAOptimizer.scala   |  4 +-
 .../BinaryClassificationMetrics.scala           |  8 ++-
 .../spark/mllib/evaluation/RankingMetrics.scala |  8 +--
 .../BinaryClassificationMetricComputers.scala   |  2 +-
 .../spark/mllib/fpm/AssociationRules.scala      |  4 +-
 .../org/apache/spark/mllib/fpm/FPGrowth.scala   |  6 +-
 .../org/apache/spark/mllib/fpm/PrefixSpan.scala | 10 ++--
 .../mllib/linalg/EigenValueDecomposition.scala  |  2 +-
 .../org/apache/spark/mllib/linalg/Vectors.scala |  4 +-
 .../mllib/optimization/GradientDescent.scala    |  6 +-
 .../apache/spark/mllib/optimization/LBFGS.scala |  3 +-
 .../apache/spark/mllib/optimization/NNLS.scala  |  9 ++-
 .../spark/mllib/optimization/Updater.scala      |  6 +-
 .../apache/spark/mllib/random/RandomRDDs.scala  |  8 +--
 .../org/apache/spark/mllib/rdd/SlidingRDD.scala |  4 +-
 .../mllib/stat/test/KolmogorovSmirnovTest.scala |  3 +-
 .../spark/mllib/stat/test/StreamingTest.scala   |  6 +-
 .../mllib/stat/test/StreamingTestMethod.scala   |  4 +-
 .../apache/spark/mllib/tree/DecisionTree.scala  |  6 +-
 .../spark/mllib/tree/GradientBoostedTrees.scala |  6 +-
 .../apache/spark/mllib/tree/RandomForest.scala  | 19 ++++---
 .../tree/configuration/BoostingStrategy.scala   | 12 ++--
 .../mllib/tree/configuration/Strategy.scala     |  8 +--
 .../apache/spark/mllib/tree/model/Split.scala   |  2 +-
 .../apache/spark/sql/InternalOutputModes.scala  |  2 +-
 .../main/scala/org/apache/spark/sql/Row.scala   |  4 +-
 .../apache/spark/sql/types/DecimalType.scala    |  3 +-
 .../scala/org/apache/spark/sql/Column.scala     | 40 ++++++-------
 .../apache/spark/sql/DataFrameNaFunctions.scala | 36 ++++++------
 .../org/apache/spark/sql/DataFrameReader.scala  | 43 +++++++-------
 .../spark/sql/DataFrameStatFunctions.scala      | 28 ++++-----
 .../org/apache/spark/sql/DataFrameWriter.scala  | 30 +++++-----
 .../scala/org/apache/spark/sql/Dataset.scala    | 44 +++++++-------
 .../org/apache/spark/sql/ForeachWriter.scala    |  3 +-
 .../spark/sql/KeyValueGroupedDataset.scala      |  8 +--
 .../spark/sql/RelationalGroupedDataset.scala    | 30 +++++-----
 .../org/apache/spark/sql/RuntimeConfig.scala    |  5 +-
 .../scala/org/apache/spark/sql/SQLContext.scala |  4 +-
 .../org/apache/spark/sql/SparkSession.scala     | 60 ++++++++++----------
 .../org/apache/spark/sql/UDFRegistration.scala  |  2 +-
 .../scala/org/apache/spark/sql/functions.scala  | 42 ++++++++------
 .../apache/spark/sql/internal/CatalogImpl.scala | 14 ++---
 .../sql/internal/VariableSubstitution.scala     |  2 +-
 .../apache/spark/sql/jdbc/JdbcDialects.scala    |  4 +-
 .../apache/spark/sql/sources/interfaces.scala   | 12 ++--
 .../hive/execution/InsertIntoHiveTable.scala    | 12 +++-
 .../org/apache/spark/sql/hive/hiveUDFs.scala    |  2 +-
 .../spark/sql/hive/hiveWriterContainers.scala   |  2 +-
 .../org/apache/spark/streaming/StateSpec.scala  |  4 +-
 .../streaming/api/java/JavaPairDStream.scala    |  4 +-
 .../api/java/JavaStreamingContext.scala         |  2 +-
 .../dstream/PairDStreamFunctions.scala          |  4 +-
 125 files changed, 611 insertions(+), 524 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/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 9d1f1d5..7bea636 100644
--- a/core/src/main/scala/org/apache/spark/Accumulator.scala
+++ b/core/src/main/scala/org/apache/spark/Accumulator.scala
@@ -26,7 +26,7 @@ package org.apache.spark
  *
  * An accumulator is created from an initial value `v` by calling
  * [[SparkContext#accumulator SparkContext.accumulator]].
- * Tasks running on the cluster can then add to it using the [[Accumulable#+= +=]] operator.
+ * Tasks running on the cluster can then add to it using the `+=` operator.
  * 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/f830bb91/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 04d657c..0c1c68d 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 NoSuchElementException
+   * @throws java.util.NoSuchElementException
    */
   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 NoSuchElementException
+   * @throws java.util.NoSuchElementException
    */
   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 NoSuchElementException
+   * @throws java.util.NoSuchElementException
    */
   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 NoSuchElementException
+   * @throws java.util.NoSuchElementException
    */
   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 NoSuchElementException
+   * @throws java.util.NoSuchElementException
    */
   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 NoSuchElementException
+   * @throws java.util.NoSuchElementException
    */
   def getSizeAsGb(key: String): Long = {
     Utils.byteStringAsGb(get(key))

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/core/src/main/scala/org/apache/spark/SparkContext.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index a159a17..1cb39a4 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -641,7 +641,7 @@ class SparkContext(config: SparkConf) extends Logging {
 
   /**
    * Get a local property set in this thread, or null if it is missing. See
-   * [[org.apache.spark.SparkContext.setLocalProperty]].
+   * `org.apache.spark.SparkContext.setLocalProperty`.
    */
   def getLocalProperty(key: String): String =
     Option(localProperties.get).map(_.getProperty(key)).orNull
@@ -659,7 +659,7 @@ class SparkContext(config: SparkConf) extends Logging {
    * Application programmers can use this method to group all those jobs together and give a
    * group description. Once set, the Spark web UI will associate such jobs with this group.
    *
-   * The application can also use [[org.apache.spark.SparkContext.cancelJobGroup]] to cancel all
+   * The application can also use `org.apache.spark.SparkContext.cancelJobGroup` to cancel all
    * running jobs in this group. For example,
    * {{{
    * // In the main thread:
@@ -1380,7 +1380,7 @@ class SparkContext(config: SparkConf) extends Logging {
   }
 
   /**
-   * Create and register a [[CollectionAccumulator]], which starts with empty list and accumulates
+   * Create and register a `CollectionAccumulator`, which starts with empty list and accumulates
    * inputs by adding them into the list.
    */
   def collectionAccumulator[T]: CollectionAccumulator[T] = {
@@ -1390,7 +1390,7 @@ class SparkContext(config: SparkConf) extends Logging {
   }
 
   /**
-   * Create and register a [[CollectionAccumulator]], which starts with empty list and accumulates
+   * Create and register a `CollectionAccumulator`, which starts with empty list and accumulates
    * inputs by adding them into the list.
    */
   def collectionAccumulator[T](name: String): CollectionAccumulator[T] = {
@@ -2039,7 +2039,7 @@ class SparkContext(config: SparkConf) extends Logging {
   }
 
   /**
-   * Cancel active jobs for the specified group. See [[org.apache.spark.SparkContext.setJobGroup]]
+   * Cancel active jobs for the specified group. See `org.apache.spark.SparkContext.setJobGroup`
    * for more information.
    */
   def cancelJobGroup(groupId: String) {
@@ -2057,7 +2057,7 @@ class SparkContext(config: SparkConf) extends Logging {
    * Cancel a given job if it's scheduled or running.
    *
    * @param jobId the job ID to cancel
-   * @throws InterruptedException if the cancel message cannot be sent
+   * @note Throws `InterruptedException` if the cancel message cannot be sent
    */
   def cancelJob(jobId: Int) {
     dagScheduler.cancelJob(jobId)
@@ -2067,7 +2067,7 @@ class SparkContext(config: SparkConf) extends Logging {
    * Cancel a given stage and all jobs associated with it.
    *
    * @param stageId the stage ID to cancel
-   * @throws InterruptedException if the cancel message cannot be sent
+   * @note Throws `InterruptedException` if the cancel message cannot be sent
    */
   def cancelStage(stageId: Int) {
     dagScheduler.cancelStage(stageId)

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/core/src/main/scala/org/apache/spark/TaskContext.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala
index 27abccf..0fd777e 100644
--- a/core/src/main/scala/org/apache/spark/TaskContext.scala
+++ b/core/src/main/scala/org/apache/spark/TaskContext.scala
@@ -164,7 +164,7 @@ abstract class TaskContext extends Serializable {
 
   /**
    * Get a local property set upstream in the driver, or null if it is missing. See also
-   * [[org.apache.spark.SparkContext.setLocalProperty]].
+   * `org.apache.spark.SparkContext.setLocalProperty`.
    */
   def getLocalProperty(key: String): String
 
@@ -174,7 +174,7 @@ abstract class TaskContext extends Serializable {
   /**
    * ::DeveloperApi::
    * Returns all metrics sources with the given name which are associated with the instance
-   * which runs the task. For more information see [[org.apache.spark.metrics.MetricsSystem!]].
+   * which runs the task. For more information see `org.apache.spark.metrics.MetricsSystem`.
    */
   @DeveloperApi
   def getMetricsSources(sourceName: String): Seq[Source]

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/core/src/main/scala/org/apache/spark/TaskEndReason.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala
index 7ca3c10..7745387 100644
--- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala
+++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala
@@ -65,7 +65,7 @@ sealed trait TaskFailedReason extends TaskEndReason {
 
 /**
  * :: DeveloperApi ::
- * A [[org.apache.spark.scheduler.ShuffleMapTask]] that completed successfully earlier, but we
+ * A `org.apache.spark.scheduler.ShuffleMapTask` that completed successfully earlier, but we
  * lost the executor before the stage completed. This means Spark needs to reschedule the task
  * to be re-executed on a different executor.
  */

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/core/src/main/scala/org/apache/spark/TestUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala
index 871b9d1..2909191 100644
--- a/core/src/main/scala/org/apache/spark/TestUtils.scala
+++ b/core/src/main/scala/org/apache/spark/TestUtils.scala
@@ -186,7 +186,7 @@ private[spark] object TestUtils {
 
 
 /**
- * A [[SparkListener]] that detects whether spills have occurred in Spark jobs.
+ * A `SparkListener` that detects whether spills have occurred in Spark jobs.
  */
 private class SpillListener extends SparkListener {
   private val stageIdToTaskMetrics = new mutable.HashMap[Int, ArrayBuffer[TaskMetrics]]

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala
index a20d264..94e26e6 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala
@@ -103,7 +103,8 @@ class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T])
    * @param withReplacement can elements be sampled multiple times (replaced when sampled out)
    * @param fraction expected size of the sample as a fraction of this RDD's size
    *  without replacement: probability that each element is chosen; fraction must be [0, 1]
-   *  with replacement: expected number of times each element is chosen; fraction must be &gt;= 0
+   *  with replacement: expected number of times each element is chosen; fraction must be greater
+   *  than or equal to 0
    *
    * @note This is NOT guaranteed to provide exactly the fraction of the count
    * of the given `RDD`.
@@ -117,7 +118,8 @@ class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T])
    * @param withReplacement can elements be sampled multiple times (replaced when sampled out)
    * @param fraction expected size of the sample as a fraction of this RDD's size
    *  without replacement: probability that each element is chosen; fraction must be [0, 1]
-   *  with replacement: expected number of times each element is chosen; fraction must be &gt;= 0
+   *  with replacement: expected number of times each element is chosen; fraction must be greater
+   *  than or equal to 0
    * @param seed seed for the random number generator
    *
    * @note This is NOT guaranteed to provide exactly the fraction of the count
@@ -167,7 +169,7 @@ class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T])
    * Return an RDD with the elements from `this` that are not in `other`.
    *
    * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting
-   * RDD will be &lt;= us.
+   * RDD will be less than or equal to us.
    */
   def subtract(other: JavaRDD[T]): JavaRDD[T] = wrapRDD(rdd.subtract(other))
 

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala
index f3ab324..14331df 100644
--- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala
@@ -152,10 +152,10 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable {
 
   /**
    * Compute a histogram using the provided buckets. The buckets are all open
-   * to the right except for the last which is closed
+   * to the right except for the last which is closed.
    *  e.g. for the array
    *  [1, 10, 20, 50] the buckets are [1, 10) [10, 20) [20, 50]
-   *  e.g 1<=x<10 , 10<=x<20, 20<=x<=50
+   *  e.g {@code <=x<10, 10<=x<20, 20<=x<=50}
    *  And on the input of 1 and 50 we would have a histogram of 1, 0, 1
    *
    * @note If your histogram is evenly spaced (e.g. [0, 10, 20, 30]) this can be switched

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
index 86351b8..ae4320d 100644
--- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
@@ -96,7 +96,7 @@ private[spark] class HadoopPartition(rddId: Int, override val index: Int, s: Inp
  * @param minPartitions Minimum number of HadoopRDD partitions (Hadoop Splits) to generate.
  *
  * @note Instantiating this class directly is not recommended, please use
- * [[org.apache.spark.SparkContext.hadoopRDD()]]
+ * `org.apache.spark.SparkContext.hadoopRDD()`
  */
 @DeveloperApi
 class HadoopRDD[K, V](

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala
index 0970b98..aab46b8 100644
--- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala
@@ -41,7 +41,10 @@ private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) e
  *   The RDD takes care of closing the connection.
  * @param sql the text of the query.
  *   The query must contain two ? placeholders for parameters used to partition the results.
- *   E.g. "select title, author from books where ? <= id and id <= ?"
+ *   For example,
+ *   {{{
+ *   select title, author from books where ? <= id and id <= ?
+ *   }}}
  * @param lowerBound the minimum value of the first placeholder
  * @param upperBound the maximum value of the second placeholder
  *   The lower and upper bounds are inclusive.
@@ -151,7 +154,10 @@ object JdbcRDD {
    *   The RDD takes care of closing the connection.
    * @param sql the text of the query.
    *   The query must contain two ? placeholders for parameters used to partition the results.
-   *   E.g. "select title, author from books where ? <= id and id <= ?"
+   *   For example,
+   *   {{{
+   *   select title, author from books where ? <= id and id <= ?
+   *   }}}
    * @param lowerBound the minimum value of the first placeholder
    * @param upperBound the maximum value of the second placeholder
    *   The lower and upper bounds are inclusive.
@@ -191,7 +197,10 @@ object JdbcRDD {
    *   The RDD takes care of closing the connection.
    * @param sql the text of the query.
    *   The query must contain two ? placeholders for parameters used to partition the results.
-   *   E.g. "select title, author from books where ? <= id and id <= ?"
+   *   For example,
+   *   {{{
+   *   select title, author from books where ? <= id and id <= ?
+   *   }}}
    * @param lowerBound the minimum value of the first placeholder
    * @param upperBound the maximum value of the second placeholder
    *   The lower and upper bounds are inclusive.

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
index a5965f5..c783e13 100644
--- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
@@ -63,7 +63,7 @@ private[spark] class NewHadoopPartition(
  * @param valueClass Class of the value associated with the inputFormatClass.
  *
  * @note Instantiating this class directly is not recommended, please use
- * [[org.apache.spark.SparkContext.newAPIHadoopRDD()]]
+ * `org.apache.spark.SparkContext.newAPIHadoopRDD()`
  */
 @DeveloperApi
 class NewHadoopRDD[K, V](

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/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 33e695e..aad99e3 100644
--- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
@@ -399,9 +399,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
    * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available
    * <a href="http://dx.doi.org/10.1145/2452376.2452456">here</a>.
    *
-   * The relative accuracy is approximately `1.054 / sqrt(2^p)`. Setting a nonzero `sp > p`
-   * would trigger sparse representation of registers, which may reduce the memory consumption
-   * and increase accuracy when the cardinality is small.
+   * The relative accuracy is approximately `1.054 / sqrt(2^p)`. Setting a nonzero (`sp` is
+   * greater than `p`) would trigger sparse representation of registers, which may reduce the
+   * memory consumption and increase accuracy when the cardinality is small.
    *
    * @param p The precision value for the normal set.
    *          `p` must be a value between 4 and `sp` if `sp` is not zero (32 max).
@@ -492,8 +492,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
    * each time the resulting RDD is evaluated.
    *
    * @note This operation may be very expensive. If you are grouping in order to perform an
-   * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]]
-   * or [[PairRDDFunctions.reduceByKey]] will provide much better performance.
+   * aggregation (such as a sum or average) over each key, using `PairRDDFunctions.aggregateByKey`
+   * or `PairRDDFunctions.reduceByKey` will provide much better performance.
    *
    * @note As currently implemented, groupByKey must be able to hold all the key-value pairs for any
    * key in memory. If a key has too many values, it can result in an [[OutOfMemoryError]].
@@ -516,8 +516,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
    * each group is not guaranteed, and may even differ each time the resulting RDD is evaluated.
    *
    * @note This operation may be very expensive. If you are grouping in order to perform an
-   * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]]
-   * or [[PairRDDFunctions.reduceByKey]] will provide much better performance.
+   * aggregation (such as a sum or average) over each key, using `PairRDDFunctions.aggregateByKey`
+   * or `PairRDDFunctions.reduceByKey` will provide much better performance.
    *
    * @note As currently implemented, groupByKey must be able to hold all the key-value pairs for any
    * key in memory. If a key has too many values, it can result in an [[OutOfMemoryError]].
@@ -637,8 +637,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
    * evaluated.
    *
    * @note This operation may be very expensive. If you are grouping in order to perform an
-   * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]]
-   * or [[PairRDDFunctions.reduceByKey]] will provide much better performance.
+   * aggregation (such as a sum or average) over each key, using `PairRDDFunctions.aggregateByKey`
+   * or `PairRDDFunctions.reduceByKey` will provide much better performance.
    */
   def groupByKey(): RDD[(K, Iterable[V])] = self.withScope {
     groupByKey(defaultPartitioner(self))
@@ -908,7 +908,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
    * Return an RDD with the pairs from `this` whose keys are not in `other`.
    *
    * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting
-   * RDD will be <= us.
+   * RDD will be less than or equal to us.
    */
   def subtractByKey[W: ClassTag](other: RDD[(K, W)]): RDD[(K, V)] = self.withScope {
     subtractByKey(other, self.partitioner.getOrElse(new HashPartitioner(self.partitions.length)))

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/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 8e67344..f723fcb 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
@@ -469,7 +469,8 @@ abstract class RDD[T: ClassTag](
    * @param withReplacement can elements be sampled multiple times (replaced when sampled out)
    * @param fraction expected size of the sample as a fraction of this RDD's size
    *  without replacement: probability that each element is chosen; fraction must be [0, 1]
-   *  with replacement: expected number of times each element is chosen; fraction must be &gt;= 0
+   *  with replacement: expected number of times each element is chosen; fraction must be greater
+   *  than or equal to 0
    * @param seed seed for the random number generator
    *
    * @note This is NOT guaranteed to provide exactly the fraction of the count
@@ -750,8 +751,10 @@ abstract class RDD[T: ClassTag](
    *                        print line function (like out.println()) as the 2nd parameter.
    *                        An example of pipe the RDD data of groupBy() in a streaming way,
    *                        instead of constructing a huge String to concat all the elements:
-   *                        def printRDDElement(record:(String, Seq[String]), f:String=&gt;Unit) =
-   *                          for (e &lt;- record._2) {f(e)}
+   *                        {{{
+   *                        def printRDDElement(record:(String, Seq[String]), f:String=>Unit) =
+   *                          for (e <- record._2) {f(e)}
+   *                        }}}
    * @param separateWorkingDir Use separate working directories for each task.
    * @param bufferSize Buffer size for the stdin writer for the piped process.
    * @param encoding Char encoding used for interacting (via stdin, stdout and stderr) with
@@ -1184,8 +1187,13 @@ abstract class RDD[T: ClassTag](
    *
    * @note This method should only be used if the resulting map is expected to be small, as
    * the whole thing is loaded into the driver's memory.
-   * To handle very large results, consider using rdd.map(x =&gt; (x, 1L)).reduceByKey(_ + _), which
-   * returns an RDD[T, Long] instead of a map.
+   * To handle very large results, consider using
+   *
+   * {{{
+   * rdd.map(x => (x, 1L)).reduceByKey(_ + _)
+   * }}}
+   *
+   * , which returns an RDD[T, Long] instead of a map.
    */
   def countByValue()(implicit ord: Ordering[T] = null): Map[T, Long] = withScope {
     map(value => (value, null)).countByKey()
@@ -1223,9 +1231,9 @@ abstract class RDD[T: ClassTag](
    * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available
    * <a href="http://dx.doi.org/10.1145/2452376.2452456">here</a>.
    *
-   * The relative accuracy is approximately `1.054 / sqrt(2^p)`. Setting a nonzero `sp &gt; p`
-   * would trigger sparse representation of registers, which may reduce the memory consumption
-   * and increase accuracy when the cardinality is small.
+   * The relative accuracy is approximately `1.054 / sqrt(2^p)`. Setting a nonzero (`sp` is greater
+   * than `p`) would trigger sparse representation of registers, which may reduce the memory
+   * consumption and increase accuracy when the cardinality is small.
    *
    * @param p The precision value for the normal set.
    *          `p` must be a value between 4 and `sp` if `sp` is not zero (32 max).

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala
index 1070bb9..6c552d4 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala
@@ -23,7 +23,8 @@ import org.apache.spark.Partition
 
 /**
  * Enumeration to manage state transitions of an RDD through checkpointing
- * [ Initialized --> checkpointing in progress --> checkpointed ].
+ *
+ * [ Initialized --{@literal >} checkpointing in progress --{@literal >} checkpointed ]
  */
 private[spark] object CheckpointState extends Enumeration {
   type CheckpointState = Value

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/core/src/main/scala/org/apache/spark/rdd/coalesce-public.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/rdd/coalesce-public.scala b/core/src/main/scala/org/apache/spark/rdd/coalesce-public.scala
index d8a80aa..e00bc22 100644
--- a/core/src/main/scala/org/apache/spark/rdd/coalesce-public.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/coalesce-public.scala
@@ -35,14 +35,14 @@ trait PartitionCoalescer {
    * @param maxPartitions the maximum number of partitions to have after coalescing
    * @param parent the parent RDD whose partitions to coalesce
    * @return an array of [[PartitionGroup]]s, where each element is itself an array of
-   * [[Partition]]s and represents a partition after coalescing is performed.
+   * `Partition`s and represents a partition after coalescing is performed.
    */
   def coalesce(maxPartitions: Int, parent: RDD[_]): Array[PartitionGroup]
 }
 
 /**
  * ::DeveloperApi::
- * A group of [[Partition]]s
+ * A group of `Partition`s
  * @param prefLoc preferred location for the partition group
  */
 @DeveloperApi

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/core/src/main/scala/org/apache/spark/rpc/netty/RpcEndpointVerifier.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/RpcEndpointVerifier.scala b/core/src/main/scala/org/apache/spark/rpc/netty/RpcEndpointVerifier.scala
index 99f20da2..430dcc5 100644
--- a/core/src/main/scala/org/apache/spark/rpc/netty/RpcEndpointVerifier.scala
+++ b/core/src/main/scala/org/apache/spark/rpc/netty/RpcEndpointVerifier.scala
@@ -20,7 +20,7 @@ package org.apache.spark.rpc.netty
 import org.apache.spark.rpc.{RpcCallContext, RpcEndpoint, RpcEnv}
 
 /**
- * An [[RpcEndpoint]] for remote [[RpcEnv]]s to query if an [[RpcEndpoint]] exists.
+ * An [[RpcEndpoint]] for remote [[RpcEnv]]s to query if an `RpcEndpoint` exists.
  *
  * This is used when setting up a remote endpoint reference.
  */
@@ -35,6 +35,6 @@ private[netty] class RpcEndpointVerifier(override val rpcEnv: RpcEnv, dispatcher
 private[netty] object RpcEndpointVerifier {
   val NAME = "endpoint-verifier"
 
-  /** A message used to ask the remote [[RpcEndpointVerifier]] if an [[RpcEndpoint]] exists. */
+  /** A message used to ask the remote [[RpcEndpointVerifier]] if an `RpcEndpoint` exists. */
   case class CheckExistence(name: String)
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala
index a6b032c..66ab9a5 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala
@@ -153,7 +153,7 @@ object InputFormatInfo {
 
     a) For each host, count number of splits hosted on that host.
     b) Decrement the currently allocated containers on that host.
-    c) Compute rack info for each host and update rack -> count map based on (b).
+    c) Compute rack info for each host and update rack to count map based on (b).
     d) Allocate nodes based on (c)
     e) On the allocation result, ensure that we don't allocate "too many" jobs on a single node
        (even if data locality on that is very high) : this is to prevent fragility of job if a

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
index 1e7c63a..d19353f 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
@@ -42,7 +42,7 @@ import org.apache.spark.rdd.RDD
  * @param outputId index of the task in this job (a job can launch tasks on only a subset of the
  *                 input RDD's partitions).
  * @param localProperties copy of thread-local properties set by the user on the driver side.
- * @param metrics a [[TaskMetrics]] that is created at driver side and sent to executor side.
+ * @param metrics a `TaskMetrics` that is created at driver side and sent to executor side.
  *
  * The parameters below are optional:
  * @param jobId id of the job this task belongs to

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
index 66d6790..31011de 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
@@ -42,7 +42,7 @@ import org.apache.spark.shuffle.ShuffleWriter
  *                   the type should be (RDD[_], ShuffleDependency[_, _, _]).
  * @param partition partition of the RDD this task is associated with
  * @param locs preferred task execution locations for locality scheduling
- * @param metrics a [[TaskMetrics]] that is created at driver side and sent to executor side.
+ * @param metrics a `TaskMetrics` that is created at driver side and sent to executor side.
  * @param localProperties copy of thread-local properties set by the user on the driver side.
  *
  * The parameters below are optional:

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/core/src/main/scala/org/apache/spark/scheduler/Task.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala
index d39651a..1554200 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala
@@ -46,7 +46,7 @@ import org.apache.spark.util._
  * @param stageId id of the stage this task belongs to
  * @param stageAttemptId attempt id of the stage this task belongs to
  * @param partitionId index of the number in the RDD
- * @param metrics a [[TaskMetrics]] that is created at driver side and sent to executor side.
+ * @param metrics a `TaskMetrics` that is created at driver side and sent to executor side.
  * @param localProperties copy of thread-local properties set by the user on the driver side.
  *
  * The parameters below are optional:

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala
index 1c7c81c..45c742c 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala
@@ -23,7 +23,7 @@ import org.apache.spark.util.SerializableBuffer
 
 /**
  * Description of a task that gets passed onto executors to be executed, usually created by
- * [[TaskSetManager.resourceOffer]].
+ * `TaskSetManager.resourceOffer`.
  */
 private[spark] class TaskDescription(
     val taskId: Long,

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala
index 6bded92..d71acbb 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala
@@ -43,7 +43,7 @@ private[spark] object BlockManagerMessages {
     extends ToBlockManagerSlave
 
   /**
-   * Driver -> Executor message to trigger a thread dump.
+   * Driver to Executor message to trigger a thread dump.
    */
   case object TriggerThreadDump extends ToBlockManagerSlave
 

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
index 4dc2f36..269c12d 100644
--- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
+++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
@@ -247,7 +247,7 @@ final class ShuffleBlockFetcherIterator(
 
   /**
    * Fetch the local blocks while we are fetching remote blocks. This is ok because
-   * [[ManagedBuffer]]'s memory is allocated lazily when we create the input stream, so all we
+   * `ManagedBuffer`'s memory is allocated lazily when we create the input stream, so all we
    * track in-memory are the ManagedBuffer references themselves.
    */
   private[this] def fetchLocalBlocks() {
@@ -423,7 +423,7 @@ object ShuffleBlockFetcherIterator {
    * @param address BlockManager that the block was fetched from.
    * @param size estimated size of the block, used to calculate bytesInFlight.
    *             Note that this is NOT the exact bytes.
-   * @param buf [[ManagedBuffer]] for the content.
+   * @param buf `ManagedBuffer` for the content.
    * @param isNetworkReqDone Is this the last network request for this host in this fetch request.
    */
   private[storage] case class SuccessFetchResult(

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
index dbeb970..d161843 100644
--- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
+++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
@@ -422,7 +422,7 @@ private[spark] object UIUtils extends Logging {
    * the whole string will rendered as a simple escaped text.
    *
    * Note: In terms of security, only anchor tags with root relative links are supported. So any
-   * attempts to embed links outside Spark UI, or other tags like &lt;script&gt; will cause in
+   * attempts to embed links outside Spark UI, or other tags like {@code <script>} will cause in
    * the whole description to be treated as plain text.
    *
    * @param desc        the original job or stage description string, which may contain html tags.

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/core/src/main/scala/org/apache/spark/util/Utils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala
index 5377050..91f5606 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -1674,7 +1674,7 @@ private[spark] object Utils extends Logging {
 
   /**
    * NaN-safe version of `java.lang.Double.compare()` which allows NaN values to be compared
-   * according to semantics where NaN == NaN and NaN &gt; any non-NaN double.
+   * according to semantics where NaN == NaN and NaN is greater than any non-NaN double.
    */
   def nanSafeCompareDoubles(x: Double, y: Double): Int = {
     val xIsNan: Boolean = java.lang.Double.isNaN(x)
@@ -1688,7 +1688,7 @@ private[spark] object Utils extends Logging {
 
   /**
    * NaN-safe version of `java.lang.Float.compare()` which allows NaN values to be compared
-   * according to semantics where NaN == NaN and NaN &gt; any non-NaN float.
+   * according to semantics where NaN == NaN and NaN is greater than any non-NaN float.
    */
   def nanSafeCompareFloats(x: Float, y: Float): Int = {
     val xIsNan: Boolean = java.lang.Float.isNaN(x)
@@ -2354,8 +2354,9 @@ private[spark] object Utils extends Logging {
    * A spark url (`spark://host:port`) is a special URI that its scheme is `spark` and only contains
    * host and port.
    *
-   * @note Throws `SparkException` if sparkUrl is invalid.
+   * @throws org.apache.spark.SparkException if sparkUrl is invalid.
    */
+  @throws(classOf[SparkException])
   def extractHostPortFromSparkUrl(sparkUrl: String): (String, Int) = {
     try {
       val uri = new java.net.URI(sparkUrl)

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala
index f98932a..297524c 100644
--- a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala
+++ b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala
@@ -67,17 +67,19 @@ private[spark] object SamplingUtils {
   }
 
   /**
-   * Returns a sampling rate that guarantees a sample of size >= sampleSizeLowerBound 99.99% of
-   * the time.
+   * Returns a sampling rate that guarantees a sample of size greater than or equal to
+   * sampleSizeLowerBound 99.99% of the time.
    *
    * How the sampling rate is determined:
+   *
    * Let p = num / total, where num is the sample size and total is the total number of
-   * datapoints in the RDD. We're trying to compute q > p such that
+   * datapoints in the RDD. We're trying to compute q {@literal >} p such that
    *   - when sampling with replacement, we're drawing each datapoint with prob_i ~ Pois(q),
-   *     where we want to guarantee Pr[s < num] < 0.0001 for s = sum(prob_i for i from 0 to total),
-   *     i.e. the failure rate of not having a sufficiently large sample < 0.0001.
+   *     where we want to guarantee
+   *     Pr[s {@literal <} num] {@literal <} 0.0001 for s = sum(prob_i for i from 0 to total),
+   *     i.e. the failure rate of not having a sufficiently large sample {@literal <} 0.0001.
    *     Setting q = p + 5 * sqrt(p/total) is sufficient to guarantee 0.9999 success rate for
-   *     num > 12, but we need a slightly larger q (9 empirically determined).
+   *     num {@literal >} 12, but we need a slightly larger q (9 empirically determined).
    *   - when sampling without replacement, we're drawing each datapoint with prob_i
    *     ~ Binomial(total, fraction) and our choice of q guarantees 1-delta, or 0.9999 success
    *     rate, where success rate is defined the same as in sampling with replacement.
@@ -108,14 +110,14 @@ private[spark] object SamplingUtils {
 private[spark] object PoissonBounds {
 
   /**
-   * Returns a lambda such that Pr[X > s] is very small, where X ~ Pois(lambda).
+   * Returns a lambda such that Pr[X {@literal >} s] is very small, where X ~ Pois(lambda).
    */
   def getLowerBound(s: Double): Double = {
     math.max(s - numStd(s) * math.sqrt(s), 1e-15)
   }
 
   /**
-   * Returns a lambda such that Pr[X < s] is very small, where X ~ Pois(lambda).
+   * Returns a lambda such that Pr[X {@literal <} s] is very small, where X ~ Pois(lambda).
    *
    * @param s sample size
    */

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala b/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala
index 6782274..ce46fc8 100644
--- a/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala
+++ b/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala
@@ -35,13 +35,14 @@ import org.apache.spark.rdd.RDD
  * high probability. This is achieved by maintaining a waitlist of size O(log(s)), where s is the
  * desired sample size for each stratum.
  *
- * Like in simple random sampling, we generate a random value for each item from the
- * uniform  distribution [0.0, 1.0]. All items with values <= min(values of items in the waitlist)
- * are accepted into the sample instantly. The threshold for instant accept is designed so that
- * s - numAccepted = O(sqrt(s)), where s is again the desired sample size. Thus, by maintaining a
- * waitlist size = O(sqrt(s)), we will be able to create a sample of the exact size s by adding
- * a portion of the waitlist to the set of items that are instantly accepted. The exact threshold
- * is computed by sorting the values in the waitlist and picking the value at (s - numAccepted).
+ * Like in simple random sampling, we generate a random value for each item from the uniform
+ * distribution [0.0, 1.0]. All items with values less than or equal to min(values of items in the
+ * waitlist) are accepted into the sample instantly. The threshold for instant accept is designed
+ * so that s - numAccepted = O(sqrt(s)), where s is again the desired sample size. Thus, by
+ * maintaining a waitlist size = O(sqrt(s)), we will be able to create a sample of the exact size
+ * s by adding a portion of the waitlist to the set of items that are instantly accepted. The exact
+ * threshold is computed by sorting the values in the waitlist and picking the value at
+ * (s - numAccepted).
  *
  * Note that since we use the same seed for the RNG when computing the thresholds and the actual
  * sample, our computed thresholds are guaranteed to produce the desired sample size.
@@ -160,12 +161,20 @@ private[spark] object StratifiedSamplingUtils extends Logging {
    *
    * To do so, we compute sampleSize = math.ceil(size * samplingRate) for each stratum and compare
    * it to the number of items that were accepted instantly and the number of items in the waitlist
-   * for that stratum. Most of the time, numAccepted <= sampleSize <= (numAccepted + numWaitlisted),
+   * for that stratum.
+   *
+   * Most of the time,
+   * {{{
+   * numAccepted <= sampleSize <= (numAccepted + numWaitlisted)
+   * }}}
    * which means we need to sort the elements in the waitlist by their associated values in order
-   * to find the value T s.t. |{elements in the stratum whose associated values <= T}| = sampleSize.
-   * Note that all elements in the waitlist have values >= bound for instant accept, so a T value
-   * in the waitlist range would allow all elements that were instantly accepted on the first pass
-   * to be included in the sample.
+   * to find the value T s.t.
+   * {{{
+   * |{elements in the stratum whose associated values <= T}| = sampleSize
+   * }}}.
+   * Note that all elements in the waitlist have values greater than or equal to bound for instant
+   * accept, so a T value in the waitlist range would allow all elements that were instantly
+   * accepted on the first pass to be included in the sample.
    */
   def computeThresholdByKey[K](finalResult: Map[K, AcceptanceResult],
       fractions: Map[K, Double]): Map[K, Double] = {

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala
----------------------------------------------------------------------
diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala
index 5456584..d84e289 100644
--- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala
+++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala
@@ -36,7 +36,7 @@ import org.apache.spark.streaming.flume.sink._
 import org.apache.spark.streaming.receiver.Receiver
 
 /**
- * A [[ReceiverInputDStream]] that can be used to read data from several Flume agents running
+ * A `ReceiverInputDStream` that can be used to read data from several Flume agents running
  * [[org.apache.spark.streaming.flume.sink.SparkSink]]s.
  * @param _ssc Streaming context that will execute this input stream
  * @param addresses List of addresses at which SparkSinks are listening

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala
----------------------------------------------------------------------
diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala
index 35acb7b..e0e44d4 100644
--- a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala
+++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala
@@ -231,7 +231,10 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
   // this 0 here indicates api version, in this case the original ZK backed api.
   private def defaultConsumerApiVersion: Short = 0
 
-  /** Requires Kafka >= 0.8.1.1.  Defaults to the original ZooKeeper backed api version. */
+  /**
+   * Requires Kafka 0.8.1.1 or later.
+   * Defaults to the original ZooKeeper backed API version.
+   */
   def getConsumerOffsets(
       groupId: String,
       topicAndPartitions: Set[TopicAndPartition]
@@ -250,7 +253,10 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
     }
   }
 
-  /** Requires Kafka >= 0.8.1.1.  Defaults to the original ZooKeeper backed api version. */
+  /**
+   * Requires Kafka 0.8.1.1 or later.
+   * Defaults to the original ZooKeeper backed API version.
+   */
   def getConsumerOffsetMetadata(
       groupId: String,
       topicAndPartitions: Set[TopicAndPartition]
@@ -287,7 +293,10 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
     Left(errs)
   }
 
-  /** Requires Kafka >= 0.8.1.1.  Defaults to the original ZooKeeper backed api version. */
+  /**
+   * Requires Kafka 0.8.1.1 or later.
+   * Defaults to the original ZooKeeper backed API version.
+   */
   def setConsumerOffsets(
       groupId: String,
       offsets: Map[TopicAndPartition, Long]
@@ -305,7 +314,10 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
     setConsumerOffsetMetadata(groupId, meta, consumerApiVersion)
   }
 
-  /** Requires Kafka >= 0.8.1.1.  Defaults to the original ZooKeeper backed api version. */
+  /**
+   * Requires Kafka 0.8.1.1 or later.
+   * Defaults to the original ZooKeeper backed API version.
+   */
   def setConsumerOffsetMetadata(
       groupId: String,
       metadata: Map[TopicAndPartition, OffsetAndMetadata]

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala
----------------------------------------------------------------------
diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala
index 3713bda..7ff3a98 100644
--- a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala
+++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala
@@ -38,7 +38,7 @@ import org.apache.spark.util.ThreadUtils
  *
  * @param kafkaParams Map of kafka configuration parameters.
  *                    See: http://kafka.apache.org/configuration.html
- * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+ * @param topics Map of (topic_name to numPartitions) to consume. Each partition is consumed
  * in its own thread.
  * @param storageLevel RDD storage level.
  */

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala
----------------------------------------------------------------------
diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala
index 56f0cb0..d5aef81 100644
--- a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala
+++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala
@@ -47,7 +47,7 @@ object KafkaUtils {
    * @param ssc       StreamingContext object
    * @param zkQuorum  Zookeeper quorum (hostname:port,hostname:port,..)
    * @param groupId   The group id for this consumer
-   * @param topics    Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+   * @param topics    Map of (topic_name to numPartitions) to consume. Each partition is consumed
    *                  in its own thread
    * @param storageLevel  Storage level to use for storing the received objects
    *                      (default: StorageLevel.MEMORY_AND_DISK_SER_2)
@@ -72,7 +72,7 @@ object KafkaUtils {
    * @param ssc         StreamingContext object
    * @param kafkaParams Map of kafka configuration parameters,
    *                    see http://kafka.apache.org/08/configuration.html
-   * @param topics      Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+   * @param topics      Map of (topic_name to numPartitions) to consume. Each partition is consumed
    *                    in its own thread.
    * @param storageLevel Storage level to use for storing the received objects
    * @tparam K type of Kafka message key
@@ -97,7 +97,7 @@ object KafkaUtils {
    * @param jssc      JavaStreamingContext object
    * @param zkQuorum  Zookeeper quorum (hostname:port,hostname:port,..)
    * @param groupId   The group id for this consumer
-   * @param topics    Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+   * @param topics    Map of (topic_name to numPartitions) to consume. Each partition is consumed
    *                  in its own thread
    * @return DStream of (Kafka message key, Kafka message value)
    */
@@ -115,7 +115,7 @@ object KafkaUtils {
    * @param jssc      JavaStreamingContext object
    * @param zkQuorum  Zookeeper quorum (hostname:port,hostname:port,..).
    * @param groupId   The group id for this consumer.
-   * @param topics    Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+   * @param topics    Map of (topic_name to numPartitions) to consume. Each partition is consumed
    *                  in its own thread.
    * @param storageLevel RDD storage level.
    * @return DStream of (Kafka message key, Kafka message value)
@@ -140,7 +140,7 @@ object KafkaUtils {
    * @param valueDecoderClass Type of kafka value decoder
    * @param kafkaParams Map of kafka configuration parameters,
    *                    see http://kafka.apache.org/08/configuration.html
-   * @param topics  Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+   * @param topics  Map of (topic_name to numPartitions) to consume. Each partition is consumed
    *                in its own thread
    * @param storageLevel RDD storage level.
    * @tparam K type of Kafka message key
@@ -396,7 +396,7 @@ object KafkaUtils {
    *    You can access the offsets used in each batch from the generated RDDs (see
    *    [[org.apache.spark.streaming.kafka.HasOffsetRanges]]).
    *  - Failure Recovery: To recover from driver failures, you have to enable checkpointing
-   *    in the [[StreamingContext]]. The information on consumed offset can be
+   *    in the `StreamingContext`. The information on consumed offset can be
    *    recovered from the checkpoint. See the programming guide for details (constraints, etc.).
    *  - End-to-end semantics: This stream ensures that every records is effectively received and
    *    transformed exactly once, but gives no guarantees on whether the transformed data are
@@ -448,7 +448,7 @@ object KafkaUtils {
    *    You can access the offsets used in each batch from the generated RDDs (see
    *    [[org.apache.spark.streaming.kafka.HasOffsetRanges]]).
    *  - Failure Recovery: To recover from driver failures, you have to enable checkpointing
-   *    in the [[StreamingContext]]. The information on consumed offset can be
+   *    in the `StreamingContext`. The information on consumed offset can be
    *    recovered from the checkpoint. See the programming guide for details (constraints, etc.).
    *  - End-to-end semantics: This stream ensures that every records is effectively received and
    *    transformed exactly once, but gives no guarantees on whether the transformed data are
@@ -499,7 +499,7 @@ object KafkaUtils {
    *    You can access the offsets used in each batch from the generated RDDs (see
    *    [[org.apache.spark.streaming.kafka.HasOffsetRanges]]).
    *  - Failure Recovery: To recover from driver failures, you have to enable checkpointing
-   *    in the [[StreamingContext]]. The information on consumed offset can be
+   *    in the `StreamingContext`. The information on consumed offset can be
    *    recovered from the checkpoint. See the programming guide for details (constraints, etc.).
    *  - End-to-end semantics: This stream ensures that every records is effectively received and
    *    transformed exactly once, but gives no guarantees on whether the transformed data are
@@ -565,7 +565,7 @@ object KafkaUtils {
    *    You can access the offsets used in each batch from the generated RDDs (see
    *    [[org.apache.spark.streaming.kafka.HasOffsetRanges]]).
    *  - Failure Recovery: To recover from driver failures, you have to enable checkpointing
-   *    in the [[StreamingContext]]. The information on consumed offset can be
+   *    in the `StreamingContext`. The information on consumed offset can be
    *    recovered from the checkpoint. See the programming guide for details (constraints, etc.).
    *  - End-to-end semantics: This stream ensures that every records is effectively received and
    *    transformed exactly once, but gives no guarantees on whether the transformed data are

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala
----------------------------------------------------------------------
diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala
index d9b856e..10d364f 100644
--- a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala
+++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala
@@ -22,7 +22,7 @@ import kafka.common.TopicAndPartition
 /**
  * Represents any object that has a collection of [[OffsetRange]]s. This can be used to access the
  * offset ranges in RDDs generated by the direct Kafka DStream (see
- * [[KafkaUtils.createDirectStream()]]).
+ * `KafkaUtils.createDirectStream()`).
  * {{{
  *   KafkaUtils.createDirectStream(...).foreachRDD { rdd =>
  *      val offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala
index add21f4..f665727 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala
@@ -32,7 +32,7 @@ object GraphLoader extends Logging {
    * id and a target id. Skips lines that begin with `#`.
    *
    * If desired the edges can be automatically oriented in the positive
-   * direction (source Id &lt; target Id) by setting `canonicalOrientation` to
+   * direction (source Id is less than target Id) by setting `canonicalOrientation` to
    * true.
    *
    * @example Loads a file in the following format:

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala
index 8d608c9..8da46db 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala
@@ -57,7 +57,7 @@ private[graphx] object VertexPartitionBase {
  * concrete implementation. [[VertexPartitionBaseOps]] provides a variety of operations for
  * VertexPartitionBase and subclasses that provide implicit evidence of membership in the
  * `VertexPartitionBaseOpsConstructor` typeclass (for example,
- * [[VertexPartition.VertexPartitionOpsConstructor]]).
+ * `VertexPartition.VertexPartitionOpsConstructor`).
  */
 private[graphx] abstract class VertexPartitionBase[@specialized(Long, Int, Double) VD: ClassTag]
   extends Serializable {

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala
index 4359457..a8ed59b 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala
@@ -29,7 +29,7 @@ import org.apache.spark.util.collection.BitSet
 /**
  * A class containing additional operations for subclasses of VertexPartitionBase that provide
  * implicit evidence of membership in the `VertexPartitionBaseOpsConstructor` typeclass (for
- * example, [[VertexPartition.VertexPartitionOpsConstructor]]).
+ * example, `VertexPartition.VertexPartitionOpsConstructor`).
  */
 private[graphx] abstract class VertexPartitionBaseOps
     [VD: ClassTag, Self[X] <: VertexPartitionBase[X]: VertexPartitionBaseOpsConstructor]

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala
index 21b2296..2715137 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala
@@ -36,7 +36,7 @@ import org.apache.spark.graphx._
  * self cycles and canonicalizes the graph to ensure that the following conditions hold:
  * <ul>
  * <li> There are no self edges</li>
- * <li> All edges are oriented src &gt; dst</li>
+ * <li> All edges are oriented (src is greater than dst)</li>
  * <li> There are no duplicate edges</li>
  * </ul>
  * However, the canonicalization procedure is costly as it requires repartitioning the graph.

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala
index 41b84f4..ec58226 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala
@@ -56,8 +56,8 @@ private[classification] trait LogisticRegressionParams extends ProbabilisticClas
   /**
    * Set threshold in binary classification, in range [0, 1].
    *
-   * If the estimated probability of class label 1 is &gt; threshold, then predict 1, else 0.
-   * A high threshold encourages the model to predict 0 more often;
+   * If the estimated probability of class label 1 is greater than threshold, then predict 1,
+   * else 0. A high threshold encourages the model to predict 0 more often;
    * a low threshold encourages the model to predict 1 more often.
    *
    * Note: Calling this with threshold p is equivalent to calling `setThresholds(Array(1-p, p))`.
@@ -123,7 +123,7 @@ private[classification] trait LogisticRegressionParams extends ProbabilisticClas
   /**
    * Set thresholds in multiclass (or binary) classification to adjust the probability of
    * predicting each class. Array must have length equal to the number of classes,
-   * with values &gt; 0, excepting that at most one value may be 0.
+   * with values greater than 0, excepting that at most one value may be 0.
    * The class with largest value p/t is predicted, where p is the original probability of that
    * class and t is the class's threshold.
    *
@@ -210,8 +210,9 @@ class LogisticRegression @Since("1.2.0") (
 
   /**
    * Set the ElasticNet mixing parameter.
-   * For alpha = 0, the penalty is an L2 penalty. For alpha = 1, it is an L1 penalty.
-   * For 0 &lt; alpha &lt; 1, the penalty is a combination of L1 and L2.
+   * For alpha = 0, the penalty is an L2 penalty.
+   * For alpha = 1, it is an L1 penalty.
+   * For alpha in (0,1), the penalty is a combination of L1 and L2.
    * Default is 0.0 which is an L2 penalty.
    *
    * @group setParam
@@ -298,7 +299,7 @@ class LogisticRegression @Since("1.2.0") (
   override def getThresholds: Array[Double] = super.getThresholds
 
   /**
-   * Suggested depth for treeAggregate (&gt;= 2).
+   * Suggested depth for treeAggregate (greater than or equal to 2).
    * If the dimensions of features or the number of partitions are large,
    * this param could be adjusted to a larger size.
    * Default is 2.
@@ -1517,7 +1518,7 @@ private class LogisticAggregator(
     }
 
     /**
-     * When maxMargin &gt; 0, the original formula could cause overflow.
+     * When maxMargin is greater than 0, the original formula could cause overflow.
      * We address this by subtracting maxMargin from all the margins, so it's guaranteed
      * that all of the new margins will be smaller than zero to prevent arithmetic overflow.
      */

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala
index c7a170d..e58df6b 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala
@@ -55,8 +55,8 @@ private[clustering] trait BisectingKMeansParams extends Params
   def getK: Int = $(k)
 
   /**
-   * The minimum number of points (if &gt;= 1.0) or the minimum proportion
-   * of points (if &lt; 1.0) of a divisible cluster (default: 1.0).
+   * The minimum number of points (if greater than or equal to 1.0) or the minimum proportion
+   * of points (if less than 1.0) of a divisible cluster (default: 1.0).
    * @group expertParam
    */
   @Since("2.0.0")

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala
index 7410934..c764c3a 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala
@@ -44,7 +44,7 @@ private[clustering] trait GaussianMixtureParams extends Params with HasMaxIter w
   with HasSeed with HasPredictionCol with HasProbabilityCol with HasTol {
 
   /**
-   * Number of independent Gaussians in the mixture model. Must be &gt; 1. Default: 2.
+   * Number of independent Gaussians in the mixture model. Must be greater than 1. Default: 2.
    * @group param
    */
   @Since("2.0.0")

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala
index 6032ab3..cd403d8 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala
@@ -78,11 +78,11 @@ private[clustering] trait LDAParams extends Params with HasFeaturesCol with HasM
    *  - EM
    *     - Currently only supports symmetric distributions, so all values in the vector should be
    *       the same.
-   *     - Values should be &gt; 1.0
+   *     - Values should be greater than 1.0
    *     - default = uniformly (50 / k) + 1, where 50/k is common in LDA libraries and +1 follows
    *       from Asuncion et al. (2009), who recommend a +1 adjustment for EM.
    *  - Online
-   *     - Values should be &gt;= 0
+   *     - Values should be greater than or equal to 0
    *     - default = uniformly (1.0 / k), following the implementation from
    *       <a href="https://github.com/Blei-Lab/onlineldavb">here</a>.
    *
@@ -120,11 +120,11 @@ private[clustering] trait LDAParams extends Params with HasFeaturesCol with HasM
    *
    * Optimizer-specific parameter settings:
    *  - EM
-   *     - Value should be &gt; 1.0
+   *     - Value should be greater than 1.0
    *     - default = 0.1 + 1, where 0.1 gives a small amount of smoothing and +1 follows
    *       Asuncion et al. (2009), who recommend a +1 adjustment for EM.
    *  - Online
-   *     - Value should be &gt;= 0
+   *     - Value should be greater than or equal to 0
    *     - default = (1.0 / k), following the implementation from
    *       <a href="https://github.com/Blei-Lab/onlineldavb">here</a>.
    *
@@ -247,7 +247,7 @@ private[clustering] trait LDAParams extends Params with HasFeaturesCol with HasM
    *
    * Note that this should be adjusted in synch with `LDA.maxIter`
    * so the entire corpus is used.  Specifically, set both so that
-   * maxIterations * miniBatchFraction &gt;= 1.
+   * maxIterations * miniBatchFraction greater than or equal to 1.
    *
    * Note: This is the same as the `miniBatchFraction` parameter in
    *       [[org.apache.spark.mllib.clustering.OnlineLDAOptimizer]].

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala
index 1143f0f..260159f 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala
@@ -44,7 +44,7 @@ final class Bucketizer @Since("1.4.0") (@Since("1.4.0") override val uid: String
   /**
    * Parameter for mapping continuous features into buckets. With n+1 splits, there are n buckets.
    * A bucket defined by splits x,y holds values in the range [x,y) except the last bucket, which
-   * also includes y. Splits should be of length >= 3 and strictly increasing.
+   * also includes y. Splits should be of length greater than or equal to 3 and strictly increasing.
    * Values at -inf, inf must be explicitly provided to cover all Double values;
    * otherwise, values outside the splits specified will be treated as errors.
    *

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala
index 6299f74..1ebe297 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala
@@ -53,8 +53,9 @@ private[feature] trait CountVectorizerParams extends Params with HasInputCol wit
   /**
    * Specifies the minimum number of different documents a term must appear in to be included
    * in the vocabulary.
-   * If this is an integer >= 1, this specifies the number of documents the term must appear in;
-   * if this is a double in [0,1), then this specifies the fraction of documents.
+   * If this is an integer greater than or equal to 1, this specifies the number of documents
+   * the term must appear in; if this is a double in [0,1), then this specifies the fraction of
+   * documents.
    *
    * Default: 1.0
    * @group param
@@ -78,8 +79,8 @@ private[feature] trait CountVectorizerParams extends Params with HasInputCol wit
   /**
    * Filter to ignore rare words in a document. For each document, terms with
    * frequency/count less than the given threshold are ignored.
-   * If this is an integer >= 1, then this specifies a count (of times the term must appear
-   * in the document);
+   * If this is an integer greater than or equal to 1, then this specifies a count (of times the
+   * term must appear in the document);
    * if this is a double in [0,1), then this specifies a fraction (out of the document's token
    * count).
    *

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala
index a8792a3..db432b6 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala
@@ -52,7 +52,7 @@ class HashingTF @Since("1.4.0") (@Since("1.4.0") override val uid: String)
   def setOutputCol(value: String): this.type = set(outputCol, value)
 
   /**
-   * Number of features.  Should be > 0.
+   * Number of features. Should be greater than 0.
    * (default = 2^18^)
    * @group param
    */

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala
index 4463aea..c8760f9 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala
@@ -41,7 +41,7 @@ class NGram @Since("1.5.0") (@Since("1.5.0") override val uid: String)
   def this() = this(Identifiable.randomUID("ngram"))
 
   /**
-   * Minimum n-gram length, >= 1.
+   * Minimum n-gram length, greater than or equal to 1.
    * Default: 2, bigram features
    * @group param
    */

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala
index eb06900..6e96545 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala
@@ -37,7 +37,7 @@ class Normalizer @Since("1.4.0") (@Since("1.4.0") override val uid: String)
   def this() = this(Identifiable.randomUID("normalizer"))
 
   /**
-   * Normalization in L^p^ space.  Must be >= 1.
+   * Normalization in L^p^ space. Must be greater than equal to 1.
    * (default: p = 2)
    * @group param
    */

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala
index ea40121..ba1380b 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala
@@ -33,14 +33,14 @@ import org.apache.spark.sql.types.{DoubleType, NumericType, StructType}
  * at most a single one-value per row that indicates the input category index.
  * For example with 5 categories, an input value of 2.0 would map to an output vector of
  * `[0.0, 0.0, 1.0, 0.0]`.
- * The last category is not included by default (configurable via [[OneHotEncoder!.dropLast]]
+ * The last category is not included by default (configurable via `OneHotEncoder!.dropLast`
  * because it makes the vector entries sum up to one, and hence linearly dependent.
  * So an input value of 4.0 maps to `[0.0, 0.0, 0.0, 0.0]`.
  *
  * @note This is different from scikit-learn's OneHotEncoder, which keeps all categories.
  * The output vectors are sparse.
  *
- * @see [[StringIndexer]] for converting categorical values into category indices
+ * @see `StringIndexer` for converting categorical values into category indices
  */
 @Since("1.4.0")
 class OneHotEncoder @Since("1.4.0") (@Since("1.4.0") override val uid: String) extends Transformer

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala
index 6e08bf0..4143d86 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala
@@ -63,7 +63,7 @@ private[feature] trait PCAParams extends Params with HasInputCol with HasOutputC
 }
 
 /**
- * PCA trains a model to project vectors to a lower dimensional space of the top [[PCA!.k]]
+ * PCA trains a model to project vectors to a lower dimensional space of the top `PCA!.k`
  * principal components.
  */
 @Since("1.5.0")
@@ -144,7 +144,7 @@ class PCAModel private[ml] (
    * Transform a vector by computed Principal Components.
    *
    * @note Vectors to be transformed must be the same length as the source vectors given
-   * to [[PCA.fit()]].
+   * to `PCA.fit()`.
    */
   @Since("2.0.0")
   override def transform(dataset: Dataset[_]): DataFrame = {

http://git-wip-us.apache.org/repos/asf/spark/blob/f830bb91/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala
index 4be17da..292f949 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala
@@ -45,7 +45,8 @@ class PolynomialExpansion @Since("1.4.0") (@Since("1.4.0") override val uid: Str
   def this() = this(Identifiable.randomUID("poly"))
 
   /**
-   * The polynomial degree to expand, which should be >= 1.  A value of 1 means no expansion.
+   * The polynomial degree to expand, which should be greater than equal to 1. A value of 1 means
+   * no expansion.
    * Default: 2
    * @group param
    */


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