From 51b1c1551d3a7147403b9e821fcc7c8f57b4824c Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Fri, 25 Nov 2016 11:27:07 +0000 Subject: [SPARK-3359][BUILD][DOCS] More changes to resolve javadoc 8 errors that will help unidoc/genjavadoc compatibility ## What changes were proposed in this pull request? This PR only tries to fix things that looks pretty straightforward and were fixed in other previous PRs before. This PR roughly fixes several things as below: - Fix unrecognisable class and method links in javadoc by changing it from `[[..]]` to `` `...` `` ``` [error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/DataStreamReader.java:226: error: reference not found [error] * Loads text files and returns a {link DataFrame} whose schema starts with a string column named ``` - Fix an exception annotation and remove code backticks in `throws` annotation Currently, sbt unidoc with Java 8 complains as below: ``` [error] .../java/org/apache/spark/sql/streaming/StreamingQuery.java:72: error: unexpected text [error] * throws StreamingQueryException, if this query has terminated with an exception. ``` `throws` should specify the correct class name from `StreamingQueryException,` to `StreamingQueryException` without backticks. (see [JDK-8007644](https://bugs.openjdk.java.net/browse/JDK-8007644)). - Fix `[[http..]]` to ``. ```diff - * [[https://blogs.oracle.com/java-platform-group/entry/diagnosing_tls_ssl_and_https Oracle - * blog page]]. + * + * Oracle blog page. ``` `[[http...]]` link markdown in scaladoc is unrecognisable in javadoc. - It seems class can't have `return` annotation. So, two cases of this were removed. ``` [error] .../java/org/apache/spark/mllib/regression/IsotonicRegression.java:27: error: invalid use of return [error] * return New instance of IsotonicRegression. ``` - Fix < to `<` and > to `>` according to HTML rules. - Fix `

