From 824011bf11aa54780ffde14a25d7141634b8d0f3 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 7 Apr 2014 23:53:29 -0700 Subject: [PATCH 1/5] Add support for injecting arbitrary JavaScript to API docs --- .../scala/org/apache/spark/Aggregator.scala | 3 +- .../scala/org/apache/spark/SparkContext.scala | 3 +- .../org/apache/spark/TaskEndReason.scala | 4 +-- .../spark/annotations/DeveloperAPI.java | 25 +++++++++++++ .../spark/annotations/Experimental.java | 25 +++++++++++++ .../apache/spark/scheduler/StageInfo.scala | 3 +- docs/_config.yml | 2 +- docs/_plugins/copy_api_dirs.rb | 35 +++++++++++-------- docs/js/api-docs.js | 5 +++ 9 files changed, 83 insertions(+), 22 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/annotations/DeveloperAPI.java create mode 100644 core/src/main/scala/org/apache/spark/annotations/Experimental.java create mode 100644 docs/js/api-docs.js diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index d43ef6c430cb1..43ab09004fabf 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -17,16 +17,17 @@ package org.apache.spark +import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.util.collection.{AppendOnlyMap, ExternalAppendOnlyMap} /** - * Developer API * A set of functions used to aggregate data. * * @param createCombiner function to create the initial value of the aggregation. * @param mergeValue function to merge a new value into the aggregation result. * @param mergeCombiners function to merge outputs from multiple mergeValue function. */ +@DeveloperAPI case class Aggregator[K, V, C] ( createCombiner: V => C, mergeValue: (C, V) => C, diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index dd4833102b72e..2884037e7fb30 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -54,8 +54,7 @@ import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerTy * @param config a Spark Config object describing the application configuration. Any settings in * this config overrides the default configs as well as system properties. */ -class SparkContext(config: SparkConf) - extends Logging { +class SparkContext(config: SparkConf) extends Logging { // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, // etc) too. This is typically generated from InputFormatInfo.computePreferredLocations. It diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 626f1260cff04..8dacf9cddd3f3 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -17,16 +17,16 @@ package org.apache.spark +import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId /** - * Developer API * 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. */ - +@DeveloperAPI sealed trait TaskEndReason /** Developer API */ diff --git a/core/src/main/scala/org/apache/spark/annotations/DeveloperAPI.java b/core/src/main/scala/org/apache/spark/annotations/DeveloperAPI.java new file mode 100644 index 0000000000000..ae04a4a63b9fd --- /dev/null +++ b/core/src/main/scala/org/apache/spark/annotations/DeveloperAPI.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.annotations; + +import java.lang.annotation.*; + +@Retention(RetentionPolicy.SOURCE) +@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, + ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) +public @interface DeveloperAPI {} diff --git a/core/src/main/scala/org/apache/spark/annotations/Experimental.java b/core/src/main/scala/org/apache/spark/annotations/Experimental.java new file mode 100644 index 0000000000000..58445cc2c1e86 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/annotations/Experimental.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.annotations; + +import java.lang.annotation.*; + +@Retention(RetentionPolicy.SOURCE) +@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, + ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) +public @interface Experimental {} 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 419cd96376c04..8d2b752078a91 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -18,11 +18,12 @@ package org.apache.spark.scheduler import org.apache.spark.storage.RDDInfo +import org.apache.spark.annotations.DeveloperAPI /** - * Developer API * Stores information about a stage to pass from the scheduler to SparkListeners. */ +@DeveloperAPI class StageInfo(val stageId: Int, val name: String, val numTasks: Int, val rddInfo: RDDInfo) { /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None diff --git a/docs/_config.yml b/docs/_config.yml index aa5a5adbc1743..5e69c5626f131 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -1,5 +1,5 @@ pygments: true -markdown: kramdown +markdown: rdiscount # These allow the documentation to be updated with nerw releases # of Spark, Scala, and Mesos. diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index d4ec15bb7d6fa..786d3ef6f09a9 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -51,6 +51,11 @@ puts "cp -r " + source + "/. " + dest cp_r(source + "/.", dest) + # Append custom JavaScript + js = File.readlines("./js/api-docs.js") + js_file = dest + "/lib/template.js" + File.open(js_file, 'a') { |f| f.write("\n" + js.join()) } + # Append custom CSS css_file = dest + "/lib/template.css" extra_css = [ @@ -62,19 +67,19 @@ File.open(css_file, 'a') { |f| f.write(extra_css) } end - # Build Epydoc for Python - puts "Moving to python directory and building epydoc." - cd("../python") - puts `epydoc --config epydoc.conf` - - puts "Moving back into docs dir." - cd("../docs") - - puts "echo making directory pyspark" - mkdir_p "pyspark" - - puts "cp -r ../python/docs/. api/pyspark" - cp_r("../python/docs/.", "api/pyspark") - - cd("..") +# # Build Epydoc for Python +# puts "Moving to python directory and building epydoc." +# cd("../python") +# puts `epydoc --config epydoc.conf` +# +# puts "Moving back into docs dir." +# cd("../docs") +# +# puts "echo making directory pyspark" +# mkdir_p "pyspark" +# +# puts "cp -r ../python/docs/. api/pyspark" +# cp_r("../python/docs/.", "api/pyspark") +# +# cd("..") end diff --git a/docs/js/api-docs.js b/docs/js/api-docs.js new file mode 100644 index 0000000000000..ee63d611a18ec --- /dev/null +++ b/docs/js/api-docs.js @@ -0,0 +1,5 @@ +/* Dynamically injected post-processing code for the API docs */ + +$(document).ready(function() { + console.log("Ready") +}); From 99192ef844f6c937067c14bdfebb83b6fa8baa06 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 8 Apr 2014 11:18:58 -0700 Subject: [PATCH 2/5] 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. --- .../scala/org/apache/spark/Dependency.scala | 11 +++---- .../scala/org/apache/spark/FutureAction.scala | 7 +++-- .../main/scala/org/apache/spark/Logging.scala | 4 ++- .../apache/spark/SerializableWritable.scala | 4 ++- .../scala/org/apache/spark/SparkContext.scala | 11 ++++--- .../scala/org/apache/spark/SparkEnv.scala | 3 +- .../scala/org/apache/spark/TaskContext.scala | 3 +- .../org/apache/spark/TaskEndReason.scala | 16 +++++----- .../spark/annotations/AlphaComponent.java | 25 ++++++++++++++++ .../spark/broadcast/BroadcastFactory.scala | 5 ++-- .../apache/spark/executor/TaskMetrics.scala | 7 +++-- .../apache/spark/io/CompressionCodec.scala | 7 +++-- .../apache/spark/partial/BoundedDouble.scala | 4 ++- .../apache/spark/partial/PartialResult.scala | 6 ++-- .../apache/spark/rdd/AsyncRDDActions.scala | 3 +- .../org/apache/spark/rdd/CoGroupedRDD.scala | 3 +- .../org/apache/spark/rdd/HadoopRDD.scala | 3 +- .../org/apache/spark/rdd/NewHadoopRDD.scala | 3 +- .../spark/rdd/PartitionPruningRDD.scala | 7 ++--- .../main/scala/org/apache/spark/rdd/RDD.scala | 22 +++++++------- .../org/apache/spark/rdd/ShuffledRDD.scala | 3 +- .../scala/org/apache/spark/rdd/UnionRDD.scala | 3 +- .../spark/scheduler/InputFormatInfo.scala | 3 +- .../apache/spark/scheduler/JobLogger.scala | 3 +- .../apache/spark/scheduler/JobResult.scala | 6 +++- .../spark/scheduler/SparkListener.scala | 29 ++++++++++--------- .../apache/spark/scheduler/SplitInfo.scala | 4 ++- .../apache/spark/scheduler/StageInfo.scala | 2 +- .../org/apache/spark/scheduler/TaskInfo.scala | 4 ++- .../apache/spark/scheduler/TaskLocality.scala | 4 ++- .../spark/serializer/JavaSerializer.scala | 3 +- .../apache/spark/serializer/Serializer.scala | 11 +++---- .../apache/spark/storage/StorageUtils.scala | 3 +- .../org/apache/spark/util/MutablePair.scala | 4 ++- .../spark/util/collection/AppendOnlyMap.scala | 4 ++- .../collection/ExternalAppendOnlyMap.scala | 3 +- .../spark/util/collection/OpenHashMap.scala | 4 ++- .../spark/util/random/Pseudorandom.scala | 4 ++- .../spark/util/random/RandomSampler.scala | 8 +++-- docs/js/api-docs.js | 18 +++++++++++- .../org/apache/spark/sql/SQLContext.scala | 5 ++-- .../org/apache/spark/sql/SchemaRDD.scala | 11 +++---- 42 files changed, 192 insertions(+), 101 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/annotations/AlphaComponent.java diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index ae3a921e61658..a37f4691dcbd0 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -17,21 +17,22 @@ package org.apache.spark +import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer /** - * Developer API * Base class for dependencies. */ +@DeveloperAPI abstract class Dependency[T](val rdd: RDD[T]) extends Serializable /** - * Developer API * 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. */ +@DeveloperAPI abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { /** * Get the parent partitions for a child partition. @@ -43,7 +44,6 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { /** - * Developer API * Represents a dependency on the output of a shuffle stage. * @param rdd the parent RDD * @param partitioner partitioner used to partition the shuffle output @@ -51,6 +51,7 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { * the default serializer, as specified by `spark.serializer` config option, will * be used. */ +@DeveloperAPI class ShuffleDependency[K, V]( @transient rdd: RDD[_ <: Product2[K, V]], val partitioner: Partitioner, @@ -62,22 +63,22 @@ class ShuffleDependency[K, V]( /** - * Developer API * Represents a one-to-one dependency between partitions of the parent and child RDDs. */ +@DeveloperAPI class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) { override def getParents(partitionId: Int) = List(partitionId) } /** - * Developer API * 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 * @param outStart the start of the range in the child RDD * @param length the length of the range */ +@DeveloperAPI class RangeDependency[T](rdd: RDD[T], inStart: Int, outStart: Int, length: Int) extends NarrowDependency[T](rdd) { diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index 93b180d180ecf..e71f0e6c5d276 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -21,14 +21,15 @@ import scala.concurrent._ import scala.concurrent.duration.Duration import scala.util.Try +import org.apache.spark.annotations.Experimental 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. */ +@Experimental trait FutureAction[T] extends Future[T] { // Note that we redefine methods of the Future trait here explicitly so we can specify a different // documentation (with reference to the word "action"). @@ -85,10 +86,10 @@ 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. */ +@Experimental class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: => T) extends FutureAction[T] { @@ -150,11 +151,11 @@ 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. */ +@Experimental class ComplexFutureAction[T] extends FutureAction[T] { // Pointer to the thread that is executing the action. It is set when the action is run. diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index f4cb7973e52a0..ef6f44c9075e8 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -21,8 +21,9 @@ import org.apache.log4j.{LogManager, PropertyConfigurator} import org.slf4j.{Logger, LoggerFactory} import org.slf4j.impl.StaticLoggerBinder +import org.apache.spark.annotations.DeveloperAPI + /** - * Developer API * 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. @@ -30,6 +31,7 @@ import org.slf4j.impl.StaticLoggerBinder * NOTE: DO NOT USE this class outside of Spark. It is intended as an internal utility. * This will likely be changed or removed in future releases. */ +@DeveloperAPI trait Logging { // Make the log field transient so that objects with Logging can // be serialized and used on another machine diff --git a/core/src/main/scala/org/apache/spark/SerializableWritable.scala b/core/src/main/scala/org/apache/spark/SerializableWritable.scala index 73694d09485b1..611be5ddf2e70 100644 --- a/core/src/main/scala/org/apache/spark/SerializableWritable.scala +++ b/core/src/main/scala/org/apache/spark/SerializableWritable.scala @@ -23,7 +23,9 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.ObjectWritable import org.apache.hadoop.io.Writable -/** Developer API */ +import org.apache.spark.annotations.DeveloperAPI + +@DeveloperAPI class SerializableWritable[T <: Writable](@transient var t: T) extends Serializable { def value = t override def toString = t.toString diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 2884037e7fb30..8e6a68a18a267 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -35,6 +35,7 @@ import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHad import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} import org.apache.mesos.MesosNativeLibrary +import org.apache.spark.annotations.{DeveloperAPI, Experimental} import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} @@ -54,6 +55,8 @@ import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerTy * @param config a Spark Config object describing the application configuration. Any settings in * this config overrides the default configs as well as system properties. */ + +@DeveloperAPI class SparkContext(config: SparkConf) extends Logging { // 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 { private[spark] var preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map() /** - * Developer API * 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 * be generated using [[org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations]] * from a list of input files or InputFormats for the application. */ + @DeveloperAPI def this(config: SparkConf, preferredNodeLocationData: Map[String, Set[SplitInfo]]) = { this(config) this.preferredNodeLocationData = preferredNodeLocationData @@ -713,9 +716,9 @@ class SparkContext(config: SparkConf) extends Logging { } /** - * Developer API * Register a listener to receive up-calls from events that happen during execution. */ + @DeveloperAPI def addSparkListener(listener: SparkListener) { listenerBus.addListener(listener) } @@ -1025,9 +1028,9 @@ class SparkContext(config: SparkConf) extends Logging { } /** - * Developer API * Run a job that can return approximate results. */ + @DeveloperAPI def runApproximateJob[T, U, R]( rdd: RDD[T], func: (TaskContext, Iterator[T]) => U, @@ -1043,9 +1046,9 @@ class SparkContext(config: SparkConf) extends Logging { } /** - * Experimental * Submit a job for execution and return a FutureJob holding the result. */ + @Experimental def submitJob[T, U, R]( rdd: RDD[T], processPartition: Iterator[T] => U, diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 599688092791e..2850b19cd1503 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -25,6 +25,7 @@ import scala.util.Properties import akka.actor._ import com.google.common.collect.MapMaker +import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.api.python.PythonWorkerFactory import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.metrics.MetricsSystem @@ -35,13 +36,13 @@ import org.apache.spark.storage._ import org.apache.spark.util.{AkkaUtils, Utils} /** - * Developer API * 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 * objects needs to have the right SparkEnv set. You can get the current environment with * SparkEnv.get (e.g. after creating a SparkContext) and set it with SparkEnv.set. */ +@DeveloperAPI class SparkEnv ( val executorId: String, val actorSystem: ActorSystem, diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index faccafbdc9a1d..28ef01f4fe6fd 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -19,12 +19,13 @@ package org.apache.spark import scala.collection.mutable.ArrayBuffer +import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.executor.TaskMetrics /** - * Developer API * Contextual information about a task which can be read or mutated during execution. */ +@DeveloperAPI class TaskContext( val stageId: Int, val partitionId: Int, diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 8dacf9cddd3f3..160739bdfc6a9 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -29,13 +29,13 @@ import org.apache.spark.storage.BlockManagerId @DeveloperAPI sealed trait TaskEndReason -/** Developer API */ +@DeveloperAPI case object Success extends TaskEndReason -/** Developer API */ +@DeveloperAPI case object Resubmitted extends TaskEndReason // Task was finished earlier but we've now lost it -/** Developer API */ +@DeveloperAPI case class FetchFailed( bmAddress: BlockManagerId, shuffleId: Int, @@ -43,7 +43,7 @@ case class FetchFailed( reduceId: Int) extends TaskEndReason -/** Developer API */ +@DeveloperAPI case class ExceptionFailure( className: String, description: String, @@ -52,25 +52,25 @@ case class ExceptionFailure( extends TaskEndReason /** - * Developer API * The task finished successfully, but the result was lost from the executor's block manager before * it was fetched. */ +@DeveloperAPI case object TaskResultLost extends TaskEndReason -/** Developer API */ +@DeveloperAPI case object TaskKilled extends TaskEndReason /** - * Developer API * The task failed because the executor that it was running on was lost. This may happen because * the task crashed the JVM. */ +@DeveloperAPI case object ExecutorLostFailure extends TaskEndReason /** - * Developer API * We don't know why the task ended -- for example, because of a ClassNotFound exception when * deserializing the task result. */ +@DeveloperAPI case object UnknownReason extends TaskEndReason diff --git a/core/src/main/scala/org/apache/spark/annotations/AlphaComponent.java b/core/src/main/scala/org/apache/spark/annotations/AlphaComponent.java new file mode 100644 index 0000000000000..d54767ab84c7e --- /dev/null +++ b/core/src/main/scala/org/apache/spark/annotations/AlphaComponent.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.annotations; + +import java.lang.annotation.*; + +@Retention(RetentionPolicy.SOURCE) +@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, + ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) +public @interface AlphaComponent {} 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 051547a199cbb..d28860122b2da 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala @@ -16,17 +16,18 @@ */ package org.apache.spark.broadcast -import org.apache.spark.SecurityManager +import org.apache.spark.SecurityManager import org.apache.spark.SparkConf +import org.apache.spark.annotations.DeveloperAPI /** - * Developer API * 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. */ +@DeveloperAPI trait BroadcastFactory { def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager): Unit def newBroadcast[T](value: T, isLocal: Boolean, id: Long): Broadcast[T] 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 af8ff39313187..b7fe0c3daec8e 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,12 +17,13 @@ package org.apache.spark.executor +import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.storage.{BlockId, BlockStatus} /** - * Developer API * Metrics tracked during the execution of a task. */ +@DeveloperAPI class TaskMetrics extends Serializable { /** * Host's name the task runs on @@ -87,9 +88,9 @@ private[spark] object TaskMetrics { /** - * Developer API * Metrics pertaining to shuffle data read in a given task. */ +@DeveloperAPI class ShuffleReadMetrics extends Serializable { /** * Absolute time when this task finished reading shuffle data @@ -125,9 +126,9 @@ class ShuffleReadMetrics extends Serializable { } /** - * Developer API * Metrics pertaining to shuffle data written in a given task. */ +@DeveloperAPI class ShuffleWriteMetrics extends Serializable { /** * Number of bytes written for the shuffle by this task 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 0a8fa9f680e5d..3da1551643820 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -23,9 +23,9 @@ import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} import org.apache.spark.SparkConf +import org.apache.spark.annotations.DeveloperAPI /** - * Developer API * CompressionCodec allows the customization of choosing different compression implementations * to be used in block storage. * @@ -33,6 +33,7 @@ import org.apache.spark.SparkConf * This is intended for use as an internal compression utility within a single * Spark application. */ +@DeveloperAPI trait CompressionCodec { def compressedOutputStream(s: OutputStream): OutputStream @@ -57,13 +58,13 @@ private[spark] object CompressionCodec { /** - * Developer API * LZF implementation of [[org.apache.spark.io.CompressionCodec]]. * * Note: The wire protocol for this codec is not guaranteed to be compatible across versions * of Spark. This is intended for use as an internal compression utility within a single Spark * application. */ +@DeveloperAPI class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { override def compressedOutputStream(s: OutputStream): OutputStream = { @@ -75,7 +76,6 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { /** - * Developer API * Snappy implementation of [[org.apache.spark.io.CompressionCodec]]. * Block size can be configured by spark.io.compression.snappy.block.size. * @@ -83,6 +83,7 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { * of Spark. This is intended for use as an internal compression utility within a single Spark * application. */ +@DeveloperAPI class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec { override def compressedOutputStream(s: OutputStream): OutputStream = { 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 55b979bba0c49..dcd13554c84e1 100644 --- a/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala +++ b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala @@ -17,10 +17,12 @@ package org.apache.spark.partial +import org.apache.spark.annotations.Experimental + /** - * Experimental * A Double value with error bars and associated confidence. */ +@Experimental class BoundedDouble(val mean: Double, val confidence: Double, val low: Double, val high: Double) { override def toString(): String = "[%.3f, %.3f]".format(low, high) } diff --git a/core/src/main/scala/org/apache/spark/partial/PartialResult.scala b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala index 9fa4c12326c11..6d5f4051c11f4 100644 --- a/core/src/main/scala/org/apache/spark/partial/PartialResult.scala +++ b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala @@ -17,9 +17,9 @@ package org.apache.spark.partial -/** - * Experimental - */ +import org.apache.spark.annotations.Experimental + +@Experimental class PartialResult[R](initialVal: R, isFinal: Boolean) { private var finalValue: Option[R] = if (isFinal) Some(initialVal) else None private var failure: Option[Exception] = None 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 05eda70db4545..594d7d2e029df 100644 --- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala @@ -24,12 +24,13 @@ import scala.concurrent.ExecutionContext.Implicits.global import scala.reflect.ClassTag 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. */ +@Experimental class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Logging { /** 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 320a33f64ddd7..b3c42d014c0df 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -23,6 +23,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.{InterruptibleIterator, Partition, Partitioner, SparkEnv, TaskContext} import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency} +import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.util.collection.{ExternalAppendOnlyMap, AppendOnlyMap} import org.apache.spark.serializer.Serializer @@ -51,7 +52,6 @@ private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep] } /** - * Developer API * 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. * @@ -61,6 +61,7 @@ private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep] * @param rdds parent RDDs. * @param part partitioner used to partition the shuffle output */ +@DeveloperAPI class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: Partitioner) extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) { 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 8807e8ca9aa3b..c1beb2aa4116d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -35,6 +35,7 @@ import org.apache.hadoop.mapred.TaskID import org.apache.hadoop.util.ReflectionUtils import org.apache.spark._ +import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.NextIterator @@ -70,7 +71,6 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp } /** - * Developer API * 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`). * @@ -88,6 +88,7 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp * @param valueClass Class of the value associated with the inputFormatClass. * @param minSplits Minimum number of Hadoop Splits (HadoopRDD partitions) to generate. */ +@DeveloperAPI class HadoopRDD[K, V]( sc: SparkContext, broadcastedConf: Broadcast[SerializableWritable[Configuration]], 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 5c97965556ac7..944bb2e1f9305 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -25,6 +25,7 @@ import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ import org.apache.spark.{InterruptibleIterator, Logging, Partition, SerializableWritable, SparkContext, TaskContext} +import org.apache.spark.annotations.DeveloperAPI private[spark] class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputSplit with Writable) @@ -36,7 +37,6 @@ class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputS } /** - * Developer API * 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`). * @@ -49,6 +49,7 @@ class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputS * @param valueClass Class of the value associated with the inputFormatClass. * @param conf The Hadoop configuration. */ +@DeveloperAPI class NewHadoopRDD[K, V]( sc : SparkContext, inputFormatClass: Class[_ <: InputFormat[K, V]], 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 efeb28da4f083..3cbead2e183af 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala @@ -20,6 +20,7 @@ package org.apache.spark.rdd import scala.reflect.ClassTag import org.apache.spark.{NarrowDependency, Partition, TaskContext} +import org.apache.spark.annotations.DeveloperAPI private[spark] class PartitionPruningRDDPartition(idx: Int, val parentSplit: Partition) extends Partition { @@ -46,12 +47,12 @@ private[spark] class PruneDependency[T](rdd: RDD[T], @transient partitionFilterF /** - * Developer API * 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 * on partitions that don't have the range covering the key. */ +@DeveloperAPI class PartitionPruningRDD[T: ClassTag]( @transient prev: RDD[T], @transient partitionFilterFunc: Int => Boolean) @@ -65,9 +66,7 @@ class PartitionPruningRDD[T: ClassTag]( } -/** - * Developer API - */ +@DeveloperAPI object PartitionPruningRDD { /** 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 67f2fe65cf6d2..3373b81356745 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -35,6 +35,7 @@ import org.apache.hadoop.mapred.TextOutputFormat import org.apache.spark._ import org.apache.spark.Partitioner._ import org.apache.spark.SparkContext._ +import org.apache.spark.annotations.{DeveloperAPI, Experimental} import org.apache.spark.api.java.JavaRDD import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.CountEvaluator @@ -87,28 +88,29 @@ abstract class RDD[T: ClassTag]( // ======================================================================= /** - * Developer API * Implemented by subclasses to compute a given partition. */ + @DeveloperAPI def compute(split: Partition, context: TaskContext): Iterator[T] /** - * Developer API * 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. */ + @DeveloperAPI protected def getPartitions: Array[Partition] /** - * Developer API * 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. */ + @DeveloperAPI protected def getDependencies: Seq[Dependency[_]] = deps /** - * Developer API - * Optionally overridden by subclasses to specify placement preferences. */ + * Optionally overridden by subclasses to specify placement preferences. + */ + @DeveloperAPI protected def getPreferredLocations(split: Partition): Seq[String] = Nil /** Optionally overridden by subclasses to specify how they are partitioned. */ @@ -520,10 +522,10 @@ abstract class RDD[T: ClassTag]( } /** - * Developer API * 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. */ + @DeveloperAPI def mapPartitionsWithContext[U: ClassTag]( f: (TaskContext, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = { @@ -783,11 +785,10 @@ 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. */ + @Experimental def countApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { val countElements: (TaskContext, Iterator[T]) => Long = { (ctx, iter) => var result = 0L @@ -831,10 +832,9 @@ abstract class RDD[T: ClassTag]( } /** - * Experimental - * * Approximate version of countByValue(). */ + @Experimental def countByValueApprox( timeout: Long, confidence: Double = 0.95 @@ -855,7 +855,6 @@ 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 @@ -863,6 +862,7 @@ abstract class RDD[T: ClassTag]( * more accurate counts but increase the memory footprint and vise versa. The default value of * relativeSD is 0.05. */ + @Experimental def countApproxDistinct(relativeSD: Double = 0.05): Long = { val zeroCounter = new SerializableHyperLogLog(new HyperLogLog(relativeSD)) aggregate(zeroCounter)(_.add(_), _.merge(_)).value.cardinality() 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 679759539cfd9..b544e8b4b616e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -20,6 +20,7 @@ package org.apache.spark.rdd import scala.reflect.ClassTag import org.apache.spark.{Dependency, Partition, Partitioner, ShuffleDependency, SparkEnv, TaskContext} +import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.serializer.Serializer private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { @@ -28,13 +29,13 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { } /** - * Developer API * 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 * @tparam K the key class. * @tparam V the value class. */ +@DeveloperAPI class ShuffledRDD[K, V, P <: Product2[K, V] : ClassTag]( @transient var prev: RDD[P], part: Partitioner) diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala index c40961497a34c..f72400a7e8b35 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -23,6 +23,7 @@ import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag import org.apache.spark.{Dependency, Partition, RangeDependency, SparkContext, TaskContext} +import org.apache.spark.annotations.DeveloperAPI private[spark] class UnionPartition[T: ClassTag](idx: Int, rdd: RDD[T], splitIndex: Int) extends Partition { @@ -43,7 +44,7 @@ private[spark] class UnionPartition[T: ClassTag](idx: Int, rdd: RDD[T], splitInd } } -/** Developer API */ +@DeveloperAPI class UnionRDD[T: ClassTag]( sc: SparkContext, @transient var rdds: Seq[RDD[T]]) 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 79eda49c6fdf9..126dcbe7ed67b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala @@ -27,12 +27,13 @@ import org.apache.hadoop.mapreduce.Job import org.apache.hadoop.util.ReflectionUtils import org.apache.spark.Logging +import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.deploy.SparkHadoopUtil /** - * Developer API * Parses and holds information about inputFormat (and files) specified as a parameter. */ +@DeveloperAPI class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Class[_], val path: String) extends Logging { 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 f85ac36597e30..4a9f409868e94 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -25,10 +25,10 @@ import java.util.concurrent.LinkedBlockingQueue import scala.collection.mutable.HashMap import org.apache.spark._ +import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.executor.TaskMetrics /** - * Developer API * 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 @@ -40,6 +40,7 @@ import org.apache.spark.executor.TaskMetrics * 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 1fb6196718445..31543464bdf1c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala @@ -17,13 +17,17 @@ package org.apache.spark.scheduler +import org.apache.spark.annotations.DeveloperAPI + /** - * Developer API * A result of a job in the DAGScheduler. */ +@DeveloperAPI sealed trait JobResult +@DeveloperAPI case object JobSucceeded extends JobResult // A failed stage ID of -1 means there is not a particular stage that caused the failure +@DeveloperAPI case class JobFailed(exception: Exception, failedStageId: Int) extends JobResult 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 cbe8c25de5d55..657727d95e395 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -23,27 +23,28 @@ import scala.collection.Map import scala.collection.mutable import org.apache.spark.{Logging, TaskEndReason} +import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.{Distribution, Utils} -/** Developer API */ +@DeveloperAPI sealed trait SparkListenerEvent -/** Developer API */ +@DeveloperAPI case class SparkListenerStageSubmitted(stageInfo: StageInfo, properties: Properties = null) extends SparkListenerEvent -/** Developer API */ +@DeveloperAPI case class SparkListenerStageCompleted(stageInfo: StageInfo) extends SparkListenerEvent -/** Developer API */ +@DeveloperAPI case class SparkListenerTaskStart(stageId: Int, taskInfo: TaskInfo) extends SparkListenerEvent -/** Developer API */ +@DeveloperAPI case class SparkListenerTaskGettingResult(taskInfo: TaskInfo) extends SparkListenerEvent -/** Developer API */ +@DeveloperAPI case class SparkListenerTaskEnd( stageId: Int, taskType: String, @@ -52,26 +53,26 @@ case class SparkListenerTaskEnd( taskMetrics: TaskMetrics) extends SparkListenerEvent -/** Developer API */ +@DeveloperAPI case class SparkListenerJobStart(jobId: Int, stageIds: Seq[Int], properties: Properties = null) extends SparkListenerEvent -/** Developer API */ +@DeveloperAPI case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkListenerEvent -/** Developer API */ +@DeveloperAPI case class SparkListenerEnvironmentUpdate(environmentDetails: Map[String, Seq[(String, String)]]) extends SparkListenerEvent -/** Developer API */ +@DeveloperAPI case class SparkListenerBlockManagerAdded(blockManagerId: BlockManagerId, maxMem: Long) extends SparkListenerEvent -/** Developer API */ +@DeveloperAPI case class SparkListenerBlockManagerRemoved(blockManagerId: BlockManagerId) extends SparkListenerEvent -/** Developer API */ +@DeveloperAPI case class SparkListenerUnpersistRDD(rddId: Int) extends SparkListenerEvent /** An event used in the listener to shutdown the listener daemon thread. */ @@ -79,10 +80,10 @@ private[spark] case object SparkListenerShutdown extends SparkListenerEvent /** - * Developer API * Interface for listening to events from the Spark scheduler. Note that this is an internal * interface which might change in different Spark releases. */ +@DeveloperAPI trait SparkListener { /** * Called when a stage is completed, with information on the completed stage @@ -142,9 +143,9 @@ trait SparkListener { } /** - * Developer API * Simple SparkListener that logs a few summary statistics when each stage completes */ +@DeveloperAPI class StatsReportListener extends SparkListener with Logging { import org.apache.spark.scheduler.StatsReportListener._ diff --git a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala index 4f28bdd991d8c..1c58b5103532c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala @@ -19,9 +19,11 @@ package org.apache.spark.scheduler import collection.mutable.ArrayBuffer +import org.apache.spark.annotations.DeveloperAPI + // information about a specific split instance : handles both split instances. // So that we do not need to worry about the differences. -/** Developer API */ +@DeveloperAPI class SplitInfo(val inputFormatClazz: Class[_], val hostLocation: String, val path: String, val length: Long, val underlyingSplit: Any) { override def toString(): String = { 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 8d2b752078a91..ecca565628cae 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -17,8 +17,8 @@ package org.apache.spark.scheduler -import org.apache.spark.storage.RDDInfo import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.storage.RDDInfo /** * Stores information about a stage to pass from the scheduler to SparkListeners. 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 515755a93c6e6..f7ca08bc4603a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala @@ -17,10 +17,12 @@ package org.apache.spark.scheduler +import org.apache.spark.annotations.DeveloperAPI + /** - * Developer API * Information about a running task attempt inside a TaskSet. */ +@DeveloperAPI class TaskInfo( val taskId: Long, val index: Int, diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala index d2d05a0b81bba..dfe447a294d39 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala @@ -17,7 +17,9 @@ package org.apache.spark.scheduler -/** Developer API */ +import org.apache.spark.annotations.DeveloperAPI + +@DeveloperAPI object TaskLocality extends Enumeration { // Process local is expected to be used ONLY within TaskSetManager for now. val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value 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 cf0576713f4ad..8c818f98a0cda 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -21,6 +21,7 @@ import java.io._ import java.nio.ByteBuffer import org.apache.spark.SparkConf +import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.util.ByteBufferInputStream private[spark] class JavaSerializationStream(out: OutputStream, counterReset: Int) @@ -94,13 +95,13 @@ private[spark] class JavaSerializerInstance(counterReset: Int) extends Serialize } /** - * Developer API * 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 * Spark. It is intended to be used to serialize/de-serialize data within a single * Spark application. */ +@DeveloperAPI class JavaSerializer(conf: SparkConf) extends Serializer with Externalizable { private var counterReset = conf.getInt("spark.serializer.objectStreamReset", 10000) 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 015c4f62e110b..559c223d4e175 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -22,11 +22,11 @@ import java.nio.ByteBuffer import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream -import org.apache.spark.util.{ByteBufferInputStream, NextIterator} import org.apache.spark.SparkEnv +import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.util.{ByteBufferInputStream, NextIterator} /** - * Developer API * 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. @@ -41,6 +41,7 @@ import org.apache.spark.SparkEnv * Note that serializers are not required to be wire-compatible across different versions of Spark. * They are intended to be used to serialize/de-serialize data within a single Spark application. */ +@DeveloperAPI trait Serializer { def newInstance(): SerializerInstance } @@ -54,9 +55,9 @@ object Serializer { /** - * Developer API * An instance of a serializer, for use by one thread at a time. */ +@DeveloperAPI trait SerializerInstance { def serialize[T](t: T): ByteBuffer @@ -87,9 +88,9 @@ trait SerializerInstance { /** - * Developer API * A stream for writing serialized objects. */ +@DeveloperAPI trait SerializationStream { def writeObject[T](t: T): SerializationStream def flush(): Unit @@ -105,9 +106,9 @@ trait SerializationStream { /** - * Developer API * A stream for reading serialized objects. */ +@DeveloperAPI trait DeserializationStream { def readObject[T](): T def close(): Unit diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 63be6917d5131..182c4127a1952 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -21,6 +21,7 @@ import scala.collection.Map import scala.collection.mutable import org.apache.spark.SparkContext +import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.util.Utils private[spark] @@ -48,7 +49,7 @@ class StorageStatus( } -/** Developer API */ +@DeveloperAPI class RDDInfo(val id: Int, val name: String, val numPartitions: Int, val storageLevel: StorageLevel) extends Ordered[RDDInfo] { 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 75ff7f68e51b7..da8b1c0f5a37e 100644 --- a/core/src/main/scala/org/apache/spark/util/MutablePair.scala +++ b/core/src/main/scala/org/apache/spark/util/MutablePair.scala @@ -17,14 +17,16 @@ package org.apache.spark.util +import org.apache.spark.annotations.DeveloperAPI + /** - * Developer API * A tuple of 2 elements. This can be used as an alternative to Scala's Tuple2 when we want to * minimize object allocation. * * @param _1 Element 1 of this MutablePair * @param _2 Element 2 of this MutablePair */ +@DeveloperAPI case class MutablePair[@specialized(Int, Long, Double, Char, Boolean/* , AnyRef */) T1, @specialized(Int, Long, Double, Char, Boolean/* , AnyRef */) T2] (var _1: T1, var _2: T2) 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 6ee1d96bbc894..4891789d9c4e3 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 @@ -19,8 +19,9 @@ package org.apache.spark.util.collection import java.util.{Arrays, Comparator} +import org.apache.spark.annotations.DeveloperAPI + /** - * Developer API * 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. * @@ -30,6 +31,7 @@ import java.util.{Arrays, Comparator} * * TODO: Cache the hash values of each key? java.util.HashMap does that. */ +@DeveloperAPI class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] with Serializable { require(initialCapacity <= (1 << 29), "Can't make capacity bigger than 2^29 elements") 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 f255b258889c0..37c85697d2f5a 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 @@ -27,11 +27,11 @@ import com.google.common.io.ByteStreams import it.unimi.dsi.fastutil.io.FastBufferedInputStream import org.apache.spark.{Logging, SparkEnv} +import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{BlockId, BlockManager} /** - * Developer API * An append-only map that spills sorted content to disk when there is insufficient space for it * to grow. * @@ -56,6 +56,7 @@ import org.apache.spark.storage.{BlockId, BlockManager} * `spark.shuffle.safetyFraction` specifies an additional margin of safety as a fraction of * this threshold, in case map size estimation is not sufficiently accurate. */ +@DeveloperAPI class ExternalAppendOnlyMap[K, V, C]( createCombiner: V => C, mergeValue: (C, V) => C, 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 959fe44f5132a..b039d426161cf 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 @@ -19,14 +19,16 @@ package org.apache.spark.util.collection import scala.reflect.ClassTag +import org.apache.spark.annotations.DeveloperAPI + /** - * Developer API * 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. * * Under the hood, it uses our OpenHashSet implementation. */ +@DeveloperAPI class OpenHashMap[K >: Null : ClassTag, @specialized(Long, Int, Double) V: ClassTag]( initialCapacity: Int) extends Iterable[(K, V)] 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 e8adc28c2acda..b445318518c5b 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 @@ -17,10 +17,12 @@ package org.apache.spark.util.random +import org.apache.spark.annotations.DeveloperAPI + /** - * Developer API * A class with pseudorandom behavior. */ +@DeveloperAPI trait Pseudorandom { /** Set random seed. */ def setSeed(seed: Long) 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 a999a607d1b31..bd621d336d481 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 @@ -22,8 +22,9 @@ import java.util.Random import cern.jet.random.Poisson import cern.jet.random.engine.DRand +import org.apache.spark.annotations.DeveloperAPI + /** - * Developer API * 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. @@ -31,6 +32,7 @@ import cern.jet.random.engine.DRand * @tparam T item type * @tparam U sampled item type */ +@DeveloperAPI trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable { /** take a random sample */ @@ -41,7 +43,6 @@ trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable } /** - * Developer API * A sampler based on Bernoulli trials. * * @param lb lower bound of the acceptance range @@ -49,6 +50,7 @@ trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable * @param complement whether to use the complement of the range specified, default to false * @tparam T item type */ +@DeveloperAPI class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) (implicit random: Random = new XORShiftRandom) extends RandomSampler[T, T] { @@ -69,12 +71,12 @@ class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) } /** - * Developer API * A sampler based on values drawn from Poisson distribution. * * @param poisson a Poisson random number generator * @tparam T item type */ +@DeveloperAPI class PoissonSampler[T](mean: Double) (implicit var poisson: Poisson = new Poisson(mean, new DRand)) extends RandomSampler[T, T] { diff --git a/docs/js/api-docs.js b/docs/js/api-docs.js index ee63d611a18ec..3040910d0c9fd 100644 --- a/docs/js/api-docs.js +++ b/docs/js/api-docs.js @@ -1,5 +1,21 @@ /* Dynamically injected post-processing code for the API docs */ $(document).ready(function() { - console.log("Ready") + // 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) }); + 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 5cdae65e29721..18d89f1058f0a 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 @@ -21,6 +21,7 @@ import scala.language.implicitConversions import scala.reflect.runtime.universe.TypeTag import org.apache.spark.SparkContext +import org.apache.spark.annotations.{AlphaComponent, Experimental} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.dsl @@ -31,13 +32,13 @@ import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.execution._ /** - * ALPHA COMPONENT * The entry point for running relational queries using Spark. Allows the creation of [[SchemaRDD]] * objects and the execution of SQL queries. * * @groupname userf Spark SQL Functions * @groupname Ungrouped Support functions for language integrated queries. */ +@AlphaComponent class SQLContext(@transient val sparkContext: SparkContext) extends Logging with dsl.ExpressionConversions @@ -61,11 +62,11 @@ 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. */ + @Experimental implicit def logicalPlanToSparkQuery(plan: LogicalPlan): SchemaRDD = new SchemaRDD(this, plan) /** 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 b7ebce240578a..35e9bb70d070a 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 @@ -17,6 +17,7 @@ package org.apache.spark.sql +import org.apache.spark.annotations.{AlphaComponent, Experimental} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ @@ -26,7 +27,6 @@ import org.apache.spark.sql.catalyst.types.BooleanType import org.apache.spark.{Dependency, OneToOneDependency, Partition, TaskContext} /** - * ALPHA COMPONENT * 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. * @@ -89,6 +89,7 @@ import org.apache.spark.{Dependency, OneToOneDependency, Partition, TaskContext} * @groupprio schema -1 * @groupname Ungrouped Base RDD Functions */ +@AlphaComponent class SchemaRDD( @transient val sqlContext: SQLContext, @transient val logicalPlan: LogicalPlan) @@ -240,7 +241,6 @@ 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 @@ -252,17 +252,18 @@ class SchemaRDD( * * @group Query */ + @Experimental def where(dynamicUdf: (DynamicRow) => Boolean) = new SchemaRDD( sqlContext, Filter(ScalaUdf(dynamicUdf, BooleanType, Seq(WrapDynamic(logicalPlan.output))), logicalPlan)) /** - * Experimental * Returns a sampled version of the underlying dataset. * * @group Query */ + @Experimental def sample( fraction: Double, withReplacement: Boolean = true, @@ -270,7 +271,6 @@ 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 @@ -286,6 +286,7 @@ class SchemaRDD( * * @group Query */ + @Experimental def generate( generator: Generator, join: Boolean = false, @@ -294,7 +295,6 @@ 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 @@ -302,6 +302,7 @@ class SchemaRDD( * * @group schema */ + @Experimental def insertInto(tableName: String, overwrite: Boolean = false) = new SchemaRDD( sqlContext, From f3954e0414590e5817a2d3f0517b191ee22b20ee Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 8 Apr 2014 15:35:23 -0700 Subject: [PATCH 3/5] Add identifier tags in comments to work around scaladocs bug The bug is that annotations and comments cannot both be displayed in the summary page. The workaround is adding a special pattern that we grep for as we post-process the DOM tree in JavaScript. Example: A @DeveloperAPI annotated class's comment must begin with ":: Developer API ::" --- .../scala/org/apache/spark/Aggregator.scala | 1 + .../scala/org/apache/spark/Dependency.scala | 5 +++ .../scala/org/apache/spark/FutureAction.scala | 3 ++ .../main/scala/org/apache/spark/Logging.scala | 1 + .../scala/org/apache/spark/SparkContext.scala | 5 +++ .../scala/org/apache/spark/SparkEnv.scala | 1 + .../scala/org/apache/spark/TaskContext.scala | 1 + .../org/apache/spark/TaskEndReason.scala | 4 ++ .../spark/broadcast/BroadcastFactory.scala | 3 +- .../apache/spark/executor/TaskMetrics.scala | 3 ++ .../apache/spark/io/CompressionCodec.scala | 3 ++ .../apache/spark/partial/BoundedDouble.scala | 1 + .../apache/spark/rdd/AsyncRDDActions.scala | 1 + .../org/apache/spark/rdd/CoGroupedRDD.scala | 1 + .../org/apache/spark/rdd/HadoopRDD.scala | 1 + .../org/apache/spark/rdd/NewHadoopRDD.scala | 1 + .../spark/rdd/PartitionPruningRDD.scala | 1 + .../main/scala/org/apache/spark/rdd/RDD.scala | 8 ++++ .../org/apache/spark/rdd/ShuffledRDD.scala | 1 + .../spark/scheduler/InputFormatInfo.scala | 1 + .../apache/spark/scheduler/JobLogger.scala | 2 +- .../apache/spark/scheduler/JobResult.scala | 1 + .../spark/scheduler/SparkListener.scala | 2 + .../apache/spark/scheduler/StageInfo.scala | 1 + .../org/apache/spark/scheduler/TaskInfo.scala | 1 + .../spark/serializer/JavaSerializer.scala | 1 + .../apache/spark/serializer/Serializer.scala | 4 ++ .../org/apache/spark/util/MutablePair.scala | 1 + .../spark/util/collection/AppendOnlyMap.scala | 1 + .../collection/ExternalAppendOnlyMap.scala | 1 + .../spark/util/collection/OpenHashMap.scala | 1 + .../spark/util/random/Pseudorandom.scala | 1 + .../spark/util/random/RandomSampler.scala | 3 ++ docs/_plugins/copy_api_dirs.rb | 9 +---- docs/css/api-docs.css | 14 +++++++ docs/js/api-docs.js | 38 +++++++++++-------- .../org/apache/spark/sql/SQLContext.scala | 2 + .../org/apache/spark/sql/SchemaRDD.scala | 5 +++ 38 files changed, 109 insertions(+), 25 deletions(-) create mode 100644 docs/css/api-docs.css 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 From 0d489081ca4f55fb48ea5b38d8b7784e944213e2 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 8 Apr 2014 15:45:03 -0700 Subject: [PATCH 4/5] Comments and new lines (minor) --- .../scala/org/apache/spark/SparkContext.scala | 1 - docs/_config.yml | 2 +- docs/_plugins/copy_api_dirs.rb | 30 +++++++++---------- docs/js/api-docs.js | 2 +- 4 files changed, 17 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index d9aa8e614eb9b..46f1980ffe942 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1050,7 +1050,6 @@ class SparkContext(config: SparkConf) extends Logging { } /** - * * Submit a job for execution and return a FutureJob holding the result. */ @Experimental diff --git a/docs/_config.yml b/docs/_config.yml index 5e69c5626f131..aa5a5adbc1743 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -1,5 +1,5 @@ pygments: true -markdown: rdiscount +markdown: kramdown # These allow the documentation to be updated with nerw releases # of Spark, Scala, and Mesos. diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index df03ea5c620ae..bbd56d2fd13bb 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -62,19 +62,19 @@ File.open(css_file, 'a') { |f| f.write("\n" + css.join()) } end -# # Build Epydoc for Python -# puts "Moving to python directory and building epydoc." -# cd("../python") -# puts `epydoc --config epydoc.conf` -# -# puts "Moving back into docs dir." -# cd("../docs") -# -# puts "echo making directory pyspark" -# mkdir_p "pyspark" -# -# puts "cp -r ../python/docs/. api/pyspark" -# cp_r("../python/docs/.", "api/pyspark") -# -# cd("..") + # Build Epydoc for Python + puts "Moving to python directory and building epydoc." + cd("../python") + puts `epydoc --config epydoc.conf` + + puts "Moving back into docs dir." + cd("../docs") + + puts "echo making directory pyspark" + mkdir_p "pyspark" + + puts "cp -r ../python/docs/. api/pyspark" + cp_r("../python/docs/.", "api/pyspark") + + cd("..") end diff --git a/docs/js/api-docs.js b/docs/js/api-docs.js index 832f41958fc81..1d234658e7ee9 100644 --- a/docs/js/api-docs.js +++ b/docs/js/api-docs.js @@ -24,4 +24,4 @@ function addBadges(allAnnotations, name, tag, html) { 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 +} From c1bcb4177f3a544fcdde2bc2d05f98de48ebfcf2 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 8 Apr 2014 16:35:43 -0700 Subject: [PATCH 5/5] DeveloperAPI -> DeveloperApi --- .../scala/org/apache/spark/Aggregator.scala | 6 ++-- .../scala/org/apache/spark/Dependency.scala | 22 ++++++------ .../main/scala/org/apache/spark/Logging.scala | 6 ++-- .../apache/spark/SerializableWritable.scala | 4 +-- .../scala/org/apache/spark/SparkContext.scala | 18 +++++----- .../scala/org/apache/spark/SparkEnv.scala | 6 ++-- .../scala/org/apache/spark/TaskContext.scala | 6 ++-- .../org/apache/spark/TaskEndReason.scala | 28 +++++++-------- .../{DeveloperAPI.java => DeveloperApi.java} | 2 +- .../spark/broadcast/BroadcastFactory.scala | 6 ++-- .../apache/spark/executor/TaskMetrics.scala | 14 ++++---- .../apache/spark/io/CompressionCodec.scala | 14 ++++---- .../org/apache/spark/rdd/CoGroupedRDD.scala | 6 ++-- .../org/apache/spark/rdd/HadoopRDD.scala | 6 ++-- .../org/apache/spark/rdd/NewHadoopRDD.scala | 6 ++-- .../spark/rdd/PartitionPruningRDD.scala | 8 ++--- .../main/scala/org/apache/spark/rdd/RDD.scala | 22 ++++++------ .../org/apache/spark/rdd/ShuffledRDD.scala | 6 ++-- .../scala/org/apache/spark/rdd/UnionRDD.scala | 4 +-- .../spark/scheduler/InputFormatInfo.scala | 6 ++-- .../apache/spark/scheduler/JobLogger.scala | 6 ++-- .../apache/spark/scheduler/JobResult.scala | 10 +++--- .../spark/scheduler/SparkListener.scala | 34 +++++++++---------- .../apache/spark/scheduler/SplitInfo.scala | 4 +-- .../apache/spark/scheduler/StageInfo.scala | 6 ++-- .../org/apache/spark/scheduler/TaskInfo.scala | 6 ++-- .../apache/spark/scheduler/TaskLocality.scala | 4 +-- .../spark/serializer/JavaSerializer.scala | 6 ++-- .../apache/spark/serializer/Serializer.scala | 18 +++++----- .../apache/spark/storage/StorageUtils.scala | 4 +-- .../org/apache/spark/util/MutablePair.scala | 6 ++-- .../spark/util/collection/AppendOnlyMap.scala | 6 ++-- .../collection/ExternalAppendOnlyMap.scala | 6 ++-- .../spark/util/collection/OpenHashMap.scala | 6 ++-- .../spark/util/random/Pseudorandom.scala | 6 ++-- .../spark/util/random/RandomSampler.scala | 14 ++++---- docs/js/api-docs.js | 2 +- 37 files changed, 170 insertions(+), 170 deletions(-) rename core/src/main/scala/org/apache/spark/annotations/{DeveloperAPI.java => DeveloperApi.java} (96%) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index 5bc9b880f4e94..06d957a1b2cb8 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -17,18 +17,18 @@ package org.apache.spark -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.util.collection.{AppendOnlyMap, ExternalAppendOnlyMap} /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * A set of functions used to aggregate data. * * @param createCombiner function to create the initial value of the aggregation. * @param mergeValue function to merge a new value into the aggregation result. * @param mergeCombiners function to merge outputs from multiple mergeValue function. */ -@DeveloperAPI +@DeveloperApi case class Aggregator[K, V, C] ( createCombiner: V => C, mergeValue: (C, V) => C, diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 73d1c40d523e6..13f3dcd84d721 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -17,24 +17,24 @@ package org.apache.spark -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Base class for dependencies. */ -@DeveloperAPI +@DeveloperApi abstract class Dependency[T](val rdd: RDD[T]) extends Serializable /** - * :: DeveloperAPI :: + * :: 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. */ -@DeveloperAPI +@DeveloperApi abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { /** * Get the parent partitions for a child partition. @@ -46,7 +46,7 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { /** - * :: DeveloperAPI :: + * :: 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 @@ -54,7 +54,7 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { * the default serializer, as specified by `spark.serializer` config option, will * be used. */ -@DeveloperAPI +@DeveloperApi class ShuffleDependency[K, V]( @transient rdd: RDD[_ <: Product2[K, V]], val partitioner: Partitioner, @@ -66,24 +66,24 @@ class ShuffleDependency[K, V]( /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Represents a one-to-one dependency between partitions of the parent and child RDDs. */ -@DeveloperAPI +@DeveloperApi class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) { override def getParents(partitionId: Int) = List(partitionId) } /** - * :: DeveloperAPI :: + * :: 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 * @param outStart the start of the range in the child RDD * @param length the length of the range */ -@DeveloperAPI +@DeveloperApi class RangeDependency[T](rdd: RDD[T], inStart: Int, outStart: Int, length: Int) extends NarrowDependency[T](rdd) { diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index 2c6b62dcc09e0..ed33c7d2be88b 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -21,10 +21,10 @@ import org.apache.log4j.{LogManager, PropertyConfigurator} import org.slf4j.{Logger, LoggerFactory} import org.slf4j.impl.StaticLoggerBinder -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi /** - * :: 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. @@ -32,7 +32,7 @@ import org.apache.spark.annotations.DeveloperAPI * NOTE: DO NOT USE this class outside of Spark. It is intended as an internal utility. * This will likely be changed or removed in future releases. */ -@DeveloperAPI +@DeveloperApi trait Logging { // Make the log field transient so that objects with Logging can // be serialized and used on another machine diff --git a/core/src/main/scala/org/apache/spark/SerializableWritable.scala b/core/src/main/scala/org/apache/spark/SerializableWritable.scala index 611be5ddf2e70..9c44340cacadb 100644 --- a/core/src/main/scala/org/apache/spark/SerializableWritable.scala +++ b/core/src/main/scala/org/apache/spark/SerializableWritable.scala @@ -23,9 +23,9 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.ObjectWritable import org.apache.hadoop.io.Writable -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi -@DeveloperAPI +@DeveloperApi class SerializableWritable[T <: Writable](@transient var t: T) extends Serializable { def value = t override def toString = t.toString diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 46f1980ffe942..6464b3bc4572f 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -35,7 +35,7 @@ import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHad import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} import org.apache.mesos.MesosNativeLibrary -import org.apache.spark.annotations.{DeveloperAPI, Experimental} +import org.apache.spark.annotations.{DeveloperApi, Experimental} import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} @@ -49,7 +49,7 @@ import org.apache.spark.ui.SparkUI import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedHashMap, Utils} /** - * :: DeveloperAPI :: + * :: 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. * @@ -57,7 +57,7 @@ import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerTy * this config overrides the default configs as well as system properties. */ -@DeveloperAPI +@DeveloperApi class SparkContext(config: SparkConf) extends Logging { // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, @@ -66,14 +66,14 @@ class SparkContext(config: SparkConf) extends Logging { private[spark] var preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map() /** - * :: DeveloperAPI :: + * :: 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 * be generated using [[org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations]] * from a list of input files or InputFormats for the application. */ - @DeveloperAPI + @DeveloperApi def this(config: SparkConf, preferredNodeLocationData: Map[String, Set[SplitInfo]]) = { this(config) this.preferredNodeLocationData = preferredNodeLocationData @@ -718,10 +718,10 @@ class SparkContext(config: SparkConf) extends Logging { } /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Register a listener to receive up-calls from events that happen during execution. */ - @DeveloperAPI + @DeveloperApi def addSparkListener(listener: SparkListener) { listenerBus.addListener(listener) } @@ -1031,10 +1031,10 @@ class SparkContext(config: SparkConf) extends Logging { } /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Run a job that can return approximate results. */ - @DeveloperAPI + @DeveloperApi def runApproximateJob[T, U, R]( rdd: RDD[T], func: (TaskContext, Iterator[T]) => U, diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 5f63b6b5bd4f0..e4e1f5e6378da 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -25,7 +25,7 @@ import scala.util.Properties import akka.actor._ import com.google.common.collect.MapMaker -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.api.python.PythonWorkerFactory import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.metrics.MetricsSystem @@ -36,14 +36,14 @@ import org.apache.spark.storage._ import org.apache.spark.util.{AkkaUtils, Utils} /** - * :: DeveloperAPI :: + * :: 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 * objects needs to have the right SparkEnv set. You can get the current environment with * SparkEnv.get (e.g. after creating a SparkContext) and set it with SparkEnv.set. */ -@DeveloperAPI +@DeveloperApi class SparkEnv ( val executorId: String, val actorSystem: ActorSystem, diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 58e39abd6e0bd..ec42f75598696 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -19,14 +19,14 @@ package org.apache.spark import scala.collection.mutable.ArrayBuffer -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.executor.TaskMetrics /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Contextual information about a task which can be read or mutated during execution. */ -@DeveloperAPI +@DeveloperApi class TaskContext( val stageId: Int, val partitionId: Int, diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index cd73508711a1e..01312a44714a4 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -17,26 +17,26 @@ package org.apache.spark -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId /** - * :: DeveloperAPI :: + * :: 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. */ -@DeveloperAPI +@DeveloperApi sealed trait TaskEndReason -@DeveloperAPI +@DeveloperApi case object Success extends TaskEndReason -@DeveloperAPI +@DeveloperApi case object Resubmitted extends TaskEndReason // Task was finished earlier but we've now lost it -@DeveloperAPI +@DeveloperApi case class FetchFailed( bmAddress: BlockManagerId, shuffleId: Int, @@ -44,7 +44,7 @@ case class FetchFailed( reduceId: Int) extends TaskEndReason -@DeveloperAPI +@DeveloperApi case class ExceptionFailure( className: String, description: String, @@ -53,28 +53,28 @@ case class ExceptionFailure( extends TaskEndReason /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * The task finished successfully, but the result was lost from the executor's block manager before * it was fetched. */ -@DeveloperAPI +@DeveloperApi case object TaskResultLost extends TaskEndReason -@DeveloperAPI +@DeveloperApi case object TaskKilled extends TaskEndReason /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * The task failed because the executor that it was running on was lost. This may happen because * the task crashed the JVM. */ -@DeveloperAPI +@DeveloperApi case object ExecutorLostFailure extends TaskEndReason /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * We don't know why the task ended -- for example, because of a ClassNotFound exception when * deserializing the task result. */ -@DeveloperAPI +@DeveloperApi case object UnknownReason extends TaskEndReason diff --git a/core/src/main/scala/org/apache/spark/annotations/DeveloperAPI.java b/core/src/main/scala/org/apache/spark/annotations/DeveloperApi.java similarity index 96% rename from core/src/main/scala/org/apache/spark/annotations/DeveloperAPI.java rename to core/src/main/scala/org/apache/spark/annotations/DeveloperApi.java index ae04a4a63b9fd..74c63ea949b3e 100644 --- a/core/src/main/scala/org/apache/spark/annotations/DeveloperAPI.java +++ b/core/src/main/scala/org/apache/spark/annotations/DeveloperApi.java @@ -22,4 +22,4 @@ @Retention(RetentionPolicy.SOURCE) @Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) -public @interface DeveloperAPI {} +public @interface DeveloperApi {} 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 67faa217c46bf..fd44abc37e16a 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala @@ -19,16 +19,16 @@ package org.apache.spark.broadcast import org.apache.spark.SecurityManager import org.apache.spark.SparkConf -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi /** - * :: DeveloperAPI :: + * :: 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. */ -@DeveloperAPI +@DeveloperApi trait BroadcastFactory { def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager): Unit def newBroadcast[T](value: T, isLocal: Boolean, id: Long): Broadcast[T] 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 ed1b45f5d485b..cbddcb3c34f7d 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,14 +17,14 @@ package org.apache.spark.executor -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.storage.{BlockId, BlockStatus} /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Metrics tracked during the execution of a task. */ -@DeveloperAPI +@DeveloperApi class TaskMetrics extends Serializable { /** * Host's name the task runs on @@ -89,10 +89,10 @@ private[spark] object TaskMetrics { /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Metrics pertaining to shuffle data read in a given task. */ -@DeveloperAPI +@DeveloperApi class ShuffleReadMetrics extends Serializable { /** * Absolute time when this task finished reading shuffle data @@ -128,10 +128,10 @@ class ShuffleReadMetrics extends Serializable { } /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Metrics pertaining to shuffle data written in a given task. */ -@DeveloperAPI +@DeveloperApi class ShuffleWriteMetrics extends Serializable { /** * Number of bytes written for the shuffle by this task 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 1f0134d0cdf3e..a0df43bafc30f 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -23,10 +23,10 @@ import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} import org.apache.spark.SparkConf -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * CompressionCodec allows the customization of choosing different compression implementations * to be used in block storage. * @@ -34,7 +34,7 @@ import org.apache.spark.annotations.DeveloperAPI * This is intended for use as an internal compression utility within a single * Spark application. */ -@DeveloperAPI +@DeveloperApi trait CompressionCodec { def compressedOutputStream(s: OutputStream): OutputStream @@ -59,14 +59,14 @@ private[spark] object CompressionCodec { /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * LZF implementation of [[org.apache.spark.io.CompressionCodec]]. * * Note: The wire protocol for this codec is not guaranteed to be compatible across versions * of Spark. This is intended for use as an internal compression utility within a single Spark * application. */ -@DeveloperAPI +@DeveloperApi class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { override def compressedOutputStream(s: OutputStream): OutputStream = { @@ -78,7 +78,7 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Snappy implementation of [[org.apache.spark.io.CompressionCodec]]. * Block size can be configured by spark.io.compression.snappy.block.size. * @@ -86,7 +86,7 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { * of Spark. This is intended for use as an internal compression utility within a single Spark * application. */ -@DeveloperAPI +@DeveloperApi class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec { override def compressedOutputStream(s: OutputStream): OutputStream = { 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 f8c3cda91781c..9f0ce8fe919c7 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -23,7 +23,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.{InterruptibleIterator, Partition, Partitioner, SparkEnv, TaskContext} import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency} -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.util.collection.{ExternalAppendOnlyMap, AppendOnlyMap} import org.apache.spark.serializer.Serializer @@ -52,7 +52,7 @@ private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep] } /** - * :: DeveloperAPI :: + * :: 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. * @@ -62,7 +62,7 @@ private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep] * @param rdds parent RDDs. * @param part partitioner used to partition the shuffle output */ -@DeveloperAPI +@DeveloperApi class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: Partitioner) extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) { 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 ea03b3b8e4861..1fad40bcf8242 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -35,7 +35,7 @@ import org.apache.hadoop.mapred.TaskID import org.apache.hadoop.util.ReflectionUtils import org.apache.spark._ -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.NextIterator @@ -71,7 +71,7 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp } /** - * :: DeveloperAPI :: + * :: 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`). * @@ -89,7 +89,7 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp * @param valueClass Class of the value associated with the inputFormatClass. * @param minSplits Minimum number of Hadoop Splits (HadoopRDD partitions) to generate. */ -@DeveloperAPI +@DeveloperApi class HadoopRDD[K, V]( sc: SparkContext, broadcastedConf: Broadcast[SerializableWritable[Configuration]], 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 510f0cfbd8be2..bdfe9a41041e6 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ import org.apache.spark.{InterruptibleIterator, Logging, Partition, SerializableWritable, SparkContext, TaskContext} -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi private[spark] class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputSplit with Writable) @@ -37,7 +37,7 @@ class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputS } /** - * :: DeveloperAPI :: + * :: 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`). * @@ -50,7 +50,7 @@ class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputS * @param valueClass Class of the value associated with the inputFormatClass. * @param conf The Hadoop configuration. */ -@DeveloperAPI +@DeveloperApi class NewHadoopRDD[K, V]( sc : SparkContext, inputFormatClass: Class[_ <: InputFormat[K, V]], 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 160236f943715..5a42ba6a9d05a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala @@ -20,7 +20,7 @@ package org.apache.spark.rdd import scala.reflect.ClassTag import org.apache.spark.{NarrowDependency, Partition, TaskContext} -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi private[spark] class PartitionPruningRDDPartition(idx: Int, val parentSplit: Partition) extends Partition { @@ -47,13 +47,13 @@ private[spark] class PruneDependency[T](rdd: RDD[T], @transient partitionFilterF /** - * :: DeveloperAPI :: + * :: 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 * on partitions that don't have the range covering the key. */ -@DeveloperAPI +@DeveloperApi class PartitionPruningRDD[T: ClassTag]( @transient prev: RDD[T], @transient partitionFilterFunc: Int => Boolean) @@ -67,7 +67,7 @@ class PartitionPruningRDD[T: ClassTag]( } -@DeveloperAPI +@DeveloperApi object PartitionPruningRDD { /** 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 c56b67ef9b643..e9af8b358acaa 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -35,7 +35,7 @@ import org.apache.hadoop.mapred.TextOutputFormat import org.apache.spark._ import org.apache.spark.Partitioner._ import org.apache.spark.SparkContext._ -import org.apache.spark.annotations.{DeveloperAPI, Experimental} +import org.apache.spark.annotations.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaRDD import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.CountEvaluator @@ -88,33 +88,33 @@ abstract class RDD[T: ClassTag]( // ======================================================================= /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Implemented by subclasses to compute a given partition. */ - @DeveloperAPI + @DeveloperApi def compute(split: Partition, context: TaskContext): Iterator[T] /** - * :: DeveloperAPI :: + * :: 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. */ - @DeveloperAPI + @DeveloperApi protected def getPartitions: Array[Partition] /** - * :: DeveloperAPI :: + * :: 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. */ - @DeveloperAPI + @DeveloperApi protected def getDependencies: Seq[Dependency[_]] = deps /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Optionally overridden by subclasses to specify placement preferences. */ - @DeveloperAPI + @DeveloperApi protected def getPreferredLocations(split: Partition): Seq[String] = Nil /** Optionally overridden by subclasses to specify how they are partitioned. */ @@ -526,11 +526,11 @@ abstract class RDD[T: ClassTag]( } /** - * :: DeveloperAPI :: + * :: 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. */ - @DeveloperAPI + @DeveloperApi def mapPartitionsWithContext[U: ClassTag]( f: (TaskContext, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = { 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 e9971cdabdba6..3661fe37f6f53 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -20,7 +20,7 @@ package org.apache.spark.rdd import scala.reflect.ClassTag import org.apache.spark.{Dependency, Partition, Partitioner, ShuffleDependency, SparkEnv, TaskContext} -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.serializer.Serializer private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { @@ -29,14 +29,14 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { } /** - * :: DeveloperAPI :: + * :: 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 * @tparam K the key class. * @tparam V the value class. */ -@DeveloperAPI +@DeveloperApi class ShuffledRDD[K, V, P <: Product2[K, V] : ClassTag]( @transient var prev: RDD[P], part: Partitioner) diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala index f72400a7e8b35..0313a12867374 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -23,7 +23,7 @@ import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag import org.apache.spark.{Dependency, Partition, RangeDependency, SparkContext, TaskContext} -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi private[spark] class UnionPartition[T: ClassTag](idx: Int, rdd: RDD[T], splitIndex: Int) extends Partition { @@ -44,7 +44,7 @@ private[spark] class UnionPartition[T: ClassTag](idx: Int, rdd: RDD[T], splitInd } } -@DeveloperAPI +@DeveloperApi class UnionRDD[T: ClassTag]( sc: SparkContext, @transient var rdds: Seq[RDD[T]]) 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 6e02ea4ac7d7d..3fd2c4ff3b570 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala @@ -27,14 +27,14 @@ import org.apache.hadoop.mapreduce.Job import org.apache.hadoop.util.ReflectionUtils import org.apache.spark.Logging -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.deploy.SparkHadoopUtil /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Parses and holds information about inputFormat (and files) specified as a parameter. */ -@DeveloperAPI +@DeveloperApi class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Class[_], val path: String) extends Logging { 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 fd458734ab09e..30d898c991548 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -25,11 +25,11 @@ import java.util.concurrent.LinkedBlockingQueue import scala.collection.mutable.HashMap import org.apache.spark._ -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.executor.TaskMetrics /** - * :: DeveloperAPI :: + * :: 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 @@ -40,7 +40,7 @@ import org.apache.spark.executor.TaskMetrics * to log application information as SparkListenerEvents. To enable this functionality, set * spark.eventLog.enabled to true. */ -@DeveloperAPI +@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 d8c443f6130ae..dc41effd59950 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala @@ -17,18 +17,18 @@ package org.apache.spark.scheduler -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * A result of a job in the DAGScheduler. */ -@DeveloperAPI +@DeveloperApi sealed trait JobResult -@DeveloperAPI +@DeveloperApi case object JobSucceeded extends JobResult // A failed stage ID of -1 means there is not a particular stage that caused the failure -@DeveloperAPI +@DeveloperApi case class JobFailed(exception: Exception, failedStageId: Int) extends JobResult 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 72ff68f97e0ab..50de7c2a9df8e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -23,28 +23,28 @@ import scala.collection.Map import scala.collection.mutable import org.apache.spark.{Logging, TaskEndReason} -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.{Distribution, Utils} -@DeveloperAPI +@DeveloperApi sealed trait SparkListenerEvent -@DeveloperAPI +@DeveloperApi case class SparkListenerStageSubmitted(stageInfo: StageInfo, properties: Properties = null) extends SparkListenerEvent -@DeveloperAPI +@DeveloperApi case class SparkListenerStageCompleted(stageInfo: StageInfo) extends SparkListenerEvent -@DeveloperAPI +@DeveloperApi case class SparkListenerTaskStart(stageId: Int, taskInfo: TaskInfo) extends SparkListenerEvent -@DeveloperAPI +@DeveloperApi case class SparkListenerTaskGettingResult(taskInfo: TaskInfo) extends SparkListenerEvent -@DeveloperAPI +@DeveloperApi case class SparkListenerTaskEnd( stageId: Int, taskType: String, @@ -53,26 +53,26 @@ case class SparkListenerTaskEnd( taskMetrics: TaskMetrics) extends SparkListenerEvent -@DeveloperAPI +@DeveloperApi case class SparkListenerJobStart(jobId: Int, stageIds: Seq[Int], properties: Properties = null) extends SparkListenerEvent -@DeveloperAPI +@DeveloperApi case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkListenerEvent -@DeveloperAPI +@DeveloperApi case class SparkListenerEnvironmentUpdate(environmentDetails: Map[String, Seq[(String, String)]]) extends SparkListenerEvent -@DeveloperAPI +@DeveloperApi case class SparkListenerBlockManagerAdded(blockManagerId: BlockManagerId, maxMem: Long) extends SparkListenerEvent -@DeveloperAPI +@DeveloperApi case class SparkListenerBlockManagerRemoved(blockManagerId: BlockManagerId) extends SparkListenerEvent -@DeveloperAPI +@DeveloperApi case class SparkListenerUnpersistRDD(rddId: Int) extends SparkListenerEvent /** An event used in the listener to shutdown the listener daemon thread. */ @@ -80,11 +80,11 @@ private[spark] case object SparkListenerShutdown extends SparkListenerEvent /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Interface for listening to events from the Spark scheduler. Note that this is an internal * interface which might change in different Spark releases. */ -@DeveloperAPI +@DeveloperApi trait SparkListener { /** * Called when a stage is completed, with information on the completed stage @@ -144,10 +144,10 @@ trait SparkListener { } /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Simple SparkListener that logs a few summary statistics when each stage completes */ -@DeveloperAPI +@DeveloperApi class StatsReportListener extends SparkListener with Logging { import org.apache.spark.scheduler.StatsReportListener._ diff --git a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala index 1c58b5103532c..e245ff8c25e0b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala @@ -19,11 +19,11 @@ package org.apache.spark.scheduler import collection.mutable.ArrayBuffer -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi // information about a specific split instance : handles both split instances. // So that we do not need to worry about the differences. -@DeveloperAPI +@DeveloperApi class SplitInfo(val inputFormatClazz: Class[_], val hostLocation: String, val path: String, val length: Long, val underlyingSplit: Any) { override def toString(): String = { 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 460f4109d536b..e2e287666834b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -17,14 +17,14 @@ package org.apache.spark.scheduler -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.storage.RDDInfo /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Stores information about a stage to pass from the scheduler to SparkListeners. */ -@DeveloperAPI +@DeveloperApi class StageInfo(val stageId: Int, val name: String, val numTasks: Int, val rddInfo: RDDInfo) { /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None 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 04f891b6e72ed..a37ab5a338e4f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala @@ -17,13 +17,13 @@ package org.apache.spark.scheduler -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Information about a running task attempt inside a TaskSet. */ -@DeveloperAPI +@DeveloperApi class TaskInfo( val taskId: Long, val index: Int, diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala index dfe447a294d39..eaf736fb20ea1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala @@ -17,9 +17,9 @@ package org.apache.spark.scheduler -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi -@DeveloperAPI +@DeveloperApi object TaskLocality extends Enumeration { // Process local is expected to be used ONLY within TaskSetManager for now. val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value 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 9880bd916a9bc..6c6463b1c8827 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -21,7 +21,7 @@ import java.io._ import java.nio.ByteBuffer import org.apache.spark.SparkConf -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.util.ByteBufferInputStream private[spark] class JavaSerializationStream(out: OutputStream, counterReset: Int) @@ -95,14 +95,14 @@ private[spark] class JavaSerializerInstance(counterReset: Int) extends Serialize } /** - * :: DeveloperAPI :: + * :: 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 * Spark. It is intended to be used to serialize/de-serialize data within a single * Spark application. */ -@DeveloperAPI +@DeveloperApi class JavaSerializer(conf: SparkConf) extends Serializer with Externalizable { private var counterReset = conf.getInt("spark.serializer.objectStreamReset", 10000) 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 cd19e45132ad4..d404035646065 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -23,11 +23,11 @@ import java.nio.ByteBuffer import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream import org.apache.spark.SparkEnv -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.util.{ByteBufferInputStream, NextIterator} /** - * :: DeveloperAPI :: + * :: 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. @@ -42,7 +42,7 @@ import org.apache.spark.util.{ByteBufferInputStream, NextIterator} * Note that serializers are not required to be wire-compatible across different versions of Spark. * They are intended to be used to serialize/de-serialize data within a single Spark application. */ -@DeveloperAPI +@DeveloperApi trait Serializer { def newInstance(): SerializerInstance } @@ -56,10 +56,10 @@ object Serializer { /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * An instance of a serializer, for use by one thread at a time. */ -@DeveloperAPI +@DeveloperApi trait SerializerInstance { def serialize[T](t: T): ByteBuffer @@ -90,10 +90,10 @@ trait SerializerInstance { /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * A stream for writing serialized objects. */ -@DeveloperAPI +@DeveloperApi trait SerializationStream { def writeObject[T](t: T): SerializationStream def flush(): Unit @@ -109,10 +109,10 @@ trait SerializationStream { /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * A stream for reading serialized objects. */ -@DeveloperAPI +@DeveloperApi trait DeserializationStream { def readObject[T](): T def close(): Unit diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 182c4127a1952..d2df92ea7655d 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -21,7 +21,7 @@ import scala.collection.Map import scala.collection.mutable import org.apache.spark.SparkContext -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.util.Utils private[spark] @@ -49,7 +49,7 @@ class StorageStatus( } -@DeveloperAPI +@DeveloperApi class RDDInfo(val id: Int, val name: String, val numPartitions: Int, val storageLevel: StorageLevel) extends Ordered[RDDInfo] { 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 2b62e9e4e25e8..a68859a1f7669 100644 --- a/core/src/main/scala/org/apache/spark/util/MutablePair.scala +++ b/core/src/main/scala/org/apache/spark/util/MutablePair.scala @@ -17,17 +17,17 @@ package org.apache.spark.util -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi /** - * :: 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. * * @param _1 Element 1 of this MutablePair * @param _2 Element 2 of this MutablePair */ -@DeveloperAPI +@DeveloperApi case class MutablePair[@specialized(Int, Long, Double, Char, Boolean/* , AnyRef */) T1, @specialized(Int, Long, Double, Char, Boolean/* , AnyRef */) T2] (var _1: T1, var _2: T2) 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 35d99f7e52d79..f1fbfdd51896d 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 @@ -19,10 +19,10 @@ package org.apache.spark.util.collection import java.util.{Arrays, Comparator} -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi /** - * :: 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. * @@ -32,7 +32,7 @@ import org.apache.spark.annotations.DeveloperAPI * * TODO: Cache the hash values of each key? java.util.HashMap does that. */ -@DeveloperAPI +@DeveloperApi class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] with Serializable { require(initialCapacity <= (1 << 29), "Can't make capacity bigger than 2^29 elements") 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 3e397f53aaaf3..abd3a6d61731b 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 @@ -27,12 +27,12 @@ import com.google.common.io.ByteStreams import it.unimi.dsi.fastutil.io.FastBufferedInputStream import org.apache.spark.{Logging, SparkEnv} -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{BlockId, BlockManager} /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * An append-only map that spills sorted content to disk when there is insufficient space for it * to grow. * @@ -57,7 +57,7 @@ import org.apache.spark.storage.{BlockId, BlockManager} * `spark.shuffle.safetyFraction` specifies an additional margin of safety as a fraction of * this threshold, in case map size estimation is not sufficiently accurate. */ -@DeveloperAPI +@DeveloperApi class ExternalAppendOnlyMap[K, V, C]( createCombiner: V => C, mergeValue: (C, V) => C, 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 418b6169c16ca..73011361b0676 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 @@ -19,17 +19,17 @@ package org.apache.spark.util.collection import scala.reflect.ClassTag -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi /** - * :: 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. * * Under the hood, it uses our OpenHashSet implementation. */ -@DeveloperAPI +@DeveloperApi class OpenHashMap[K >: Null : ClassTag, @specialized(Long, Int, Double) V: ClassTag]( initialCapacity: Int) extends Iterable[(K, V)] 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 77862518bd678..745c38ef72143 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 @@ -17,13 +17,13 @@ package org.apache.spark.util.random -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * A class with pseudorandom behavior. */ -@DeveloperAPI +@DeveloperApi trait Pseudorandom { /** Set random seed. */ def setSeed(seed: Long) 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 cbc15873f8cce..5306b89fddc86 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 @@ -22,10 +22,10 @@ import java.util.Random import cern.jet.random.Poisson import cern.jet.random.engine.DRand -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi /** - * :: 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. @@ -33,7 +33,7 @@ import org.apache.spark.annotations.DeveloperAPI * @tparam T item type * @tparam U sampled item type */ -@DeveloperAPI +@DeveloperApi trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable { /** take a random sample */ @@ -44,7 +44,7 @@ trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable } /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * A sampler based on Bernoulli trials. * * @param lb lower bound of the acceptance range @@ -52,7 +52,7 @@ trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable * @param complement whether to use the complement of the range specified, default to false * @tparam T item type */ -@DeveloperAPI +@DeveloperApi class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) (implicit random: Random = new XORShiftRandom) extends RandomSampler[T, T] { @@ -73,13 +73,13 @@ class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) } /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * A sampler based on values drawn from Poisson distribution. * * @param poisson a Poisson random number generator * @tparam T item type */ -@DeveloperAPI +@DeveloperApi class PoissonSampler[T](mean: Double) (implicit var poisson: Poisson = new Poisson(mean, new DRand)) extends RandomSampler[T, T] { diff --git a/docs/js/api-docs.js b/docs/js/api-docs.js index 1d234658e7ee9..2c3ca434ec46c 100644 --- a/docs/js/api-docs.js +++ b/docs/js/api-docs.js @@ -3,7 +3,7 @@ $(document).ready(function() { var annotations = $("dt:contains('Annotations')").next("dd").children("span.name"); addBadges(annotations, "AlphaComponent", ":: AlphaComponent ::", "ALPHA COMPONENT"); - addBadges(annotations, "DeveloperAPI", ":: DeveloperAPI ::", "Developer API"); + addBadges(annotations, "DeveloperApi", ":: DeveloperApi ::", "Developer API"); addBadges(annotations, "Experimental", ":: Experimental ::", "Experimental"); });