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,