Skip to content

Commit 898fc34

Browse files
andrewor14pwendell
authored andcommitted
[Fix #274] Document + fix annotation usages
... so that we don't follow an unspoken set of forbidden rules for adding **@AlphaComponent**, **@DeveloperAPI**, and **@experimental** annotations in the code. In addition, this PR (1) removes unnecessary `:: * ::` tags, (2) adds missing `:: * ::` tags, and (3) removes annotations for internal APIs. Author: Andrew Or <andrewor14@gmail.com> Closes #470 from andrewor14/annotations-fix and squashes the following commits: 92a7f42 [Andrew Or] Document + fix annotation usages (cherry picked from commit b3e5366) Signed-off-by: Patrick Wendell <pwendell@gmail.com>
1 parent 61d7401 commit 898fc34

File tree

11 files changed

+29
-19
lines changed

11 files changed

+29
-19
lines changed

core/src/main/scala/org/apache/spark/SparkContext.scala

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1110,6 +1110,7 @@ class SparkContext(config: SparkConf) extends Logging {
11101110
}
11111111

11121112
/**
1113+
* :: Experimental ::
11131114
* Submit a job for execution and return a FutureJob holding the result.
11141115
*/
11151116
@Experimental

core/src/main/scala/org/apache/spark/annotation/AlphaComponent.java

Lines changed: 8 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,14 @@
1919

2020
import java.lang.annotation.*;
2121

22-
/** A new component of Spark which may have unstable API's. */
22+
/**
23+
* A new component of Spark which may have unstable API's.
24+
*
25+
* NOTE: If there exists a Scaladoc comment that immediately precedes this annotation, the first
26+
* line of the comment must be ":: AlphaComponent ::" with no trailing blank line. This is because
27+
* of the known issue that Scaladoc displays only either the annotation or the comment, whichever
28+
* comes first.
29+
*/
2330
@Retention(RetentionPolicy.RUNTIME)
2431
@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER,
2532
ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE})

core/src/main/scala/org/apache/spark/annotation/DeveloperApi.java

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,11 @@
2323
* A lower-level, unstable API intended for developers.
2424
*
2525
* Developer API's might change or be removed in minor versions of Spark.
26+
*
27+
* NOTE: If there exists a Scaladoc comment that immediately precedes this annotation, the first
28+
* line of the comment must be ":: DeveloperApi ::" with no trailing blank line. This is because
29+
* of the known issue that Scaladoc displays only either the annotation or the comment, whichever
30+
* comes first.
2631
*/
2732
@Retention(RetentionPolicy.RUNTIME)
2833
@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER,

core/src/main/scala/org/apache/spark/annotation/Experimental.java

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,11 @@
2424
*
2525
* Experimental API's might change or be removed in minor versions of Spark, or be adopted as
2626
* first-class Spark API's.
27+
*
28+
* NOTE: If there exists a Scaladoc comment that immediately precedes this annotation, the first
29+
* line of the comment must be ":: Experimental ::" with no trailing blank line. This is because
30+
* of the known issue that Scaladoc displays only either the annotation or the comment, whichever
31+
* comes first.
2732
*/
2833
@Retention(RetentionPolicy.RUNTIME)
2934
@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER,

core/src/main/scala/org/apache/spark/rdd/RDD.scala

