Skip to content

Commit 99192ef

Browse files
committed
Dynamically add badges based on annotations
This has known problems due to bugs in scaladoc. In particular, on the packages page, annotations appear only if there are no comments for the element. As soon as this is fixed (or there is a workaround) this should basically be ready.
1 parent 824011b commit 99192ef

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

42 files changed

+192
-101
lines changed

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

Lines changed: 6 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -17,21 +17,22 @@
1717

1818
package org.apache.spark
1919

20+
import org.apache.spark.annotations.DeveloperAPI
2021
import org.apache.spark.rdd.RDD
2122
import org.apache.spark.serializer.Serializer
2223

2324
/**
24-
* <span class="developer badge">Developer API</span>
2525
* Base class for dependencies.
2626
*/
27+
@DeveloperAPI
2728
abstract class Dependency[T](val rdd: RDD[T]) extends Serializable
2829

2930

3031
/**
31-
* <span class="developer badge">Developer API</span>
3232
* Base class for dependencies where each partition of the parent RDD is used by at most one
3333
* partition of the child RDD. Narrow dependencies allow for pipelined execution.
3434
*/
35+
@DeveloperAPI
3536
abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) {
3637
/**
3738
* Get the parent partitions for a child partition.
@@ -43,14 +44,14 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) {
4344

4445

4546
/**
46-
* <span class="developer badge">Developer API</span>
4747
* Represents a dependency on the output of a shuffle stage.
4848
* @param rdd the parent RDD
4949
* @param partitioner partitioner used to partition the shuffle output
5050
* @param serializer [[org.apache.spark.serializer.Serializer Serializer]] to use. If set to null,
5151
* the default serializer, as specified by `spark.serializer` config option, will
5252
* be used.
5353
*/
54+
@DeveloperAPI
5455
class ShuffleDependency[K, V](
5556
@transient rdd: RDD[_ <: Product2[K, V]],
5657
val partitioner: Partitioner,
@@ -62,22 +63,22 @@ class ShuffleDependency[K, V](
6263

6364

6465
/**
65-
* <span class="developer badge">Developer API</span>
6666
* Represents a one-to-one dependency between partitions of the parent and child RDDs.
6767
*/
68+
@DeveloperAPI
6869
class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) {
6970
override def getParents(partitionId: Int) = List(partitionId)
7071
}
7172

7273

7374
/**
74-
* <span class="developer badge">Developer API</span>
7575
* Represents a one-to-one dependency between ranges of partitions in the parent and child RDDs.
7676
* @param rdd the parent RDD
7777
* @param inStart the start of the range in the parent RDD
7878
* @param outStart the start of the range in the child RDD
7979
* @param length the length of the range
8080
*/
81+
@DeveloperAPI
8182
class RangeDependency[T](rdd: RDD[T], inStart: Int, outStart: Int, length: Int)
8283
extends NarrowDependency[T](rdd) {
8384

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

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -21,14 +21,15 @@ import scala.concurrent._
2121
import scala.concurrent.duration.Duration
2222
import scala.util.Try
2323

24+
import org.apache.spark.annotations.Experimental
2425
import org.apache.spark.rdd.RDD
2526
import org.apache.spark.scheduler.{JobFailed, JobSucceeded, JobWaiter}
2627

2728
/**
28-
* <span class="experimental badge">Experimental</span>
2929
* A future for the result of an action to support cancellation. This is an extension of the
3030
* Scala Future interface to support cancellation.
3131
*/
32+
@Experimental
3233
trait FutureAction[T] extends Future[T] {
3334
// Note that we redefine methods of the Future trait here explicitly so we can specify a different
3435
// documentation (with reference to the word "action").
@@ -85,10 +86,10 @@ trait FutureAction[T] extends Future[T] {
8586

8687

8788
/**
88-
* <span class="experimental badge">Experimental</span>
8989
* A [[FutureAction]] holding the result of an action that triggers a single job. Examples include
9090
* count, collect, reduce.
9191
*/
92+
@Experimental
9293
class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: => T)
9394
extends FutureAction[T] {
9495

@@ -150,11 +151,11 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc:
150151

151152

152153
/**
153-
* <span class="experimental badge">Experimental</span>
154154
* A [[FutureAction]] for actions that could trigger multiple Spark jobs. Examples include take,
155155
* takeSample. Cancellation works by setting the cancelled flag to true and interrupting the
156156
* action thread if it is being blocked by a job.
157157
*/
158+
@Experimental
158159
class ComplexFutureAction[T] extends FutureAction[T] {
159160

160161
// Pointer to the thread that is executing the action. It is set when the action is run.

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

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -21,15 +21,17 @@ import org.apache.log4j.{LogManager, PropertyConfigurator}
2121
import org.slf4j.{Logger, LoggerFactory}
2222
import org.slf4j.impl.StaticLoggerBinder
2323

24+
import org.apache.spark.annotations.DeveloperAPI
25+
2426
/**
25-
* <span class="developer badge">Developer API</span>
2627
* Utility trait for classes that want to log data. Creates a SLF4J logger for the class and allows
2728
* logging messages at different levels using methods that only evaluate parameters lazily if the
2829
* log level is enabled.
2930
*
3031
* NOTE: DO NOT USE this class outside of Spark. It is intended as an internal utility.
3132
* This will likely be changed or removed in future releases.
3233
*/
34+
@DeveloperAPI
3335
trait Logging {
3436
// Make the log field transient so that objects with Logging can
3537
// be serialized and used on another machine

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

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,9 @@ import org.apache.hadoop.conf.Configuration
2323
import org.apache.hadoop.io.ObjectWritable
2424
import org.apache.hadoop.io.Writable
2525

26-
/** <span class="developer badge">Developer API</span> */
26+
import org.apache.spark.annotations.DeveloperAPI
27+
28+
@DeveloperAPI
2729
class SerializableWritable[T <: Writable](@transient var t: T) extends Serializable {
2830
def value = t
2931
override def toString = t.toString

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

Lines changed: 7 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -35,6 +35,7 @@ import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHad
3535
import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat}
3636
import org.apache.mesos.MesosNativeLibrary
3737

38+
import org.apache.spark.annotations.{DeveloperAPI, Experimental}
3839
import org.apache.spark.broadcast.Broadcast
3940
import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil}
4041
import org.apache.spark.partial.{ApproximateEvaluator, PartialResult}
@@ -54,6 +55,8 @@ import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerTy
5455
* @param config a Spark Config object describing the application configuration. Any settings in
5556
* this config overrides the default configs as well as system properties.
5657
*/
58+
59+
@DeveloperAPI
5760
class SparkContext(config: SparkConf) extends Logging {
5861

5962
// This is used only by YARN for now, but should be relevant to other cluster types (Mesos,
@@ -62,13 +65,13 @@ class SparkContext(config: SparkConf) extends Logging {
6265
private[spark] var preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map()
6366

6467
/**
65-
* <span class="developer badge">Developer API</span>
6668
* Alternative constructor for setting preferred locations where Spark will create executors.
6769
*
6870
* @param preferredNodeLocationData used in YARN mode to select nodes to launch containers on. Ca
6971
* be generated using [[org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations]]
7072
* from a list of input files or InputFormats for the application.
7173
*/
74+
@DeveloperAPI
7275
def this(config: SparkConf, preferredNodeLocationData: Map[String, Set[SplitInfo]]) = {
7376
this(config)
7477
this.preferredNodeLocationData = preferredNodeLocationData
@@ -713,9 +716,9 @@ class SparkContext(config: SparkConf) extends Logging {
713716
}
714717

715718
/**
716-
* <span class="developer badge">Developer API</span>
717719
* Register a listener to receive up-calls from events that happen during execution.
718720
*/
721+
@DeveloperAPI
719722
def addSparkListener(listener: SparkListener) {
720723
listenerBus.addListener(listener)
721724
}
@@ -1025,9 +1028,9 @@ class SparkContext(config: SparkConf) extends Logging {
10251028
}
10261029

10271030
/**
1028-
* <span class="developer badge">Developer API</span>
10291031
* Run a job that can return approximate results.
10301032
*/
1033+
@DeveloperAPI
10311034
def runApproximateJob[T, U, R](
10321035
rdd: RDD[T],
10331036
func: (TaskContext, Iterator[T]) => U,
@@ -1043,9 +1046,9 @@ class SparkContext(config: SparkConf) extends Logging {
10431046
}
10441047

10451048
/**
1046-
* <span class="experimental badge">Experimental</span>
10471049
* Submit a job for execution and return a FutureJob holding the result.
10481050
*/
1051+
@Experimental
10491052
def submitJob[T, U, R](
10501053
rdd: RDD[T],
10511054
processPartition: Iterator[T] => U,

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

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@ import scala.util.Properties
2525
import akka.actor._
2626
import com.google.common.collect.MapMaker
2727

28+
import org.apache.spark.annotations.DeveloperAPI
2829
import org.apache.spark.api.python.PythonWorkerFactory
2930
import org.apache.spark.broadcast.BroadcastManager
3031
import org.apache.spark.metrics.MetricsSystem
@@ -35,13 +36,13 @@ import org.apache.spark.storage._
3536
import org.apache.spark.util.{AkkaUtils, Utils}
3637

3738
/**
38-
* <span class="developer badge">Developer API</span>
3939
* Holds all the runtime environment objects for a running Spark instance (either master or worker),
4040
* including the serializer, Akka actor system, block manager, map output tracker, etc. Currently
4141
* Spark code finds the SparkEnv through a thread-local variable, so each thread that accesses these
4242
* objects needs to have the right SparkEnv set. You can get the current environment with
4343
* SparkEnv.get (e.g. after creating a SparkContext) and set it with SparkEnv.set.
4444
*/
45+
@DeveloperAPI
4546
class SparkEnv (
4647
val executorId: String,
4748
val actorSystem: ActorSystem,

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

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -19,12 +19,13 @@ package org.apache.spark
1919

2020
import scala.collection.mutable.ArrayBuffer
2121

22+
import org.apache.spark.annotations.DeveloperAPI
2223
import org.apache.spark.executor.TaskMetrics
2324

2425
/**
25-
* <span class="developer badge">Developer API</span>
2626
* Contextual information about a task which can be read or mutated during execution.
2727
*/
28+
@DeveloperAPI
2829
class TaskContext(
2930
val stageId: Int,
3031
val partitionId: Int,

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

Lines changed: 8 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -29,21 +29,21 @@ import org.apache.spark.storage.BlockManagerId
2929
@DeveloperAPI
3030
sealed trait TaskEndReason
3131

32-
/** <span class="developer badge">Developer API</span> */
32+
@DeveloperAPI
3333
case object Success extends TaskEndReason
3434

35-
/** <span class="developer badge">Developer API</span> */
35+
@DeveloperAPI
3636
case object Resubmitted extends TaskEndReason // Task was finished earlier but we've now lost it
3737

38-
/** <span class="developer badge">Developer API</span> */
38+
@DeveloperAPI
3939
case class FetchFailed(
4040
bmAddress: BlockManagerId,
4141
shuffleId: Int,
4242
mapId: Int,
4343
reduceId: Int)
4444
extends TaskEndReason
4545

46-
/** <span class="developer badge">Developer API</span> */
46+
@DeveloperAPI
4747
case class ExceptionFailure(
4848
className: String,
4949
description: String,
@@ -52,25 +52,25 @@ case class ExceptionFailure(
5252
extends TaskEndReason
5353

5454
/**
55-
* <span class="developer badge">Developer API</span>
5655
* The task finished successfully, but the result was lost from the executor's block manager before
5756
* it was fetched.
5857
*/
58+
@DeveloperAPI
5959
case object TaskResultLost extends TaskEndReason
6060

61-
/** <span class="developer badge">Developer API</span> */
61+
@DeveloperAPI
6262
case object TaskKilled extends TaskEndReason
6363

6464
/**
65-
* <span class="developer badge">Developer API</span>
6665
* The task failed because the executor that it was running on was lost. This may happen because
6766
* the task crashed the JVM.
6867
*/
68+
@DeveloperAPI
6969
case object ExecutorLostFailure extends TaskEndReason
7070

7171
/**
72-
* <span class="developer badge">Developer API</span>
7372
* We don't know why the task ended -- for example, because of a ClassNotFound exception when
7473
* deserializing the task result.
7574
*/
75+
@DeveloperAPI
7676
case object UnknownReason extends TaskEndReason
Lines changed: 25 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,25 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.annotations;
19+
20+
import java.lang.annotation.*;
21+
22+
@Retention(RetentionPolicy.SOURCE)
23+
@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD,
24+
ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE})
25+
public @interface AlphaComponent {}

core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -16,17 +16,18 @@
1616
*/
1717

1818
package org.apache.spark.broadcast
19-
import org.apache.spark.SecurityManager
2019

20+
import org.apache.spark.SecurityManager
2121
import org.apache.spark.SparkConf
22+
import org.apache.spark.annotations.DeveloperAPI
2223

2324
/**
24-
* <span class="developer badge">Developer API</span>
2525
* An interface for all the broadcast implementations in Spark (to allow
2626
* multiple broadcast implementations). SparkContext uses a user-specified
2727
* BroadcastFactory implementation to instantiate a particular broadcast for the
2828
* entire Spark job.
2929
*/
30+
@DeveloperAPI
3031
trait BroadcastFactory {
3132
def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager): Unit
3233
def newBroadcast[T](value: T, isLocal: Boolean, id: Long): Broadcast[T]

0 commit comments

Comments
 (0)