diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala
index 43ab09004fabf..5bc9b880f4e94 100644
--- a/core/src/main/scala/org/apache/spark/Aggregator.scala
+++ b/core/src/main/scala/org/apache/spark/Aggregator.scala
@@ -21,6 +21,7 @@ import org.apache.spark.annotations.DeveloperAPI
import org.apache.spark.util.collection.{AppendOnlyMap, ExternalAppendOnlyMap}
/**
+ * :: DeveloperAPI ::
* A set of functions used to aggregate data.
*
* @param createCombiner function to create the initial value of the aggregation.
diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala
index a37f4691dcbd0..73d1c40d523e6 100644
--- a/core/src/main/scala/org/apache/spark/Dependency.scala
+++ b/core/src/main/scala/org/apache/spark/Dependency.scala
@@ -22,6 +22,7 @@ import org.apache.spark.rdd.RDD
import org.apache.spark.serializer.Serializer
/**
+ * :: DeveloperAPI ::
* Base class for dependencies.
*/
@DeveloperAPI
@@ -29,6 +30,7 @@ abstract class Dependency[T](val rdd: RDD[T]) extends Serializable
/**
+ * :: DeveloperAPI ::
* Base class for dependencies where each partition of the parent RDD is used by at most one
* partition of the child RDD. Narrow dependencies allow for pipelined execution.
*/
@@ -44,6 +46,7 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) {
/**
+ * :: DeveloperAPI ::
* Represents a dependency on the output of a shuffle stage.
* @param rdd the parent RDD
* @param partitioner partitioner used to partition the shuffle output
@@ -63,6 +66,7 @@ class ShuffleDependency[K, V](
/**
+ * :: DeveloperAPI ::
* Represents a one-to-one dependency between partitions of the parent and child RDDs.
*/
@DeveloperAPI
@@ -72,6 +76,7 @@ class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) {
/**
+ * :: DeveloperAPI ::
* Represents a one-to-one dependency between ranges of partitions in the parent and child RDDs.
* @param rdd the parent RDD
* @param inStart the start of the range in the parent RDD
diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala
index e71f0e6c5d276..07c499f49d2b1 100644
--- a/core/src/main/scala/org/apache/spark/FutureAction.scala
+++ b/core/src/main/scala/org/apache/spark/FutureAction.scala
@@ -26,6 +26,7 @@ import org.apache.spark.rdd.RDD
import org.apache.spark.scheduler.{JobFailed, JobSucceeded, JobWaiter}
/**
+ * :: Experimental ::
* A future for the result of an action to support cancellation. This is an extension of the
* Scala Future interface to support cancellation.
*/
@@ -86,6 +87,7 @@ trait FutureAction[T] extends Future[T] {
/**
+ * :: Experimental ::
* A [[FutureAction]] holding the result of an action that triggers a single job. Examples include
* count, collect, reduce.
*/
@@ -151,6 +153,7 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc:
/**
+ * :: Experimental ::
* A [[FutureAction]] for actions that could trigger multiple Spark jobs. Examples include take,
* takeSample. Cancellation works by setting the cancelled flag to true and interrupting the
* action thread if it is being blocked by a job.
diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala
index ef6f44c9075e8..2c6b62dcc09e0 100644
--- a/core/src/main/scala/org/apache/spark/Logging.scala
+++ b/core/src/main/scala/org/apache/spark/Logging.scala
@@ -24,6 +24,7 @@ import org.slf4j.impl.StaticLoggerBinder
import org.apache.spark.annotations.DeveloperAPI
/**
+ * :: DeveloperAPI ::
* Utility trait for classes that want to log data. Creates a SLF4J logger for the class and allows
* logging messages at different levels using methods that only evaluate parameters lazily if the
* log level is enabled.
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 8e6a68a18a267..d9aa8e614eb9b 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -49,6 +49,7 @@ import org.apache.spark.ui.SparkUI
import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedHashMap, Utils}
/**
+ * :: DeveloperAPI ::
* Main entry point for Spark functionality. A SparkContext represents the connection to a Spark
* cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster.
*
@@ -65,6 +66,7 @@ class SparkContext(config: SparkConf) extends Logging {
private[spark] var preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map()
/**
+ * :: DeveloperAPI ::
* Alternative constructor for setting preferred locations where Spark will create executors.
*
* @param preferredNodeLocationData used in YARN mode to select nodes to launch containers on. Ca
@@ -716,6 +718,7 @@ class SparkContext(config: SparkConf) extends Logging {
}
/**
+ * :: DeveloperAPI ::
* Register a listener to receive up-calls from events that happen during execution.
*/
@DeveloperAPI
@@ -1028,6 +1031,7 @@ class SparkContext(config: SparkConf) extends Logging {
}
/**
+ * :: DeveloperAPI ::
* Run a job that can return approximate results.
*/
@DeveloperAPI
@@ -1046,6 +1050,7 @@ class SparkContext(config: SparkConf) extends Logging {
}
/**
+ *
* Submit a job for execution and return a FutureJob holding the result.
*/
@Experimental
diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala
index 2850b19cd1503..5f63b6b5bd4f0 100644
--- a/core/src/main/scala/org/apache/spark/SparkEnv.scala
+++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala
@@ -36,6 +36,7 @@ import org.apache.spark.storage._
import org.apache.spark.util.{AkkaUtils, Utils}
/**
+ * :: DeveloperAPI ::
* Holds all the runtime environment objects for a running Spark instance (either master or worker),
* including the serializer, Akka actor system, block manager, map output tracker, etc. Currently
* Spark code finds the SparkEnv through a thread-local variable, so each thread that accesses these
diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala
index 28ef01f4fe6fd..58e39abd6e0bd 100644
--- a/core/src/main/scala/org/apache/spark/TaskContext.scala
+++ b/core/src/main/scala/org/apache/spark/TaskContext.scala
@@ -23,6 +23,7 @@ import org.apache.spark.annotations.DeveloperAPI
import org.apache.spark.executor.TaskMetrics
/**
+ * :: DeveloperAPI ::
* Contextual information about a task which can be read or mutated during execution.
*/
@DeveloperAPI
diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala
index 160739bdfc6a9..cd73508711a1e 100644
--- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala
+++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala
@@ -22,6 +22,7 @@ import org.apache.spark.executor.TaskMetrics
import org.apache.spark.storage.BlockManagerId
/**
+ * :: DeveloperAPI ::
* Various possible reasons why a task ended. The low-level TaskScheduler is supposed to retry
* tasks several times for "ephemeral" failures, and only report back failures that require some
* old stages to be resubmitted, such as shuffle map fetch failures.
@@ -52,6 +53,7 @@ case class ExceptionFailure(
extends TaskEndReason
/**
+ * :: DeveloperAPI ::
* The task finished successfully, but the result was lost from the executor's block manager before
* it was fetched.
*/
@@ -62,6 +64,7 @@ case object TaskResultLost extends TaskEndReason
case object TaskKilled extends TaskEndReason
/**
+ * :: DeveloperAPI ::
* The task failed because the executor that it was running on was lost. This may happen because
* the task crashed the JVM.
*/
@@ -69,6 +72,7 @@ case object TaskKilled extends TaskEndReason
case object ExecutorLostFailure extends TaskEndReason
/**
+ * :: DeveloperAPI ::
* We don't know why the task ended -- for example, because of a ClassNotFound exception when
* deserializing the task result.
*/
diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala
index d28860122b2da..67faa217c46bf 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala
@@ -22,7 +22,8 @@ import org.apache.spark.SparkConf
import org.apache.spark.annotations.DeveloperAPI
/**
- * An interface for all the broadcast implementations in Spark (to allow
+ * :: DeveloperAPI ::
+ * An interface for all the broadcast implementations in Spark (to allow
* multiple broadcast implementations). SparkContext uses a user-specified
* BroadcastFactory implementation to instantiate a particular broadcast for the
* entire Spark job.
diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala
index b7fe0c3daec8e..ed1b45f5d485b 100644
--- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala
+++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala
@@ -21,6 +21,7 @@ import org.apache.spark.annotations.DeveloperAPI
import org.apache.spark.storage.{BlockId, BlockStatus}
/**
+ * :: DeveloperAPI ::
* Metrics tracked during the execution of a task.
*/
@DeveloperAPI
@@ -88,6 +89,7 @@ private[spark] object TaskMetrics {
/**
+ * :: DeveloperAPI ::
* Metrics pertaining to shuffle data read in a given task.
*/
@DeveloperAPI
@@ -126,6 +128,7 @@ class ShuffleReadMetrics extends Serializable {
}
/**
+ * :: DeveloperAPI ::
* Metrics pertaining to shuffle data written in a given task.
*/
@DeveloperAPI
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 3da1551643820..1f0134d0cdf3e 100644
--- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
+++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
@@ -26,6 +26,7 @@ import org.apache.spark.SparkConf
import org.apache.spark.annotations.DeveloperAPI
/**
+ * :: DeveloperAPI ::
* CompressionCodec allows the customization of choosing different compression implementations
* to be used in block storage.
*
@@ -58,6 +59,7 @@ private[spark] object CompressionCodec {
/**
+ * :: DeveloperAPI ::
* LZF implementation of [[org.apache.spark.io.CompressionCodec]].
*
* Note: The wire protocol for this codec is not guaranteed to be compatible across versions
@@ -76,6 +78,7 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec {
/**
+ * :: DeveloperAPI ::
* Snappy implementation of [[org.apache.spark.io.CompressionCodec]].
* Block size can be configured by spark.io.compression.snappy.block.size.
*
diff --git a/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala
index dcd13554c84e1..ad11fccaf20a2 100644
--- a/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala
+++ b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala
@@ -20,6 +20,7 @@ package org.apache.spark.partial
import org.apache.spark.annotations.Experimental
/**
+ * :: Experimental ::
* A Double value with error bars and associated confidence.
*/
@Experimental
diff --git a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala
index 594d7d2e029df..85ea8f42b39ee 100644
--- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala
@@ -27,6 +27,7 @@ import org.apache.spark.{ComplexFutureAction, FutureAction, Logging}
import org.apache.spark.annotations.Experimental
/**
+ * :: Experimental ::
* A set of asynchronous RDD actions available through an implicit conversion.
* Import `org.apache.spark.SparkContext._` at the top of your program to use these functions.
*/
diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
index b3c42d014c0df..f8c3cda91781c 100644
--- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
@@ -52,6 +52,7 @@ private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep]
}
/**
+ * :: DeveloperAPI ::
* A RDD that cogroups its parents. For each key k in parent RDDs, the resulting RDD contains a
* tuple with the list of values for that key.
*
diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
index c1beb2aa4116d..ea03b3b8e4861 100644
--- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
@@ -71,6 +71,7 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp
}
/**
+ * :: DeveloperAPI ::
* An RDD that provides core functionality for reading data stored in Hadoop (e.g., files in HDFS,
* sources in HBase, or S3), using the older MapReduce API (`org.apache.hadoop.mapred`).
*
diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
index 944bb2e1f9305..510f0cfbd8be2 100644
--- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
@@ -37,6 +37,7 @@ class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputS
}
/**
+ * :: DeveloperAPI ::
* An RDD that provides core functionality for reading data stored in Hadoop (e.g., files in HDFS,
* sources in HBase, or S3), using the new MapReduce API (`org.apache.hadoop.mapreduce`).
*
diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala
index 3cbead2e183af..160236f943715 100644
--- a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala
@@ -47,6 +47,7 @@ private[spark] class PruneDependency[T](rdd: RDD[T], @transient partitionFilterF
/**
+ * :: DeveloperAPI ::
* A RDD used to prune RDD partitions/partitions so we can avoid launching tasks on
* all partitions. An example use case: If we know the RDD is partitioned by range,
* and the execution DAG has a filter on the key, we can avoid launching tasks
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 3373b81356745..c56b67ef9b643 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
@@ -88,12 +88,14 @@ abstract class RDD[T: ClassTag](
// =======================================================================
/**
+ * :: DeveloperAPI ::
* Implemented by subclasses to compute a given partition.
*/
@DeveloperAPI
def compute(split: Partition, context: TaskContext): Iterator[T]
/**
+ * :: DeveloperAPI ::
* Implemented by subclasses to return the set of partitions in this RDD. This method will only
* be called once, so it is safe to implement a time-consuming computation in it.
*/
@@ -101,6 +103,7 @@ abstract class RDD[T: ClassTag](
protected def getPartitions: Array[Partition]
/**
+ * :: DeveloperAPI ::
* Implemented by subclasses to return how this RDD depends on parent RDDs. This method will only
* be called once, so it is safe to implement a time-consuming computation in it.
*/
@@ -108,6 +111,7 @@ abstract class RDD[T: ClassTag](
protected def getDependencies: Seq[Dependency[_]] = deps
/**
+ * :: DeveloperAPI ::
* Optionally overridden by subclasses to specify placement preferences.
*/
@DeveloperAPI
@@ -522,6 +526,7 @@ abstract class RDD[T: ClassTag](
}
/**
+ * :: DeveloperAPI ::
* Return a new RDD by applying a function to each partition of this RDD. This is a variant of
* mapPartitions that also passes the TaskContext into the closure.
*/
@@ -785,6 +790,7 @@ abstract class RDD[T: ClassTag](
def count(): Long = sc.runJob(this, Utils.getIteratorSize _).sum
/**
+ * :: Experimental ::
* Approximate version of count() that returns a potentially incomplete result
* within a timeout, even if not all tasks have finished.
*/
@@ -832,6 +838,7 @@ abstract class RDD[T: ClassTag](
}
/**
+ * :: Experimental ::
* Approximate version of countByValue().
*/
@Experimental
@@ -855,6 +862,7 @@ abstract class RDD[T: ClassTag](
}
/**
+ * :: Experimental ::
* Return approximate number of distinct elements in the RDD.
*
* The accuracy of approximation can be controlled through the relative standard deviation
diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
index b544e8b4b616e..e9971cdabdba6 100644
--- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
@@ -29,6 +29,7 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition {
}
/**
+ * :: DeveloperAPI ::
* The resulting RDD from a shuffle (e.g. repartitioning of data).
* @param prev the parent RDD.
* @param part the partitioner used to partition the RDD
diff --git a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala
index 126dcbe7ed67b..6e02ea4ac7d7d 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala
@@ -31,6 +31,7 @@ import org.apache.spark.annotations.DeveloperAPI
import org.apache.spark.deploy.SparkHadoopUtil
/**
+ * :: DeveloperAPI ::
* Parses and holds information about inputFormat (and files) specified as a parameter.
*/
@DeveloperAPI
diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala
index 4a9f409868e94..fd458734ab09e 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala
@@ -29,6 +29,7 @@ import org.apache.spark.annotations.DeveloperAPI
import org.apache.spark.executor.TaskMetrics
/**
+ * :: DeveloperAPI ::
* A logger class to record runtime information for jobs in Spark. This class outputs one log file
* for each Spark job, containing tasks start/stop and shuffle information. JobLogger is a subclass
* of SparkListener, use addSparkListener to add JobLogger to a SparkContext after the SparkContext
@@ -39,7 +40,6 @@ import org.apache.spark.executor.TaskMetrics
* to log application information as SparkListenerEvents. To enable this functionality, set
* spark.eventLog.enabled to true.
*/
-
@DeveloperAPI
@deprecated("Log application information by setting spark.eventLog.enabled.", "1.0.0")
class JobLogger(val user: String, val logDirName: String) extends SparkListener with Logging {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala
index 31543464bdf1c..d8c443f6130ae 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala
@@ -20,6 +20,7 @@ package org.apache.spark.scheduler
import org.apache.spark.annotations.DeveloperAPI
/**
+ * :: DeveloperAPI ::
* A result of a job in the DAGScheduler.
*/
@DeveloperAPI
diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
index 657727d95e395..72ff68f97e0ab 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
@@ -80,6 +80,7 @@ private[spark] case object SparkListenerShutdown extends SparkListenerEvent
/**
+ * :: DeveloperAPI ::
* Interface for listening to events from the Spark scheduler. Note that this is an internal
* interface which might change in different Spark releases.
*/
@@ -143,6 +144,7 @@ trait SparkListener {
}
/**
+ * :: DeveloperAPI ::
* Simple SparkListener that logs a few summary statistics when each stage completes
*/
@DeveloperAPI
diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
index ecca565628cae..460f4109d536b 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
@@ -21,6 +21,7 @@ import org.apache.spark.annotations.DeveloperAPI
import org.apache.spark.storage.RDDInfo
/**
+ * :: DeveloperAPI ::
* Stores information about a stage to pass from the scheduler to SparkListeners.
*/
@DeveloperAPI
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala
index f7ca08bc4603a..04f891b6e72ed 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala
@@ -20,6 +20,7 @@ package org.apache.spark.scheduler
import org.apache.spark.annotations.DeveloperAPI
/**
+ * :: DeveloperAPI ::
* Information about a running task attempt inside a TaskSet.
*/
@DeveloperAPI
diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala
index 8c818f98a0cda..9880bd916a9bc 100644
--- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala
@@ -95,6 +95,7 @@ private[spark] class JavaSerializerInstance(counterReset: Int) extends Serialize
}
/**
+ * :: DeveloperAPI ::
* A Spark serializer that uses Java's built-in serialization.
*
* Note that this serializer is not guaranteed to be wire-compatible across different versions of
diff --git a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala
index 559c223d4e175..cd19e45132ad4 100644
--- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala
@@ -27,6 +27,7 @@ import org.apache.spark.annotations.DeveloperAPI
import org.apache.spark.util.{ByteBufferInputStream, NextIterator}
/**
+ * :: DeveloperAPI ::
* A serializer. Because some serialization libraries are not thread safe, this class is used to
* create [[org.apache.spark.serializer.SerializerInstance]] objects that do the actual
* serialization and are guaranteed to only be called from one thread at a time.
@@ -55,6 +56,7 @@ object Serializer {
/**
+ * :: DeveloperAPI ::
* An instance of a serializer, for use by one thread at a time.
*/
@DeveloperAPI
@@ -88,6 +90,7 @@ trait SerializerInstance {
/**
+ * :: DeveloperAPI ::
* A stream for writing serialized objects.
*/
@DeveloperAPI
@@ -106,6 +109,7 @@ trait SerializationStream {
/**
+ * :: DeveloperAPI ::
* A stream for reading serialized objects.
*/
@DeveloperAPI
diff --git a/core/src/main/scala/org/apache/spark/util/MutablePair.scala b/core/src/main/scala/org/apache/spark/util/MutablePair.scala
index da8b1c0f5a37e..2b62e9e4e25e8 100644
--- a/core/src/main/scala/org/apache/spark/util/MutablePair.scala
+++ b/core/src/main/scala/org/apache/spark/util/MutablePair.scala
@@ -20,6 +20,7 @@ package org.apache.spark.util
import org.apache.spark.annotations.DeveloperAPI
/**
+ * :: DeveloperAPI ::
* A tuple of 2 elements. This can be used as an alternative to Scala's Tuple2 when we want to
* minimize object allocation.
*
diff --git a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala
index 4891789d9c4e3..35d99f7e52d79 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala
@@ -22,6 +22,7 @@ import java.util.{Arrays, Comparator}
import org.apache.spark.annotations.DeveloperAPI
/**
+ * :: DeveloperAPI ::
* A simple open hash table optimized for the append-only use case, where keys
* are never removed, but the value for each key may be changed.
*
diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala
index 37c85697d2f5a..3e397f53aaaf3 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala
@@ -32,6 +32,7 @@ import org.apache.spark.serializer.Serializer
import org.apache.spark.storage.{BlockId, BlockManager}
/**
+ * :: DeveloperAPI ::
* An append-only map that spills sorted content to disk when there is insufficient space for it
* to grow.
*
diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala
index b039d426161cf..418b6169c16ca 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala
@@ -22,6 +22,7 @@ import scala.reflect.ClassTag
import org.apache.spark.annotations.DeveloperAPI
/**
+ * :: DeveloperAPI ::
* A fast hash map implementation for nullable keys. This hash map supports insertions and updates,
* but not deletions. This map is about 5X faster than java.util.HashMap, while using much less
* space overhead.
diff --git a/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala b/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala
index b445318518c5b..77862518bd678 100644
--- a/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala
+++ b/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala
@@ -20,6 +20,7 @@ package org.apache.spark.util.random
import org.apache.spark.annotations.DeveloperAPI
/**
+ * :: DeveloperAPI ::
* A class with pseudorandom behavior.
*/
@DeveloperAPI
diff --git a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala
index bd621d336d481..cbc15873f8cce 100644
--- a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala
+++ b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala
@@ -25,6 +25,7 @@ import cern.jet.random.engine.DRand
import org.apache.spark.annotations.DeveloperAPI
/**
+ * :: DeveloperAPI ::
* A pseudorandom sampler. It is possible to change the sampled item type. For example, we might
* want to add weights for stratified sampling or importance sampling. Should only use
* transformations that are tied to the sampler and cannot be applied after sampling.
@@ -43,6 +44,7 @@ trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable
}
/**
+ * :: DeveloperAPI ::
* A sampler based on Bernoulli trials.
*
* @param lb lower bound of the acceptance range
@@ -71,6 +73,7 @@ class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false)
}
/**
+ * :: DeveloperAPI ::
* A sampler based on values drawn from Poisson distribution.
*
* @param poisson a Poisson random number generator
diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb
index 786d3ef6f09a9..df03ea5c620ae 100644
--- a/docs/_plugins/copy_api_dirs.rb
+++ b/docs/_plugins/copy_api_dirs.rb
@@ -57,14 +57,9 @@
File.open(js_file, 'a') { |f| f.write("\n" + js.join()) }
# Append custom CSS
+ css = File.readlines("./css/api-docs.css")
css_file = dest + "/lib/template.css"
- extra_css = [
- "",
- "/* Styles added by spark jekyll plug-in */",
- ".developer {background-color: #44751E; float: right; text-transform: uppercase;}",
- ".experimental {background-color: #257080; float: right; text-transform: uppercase;}",
- ].join("\n")
- File.open(css_file, 'a') { |f| f.write(extra_css) }
+ File.open(css_file, 'a') { |f| f.write("\n" + css.join()) }
end
# # Build Epydoc for Python
diff --git a/docs/css/api-docs.css b/docs/css/api-docs.css
new file mode 100644
index 0000000000000..cc5f37bbdb42e
--- /dev/null
+++ b/docs/css/api-docs.css
@@ -0,0 +1,14 @@
+/* Dynamically injected style for the API docs */
+
+.developer {
+ background-color: #44751E;
+}
+
+.experimental {
+ background-color: #257080;
+}
+
+.badge {
+ font-family: Arial, san-serif;
+ float: right;
+}
diff --git a/docs/js/api-docs.js b/docs/js/api-docs.js
index 3040910d0c9fd..832f41958fc81 100644
--- a/docs/js/api-docs.js
+++ b/docs/js/api-docs.js
@@ -1,21 +1,27 @@
/* Dynamically injected post-processing code for the API docs */
$(document).ready(function() {
- // Find annotations
- var annotations = $("dt:contains('Annotations')").next("dd").children("span.name")
- var alphaComponentElements = annotations.children("a[name='org.apache.spark.annotations.AlphaComponent']")
- var developerAPIElements = annotations.children("a[name='org.apache.spark.annotations.DeveloperAPI']")
- var experimentalElements = annotations.children("a[name='org.apache.spark.annotations.Experimental']")
-
- // Insert badges into DOM tree
- var alphaComponentHTML = "ALPHA COMPONENT"
- var developerAPIHTML = "Developer API"
- var experimentalHTML = "Experimental"
- alphaComponentElements.closest("div.fullcomment").prevAll("h4.signature").prepend(alphaComponentHTML)
- alphaComponentElements.closest("div.fullcommenttop").prepend(alphaComponentHTML)
- developerAPIElements.closest("div.fullcomment").prevAll("h4.signature").prepend(developerAPIHTML)
- developerAPIElements.closest("div.fullcommenttop").prepend(developerAPIHTML)
- experimentalElements.closest("div.fullcomment").prevAll("h4.signature").prepend(experimentalHTML)
- experimentalElements.closest("div.fullcommenttop").prepend(experimentalHTML)
+ var annotations = $("dt:contains('Annotations')").next("dd").children("span.name");
+ addBadges(annotations, "AlphaComponent", ":: AlphaComponent ::", "ALPHA COMPONENT");
+ addBadges(annotations, "DeveloperAPI", ":: DeveloperAPI ::", "Developer API");
+ addBadges(annotations, "Experimental", ":: Experimental ::", "Experimental");
});
+function addBadges(allAnnotations, name, tag, html) {
+ var fullName = "org.apache.spark.annotations." + name;
+ var annotations = allAnnotations.children("a[name='" + fullName + "']");
+ var tags = $("p.comment:contains(" + tag + ")").add(
+ $("div.comment p:contains(" + tag + ")"));
+
+ // Remove identifier tags from comments
+ tags.each(function(index) {
+ var oldHTML = $(this).html();
+ var newHTML = oldHTML.replace(tag, "");
+ $(this).html(newHTML);
+ });
+
+ // Add badges to all containers
+ tags.prevAll("h4.signature").prepend(html);
+ annotations.closest("div.fullcomment").prevAll("h4.signature").prepend(html);
+ annotations.closest("div.fullcommenttop").prepend(html);
+}
\ No newline at end of file
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index 18d89f1058f0a..b8e4867f81a7f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -32,6 +32,7 @@ import org.apache.spark.sql.catalyst.rules.RuleExecutor
import org.apache.spark.sql.execution._
/**
+ * :: AlphaComponent ::
* The entry point for running relational queries using Spark. Allows the creation of [[SchemaRDD]]
* objects and the execution of SQL queries.
*
@@ -62,6 +63,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
new this.QueryExecution { val logical = plan }
/**
+ * :: Experimental ::
* Allows catalyst LogicalPlans to be executed as a SchemaRDD. Note that the LogicalPlan
* interface is considered internal, and thus not guranteed to be stable. As a result, using
* them directly is not reccomended.
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala
index 35e9bb70d070a..8eaddd5d0770e 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala
@@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.types.BooleanType
import org.apache.spark.{Dependency, OneToOneDependency, Partition, TaskContext}
/**
+ * :: AlphaComponent ::
* An RDD of [[Row]] objects that has an associated schema. In addition to standard RDD functions,
* SchemaRDDs can be used in relational queries, as shown in the examples below.
*
@@ -241,6 +242,7 @@ class SchemaRDD(
Filter(ScalaUdf(udf, BooleanType, Seq(UnresolvedAttribute(arg1.name))), logicalPlan))
/**
+ * :: Experimental ::
* Filters tuples using a function over a `Dynamic` version of a given Row. DynamicRows use
* scala's Dynamic trait to emulate an ORM of in a dynamically typed language. Since the type of
* the column is not known at compile time, all attributes are converted to strings before
@@ -259,6 +261,7 @@ class SchemaRDD(
Filter(ScalaUdf(dynamicUdf, BooleanType, Seq(WrapDynamic(logicalPlan.output))), logicalPlan))
/**
+ * :: Experimental ::
* Returns a sampled version of the underlying dataset.
*
* @group Query
@@ -271,6 +274,7 @@ class SchemaRDD(
new SchemaRDD(sqlContext, Sample(fraction, withReplacement, seed, logicalPlan))
/**
+ * :: Experimental ::
* Applies the given Generator, or table generating function, to this relation.
*
* @param generator A table generating function. The API for such functions is likely to change
@@ -295,6 +299,7 @@ class SchemaRDD(
new SchemaRDD(sqlContext, Generate(generator, join, outer, None, logicalPlan))
/**
+ * :: Experimental ::
* Adds the rows from this RDD to the specified table. Note in a standard [[SQLContext]] there is
* no notion of persistent tables, and thus queries that contain this operator will fail to
* optimize. When working with an extension of a SQLContext that has a persistent catalog, such