Lines changed: 0 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -94,26 +94,20 @@ abstract class RDD[T: ClassTag](
9494
def compute(split: Partition, context: TaskContext): Iterator[T]
9595

9696
/**
97-
* :: DeveloperApi ::
9897
* Implemented by subclasses to return the set of partitions in this RDD. This method will only
9998
* be called once, so it is safe to implement a time-consuming computation in it.
10099
*/
101-
@DeveloperApi
102100
protected def getPartitions: Array[Partition]
103101

104102
/**
105-
* :: DeveloperApi ::
106103
* Implemented by subclasses to return how this RDD depends on parent RDDs. This method will only
107104
* be called once, so it is safe to implement a time-consuming computation in it.
108105
*/
109-
@DeveloperApi
110106
protected def getDependencies: Seq[Dependency[_]] = deps
111107

112108
/**
113-
* :: DeveloperApi ::
114109
* Optionally overridden by subclasses to specify placement preferences.
115110
*/
116-
@DeveloperApi
117111
protected def getPreferredLocations(split: Partition): Seq[String] = Nil
118112

119113
/** Optionally overridden by subclasses to specify how they are partitioned. */

core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -24,8 +24,12 @@ import org.apache.spark.annotation.DeveloperApi
2424
// information about a specific split instance : handles both split instances.
2525
// So that we do not need to worry about the differences.
2626
@DeveloperApi
27-
class SplitInfo(val inputFormatClazz: Class[_], val hostLocation: String, val path: String,
28-
val length: Long, val underlyingSplit: Any) {
27+
class SplitInfo(
28+
val inputFormatClazz: Class[_],
29+
val hostLocation: String,
30+
val path: String,
31+
val length: Long,
32+
val underlyingSplit: Any) {
2933
override def toString(): String = {
3034
"SplitInfo " + super.toString + " .. inputFormatClazz " + inputFormatClazz +
3135
", hostLocation : " + hostLocation + ", path : " + path +

mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -391,9 +391,9 @@ object KMeans {
391391
* Returns the squared Euclidean distance between two vectors computed by
392392
* [[org.apache.spark.mllib.util.MLUtils#fastSquaredDistance]].
393393
*/
394-
private[clustering]
395-
def fastSquaredDistance(v1: BreezeVectorWithNorm, v2: BreezeVectorWithNorm)
396-
: Double = {
394+
private[clustering] def fastSquaredDistance(
395+
v1: BreezeVectorWithNorm,
396+
v2: BreezeVectorWithNorm): Double = {
397397
MLUtils.fastSquaredDistance(v1.vector, v1.norm, v2.vector, v2.norm)
398398
}
399399

mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala

Lines changed: 0 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -21,9 +21,6 @@ import org.apache.spark.rdd.RDD
2121
import org.apache.spark.mllib.linalg.Vector
2222
import org.apache.spark.annotation.Experimental
2323

24-
/**
25-
* :: Experimental ::
26-
*/
2724
@Experimental
2825
trait RegressionModel extends Serializable {
2926
/**

sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -249,11 +249,9 @@ class SQLContext(@transient val sparkContext: SparkContext)
249249
}
250250

251251
/**
252-
* :: DeveloperApi ::
253252
* The primary workflow for executing relational queries using Spark. Designed to allow easy
254253
* access to the intermediate phases of query execution for developers.
255254
*/
256-
@DeveloperApi
257255
protected abstract class QueryExecution {
258256
def logical: LogicalPlan
259257

sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,7 @@ package org.apache.spark.sql
2020
import net.razorvine.pickle.Pickler
2121

2222
import org.apache.spark.{Dependency, OneToOneDependency, Partition, TaskContext}
23-
import org.apache.spark.annotation.{AlphaComponent, Experimental, DeveloperApi}
23+
import org.apache.spark.annotation.{AlphaComponent, Experimental}
2424
import org.apache.spark.rdd.RDD
2525
import org.apache.spark.sql.catalyst.analysis._
2626
import org.apache.spark.sql.catalyst.expressions._

sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -239,7 +239,6 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
239239
sparkContext.parallelize(Seq(new GenericRow(Array[Any]()): Row), 1)
240240

241241
/** Extends QueryExecution with hive specific features. */
242-
@DeveloperApi
243242
protected[sql] abstract class QueryExecution extends super.QueryExecution {
244243
// TODO: Create mixin for the analyzer instead of overriding things here.
245244
override lazy val optimizedPlan =

0 commit comments

Comments
 (0)