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"); });