aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--core/src/main/scala/org/apache/spark/rpc/RpcEndpoint.scala10
-rw-r--r--core/src/main/scala/org/apache/spark/rpc/RpcTimeout.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/ExternalClusterManager.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala8
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockManager.scala2
-rw-r--r--core/src/test/scala/org/apache/spark/AccumulatorSuite.scala4
-rw-r--r--core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala2
-rw-r--r--core/src/test/scala/org/apache/spark/LocalSparkContext.scala2
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala4
-rw-r--r--core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala2
-rwxr-xr-xdev/run-tests.py15
-rw-r--r--graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala2
-rw-r--r--mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala2
-rw-r--r--mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala8
-rw-r--r--mllib/src/test/scala/org/apache/spark/ml/feature/LSHTest.scala12
-rw-r--r--mllib/src/test/scala/org/apache/spark/ml/param/ParamsSuite.scala2
-rw-r--r--mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala6
-rw-r--r--mllib/src/test/scala/org/apache/spark/ml/util/DefaultReadWriteTest.scala18
-rw-r--r--mllib/src/test/scala/org/apache/spark/ml/util/StopwatchSuite.scala4
-rw-r--r--mllib/src/test/scala/org/apache/spark/ml/util/TempDirectory.scala4
-rw-r--r--mllib/src/test/scala/org/apache/spark/mllib/tree/ImpuritySuite.scala2
-rw-r--r--mllib/src/test/scala/org/apache/spark/mllib/util/MLlibTestSparkContext.scala2
-rw-r--r--resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala6
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala8
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/UnsafeProjectionBenchmark.scala2
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala2
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala4
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala4
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala4
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala12
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala18
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala2
-rw-r--r--sql/hive-thriftserver/src/main/java/org/apache/hive/service/Service.java2
-rw-r--r--sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceOperations.java12
-rw-r--r--sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java2
-rw-r--r--sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PasswdAuthenticationProvider.java2
-rw-r--r--sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java9
-rw-r--r--sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIServiceUtils.java2
-rw-r--r--sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java6
-rw-r--r--sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java2
-rw-r--r--sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java4
-rw-r--r--sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadFactoryWithGarbageCleanup.java6
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala4
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala2
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcTest.scala4
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/rdd/MapWithStateRDD.scala4
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimator.scala2
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/scheduler/rate/RateEstimator.scala2
49 files changed, 140 insertions, 106 deletions
diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcEndpoint.scala b/core/src/main/scala/org/apache/spark/rpc/RpcEndpoint.scala
index 0ba9516952..97eed540b8 100644
--- a/core/src/main/scala/org/apache/spark/rpc/RpcEndpoint.scala
+++ b/core/src/main/scala/org/apache/spark/rpc/RpcEndpoint.scala
@@ -35,7 +35,7 @@ private[spark] trait RpcEnvFactory {
*
* The life-cycle of an endpoint is:
*
- * constructor -> onStart -> receive* -> onStop
+ * {@code constructor -> onStart -> receive* -> onStop}
*
* Note: `receive` can be called concurrently. If you want `receive` to be thread-safe, please use
* [[ThreadSafeRpcEndpoint]]
@@ -63,16 +63,16 @@ private[spark] trait RpcEndpoint {
}
/**
- * Process messages from [[RpcEndpointRef.send]] or [[RpcCallContext.reply)]]. If receiving a
- * unmatched message, [[SparkException]] will be thrown and sent to `onError`.
+ * Process messages from `RpcEndpointRef.send` or `RpcCallContext.reply`. If receiving a
+ * unmatched message, `SparkException` will be thrown and sent to `onError`.
*/
def receive: PartialFunction[Any, Unit] = {
case _ => throw new SparkException(self + " does not implement 'receive'")
}
/**
- * Process messages from [[RpcEndpointRef.ask]]. If receiving a unmatched message,
- * [[SparkException]] will be thrown and sent to `onError`.
+ * Process messages from `RpcEndpointRef.ask`. If receiving a unmatched message,
+ * `SparkException` will be thrown and sent to `onError`.
*/
def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
case _ => context.sendFailure(new SparkException(self + " won't reply anything"))
diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcTimeout.scala b/core/src/main/scala/org/apache/spark/rpc/RpcTimeout.scala
index 2c9a976e76..0557b7a3cc 100644
--- a/core/src/main/scala/org/apache/spark/rpc/RpcTimeout.scala
+++ b/core/src/main/scala/org/apache/spark/rpc/RpcTimeout.scala
@@ -26,7 +26,7 @@ import org.apache.spark.SparkConf
import org.apache.spark.util.{ThreadUtils, Utils}
/**
- * An exception thrown if RpcTimeout modifies a [[TimeoutException]].
+ * An exception thrown if RpcTimeout modifies a `TimeoutException`.
*/
private[rpc] class RpcTimeoutException(message: String, cause: TimeoutException)
extends TimeoutException(message) { initCause(cause) }
diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
index 0971731683..aab177f257 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -607,7 +607,7 @@ class DAGScheduler(
* @param resultHandler callback to pass each result to
* @param properties scheduler properties to attach to this job, e.g. fair scheduler pool name
*
- * @throws Exception when the job fails
+ * @note Throws `Exception` when the job fails
*/
def runJob[T, U](
rdd: RDD[T],
@@ -644,7 +644,7 @@ class DAGScheduler(
*
* @param rdd target RDD to run tasks on
* @param func a function to run on each partition of the RDD
- * @param evaluator [[ApproximateEvaluator]] to receive the partial results
+ * @param evaluator `ApproximateEvaluator` to receive the partial results
* @param callSite where in the user program this job was called
* @param timeout maximum time to wait for the job, in milliseconds
* @param properties scheduler properties to attach to this job, e.g. fair scheduler pool name
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExternalClusterManager.scala b/core/src/main/scala/org/apache/spark/scheduler/ExternalClusterManager.scala
index d1ac7131ba..47f3527a32 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ExternalClusterManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ExternalClusterManager.scala
@@ -42,7 +42,7 @@ private[spark] trait ExternalClusterManager {
/**
* Create a scheduler backend for the given SparkContext and scheduler. This is
- * called after task scheduler is created using [[ExternalClusterManager.createTaskScheduler()]].
+ * called after task scheduler is created using `ExternalClusterManager.createTaskScheduler()`.
* @param sc SparkContext
* @param masterURL the master URL
* @param scheduler TaskScheduler that will be used with the scheduler backend.
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
index c849a16023..1b6bc9139f 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
@@ -38,7 +38,7 @@ import org.apache.spark.util.{AccumulatorV2, ThreadUtils, Utils}
/**
* Schedules tasks for multiple types of clusters by acting through a SchedulerBackend.
- * It can also work with a local setup by using a [[LocalSchedulerBackend]] and setting
+ * It can also work with a local setup by using a `LocalSchedulerBackend` and setting
* isLocal to true. It handles common logic, like determining a scheduling order across jobs, waking
* up to launch speculative tasks, etc.
*
@@ -704,12 +704,12 @@ private[spark] object TaskSchedulerImpl {
* Used to balance containers across hosts.
*
* Accepts a map of hosts to resource offers for that host, and returns a prioritized list of
- * resource offers representing the order in which the offers should be used. The resource
+ * resource offers representing the order in which the offers should be used. The resource
* offers are ordered such that we'll allocate one container on each host before allocating a
* second container on any host, and so on, in order to reduce the damage if a host fails.
*
- * For example, given <h1, [o1, o2, o3]>, <h2, [o4]>, <h1, [o5, o6]>, returns
- * [o1, o5, o4, 02, o6, o3]
+ * For example, given {@literal <h1, [o1, o2, o3]>}, {@literal <h2, [o4]>} and
+ * {@literal <h3, [o5, o6]>}, returns {@literal [o1, o5, o4, o2, o6, o3]}.
*/
def prioritizeContainers[K, T] (map: HashMap[K, ArrayBuffer[T]]): List[T] = {
val _keyList = new ArrayBuffer[K](map.size)
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
index 63acba65d3..3219969bcd 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
@@ -66,7 +66,7 @@ private[spark] trait BlockData {
/**
* Returns a Netty-friendly wrapper for the block's data.
*
- * @see [[ManagedBuffer#convertToNetty()]]
+ * Please see `ManagedBuffer.convertToNetty()` for more details.
*/
def toNetty(): Object
diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala
index 6d03ee091e..ddbcb2d19d 100644
--- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala
+++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala
@@ -243,7 +243,7 @@ private[spark] object AccumulatorSuite {
import InternalAccumulator._
/**
- * Create a long accumulator and register it to [[AccumulatorContext]].
+ * Create a long accumulator and register it to `AccumulatorContext`.
*/
def createLongAccum(
name: String,
@@ -258,7 +258,7 @@ private[spark] object AccumulatorSuite {
}
/**
- * Make an [[AccumulableInfo]] out of an [[Accumulable]] with the intent to use the
+ * Make an `AccumulableInfo` out of an [[Accumulable]] with the intent to use the
* info as an accumulator update.
*/
def makeInfo(a: AccumulatorV2[_, _]): AccumulableInfo = a.toInfo(Some(a.value), None)
diff --git a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala
index eb3fb99747..fe944031bc 100644
--- a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala
@@ -27,7 +27,7 @@ import org.apache.spark.network.shuffle.{ExternalShuffleBlockHandler, ExternalSh
/**
* This suite creates an external shuffle server and routes all shuffle fetches through it.
* Note that failures in this suite may arise due to changes in Spark that invalidate expectations
- * set up in [[ExternalShuffleBlockHandler]], such as changing the format of shuffle files or how
+ * set up in `ExternalShuffleBlockHandler`, such as changing the format of shuffle files or how
* we hash files into folders.
*/
class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll {
diff --git a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala
index 24ec99c7e5..1dd89bcbe3 100644
--- a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala
+++ b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala
@@ -22,7 +22,7 @@ import org.scalatest.BeforeAndAfterAll
import org.scalatest.BeforeAndAfterEach
import org.scalatest.Suite
-/** Manages a local `sc` {@link SparkContext} variable, correctly stopping it after each test. */
+/** Manages a local `sc` `SparkContext` variable, correctly stopping it after each test. */
trait LocalSparkContext extends BeforeAndAfterEach with BeforeAndAfterAll { self: Suite =>
@transient var sc: SparkContext = _
diff --git a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala
index 8103983c43..8300607ea8 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala
@@ -95,12 +95,12 @@ abstract class SchedulerIntegrationSuite[T <: MockBackend: ClassTag] extends Spa
}
/**
- * A map from partition -> results for all tasks of a job when you call this test framework's
+ * A map from partition to results for all tasks of a job when you call this test framework's
* [[submit]] method. Two important considerations:
*
* 1. If there is a job failure, results may or may not be empty. If any tasks succeed before
* the job has failed, they will get included in `results`. Instead, check for job failure by
- * checking [[failure]]. (Also see [[assertDataStructuresEmpty()]])
+ * checking [[failure]]. (Also see `assertDataStructuresEmpty()`)
*
* 2. This only gets cleared between tests. So you'll need to do special handling if you submit
* more than one job in one test.
diff --git a/core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala b/core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala
index 4ce3b941be..99882bf76e 100644
--- a/core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala
+++ b/core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala
@@ -29,7 +29,7 @@ import org.apache.spark.serializer.KryoTest.RegistratorWithoutAutoReset
/**
* Tests to ensure that [[Serializer]] implementations obey the API contracts for methods that
* describe properties of the serialized stream, such as
- * [[Serializer.supportsRelocationOfSerializedObjects]].
+ * `Serializer.supportsRelocationOfSerializedObjects`.
*/
class SerializerPropertiesSuite extends SparkFunSuite {
diff --git a/dev/run-tests.py b/dev/run-tests.py
index 04035b33e6..450b68123e 100755
--- a/dev/run-tests.py
+++ b/dev/run-tests.py
@@ -344,6 +344,19 @@ def build_spark_sbt(hadoop_version):
exec_sbt(profiles_and_goals)
+def build_spark_unidoc_sbt(hadoop_version):
+ set_title_and_block("Building Unidoc API Documentation", "BLOCK_DOCUMENTATION")
+ # Enable all of the profiles for the build:
+ build_profiles = get_hadoop_profiles(hadoop_version) + modules.root.build_profile_flags
+ sbt_goals = ["unidoc"]
+ profiles_and_goals = build_profiles + sbt_goals
+
+ print("[info] Building Spark unidoc (w/Hive 1.2.1) using SBT with these arguments: ",
+ " ".join(profiles_and_goals))
+
+ exec_sbt(profiles_and_goals)
+
+
def build_spark_assembly_sbt(hadoop_version):
# Enable all of the profiles for the build:
build_profiles = get_hadoop_profiles(hadoop_version) + modules.root.build_profile_flags
@@ -352,6 +365,8 @@ def build_spark_assembly_sbt(hadoop_version):
print("[info] Building Spark assembly (w/Hive 1.2.1) using SBT with these arguments: ",
" ".join(profiles_and_goals))
exec_sbt(profiles_and_goals)
+ # Make sure that Java and Scala API documentation can be generated
+ build_spark_unidoc_sbt(hadoop_version)
def build_apache_spark(build_tool, hadoop_version):
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala b/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala
index d2ad9be555..66c4747fec 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala
@@ -21,7 +21,7 @@ import org.apache.spark.SparkConf
import org.apache.spark.SparkContext
/**
- * Provides a method to run tests against a {@link SparkContext} variable that is correctly stopped
+ * Provides a method to run tests against a `SparkContext` variable that is correctly stopped
* after each test.
*/
trait LocalSparkContext {
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 d8608d885d..bc0b49d48d 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
@@ -74,7 +74,7 @@ abstract class Classifier[
* 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
+ * @note Throws `SparkException` if any label is a non-integer or is negative
*/
protected def extractLabeledPoints(dataset: Dataset[_], numClasses: Int): RDD[LabeledPoint] = {
require(numClasses > 0, s"Classifier (in extractLabeledPoints) found numClasses =" +
diff --git a/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala
index 4cdbf845ae..4a7e4dd80f 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala
@@ -230,7 +230,9 @@ class PipelineSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul
}
-/** Used to test [[Pipeline]] with [[MLWritable]] stages */
+/**
+ * Used to test [[Pipeline]] with `MLWritable` stages
+ */
class WritableStage(override val uid: String) extends Transformer with MLWritable {
final val intParam: IntParam = new IntParam(this, "intParam", "doc")
@@ -257,7 +259,9 @@ object WritableStage extends MLReadable[WritableStage] {
override def load(path: String): WritableStage = super.load(path)
}
-/** Used to test [[Pipeline]] with non-[[MLWritable]] stages */
+/**
+ * Used to test [[Pipeline]] with non-`MLWritable` stages
+ */
class UnWritableStage(override val uid: String) extends Transformer {
final val intParam: IntParam = new IntParam(this, "intParam", "doc")
diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/LSHTest.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/LSHTest.scala
index dd4dd62b8c..db4f56ed60 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/feature/LSHTest.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/feature/LSHTest.scala
@@ -29,8 +29,10 @@ private[ml] object LSHTest {
* the following property is satisfied.
*
* There exist dist1, dist2, p1, p2, so that for any two elements e1 and e2,
- * If dist(e1, e2) <= dist1, then Pr{h(x) == h(y)} >= p1
- * If dist(e1, e2) >= dist2, then Pr{h(x) == h(y)} <= p2
+ * If dist(e1, e2) is less than or equal to dist1, then Pr{h(x) == h(y)} is greater than
+ * or equal to p1
+ * If dist(e1, e2) is greater than or equal to dist2, then Pr{h(x) == h(y)} is less than
+ * or equal to p2
*
* This is called locality sensitive property. This method checks the property on an
* existing dataset and calculate the probabilities.
@@ -38,8 +40,10 @@ private[ml] object LSHTest {
*
* This method hashes each elements to hash buckets using LSH, and calculate the false positive
* and false negative:
- * False positive: Of all (e1, e2) sharing any bucket, the probability of dist(e1, e2) > distFP
- * False negative: Of all (e1, e2) not sharing buckets, the probability of dist(e1, e2) < distFN
+ * False positive: Of all (e1, e2) sharing any bucket, the probability of dist(e1, e2) is greater
+ * than distFP
+ * False negative: Of all (e1, e2) not sharing buckets, the probability of dist(e1, e2) is less
+ * than distFN
*
* @param dataset The dataset to verify the locality sensitive hashing property.
* @param lsh The lsh instance to perform the hashing
diff --git a/mllib/src/test/scala/org/apache/spark/ml/param/ParamsSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/param/ParamsSuite.scala
index aa9c53ca30..78a33e05e0 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/param/ParamsSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/param/ParamsSuite.scala
@@ -377,7 +377,7 @@ class ParamsSuite extends SparkFunSuite {
object ParamsSuite extends SparkFunSuite {
/**
- * Checks common requirements for [[Params.params]]:
+ * Checks common requirements for `Params.params`:
* - params are ordered by names
* - param parent has the same UID as the object's UID
* - param name is the same as the param method name
diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala
index c90cb8ca10..92a236928e 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala
@@ -34,7 +34,7 @@ private[ml] object TreeTests extends SparkFunSuite {
* Convert the given data to a DataFrame, and set the features and label metadata.
* @param data Dataset. Categorical features and labels must already have 0-based indices.
* This must be non-empty.
- * @param categoricalFeatures Map: categorical feature index -> number of distinct values
+ * @param categoricalFeatures Map: categorical feature index to number of distinct values
* @param numClasses Number of classes label can take. If 0, mark as continuous.
* @return DataFrame with metadata
*/
@@ -69,7 +69,9 @@ private[ml] object TreeTests extends SparkFunSuite {
df("label").as("label", labelMetadata))
}
- /** Java-friendly version of [[setMetadata()]] */
+ /**
+ * Java-friendly version of `setMetadata()`
+ */
def setMetadata(
data: JavaRDD[LabeledPoint],
categoricalFeatures: java.util.Map[java.lang.Integer, java.lang.Integer],
diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/DefaultReadWriteTest.scala b/mllib/src/test/scala/org/apache/spark/ml/util/DefaultReadWriteTest.scala
index bfe8f12258..27d606cb05 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/util/DefaultReadWriteTest.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/util/DefaultReadWriteTest.scala
@@ -81,20 +81,20 @@ trait DefaultReadWriteTest extends TempDirectory { self: Suite =>
/**
* Default test for Estimator, Model pairs:
* - Explicitly set Params, and train model
- * - Test save/load using [[testDefaultReadWrite()]] on Estimator and Model
+ * - Test save/load using `testDefaultReadWrite` on Estimator and Model
* - Check Params on Estimator and Model
* - Compare model data
*
- * This requires that [[Model]]'s [[Param]]s should be a subset of [[Estimator]]'s [[Param]]s.
+ * This requires that `Model`'s `Param`s should be a subset of `Estimator`'s `Param`s.
*
* @param estimator Estimator to test
- * @param dataset Dataset to pass to [[Estimator.fit()]]
- * @param testEstimatorParams Set of [[Param]] values to set in estimator
- * @param testModelParams Set of [[Param]] values to set in model
- * @param checkModelData Method which takes the original and loaded [[Model]] and compares their
- * data. This method does not need to check [[Param]] values.
- * @tparam E Type of [[Estimator]]
- * @tparam M Type of [[Model]] produced by estimator
+ * @param dataset Dataset to pass to `Estimator.fit()`
+ * @param testEstimatorParams Set of `Param` values to set in estimator
+ * @param testModelParams Set of `Param` values to set in model
+ * @param checkModelData Method which takes the original and loaded `Model` and compares their
+ * data. This method does not need to check `Param` values.
+ * @tparam E Type of `Estimator`
+ * @tparam M Type of `Model` produced by estimator
*/
def testEstimatorAndModelReadWrite[
E <: Estimator[M] with MLWritable, M <: Model[M] with MLWritable](
diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/StopwatchSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/util/StopwatchSuite.scala
index 141249a427..54e363a8b9 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/util/StopwatchSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/util/StopwatchSuite.scala
@@ -105,8 +105,8 @@ class StopwatchSuite extends SparkFunSuite with MLlibTestSparkContext {
private object StopwatchSuite extends SparkFunSuite {
/**
- * Checks the input stopwatch on a task that takes a random time (<10ms) to finish. Validates and
- * returns the duration reported by the stopwatch.
+ * Checks the input stopwatch on a task that takes a random time (less than 10ms) to finish.
+ * Validates and returns the duration reported by the stopwatch.
*/
def checkStopwatch(sw: Stopwatch): Long = {
val ubStart = now
diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/TempDirectory.scala b/mllib/src/test/scala/org/apache/spark/ml/util/TempDirectory.scala
index 8f11bbc8e4..50b73e0e99 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/util/TempDirectory.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/util/TempDirectory.scala
@@ -30,7 +30,9 @@ trait TempDirectory extends BeforeAndAfterAll { self: Suite =>
private var _tempDir: File = _
- /** Returns the temporary directory as a [[File]] instance. */
+ /**
+ * Returns the temporary directory as a `File` instance.
+ */
protected def tempDir: File = _tempDir
override def beforeAll(): Unit = {
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/ImpuritySuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/ImpuritySuite.scala
index 14152cdd63..d0f02dd966 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/tree/ImpuritySuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/ImpuritySuite.scala
@@ -21,7 +21,7 @@ import org.apache.spark.SparkFunSuite
import org.apache.spark.mllib.tree.impurity.{EntropyAggregator, GiniAggregator}
/**
- * Test suites for [[GiniAggregator]] and [[EntropyAggregator]].
+ * Test suites for `GiniAggregator` and `EntropyAggregator`.
*/
class ImpuritySuite extends SparkFunSuite {
test("Gini impurity does not support negative labels") {
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLlibTestSparkContext.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLlibTestSparkContext.scala
index 6bb7ed9c95..720237bd2d 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLlibTestSparkContext.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLlibTestSparkContext.scala
@@ -60,7 +60,7 @@ trait MLlibTestSparkContext extends TempDirectory { self: Suite =>
* A helper object for importing SQL implicits.
*
* Note that the alternative of importing `spark.implicits._` is not possible here.
- * This is because we create the [[SQLContext]] immediately before the first test is run,
+ * This is because we create the `SQLContext` immediately before the first test is run,
* but the implicits import is needed in the constructor.
*/
protected object testImplicits extends SQLImplicits {
diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala
index 3f25535cb5..9d81025a30 100644
--- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala
+++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala
@@ -239,7 +239,7 @@ trait MesosSchedulerUtils extends Logging {
}
/**
- * Converts the attributes from the resource offer into a Map of name -> Attribute Value
+ * Converts the attributes from the resource offer into a Map of name to Attribute Value
* The attribute values are the mesos attribute types and they are
*
* @param offerAttributes the attributes offered
@@ -296,7 +296,7 @@ trait MesosSchedulerUtils extends Logging {
/**
* Parses the attributes constraints provided to spark and build a matching data struct:
- * Map[<attribute-name>, Set[values-to-match]]
+ * {@literal Map[<attribute-name>, Set[values-to-match]}
* The constraints are specified as ';' separated key-value pairs where keys and values
* are separated by ':'. The ':' implies equality (for singular values) and "is one of" for
* multiple values (comma separated). For example:
@@ -354,7 +354,7 @@ trait MesosSchedulerUtils extends Logging {
* container overheads.
*
* @param sc SparkContext to use to get `spark.mesos.executor.memoryOverhead` value
- * @return memory requirement as (0.1 * <memoryOverhead>) or MEMORY_OVERHEAD_MINIMUM
+ * @return memory requirement as (0.1 * memoryOverhead) or MEMORY_OVERHEAD_MINIMUM
* (whichever is larger)
*/
def executorMemory(sc: SparkContext): Int = {
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala
index 8508697995..8ae3ff5043 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala
@@ -117,11 +117,11 @@ object RandomDataGenerator {
}
/**
- * Returns a function which generates random values for the given [[DataType]], or `None` if no
+ * Returns a function which generates random values for the given `DataType`, or `None` if no
* random data generator is defined for that data type. The generated values will use an external
- * representation of the data type; for example, the random generator for [[DateType]] will return
- * instances of [[java.sql.Date]] and the generator for [[StructType]] will return a [[Row]].
- * For a [[UserDefinedType]] for a class X, an instance of class X is returned.
+ * representation of the data type; for example, the random generator for `DateType` will return
+ * instances of [[java.sql.Date]] and the generator for `StructType` will return a [[Row]].
+ * For a `UserDefinedType` for a class X, an instance of class X is returned.
*
* @param dataType the type to generate values for
* @param nullable whether null values should be generated
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/UnsafeProjectionBenchmark.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/UnsafeProjectionBenchmark.scala
index a6d9040938..769addf3b2 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/UnsafeProjectionBenchmark.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/UnsafeProjectionBenchmark.scala
@@ -24,7 +24,7 @@ import org.apache.spark.sql.types._
import org.apache.spark.util.Benchmark
/**
- * Benchmark [[UnsafeProjection]] for fixed-length/primitive-type fields.
+ * Benchmark `UnsafeProjection` for fixed-length/primitive-type fields.
*/
object UnsafeProjectionBenchmark {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala
index 074952ff79..7e5da012f8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala
@@ -510,7 +510,7 @@ abstract class Catalog {
def refreshTable(tableName: String): Unit
/**
- * Invalidates and refreshes all the cached data (and the associated metadata) for any [[Dataset]]
+ * Invalidates and refreshes all the cached data (and the associated metadata) for any `Dataset`
* that contains the given data source path. Path matching is by prefix, i.e. "/" would invalidate
* everything that is cached.
*
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala
index 0f3d0cefe3..92c5656f65 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala
@@ -56,7 +56,9 @@ object TestRegistrator {
def apply(): TestRegistrator = new TestRegistrator()
}
-/** A [[Serializer]] that takes a [[KryoData]] and serializes it as KryoData(0). */
+/**
+ * A `Serializer` that takes a [[KryoData]] and serializes it as KryoData(0).
+ */
class ZeroKryoDataSerializer extends Serializer[KryoData] {
override def write(kryo: Kryo, output: Output, t: KryoData): Unit = {
output.writeInt(0)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala
index 26967782f7..2108b118bf 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala
@@ -44,8 +44,8 @@ abstract class FileStreamSourceTest
import testImplicits._
/**
- * A subclass [[AddData]] for adding data to files. This is meant to use the
- * [[FileStreamSource]] actually being used in the execution.
+ * A subclass `AddData` for adding data to files. This is meant to use the
+ * `FileStreamSource` actually being used in the execution.
*/
abstract class AddFileData extends AddData {
override def addData(query: Option[StreamExecution]): (Source, Offset) = {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala
index 5ab9dc2bc7..13fe51a557 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala
@@ -569,7 +569,7 @@ class ThrowingIOExceptionLikeHadoop12074 extends FakeSource {
object ThrowingIOExceptionLikeHadoop12074 {
/**
- * A latch to allow the user to wait until [[ThrowingIOExceptionLikeHadoop12074.createSource]] is
+ * A latch to allow the user to wait until `ThrowingIOExceptionLikeHadoop12074.createSource` is
* called.
*/
@volatile var createSourceLatch: CountDownLatch = null
@@ -600,7 +600,7 @@ class ThrowingInterruptedIOException extends FakeSource {
object ThrowingInterruptedIOException {
/**
- * A latch to allow the user to wait until [[ThrowingInterruptedIOException.createSource]] is
+ * A latch to allow the user to wait until `ThrowingInterruptedIOException.createSource` is
* called.
*/
@volatile var createSourceLatch: CountDownLatch = null
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
index 2ebbfcd22b..b69536ed37 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
@@ -642,8 +642,10 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi
*
* @param expectedBehavior Expected behavior (not blocked, blocked, or exception thrown)
* @param timeoutMs Timeout in milliseconds
- * When timeoutMs <= 0, awaitTermination() is tested (i.e. w/o timeout)
- * When timeoutMs > 0, awaitTermination(timeoutMs) is tested
+ * When timeoutMs is less than or equal to 0, awaitTermination() is
+ * tested (i.e. w/o timeout)
+ * When timeoutMs is greater than 0, awaitTermination(timeoutMs) is
+ * tested
* @param expectedReturnValue Expected return value when awaitTermination(timeoutMs) is used
*/
case class TestAwaitTermination(
@@ -667,8 +669,10 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi
*
* @param expectedBehavior Expected behavior (not blocked, blocked, or exception thrown)
* @param timeoutMs Timeout in milliseconds
- * When timeoutMs <= 0, awaitTermination() is tested (i.e. w/o timeout)
- * When timeoutMs > 0, awaitTermination(timeoutMs) is tested
+ * When timeoutMs is less than or equal to 0, awaitTermination() is
+ * tested (i.e. w/o timeout)
+ * When timeoutMs is greater than 0, awaitTermination(timeoutMs) is
+ * tested
* @param expectedReturnValue Expected return value when awaitTermination(timeoutMs) is used
*/
def assertOnQueryCondition(
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala
index cab219216d..6a4cc95d36 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala
@@ -41,11 +41,11 @@ import org.apache.spark.util.{UninterruptibleThread, Utils}
/**
* Helper trait that should be extended by all SQL test suites.
*
- * This allows subclasses to plugin a custom [[SQLContext]]. It comes with test data
+ * This allows subclasses to plugin a custom `SQLContext`. It comes with test data
* prepared in advance as well as all implicit conversions used extensively by dataframes.
- * To use implicit methods, import `testImplicits._` instead of through the [[SQLContext]].
+ * To use implicit methods, import `testImplicits._` instead of through the `SQLContext`.
*
- * Subclasses should *not* create [[SQLContext]]s in the test suite constructor, which is
+ * Subclasses should *not* create `SQLContext`s in the test suite constructor, which is
* prone to leaving multiple overlapping [[org.apache.spark.SparkContext]]s in the same JVM.
*/
private[sql] trait SQLTestUtils
@@ -65,7 +65,7 @@ private[sql] trait SQLTestUtils
* A helper object for importing SQL implicits.
*
* Note that the alternative of importing `spark.implicits._` is not possible here.
- * This is because we create the [[SQLContext]] immediately before the first test is run,
+ * This is because we create the `SQLContext` immediately before the first test is run,
* but the implicits import is needed in the constructor.
*/
protected object testImplicits extends SQLImplicits {
@@ -73,7 +73,7 @@ private[sql] trait SQLTestUtils
}
/**
- * Materialize the test data immediately after the [[SQLContext]] is set up.
+ * Materialize the test data immediately after the `SQLContext` is set up.
* This is necessary if the data is accessed by name but not through direct reference.
*/
protected def setupTestData(): Unit = {
@@ -250,8 +250,8 @@ private[sql] trait SQLTestUtils
}
/**
- * Turn a logical plan into a [[DataFrame]]. This should be removed once we have an easier
- * way to construct [[DataFrame]] directly out of local data without relying on implicits.
+ * Turn a logical plan into a `DataFrame`. This should be removed once we have an easier
+ * way to construct `DataFrame` directly out of local data without relying on implicits.
*/
protected implicit def logicalPlanToSparkQuery(plan: LogicalPlan): DataFrame = {
Dataset.ofRows(spark, plan)
@@ -271,7 +271,9 @@ private[sql] trait SQLTestUtils
}
}
- /** Run a test on a separate [[UninterruptibleThread]]. */
+ /**
+ * Run a test on a separate `UninterruptibleThread`.
+ */
protected def testWithUninterruptibleThread(name: String, quietly: Boolean = false)
(body: => Unit): Unit = {
val timeoutMillis = 10000
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala
index b01977a238..959edf9a49 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala
@@ -22,7 +22,7 @@ import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.internal.{SessionState, SessionStateBuilder, SQLConf, WithTestConf}
/**
- * A special [[SparkSession]] prepared for testing.
+ * A special `SparkSession` prepared for testing.
*/
private[sql] class TestSparkSession(sc: SparkContext) extends SparkSession(sc) { self =>
def this(sparkConf: SparkConf) {
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/Service.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/Service.java
index b95077cd62..0d0e3e4011 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/Service.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/Service.java
@@ -49,7 +49,7 @@ public interface Service {
* The transition must be from {@link STATE#NOTINITED} to {@link STATE#INITED} unless the
* operation failed and an exception was raised.
*
- * @param config
+ * @param conf
* the configuration of the service
*/
void init(HiveConf conf);
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceOperations.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceOperations.java
index a2c580d6ac..c3219aabfc 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceOperations.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceOperations.java
@@ -51,7 +51,7 @@ public final class ServiceOperations {
/**
* Initialize a service.
- * <p/>
+ *
* The service state is checked <i>before</i> the operation begins.
* This process is <i>not</i> thread safe.
* @param service a service that must be in the state
@@ -69,7 +69,7 @@ public final class ServiceOperations {
/**
* Start a service.
- * <p/>
+ *
* The service state is checked <i>before</i> the operation begins.
* This process is <i>not</i> thread safe.
* @param service a service that must be in the state
@@ -86,7 +86,7 @@ public final class ServiceOperations {
/**
* Initialize then start a service.
- * <p/>
+ *
* The service state is checked <i>before</i> the operation begins.
* This process is <i>not</i> thread safe.
* @param service a service that must be in the state
@@ -102,9 +102,9 @@ public final class ServiceOperations {
/**
* Stop a service.
- * <p/>Do nothing if the service is null or not
- * in a state in which it can be/needs to be stopped.
- * <p/>
+ *
+ * Do nothing if the service is null or not in a state in which it can be/needs to be stopped.
+ *
* The service state is checked <i>before</i> the operation begins.
* This process is <i>not</i> thread safe.
* @param service a service or null
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java
index 5021528299..f7375ee707 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java
@@ -89,7 +89,7 @@ public final class HttpAuthUtils {
* @param clientUserName Client User name.
* @return An unsigned cookie token generated from input parameters.
* The final cookie generated is of the following format :
- * cu=<username>&rn=<randomNumber>&s=<cookieSignature>
+ * {@code cu=<username>&rn=<randomNumber>&s=<cookieSignature>}
*/
public static String createCookieToken(String clientUserName) {
StringBuffer sb = new StringBuffer();
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PasswdAuthenticationProvider.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PasswdAuthenticationProvider.java
index e2a6de165a..1af1c1d06e 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PasswdAuthenticationProvider.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PasswdAuthenticationProvider.java
@@ -26,7 +26,7 @@ public interface PasswdAuthenticationProvider {
* to authenticate users for their requests.
* If a user is to be granted, return nothing/throw nothing.
* When a user is to be disallowed, throw an appropriate {@link AuthenticationException}.
- * <p/>
+ *
* For an example implementation, see {@link LdapAuthenticationProviderImpl}.
*
* @param user The username received over the connection request
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java
index 645e3e2bbd..9a61ad4994 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java
@@ -31,12 +31,9 @@ import org.slf4j.LoggerFactory;
/**
* This class is responsible for setting the ipAddress for operations executed via HiveServer2.
- * <p>
- * <ul>
- * <li>IP address is only set for operations that calls listeners with hookContext</li>
- * <li>IP address is only set if the underlying transport mechanism is socket</li>
- * </ul>
- * </p>
+ *
+ * - IP address is only set for operations that calls listeners with hookContext
+ * - IP address is only set if the underlying transport mechanism is socket
*
* @see org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext
*/
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIServiceUtils.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIServiceUtils.java
index 9d64b102e0..bf2380632f 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIServiceUtils.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIServiceUtils.java
@@ -38,7 +38,7 @@ public class CLIServiceUtils {
* Convert a SQL search pattern into an equivalent Java Regex.
*
* @param pattern input which may contain '%' or '_' wildcard characters, or
- * these characters escaped using {@link #getSearchStringEscape()}.
+ * these characters escaped using {@code getSearchStringEscape()}.
* @return replace %/_ with regex search characters, also handle escaped
* characters.
*/
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java
index 05a6bf9384..af36057bda 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java
@@ -28,9 +28,9 @@ import org.apache.hadoop.hive.metastore.TableType;
/**
* ClassicTableTypeMapping.
* Classic table type mapping :
- * Managed Table ==> Table
- * External Table ==> Table
- * Virtual View ==> View
+ * Managed Table to Table
+ * External Table to Table
+ * Virtual View to View
*/
public class ClassicTableTypeMapping implements TableTypeMapping {
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java
index e392c459cf..e59d19ea6b 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java
@@ -31,7 +31,7 @@ public interface TableTypeMapping {
/**
* Map hive's table type name to client's table type
- * @param clientTypeName
+ * @param hiveTypeName
* @return
*/
String mapToClientType(String hiveTypeName);
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java
index de066dd406..c1b3892f52 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java
@@ -224,7 +224,9 @@ public class SessionManager extends CompositeService {
* The username passed to this method is the effective username.
* If withImpersonation is true (==doAs true) we wrap all the calls in HiveSession
* within a UGI.doAs, where UGI corresponds to the effective user.
- * @see org.apache.hive.service.cli.thrift.ThriftCLIService#getUserName()
+ *
+ * Please see {@code org.apache.hive.service.cli.thrift.ThriftCLIService.getUserName()} for
+ * more details.
*
* @param protocol
* @param username
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadFactoryWithGarbageCleanup.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadFactoryWithGarbageCleanup.java
index fb8141a905..94f8126552 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadFactoryWithGarbageCleanup.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadFactoryWithGarbageCleanup.java
@@ -30,12 +30,12 @@ import org.apache.hadoop.hive.metastore.RawStore;
* in custom cleanup code to be called before this thread is GC-ed.
* Currently cleans up the following:
* 1. ThreadLocal RawStore object:
- * In case of an embedded metastore, HiveServer2 threads (foreground & background)
+ * In case of an embedded metastore, HiveServer2 threads (foreground and background)
* end up caching a ThreadLocal RawStore object. The ThreadLocal RawStore object has
- * an instance of PersistenceManagerFactory & PersistenceManager.
+ * an instance of PersistenceManagerFactory and PersistenceManager.
* The PersistenceManagerFactory keeps a cache of PersistenceManager objects,
* which are only removed when PersistenceManager#close method is called.
- * HiveServer2 uses ExecutorService for managing thread pools for foreground & background threads.
+ * HiveServer2 uses ExecutorService for managing thread pools for foreground and background threads.
* ExecutorService unfortunately does not provide any hooks to be called,
* when a thread from the pool is terminated.
* As a solution, we're using this ThreadFactory to keep a cache of RawStore objects per thread.
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 6f5b923cd4..4dec2f71b8 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
@@ -53,8 +53,8 @@ import org.apache.spark.unsafe.types.UTF8String
* java.sql.Date
* java.sql.Timestamp
* Complex Types =>
- * Map: [[MapData]]
- * List: [[ArrayData]]
+ * Map: `MapData`
+ * List: `ArrayData`
* Struct: [[org.apache.spark.sql.catalyst.InternalRow]]
* Union: NOT SUPPORTED YET
* The Complex types plays as a container, which can hold arbitrary data types.
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala
index e772324a57..bb4ce6d3aa 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala
@@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.util._
/**
* A framework for running the query tests that are listed as a set of text files.
*
- * TestSuites that derive from this class must provide a map of testCaseName -> testCaseFiles
+ * TestSuites that derive from this class must provide a map of testCaseName to testCaseFiles
* that should be included. Additionally, there is support for whitelisting and blacklisting
* tests as development progresses.
*/
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcTest.scala
index 7226ed521e..a2f08c5ba7 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcTest.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcTest.scala
@@ -43,7 +43,7 @@ private[sql] trait OrcTest extends SQLTestUtils with TestHiveSingleton {
}
/**
- * Writes `data` to a Orc file and reads it back as a [[DataFrame]],
+ * Writes `data` to a Orc file and reads it back as a `DataFrame`,
* which is then passed to `f`. The Orc file will be deleted after `f` returns.
*/
protected def withOrcDataFrame[T <: Product: ClassTag: TypeTag]
@@ -53,7 +53,7 @@ private[sql] trait OrcTest extends SQLTestUtils with TestHiveSingleton {
}
/**
- * Writes `data` to a Orc file, reads it back as a [[DataFrame]] and registers it as a
+ * Writes `data` to a Orc file, reads it back as a `DataFrame` and registers it as a
* temporary table named `tableName`, then call `f`. The temporary table together with the
* Orc file will be dropped/deleted after `f` returns.
*/
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/rdd/MapWithStateRDD.scala b/streaming/src/main/scala/org/apache/spark/streaming/rdd/MapWithStateRDD.scala
index 58b7031d5e..15d3c7e54b 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/rdd/MapWithStateRDD.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/rdd/MapWithStateRDD.scala
@@ -29,7 +29,7 @@ import org.apache.spark.streaming.util.{EmptyStateMap, StateMap}
import org.apache.spark.util.Utils
/**
- * Record storing the keyed-state [[MapWithStateRDD]]. Each record contains a [[StateMap]] and a
+ * Record storing the keyed-state [[MapWithStateRDD]]. Each record contains a `StateMap` and a
* sequence of records returned by the mapping function of `mapWithState`.
*/
private[streaming] case class MapWithStateRDDRecord[K, S, E](
@@ -111,7 +111,7 @@ private[streaming] class MapWithStateRDDPartition(
/**
* RDD storing the keyed states of `mapWithState` operation and corresponding mapped data.
* Each partition of this RDD has a single record of type [[MapWithStateRDDRecord]]. This contains a
- * [[StateMap]] (containing the keyed-states) and the sequence of records returned by the mapping
+ * `StateMap` (containing the keyed-states) and the sequence of records returned by the mapping
* function of `mapWithState`.
* @param prevStateRDD The previous MapWithStateRDD on whose StateMap data `this` RDD
* will be created
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimator.scala
index a73e6cc2cd..dc02062b9e 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimator.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimator.scala
@@ -26,7 +26,7 @@ import org.apache.spark.internal.Logging
* case of Spark Streaming the error is the difference between the measured processing
* rate (number of elements/processing delay) and the previous rate.
*
- * @see https://en.wikipedia.org/wiki/PID_controller
+ * @see <a href="https://en.wikipedia.org/wiki/PID_controller">PID controller (Wikipedia)</a>
*
* @param batchIntervalMillis the batch duration, in milliseconds
* @param proportional how much the correction should depend on the current
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/rate/RateEstimator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/rate/RateEstimator.scala
index 7b2ef6881d..e4b9dffee0 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/rate/RateEstimator.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/rate/RateEstimator.scala
@@ -24,7 +24,7 @@ import org.apache.spark.streaming.Duration
* A component that estimates the rate at which an `InputDStream` should ingest
* records, based on updates at every batch completion.
*
- * @see [[org.apache.spark.streaming.scheduler.RateController]]
+ * Please see `org.apache.spark.streaming.scheduler.RateController` for more details.
*/
private[streaming] trait RateEstimator extends Serializable {