` complaint - Exclude unrecognisable in javadoc, `constructor`, `todo` and `groupname`. ## 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) ``` Note: this does not yet make sbt unidoc suceed with Java 8 yet but it reduces the number of errors with Java 8. Author: hyukjinkwon Closes #15999 from HyukjinKwon/SPARK-3359-errors. --- .../main/scala/org/apache/spark/SSLOptions.scala | 4 +- .../org/apache/spark/api/java/JavaPairRDD.scala | 6 +- .../scala/org/apache/spark/api/java/JavaRDD.scala | 10 +-- .../apache/spark/api/java/JavaSparkContext.scala | 14 ++-- .../org/apache/spark/io/CompressionCodec.scala | 2 +- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 18 +++--- .../apache/spark/security/CryptoStreamUtils.scala | 4 +- .../apache/spark/serializer/KryoSerializer.scala | 3 +- .../spark/storage/BlockReplicationPolicy.scala | 7 +- .../main/scala/org/apache/spark/ui/UIUtils.scala | 4 +- .../org/apache/spark/util/AccumulatorV2.scala | 2 +- .../scala/org/apache/spark/util/RpcUtils.scala | 2 +- .../scala/org/apache/spark/util/StatCounter.scala | 4 +- .../scala/org/apache/spark/util/ThreadUtils.scala | 6 +- .../main/scala/org/apache/spark/util/Utils.scala | 10 +-- .../apache/spark/util/io/ChunkedByteBuffer.scala | 2 +- .../main/scala/org/apache/spark/graphx/Graph.scala | 4 +- .../org/apache/spark/graphx/GraphLoader.scala | 2 +- .../org/apache/spark/graphx/impl/EdgeRDDImpl.scala | 2 +- .../org/apache/spark/graphx/lib/PageRank.scala | 4 +- .../org/apache/spark/graphx/lib/SVDPlusPlus.scala | 3 +- .../apache/spark/graphx/lib/TriangleCount.scala | 2 +- .../stat/distribution/MultivariateGaussian.scala | 3 +- .../main/scala/org/apache/spark/ml/Predictor.scala | 2 +- .../apache/spark/ml/attribute/AttributeGroup.scala | 2 +- .../org/apache/spark/ml/attribute/attributes.scala | 4 +- .../ml/classification/LogisticRegression.scala | 74 +++++++++++----------- .../MultilayerPerceptronClassifier.scala | 1 - .../spark/ml/classification/NaiveBayes.scala | 8 ++- .../ml/classification/RandomForestClassifier.scala | 6 +- .../spark/ml/clustering/BisectingKMeans.scala | 14 ++-- .../spark/ml/clustering/ClusteringSummary.scala | 2 +- .../spark/ml/clustering/GaussianMixture.scala | 6 +- .../org/apache/spark/ml/clustering/KMeans.scala | 8 +-- .../scala/org/apache/spark/ml/clustering/LDA.scala | 42 ++++++------ .../scala/org/apache/spark/ml/feature/DCT.scala | 3 +- .../org/apache/spark/ml/feature/MinHash.scala | 5 +- .../org/apache/spark/ml/feature/MinMaxScaler.scala | 4 +- .../spark/ml/feature/PolynomialExpansion.scala | 14 ++-- .../apache/spark/ml/feature/RandomProjection.scala | 4 +- .../apache/spark/ml/feature/StandardScaler.scala | 4 +- .../apache/spark/ml/feature/StopWordsRemover.scala | 5 +- .../org/apache/spark/ml/feature/package.scala | 3 +- .../optim/IterativelyReweightedLeastSquares.scala | 7 +- .../spark/ml/param/shared/sharedParams.scala | 12 ++-- .../ml/regression/AFTSurvivalRegression.scala | 27 ++++---- .../ml/regression/DecisionTreeRegressor.scala | 4 +- .../apache/spark/ml/regression/GBTRegressor.scala | 4 +- .../regression/GeneralizedLinearRegression.scala | 12 ++-- .../spark/ml/regression/LinearRegression.scala | 38 +++++------ .../ml/regression/RandomForestRegressor.scala | 5 +- .../spark/ml/source/libsvm/LibSVMDataSource.scala | 13 ++-- .../spark/ml/tree/impl/GradientBoostedTrees.scala | 10 +-- .../apache/spark/ml/tree/impl/RandomForest.scala | 2 +- .../org/apache/spark/ml/tree/treeParams.scala | 6 +- .../apache/spark/ml/tuning/CrossValidator.scala | 4 +- .../scala/org/apache/spark/ml/util/ReadWrite.scala | 10 +-- .../spark/mllib/classification/NaiveBayes.scala | 28 ++++---- .../spark/mllib/clustering/BisectingKMeans.scala | 21 +++--- .../mllib/clustering/BisectingKMeansModel.scala | 4 +- .../spark/mllib/clustering/GaussianMixture.scala | 6 +- .../mllib/clustering/GaussianMixtureModel.scala | 2 +- .../org/apache/spark/mllib/clustering/LDA.scala | 24 +++---- .../apache/spark/mllib/clustering/LDAModel.scala | 2 +- .../spark/mllib/clustering/LDAOptimizer.scala | 2 +- .../clustering/PowerIterationClustering.scala | 13 ++-- .../spark/mllib/clustering/StreamingKMeans.scala | 4 +- .../spark/mllib/evaluation/RegressionMetrics.scala | 10 +-- .../org/apache/spark/mllib/fpm/FPGrowth.scala | 12 ++-- .../org/apache/spark/mllib/fpm/PrefixSpan.scala | 7 +- .../mllib/linalg/distributed/BlockMatrix.scala | 20 +++--- .../linalg/distributed/CoordinateMatrix.scala | 4 +- .../linalg/distributed/IndexedRowMatrix.scala | 4 +- .../spark/mllib/linalg/distributed/RowMatrix.scala | 2 +- .../apache/spark/mllib/optimization/Gradient.scala | 24 +++---- .../spark/mllib/optimization/GradientDescent.scala | 4 +- .../apache/spark/mllib/optimization/LBFGS.scala | 7 +- .../org/apache/spark/mllib/optimization/NNLS.scala | 2 +- .../apache/spark/mllib/optimization/Updater.scala | 6 +- .../scala/org/apache/spark/mllib/package.scala | 4 +- .../org/apache/spark/mllib/rdd/RDDFunctions.scala | 2 +- .../apache/spark/mllib/recommendation/ALS.scala | 7 +- .../recommendation/MatrixFactorizationModel.scala | 6 +- .../mllib/regression/IsotonicRegression.scala | 9 ++- .../mllib/stat/MultivariateOnlineSummarizer.scala | 7 +- .../org/apache/spark/mllib/stat/Statistics.scala | 11 ++-- .../stat/distribution/MultivariateGaussian.scala | 3 +- .../spark/mllib/tree/GradientBoostedTrees.scala | 2 +- .../org/apache/spark/mllib/tree/RandomForest.scala | 8 +-- .../org/apache/spark/mllib/tree/model/Split.scala | 2 +- .../org/apache/spark/mllib/util/MLUtils.scala | 10 +-- .../apache/spark/mllib/util/modelSaveLoad.scala | 2 +- pom.xml | 12 ++++ project/SparkBuild.scala | 5 +- .../src/main/scala/org/apache/spark/sql/Row.scala | 2 +- .../expressions/aggregate/CentralMomentAgg.scala | 4 +- .../org/apache/spark/sql/types/BinaryType.scala | 2 +- .../org/apache/spark/sql/types/BooleanType.scala | 2 +- .../org/apache/spark/sql/types/ByteType.scala | 2 +- .../spark/sql/types/CalendarIntervalType.scala | 2 +- .../org/apache/spark/sql/types/DateType.scala | 2 +- .../org/apache/spark/sql/types/DecimalType.scala | 4 +- .../org/apache/spark/sql/types/DoubleType.scala | 2 +- .../org/apache/spark/sql/types/FloatType.scala | 2 +- .../org/apache/spark/sql/types/IntegerType.scala | 2 +- .../org/apache/spark/sql/types/LongType.scala | 2 +- .../scala/org/apache/spark/sql/types/MapType.scala | 2 +- .../org/apache/spark/sql/types/NullType.scala | 2 +- .../org/apache/spark/sql/types/ShortType.scala | 2 +- .../org/apache/spark/sql/types/StringType.scala | 2 +- .../org/apache/spark/sql/types/TimestampType.scala | 2 +- .../org/apache/spark/sql/DataFrameReader.scala | 17 ++--- .../apache/spark/sql/DataFrameStatFunctions.scala | 16 +++-- .../org/apache/spark/sql/DataFrameWriter.scala | 4 +- .../scala/org/apache/spark/sql/SQLContext.scala | 62 +++++++++--------- .../spark/sql/execution/stat/FrequentItems.scala | 3 +- .../spark/sql/execution/stat/StatFunctions.scala | 4 +- .../apache/spark/sql/expressions/Aggregator.scala | 8 +-- .../sql/expressions/UserDefinedFunction.scala | 2 +- .../org/apache/spark/sql/expressions/Window.scala | 16 ++--- .../apache/spark/sql/expressions/WindowSpec.scala | 16 ++--- .../spark/sql/expressions/scalalang/typed.scala | 2 +- .../org/apache/spark/sql/expressions/udaf.scala | 24 +++---- .../org/apache/spark/sql/jdbc/JdbcDialects.scala | 6 +- .../spark/sql/streaming/DataStreamReader.scala | 20 +++--- .../spark/sql/streaming/DataStreamWriter.scala | 8 +-- .../spark/sql/streaming/StreamingQuery.scala | 10 +-- .../sql/streaming/StreamingQueryManager.scala | 8 ++- .../spark/sql/util/QueryExecutionListener.scala | 2 +- .../sql/hive/execution/InsertIntoHiveTable.scala | 4 +- .../apache/spark/sql/hive/orc/OrcFileFormat.scala | 4 +- .../spark/sql/hive/orc/OrcFileOperator.scala | 2 +- 132 files changed, 558 insertions(+), 499 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SSLOptions.scala b/core/src/main/scala/org/apache/spark/SSLOptions.scala index be19179b00..5f14102c3c 100644 --- a/core/src/main/scala/org/apache/spark/SSLOptions.scala +++ b/core/src/main/scala/org/apache/spark/SSLOptions.scala @@ -150,8 +150,8 @@ private[spark] object SSLOptions extends Logging { * $ - `[ns].enabledAlgorithms` - a comma separated list of ciphers * * For a list of protocols and ciphers supported by particular Java versions, you may go to - * [[https://blogs.oracle.com/java-platform-group/entry/diagnosing_tls_ssl_and_https Oracle - * blog page]]. + * + * Oracle blog page. * * You can optionally specify the default configuration. If you do, for each setting which is * missing in SparkConf, the corresponding setting is used from the default configuration. diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index bff5a29bb6..d7e3a1b1be 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -405,7 +405,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * partitioning of the resulting key-value pair RDD by passing a Partitioner. * * @note If you are grouping in order to perform an aggregation (such as a sum or average) over - * each key, using [[JavaPairRDD.reduceByKey]] or [[JavaPairRDD.combineByKey]] + * each key, using `JavaPairRDD.reduceByKey` or `JavaPairRDD.combineByKey` * will provide much better performance. */ def groupByKey(partitioner: Partitioner): JavaPairRDD[K, JIterable[V]] = @@ -416,7 +416,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * resulting RDD with into `numPartitions` partitions. * * @note If you are grouping in order to perform an aggregation (such as a sum or average) over - * each key, using [[JavaPairRDD.reduceByKey]] or [[JavaPairRDD.combineByKey]] + * each key, using `JavaPairRDD.reduceByKey` or `JavaPairRDD.combineByKey` * will provide much better performance. */ def groupByKey(numPartitions: Int): JavaPairRDD[K, JIterable[V]] = @@ -546,7 +546,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * resulting RDD with the existing partitioner/parallelism level. * * @note If you are grouping in order to perform an aggregation (such as a sum or average) over - * each key, using [[JavaPairRDD.reduceByKey]] or [[JavaPairRDD.combineByKey]] + * each key, using `JavaPairRDD.reduceByKey` or `JavaPairRDD.combineByKey` * will provide much better performance. */ def groupByKey(): JavaPairRDD[K, JIterable[V]] = 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 ccd94f876e..a20d264be5 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,10 +103,10 @@ 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 >= 0 + * with replacement: expected number of times each element is chosen; fraction must be >= 0 * * @note This is NOT guaranteed to provide exactly the fraction of the count - * of the given [[RDD]]. + * of the given `RDD`. */ def sample(withReplacement: Boolean, fraction: Double): JavaRDD[T] = sample(withReplacement, fraction, Utils.random.nextLong) @@ -117,11 +117,11 @@ 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 >= 0 + * with replacement: expected number of times each element is chosen; fraction must be >= 0 * @param seed seed for the random number generator * * @note This is NOT guaranteed to provide exactly the fraction of the count - * of the given [[RDD]]. + * of the given `RDD`. */ def sample(withReplacement: Boolean, fraction: Double, seed: Long): JavaRDD[T] = wrapRDD(rdd.sample(withReplacement, fraction, seed)) @@ -167,7 +167,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 <= us. + * RDD will be <= us. */ def subtract(other: JavaRDD[T]): JavaRDD[T] = wrapRDD(rdd.subtract(other)) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 38d347aeab..9481156bc9 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -238,7 +238,9 @@ class JavaSparkContext(val sc: SparkContext) * }}} * * Do - * `JavaPairRDD rdd = sparkContext.dataStreamFiles("hdfs://a-hdfs-path")`, + * {{{ + * JavaPairRDD rdd = sparkContext.dataStreamFiles("hdfs://a-hdfs-path") + * }}} * * then `rdd` contains * {{{ @@ -270,7 +272,9 @@ class JavaSparkContext(val sc: SparkContext) * }}} * * Do - * `JavaPairRDD rdd = sparkContext.dataStreamFiles("hdfs://a-hdfs-path")`, + * {{{ + * JavaPairRDD rdd = sparkContext.dataStreamFiles("hdfs://a-hdfs-path") + * }}}, * * then `rdd` contains * {{{ @@ -749,7 +753,7 @@ class JavaSparkContext(val sc: SparkContext) /** * Get a local property set in this thread, or null if it is missing. See - * [[org.apache.spark.api.java.JavaSparkContext.setLocalProperty]]. + * `org.apache.spark.api.java.JavaSparkContext.setLocalProperty`. */ def getLocalProperty(key: String): String = sc.getLocalProperty(key) @@ -769,7 +773,7 @@ class JavaSparkContext(val sc: SparkContext) * 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.api.java.JavaSparkContext.cancelJobGroup]] + * The application can also use `org.apache.spark.api.java.JavaSparkContext.cancelJobGroup` * to cancel all running jobs in this group. For example, * {{{ * // In the main thread: @@ -802,7 +806,7 @@ class JavaSparkContext(val sc: SparkContext) /** * Cancel active jobs for the specified group. See - * [[org.apache.spark.api.java.JavaSparkContext.setJobGroup]] for more information. + * `org.apache.spark.api.java.JavaSparkContext.setJobGroup` for more information. */ def cancelJobGroup(groupId: String): Unit = sc.cancelJobGroup(groupId) diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index 6ba79e506a..2e991ce394 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -172,7 +172,7 @@ private final object SnappyCompressionCodec { } /** - * Wrapper over [[SnappyOutputStream]] which guards against write-after-close and double-close + * Wrapper over `SnappyOutputStream` which guards against write-after-close and double-close * issues. See SPARK-7660 for more details. This wrapping can be removed if we upgrade to a version * of snappy-java that contains the fix for https://github.com/xerial/snappy-java/issues/107. */ 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 bff2b8f1d0..8e67344758 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -70,8 +70,8 @@ import org.apache.spark.util.random.{BernoulliCellSampler, BernoulliSampler, Poi * All of the scheduling and execution in Spark is done based on these methods, allowing each RDD * to implement its own way of computing itself. Indeed, users can implement custom RDDs (e.g. for * reading data from a new storage system) by overriding these functions. Please refer to the - * [[http://people.csail.mit.edu/matei/papers/2012/nsdi_spark.pdf Spark paper]] for more details - * on RDD internals. + * Spark paper + * for more details on RDD internals. */ abstract class RDD[T: ClassTag]( @transient private var _sc: SparkContext, @@ -469,7 +469,7 @@ 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 >= 0 + * with replacement: expected number of times each element is chosen; fraction must be >= 0 * @param seed seed for the random number generator * * @note This is NOT guaranteed to provide exactly the fraction of the count @@ -675,8 +675,8 @@ abstract class RDD[T: ClassTag]( * 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. */ def groupBy[K](f: T => K)(implicit kt: ClassTag[K]): RDD[(K, Iterable[T])] = withScope { groupBy[K](f, defaultPartitioner(this)) @@ -688,8 +688,8 @@ abstract class RDD[T: ClassTag]( * 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. */ def groupBy[K]( f: T => K, @@ -703,8 +703,8 @@ abstract class RDD[T: ClassTag]( * 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. */ def groupBy[K](f: T => K, p: Partitioner)(implicit kt: ClassTag[K], ord: Ordering[K] = null) : RDD[(K, Iterable[T])] = withScope { diff --git a/core/src/main/scala/org/apache/spark/security/CryptoStreamUtils.scala b/core/src/main/scala/org/apache/spark/security/CryptoStreamUtils.scala index 8f15f50bee..f41fc38be2 100644 --- a/core/src/main/scala/org/apache/spark/security/CryptoStreamUtils.scala +++ b/core/src/main/scala/org/apache/spark/security/CryptoStreamUtils.scala @@ -46,7 +46,7 @@ private[spark] object CryptoStreamUtils extends Logging { val COMMONS_CRYPTO_CONF_PREFIX = "commons.crypto." /** - * Helper method to wrap [[OutputStream]] with [[CryptoOutputStream]] for encryption. + * Helper method to wrap `OutputStream` with `CryptoOutputStream` for encryption. */ def createCryptoOutputStream( os: OutputStream, @@ -62,7 +62,7 @@ private[spark] object CryptoStreamUtils extends Logging { } /** - * Helper method to wrap [[InputStream]] with [[CryptoInputStream]] for decryption. + * Helper method to wrap `InputStream` with `CryptoInputStream` for decryption. */ def createCryptoInputStream( is: InputStream, diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 19e020c968..7eb2da1c27 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -43,7 +43,8 @@ import org.apache.spark.util.{BoundedPriorityQueue, SerializableConfiguration, S import org.apache.spark.util.collection.CompactBuffer /** - * A Spark serializer that uses the [[https://code.google.com/p/kryo/ Kryo serialization library]]. + * A Spark serializer that uses the + * Kryo serialization library. * * @note This serializer is not guaranteed to be wire-compatible across different versions of * Spark. It is intended to be used to serialize/de-serialize data within a single diff --git a/core/src/main/scala/org/apache/spark/storage/BlockReplicationPolicy.scala b/core/src/main/scala/org/apache/spark/storage/BlockReplicationPolicy.scala index bf087af16a..bb8a684b4c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockReplicationPolicy.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockReplicationPolicy.scala @@ -89,17 +89,18 @@ class RandomBlockReplicationPolicy prioritizedPeers } + // scalastyle:off line.size.limit /** * Uses sampling algorithm by Robert Floyd. Finds a random sample in O(n) while - * minimizing space usage - * [[http://math.stackexchange.com/questions/178690/ - * whats-the-proof-of-correctness-for-robert-floyds-algorithm-for-selecting-a-sin]] + * minimizing space usage. Please see + * here. * * @param n total number of indices * @param m number of samples needed * @param r random number generator * @return list of m random unique indices */ + // scalastyle:on line.size.limit private def getSampleIds(n: Int, m: Int, r: Random): List[Int] = { val indices = (n - m + 1 to n).foldLeft(Set.empty[Int]) {case (set, i) => val t = r.nextInt(i) + 1 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 57f6f2f0a9..dbeb970c81 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -422,8 +422,8 @@ 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