From 9d48cbf0caf4d04cd3f9deaa02abadb5856f9033 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 21 Mar 2014 19:37:37 -0700 Subject: [PATCH 01/31] Initial pass --- .../scala/org/apache/spark/Aggregator.scala | 2 ++ .../scala/org/apache/spark/Dependency.scala | 10 ++++++++++ .../scala/org/apache/spark/FutureAction.scala | 4 ++++ .../apache/spark/SerializableWritable.scala | 2 +- .../scala/org/apache/spark/SparkEnv.scala | 2 +- .../scala/org/apache/spark/TaskContext.scala | 5 +++++ .../org/apache/spark/api/java/JavaUtils.scala | 2 +- .../spark/executor/ExecutorSource.scala | 2 +- .../apache/spark/executor/TaskMetrics.scala | 15 +++++++++++++++ .../apache/spark/io/CompressionCodec.scala | 18 ++++++++++++++++++ .../spark/metrics/sink/ConsoleSink.scala | 2 +- .../apache/spark/metrics/sink/CsvSink.scala | 2 +- .../spark/metrics/sink/GraphiteSink.scala | 2 +- .../apache/spark/metrics/sink/JmxSink.scala | 2 +- .../spark/metrics/sink/MetricsServlet.scala | 2 +- .../org/apache/spark/metrics/sink/Sink.scala | 2 +- .../spark/metrics/source/JvmSource.scala | 2 +- .../apache/spark/metrics/source/Source.scala | 2 +- .../apache/spark/partial/BoundedDouble.scala | 4 +++- .../apache/spark/partial/PartialResult.scala | 3 +++ .../apache/spark/rdd/AsyncRDDActions.scala | 2 ++ .../org/apache/spark/rdd/CoGroupedRDD.scala | 1 + .../org/apache/spark/rdd/CoalescedRDD.scala | 4 ++-- .../scala/org/apache/spark/rdd/EmptyRDD.scala | 2 +- .../org/apache/spark/rdd/HadoopRDD.scala | 5 +++++ .../scala/org/apache/spark/rdd/JdbcRDD.scala | 2 +- .../org/apache/spark/rdd/NewHadoopRDD.scala | 5 +++++ .../spark/rdd/PartitionPruningRDD.scala | 10 ++++++++-- .../spark/rdd/PartitionwiseSampledRDD.scala | 2 +- .../scala/org/apache/spark/rdd/PipedRDD.scala | 2 +- .../org/apache/spark/rdd/SampledRDD.scala | 2 +- .../org/apache/spark/rdd/ShuffledRDD.scala | 2 ++ .../scala/org/apache/spark/rdd/UnionRDD.scala | 2 +- .../spark/rdd/ZippedPartitionsRDD.scala | 8 ++++---- .../org/apache/spark/rdd/ZippedRDD.scala | 2 +- .../spark/scheduler/SparkListener.scala | 19 ++++++++++++++++++- .../spark/serializer/JavaSerializer.scala | 6 ++++++ .../spark/serializer/KryoSerializer.scala | 4 ++++ .../apache/spark/serializer/Serializer.scala | 12 ++++++++++++ .../spark/util/BoundedPriorityQueue.scala | 2 ++ .../apache/spark/util/CollectionsUtil.scala | 2 +- .../org/apache/spark/util/Distribution.scala | 2 +- .../org/apache/spark/util/FileLogger.scala | 2 +- .../org/apache/spark/util/MutablePair.scala | 2 ++ .../spark/util/TimeStampedHashMap.scala | 2 +- .../spark/util/TimeStampedHashSet.scala | 2 +- .../spark/util/random/Pseudorandom.scala | 2 ++ .../spark/util/random/RandomSampler.scala | 8 +++++++- 48 files changed, 166 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index ceead59b79ed6..09add7a497ca1 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -20,6 +20,8 @@ package org.apache.spark import org.apache.spark.util.collection.{AppendOnlyMap, ExternalAppendOnlyMap} /** + * SEMI-PRIVATE + * * A set of functions used to aggregate data. * * @param createCombiner function to create the initial value of the aggregation. diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 3132dcf745e19..e23b7ad4a11b0 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -21,12 +21,16 @@ import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer /** + * SEMI-PRIVATE + * * Base class for dependencies. */ abstract class Dependency[T](val rdd: RDD[T]) extends Serializable /** + * SEMI-PRIVATE + * * 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. */ @@ -41,6 +45,8 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { /** + * SEMI-PRIVATE + * * Represents a dependency on the output of a shuffle stage. * @param rdd the parent RDD * @param partitioner partitioner used to partition the shuffle output @@ -59,6 +65,8 @@ class ShuffleDependency[K, V]( /** + * SEMI-PRIVATE + * * Represents a one-to-one dependency between partitions of the parent and child RDDs. */ class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) { @@ -67,6 +75,8 @@ class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) { /** + * SEMI-PRIVATE + * * Represents a one-to-one dependency between ranges of partitions in the parent and child RDDs. * @param rdd the parent RDD * @param inStart the start of the range in the parent RDD diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index f2decd14ef6d9..6eb5689b5004b 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -25,6 +25,8 @@ 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. */ @@ -148,6 +150,8 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: /** + * EXPERIMENTAL + * * A [[FutureAction]] for actions that could trigger multiple Spark jobs. Examples include take, * takeSample. Cancellation works by setting the cancelled flag to true and interrupting the * action thread if it is being blocked by a job. diff --git a/core/src/main/scala/org/apache/spark/SerializableWritable.scala b/core/src/main/scala/org/apache/spark/SerializableWritable.scala index dff665cae6cb6..e250ebe84e458 100644 --- a/core/src/main/scala/org/apache/spark/SerializableWritable.scala +++ b/core/src/main/scala/org/apache/spark/SerializableWritable.scala @@ -23,7 +23,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.ObjectWritable import org.apache.hadoop.io.Writable -class SerializableWritable[T <: Writable](@transient var t: T) extends Serializable { +private[spark] 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/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 5ceac28fe7afb..36748766f377a 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -41,7 +41,7 @@ import org.apache.spark.util.{AkkaUtils, Utils} * 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. */ -class SparkEnv private[spark] ( +private[spark] class SparkEnv private[spark] ( val executorId: String, val actorSystem: ActorSystem, val serializer: Serializer, diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index be53ca2968cfb..928ad0830f6f8 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -21,6 +21,11 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.executor.TaskMetrics +/** + * SEMI-PRIVATE + * + * Contextual information about a task which can be read or mutated during execution. + */ class TaskContext( val stageId: Int, val partitionId: Int, diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala index ecbf18849ad48..22810cb1c662d 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala @@ -19,7 +19,7 @@ package org.apache.spark.api.java import com.google.common.base.Optional -object JavaUtils { +private[spark] object JavaUtils { def optionToOptional[T](option: Option[T]): Optional[T] = option match { case Some(value) => Optional.of(value) diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala index 127f5e90f3e1a..0ed52cfe9df61 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala @@ -24,7 +24,7 @@ import org.apache.hadoop.fs.FileSystem import org.apache.spark.metrics.source.Source -class ExecutorSource(val executor: Executor, executorId: String) extends Source { +private[spark] class ExecutorSource(val executor: Executor, executorId: String) extends Source { private def fileStats(scheme: String) : Option[FileSystem.Statistics] = FileSystem.getAllStatistics().filter(s => s.getScheme.equals(scheme)).headOption 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 88625e79a5c68..bd63b1148d54b 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -19,6 +19,11 @@ package org.apache.spark.executor import org.apache.spark.storage.{BlockId, BlockStatus} +/** + * SEMI-PRIVATE + * + * Metrics tracked during the execution of a task. + */ class TaskMetrics extends Serializable { /** * Host's name the task runs on @@ -82,6 +87,11 @@ object TaskMetrics { } +/** + * SEMI-PRIVATE + * + * Metrics pertaining to shuffle data read in a given task. + */ class ShuffleReadMetrics extends Serializable { /** * Absolute time when this task finished reading shuffle data @@ -116,6 +126,11 @@ class ShuffleReadMetrics extends Serializable { var remoteBytesRead: Long = _ } +/** + * SEMI-PRIVATE + * + * Metrics pertaining to shuffle data written in a given task. + */ 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 059e58824c39b..a7035b647a701 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -25,8 +25,14 @@ import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} import org.apache.spark.SparkConf /** + * SEMI-PRIVATE + * * CompressionCodec allows the customization of choosing different compression implementations * to be used in block storage. + * + * Note: The wire protocol for a codec is not guaranteed compatible across versions of Spark. + * This is intended for use as an internal compression utility within a single + * Spark application. */ trait CompressionCodec { @@ -52,7 +58,13 @@ private[spark] object CompressionCodec { /** + * SEMI-PRIVATE + * * 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. */ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { @@ -65,8 +77,14 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { /** + * SEMI-PRIVATE + * * Snappy implementation of [[org.apache.spark.io.CompressionCodec]]. * Block size can be configured by spark.io.compression.snappy.block.size. + * + * 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. */ class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec { diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala index 64eac73605388..05852f1f98993 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala @@ -25,7 +25,7 @@ import com.codahale.metrics.{ConsoleReporter, MetricRegistry} import org.apache.spark.SecurityManager import org.apache.spark.metrics.MetricsSystem -class ConsoleSink(val property: Properties, val registry: MetricRegistry, +private[spark] class ConsoleSink(val property: Properties, val registry: MetricRegistry, securityMgr: SecurityManager) extends Sink { val CONSOLE_DEFAULT_PERIOD = 10 val CONSOLE_DEFAULT_UNIT = "SECONDS" diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala index 544848d4150b6..42c1200926fea 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala @@ -26,7 +26,7 @@ import com.codahale.metrics.{CsvReporter, MetricRegistry} import org.apache.spark.SecurityManager import org.apache.spark.metrics.MetricsSystem -class CsvSink(val property: Properties, val registry: MetricRegistry, +private[spark] class CsvSink(val property: Properties, val registry: MetricRegistry, securityMgr: SecurityManager) extends Sink { val CSV_KEY_PERIOD = "period" val CSV_KEY_UNIT = "unit" diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala index 7f0a2fd16fa99..aeb4ad44a0647 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala @@ -27,7 +27,7 @@ import com.codahale.metrics.graphite.{Graphite, GraphiteReporter} import org.apache.spark.SecurityManager import org.apache.spark.metrics.MetricsSystem -class GraphiteSink(val property: Properties, val registry: MetricRegistry, +private[spark] class GraphiteSink(val property: Properties, val registry: MetricRegistry, securityMgr: SecurityManager) extends Sink { val GRAPHITE_DEFAULT_PERIOD = 10 val GRAPHITE_DEFAULT_UNIT = "SECONDS" diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala index 3b5edd5c376f0..ed27234b4e760 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala @@ -22,7 +22,7 @@ import java.util.Properties import com.codahale.metrics.{JmxReporter, MetricRegistry} import org.apache.spark.SecurityManager -class JmxSink(val property: Properties, val registry: MetricRegistry, +private[spark] class JmxSink(val property: Properties, val registry: MetricRegistry, securityMgr: SecurityManager) extends Sink { val reporter: JmxReporter = JmxReporter.forRegistry(registry).build() diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala index 854b52c510e3d..571539ba5e467 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala @@ -30,7 +30,7 @@ import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.SecurityManager import org.apache.spark.ui.JettyUtils._ -class MetricsServlet(val property: Properties, val registry: MetricRegistry, +private[spark] class MetricsServlet(val property: Properties, val registry: MetricRegistry, securityMgr: SecurityManager) extends Sink { val SERVLET_KEY_PATH = "path" val SERVLET_KEY_SAMPLE = "sample" diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala index 3a739aa563eae..6f2b5a06027ea 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala @@ -17,7 +17,7 @@ package org.apache.spark.metrics.sink -trait Sink { +private[spark] trait Sink { def start: Unit def stop: Unit } diff --git a/core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala b/core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala index 75cb2b8973aa1..f865f9648a91e 100644 --- a/core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala +++ b/core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala @@ -20,7 +20,7 @@ package org.apache.spark.metrics.source import com.codahale.metrics.MetricRegistry import com.codahale.metrics.jvm.{GarbageCollectorMetricSet, MemoryUsageGaugeSet} -class JvmSource extends Source { +private[spark] class JvmSource extends Source { val sourceName = "jvm" val metricRegistry = new MetricRegistry() diff --git a/core/src/main/scala/org/apache/spark/metrics/source/Source.scala b/core/src/main/scala/org/apache/spark/metrics/source/Source.scala index 3fee55cc6dcd5..1dda2cd83b2a9 100644 --- a/core/src/main/scala/org/apache/spark/metrics/source/Source.scala +++ b/core/src/main/scala/org/apache/spark/metrics/source/Source.scala @@ -19,7 +19,7 @@ package org.apache.spark.metrics.source import com.codahale.metrics.MetricRegistry -trait Source { +private[spark] trait Source { def sourceName: String def metricRegistry: MetricRegistry } 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 5f4450859cc9b..a3ec79775e05f 100644 --- a/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala +++ b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala @@ -18,7 +18,9 @@ package org.apache.spark.partial /** - * A Double with error bars on it. + * EXPERIMENTAL + * + * A Double value with error bars and associated confidence. */ 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 812368e04ac0d..4d8f49b5a354b 100644 --- a/core/src/main/scala/org/apache/spark/partial/PartialResult.scala +++ b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala @@ -17,6 +17,9 @@ package org.apache.spark.partial +/** + * 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 d1c74a5063510..e0e5d08ffe6fc 100644 --- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala @@ -26,6 +26,8 @@ import scala.reflect.ClassTag import org.apache.spark.{ComplexFutureAction, FutureAction, Logging} /** + * EXPERIMENTAL + * * A set of asynchronous RDD actions available through an implicit conversion. * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions. */ diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 9aa454a5c8b88..ad35f43dd9825 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -57,6 +57,7 @@ private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep] * @param rdds parent RDDs. * @param part partitioner used to partition the shuffle output. */ +private[spark] 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/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala index 4e82b51313bf0..44401a663440c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -32,7 +32,7 @@ import org.apache.spark._ * @param parentsIndices list of indices in the parent that have been coalesced into this partition * @param preferredLocation the preferred location for this partition */ -case class CoalescedRDDPartition( +private[spark] case class CoalescedRDDPartition( index: Int, @transient rdd: RDD[_], parentsIndices: Array[Int], @@ -70,7 +70,7 @@ case class CoalescedRDDPartition( * @param maxPartitions number of desired partitions in the coalesced RDD * @param balanceSlack used to trade-off balance and locality. 1.0 is all locality, 0 is all balance */ -class CoalescedRDD[T: ClassTag]( +private[spark] class CoalescedRDD[T: ClassTag]( @transient var prev: RDD[T], maxPartitions: Int, balanceSlack: Double = 0.10) diff --git a/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala index a84e5f9fd8ef8..6f28ac3ed37f6 100644 --- a/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala @@ -24,7 +24,7 @@ import org.apache.spark.{Partition, SparkContext, TaskContext} /** * An RDD that is empty, i.e. has no element in it. */ -class EmptyRDD[T: ClassTag](sc: SparkContext) extends RDD[T](sc, Nil) { +private[spark] class EmptyRDD[T: ClassTag](sc: SparkContext) extends RDD[T](sc, Nil) { override def getPartitions: Array[Partition] = Array.empty 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 3af008bd72378..30970af325c6c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -70,9 +70,14 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp } /** + * SEMI-PRIVATE + * * 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`). * + * Note: Instantiating this class directly is not recommended, please use + * [[org.apache.spark.SparkContext.hadoopRDD()]] + * * @param sc The SparkContext to associate the RDD with. * @param broadcastedConf A general Hadoop Configuration, or a subclass of it. If the enclosed * variabe references an instance of JobConf, then that JobConf will be used for the Hadoop job. diff --git a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala index 1b503743ac117..a76a070b5b863 100644 --- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala @@ -27,7 +27,7 @@ import org.apache.spark.util.NextIterator private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) extends Partition { override def index = idx } - +// TODO: Expose a jdbcRDD function in SparkContext and mark this as semi-private /** * An RDD that executes an SQL query on a JDBC connection and reads results. * For usage example, see test case JdbcRDDSuite. 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 461a749eac48b..8a34292bf0e85 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -36,9 +36,14 @@ class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputS } /** + * SEMI-PRIVATE + * * 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`). * + * Note: Instantiating this class directly is not recommended, please use + * [[org.apache.spark.SparkContext.newAPIHadoopRDD()]] + * * @param sc The SparkContext to associate the RDD with. * @param inputFormatClass Storage format of the data to be read. * @param keyClass Class of the key associated with the inputFormatClass. 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 b0440ca7f32cf..9f3fa08a5ccb8 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala @@ -21,7 +21,8 @@ import scala.reflect.ClassTag import org.apache.spark.{NarrowDependency, Partition, TaskContext} -class PartitionPruningRDDPartition(idx: Int, val parentSplit: Partition) extends Partition { +private[spark] class PartitionPruningRDDPartition(idx: Int, val parentSplit: Partition) + extends Partition { override val index = idx } @@ -30,7 +31,7 @@ class PartitionPruningRDDPartition(idx: Int, val parentSplit: Partition) extends * Represents a dependency between the PartitionPruningRDD and its parent. In this * case, the child RDD contains a subset of partitions of the parents'. */ -class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boolean) +private[spark] class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boolean) extends NarrowDependency[T](rdd) { @transient @@ -45,6 +46,8 @@ class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boo /** + * SEMI-PRIVATE + * * 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 @@ -63,6 +66,9 @@ class PartitionPruningRDD[T: ClassTag]( } +/** + * SEMI-PRIVATE + */ object PartitionPruningRDD { /** diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala index ce4c0d382baab..b4e3bb5d75e17 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala @@ -42,7 +42,7 @@ class PartitionwiseSampledRDDPartition(val prev: Partition, val seed: Long) * @tparam T input RDD item type * @tparam U sampled RDD item type */ -class PartitionwiseSampledRDD[T: ClassTag, U: ClassTag]( +private[spark] class PartitionwiseSampledRDD[T: ClassTag, U: ClassTag]( prev: RDD[T], sampler: RandomSampler[T, U], @transient seed: Long = System.nanoTime) diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala index 4250a9d02f764..9ff4e5d33dd8e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala @@ -33,7 +33,7 @@ import org.apache.spark.{Partition, SparkEnv, TaskContext} * An RDD that pipes the contents of each parent partition through an external command * (printing them one per line) and returns the output as a collection of strings. */ -class PipedRDD[T: ClassTag]( +private[spark] class PipedRDD[T: ClassTag]( prev: RDD[T], command: Seq[String], envVars: Map[String, String], diff --git a/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala index 4ceea557f569c..b097c30f8c231 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala @@ -33,7 +33,7 @@ class SampledRDDPartition(val prev: Partition, val seed: Int) extends Partition } @deprecated("Replaced by PartitionwiseSampledRDD", "1.0.0") -class SampledRDD[T: ClassTag]( +private[spark] class SampledRDD[T: ClassTag]( prev: RDD[T], withReplacement: Boolean, frac: Double, 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 02660ea6a45c5..80843c5addff8 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -28,6 +28,8 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { } /** + * SEMI-PRIVATE + * * The resulting RDD from a shuffle (e.g. repartitioning of data). * @param prev the parent RDD. * @param part the partitioner used to partition the RDD diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala index a447030752096..c856f2b3db710 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -43,7 +43,7 @@ private[spark] class UnionPartition[T: ClassTag](idx: Int, rdd: RDD[T], splitInd } } -class UnionRDD[T: ClassTag]( +private[spark] class UnionRDD[T: ClassTag]( sc: SparkContext, @transient var rdds: Seq[RDD[T]]) extends RDD[T](sc, Nil) { // Nil since we implement getDependencies diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala index b56643444aa40..f3d30f6c9b32f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala @@ -41,7 +41,7 @@ private[spark] class ZippedPartitionsPartition( } } -abstract class ZippedPartitionsBaseRDD[V: ClassTag]( +private[spark] abstract class ZippedPartitionsBaseRDD[V: ClassTag]( sc: SparkContext, var rdds: Seq[RDD[_]], preservesPartitioning: Boolean = false) @@ -74,7 +74,7 @@ abstract class ZippedPartitionsBaseRDD[V: ClassTag]( } } -class ZippedPartitionsRDD2[A: ClassTag, B: ClassTag, V: ClassTag]( +private[spark] class ZippedPartitionsRDD2[A: ClassTag, B: ClassTag, V: ClassTag]( sc: SparkContext, f: (Iterator[A], Iterator[B]) => Iterator[V], var rdd1: RDD[A], @@ -94,7 +94,7 @@ class ZippedPartitionsRDD2[A: ClassTag, B: ClassTag, V: ClassTag]( } } -class ZippedPartitionsRDD3 +private[spark] class ZippedPartitionsRDD3 [A: ClassTag, B: ClassTag, C: ClassTag, V: ClassTag]( sc: SparkContext, f: (Iterator[A], Iterator[B], Iterator[C]) => Iterator[V], @@ -119,7 +119,7 @@ class ZippedPartitionsRDD3 } } -class ZippedPartitionsRDD4 +private[spark] class ZippedPartitionsRDD4 [A: ClassTag, B: ClassTag, C: ClassTag, D:ClassTag, V: ClassTag]( sc: SparkContext, f: (Iterator[A], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V], diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala index 2119e76f0e032..b8110ffc42f2d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala @@ -44,7 +44,7 @@ private[spark] class ZippedPartition[T: ClassTag, U: ClassTag]( } } -class ZippedRDD[T: ClassTag, U: ClassTag]( +private[spark] class ZippedRDD[T: ClassTag, U: ClassTag]( sc: SparkContext, var rdd1: RDD[T], var rdd2: RDD[U]) 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 d4eb0ac88d8e8..67f6ac9995926 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -27,17 +27,23 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.{Distribution, Utils} +/** SEMI-PRIVATE */ sealed trait SparkListenerEvent +/** SEMI-PRIVATE */ case class SparkListenerStageSubmitted(stageInfo: StageInfo, properties: Properties = null) extends SparkListenerEvent +/** SEMI-PRIVATE */ case class SparkListenerStageCompleted(stageInfo: StageInfo) extends SparkListenerEvent +/** SEMI-PRIVATE */ case class SparkListenerTaskStart(stageId: Int, taskInfo: TaskInfo) extends SparkListenerEvent +/** SEMI-PRIVATE */ case class SparkListenerTaskGettingResult(taskInfo: TaskInfo) extends SparkListenerEvent +/** SEMI-PRIVATE */ case class SparkListenerTaskEnd( stageId: Int, taskType: String, @@ -46,20 +52,26 @@ case class SparkListenerTaskEnd( taskMetrics: TaskMetrics) extends SparkListenerEvent +/** SEMI-PRIVATE */ case class SparkListenerJobStart(jobId: Int, stageIds: Seq[Int], properties: Properties = null) extends SparkListenerEvent +/** SEMI-PRIVATE */ case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkListenerEvent +/** SEMI-PRIVATE */ case class SparkListenerEnvironmentUpdate(environmentDetails: Map[String, Seq[(String, String)]]) extends SparkListenerEvent +/** SEMI-PRIVATE */ case class SparkListenerBlockManagerAdded(blockManagerId: BlockManagerId, maxMem: Long) extends SparkListenerEvent +/** SEMI-PRIVATE */ case class SparkListenerBlockManagerRemoved(blockManagerId: BlockManagerId) extends SparkListenerEvent +/** SEMI-PRIVATE */ case class SparkListenerUnpersistRDD(rddId: Int) extends SparkListenerEvent /** An event used in the listener to shutdown the listener daemon thread. */ @@ -67,7 +79,10 @@ private[spark] case object SparkListenerShutdown extends SparkListenerEvent /** - * Interface for listening to events from the Spark scheduler. + * SEMI-PRIVATE + * + * Interface for listening to events from the Spark scheduler. Note that this is an internal + * interface which might change in different Spark releases. */ trait SparkListener { /** @@ -128,6 +143,8 @@ trait SparkListener { } /** + * SEMI-PRIVATE + * * Simple SparkListener that logs a few summary statistics when each stage completes */ class StatsReportListener extends SparkListener with Logging { 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 18a68b05fa853..bd75d31601204 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -94,7 +94,13 @@ private[spark] class JavaSerializerInstance(counterReset: Int) extends Serialize } /** + * SEMI-PRIVATE + * * 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. */ 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/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 926e71573be32..d1e8c3ef63622 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -33,6 +33,10 @@ import org.apache.spark.storage.{GetBlock, GotBlock, PutBlock} /** * A Spark serializer that uses the [[https://code.google.com/p/kryo/ Kryo serialization library]]. + * + * 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. */ class KryoSerializer(conf: SparkConf) extends org.apache.spark.serializer.Serializer 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 099143494b851..c7fd92ce59548 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -26,15 +26,21 @@ import org.apache.spark.util.{ByteBufferInputStream, NextIterator} import org.apache.spark.SparkEnv /** + * SEMI-PRIVATE + * * 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. * * Implementations of this trait should implement: + * * 1. a zero-arg constructor or a constructor that accepts a [[org.apache.spark.SparkConf]] * as parameter. If both constructors are defined, the latter takes precedence. * * 2. Java serialization interface. + * + * 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. */ trait Serializer { def newInstance(): SerializerInstance @@ -49,6 +55,8 @@ object Serializer { /** + * SEMI-PRIVATE + * * An instance of a serializer, for use by one thread at a time. */ trait SerializerInstance { @@ -81,6 +89,8 @@ trait SerializerInstance { /** + * SEMI-PRIVATE + * * A stream for writing serialized objects. */ trait SerializationStream { @@ -98,6 +108,8 @@ trait SerializationStream { /** + * SEMI-PRIVATE + * * A stream for reading serialized objects. */ trait DeserializationStream { diff --git a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala index c3692f2fd929b..a118048ce2865 100644 --- a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala +++ b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala @@ -24,6 +24,8 @@ import scala.collection.JavaConverters._ import scala.collection.generic.Growable /** + * SEMI-PRIVATE + * * Bounded priority queue. This class wraps the original PriorityQueue * class and modifies it such that only the top K elements are retained. * The top K elements are defined by an implicit Ordering[A]. diff --git a/core/src/main/scala/org/apache/spark/util/CollectionsUtil.scala b/core/src/main/scala/org/apache/spark/util/CollectionsUtil.scala index db3db87e6618e..93235031f3ad5 100644 --- a/core/src/main/scala/org/apache/spark/util/CollectionsUtil.scala +++ b/core/src/main/scala/org/apache/spark/util/CollectionsUtil.scala @@ -22,7 +22,7 @@ import java.util import scala.Array import scala.reflect._ -object CollectionsUtils { +private[spark] object CollectionsUtils { def makeBinarySearch[K <% Ordered[K] : ClassTag] : (Array[K], K) => Int = { classTag[K] match { case ClassTag.Float => diff --git a/core/src/main/scala/org/apache/spark/util/Distribution.scala b/core/src/main/scala/org/apache/spark/util/Distribution.scala index 5b347555fe708..4a2a852334a38 100644 --- a/core/src/main/scala/org/apache/spark/util/Distribution.scala +++ b/core/src/main/scala/org/apache/spark/util/Distribution.scala @@ -29,7 +29,7 @@ import scala.collection.immutable.IndexedSeq * * Assumes you are giving it a non-empty set of data */ -class Distribution(val data: Array[Double], val startIdx: Int, val endIdx: Int) { +private[spark] class Distribution(val data: Array[Double], val startIdx: Int, val endIdx: Int) { require(startIdx < endIdx) def this(data: Traversable[Double]) = this(data.toArray, 0, data.size) java.util.Arrays.sort(data, startIdx, endIdx) diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index a0c07e32fdc98..b5f2ec6831d26 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -36,7 +36,7 @@ import org.apache.spark.io.CompressionCodec * @param compress Whether to compress output * @param overwrite Whether to overwrite existing files */ -class FileLogger( +private[spark] class FileLogger( logDir: String, conf: SparkConf = new SparkConf, outputBufferSize: Int = 8 * 1024, // 8 KB 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 a6b39247a54ca..482a1858e7fb3 100644 --- a/core/src/main/scala/org/apache/spark/util/MutablePair.scala +++ b/core/src/main/scala/org/apache/spark/util/MutablePair.scala @@ -18,6 +18,8 @@ package org.apache.spark.util /** + * SEMI-PRIVATE + * * A tuple of 2 elements. This can be used as an alternative to Scala's Tuple2 when we want to * minimize object allocation. * diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala index ddbd084ed7f01..a3b8ac77f0ffc 100644 --- a/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala @@ -34,7 +34,7 @@ import org.apache.spark.Logging * @param updateTimeStampOnGet When enabled, the timestamp of a pair will be * updated when it is accessed */ -class TimeStampedHashMap[A, B](updateTimeStampOnGet: Boolean = false) +private[spark] class TimeStampedHashMap[A, B](updateTimeStampOnGet: Boolean = false) extends Map[A, B]() with Logging { val internalMap = new ConcurrentHashMap[A, (B, Long)]() diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala index 19bece86b36b4..7cd8f28b12dd6 100644 --- a/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala +++ b/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala @@ -22,7 +22,7 @@ import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConversions import scala.collection.mutable.Set -class TimeStampedHashSet[A] extends Set[A] { +private[spark] class TimeStampedHashSet[A] extends Set[A] { val internalMap = new ConcurrentHashMap[A, Long]() def contains(key: A): Boolean = { 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 98569143ee1e3..768c0c31efb78 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 @@ -18,6 +18,8 @@ package org.apache.spark.util.random /** + * SEMI-PRIVATE + * * A class with pseudorandom behavior. */ trait Pseudorandom { 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 0f1fca4813ba9..406d32c4e0a1a 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 @@ -23,6 +23,8 @@ import cern.jet.random.Poisson import cern.jet.random.engine.DRand /** + * SEMI-PRIVATE + * * 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. @@ -40,6 +42,8 @@ trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable } /** + * SEMI-PRIVATE + * * A sampler based on Bernoulli trials. * * @param lb lower bound of the acceptance range @@ -66,7 +70,9 @@ class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) override def clone = new BernoulliSampler[T](lb, ub) } -/** +/** Ps + * SEMI-PRIVATE + * * A sampler based on values drawn from Poisson distribution. * * @param poisson a Poisson random number generator From 42c1f0991567a6b1f8f9c25ab946eee1ac404538 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 30 Mar 2014 15:46:18 -0700 Subject: [PATCH 02/31] Using better labels --- .../scala/org/apache/spark/Aggregator.scala | 2 +- .../scala/org/apache/spark/Dependency.scala | 10 +++---- .../scala/org/apache/spark/FutureAction.scala | 4 +-- .../scala/org/apache/spark/TaskContext.scala | 2 +- .../apache/spark/executor/TaskMetrics.scala | 6 ++-- .../apache/spark/io/CompressionCodec.scala | 6 ++-- .../apache/spark/partial/BoundedDouble.scala | 2 +- .../apache/spark/partial/PartialResult.scala | 2 +- .../apache/spark/rdd/AsyncRDDActions.scala | 2 +- .../org/apache/spark/rdd/HadoopRDD.scala | 2 +- .../org/apache/spark/rdd/NewHadoopRDD.scala | 2 +- .../spark/rdd/PartitionPruningRDD.scala | 4 +-- .../org/apache/spark/rdd/ShuffledRDD.scala | 2 +- .../spark/scheduler/SparkListener.scala | 28 +++++++++---------- .../spark/serializer/JavaSerializer.scala | 2 +- .../apache/spark/serializer/Serializer.scala | 8 +++--- .../spark/util/BoundedPriorityQueue.scala | 2 +- .../org/apache/spark/util/MutablePair.scala | 2 +- .../spark/util/random/Pseudorandom.scala | 2 +- .../spark/util/random/RandomSampler.scala | 6 ++-- 20 files changed, 48 insertions(+), 48 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index 09add7a497ca1..ec643251bd5e9 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -20,7 +20,7 @@ package org.apache.spark import org.apache.spark.util.collection.{AppendOnlyMap, ExternalAppendOnlyMap} /** - * SEMI-PRIVATE + * UNSTABLE DEVELOPER API * * A set of functions used to aggregate data. * diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index e23b7ad4a11b0..aae8409ec083c 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -21,7 +21,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer /** - * SEMI-PRIVATE + * UNSTABLE DEVELOPER API * * Base class for dependencies. */ @@ -29,7 +29,7 @@ abstract class Dependency[T](val rdd: RDD[T]) extends Serializable /** - * SEMI-PRIVATE + * UNSTABLE 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. @@ -45,7 +45,7 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { /** - * SEMI-PRIVATE + * UNSTABLE DEVELOPER API * * Represents a dependency on the output of a shuffle stage. * @param rdd the parent RDD @@ -65,7 +65,7 @@ class ShuffleDependency[K, V]( /** - * SEMI-PRIVATE + * UNSTABLE DEVELOPER API * * Represents a one-to-one dependency between partitions of the parent and child RDDs. */ @@ -75,7 +75,7 @@ class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) { /** - * SEMI-PRIVATE + * UNSTABLE DEVELOPER API * * Represents a one-to-one dependency between ranges of partitions in the parent and child RDDs. * @param rdd the parent RDD diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index 6eb5689b5004b..bea19913228fc 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -25,7 +25,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.{JobFailed, JobSucceeded, JobWaiter} /** - * EXPERIMENTAL + * EXPERIMENTAL API * * A future for the result of an action to support cancellation. This is an extension of the * Scala Future interface to support cancellation. @@ -150,7 +150,7 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: /** - * EXPERIMENTAL + * EXPERIMENTAL API * * 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 diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 928ad0830f6f8..d4f8fcf8f4320 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -22,7 +22,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.executor.TaskMetrics /** - * SEMI-PRIVATE + * UNSTABLE DEVELOPER API * * Contextual information about a task which can be read or mutated during execution. */ 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 bd63b1148d54b..656c9d337c637 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -20,7 +20,7 @@ package org.apache.spark.executor import org.apache.spark.storage.{BlockId, BlockStatus} /** - * SEMI-PRIVATE + * UNSTABLE DEVELOPER API * * Metrics tracked during the execution of a task. */ @@ -88,7 +88,7 @@ object TaskMetrics { /** - * SEMI-PRIVATE + * UNSTABLE DEVELOPER API * * Metrics pertaining to shuffle data read in a given task. */ @@ -127,7 +127,7 @@ class ShuffleReadMetrics extends Serializable { } /** - * SEMI-PRIVATE + * UNSTABLE DEVELOPER API * * Metrics pertaining to shuffle data written in a given 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 a7035b647a701..4f9e24f452d52 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -25,7 +25,7 @@ import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} import org.apache.spark.SparkConf /** - * SEMI-PRIVATE + * UNSTABLE DEVELOPER API * * CompressionCodec allows the customization of choosing different compression implementations * to be used in block storage. @@ -58,7 +58,7 @@ private[spark] object CompressionCodec { /** - * SEMI-PRIVATE + * UNSTABLE DEVELOPER API * * LZF implementation of [[org.apache.spark.io.CompressionCodec]]. * @@ -77,7 +77,7 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { /** - * SEMI-PRIVATE + * UNSTABLE DEVELOPER API * * Snappy implementation of [[org.apache.spark.io.CompressionCodec]]. * Block size can be configured by spark.io.compression.snappy.block.size. diff --git a/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala index a3ec79775e05f..0aa79de87bf99 100644 --- a/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala +++ b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala @@ -18,7 +18,7 @@ package org.apache.spark.partial /** - * EXPERIMENTAL + * EXPERIMENTAL API * * A Double value with error bars and associated confidence. */ 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 4d8f49b5a354b..2e2591973c665 100644 --- a/core/src/main/scala/org/apache/spark/partial/PartialResult.scala +++ b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala @@ -18,7 +18,7 @@ package org.apache.spark.partial /** - * EXPERIMENTAL + * EXPERIMENTAL API */ class PartialResult[R](initialVal: R, isFinal: Boolean) { private var finalValue: Option[R] = if (isFinal) Some(initialVal) else 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 e0e5d08ffe6fc..992b67dfff1af 100644 --- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala @@ -26,7 +26,7 @@ import scala.reflect.ClassTag import org.apache.spark.{ComplexFutureAction, FutureAction, Logging} /** - * EXPERIMENTAL + * EXPERIMENTAL API * * A set of asynchronous RDD actions available through an implicit conversion. * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions. diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 30970af325c6c..15d0aea052b1b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -70,7 +70,7 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp } /** - * SEMI-PRIVATE + * UNSTABLE 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`). 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 8a34292bf0e85..da178db4bd500 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -36,7 +36,7 @@ class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputS } /** - * SEMI-PRIVATE + * UNSTABLE 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`). 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 9f3fa08a5ccb8..f20cb20d66f61 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala @@ -46,7 +46,7 @@ private[spark] class PruneDependency[T](rdd: RDD[T], @transient partitionFilterF /** - * SEMI-PRIVATE + * UNSTABLE 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, @@ -67,7 +67,7 @@ class PartitionPruningRDD[T: ClassTag]( /** - * SEMI-PRIVATE + * UNSTABLE DEVELOPER API */ object PartitionPruningRDD { 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 80843c5addff8..72f9e7dff84cf 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -28,7 +28,7 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { } /** - * SEMI-PRIVATE + * UNSTABLE DEVELOPER API * * The resulting RDD from a shuffle (e.g. repartitioning of data). * @param prev the parent RDD. 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 67f6ac9995926..99ce7db947ea1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -27,23 +27,23 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.{Distribution, Utils} -/** SEMI-PRIVATE */ +/** UNSTABLE DEVELOPER API */ sealed trait SparkListenerEvent -/** SEMI-PRIVATE */ +/** UNSTABLE DEVELOPER API */ case class SparkListenerStageSubmitted(stageInfo: StageInfo, properties: Properties = null) extends SparkListenerEvent -/** SEMI-PRIVATE */ +/** UNSTABLE DEVELOPER API */ case class SparkListenerStageCompleted(stageInfo: StageInfo) extends SparkListenerEvent -/** SEMI-PRIVATE */ +/** UNSTABLE DEVELOPER API */ case class SparkListenerTaskStart(stageId: Int, taskInfo: TaskInfo) extends SparkListenerEvent -/** SEMI-PRIVATE */ +/** UNSTABLE DEVELOPER API */ case class SparkListenerTaskGettingResult(taskInfo: TaskInfo) extends SparkListenerEvent -/** SEMI-PRIVATE */ +/** UNSTABLE DEVELOPER API */ case class SparkListenerTaskEnd( stageId: Int, taskType: String, @@ -52,26 +52,26 @@ case class SparkListenerTaskEnd( taskMetrics: TaskMetrics) extends SparkListenerEvent -/** SEMI-PRIVATE */ +/** UNSTABLE DEVELOPER API */ case class SparkListenerJobStart(jobId: Int, stageIds: Seq[Int], properties: Properties = null) extends SparkListenerEvent -/** SEMI-PRIVATE */ +/** UNSTABLE DEVELOPER API */ case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkListenerEvent -/** SEMI-PRIVATE */ +/** UNSTABLE DEVELOPER API */ case class SparkListenerEnvironmentUpdate(environmentDetails: Map[String, Seq[(String, String)]]) extends SparkListenerEvent -/** SEMI-PRIVATE */ +/** UNSTABLE DEVELOPER API */ case class SparkListenerBlockManagerAdded(blockManagerId: BlockManagerId, maxMem: Long) extends SparkListenerEvent -/** SEMI-PRIVATE */ +/** UNSTABLE DEVELOPER API */ case class SparkListenerBlockManagerRemoved(blockManagerId: BlockManagerId) extends SparkListenerEvent -/** SEMI-PRIVATE */ +/** UNSTABLE DEVELOPER API */ case class SparkListenerUnpersistRDD(rddId: Int) extends SparkListenerEvent /** An event used in the listener to shutdown the listener daemon thread. */ @@ -79,7 +79,7 @@ private[spark] case object SparkListenerShutdown extends SparkListenerEvent /** - * SEMI-PRIVATE + * UNSTABLE 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. @@ -143,7 +143,7 @@ trait SparkListener { } /** - * SEMI-PRIVATE + * UNSTABLE DEVELOPER API * * Simple SparkListener that logs a few summary statistics when each stage completes */ 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 bd75d31601204..2d901735e1324 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -94,7 +94,7 @@ private[spark] class JavaSerializerInstance(counterReset: Int) extends Serialize } /** - * SEMI-PRIVATE + * UNSTABLE DEVELOPER API * * A Spark serializer that uses Java's built-in serialization. * 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 c7fd92ce59548..1133cc9104d10 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -26,7 +26,7 @@ import org.apache.spark.util.{ByteBufferInputStream, NextIterator} import org.apache.spark.SparkEnv /** - * SEMI-PRIVATE + * UNSTABLE 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 @@ -55,7 +55,7 @@ object Serializer { /** - * SEMI-PRIVATE + * UNSTABLE DEVELOPER API * * An instance of a serializer, for use by one thread at a time. */ @@ -89,7 +89,7 @@ trait SerializerInstance { /** - * SEMI-PRIVATE + * UNSTABLE DEVELOPER API * * A stream for writing serialized objects. */ @@ -108,7 +108,7 @@ trait SerializationStream { /** - * SEMI-PRIVATE + * UNSTABLE DEVELOPER API * * A stream for reading serialized objects. */ diff --git a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala index a118048ce2865..5767daf7f094a 100644 --- a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala +++ b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala @@ -24,7 +24,7 @@ import scala.collection.JavaConverters._ import scala.collection.generic.Growable /** - * SEMI-PRIVATE + * UNSTABLE DEVELOPER API * * Bounded priority queue. This class wraps the original PriorityQueue * class and modifies it such that only the top K elements are retained. 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 482a1858e7fb3..34fd30044170f 100644 --- a/core/src/main/scala/org/apache/spark/util/MutablePair.scala +++ b/core/src/main/scala/org/apache/spark/util/MutablePair.scala @@ -18,7 +18,7 @@ package org.apache.spark.util /** - * SEMI-PRIVATE + * UNSTABLE 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. 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 768c0c31efb78..80dd5deed8502 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 @@ -18,7 +18,7 @@ package org.apache.spark.util.random /** - * SEMI-PRIVATE + * UNSTABLE DEVELOPER API * * A class with pseudorandom behavior. */ 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 406d32c4e0a1a..57d1927b7d686 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 @@ -23,7 +23,7 @@ import cern.jet.random.Poisson import cern.jet.random.engine.DRand /** - * SEMI-PRIVATE + * UNSTABLE 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 @@ -42,7 +42,7 @@ trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable } /** - * SEMI-PRIVATE + * UNSTABLE DEVELOPER API * * A sampler based on Bernoulli trials. * @@ -71,7 +71,7 @@ class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) } /** Ps - * SEMI-PRIVATE + * UNSTABLE DEVELOPER API * * A sampler based on values drawn from Poisson distribution. * From 5a76ed6ee875c76d04c19a07c6f6c98edf8b4b70 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 30 Mar 2014 15:49:52 -0700 Subject: [PATCH 03/31] More visiblity clean-up --- .../scala/org/apache/spark/storage/BlockManager.scala | 8 ++++---- .../org/apache/spark/storage/StoragePerfTester.scala | 2 +- .../main/scala/org/apache/spark/util/Distribution.scala | 2 +- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 71584b6eb102a..ca8ca650528b7 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -35,11 +35,11 @@ import org.apache.spark.network._ import org.apache.spark.serializer.Serializer import org.apache.spark.util._ -sealed trait Values +private[spark] sealed trait Values -case class ByteBufferValues(buffer: ByteBuffer) extends Values -case class IteratorValues(iterator: Iterator[Any]) extends Values -case class ArrayBufferValues(buffer: ArrayBuffer[Any]) extends Values +private[spark] case class ByteBufferValues(buffer: ByteBuffer) extends Values +private[spark]case class IteratorValues(iterator: Iterator[Any]) extends Values +private[spark]case class ArrayBufferValues(buffer: ArrayBuffer[Any]) extends Values private[spark] class BlockManager( executorId: String, diff --git a/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala b/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala index 8cea302eb14c3..5ba0363a14b3d 100644 --- a/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala +++ b/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala @@ -25,7 +25,7 @@ import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.Utils /** - * Utility for micro-benchmarking shuffle write performance. + * Internal utility for micro-benchmarking shuffle write performance. * * Writes simulated shuffle output from several threads and records the observed throughput. */ diff --git a/core/src/main/scala/org/apache/spark/util/Distribution.scala b/core/src/main/scala/org/apache/spark/util/Distribution.scala index 4a2a852334a38..a465298c8c5ab 100644 --- a/core/src/main/scala/org/apache/spark/util/Distribution.scala +++ b/core/src/main/scala/org/apache/spark/util/Distribution.scala @@ -69,7 +69,7 @@ private[spark] class Distribution(val data: Array[Double], val startIdx: Int, va } } -object Distribution { +private[spark] object Distribution { def apply(data: Traversable[Double]): Option[Distribution] = { if (data.size > 0) { From d86818e57986998a747cfd0fb3bd68cd1814a675 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 30 Mar 2014 17:15:34 -0700 Subject: [PATCH 04/31] Another naming change --- .../scala/org/apache/spark/Aggregator.scala | 2 +- .../scala/org/apache/spark/Dependency.scala | 10 +++---- .../scala/org/apache/spark/TaskContext.scala | 2 +- .../apache/spark/executor/TaskMetrics.scala | 6 ++-- .../apache/spark/io/CompressionCodec.scala | 6 ++-- .../org/apache/spark/rdd/HadoopRDD.scala | 2 +- .../org/apache/spark/rdd/NewHadoopRDD.scala | 2 +- .../spark/rdd/PartitionPruningRDD.scala | 4 +-- .../org/apache/spark/rdd/ShuffledRDD.scala | 2 +- .../spark/scheduler/SparkListener.scala | 28 +++++++++---------- .../spark/serializer/JavaSerializer.scala | 2 +- .../apache/spark/serializer/Serializer.scala | 8 +++--- .../spark/util/BoundedPriorityQueue.scala | 2 +- .../org/apache/spark/util/MutablePair.scala | 2 +- .../spark/util/random/Pseudorandom.scala | 2 +- .../spark/util/random/RandomSampler.scala | 6 ++-- 16 files changed, 43 insertions(+), 43 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index ec643251bd5e9..97f4f36bd7c49 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -20,7 +20,7 @@ package org.apache.spark import org.apache.spark.util.collection.{AppendOnlyMap, ExternalAppendOnlyMap} /** - * UNSTABLE DEVELOPER API + * DEVELOPER API - UNSTABLE * * A set of functions used to aggregate data. * diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index aae8409ec083c..df5c798d9855c 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -21,7 +21,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer /** - * UNSTABLE DEVELOPER API + * DEVELOPER API - UNSTABLE * * Base class for dependencies. */ @@ -29,7 +29,7 @@ abstract class Dependency[T](val rdd: RDD[T]) extends Serializable /** - * UNSTABLE DEVELOPER API + * DEVELOPER API - UNSTABLE * * 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. @@ -45,7 +45,7 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { /** - * UNSTABLE DEVELOPER API + * DEVELOPER API - UNSTABLE * * Represents a dependency on the output of a shuffle stage. * @param rdd the parent RDD @@ -65,7 +65,7 @@ class ShuffleDependency[K, V]( /** - * UNSTABLE DEVELOPER API + * DEVELOPER API - UNSTABLE * * Represents a one-to-one dependency between partitions of the parent and child RDDs. */ @@ -75,7 +75,7 @@ class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) { /** - * UNSTABLE DEVELOPER API + * DEVELOPER API - UNSTABLE * * Represents a one-to-one dependency between ranges of partitions in the parent and child RDDs. * @param rdd the parent RDD diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index d4f8fcf8f4320..ddb4b5ce453ff 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -22,7 +22,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.executor.TaskMetrics /** - * UNSTABLE DEVELOPER API + * DEVELOPER API - UNSTABLE * * Contextual information about a task which can be read or mutated during execution. */ 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 656c9d337c637..31c871d6e3db7 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -20,7 +20,7 @@ package org.apache.spark.executor import org.apache.spark.storage.{BlockId, BlockStatus} /** - * UNSTABLE DEVELOPER API + * DEVELOPER API - UNSTABLE * * Metrics tracked during the execution of a task. */ @@ -88,7 +88,7 @@ object TaskMetrics { /** - * UNSTABLE DEVELOPER API + * DEVELOPER API - UNSTABLE * * Metrics pertaining to shuffle data read in a given task. */ @@ -127,7 +127,7 @@ class ShuffleReadMetrics extends Serializable { } /** - * UNSTABLE DEVELOPER API + * DEVELOPER API - UNSTABLE * * Metrics pertaining to shuffle data written in a given 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 4f9e24f452d52..d65022d2356fb 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -25,7 +25,7 @@ import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} import org.apache.spark.SparkConf /** - * UNSTABLE DEVELOPER API + * DEVELOPER API - UNSTABLE * * CompressionCodec allows the customization of choosing different compression implementations * to be used in block storage. @@ -58,7 +58,7 @@ private[spark] object CompressionCodec { /** - * UNSTABLE DEVELOPER API + * DEVELOPER API - UNSTABLE * * LZF implementation of [[org.apache.spark.io.CompressionCodec]]. * @@ -77,7 +77,7 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { /** - * UNSTABLE DEVELOPER API + * DEVELOPER API - UNSTABLE * * Snappy implementation of [[org.apache.spark.io.CompressionCodec]]. * Block size can be configured by spark.io.compression.snappy.block.size. diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 15d0aea052b1b..8483c13c989a1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -70,7 +70,7 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp } /** - * UNSTABLE DEVELOPER API + * DEVELOPER API - UNSTABLE * * An RDD that provides core functionality for reading data stored in Hadoop (e.g., files in HDFS, * sources in HBase, or S3), using the older MapReduce API (`org.apache.hadoop.mapred`). diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index da178db4bd500..71a055f05691f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -36,7 +36,7 @@ class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputS } /** - * UNSTABLE DEVELOPER API + * DEVELOPER API - UNSTABLE * * An RDD that provides core functionality for reading data stored in Hadoop (e.g., files in HDFS, * sources in HBase, or S3), using the new MapReduce API (`org.apache.hadoop.mapreduce`). diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala index f20cb20d66f61..ff9e9cde4225c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala @@ -46,7 +46,7 @@ private[spark] class PruneDependency[T](rdd: RDD[T], @transient partitionFilterF /** - * UNSTABLE DEVELOPER API + * DEVELOPER API - UNSTABLE * * 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, @@ -67,7 +67,7 @@ class PartitionPruningRDD[T: ClassTag]( /** - * UNSTABLE DEVELOPER API + * DEVELOPER API - UNSTABLE */ object PartitionPruningRDD { 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 72f9e7dff84cf..166b35d9508c0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -28,7 +28,7 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { } /** - * UNSTABLE DEVELOPER API + * DEVELOPER API - UNSTABLE * * The resulting RDD from a shuffle (e.g. repartitioning of data). * @param prev the parent RDD. 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 99ce7db947ea1..13a17f8fd84cf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -27,23 +27,23 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.{Distribution, Utils} -/** UNSTABLE DEVELOPER API */ +/** DEVELOPER API - UNSTABLE */ sealed trait SparkListenerEvent -/** UNSTABLE DEVELOPER API */ +/** DEVELOPER API - UNSTABLE */ case class SparkListenerStageSubmitted(stageInfo: StageInfo, properties: Properties = null) extends SparkListenerEvent -/** UNSTABLE DEVELOPER API */ +/** DEVELOPER API - UNSTABLE */ case class SparkListenerStageCompleted(stageInfo: StageInfo) extends SparkListenerEvent -/** UNSTABLE DEVELOPER API */ +/** DEVELOPER API - UNSTABLE */ case class SparkListenerTaskStart(stageId: Int, taskInfo: TaskInfo) extends SparkListenerEvent -/** UNSTABLE DEVELOPER API */ +/** DEVELOPER API - UNSTABLE */ case class SparkListenerTaskGettingResult(taskInfo: TaskInfo) extends SparkListenerEvent -/** UNSTABLE DEVELOPER API */ +/** DEVELOPER API - UNSTABLE */ case class SparkListenerTaskEnd( stageId: Int, taskType: String, @@ -52,26 +52,26 @@ case class SparkListenerTaskEnd( taskMetrics: TaskMetrics) extends SparkListenerEvent -/** UNSTABLE DEVELOPER API */ +/** DEVELOPER API - UNSTABLE */ case class SparkListenerJobStart(jobId: Int, stageIds: Seq[Int], properties: Properties = null) extends SparkListenerEvent -/** UNSTABLE DEVELOPER API */ +/** DEVELOPER API - UNSTABLE */ case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkListenerEvent -/** UNSTABLE DEVELOPER API */ +/** DEVELOPER API - UNSTABLE */ case class SparkListenerEnvironmentUpdate(environmentDetails: Map[String, Seq[(String, String)]]) extends SparkListenerEvent -/** UNSTABLE DEVELOPER API */ +/** DEVELOPER API - UNSTABLE */ case class SparkListenerBlockManagerAdded(blockManagerId: BlockManagerId, maxMem: Long) extends SparkListenerEvent -/** UNSTABLE DEVELOPER API */ +/** DEVELOPER API - UNSTABLE */ case class SparkListenerBlockManagerRemoved(blockManagerId: BlockManagerId) extends SparkListenerEvent -/** UNSTABLE DEVELOPER API */ +/** DEVELOPER API - UNSTABLE */ case class SparkListenerUnpersistRDD(rddId: Int) extends SparkListenerEvent /** An event used in the listener to shutdown the listener daemon thread. */ @@ -79,7 +79,7 @@ private[spark] case object SparkListenerShutdown extends SparkListenerEvent /** - * UNSTABLE DEVELOPER API + * DEVELOPER API - UNSTABLE * * Interface for listening to events from the Spark scheduler. Note that this is an internal * interface which might change in different Spark releases. @@ -143,7 +143,7 @@ trait SparkListener { } /** - * UNSTABLE DEVELOPER API + * DEVELOPER API - UNSTABLE * * Simple SparkListener that logs a few summary statistics when each stage completes */ 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 2d901735e1324..ee71ea6ec6550 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -94,7 +94,7 @@ private[spark] class JavaSerializerInstance(counterReset: Int) extends Serialize } /** - * UNSTABLE DEVELOPER API + * DEVELOPER API - UNSTABLE * * A Spark serializer that uses Java's built-in serialization. * 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 1133cc9104d10..3440fd4aa0ad2 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -26,7 +26,7 @@ import org.apache.spark.util.{ByteBufferInputStream, NextIterator} import org.apache.spark.SparkEnv /** - * UNSTABLE DEVELOPER API + * DEVELOPER API - UNSTABLE * * 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 @@ -55,7 +55,7 @@ object Serializer { /** - * UNSTABLE DEVELOPER API + * DEVELOPER API - UNSTABLE * * An instance of a serializer, for use by one thread at a time. */ @@ -89,7 +89,7 @@ trait SerializerInstance { /** - * UNSTABLE DEVELOPER API + * DEVELOPER API - UNSTABLE * * A stream for writing serialized objects. */ @@ -108,7 +108,7 @@ trait SerializationStream { /** - * UNSTABLE DEVELOPER API + * DEVELOPER API - UNSTABLE * * A stream for reading serialized objects. */ diff --git a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala index 5767daf7f094a..97c1ba15f04e9 100644 --- a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala +++ b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala @@ -24,7 +24,7 @@ import scala.collection.JavaConverters._ import scala.collection.generic.Growable /** - * UNSTABLE DEVELOPER API + * DEVELOPER API - UNSTABLE * * Bounded priority queue. This class wraps the original PriorityQueue * class and modifies it such that only the top K elements are retained. 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 34fd30044170f..27b6ccf18df73 100644 --- a/core/src/main/scala/org/apache/spark/util/MutablePair.scala +++ b/core/src/main/scala/org/apache/spark/util/MutablePair.scala @@ -18,7 +18,7 @@ package org.apache.spark.util /** - * UNSTABLE DEVELOPER API + * DEVELOPER API - UNSTABLE * * A tuple of 2 elements. This can be used as an alternative to Scala's Tuple2 when we want to * minimize object allocation. diff --git a/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala b/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala index 80dd5deed8502..a0863f41dc302 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 @@ -18,7 +18,7 @@ package org.apache.spark.util.random /** - * UNSTABLE DEVELOPER API + * DEVELOPER API - UNSTABLE * * A class with pseudorandom behavior. */ 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 57d1927b7d686..dfa6013c58083 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 @@ -23,7 +23,7 @@ import cern.jet.random.Poisson import cern.jet.random.engine.DRand /** - * UNSTABLE DEVELOPER API + * DEVELOPER API - UNSTABLE * * 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 @@ -42,7 +42,7 @@ trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable } /** - * UNSTABLE DEVELOPER API + * DEVELOPER API - UNSTABLE * * A sampler based on Bernoulli trials. * @@ -71,7 +71,7 @@ class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) } /** Ps - * UNSTABLE DEVELOPER API + * DEVELOPER API - UNSTABLE * * A sampler based on values drawn from Poisson distribution. * From cf23ec6be8a23dbd505ceb9fd4fa795cb3e7f2d0 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 30 Mar 2014 17:20:25 -0700 Subject: [PATCH 05/31] Marking GraphX as alpha --- graphx/src/main/scala/org/apache/spark/graphx/package.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/package.scala index 425a5164cad24..6f602a59cb5b8 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/package.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/package.scala @@ -19,7 +19,10 @@ package org.apache.spark import org.apache.spark.util.collection.OpenHashSet -/** GraphX is a graph processing framework built on top of Spark. */ +/** + * ALPHA COMPONENT + * + * GraphX is a graph processing framework built on top of Spark. */ package object graphx { /** * A 64-bit vertex identifier that uniquely identifies a vertex within a graph. It does not need From 542a736ad992301197a0ecd484c8cf258f46a32a Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 30 Mar 2014 17:25:25 -0700 Subject: [PATCH 06/31] Small fixes --- core/src/main/scala/org/apache/spark/SparkEnv.scala | 2 +- .../main/scala/org/apache/spark/storage/BlockManager.scala | 4 ++-- .../scala/org/apache/spark/util/random/RandomSampler.scala | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 36748766f377a..0e2965e714dd7 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -41,7 +41,7 @@ import org.apache.spark.util.{AkkaUtils, Utils} * 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. */ -private[spark] class SparkEnv private[spark] ( +private[spark] class SparkEnv ( val executorId: String, val actorSystem: ActorSystem, val serializer: Serializer, diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index ca8ca650528b7..f7ac1bddd3b8f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -38,8 +38,8 @@ import org.apache.spark.util._ private[spark] sealed trait Values private[spark] case class ByteBufferValues(buffer: ByteBuffer) extends Values -private[spark]case class IteratorValues(iterator: Iterator[Any]) extends Values -private[spark]case class ArrayBufferValues(buffer: ArrayBuffer[Any]) extends Values +private[spark] case class IteratorValues(iterator: Iterator[Any]) extends Values +private[spark] case class ArrayBufferValues(buffer: ArrayBuffer[Any]) extends Values private[spark] class BlockManager( executorId: String, 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 dfa6013c58083..1fb2f60a8cd9d 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 @@ -70,7 +70,7 @@ class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) override def clone = new BernoulliSampler[T](lb, ub) } -/** Ps +/** * DEVELOPER API - UNSTABLE * * A sampler based on values drawn from Poisson distribution. From 2f706f14a228400e7fed54a261bd78bdb67237d0 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 5 Apr 2014 13:54:41 -0700 Subject: [PATCH 07/31] Don't use floats --- .../scala/org/apache/spark/Aggregator.scala | 2 +- .../scala/org/apache/spark/Dependency.scala | 10 +++---- .../scala/org/apache/spark/FutureAction.scala | 4 +-- .../scala/org/apache/spark/TaskContext.scala | 2 +- .../apache/spark/executor/TaskMetrics.scala | 6 ++-- .../apache/spark/io/CompressionCodec.scala | 6 ++-- .../apache/spark/partial/BoundedDouble.scala | 2 +- .../apache/spark/partial/PartialResult.scala | 2 +- .../apache/spark/rdd/AsyncRDDActions.scala | 2 +- .../org/apache/spark/rdd/HadoopRDD.scala | 2 +- .../org/apache/spark/rdd/NewHadoopRDD.scala | 2 +- .../spark/rdd/PartitionPruningRDD.scala | 4 +-- .../org/apache/spark/rdd/ShuffledRDD.scala | 2 +- .../spark/scheduler/SparkListener.scala | 28 +++++++++---------- .../spark/serializer/JavaSerializer.scala | 2 +- .../apache/spark/serializer/Serializer.scala | 8 +++--- .../spark/util/BoundedPriorityQueue.scala | 2 +- .../org/apache/spark/util/MutablePair.scala | 2 +- .../spark/util/random/Pseudorandom.scala | 2 +- .../spark/util/random/RandomSampler.scala | 6 ++-- .../org/apache/spark/graphx/package.scala | 2 +- .../org/apache/spark/sql/SQLContext.scala | 2 +- .../org/apache/spark/sql/SchemaRDD.scala | 8 +++--- 23 files changed, 54 insertions(+), 54 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index 97f4f36bd7c49..32c0e8228705c 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -20,7 +20,7 @@ package org.apache.spark import org.apache.spark.util.collection.{AppendOnlyMap, ExternalAppendOnlyMap} /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API - UNSTABLE * * A set of functions used to aggregate data. * diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index df5c798d9855c..af51e6d177bf4 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -21,7 +21,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API - UNSTABLE * * Base class for dependencies. */ @@ -29,7 +29,7 @@ abstract class Dependency[T](val rdd: RDD[T]) extends Serializable /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API - UNSTABLE * * 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. @@ -45,7 +45,7 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API - UNSTABLE * * Represents a dependency on the output of a shuffle stage. * @param rdd the parent RDD @@ -65,7 +65,7 @@ class ShuffleDependency[K, V]( /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API - UNSTABLE * * Represents a one-to-one dependency between partitions of the parent and child RDDs. */ @@ -75,7 +75,7 @@ class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) { /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API - UNSTABLE * * Represents a one-to-one dependency between ranges of partitions in the parent and child RDDs. * @param rdd the parent RDD diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index bea19913228fc..392150e682c47 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -25,7 +25,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.{JobFailed, JobSucceeded, JobWaiter} /** - * EXPERIMENTAL API + * EXPERIMENTAL API * * A future for the result of an action to support cancellation. This is an extension of the * Scala Future interface to support cancellation. @@ -150,7 +150,7 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: /** - * EXPERIMENTAL API + * EXPERIMENTAL API * * 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 diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index ddb4b5ce453ff..f1f5bf5ea8d75 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -22,7 +22,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.executor.TaskMetrics /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API - UNSTABLE * * Contextual information about a task which can be read or mutated during execution. */ 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 31c871d6e3db7..7f44886ef569d 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -20,7 +20,7 @@ package org.apache.spark.executor import org.apache.spark.storage.{BlockId, BlockStatus} /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API - UNSTABLE * * Metrics tracked during the execution of a task. */ @@ -88,7 +88,7 @@ object TaskMetrics { /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API - UNSTABLE * * Metrics pertaining to shuffle data read in a given task. */ @@ -127,7 +127,7 @@ class ShuffleReadMetrics extends Serializable { } /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API - UNSTABLE * * Metrics pertaining to shuffle data written in a given 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 d65022d2356fb..cdcbaf4f0e947 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -25,7 +25,7 @@ import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} import org.apache.spark.SparkConf /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API - UNSTABLE * * CompressionCodec allows the customization of choosing different compression implementations * to be used in block storage. @@ -58,7 +58,7 @@ private[spark] object CompressionCodec { /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API - UNSTABLE * * LZF implementation of [[org.apache.spark.io.CompressionCodec]]. * @@ -77,7 +77,7 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API - UNSTABLE * * Snappy implementation of [[org.apache.spark.io.CompressionCodec]]. * Block size can be configured by spark.io.compression.snappy.block.size. diff --git a/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala index 0aa79de87bf99..811a771f007e9 100644 --- a/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala +++ b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala @@ -18,7 +18,7 @@ package org.apache.spark.partial /** - * EXPERIMENTAL API + * EXPERIMENTAL API * * A Double value with error bars and associated confidence. */ 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 2e2591973c665..46095dc69968f 100644 --- a/core/src/main/scala/org/apache/spark/partial/PartialResult.scala +++ b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala @@ -18,7 +18,7 @@ package org.apache.spark.partial /** - * EXPERIMENTAL API + * EXPERIMENTAL API */ class PartialResult[R](initialVal: R, isFinal: Boolean) { private var finalValue: Option[R] = if (isFinal) Some(initialVal) else 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 992b67dfff1af..6b6efa1f26893 100644 --- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala @@ -26,7 +26,7 @@ import scala.reflect.ClassTag import org.apache.spark.{ComplexFutureAction, FutureAction, Logging} /** - * EXPERIMENTAL API + * EXPERIMENTAL API * * A set of asynchronous RDD actions available through an implicit conversion. * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions. diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 8483c13c989a1..818bfd46150c8 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -70,7 +70,7 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp } /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API - UNSTABLE * * An RDD that provides core functionality for reading data stored in Hadoop (e.g., files in HDFS, * sources in HBase, or S3), using the older MapReduce API (`org.apache.hadoop.mapred`). diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 71a055f05691f..75f82163ef50b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -36,7 +36,7 @@ class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputS } /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API - UNSTABLE * * An RDD that provides core functionality for reading data stored in Hadoop (e.g., files in HDFS, * sources in HBase, or S3), using the new MapReduce API (`org.apache.hadoop.mapreduce`). diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala index ff9e9cde4225c..46f44c129ba5a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala @@ -46,7 +46,7 @@ private[spark] class PruneDependency[T](rdd: RDD[T], @transient partitionFilterF /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API - UNSTABLE * * 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, @@ -67,7 +67,7 @@ class PartitionPruningRDD[T: ClassTag]( /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API - UNSTABLE */ object PartitionPruningRDD { 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 166b35d9508c0..c94f67717448b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -28,7 +28,7 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { } /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API - UNSTABLE * * The resulting RDD from a shuffle (e.g. repartitioning of data). * @param prev the parent RDD. 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 13a17f8fd84cf..af882924213de 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -27,23 +27,23 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.{Distribution, Utils} -/** DEVELOPER API - UNSTABLE */ +/** DEVELOPER API - UNSTABLE */ sealed trait SparkListenerEvent -/** DEVELOPER API - UNSTABLE */ +/** DEVELOPER API - UNSTABLE */ case class SparkListenerStageSubmitted(stageInfo: StageInfo, properties: Properties = null) extends SparkListenerEvent -/** DEVELOPER API - UNSTABLE */ +/** DEVELOPER API - UNSTABLE */ case class SparkListenerStageCompleted(stageInfo: StageInfo) extends SparkListenerEvent -/** DEVELOPER API - UNSTABLE */ +/** DEVELOPER API - UNSTABLE */ case class SparkListenerTaskStart(stageId: Int, taskInfo: TaskInfo) extends SparkListenerEvent -/** DEVELOPER API - UNSTABLE */ +/** DEVELOPER API - UNSTABLE */ case class SparkListenerTaskGettingResult(taskInfo: TaskInfo) extends SparkListenerEvent -/** DEVELOPER API - UNSTABLE */ +/** DEVELOPER API - UNSTABLE */ case class SparkListenerTaskEnd( stageId: Int, taskType: String, @@ -52,26 +52,26 @@ case class SparkListenerTaskEnd( taskMetrics: TaskMetrics) extends SparkListenerEvent -/** DEVELOPER API - UNSTABLE */ +/** DEVELOPER API - UNSTABLE */ case class SparkListenerJobStart(jobId: Int, stageIds: Seq[Int], properties: Properties = null) extends SparkListenerEvent -/** DEVELOPER API - UNSTABLE */ +/** DEVELOPER API - UNSTABLE */ case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkListenerEvent -/** DEVELOPER API - UNSTABLE */ +/** DEVELOPER API - UNSTABLE */ case class SparkListenerEnvironmentUpdate(environmentDetails: Map[String, Seq[(String, String)]]) extends SparkListenerEvent -/** DEVELOPER API - UNSTABLE */ +/** DEVELOPER API - UNSTABLE */ case class SparkListenerBlockManagerAdded(blockManagerId: BlockManagerId, maxMem: Long) extends SparkListenerEvent -/** DEVELOPER API - UNSTABLE */ +/** DEVELOPER API - UNSTABLE */ case class SparkListenerBlockManagerRemoved(blockManagerId: BlockManagerId) extends SparkListenerEvent -/** DEVELOPER API - UNSTABLE */ +/** DEVELOPER API - UNSTABLE */ case class SparkListenerUnpersistRDD(rddId: Int) extends SparkListenerEvent /** An event used in the listener to shutdown the listener daemon thread. */ @@ -79,7 +79,7 @@ private[spark] case object SparkListenerShutdown extends SparkListenerEvent /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API - UNSTABLE * * Interface for listening to events from the Spark scheduler. Note that this is an internal * interface which might change in different Spark releases. @@ -143,7 +143,7 @@ trait SparkListener { } /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API - UNSTABLE * * Simple SparkListener that logs a few summary statistics when each stage completes */ 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 ee71ea6ec6550..df9b1ab8c8931 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -94,7 +94,7 @@ private[spark] class JavaSerializerInstance(counterReset: Int) extends Serialize } /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API - UNSTABLE * * A Spark serializer that uses Java's built-in serialization. * 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 3440fd4aa0ad2..d423d2e61a865 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -26,7 +26,7 @@ import org.apache.spark.util.{ByteBufferInputStream, NextIterator} import org.apache.spark.SparkEnv /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API - UNSTABLE * * 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 @@ -55,7 +55,7 @@ object Serializer { /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API - UNSTABLE * * An instance of a serializer, for use by one thread at a time. */ @@ -89,7 +89,7 @@ trait SerializerInstance { /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API - UNSTABLE * * A stream for writing serialized objects. */ @@ -108,7 +108,7 @@ trait SerializationStream { /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API - UNSTABLE * * A stream for reading serialized objects. */ diff --git a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala index 97c1ba15f04e9..6423d6948f2d7 100644 --- a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala +++ b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala @@ -24,7 +24,7 @@ import scala.collection.JavaConverters._ import scala.collection.generic.Growable /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API - UNSTABLE * * Bounded priority queue. This class wraps the original PriorityQueue * class and modifies it such that only the top K elements are retained. 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 27b6ccf18df73..db09d21bd4ec7 100644 --- a/core/src/main/scala/org/apache/spark/util/MutablePair.scala +++ b/core/src/main/scala/org/apache/spark/util/MutablePair.scala @@ -18,7 +18,7 @@ package org.apache.spark.util /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API - UNSTABLE * * A tuple of 2 elements. This can be used as an alternative to Scala's Tuple2 when we want to * minimize object allocation. diff --git a/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala b/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala index a0863f41dc302..dac0d7d8a3b3c 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 @@ -18,7 +18,7 @@ package org.apache.spark.util.random /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API - UNSTABLE * * A class with pseudorandom behavior. */ 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 1fb2f60a8cd9d..479ca086b979c 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 @@ -23,7 +23,7 @@ import cern.jet.random.Poisson import cern.jet.random.engine.DRand /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API - UNSTABLE * * 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 @@ -42,7 +42,7 @@ trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable } /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API - UNSTABLE * * A sampler based on Bernoulli trials. * @@ -71,7 +71,7 @@ class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) } /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API - UNSTABLE * * A sampler based on values drawn from Poisson distribution. * diff --git a/graphx/src/main/scala/org/apache/spark/graphx/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/package.scala index 6f602a59cb5b8..6d0e3cde812b1 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/package.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/package.scala @@ -20,7 +20,7 @@ package org.apache.spark import org.apache.spark.util.collection.OpenHashSet /** - * ALPHA COMPONENT + * ALPHA COMPONENT * * GraphX is a graph processing framework built on top of Spark. */ package object graphx { 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 cf3c06acce5b0..bace60f59cd22 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 @@ -62,7 +62,7 @@ class SQLContext(@transient val sparkContext: SparkContext) new this.QueryExecution { val logical = plan } /** - * EXPERIMENTAL + * 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 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 770cabcb31d13..57a37e138e9ff 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 @@ -241,7 +241,7 @@ class SchemaRDD( Filter(ScalaUdf(udf, BooleanType, Seq(UnresolvedAttribute(arg1.name))), logicalPlan)) /** - * EXPERIMENTAL + * 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 @@ -260,7 +260,7 @@ class SchemaRDD( Filter(ScalaUdf(dynamicUdf, BooleanType, Seq(WrapDynamic(logicalPlan.output))), logicalPlan)) /** - * EXPERIMENTAL + * EXPERIMENTAL * * Returns a sampled version of the underlying dataset. * @@ -273,7 +273,7 @@ class SchemaRDD( new SchemaRDD(sqlContext, Sample(fraction, withReplacement, seed, logicalPlan)) /** - * EXPERIMENTAL + * EXPERIMENTAL * * Applies the given Generator, or table generating function, to this relation. * @@ -298,7 +298,7 @@ class SchemaRDD( new SchemaRDD(sqlContext, Generate(generator, join, outer, None, logicalPlan)) /** - * EXPERIMENTAL + * 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 From cd7a465e864670dc3f5311101679261ba202ac5f Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 5 Apr 2014 15:47:12 -0700 Subject: [PATCH 08/31] Code review feedback --- .../src/main/scala/org/apache/spark/Aggregator.scala | 2 +- .../main/scala/org/apache/spark/FutureAction.scala | 2 ++ .../org/apache/spark/InterruptibleIterator.scala | 2 +- .../main/scala/org/apache/spark/SparkContext.scala | 11 ++++++++++- .../apache/spark/broadcast/BroadcastFactory.scala | 2 ++ core/src/main/scala/org/apache/spark/package.scala | 9 +++++++++ .../scala/org/apache/spark/rdd/CoGroupedRDD.scala | 8 ++++++-- .../main/scala/org/apache/spark/rdd/EmptyRDD.scala | 2 +- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 12 ++++++++++-- .../scala/org/apache/spark/scheduler/JobLogger.scala | 2 ++ .../scala/org/apache/spark/scheduler/SplitInfo.scala | 1 + .../org/apache/spark/util/BoundedPriorityQueue.scala | 4 +--- .../main/scala/org/apache/spark/util/Vector.scala | 1 + .../main/scala/org/apache/spark/graphx/package.scala | 3 ++- .../org/apache/spark/tools}/StoragePerfTester.scala | 2 +- 15 files changed, 50 insertions(+), 13 deletions(-) rename {core/src/main/scala/org/apache/spark/storage => tools/src/main/scala/org/apache/spark/tools}/StoragePerfTester.scala (99%) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index 32c0e8228705c..f31a6318f69a6 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -21,13 +21,13 @@ import org.apache.spark.util.collection.{AppendOnlyMap, ExternalAppendOnlyMap} /** * DEVELOPER API - UNSTABLE - * * 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. */ + case class Aggregator[K, V, C] ( createCombiner: V => C, mergeValue: (C, V) => C, diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index 392150e682c47..53ec7d129d7fa 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -86,6 +86,8 @@ trait FutureAction[T] extends Future[T] { /** + * EXPERIMENTAL API + * * A [[FutureAction]] holding the result of an action that triggers a single job. Examples include * count, collect, reduce. */ diff --git a/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala b/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala index 9b1601d5b95fa..fd1802ba2f984 100644 --- a/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala +++ b/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala @@ -21,7 +21,7 @@ package org.apache.spark * An iterator that wraps around an existing iterator to provide task killing functionality. * It works by checking the interrupted flag in [[TaskContext]]. */ -class InterruptibleIterator[+T](val context: TaskContext, val delegate: Iterator[T]) +private[spark] class InterruptibleIterator[+T](val context: TaskContext, val delegate: Iterator[T]) extends Iterator[T] { def hasNext: Boolean = !context.interrupted && delegate.hasNext diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index b23accbbb9410..e579c503362b5 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -184,7 +184,7 @@ class SparkContext( jars.foreach(addJar) } - def warnSparkMem(value: String): String = { + private def warnSparkMem(value: String): String = { logWarning("Using SPARK_MEM to set amount of memory to use per executor process is " + "deprecated, please use spark.executor.memory instead.") value @@ -665,6 +665,11 @@ class SparkContext( postEnvironmentUpdate() } + /** + * DEVELOPER API - UNSTABLE + * + * Register a listener to receive up-calls from events that happen during execution. + */ def addSparkListener(listener: SparkListener) { listenerBus.addListener(listener) } @@ -974,6 +979,8 @@ class SparkContext( } /** + * DEVELOPER API - UNSTABLE + * * Run a job that can return approximate results. */ def runApproximateJob[T, U, R]( @@ -991,6 +998,8 @@ class SparkContext( } /** + * EXPERIMENTAL API + * * Submit a job for execution and return a FutureJob holding the result. */ def submitJob[T, U, R]( 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 6beecaeced5be..dda8387a16cbe 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala @@ -21,6 +21,8 @@ import org.apache.spark.SecurityManager import org.apache.spark.SparkConf /** + * DEVELOPER API - UNSTABLE + * * 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 diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala index 2625a7f6a575a..344397cb5b2af 100644 --- a/core/src/main/scala/org/apache/spark/package.scala +++ b/core/src/main/scala/org/apache/spark/package.scala @@ -32,6 +32,15 @@ package org.apache * * Java programmers should reference the [[spark.api.java]] package * for Spark programming APIs in Java. + * + * Classes and methods marked with EXPERIMENTAL API are + * user-facing features which have not been officially adopted by the Spark project. These are + * subject to change or removal in minor releases. + * + * Classes and methods marked with DEVELOPER API - UNSTABLE + * are intended for advanced users want to extend Spark through lower level interfaces. These are + * subject to changes or removal in minor releases. + * */ package object spark { // For package docs only 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 ad35f43dd9825..408071262d158 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -51,13 +51,17 @@ private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep] } /** + * DEVELOPER API - UNSTABLE + * * 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. * + * Note: This is an internal API. We recommend users use RDD.coGroup(...) instead of + * instantiating this directly. + * @param rdds parent RDDs. - * @param part partitioner used to partition the shuffle output. + * @param part partitioner used to partition the shuffle output */ -private[spark] 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/EmptyRDD.scala b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala index 6f28ac3ed37f6..e4bd9ebf4fb68 100644 --- a/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.{Partition, SparkContext, TaskContext} /** - * An RDD that is empty, i.e. has no element in it. + * An RDD that has no partitions and no elements.. */ private[spark] class EmptyRDD[T: ClassTag](sc: SparkContext) extends RDD[T](sc, Nil) { 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 ce2b8ac27206b..e269deefa9d17 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -513,6 +513,8 @@ abstract class RDD[T: ClassTag]( } /** + * DEVELOPER API - UNSTABLE + * * 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. */ @@ -775,7 +777,9 @@ 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 + * EXPERIMENTAL API + * + * Approximate version of count() that returns a potentially incomplete result * within a timeout, even if not all tasks have finished. */ def countApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { @@ -821,7 +825,9 @@ abstract class RDD[T: ClassTag]( } /** - * (Experimental) Approximate version of countByValue(). + * EXPERIMENTAL API + * + * Approximate version of countByValue(). */ def countByValueApprox( timeout: Long, @@ -843,6 +849,8 @@ abstract class RDD[T: ClassTag]( } /** + * EXPERIMENTAL API + * * Return approximate number of distinct elements in the RDD. * * The accuracy of approximation can be controlled through the relative standard deviation diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 5cecf9416b32c..8fcc9c9d855b0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -28,6 +28,8 @@ import org.apache.spark._ import org.apache.spark.executor.TaskMetrics /** + * DEVELOPER API - UNSTABLE + * * 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 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 5b40a3eb29b30..f742291deb9e9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala @@ -21,6 +21,7 @@ import collection.mutable.ArrayBuffer // information about a specific split instance : handles both split instances. // So that we do not need to worry about the differences. +private[spark] 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/util/BoundedPriorityQueue.scala b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala index 6423d6948f2d7..b9f4a5d720b93 100644 --- a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala +++ b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala @@ -24,13 +24,11 @@ import scala.collection.JavaConverters._ import scala.collection.generic.Growable /** - * DEVELOPER API - UNSTABLE - * * Bounded priority queue. This class wraps the original PriorityQueue * class and modifies it such that only the top K elements are retained. * The top K elements are defined by an implicit Ordering[A]. */ -class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Ordering[A]) +private[spark] class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Ordering[A]) extends Iterable[A] with Growable[A] with Serializable { private val underlying = new JPriorityQueue[A](maxSize, ord) diff --git a/core/src/main/scala/org/apache/spark/util/Vector.scala b/core/src/main/scala/org/apache/spark/util/Vector.scala index dc4b8f253f259..643366db9c78b 100644 --- a/core/src/main/scala/org/apache/spark/util/Vector.scala +++ b/core/src/main/scala/org/apache/spark/util/Vector.scala @@ -21,6 +21,7 @@ import scala.util.Random import org.apache.spark.util.random.XORShiftRandom +@deprecated("Use Vector from Spark's mllib.linalg package instead.", "1.0.0") class Vector(val elements: Array[Double]) extends Serializable { def length = elements.length diff --git a/graphx/src/main/scala/org/apache/spark/graphx/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/package.scala index 6d0e3cde812b1..099e19a161a40 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/package.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/package.scala @@ -22,7 +22,8 @@ import org.apache.spark.util.collection.OpenHashSet /** * ALPHA COMPONENT * - * GraphX is a graph processing framework built on top of Spark. */ + * GraphX is a graph processing framework built on top of Spark. + */ package object graphx { /** * A 64-bit vertex identifier that uniquely identifies a vertex within a graph. It does not need diff --git a/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala similarity index 99% rename from core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala rename to tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala index 5ba0363a14b3d..8e8c35615a711 100644 --- a/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala +++ b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.storage +package org.apache.spark.tools import java.util.concurrent.{CountDownLatch, Executors} import java.util.concurrent.atomic.AtomicLong From 1ed27d211d39dbdaca22e68659fafe8ae62348fc Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 5 Apr 2014 15:53:50 -0700 Subject: [PATCH 09/31] Formatting and coloring of badges --- .../scala/org/apache/spark/Aggregator.scala | 2 +- .../scala/org/apache/spark/Dependency.scala | 15 ++++------ .../scala/org/apache/spark/FutureAction.scala | 9 ++---- .../scala/org/apache/spark/SparkContext.scala | 6 ++-- .../scala/org/apache/spark/TaskContext.scala | 3 +- .../spark/broadcast/BroadcastFactory.scala | 3 +- .../apache/spark/executor/TaskMetrics.scala | 9 ++---- .../apache/spark/io/CompressionCodec.scala | 9 ++---- .../main/scala/org/apache/spark/package.scala | 16 +++++----- .../apache/spark/partial/BoundedDouble.scala | 3 +- .../apache/spark/partial/PartialResult.scala | 2 +- .../apache/spark/rdd/AsyncRDDActions.scala | 3 +- .../org/apache/spark/rdd/CoGroupedRDD.scala | 3 +- .../org/apache/spark/rdd/HadoopRDD.scala | 3 +- .../org/apache/spark/rdd/NewHadoopRDD.scala | 3 +- .../spark/rdd/PartitionPruningRDD.scala | 5 ++-- .../main/scala/org/apache/spark/rdd/RDD.scala | 20 ++++++++----- .../org/apache/spark/rdd/ShuffledRDD.scala | 3 +- .../apache/spark/scheduler/JobLogger.scala | 3 +- .../spark/scheduler/SparkListener.scala | 30 +++++++++---------- .../spark/serializer/JavaSerializer.scala | 3 +- .../apache/spark/serializer/Serializer.scala | 12 +++----- .../org/apache/spark/util/MutablePair.scala | 3 +- .../spark/util/random/Pseudorandom.scala | 3 +- .../spark/util/random/RandomSampler.scala | 9 ++---- .../org/apache/spark/graphx/package.scala | 3 +- .../org/apache/spark/sql/SQLContext.scala | 6 ++-- .../org/apache/spark/sql/SchemaRDD.scala | 15 ++++------ 28 files changed, 83 insertions(+), 121 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index f31a6318f69a6..ad00b838e14a2 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -20,7 +20,7 @@ package org.apache.spark import org.apache.spark.util.collection.{AppendOnlyMap, ExternalAppendOnlyMap} /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API * A set of functions used to aggregate data. * * @param createCombiner function to create the initial value of the aggregation. diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index af51e6d177bf4..0d65a1ab2f9a4 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -21,16 +21,14 @@ import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer /** - * DEVELOPER API - UNSTABLE - * + * DEVELOPER API * Base class for dependencies. */ abstract class Dependency[T](val rdd: RDD[T]) extends Serializable /** - * DEVELOPER API - UNSTABLE - * + * 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. */ @@ -45,8 +43,7 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { /** - * DEVELOPER API - UNSTABLE - * + * 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 @@ -65,8 +62,7 @@ class ShuffleDependency[K, V]( /** - * DEVELOPER API - UNSTABLE - * + * DEVELOPER API * Represents a one-to-one dependency between partitions of the parent and child RDDs. */ class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) { @@ -75,8 +71,7 @@ class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) { /** - * DEVELOPER API - UNSTABLE - * + * 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 diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index 53ec7d129d7fa..0d5bad9e91e91 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -25,8 +25,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.{JobFailed, JobSucceeded, JobWaiter} /** - * EXPERIMENTAL API - * + * EXPERIMENTAL API * A future for the result of an action to support cancellation. This is an extension of the * Scala Future interface to support cancellation. */ @@ -86,8 +85,7 @@ trait FutureAction[T] extends Future[T] { /** - * EXPERIMENTAL API - * + * EXPERIMENTAL API * A [[FutureAction]] holding the result of an action that triggers a single job. Examples include * count, collect, reduce. */ @@ -152,8 +150,7 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: /** - * EXPERIMENTAL API - * + * EXPERIMENTAL API * A [[FutureAction]] for actions that could trigger multiple Spark jobs. Examples include take, * takeSample. Cancellation works by setting the cancelled flag to true and interrupting the * action thread if it is being blocked by a job. diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index e579c503362b5..40d3b67b684b6 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -666,7 +666,7 @@ class SparkContext( } /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API * * Register a listener to receive up-calls from events that happen during execution. */ @@ -979,7 +979,7 @@ class SparkContext( } /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API * * Run a job that can return approximate results. */ @@ -998,7 +998,7 @@ class SparkContext( } /** - * EXPERIMENTAL API + * EXPERIMENTAL API * * Submit a job for execution and return a FutureJob holding the result. */ diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index f1f5bf5ea8d75..97de2b7f7717a 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -22,8 +22,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.executor.TaskMetrics /** - * DEVELOPER API - UNSTABLE - * + * DEVELOPER API * Contextual information about a task which can be read or mutated during execution. */ class TaskContext( 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 dda8387a16cbe..bcb8203f2d862 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala @@ -21,8 +21,7 @@ import org.apache.spark.SecurityManager import org.apache.spark.SparkConf /** - * DEVELOPER API - UNSTABLE - * + * 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 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 7f44886ef569d..acf0a543f2d62 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -20,8 +20,7 @@ package org.apache.spark.executor import org.apache.spark.storage.{BlockId, BlockStatus} /** - * DEVELOPER API - UNSTABLE - * + * DEVELOPER API * Metrics tracked during the execution of a task. */ class TaskMetrics extends Serializable { @@ -88,8 +87,7 @@ object TaskMetrics { /** - * DEVELOPER API - UNSTABLE - * + * DEVELOPER API * Metrics pertaining to shuffle data read in a given task. */ class ShuffleReadMetrics extends Serializable { @@ -127,8 +125,7 @@ class ShuffleReadMetrics extends Serializable { } /** - * DEVELOPER API - UNSTABLE - * + * DEVELOPER API * Metrics pertaining to shuffle data written in a given task. */ class ShuffleWriteMetrics extends Serializable { 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 cdcbaf4f0e947..e872b487b414e 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -25,8 +25,7 @@ import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} import org.apache.spark.SparkConf /** - * DEVELOPER API - UNSTABLE - * + * DEVELOPER API * CompressionCodec allows the customization of choosing different compression implementations * to be used in block storage. * @@ -58,8 +57,7 @@ private[spark] object CompressionCodec { /** - * DEVELOPER API - UNSTABLE - * + * 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 @@ -77,8 +75,7 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { /** - * DEVELOPER API - UNSTABLE - * + * DEVELOPER API * Snappy implementation of [[org.apache.spark.io.CompressionCodec]]. * Block size can be configured by spark.io.compression.snappy.block.size. * diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala index 344397cb5b2af..0c98f341f7543 100644 --- a/core/src/main/scala/org/apache/spark/package.scala +++ b/core/src/main/scala/org/apache/spark/package.scala @@ -33,15 +33,15 @@ package org.apache * Java programmers should reference the [[spark.api.java]] package * for Spark programming APIs in Java. * - * Classes and methods marked with EXPERIMENTAL API are - * user-facing features which have not been officially adopted by the Spark project. These are - * subject to change or removal in minor releases. - * - * Classes and methods marked with DEVELOPER API - UNSTABLE - * are intended for advanced users want to extend Spark through lower level interfaces. These are - * subject to changes or removal in minor releases. + * Classes and methods marked with + * EXPERIMENTAL API are user-facing features which have not been officially adopted by the + * Spark project. These are subject to change or removal in minor releases. * + * Classes and methods marked with + * DEVELOPER API are intended for advanced users want to extend Spark through lower + * level interfaces. These are subject to changes or removal in minor releases. */ + package object spark { // For package docs only -} +} \ No newline at end of file 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 811a771f007e9..aee6c3c85cbea 100644 --- a/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala +++ b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala @@ -18,8 +18,7 @@ package org.apache.spark.partial /** - * EXPERIMENTAL API - * + * EXPERIMENTAL API * A Double value with error bars and associated confidence. */ class BoundedDouble(val mean: Double, val confidence: Double, val low: Double, val high: Double) { 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 46095dc69968f..f055543847dfc 100644 --- a/core/src/main/scala/org/apache/spark/partial/PartialResult.scala +++ b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala @@ -18,7 +18,7 @@ package org.apache.spark.partial /** - * EXPERIMENTAL API + * EXPERIMENTAL API */ class PartialResult[R](initialVal: R, isFinal: Boolean) { private var finalValue: Option[R] = if (isFinal) Some(initialVal) else 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 6b6efa1f26893..599324879d0f9 100644 --- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala @@ -26,8 +26,7 @@ import scala.reflect.ClassTag import org.apache.spark.{ComplexFutureAction, FutureAction, Logging} /** - * EXPERIMENTAL API - * + * EXPERIMENTAL API * A set of asynchronous RDD actions available through an implicit conversion. * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions. */ diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 408071262d158..bb0eec95848d4 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -51,8 +51,7 @@ private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep] } /** - * DEVELOPER API - UNSTABLE - * + * 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. * 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 818bfd46150c8..99ec8b9416c49 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -70,8 +70,7 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp } /** - * DEVELOPER API - UNSTABLE - * + * 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`). * 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 75f82163ef50b..bdf21b135cbf4 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -36,8 +36,7 @@ class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputS } /** - * DEVELOPER API - UNSTABLE - * + * 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`). * 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 46f44c129ba5a..7ccdd8649aac8 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala @@ -46,8 +46,7 @@ private[spark] class PruneDependency[T](rdd: RDD[T], @transient partitionFilterF /** - * DEVELOPER API - UNSTABLE - * + * 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 @@ -67,7 +66,7 @@ class PartitionPruningRDD[T: ClassTag]( /** - * DEVELOPER API - UNSTABLE + * DEVELOPER API */ 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 e269deefa9d17..a5eea008933da 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -86,22 +86,29 @@ abstract class RDD[T: ClassTag]( // Methods that should be implemented by subclasses of RDD // ======================================================================= - /** Implemented by subclasses to compute a given partition. */ + /** + * DEVELOPER API + * Implemented by subclasses to compute a given partition. + */ 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. */ 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. */ protected def getDependencies: Seq[Dependency[_]] = deps - /** Optionally overridden by subclasses to specify placement preferences. */ + /** + * DEVELOPER API + * Optionally overridden by subclasses to specify placement preferences. */ protected def getPreferredLocations(split: Partition): Seq[String] = Nil /** Optionally overridden by subclasses to specify how they are partitioned. */ @@ -513,8 +520,7 @@ abstract class RDD[T: ClassTag]( } /** - * DEVELOPER API - UNSTABLE - * + * 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. */ @@ -777,7 +783,7 @@ abstract class RDD[T: ClassTag]( def count(): Long = sc.runJob(this, Utils.getIteratorSize _).sum /** - * EXPERIMENTAL API + * EXPERIMENTAL API * * Approximate version of count() that returns a potentially incomplete result * within a timeout, even if not all tasks have finished. @@ -825,7 +831,7 @@ abstract class RDD[T: ClassTag]( } /** - * EXPERIMENTAL API + * EXPERIMENTAL API * * Approximate version of countByValue(). */ @@ -849,7 +855,7 @@ abstract class RDD[T: ClassTag]( } /** - * EXPERIMENTAL API + * EXPERIMENTAL API * * Return approximate number of distinct elements in the RDD. * 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 c94f67717448b..8a7f9efebe680 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -28,8 +28,7 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { } /** - * DEVELOPER API - UNSTABLE - * + * 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 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 8fcc9c9d855b0..66e360fcce401 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -28,8 +28,7 @@ import org.apache.spark._ import org.apache.spark.executor.TaskMetrics /** - * DEVELOPER API - UNSTABLE - * + * 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 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 af882924213de..4c77f493987c2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -27,23 +27,23 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.{Distribution, Utils} -/** DEVELOPER API - UNSTABLE */ +/** DEVELOPER API */ sealed trait SparkListenerEvent -/** DEVELOPER API - UNSTABLE */ +/** DEVELOPER API */ case class SparkListenerStageSubmitted(stageInfo: StageInfo, properties: Properties = null) extends SparkListenerEvent -/** DEVELOPER API - UNSTABLE */ +/** DEVELOPER API */ case class SparkListenerStageCompleted(stageInfo: StageInfo) extends SparkListenerEvent -/** DEVELOPER API - UNSTABLE */ +/** DEVELOPER API */ case class SparkListenerTaskStart(stageId: Int, taskInfo: TaskInfo) extends SparkListenerEvent -/** DEVELOPER API - UNSTABLE */ +/** DEVELOPER API */ case class SparkListenerTaskGettingResult(taskInfo: TaskInfo) extends SparkListenerEvent -/** DEVELOPER API - UNSTABLE */ +/** DEVELOPER API */ case class SparkListenerTaskEnd( stageId: Int, taskType: String, @@ -52,26 +52,26 @@ case class SparkListenerTaskEnd( taskMetrics: TaskMetrics) extends SparkListenerEvent -/** DEVELOPER API - UNSTABLE */ +/** DEVELOPER API */ case class SparkListenerJobStart(jobId: Int, stageIds: Seq[Int], properties: Properties = null) extends SparkListenerEvent -/** DEVELOPER API - UNSTABLE */ +/** DEVELOPER API */ case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkListenerEvent -/** DEVELOPER API - UNSTABLE */ +/** DEVELOPER API */ case class SparkListenerEnvironmentUpdate(environmentDetails: Map[String, Seq[(String, String)]]) extends SparkListenerEvent -/** DEVELOPER API - UNSTABLE */ +/** DEVELOPER API */ case class SparkListenerBlockManagerAdded(blockManagerId: BlockManagerId, maxMem: Long) extends SparkListenerEvent -/** DEVELOPER API - UNSTABLE */ +/** DEVELOPER API */ case class SparkListenerBlockManagerRemoved(blockManagerId: BlockManagerId) extends SparkListenerEvent -/** DEVELOPER API - UNSTABLE */ +/** DEVELOPER API */ case class SparkListenerUnpersistRDD(rddId: Int) extends SparkListenerEvent /** An event used in the listener to shutdown the listener daemon thread. */ @@ -79,8 +79,7 @@ private[spark] case object SparkListenerShutdown extends SparkListenerEvent /** - * DEVELOPER API - UNSTABLE - * + * 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. */ @@ -143,8 +142,7 @@ trait SparkListener { } /** - * DEVELOPER API - UNSTABLE - * + * DEVELOPER API * Simple SparkListener that logs a few summary statistics when each stage completes */ class StatsReportListener extends SparkListener with Logging { 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 df9b1ab8c8931..5aa7779aeb385 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -94,8 +94,7 @@ private[spark] class JavaSerializerInstance(counterReset: Int) extends Serialize } /** - * DEVELOPER API - UNSTABLE - * + * 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 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 d423d2e61a865..1f7caa1aad4c6 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -26,8 +26,7 @@ import org.apache.spark.util.{ByteBufferInputStream, NextIterator} import org.apache.spark.SparkEnv /** - * DEVELOPER API - UNSTABLE - * + * 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. @@ -55,8 +54,7 @@ object Serializer { /** - * DEVELOPER API - UNSTABLE - * + * DEVELOPER API * An instance of a serializer, for use by one thread at a time. */ trait SerializerInstance { @@ -89,8 +87,7 @@ trait SerializerInstance { /** - * DEVELOPER API - UNSTABLE - * + * DEVELOPER API * A stream for writing serialized objects. */ trait SerializationStream { @@ -108,8 +105,7 @@ trait SerializationStream { /** - * DEVELOPER API - UNSTABLE - * + * DEVELOPER API * A stream for reading serialized objects. */ trait DeserializationStream { 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 db09d21bd4ec7..eec1c45126099 100644 --- a/core/src/main/scala/org/apache/spark/util/MutablePair.scala +++ b/core/src/main/scala/org/apache/spark/util/MutablePair.scala @@ -18,8 +18,7 @@ package org.apache.spark.util /** - * DEVELOPER API - UNSTABLE - * + * 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. * 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 dac0d7d8a3b3c..c5f72d1e4124e 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 @@ -18,8 +18,7 @@ package org.apache.spark.util.random /** - * DEVELOPER API - UNSTABLE - * + * DEVELOPER API * A class with pseudorandom behavior. */ trait Pseudorandom { 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 479ca086b979c..15348515a724a 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 @@ -23,8 +23,7 @@ import cern.jet.random.Poisson import cern.jet.random.engine.DRand /** - * DEVELOPER API - UNSTABLE - * + * 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. @@ -42,8 +41,7 @@ trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable } /** - * DEVELOPER API - UNSTABLE - * + * DEVELOPER API * A sampler based on Bernoulli trials. * * @param lb lower bound of the acceptance range @@ -71,8 +69,7 @@ class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) } /** - * DEVELOPER API - UNSTABLE - * + * DEVELOPER API * A sampler based on values drawn from Poisson distribution. * * @param poisson a Poisson random number generator diff --git a/graphx/src/main/scala/org/apache/spark/graphx/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/package.scala index 099e19a161a40..ff17edeaf8f16 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/package.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/package.scala @@ -20,8 +20,7 @@ package org.apache.spark import org.apache.spark.util.collection.OpenHashSet /** - * ALPHA COMPONENT - * + * ALPHA COMPONENT * GraphX is a graph processing framework built on top of Spark. */ package object graphx { 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 bace60f59cd22..c00162866e058 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 @@ -31,8 +31,7 @@ import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.execution._ /** - * ALPHA COMPONENT - * + * ALPHA COMPONENT * The entry point for running relational queries using Spark. Allows the creation of [[SchemaRDD]] * objects and the execution of SQL queries. * @@ -62,8 +61,7 @@ class SQLContext(@transient val sparkContext: SparkContext) new this.QueryExecution { val logical = plan } /** - * EXPERIMENTAL - * + * EXPERIMENTAL * Allows catalyst LogicalPlans to be executed as a SchemaRDD. Note that the LogicalPlan * interface is considered internal, and thus not guranteed to be stable. As a result, using * them directly is not reccomended. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 57a37e138e9ff..d077bfc97f49d 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 @@ -26,8 +26,7 @@ import org.apache.spark.sql.catalyst.types.BooleanType import org.apache.spark.{Dependency, OneToOneDependency, Partition, TaskContext} /** - * ALPHA COMPONENT - * + * 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. * @@ -241,8 +240,7 @@ class SchemaRDD( Filter(ScalaUdf(udf, BooleanType, Seq(UnresolvedAttribute(arg1.name))), logicalPlan)) /** - * EXPERIMENTAL - * + * EXPERIMENTAL API * 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 @@ -260,8 +258,7 @@ class SchemaRDD( Filter(ScalaUdf(dynamicUdf, BooleanType, Seq(WrapDynamic(logicalPlan.output))), logicalPlan)) /** - * EXPERIMENTAL - * + * EXPERIMENTAL API * Returns a sampled version of the underlying dataset. * * @group Query @@ -273,8 +270,7 @@ class SchemaRDD( new SchemaRDD(sqlContext, Sample(fraction, withReplacement, seed, logicalPlan)) /** - * EXPERIMENTAL - * + * EXPERIMENTAL API * 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 @@ -298,8 +294,7 @@ class SchemaRDD( new SchemaRDD(sqlContext, Generate(generator, join, outer, None, logicalPlan)) /** - * EXPERIMENTAL - * + * EXPERIMENTAL API * Adds the rows from this RDD to the specified table. Note in a standard [[SQLContext]] there is * no notion of persistent tables, and thus queries that contain this operator will fail to * optimize. When working with an extension of a SQLContext that has a persistent catalog, such From 8452309a13522685d8d2fa7f7ee4bce4f006bcdc Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 5 Apr 2014 16:53:08 -0700 Subject: [PATCH 10/31] Style fixes --- core/src/main/scala/org/apache/spark/Aggregator.scala | 1 - core/src/main/scala/org/apache/spark/SparkContext.scala | 3 --- .../main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala | 2 +- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 1 - 4 files changed, 1 insertion(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index ad00b838e14a2..d03f653375898 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -27,7 +27,6 @@ import org.apache.spark.util.collection.{AppendOnlyMap, ExternalAppendOnlyMap} * @param mergeValue function to merge a new value into the aggregation result. * @param mergeCombiners function to merge outputs from multiple mergeValue function. */ - case class Aggregator[K, V, C] ( createCombiner: V => C, mergeValue: (C, V) => C, diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 40d3b67b684b6..5042d4b5bc824 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -667,7 +667,6 @@ class SparkContext( /** * DEVELOPER API - * * Register a listener to receive up-calls from events that happen during execution. */ def addSparkListener(listener: SparkListener) { @@ -980,7 +979,6 @@ class SparkContext( /** * DEVELOPER API - * * Run a job that can return approximate results. */ def runApproximateJob[T, U, R]( @@ -999,7 +997,6 @@ class SparkContext( /** * EXPERIMENTAL API - * * Submit a job for execution and return a FutureJob holding the result. */ def submitJob[T, U, R]( 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 7ccdd8649aac8..7ab40aca439af 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.{NarrowDependency, Partition, TaskContext} private[spark] class PartitionPruningRDDPartition(idx: Int, val parentSplit: Partition) - extends Partition { + extends Partition { override val index = idx } 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 a5eea008933da..66ecf1b4a1296 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -856,7 +856,6 @@ abstract class RDD[T: ClassTag]( /** * EXPERIMENTAL API - * * Return approximate number of distinct elements in the RDD. * * The accuracy of approximation can be controlled through the relative standard deviation From c581dcedc7578eea90bfbf966e9b5c1050971a37 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 5 Apr 2014 22:55:16 -0700 Subject: [PATCH 11/31] Changes after building against Shark. --- .../apache/spark/SerializableWritable.scala | 3 +- .../scala/org/apache/spark/SparkContext.scala | 31 +++++++++++++------ .../scala/org/apache/spark/SparkEnv.scala | 3 +- .../scala/org/apache/spark/rdd/EmptyRDD.scala | 4 +-- .../scala/org/apache/spark/rdd/UnionRDD.scala | 2 +- .../spark/scheduler/InputFormatInfo.scala | 1 + .../apache/spark/scheduler/SplitInfo.scala | 2 +- .../scala/org/apache/spark/util/Vector.scala | 2 +- 8 files changed, 32 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SerializableWritable.scala b/core/src/main/scala/org/apache/spark/SerializableWritable.scala index e250ebe84e458..3fd85ee9c4627 100644 --- a/core/src/main/scala/org/apache/spark/SerializableWritable.scala +++ b/core/src/main/scala/org/apache/spark/SerializableWritable.scala @@ -23,7 +23,8 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.ObjectWritable import org.apache.hadoop.io.Writable -private[spark] class SerializableWritable[T <: Writable](@transient var t: T) extends Serializable { +/** DEVELOPER API */ +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 5042d4b5bc824..2818a08b2081b 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -53,18 +53,28 @@ 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. - * @param preferredNodeLocationData used in YARN mode to select nodes to launch containers on. Can - * be generated using [[org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations]] - * from a list of input files or InputFormats for the application. */ -class SparkContext( - config: SparkConf, - // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, - // etc) too. This is typically generated from InputFormatInfo.computePreferredLocations. It - // contains a map from hostname to a list of input format splits on the host. - val preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map()) +class SparkContext(config: SparkConf) extends Logging { + // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, + // etc) too. This is typically generated from InputFormatInfo.computePreferredLocations. It + // contains a map from hostname to a list of input format splits on the host. + 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. Can + * be generated using [[org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations]] + * from a list of input files or InputFormats for the application. + */ + def this(config: SparkConf, preferredNodeLocationData: Map[String, Set[SplitInfo]]) = { + this(config) + this.preferredNodeLocationData = preferredNodeLocationData + } + /** * Alternative constructor that allows setting common Spark properties directly * @@ -606,6 +616,9 @@ class SparkContext( def union[T: ClassTag](first: RDD[T], rest: RDD[T]*): RDD[T] = new UnionRDD(this, Seq(first) ++ rest) + /** Get an RDD that has no partitions or elements. */ + def emptyRDD[T: ClassTag] = new EmptyRDD[T](this) + // Methods for creating shared variables /** diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 0e2965e714dd7..24543fcc8137d 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -35,13 +35,14 @@ 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. */ -private[spark] class SparkEnv ( +class SparkEnv ( val executorId: String, val actorSystem: ActorSystem, val serializer: Serializer, diff --git a/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala index e4bd9ebf4fb68..22e8c8c54310b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala @@ -22,9 +22,9 @@ import scala.reflect.ClassTag import org.apache.spark.{Partition, SparkContext, TaskContext} /** - * An RDD that has no partitions and no elements.. + * An RDD that has no partitions and no elements. */ -private[spark] class EmptyRDD[T: ClassTag](sc: SparkContext) extends RDD[T](sc, Nil) { +class EmptyRDD[T: ClassTag](sc: SparkContext) extends RDD[T](sc, Nil) { override def getPartitions: Array[Partition] = Array.empty 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 c856f2b3db710..a447030752096 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -43,7 +43,7 @@ private[spark] class UnionPartition[T: ClassTag](idx: Int, rdd: RDD[T], splitInd } } -private[spark] class UnionRDD[T: ClassTag]( +class UnionRDD[T: ClassTag]( sc: SparkContext, @transient var rdds: Seq[RDD[T]]) extends RDD[T](sc, Nil) { // Nil since we implement getDependencies 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 5555585c8b4cd..c943ef28802b7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala @@ -30,6 +30,7 @@ import org.apache.spark.Logging import org.apache.spark.deploy.SparkHadoopUtil /** + * DEVELOPER API * Parses and holds information about inputFormat (and files) specified as a parameter. */ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Class[_], 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 f742291deb9e9..7828d2c830d16 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala @@ -21,7 +21,7 @@ import collection.mutable.ArrayBuffer // information about a specific split instance : handles both split instances. // So that we do not need to worry about the differences. -private[spark] +/** DEVELOPER API */ 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/util/Vector.scala b/core/src/main/scala/org/apache/spark/util/Vector.scala index 643366db9c78b..3c8f94a416c65 100644 --- a/core/src/main/scala/org/apache/spark/util/Vector.scala +++ b/core/src/main/scala/org/apache/spark/util/Vector.scala @@ -21,7 +21,7 @@ import scala.util.Random import org.apache.spark.util.random.XORShiftRandom -@deprecated("Use Vector from Spark's mllib.linalg package instead.", "1.0.0") +@deprecated("Use Vectors.dense from Spark's mllib.linalg package instead.", "1.0.0") class Vector(val elements: Array[Double]) extends Serializable { def length = elements.length From 4a9e90cdc0b489619208b4098dce4d6b6582291e Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 6 Apr 2014 00:27:18 -0700 Subject: [PATCH 12/31] EXPERIMENTAL API --> EXPERIMENTAL --- core/src/main/scala/org/apache/spark/FutureAction.scala | 6 +++--- core/src/main/scala/org/apache/spark/SparkContext.scala | 2 +- core/src/main/scala/org/apache/spark/package.scala | 2 +- .../scala/org/apache/spark/partial/BoundedDouble.scala | 2 +- .../scala/org/apache/spark/partial/PartialResult.scala | 2 +- .../main/scala/org/apache/spark/rdd/AsyncRDDActions.scala | 2 +- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 6 +++--- .../src/main/scala/org/apache/spark/sql/SchemaRDD.scala | 8 ++++---- 8 files changed, 15 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index 0d5bad9e91e91..5468886bf1c6c 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -25,7 +25,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.{JobFailed, JobSucceeded, JobWaiter} /** - * EXPERIMENTAL API + * EXPERIMENTAL * A future for the result of an action to support cancellation. This is an extension of the * Scala Future interface to support cancellation. */ @@ -85,7 +85,7 @@ trait FutureAction[T] extends Future[T] { /** - * EXPERIMENTAL API + * EXPERIMENTAL * A [[FutureAction]] holding the result of an action that triggers a single job. Examples include * count, collect, reduce. */ @@ -150,7 +150,7 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: /** - * EXPERIMENTAL API + * EXPERIMENTAL * A [[FutureAction]] for actions that could trigger multiple Spark jobs. Examples include take, * takeSample. Cancellation works by setting the cancelled flag to true and interrupting the * action thread if it is being blocked by a job. diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 2818a08b2081b..eb843d1f62a08 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1009,7 +1009,7 @@ class SparkContext(config: SparkConf) } /** - * EXPERIMENTAL API + * EXPERIMENTAL * Submit a job for execution and return a FutureJob holding the result. */ def submitJob[T, U, R]( diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala index 0c98f341f7543..e1d10cab5988d 100644 --- a/core/src/main/scala/org/apache/spark/package.scala +++ b/core/src/main/scala/org/apache/spark/package.scala @@ -34,7 +34,7 @@ package org.apache * for Spark programming APIs in Java. * * Classes and methods marked with - * EXPERIMENTAL API are user-facing features which have not been officially adopted by the + * EXPERIMENTAL are user-facing features which have not been officially adopted by the * Spark project. These are subject to change or removal in minor releases. * * Classes and methods marked with 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 aee6c3c85cbea..12a7fff35e868 100644 --- a/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala +++ b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala @@ -18,7 +18,7 @@ package org.apache.spark.partial /** - * EXPERIMENTAL API + * EXPERIMENTAL * A Double value with error bars and associated confidence. */ class BoundedDouble(val mean: Double, val confidence: Double, val low: Double, val high: Double) { 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 f055543847dfc..3103b5783806f 100644 --- a/core/src/main/scala/org/apache/spark/partial/PartialResult.scala +++ b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala @@ -18,7 +18,7 @@ package org.apache.spark.partial /** - * EXPERIMENTAL API + * EXPERIMENTAL */ class PartialResult[R](initialVal: R, isFinal: Boolean) { private var finalValue: Option[R] = if (isFinal) Some(initialVal) else 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 599324879d0f9..199945287da95 100644 --- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala @@ -26,7 +26,7 @@ import scala.reflect.ClassTag import org.apache.spark.{ComplexFutureAction, FutureAction, Logging} /** - * EXPERIMENTAL API + * EXPERIMENTAL * A set of asynchronous RDD actions available through an implicit conversion. * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions. */ diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 66ecf1b4a1296..ddff5311b423f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -783,7 +783,7 @@ abstract class RDD[T: ClassTag]( def count(): Long = sc.runJob(this, Utils.getIteratorSize _).sum /** - * EXPERIMENTAL API + * EXPERIMENTAL * * Approximate version of count() that returns a potentially incomplete result * within a timeout, even if not all tasks have finished. @@ -831,7 +831,7 @@ abstract class RDD[T: ClassTag]( } /** - * EXPERIMENTAL API + * EXPERIMENTAL * * Approximate version of countByValue(). */ @@ -855,7 +855,7 @@ abstract class RDD[T: ClassTag]( } /** - * EXPERIMENTAL API + * EXPERIMENTAL * Return approximate number of distinct elements in the RDD. * * The accuracy of approximation can be controlled through the relative standard deviation diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index d077bfc97f49d..e1937f7f0d9f8 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 @@ -240,7 +240,7 @@ class SchemaRDD( Filter(ScalaUdf(udf, BooleanType, Seq(UnresolvedAttribute(arg1.name))), logicalPlan)) /** - * EXPERIMENTAL API + * 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 @@ -258,7 +258,7 @@ class SchemaRDD( Filter(ScalaUdf(dynamicUdf, BooleanType, Seq(WrapDynamic(logicalPlan.output))), logicalPlan)) /** - * EXPERIMENTAL API + * EXPERIMENTAL * Returns a sampled version of the underlying dataset. * * @group Query @@ -270,7 +270,7 @@ class SchemaRDD( new SchemaRDD(sqlContext, Sample(fraction, withReplacement, seed, logicalPlan)) /** - * EXPERIMENTAL API + * 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 @@ -294,7 +294,7 @@ class SchemaRDD( new SchemaRDD(sqlContext, Generate(generator, join, outer, None, logicalPlan)) /** - * EXPERIMENTAL API + * EXPERIMENTAL * Adds the rows from this RDD to the specified table. Note in a standard [[SQLContext]] there is * no notion of persistent tables, and thus queries that contain this operator will fail to * optimize. When working with an extension of a SQLContext that has a persistent catalog, such From 7fb13b21864ac030ae3bd09d0a4262bdb5bf66f3 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 6 Apr 2014 10:35:57 -0700 Subject: [PATCH 13/31] Changes to UnionRDD and EmptyRDD --- core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala | 2 +- core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala index 22e8c8c54310b..a2d7e344cf1b2 100644 --- a/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala @@ -24,7 +24,7 @@ import org.apache.spark.{Partition, SparkContext, TaskContext} /** * An RDD that has no partitions and no elements. */ -class EmptyRDD[T: ClassTag](sc: SparkContext) extends RDD[T](sc, Nil) { +private[spark] class EmptyRDD[T: ClassTag](sc: SparkContext) extends RDD[T](sc, Nil) { override def getPartitions: Array[Partition] = Array.empty 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 a447030752096..98b44f724a838 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -43,6 +43,7 @@ private[spark] class UnionPartition[T: ClassTag](idx: Int, rdd: RDD[T], splitInd } } +/** DEVELOPER API */ class UnionRDD[T: ClassTag]( sc: SparkContext, @transient var rdds: Seq[RDD[T]]) From 870a7ba00339d28faf8e585ef59315935d3a0b6a Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 6 Apr 2014 11:42:07 -0700 Subject: [PATCH 14/31] Work around for SI-8479 --- .../scala/org/apache/spark/SparkContext.scala | 49 ++++++++++++++++--- 1 file changed, 41 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index eb843d1f62a08..a60966a7ea89a 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -66,13 +66,13 @@ class SparkContext(config: SparkConf) * 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. Can + * @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. */ - def this(config: SparkConf, preferredNodeLocationData: Map[String, Set[SplitInfo]]) = { - this(config) - this.preferredNodeLocationData = preferredNodeLocationData + def this(config: SparkConf, preferredNodeLocationData: Map[String, Set[SplitInfo]]) = { + this(config) + this.preferredNodeLocationData = preferredNodeLocationData } /** @@ -85,6 +85,10 @@ class SparkContext(config: SparkConf) def this(master: String, appName: String, conf: SparkConf) = this(SparkContext.updatedConf(conf, master, appName)) + // NOTE: The below constructors could be consolidated using default arguments. Due to + // Scala bug SI-8479, however, this causes the compile step to fail when generating docs. + // Until we have a good workaround for that bug the constructors remain broken out. + /** * Alternative constructor that allows setting common Spark properties directly * @@ -100,13 +104,42 @@ class SparkContext(config: SparkConf) appName: String, sparkHome: String = null, jars: Seq[String] = Nil, - environment: Map[String, String] = Map(), - preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map()) = + environment: Map[String, String] = Map()) = { - this(SparkContext.updatedConf(new SparkConf(), master, appName, sparkHome, jars, environment), - preferredNodeLocationData) + this(SparkContext.updatedConf(new SparkConf(), master, appName, sparkHome, jars, environment)) } + /** + * Alternative constructor that allows setting common Spark properties directly + * + * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param appName A name for your application, to display on the cluster web UI. + */ + def this(master: String, appName: String) = + this(master, appName, null, Nil, Map()) + + /** + * Alternative constructor that allows setting common Spark properties directly + * + * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param appName A name for your application, to display on the cluster web UI. + * @param sparkHome Location where Spark is installed on cluster nodes. + */ + def this(master: String, appName: String, sparkHome: String) = + this(master, appName, sparkHome, Nil, Map()) + + /** + * Alternative constructor that allows setting common Spark properties directly + * + * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param appName A name for your application, to display on the cluster web UI. + * @param sparkHome Location where Spark is installed on cluster nodes. + * @param jars Collection of JARs to send to the cluster. These can be paths on the local file + * system or HDFS, HTTP, HTTPS, or FTP URLs. + */ + def this(master: String, appName: String, sparkHome: String, jars: Seq[String]) = + this(master, appName, sparkHome, jars, Map()) + private[spark] val conf = config.clone() /** From 403ba52ccc5c3d71c5580f43441196f2ff03b082 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 6 Apr 2014 11:51:49 -0700 Subject: [PATCH 15/31] Style fix --- core/src/main/scala/org/apache/spark/package.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala index e1d10cab5988d..5bbfb6c097580 100644 --- a/core/src/main/scala/org/apache/spark/package.scala +++ b/core/src/main/scala/org/apache/spark/package.scala @@ -44,4 +44,4 @@ package org.apache package object spark { // For package docs only -} \ No newline at end of file +} From d6b0afdd2daf744aab8b814f16f1bd6d3618b2d2 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 6 Apr 2014 12:16:45 -0700 Subject: [PATCH 16/31] Small chang to existing constructor --- core/src/main/scala/org/apache/spark/SparkContext.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index a60966a7ea89a..67b732c9582be 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -104,9 +104,11 @@ class SparkContext(config: SparkConf) appName: String, sparkHome: String = null, jars: Seq[String] = Nil, - environment: Map[String, String] = Map()) = + environment: Map[String, String] = Map(), + preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map()) = { this(SparkContext.updatedConf(new SparkConf(), master, appName, sparkHome, jars, environment)) + this.preferredNodeLocationData = preferredNodeLocationData } /** From f390b131e1ae399851d70bdce18186eb89b0dc86 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 6 Apr 2014 12:22:51 -0700 Subject: [PATCH 17/31] Better visibility for workaround constructors --- .../scala/org/apache/spark/SparkContext.scala | 20 +++++++++---------- .../spark/api/java/JavaSparkContext.scala | 2 +- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 67b732c9582be..5b450756a23c5 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -85,10 +85,6 @@ class SparkContext(config: SparkConf) def this(master: String, appName: String, conf: SparkConf) = this(SparkContext.updatedConf(conf, master, appName)) - // NOTE: The below constructors could be consolidated using default arguments. Due to - // Scala bug SI-8479, however, this causes the compile step to fail when generating docs. - // Until we have a good workaround for that bug the constructors remain broken out. - /** * Alternative constructor that allows setting common Spark properties directly * @@ -111,14 +107,18 @@ class SparkContext(config: SparkConf) this.preferredNodeLocationData = preferredNodeLocationData } + // NOTE: The below constructors could be consolidated using default arguments. Due to + // Scala bug SI-8479, however, this causes the compile step to fail when generating docs. + // Until we have a good workaround for that bug the constructors remain broken out. + /** * Alternative constructor that allows setting common Spark properties directly * * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). * @param appName A name for your application, to display on the cluster web UI. */ - def this(master: String, appName: String) = - this(master, appName, null, Nil, Map()) + private[spark] def this(master: String, appName: String) = + this(master, appName, null, Nil, Map(), Map()) /** * Alternative constructor that allows setting common Spark properties directly @@ -127,8 +127,8 @@ class SparkContext(config: SparkConf) * @param appName A name for your application, to display on the cluster web UI. * @param sparkHome Location where Spark is installed on cluster nodes. */ - def this(master: String, appName: String, sparkHome: String) = - this(master, appName, sparkHome, Nil, Map()) + private[spark] def this(master: String, appName: String, sparkHome: String) = + this(master, appName, sparkHome, Nil, Map(), Map()) /** * Alternative constructor that allows setting common Spark properties directly @@ -139,8 +139,8 @@ class SparkContext(config: SparkConf) * @param jars Collection of JARs to send to the cluster. These can be paths on the local file * system or HDFS, HTTP, HTTPS, or FTP URLs. */ - def this(master: String, appName: String, sparkHome: String, jars: Seq[String]) = - this(master, appName, sparkHome, jars, Map()) + private[spark] def this(master: String, appName: String, sparkHome: String, jars: Seq[String]) = + this(master, appName, sparkHome, jars, Map(), Map()) private[spark] val conf = config.clone() diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index e531a57aced31..2d313d73cc8f5 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -89,7 +89,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork */ def this(master: String, appName: String, sparkHome: String, jars: Array[String], environment: JMap[String, String]) = - this(new SparkContext(master, appName, sparkHome, jars.toSeq, environment)) + this(new SparkContext(master, appName, sparkHome, jars.toSeq, environment, Map())) private[spark] val env = sc.env From e4c76b98396604cecf5094161f6fbfd03996cae5 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 7 Apr 2014 14:32:11 -0700 Subject: [PATCH 18/31] Logging --- core/src/main/scala/org/apache/spark/Logging.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index 7423082e34f47..20ba27b0c167c 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -22,9 +22,13 @@ import org.slf4j.{Logger, LoggerFactory} import org.slf4j.impl.StaticLoggerBinder /** + * 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. + * + * 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. */ trait Logging { // Make the log field transient so that objects with Logging can From c318b24134d3cfbc7542982a82ecc05fe992e298 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 7 Apr 2014 16:44:30 -0700 Subject: [PATCH 19/31] Use CSS styles --- .../scala/org/apache/spark/Aggregator.scala | 2 +- .../scala/org/apache/spark/Dependency.scala | 10 +++---- .../scala/org/apache/spark/FutureAction.scala | 6 ++-- .../main/scala/org/apache/spark/Logging.scala | 2 +- .../apache/spark/SerializableWritable.scala | 2 +- .../scala/org/apache/spark/SparkContext.scala | 10 +++---- .../scala/org/apache/spark/SparkEnv.scala | 2 +- .../scala/org/apache/spark/TaskContext.scala | 2 +- .../spark/broadcast/BroadcastFactory.scala | 2 +- .../apache/spark/executor/TaskMetrics.scala | 6 ++-- .../apache/spark/io/CompressionCodec.scala | 6 ++-- .../main/scala/org/apache/spark/package.scala | 8 +++--- .../apache/spark/partial/BoundedDouble.scala | 2 +- .../apache/spark/partial/PartialResult.scala | 2 +- .../apache/spark/rdd/AsyncRDDActions.scala | 2 +- .../org/apache/spark/rdd/CoGroupedRDD.scala | 2 +- .../org/apache/spark/rdd/HadoopRDD.scala | 2 +- .../org/apache/spark/rdd/NewHadoopRDD.scala | 2 +- .../spark/rdd/PartitionPruningRDD.scala | 4 +-- .../main/scala/org/apache/spark/rdd/RDD.scala | 16 +++++------ .../org/apache/spark/rdd/ShuffledRDD.scala | 2 +- .../scala/org/apache/spark/rdd/UnionRDD.scala | 2 +- .../spark/scheduler/InputFormatInfo.scala | 2 +- .../apache/spark/scheduler/JobLogger.scala | 2 +- .../spark/scheduler/SparkListener.scala | 28 +++++++++---------- .../apache/spark/scheduler/SplitInfo.scala | 2 +- .../spark/serializer/JavaSerializer.scala | 2 +- .../apache/spark/serializer/Serializer.scala | 8 +++--- .../org/apache/spark/util/MutablePair.scala | 2 +- .../spark/util/random/Pseudorandom.scala | 2 +- .../spark/util/random/RandomSampler.scala | 6 ++-- docs/_plugins/copy_api_dirs.rb | 26 ++++++++--------- .../spark/sql/catalyst/expressions/Row.scala | 2 +- .../org/apache/spark/sql/SQLContext.scala | 4 +-- .../org/apache/spark/sql/SchemaRDD.scala | 10 +++---- 35 files changed, 95 insertions(+), 95 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index d03f653375898..d43ef6c430cb1 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -20,7 +20,7 @@ package org.apache.spark import org.apache.spark.util.collection.{AppendOnlyMap, ExternalAppendOnlyMap} /** - * DEVELOPER API + * Developer API * A set of functions used to aggregate data. * * @param createCombiner function to create the initial value of the aggregation. diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 0d65a1ab2f9a4..ae3a921e61658 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -21,14 +21,14 @@ import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer /** - * DEVELOPER API + * Developer API * Base class for dependencies. */ abstract class Dependency[T](val rdd: RDD[T]) extends Serializable /** - * DEVELOPER API + * 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. */ @@ -43,7 +43,7 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { /** - * DEVELOPER API + * 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 @@ -62,7 +62,7 @@ class ShuffleDependency[K, V]( /** - * DEVELOPER API + * Developer API * Represents a one-to-one dependency between partitions of the parent and child RDDs. */ class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) { @@ -71,7 +71,7 @@ class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) { /** - * DEVELOPER API + * 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 diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index 5468886bf1c6c..93b180d180ecf 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -25,7 +25,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.{JobFailed, JobSucceeded, JobWaiter} /** - * EXPERIMENTAL + * Experimental * A future for the result of an action to support cancellation. This is an extension of the * Scala Future interface to support cancellation. */ @@ -85,7 +85,7 @@ trait FutureAction[T] extends Future[T] { /** - * EXPERIMENTAL + * Experimental * A [[FutureAction]] holding the result of an action that triggers a single job. Examples include * count, collect, reduce. */ @@ -150,7 +150,7 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: /** - * EXPERIMENTAL + * Experimental * A [[FutureAction]] for actions that could trigger multiple Spark jobs. Examples include take, * takeSample. Cancellation works by setting the cancelled flag to true and interrupting the * action thread if it is being blocked by a job. diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index 20ba27b0c167c..f4cb7973e52a0 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -22,7 +22,7 @@ import org.slf4j.{Logger, LoggerFactory} import org.slf4j.impl.StaticLoggerBinder /** - * DEVELOPER API + * 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. diff --git a/core/src/main/scala/org/apache/spark/SerializableWritable.scala b/core/src/main/scala/org/apache/spark/SerializableWritable.scala index 3fd85ee9c4627..73694d09485b1 100644 --- a/core/src/main/scala/org/apache/spark/SerializableWritable.scala +++ b/core/src/main/scala/org/apache/spark/SerializableWritable.scala @@ -23,7 +23,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.ObjectWritable import org.apache.hadoop.io.Writable -/** DEVELOPER API */ +/** Developer API */ 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 5b450756a23c5..bf91093a9c65c 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -63,13 +63,13 @@ class SparkContext(config: SparkConf) private[spark] var preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map() /** - * DEVELOPER API + * 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. - */ + def this(config: SparkConf, preferredNodeLocationData: Map[String, Set[SplitInfo]]) = { this(config) this.preferredNodeLocationData = preferredNodeLocationData @@ -714,7 +714,7 @@ class SparkContext(config: SparkConf) } /** - * DEVELOPER API + * Developer API * Register a listener to receive up-calls from events that happen during execution. */ def addSparkListener(listener: SparkListener) { @@ -1026,7 +1026,7 @@ class SparkContext(config: SparkConf) } /** - * DEVELOPER API + * Developer API * Run a job that can return approximate results. */ def runApproximateJob[T, U, R]( @@ -1044,7 +1044,7 @@ class SparkContext(config: SparkConf) } /** - * EXPERIMENTAL + * Experimental * Submit a job for execution and return a FutureJob holding the result. */ def submitJob[T, U, R]( diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 24543fcc8137d..599688092791e 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -35,7 +35,7 @@ import org.apache.spark.storage._ import org.apache.spark.util.{AkkaUtils, Utils} /** - * DEVELOPER API + * 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 diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 97de2b7f7717a..faccafbdc9a1d 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -22,7 +22,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.executor.TaskMetrics /** - * DEVELOPER API + * Developer API * Contextual information about a task which can be read or mutated during execution. */ class TaskContext( 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 bcb8203f2d862..051547a199cbb 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala @@ -21,7 +21,7 @@ import org.apache.spark.SecurityManager import org.apache.spark.SparkConf /** - * DEVELOPER API + * 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 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 acf0a543f2d62..e844149ea1ec8 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -20,7 +20,7 @@ package org.apache.spark.executor import org.apache.spark.storage.{BlockId, BlockStatus} /** - * DEVELOPER API + * Developer API * Metrics tracked during the execution of a task. */ class TaskMetrics extends Serializable { @@ -87,7 +87,7 @@ object TaskMetrics { /** - * DEVELOPER API + * Developer API * Metrics pertaining to shuffle data read in a given task. */ class ShuffleReadMetrics extends Serializable { @@ -125,7 +125,7 @@ class ShuffleReadMetrics extends Serializable { } /** - * DEVELOPER API + * Developer API * Metrics pertaining to shuffle data written in a given task. */ class ShuffleWriteMetrics extends Serializable { 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 e872b487b414e..0a8fa9f680e5d 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -25,7 +25,7 @@ import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} import org.apache.spark.SparkConf /** - * DEVELOPER API + * Developer API * CompressionCodec allows the customization of choosing different compression implementations * to be used in block storage. * @@ -57,7 +57,7 @@ private[spark] object CompressionCodec { /** - * DEVELOPER API + * 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 @@ -75,7 +75,7 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { /** - * DEVELOPER API + * Developer API * Snappy implementation of [[org.apache.spark.io.CompressionCodec]]. * Block size can be configured by spark.io.compression.snappy.block.size. * diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala index 5bbfb6c097580..eb78a1ac172ca 100644 --- a/core/src/main/scala/org/apache/spark/package.scala +++ b/core/src/main/scala/org/apache/spark/package.scala @@ -33,12 +33,12 @@ package org.apache * Java programmers should reference the [[spark.api.java]] package * for Spark programming APIs in Java. * - * Classes and methods marked with - * EXPERIMENTAL are user-facing features which have not been officially adopted by the + * Classes and methods marked with + * Experimental are user-facing features which have not been officially adopted by the * Spark project. These are subject to change or removal in minor releases. * - * Classes and methods marked with - * DEVELOPER API are intended for advanced users want to extend Spark through lower + * Classes and methods marked with + * Developer API are intended for advanced users want to extend Spark through lower * level interfaces. These are subject to changes or removal in minor releases. */ 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 12a7fff35e868..55b979bba0c49 100644 --- a/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala +++ b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala @@ -18,7 +18,7 @@ package org.apache.spark.partial /** - * EXPERIMENTAL + * Experimental * A Double value with error bars and associated confidence. */ class BoundedDouble(val mean: Double, val confidence: Double, val low: Double, val high: Double) { 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 3103b5783806f..9fa4c12326c11 100644 --- a/core/src/main/scala/org/apache/spark/partial/PartialResult.scala +++ b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala @@ -18,7 +18,7 @@ package org.apache.spark.partial /** - * EXPERIMENTAL + * Experimental */ class PartialResult[R](initialVal: R, isFinal: Boolean) { private var finalValue: Option[R] = if (isFinal) Some(initialVal) else 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 199945287da95..05eda70db4545 100644 --- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala @@ -26,7 +26,7 @@ import scala.reflect.ClassTag import org.apache.spark.{ComplexFutureAction, FutureAction, Logging} /** - * EXPERIMENTAL + * Experimental * A set of asynchronous RDD actions available through an implicit conversion. * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions. */ diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index bb0eec95848d4..320a33f64ddd7 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -51,7 +51,7 @@ private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep] } /** - * DEVELOPER API + * 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. * 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 99ec8b9416c49..8807e8ca9aa3b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -70,7 +70,7 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp } /** - * DEVELOPER API + * 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`). * 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 bdf21b135cbf4..5c97965556ac7 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -36,7 +36,7 @@ class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputS } /** - * DEVELOPER API + * 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`). * 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 7ab40aca439af..efeb28da4f083 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala @@ -46,7 +46,7 @@ private[spark] class PruneDependency[T](rdd: RDD[T], @transient partitionFilterF /** - * DEVELOPER API + * 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 @@ -66,7 +66,7 @@ class PartitionPruningRDD[T: ClassTag]( /** - * DEVELOPER API + * Developer API */ 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 ddff5311b423f..67f2fe65cf6d2 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -87,27 +87,27 @@ abstract class RDD[T: ClassTag]( // ======================================================================= /** - * DEVELOPER API + * Developer API * Implemented by subclasses to compute a given partition. */ def compute(split: Partition, context: TaskContext): Iterator[T] /** - * DEVELOPER API + * 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. */ protected def getPartitions: Array[Partition] /** - * DEVELOPER API + * 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. */ protected def getDependencies: Seq[Dependency[_]] = deps /** - * DEVELOPER API + * Developer API * Optionally overridden by subclasses to specify placement preferences. */ protected def getPreferredLocations(split: Partition): Seq[String] = Nil @@ -520,7 +520,7 @@ abstract class RDD[T: ClassTag]( } /** - * DEVELOPER API + * 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. */ @@ -783,7 +783,7 @@ abstract class RDD[T: ClassTag]( def count(): Long = sc.runJob(this, Utils.getIteratorSize _).sum /** - * EXPERIMENTAL + * Experimental * * Approximate version of count() that returns a potentially incomplete result * within a timeout, even if not all tasks have finished. @@ -831,7 +831,7 @@ abstract class RDD[T: ClassTag]( } /** - * EXPERIMENTAL + * Experimental * * Approximate version of countByValue(). */ @@ -855,7 +855,7 @@ abstract class RDD[T: ClassTag]( } /** - * EXPERIMENTAL + * Experimental * Return approximate number of distinct elements in the RDD. * * The accuracy of approximation can be controlled through the relative standard deviation diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala index 8a7f9efebe680..679759539cfd9 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -28,7 +28,7 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { } /** - * DEVELOPER API + * 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 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 98b44f724a838..c40961497a34c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -43,7 +43,7 @@ private[spark] class UnionPartition[T: ClassTag](idx: Int, rdd: RDD[T], splitInd } } -/** DEVELOPER API */ +/** Developer API */ 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 c943ef28802b7..79eda49c6fdf9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala @@ -30,7 +30,7 @@ import org.apache.spark.Logging import org.apache.spark.deploy.SparkHadoopUtil /** - * DEVELOPER API + * Developer API * Parses and holds information about inputFormat (and files) specified as a parameter. */ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Class[_], 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 66e360fcce401..f85ac36597e30 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -28,7 +28,7 @@ import org.apache.spark._ import org.apache.spark.executor.TaskMetrics /** - * DEVELOPER API + * 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 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 4c77f493987c2..cbe8c25de5d55 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -27,23 +27,23 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.{Distribution, Utils} -/** DEVELOPER API */ +/** Developer API */ sealed trait SparkListenerEvent -/** DEVELOPER API */ +/** Developer API */ case class SparkListenerStageSubmitted(stageInfo: StageInfo, properties: Properties = null) extends SparkListenerEvent -/** DEVELOPER API */ +/** Developer API */ case class SparkListenerStageCompleted(stageInfo: StageInfo) extends SparkListenerEvent -/** DEVELOPER API */ +/** Developer API */ case class SparkListenerTaskStart(stageId: Int, taskInfo: TaskInfo) extends SparkListenerEvent -/** DEVELOPER API */ +/** Developer API */ case class SparkListenerTaskGettingResult(taskInfo: TaskInfo) extends SparkListenerEvent -/** DEVELOPER API */ +/** Developer API */ case class SparkListenerTaskEnd( stageId: Int, taskType: String, @@ -52,26 +52,26 @@ case class SparkListenerTaskEnd( taskMetrics: TaskMetrics) extends SparkListenerEvent -/** DEVELOPER API */ +/** Developer API */ case class SparkListenerJobStart(jobId: Int, stageIds: Seq[Int], properties: Properties = null) extends SparkListenerEvent -/** DEVELOPER API */ +/** Developer API */ case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkListenerEvent -/** DEVELOPER API */ +/** Developer API */ case class SparkListenerEnvironmentUpdate(environmentDetails: Map[String, Seq[(String, String)]]) extends SparkListenerEvent -/** DEVELOPER API */ +/** Developer API */ case class SparkListenerBlockManagerAdded(blockManagerId: BlockManagerId, maxMem: Long) extends SparkListenerEvent -/** DEVELOPER API */ +/** Developer API */ case class SparkListenerBlockManagerRemoved(blockManagerId: BlockManagerId) extends SparkListenerEvent -/** DEVELOPER API */ +/** Developer API */ case class SparkListenerUnpersistRDD(rddId: Int) extends SparkListenerEvent /** An event used in the listener to shutdown the listener daemon thread. */ @@ -79,7 +79,7 @@ private[spark] case object SparkListenerShutdown extends SparkListenerEvent /** - * DEVELOPER API + * 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. */ @@ -142,7 +142,7 @@ trait SparkListener { } /** - * DEVELOPER API + * Developer API * Simple SparkListener that logs a few summary statistics when each stage completes */ class StatsReportListener extends SparkListener with Logging { 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 7828d2c830d16..4f28bdd991d8c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala @@ -21,7 +21,7 @@ import collection.mutable.ArrayBuffer // information about a specific split instance : handles both split instances. // So that we do not need to worry about the differences. -/** DEVELOPER API */ +/** Developer API */ 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/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index 5aa7779aeb385..cf0576713f4ad 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -94,7 +94,7 @@ private[spark] class JavaSerializerInstance(counterReset: Int) extends Serialize } /** - * DEVELOPER API + * 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 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 1f7caa1aad4c6..015c4f62e110b 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -26,7 +26,7 @@ import org.apache.spark.util.{ByteBufferInputStream, NextIterator} import org.apache.spark.SparkEnv /** - * DEVELOPER API + * 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. @@ -54,7 +54,7 @@ object Serializer { /** - * DEVELOPER API + * Developer API * An instance of a serializer, for use by one thread at a time. */ trait SerializerInstance { @@ -87,7 +87,7 @@ trait SerializerInstance { /** - * DEVELOPER API + * Developer API * A stream for writing serialized objects. */ trait SerializationStream { @@ -105,7 +105,7 @@ trait SerializationStream { /** - * DEVELOPER API + * Developer API * A stream for reading serialized objects. */ trait DeserializationStream { 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 eec1c45126099..75ff7f68e51b7 100644 --- a/core/src/main/scala/org/apache/spark/util/MutablePair.scala +++ b/core/src/main/scala/org/apache/spark/util/MutablePair.scala @@ -18,7 +18,7 @@ package org.apache.spark.util /** - * DEVELOPER API + * 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. * 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 c5f72d1e4124e..e8adc28c2acda 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 @@ -18,7 +18,7 @@ package org.apache.spark.util.random /** - * DEVELOPER API + * Developer API * A class with pseudorandom behavior. */ trait Pseudorandom { 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 15348515a724a..a999a607d1b31 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 @@ -23,7 +23,7 @@ import cern.jet.random.Poisson import cern.jet.random.engine.DRand /** - * DEVELOPER API + * 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. @@ -41,7 +41,7 @@ trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable } /** - * DEVELOPER API + * Developer API * A sampler based on Bernoulli trials. * * @param lb lower bound of the acceptance range @@ -69,7 +69,7 @@ class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) } /** - * DEVELOPER API + * Developer API * A sampler based on values drawn from Poisson distribution. * * @param poisson a Poisson random number generator diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index 2245bcbc70f1e..8d4a525fb51a8 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -24,7 +24,9 @@ external_projects = ["flume", "kafka", "mqtt", "twitter", "zeromq"] sql_projects = ["catalyst", "core", "hive"] - projects = core_projects + external_projects.map { |project_name| "external/" + project_name } + projects = core_projects + projects = projects + external_projects.map { |project_name| "external/" + project_name } +# projects = projects + sql_projects.map { |project_name| "sql/" + project_name } puts "Moving to project root and building scaladoc." curr_dir = pwd @@ -42,24 +44,22 @@ source = "../" + project_name + "/target/scala-2.10/api" dest = "api/" + project_name - puts "echo making directory " + dest + puts "making directory " + dest mkdir_p dest # From the rubydoc: cp_r('src', 'dest') makes src/dest, but this doesn't. puts "cp -r " + source + "/. " + dest cp_r(source + "/.", dest) - end - - sql_projects.each do |project_name| - source = "../sql/" + project_name + "/target/scala-2.10/api/" - dest = "api/sql/" + project_name - - puts "echo making directory " + dest - mkdir_p dest - # From the rubydoc: cp_r('src', 'dest') makes src/dest, but this doesn't. - puts "cp -r " + source + "/. " + dest - cp_r(source + "/.", dest) + # Append custom CSS + css_file = dest + "/lib/template.css" + extra_css = [ + "", + "/* Styles added by spark jekyll plug-in */", + ".developer {background-color: #44751E; float: right; text-transform: uppercase;}", + ".experimental {background-color: #257080; float: right; text-transform: uppercase;}", + ].join("\n") + File.open(css_file, 'a') { |f| f.write(extra_css) } end # Build Epydoc for Python diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala index 31d42b9ee71a0..6a53e66ce3dbf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala @@ -65,7 +65,7 @@ trait MutableRow extends Row { def setString(ordinal: Int, value: String) /** - * EXPERIMENTAL + * Experimental * * Returns a mutable string builder for the specified column. A given row should return the * result of any mutations made to the returned buffer next time getString is called for the same 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 c00162866e058..5cdae65e29721 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 @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.execution._ /** - * ALPHA COMPONENT + * ALPHA COMPONENT * The entry point for running relational queries using Spark. Allows the creation of [[SchemaRDD]] * objects and the execution of SQL queries. * @@ -61,7 +61,7 @@ class SQLContext(@transient val sparkContext: SparkContext) new this.QueryExecution { val logical = plan } /** - * EXPERIMENTAL + * Experimental * Allows catalyst LogicalPlans to be executed as a SchemaRDD. Note that the LogicalPlan * interface is considered internal, and thus not guranteed to be stable. As a result, using * them directly is not reccomended. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index e1937f7f0d9f8..b7ebce240578a 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 @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.types.BooleanType import org.apache.spark.{Dependency, OneToOneDependency, Partition, TaskContext} /** - * ALPHA COMPONENT + * 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. * @@ -240,7 +240,7 @@ class SchemaRDD( Filter(ScalaUdf(udf, BooleanType, Seq(UnresolvedAttribute(arg1.name))), logicalPlan)) /** - * EXPERIMENTAL + * 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 @@ -258,7 +258,7 @@ class SchemaRDD( Filter(ScalaUdf(dynamicUdf, BooleanType, Seq(WrapDynamic(logicalPlan.output))), logicalPlan)) /** - * EXPERIMENTAL + * Experimental * Returns a sampled version of the underlying dataset. * * @group Query @@ -270,7 +270,7 @@ class SchemaRDD( new SchemaRDD(sqlContext, Sample(fraction, withReplacement, seed, logicalPlan)) /** - * EXPERIMENTAL + * 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 @@ -294,7 +294,7 @@ class SchemaRDD( new SchemaRDD(sqlContext, Generate(generator, join, outer, None, logicalPlan)) /** - * EXPERIMENTAL + * Experimental * Adds the rows from this RDD to the specified table. Note in a standard [[SQLContext]] there is * no notion of persistent tables, and thus queries that contain this operator will fail to * optimize. When working with an extension of a SQLContext that has a persistent catalog, such From f7d124fab52249e7d8b4256ff09580ea7bf74d49 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 7 Apr 2014 17:13:53 -0700 Subject: [PATCH 20/31] Small fixes --- core/src/main/scala/org/apache/spark/SparkContext.scala | 2 +- core/src/main/scala/org/apache/spark/package.scala | 4 ++-- docs/_plugins/copy_api_dirs.rb | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index bf91093a9c65c..dd4833102b72e 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -69,7 +69,7 @@ class SparkContext(config: SparkConf) * @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. - + */ def this(config: SparkConf, preferredNodeLocationData: Map[String, Set[SplitInfo]]) = { this(config) this.preferredNodeLocationData = preferredNodeLocationData diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala index eb78a1ac172ca..59bbb1171f239 100644 --- a/core/src/main/scala/org/apache/spark/package.scala +++ b/core/src/main/scala/org/apache/spark/package.scala @@ -33,11 +33,11 @@ package org.apache * Java programmers should reference the [[spark.api.java]] package * for Spark programming APIs in Java. * - * Classes and methods marked with + * Classes and methods marked with * Experimental are user-facing features which have not been officially adopted by the * Spark project. These are subject to change or removal in minor releases. * - * Classes and methods marked with + * Classes and methods marked with * Developer API are intended for advanced users want to extend Spark through lower * level interfaces. These are subject to changes or removal in minor releases. */ diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index 8d4a525fb51a8..d4ec15bb7d6fa 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -26,7 +26,7 @@ projects = core_projects projects = projects + external_projects.map { |project_name| "external/" + project_name } -# projects = projects + sql_projects.map { |project_name| "sql/" + project_name } + projects = projects + sql_projects.map { |project_name| "sql/" + project_name } puts "Moving to project root and building scaladoc." curr_dir = pwd From 037755cb0d90df0dfc4afc95e066ce7695fd1326 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 7 Apr 2014 18:44:45 -0700 Subject: [PATCH 21/31] Some changes after working with andrew or --- .../org/apache/spark/TaskEndReason.scala | 27 ++++++++++++------- .../apache/spark/executor/TaskMetrics.scala | 4 +-- .../apache/spark/scheduler/JobResult.scala | 7 ++--- .../apache/spark/scheduler/StageInfo.scala | 2 +- .../org/apache/spark/scheduler/TaskInfo.scala | 12 ++++----- .../apache/spark/scheduler/TaskLocality.scala | 3 ++- .../apache/spark/storage/StorageUtils.scala | 3 ++- .../spark/util/collection/AppendOnlyMap.scala | 6 ++--- .../collection/ExternalAppendOnlyMap.scala | 4 +-- .../spark/util/collection/OpenHashMap.scala | 2 +- 10 files changed, 41 insertions(+), 29 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index f1a753b6ab8a9..626f1260cff04 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -21,25 +21,30 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId /** + * Developer API * Various possible reasons why a task ended. The low-level TaskScheduler is supposed to retry * tasks several times for "ephemeral" failures, and only report back failures that require some * old stages to be resubmitted, such as shuffle map fetch failures. */ -private[spark] sealed trait TaskEndReason -private[spark] case object Success extends TaskEndReason +sealed trait TaskEndReason -private[spark] +/** Developer API */ +case object Success extends TaskEndReason + +/** Developer API */ case object Resubmitted extends TaskEndReason // Task was finished earlier but we've now lost it -private[spark] case class FetchFailed( +/** Developer API */ +case class FetchFailed( bmAddress: BlockManagerId, shuffleId: Int, mapId: Int, reduceId: Int) extends TaskEndReason -private[spark] case class ExceptionFailure( +/** Developer API */ +case class ExceptionFailure( className: String, description: String, stackTrace: Array[StackTraceElement], @@ -47,21 +52,25 @@ private[spark] 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. */ -private[spark] case object TaskResultLost extends TaskEndReason +case object TaskResultLost extends TaskEndReason -private[spark] case object TaskKilled extends TaskEndReason +/** Developer API */ +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. */ -private[spark] case object ExecutorLostFailure extends TaskEndReason +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. */ -private[spark] case object UnknownReason extends TaskEndReason +case object UnknownReason extends TaskEndReason 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 e844149ea1ec8..af8ff39313187 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -81,8 +81,8 @@ class TaskMetrics extends Serializable { var updatedBlocks: Option[Seq[(BlockId, BlockStatus)]] = None } -object TaskMetrics { - private[spark] def empty(): TaskMetrics = new TaskMetrics +private[spark] object TaskMetrics { + def empty(): TaskMetrics = new TaskMetrics } 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 3cf4e3077e4a4..1fb6196718445 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala @@ -18,11 +18,12 @@ package org.apache.spark.scheduler /** + * Developer API * A result of a job in the DAGScheduler. */ -private[spark] sealed trait JobResult +sealed trait JobResult -private[spark] case object JobSucceeded extends JobResult +case object JobSucceeded extends JobResult // A failed stage ID of -1 means there is not a particular stage that caused the failure -private[spark] case class JobFailed(exception: Exception, failedStageId: Int) extends JobResult +case class JobFailed(exception: Exception, failedStageId: Int) extends JobResult 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 8115a7ed7896d..419cd96376c04 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -20,9 +20,9 @@ package org.apache.spark.scheduler import org.apache.spark.storage.RDDInfo /** + * Developer API * Stores information about a stage to pass from the scheduler to SparkListeners. */ -private[spark] 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 6183b125def99..515755a93c6e6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala @@ -18,9 +18,9 @@ package org.apache.spark.scheduler /** + * Developer API * Information about a running task attempt inside a TaskSet. */ -private[spark] class TaskInfo( val taskId: Long, val index: Int, @@ -46,15 +46,15 @@ class TaskInfo( var serializedSize: Int = 0 - def markGettingResult(time: Long = System.currentTimeMillis) { + private[spark] def markGettingResult(time: Long = System.currentTimeMillis) { gettingResultTime = time } - def markSuccessful(time: Long = System.currentTimeMillis) { + private[spark] def markSuccessful(time: Long = System.currentTimeMillis) { finishTime = time } - def markFailed(time: Long = System.currentTimeMillis) { + private[spark] def markFailed(time: Long = System.currentTimeMillis) { finishTime = time failed = true } @@ -83,11 +83,11 @@ class TaskInfo( def duration: Long = { if (!finished) { - throw new UnsupportedOperationException("duration() called on unfinished tasks") + throw new UnsupportedOperationException("duration() called on unfinished task") } else { finishTime - launchTime } } - def timeRunning(currentTime: Long): Long = currentTime - launchTime + private[spark] def timeRunning(currentTime: Long): Long = currentTime - launchTime } 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 308edb12edd5c..d2d05a0b81bba 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,8 @@ package org.apache.spark.scheduler -private[spark] object TaskLocality extends Enumeration { +/** Developer API */ +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/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 6153dfe0b7e13..63be6917d5131 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -47,7 +47,8 @@ class StorageStatus( } } -private[spark] + +/** Developer API */ 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/collection/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala index b8c852b4ff5c7..6ee1d96bbc894 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 @@ -20,6 +20,7 @@ package org.apache.spark.util.collection import java.util.{Arrays, Comparator} /** + * 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. * @@ -29,9 +30,8 @@ import java.util.{Arrays, Comparator} * * TODO: Cache the hash values of each key? java.util.HashMap does that. */ -private[spark] -class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, - V)] with Serializable { +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") require(initialCapacity >= 1, "Invalid initial capacity") 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 caa06d5b445b4..f255b258889c0 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 @@ -31,6 +31,7 @@ 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. * @@ -55,8 +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. */ - -private[spark] class ExternalAppendOnlyMap[K, V, C]( +class ExternalAppendOnlyMap[K, V, C]( createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => 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 c26f23d50024a..959fe44f5132a 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 @@ -20,13 +20,13 @@ package org.apache.spark.util.collection import scala.reflect.ClassTag /** + * 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. */ -private[spark] class OpenHashMap[K >: Null : ClassTag, @specialized(Long, Int, Double) V: ClassTag]( initialCapacity: Int) extends Iterable[(K, V)] From 824011bf11aa54780ffde14a25d7141634b8d0f3 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 7 Apr 2014 23:53:29 -0700 Subject: [PATCH 22/31] Add support for injecting arbitrary JavaScript to API docs --- .../scala/org/apache/spark/Aggregator.scala | 3 +- .../scala/org/apache/spark/SparkContext.scala | 3 +- .../org/apache/spark/TaskEndReason.scala | 4 +-- .../spark/annotations/DeveloperAPI.java | 25 +++++++++++++ .../spark/annotations/Experimental.java | 25 +++++++++++++ .../apache/spark/scheduler/StageInfo.scala | 3 +- docs/_config.yml | 2 +- docs/_plugins/copy_api_dirs.rb | 35 +++++++++++-------- docs/js/api-docs.js | 5 +++ 9 files changed, 83 insertions(+), 22 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/annotations/DeveloperAPI.java create mode 100644 core/src/main/scala/org/apache/spark/annotations/Experimental.java create mode 100644 docs/js/api-docs.js diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index d43ef6c430cb1..43ab09004fabf 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -17,16 +17,17 @@ package org.apache.spark +import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.util.collection.{AppendOnlyMap, ExternalAppendOnlyMap} /** - * Developer API * A set of functions used to aggregate data. * * @param createCombiner function to create the initial value of the aggregation. * @param mergeValue function to merge a new value into the aggregation result. * @param mergeCombiners function to merge outputs from multiple mergeValue function. */ +@DeveloperAPI case class Aggregator[K, V, C] ( createCombiner: V => C, mergeValue: (C, V) => C, diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index dd4833102b72e..2884037e7fb30 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -54,8 +54,7 @@ import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerTy * @param config a Spark Config object describing the application configuration. Any settings in * this config overrides the default configs as well as system properties. */ -class SparkContext(config: SparkConf) - extends Logging { +class SparkContext(config: SparkConf) extends Logging { // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, // etc) too. This is typically generated from InputFormatInfo.computePreferredLocations. It diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 626f1260cff04..8dacf9cddd3f3 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -17,16 +17,16 @@ package org.apache.spark +import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId /** - * Developer API * Various possible reasons why a task ended. The low-level TaskScheduler is supposed to retry * tasks several times for "ephemeral" failures, and only report back failures that require some * old stages to be resubmitted, such as shuffle map fetch failures. */ - +@DeveloperAPI sealed trait TaskEndReason /** Developer API */ diff --git a/core/src/main/scala/org/apache/spark/annotations/DeveloperAPI.java b/core/src/main/scala/org/apache/spark/annotations/DeveloperAPI.java new file mode 100644 index 0000000000000..ae04a4a63b9fd --- /dev/null +++ b/core/src/main/scala/org/apache/spark/annotations/DeveloperAPI.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.annotations; + +import java.lang.annotation.*; + +@Retention(RetentionPolicy.SOURCE) +@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, + ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) +public @interface DeveloperAPI {} diff --git a/core/src/main/scala/org/apache/spark/annotations/Experimental.java b/core/src/main/scala/org/apache/spark/annotations/Experimental.java new file mode 100644 index 0000000000000..58445cc2c1e86 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/annotations/Experimental.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.annotations; + +import java.lang.annotation.*; + +@Retention(RetentionPolicy.SOURCE) +@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, + ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) +public @interface Experimental {} diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 419cd96376c04..8d2b752078a91 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -18,11 +18,12 @@ package org.apache.spark.scheduler import org.apache.spark.storage.RDDInfo +import org.apache.spark.annotations.DeveloperAPI /** - * Developer API * Stores information about a stage to pass from the scheduler to SparkListeners. */ +@DeveloperAPI class StageInfo(val stageId: Int, val name: String, val numTasks: Int, val rddInfo: RDDInfo) { /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None diff --git a/docs/_config.yml b/docs/_config.yml index aa5a5adbc1743..5e69c5626f131 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -1,5 +1,5 @@ pygments: true -markdown: kramdown +markdown: rdiscount # These allow the documentation to be updated with nerw releases # of Spark, Scala, and Mesos. diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index d4ec15bb7d6fa..786d3ef6f09a9 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -51,6 +51,11 @@ puts "cp -r " + source + "/. " + dest cp_r(source + "/.", dest) + # Append custom JavaScript + js = File.readlines("./js/api-docs.js") + js_file = dest + "/lib/template.js" + File.open(js_file, 'a') { |f| f.write("\n" + js.join()) } + # Append custom CSS css_file = dest + "/lib/template.css" extra_css = [ @@ -62,19 +67,19 @@ File.open(css_file, 'a') { |f| f.write(extra_css) } end - # Build Epydoc for Python - puts "Moving to python directory and building epydoc." - cd("../python") - puts `epydoc --config epydoc.conf` - - puts "Moving back into docs dir." - cd("../docs") - - puts "echo making directory pyspark" - mkdir_p "pyspark" - - puts "cp -r ../python/docs/. api/pyspark" - cp_r("../python/docs/.", "api/pyspark") - - cd("..") +# # Build Epydoc for Python +# puts "Moving to python directory and building epydoc." +# cd("../python") +# puts `epydoc --config epydoc.conf` +# +# puts "Moving back into docs dir." +# cd("../docs") +# +# puts "echo making directory pyspark" +# mkdir_p "pyspark" +# +# puts "cp -r ../python/docs/. api/pyspark" +# cp_r("../python/docs/.", "api/pyspark") +# +# cd("..") end diff --git a/docs/js/api-docs.js b/docs/js/api-docs.js new file mode 100644 index 0000000000000..ee63d611a18ec --- /dev/null +++ b/docs/js/api-docs.js @@ -0,0 +1,5 @@ +/* Dynamically injected post-processing code for the API docs */ + +$(document).ready(function() { + console.log("Ready") +}); From 99192ef844f6c937067c14bdfebb83b6fa8baa06 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 8 Apr 2014 11:18:58 -0700 Subject: [PATCH 23/31] Dynamically add badges based on annotations This has known problems due to bugs in scaladoc. In particular, on the packages page, annotations appear only if there are no comments for the element. As soon as this is fixed (or there is a workaround) this should basically be ready. --- .../scala/org/apache/spark/Dependency.scala | 11 +++---- .../scala/org/apache/spark/FutureAction.scala | 7 +++-- .../main/scala/org/apache/spark/Logging.scala | 4 ++- .../apache/spark/SerializableWritable.scala | 4 ++- .../scala/org/apache/spark/SparkContext.scala | 11 ++++--- .../scala/org/apache/spark/SparkEnv.scala | 3 +- .../scala/org/apache/spark/TaskContext.scala | 3 +- .../org/apache/spark/TaskEndReason.scala | 16 +++++----- .../spark/annotations/AlphaComponent.java | 25 ++++++++++++++++ .../spark/broadcast/BroadcastFactory.scala | 5 ++-- .../apache/spark/executor/TaskMetrics.scala | 7 +++-- .../apache/spark/io/CompressionCodec.scala | 7 +++-- .../apache/spark/partial/BoundedDouble.scala | 4 ++- .../apache/spark/partial/PartialResult.scala | 6 ++-- .../apache/spark/rdd/AsyncRDDActions.scala | 3 +- .../org/apache/spark/rdd/CoGroupedRDD.scala | 3 +- .../org/apache/spark/rdd/HadoopRDD.scala | 3 +- .../org/apache/spark/rdd/NewHadoopRDD.scala | 3 +- .../spark/rdd/PartitionPruningRDD.scala | 7 ++--- .../main/scala/org/apache/spark/rdd/RDD.scala | 22 +++++++------- .../org/apache/spark/rdd/ShuffledRDD.scala | 3 +- .../scala/org/apache/spark/rdd/UnionRDD.scala | 3 +- .../spark/scheduler/InputFormatInfo.scala | 3 +- .../apache/spark/scheduler/JobLogger.scala | 3 +- .../apache/spark/scheduler/JobResult.scala | 6 +++- .../spark/scheduler/SparkListener.scala | 29 ++++++++++--------- .../apache/spark/scheduler/SplitInfo.scala | 4 ++- .../apache/spark/scheduler/StageInfo.scala | 2 +- .../org/apache/spark/scheduler/TaskInfo.scala | 4 ++- .../apache/spark/scheduler/TaskLocality.scala | 4 ++- .../spark/serializer/JavaSerializer.scala | 3 +- .../apache/spark/serializer/Serializer.scala | 11 +++---- .../apache/spark/storage/StorageUtils.scala | 3 +- .../org/apache/spark/util/MutablePair.scala | 4 ++- .../spark/util/collection/AppendOnlyMap.scala | 4 ++- .../collection/ExternalAppendOnlyMap.scala | 3 +- .../spark/util/collection/OpenHashMap.scala | 4 ++- .../spark/util/random/Pseudorandom.scala | 4 ++- .../spark/util/random/RandomSampler.scala | 8 +++-- docs/js/api-docs.js | 18 +++++++++++- .../org/apache/spark/sql/SQLContext.scala | 5 ++-- .../org/apache/spark/sql/SchemaRDD.scala | 11 +++---- 42 files changed, 192 insertions(+), 101 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/annotations/AlphaComponent.java diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index ae3a921e61658..a37f4691dcbd0 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -17,21 +17,22 @@ package org.apache.spark +import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer /** - * Developer API * Base class for dependencies. */ +@DeveloperAPI abstract class Dependency[T](val rdd: RDD[T]) extends Serializable /** - * Developer API * Base class for dependencies where each partition of the parent RDD is used by at most one * partition of the child RDD. Narrow dependencies allow for pipelined execution. */ +@DeveloperAPI abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { /** * Get the parent partitions for a child partition. @@ -43,7 +44,6 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { /** - * Developer API * Represents a dependency on the output of a shuffle stage. * @param rdd the parent RDD * @param partitioner partitioner used to partition the shuffle output @@ -51,6 +51,7 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { * the default serializer, as specified by `spark.serializer` config option, will * be used. */ +@DeveloperAPI class ShuffleDependency[K, V]( @transient rdd: RDD[_ <: Product2[K, V]], val partitioner: Partitioner, @@ -62,22 +63,22 @@ class ShuffleDependency[K, V]( /** - * Developer API * Represents a one-to-one dependency between partitions of the parent and child RDDs. */ +@DeveloperAPI class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) { override def getParents(partitionId: Int) = List(partitionId) } /** - * Developer API * Represents a one-to-one dependency between ranges of partitions in the parent and child RDDs. * @param rdd the parent RDD * @param inStart the start of the range in the parent RDD * @param outStart the start of the range in the child RDD * @param length the length of the range */ +@DeveloperAPI class RangeDependency[T](rdd: RDD[T], inStart: Int, outStart: Int, length: Int) extends NarrowDependency[T](rdd) { diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index 93b180d180ecf..e71f0e6c5d276 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -21,14 +21,15 @@ import scala.concurrent._ import scala.concurrent.duration.Duration import scala.util.Try +import org.apache.spark.annotations.Experimental import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.{JobFailed, JobSucceeded, JobWaiter} /** - * Experimental * A future for the result of an action to support cancellation. This is an extension of the * Scala Future interface to support cancellation. */ +@Experimental trait FutureAction[T] extends Future[T] { // Note that we redefine methods of the Future trait here explicitly so we can specify a different // documentation (with reference to the word "action"). @@ -85,10 +86,10 @@ trait FutureAction[T] extends Future[T] { /** - * Experimental * A [[FutureAction]] holding the result of an action that triggers a single job. Examples include * count, collect, reduce. */ +@Experimental class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: => T) extends FutureAction[T] { @@ -150,11 +151,11 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: /** - * Experimental * A [[FutureAction]] for actions that could trigger multiple Spark jobs. Examples include take, * takeSample. Cancellation works by setting the cancelled flag to true and interrupting the * action thread if it is being blocked by a job. */ +@Experimental class ComplexFutureAction[T] extends FutureAction[T] { // Pointer to the thread that is executing the action. It is set when the action is run. diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index f4cb7973e52a0..ef6f44c9075e8 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -21,8 +21,9 @@ import org.apache.log4j.{LogManager, PropertyConfigurator} import org.slf4j.{Logger, LoggerFactory} import org.slf4j.impl.StaticLoggerBinder +import org.apache.spark.annotations.DeveloperAPI + /** - * Developer API * Utility trait for classes that want to log data. Creates a SLF4J logger for the class and allows * logging messages at different levels using methods that only evaluate parameters lazily if the * log level is enabled. @@ -30,6 +31,7 @@ import org.slf4j.impl.StaticLoggerBinder * NOTE: DO NOT USE this class outside of Spark. It is intended as an internal utility. * This will likely be changed or removed in future releases. */ +@DeveloperAPI trait Logging { // Make the log field transient so that objects with Logging can // be serialized and used on another machine diff --git a/core/src/main/scala/org/apache/spark/SerializableWritable.scala b/core/src/main/scala/org/apache/spark/SerializableWritable.scala index 73694d09485b1..611be5ddf2e70 100644 --- a/core/src/main/scala/org/apache/spark/SerializableWritable.scala +++ b/core/src/main/scala/org/apache/spark/SerializableWritable.scala @@ -23,7 +23,9 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.ObjectWritable import org.apache.hadoop.io.Writable -/** Developer API */ +import org.apache.spark.annotations.DeveloperAPI + +@DeveloperAPI class SerializableWritable[T <: Writable](@transient var t: T) extends Serializable { def value = t override def toString = t.toString diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 2884037e7fb30..8e6a68a18a267 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -35,6 +35,7 @@ import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHad import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} import org.apache.mesos.MesosNativeLibrary +import org.apache.spark.annotations.{DeveloperAPI, Experimental} import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} @@ -54,6 +55,8 @@ import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerTy * @param config a Spark Config object describing the application configuration. Any settings in * this config overrides the default configs as well as system properties. */ + +@DeveloperAPI class SparkContext(config: SparkConf) extends Logging { // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, @@ -62,13 +65,13 @@ class SparkContext(config: SparkConf) extends Logging { private[spark] var preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map() /** - * Developer API * Alternative constructor for setting preferred locations where Spark will create executors. * * @param preferredNodeLocationData used in YARN mode to select nodes to launch containers on. Ca * be generated using [[org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations]] * from a list of input files or InputFormats for the application. */ + @DeveloperAPI def this(config: SparkConf, preferredNodeLocationData: Map[String, Set[SplitInfo]]) = { this(config) this.preferredNodeLocationData = preferredNodeLocationData @@ -713,9 +716,9 @@ class SparkContext(config: SparkConf) extends Logging { } /** - * Developer API * Register a listener to receive up-calls from events that happen during execution. */ + @DeveloperAPI def addSparkListener(listener: SparkListener) { listenerBus.addListener(listener) } @@ -1025,9 +1028,9 @@ class SparkContext(config: SparkConf) extends Logging { } /** - * Developer API * Run a job that can return approximate results. */ + @DeveloperAPI def runApproximateJob[T, U, R]( rdd: RDD[T], func: (TaskContext, Iterator[T]) => U, @@ -1043,9 +1046,9 @@ class SparkContext(config: SparkConf) extends Logging { } /** - * Experimental * Submit a job for execution and return a FutureJob holding the result. */ + @Experimental def submitJob[T, U, R]( rdd: RDD[T], processPartition: Iterator[T] => U, diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 599688092791e..2850b19cd1503 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -25,6 +25,7 @@ import scala.util.Properties import akka.actor._ import com.google.common.collect.MapMaker +import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.api.python.PythonWorkerFactory import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.metrics.MetricsSystem @@ -35,13 +36,13 @@ import org.apache.spark.storage._ import org.apache.spark.util.{AkkaUtils, Utils} /** - * Developer API * Holds all the runtime environment objects for a running Spark instance (either master or worker), * including the serializer, Akka actor system, block manager, map output tracker, etc. Currently * Spark code finds the SparkEnv through a thread-local variable, so each thread that accesses these * objects needs to have the right SparkEnv set. You can get the current environment with * SparkEnv.get (e.g. after creating a SparkContext) and set it with SparkEnv.set. */ +@DeveloperAPI class SparkEnv ( val executorId: String, val actorSystem: ActorSystem, diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index faccafbdc9a1d..28ef01f4fe6fd 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -19,12 +19,13 @@ package org.apache.spark import scala.collection.mutable.ArrayBuffer +import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.executor.TaskMetrics /** - * Developer API * Contextual information about a task which can be read or mutated during execution. */ +@DeveloperAPI class TaskContext( val stageId: Int, val partitionId: Int, diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 8dacf9cddd3f3..160739bdfc6a9 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -29,13 +29,13 @@ import org.apache.spark.storage.BlockManagerId @DeveloperAPI sealed trait TaskEndReason -/** Developer API */ +@DeveloperAPI case object Success extends TaskEndReason -/** Developer API */ +@DeveloperAPI case object Resubmitted extends TaskEndReason // Task was finished earlier but we've now lost it -/** Developer API */ +@DeveloperAPI case class FetchFailed( bmAddress: BlockManagerId, shuffleId: Int, @@ -43,7 +43,7 @@ case class FetchFailed( reduceId: Int) extends TaskEndReason -/** Developer API */ +@DeveloperAPI case class ExceptionFailure( className: String, description: String, @@ -52,25 +52,25 @@ case class ExceptionFailure( extends TaskEndReason /** - * Developer API * The task finished successfully, but the result was lost from the executor's block manager before * it was fetched. */ +@DeveloperAPI case object TaskResultLost extends TaskEndReason -/** Developer API */ +@DeveloperAPI case object TaskKilled extends TaskEndReason /** - * Developer API * The task failed because the executor that it was running on was lost. This may happen because * the task crashed the JVM. */ +@DeveloperAPI case object ExecutorLostFailure extends TaskEndReason /** - * Developer API * We don't know why the task ended -- for example, because of a ClassNotFound exception when * deserializing the task result. */ +@DeveloperAPI case object UnknownReason extends TaskEndReason diff --git a/core/src/main/scala/org/apache/spark/annotations/AlphaComponent.java b/core/src/main/scala/org/apache/spark/annotations/AlphaComponent.java new file mode 100644 index 0000000000000..d54767ab84c7e --- /dev/null +++ b/core/src/main/scala/org/apache/spark/annotations/AlphaComponent.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.annotations; + +import java.lang.annotation.*; + +@Retention(RetentionPolicy.SOURCE) +@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, + ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) +public @interface AlphaComponent {} diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala index 051547a199cbb..d28860122b2da 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala @@ -16,17 +16,18 @@ */ package org.apache.spark.broadcast -import org.apache.spark.SecurityManager +import org.apache.spark.SecurityManager import org.apache.spark.SparkConf +import org.apache.spark.annotations.DeveloperAPI /** - * Developer API * An interface for all the broadcast implementations in Spark (to allow * multiple broadcast implementations). SparkContext uses a user-specified * BroadcastFactory implementation to instantiate a particular broadcast for the * entire Spark job. */ +@DeveloperAPI trait BroadcastFactory { def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager): Unit def newBroadcast[T](value: T, isLocal: Boolean, id: Long): Broadcast[T] diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index af8ff39313187..b7fe0c3daec8e 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,12 +17,13 @@ package org.apache.spark.executor +import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.storage.{BlockId, BlockStatus} /** - * Developer API * Metrics tracked during the execution of a task. */ +@DeveloperAPI class TaskMetrics extends Serializable { /** * Host's name the task runs on @@ -87,9 +88,9 @@ private[spark] object TaskMetrics { /** - * Developer API * Metrics pertaining to shuffle data read in a given task. */ +@DeveloperAPI class ShuffleReadMetrics extends Serializable { /** * Absolute time when this task finished reading shuffle data @@ -125,9 +126,9 @@ class ShuffleReadMetrics extends Serializable { } /** - * Developer API * Metrics pertaining to shuffle data written in a given task. */ +@DeveloperAPI class ShuffleWriteMetrics extends Serializable { /** * Number of bytes written for the shuffle by this task diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index 0a8fa9f680e5d..3da1551643820 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -23,9 +23,9 @@ import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} import org.apache.spark.SparkConf +import org.apache.spark.annotations.DeveloperAPI /** - * Developer API * CompressionCodec allows the customization of choosing different compression implementations * to be used in block storage. * @@ -33,6 +33,7 @@ import org.apache.spark.SparkConf * This is intended for use as an internal compression utility within a single * Spark application. */ +@DeveloperAPI trait CompressionCodec { def compressedOutputStream(s: OutputStream): OutputStream @@ -57,13 +58,13 @@ private[spark] object CompressionCodec { /** - * Developer API * LZF implementation of [[org.apache.spark.io.CompressionCodec]]. * * Note: The wire protocol for this codec is not guaranteed to be compatible across versions * of Spark. This is intended for use as an internal compression utility within a single Spark * application. */ +@DeveloperAPI class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { override def compressedOutputStream(s: OutputStream): OutputStream = { @@ -75,7 +76,6 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { /** - * Developer API * Snappy implementation of [[org.apache.spark.io.CompressionCodec]]. * Block size can be configured by spark.io.compression.snappy.block.size. * @@ -83,6 +83,7 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { * of Spark. This is intended for use as an internal compression utility within a single Spark * application. */ +@DeveloperAPI class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec { override def compressedOutputStream(s: OutputStream): OutputStream = { diff --git a/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala index 55b979bba0c49..dcd13554c84e1 100644 --- a/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala +++ b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala @@ -17,10 +17,12 @@ package org.apache.spark.partial +import org.apache.spark.annotations.Experimental + /** - * Experimental * A Double value with error bars and associated confidence. */ +@Experimental class BoundedDouble(val mean: Double, val confidence: Double, val low: Double, val high: Double) { override def toString(): String = "[%.3f, %.3f]".format(low, high) } diff --git a/core/src/main/scala/org/apache/spark/partial/PartialResult.scala b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala index 9fa4c12326c11..6d5f4051c11f4 100644 --- a/core/src/main/scala/org/apache/spark/partial/PartialResult.scala +++ b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala @@ -17,9 +17,9 @@ package org.apache.spark.partial -/** - * Experimental - */ +import org.apache.spark.annotations.Experimental + +@Experimental class PartialResult[R](initialVal: R, isFinal: Boolean) { private var finalValue: Option[R] = if (isFinal) Some(initialVal) else None private var failure: Option[Exception] = None diff --git a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala index 05eda70db4545..594d7d2e029df 100644 --- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala @@ -24,12 +24,13 @@ import scala.concurrent.ExecutionContext.Implicits.global import scala.reflect.ClassTag import org.apache.spark.{ComplexFutureAction, FutureAction, Logging} +import org.apache.spark.annotations.Experimental /** - * Experimental * A set of asynchronous RDD actions available through an implicit conversion. * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions. */ +@Experimental class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Logging { /** diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 320a33f64ddd7..b3c42d014c0df 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -23,6 +23,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.{InterruptibleIterator, Partition, Partitioner, SparkEnv, TaskContext} import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency} +import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.util.collection.{ExternalAppendOnlyMap, AppendOnlyMap} import org.apache.spark.serializer.Serializer @@ -51,7 +52,6 @@ private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep] } /** - * Developer API * A RDD that cogroups its parents. For each key k in parent RDDs, the resulting RDD contains a * tuple with the list of values for that key. * @@ -61,6 +61,7 @@ private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep] * @param rdds parent RDDs. * @param part partitioner used to partition the shuffle output */ +@DeveloperAPI class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: Partitioner) extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) { diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 8807e8ca9aa3b..c1beb2aa4116d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -35,6 +35,7 @@ import org.apache.hadoop.mapred.TaskID import org.apache.hadoop.util.ReflectionUtils import org.apache.spark._ +import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.NextIterator @@ -70,7 +71,6 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp } /** - * Developer API * An RDD that provides core functionality for reading data stored in Hadoop (e.g., files in HDFS, * sources in HBase, or S3), using the older MapReduce API (`org.apache.hadoop.mapred`). * @@ -88,6 +88,7 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp * @param valueClass Class of the value associated with the inputFormatClass. * @param minSplits Minimum number of Hadoop Splits (HadoopRDD partitions) to generate. */ +@DeveloperAPI class HadoopRDD[K, V]( sc: SparkContext, broadcastedConf: Broadcast[SerializableWritable[Configuration]], diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 5c97965556ac7..944bb2e1f9305 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -25,6 +25,7 @@ import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ import org.apache.spark.{InterruptibleIterator, Logging, Partition, SerializableWritable, SparkContext, TaskContext} +import org.apache.spark.annotations.DeveloperAPI private[spark] class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputSplit with Writable) @@ -36,7 +37,6 @@ class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputS } /** - * Developer API * An RDD that provides core functionality for reading data stored in Hadoop (e.g., files in HDFS, * sources in HBase, or S3), using the new MapReduce API (`org.apache.hadoop.mapreduce`). * @@ -49,6 +49,7 @@ class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputS * @param valueClass Class of the value associated with the inputFormatClass. * @param conf The Hadoop configuration. */ +@DeveloperAPI class NewHadoopRDD[K, V]( sc : SparkContext, inputFormatClass: Class[_ <: InputFormat[K, V]], diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala index efeb28da4f083..3cbead2e183af 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala @@ -20,6 +20,7 @@ package org.apache.spark.rdd import scala.reflect.ClassTag import org.apache.spark.{NarrowDependency, Partition, TaskContext} +import org.apache.spark.annotations.DeveloperAPI private[spark] class PartitionPruningRDDPartition(idx: Int, val parentSplit: Partition) extends Partition { @@ -46,12 +47,12 @@ private[spark] class PruneDependency[T](rdd: RDD[T], @transient partitionFilterF /** - * Developer API * A RDD used to prune RDD partitions/partitions so we can avoid launching tasks on * all partitions. An example use case: If we know the RDD is partitioned by range, * and the execution DAG has a filter on the key, we can avoid launching tasks * on partitions that don't have the range covering the key. */ +@DeveloperAPI class PartitionPruningRDD[T: ClassTag]( @transient prev: RDD[T], @transient partitionFilterFunc: Int => Boolean) @@ -65,9 +66,7 @@ class PartitionPruningRDD[T: ClassTag]( } -/** - * Developer API - */ +@DeveloperAPI object PartitionPruningRDD { /** diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 67f2fe65cf6d2..3373b81356745 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -35,6 +35,7 @@ import org.apache.hadoop.mapred.TextOutputFormat import org.apache.spark._ import org.apache.spark.Partitioner._ import org.apache.spark.SparkContext._ +import org.apache.spark.annotations.{DeveloperAPI, Experimental} import org.apache.spark.api.java.JavaRDD import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.CountEvaluator @@ -87,28 +88,29 @@ abstract class RDD[T: ClassTag]( // ======================================================================= /** - * Developer API * Implemented by subclasses to compute a given partition. */ + @DeveloperAPI def compute(split: Partition, context: TaskContext): Iterator[T] /** - * Developer API * Implemented by subclasses to return the set of partitions in this RDD. This method will only * be called once, so it is safe to implement a time-consuming computation in it. */ + @DeveloperAPI protected def getPartitions: Array[Partition] /** - * Developer API * Implemented by subclasses to return how this RDD depends on parent RDDs. This method will only * be called once, so it is safe to implement a time-consuming computation in it. */ + @DeveloperAPI protected def getDependencies: Seq[Dependency[_]] = deps /** - * Developer API - * Optionally overridden by subclasses to specify placement preferences. */ + * Optionally overridden by subclasses to specify placement preferences. + */ + @DeveloperAPI protected def getPreferredLocations(split: Partition): Seq[String] = Nil /** Optionally overridden by subclasses to specify how they are partitioned. */ @@ -520,10 +522,10 @@ abstract class RDD[T: ClassTag]( } /** - * Developer API * Return a new RDD by applying a function to each partition of this RDD. This is a variant of * mapPartitions that also passes the TaskContext into the closure. */ + @DeveloperAPI def mapPartitionsWithContext[U: ClassTag]( f: (TaskContext, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = { @@ -783,11 +785,10 @@ abstract class RDD[T: ClassTag]( def count(): Long = sc.runJob(this, Utils.getIteratorSize _).sum /** - * Experimental - * * Approximate version of count() that returns a potentially incomplete result * within a timeout, even if not all tasks have finished. */ + @Experimental def countApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { val countElements: (TaskContext, Iterator[T]) => Long = { (ctx, iter) => var result = 0L @@ -831,10 +832,9 @@ abstract class RDD[T: ClassTag]( } /** - * Experimental - * * Approximate version of countByValue(). */ + @Experimental def countByValueApprox( timeout: Long, confidence: Double = 0.95 @@ -855,7 +855,6 @@ abstract class RDD[T: ClassTag]( } /** - * Experimental * Return approximate number of distinct elements in the RDD. * * The accuracy of approximation can be controlled through the relative standard deviation @@ -863,6 +862,7 @@ abstract class RDD[T: ClassTag]( * more accurate counts but increase the memory footprint and vise versa. The default value of * relativeSD is 0.05. */ + @Experimental def countApproxDistinct(relativeSD: Double = 0.05): Long = { val zeroCounter = new SerializableHyperLogLog(new HyperLogLog(relativeSD)) aggregate(zeroCounter)(_.add(_), _.merge(_)).value.cardinality() diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala index 679759539cfd9..b544e8b4b616e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -20,6 +20,7 @@ package org.apache.spark.rdd import scala.reflect.ClassTag import org.apache.spark.{Dependency, Partition, Partitioner, ShuffleDependency, SparkEnv, TaskContext} +import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.serializer.Serializer private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { @@ -28,13 +29,13 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { } /** - * Developer API * The resulting RDD from a shuffle (e.g. repartitioning of data). * @param prev the parent RDD. * @param part the partitioner used to partition the RDD * @tparam K the key class. * @tparam V the value class. */ +@DeveloperAPI class ShuffledRDD[K, V, P <: Product2[K, V] : ClassTag]( @transient var prev: RDD[P], part: Partitioner) diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala index c40961497a34c..f72400a7e8b35 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -23,6 +23,7 @@ import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag import org.apache.spark.{Dependency, Partition, RangeDependency, SparkContext, TaskContext} +import org.apache.spark.annotations.DeveloperAPI private[spark] class UnionPartition[T: ClassTag](idx: Int, rdd: RDD[T], splitIndex: Int) extends Partition { @@ -43,7 +44,7 @@ private[spark] class UnionPartition[T: ClassTag](idx: Int, rdd: RDD[T], splitInd } } -/** Developer API */ +@DeveloperAPI class UnionRDD[T: ClassTag]( sc: SparkContext, @transient var rdds: Seq[RDD[T]]) diff --git a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala index 79eda49c6fdf9..126dcbe7ed67b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala @@ -27,12 +27,13 @@ import org.apache.hadoop.mapreduce.Job import org.apache.hadoop.util.ReflectionUtils import org.apache.spark.Logging +import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.deploy.SparkHadoopUtil /** - * Developer API * Parses and holds information about inputFormat (and files) specified as a parameter. */ +@DeveloperAPI class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Class[_], val path: String) extends Logging { diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index f85ac36597e30..4a9f409868e94 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -25,10 +25,10 @@ import java.util.concurrent.LinkedBlockingQueue import scala.collection.mutable.HashMap import org.apache.spark._ +import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.executor.TaskMetrics /** - * Developer API * A logger class to record runtime information for jobs in Spark. This class outputs one log file * for each Spark job, containing tasks start/stop and shuffle information. JobLogger is a subclass * of SparkListener, use addSparkListener to add JobLogger to a SparkContext after the SparkContext @@ -40,6 +40,7 @@ import org.apache.spark.executor.TaskMetrics * spark.eventLog.enabled to true. */ +@DeveloperAPI @deprecated("Log application information by setting spark.eventLog.enabled.", "1.0.0") class JobLogger(val user: String, val logDirName: String) extends SparkListener with Logging { diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala index 1fb6196718445..31543464bdf1c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala @@ -17,13 +17,17 @@ package org.apache.spark.scheduler +import org.apache.spark.annotations.DeveloperAPI + /** - * Developer API * A result of a job in the DAGScheduler. */ +@DeveloperAPI sealed trait JobResult +@DeveloperAPI case object JobSucceeded extends JobResult // A failed stage ID of -1 means there is not a particular stage that caused the failure +@DeveloperAPI case class JobFailed(exception: Exception, failedStageId: Int) extends JobResult diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index cbe8c25de5d55..657727d95e395 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -23,27 +23,28 @@ import scala.collection.Map import scala.collection.mutable import org.apache.spark.{Logging, TaskEndReason} +import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.{Distribution, Utils} -/** Developer API */ +@DeveloperAPI sealed trait SparkListenerEvent -/** Developer API */ +@DeveloperAPI case class SparkListenerStageSubmitted(stageInfo: StageInfo, properties: Properties = null) extends SparkListenerEvent -/** Developer API */ +@DeveloperAPI case class SparkListenerStageCompleted(stageInfo: StageInfo) extends SparkListenerEvent -/** Developer API */ +@DeveloperAPI case class SparkListenerTaskStart(stageId: Int, taskInfo: TaskInfo) extends SparkListenerEvent -/** Developer API */ +@DeveloperAPI case class SparkListenerTaskGettingResult(taskInfo: TaskInfo) extends SparkListenerEvent -/** Developer API */ +@DeveloperAPI case class SparkListenerTaskEnd( stageId: Int, taskType: String, @@ -52,26 +53,26 @@ case class SparkListenerTaskEnd( taskMetrics: TaskMetrics) extends SparkListenerEvent -/** Developer API */ +@DeveloperAPI case class SparkListenerJobStart(jobId: Int, stageIds: Seq[Int], properties: Properties = null) extends SparkListenerEvent -/** Developer API */ +@DeveloperAPI case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkListenerEvent -/** Developer API */ +@DeveloperAPI case class SparkListenerEnvironmentUpdate(environmentDetails: Map[String, Seq[(String, String)]]) extends SparkListenerEvent -/** Developer API */ +@DeveloperAPI case class SparkListenerBlockManagerAdded(blockManagerId: BlockManagerId, maxMem: Long) extends SparkListenerEvent -/** Developer API */ +@DeveloperAPI case class SparkListenerBlockManagerRemoved(blockManagerId: BlockManagerId) extends SparkListenerEvent -/** Developer API */ +@DeveloperAPI case class SparkListenerUnpersistRDD(rddId: Int) extends SparkListenerEvent /** An event used in the listener to shutdown the listener daemon thread. */ @@ -79,10 +80,10 @@ private[spark] case object SparkListenerShutdown extends SparkListenerEvent /** - * Developer API * Interface for listening to events from the Spark scheduler. Note that this is an internal * interface which might change in different Spark releases. */ +@DeveloperAPI trait SparkListener { /** * Called when a stage is completed, with information on the completed stage @@ -142,9 +143,9 @@ trait SparkListener { } /** - * Developer API * Simple SparkListener that logs a few summary statistics when each stage completes */ +@DeveloperAPI class StatsReportListener extends SparkListener with Logging { import org.apache.spark.scheduler.StatsReportListener._ diff --git a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala index 4f28bdd991d8c..1c58b5103532c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala @@ -19,9 +19,11 @@ package org.apache.spark.scheduler import collection.mutable.ArrayBuffer +import org.apache.spark.annotations.DeveloperAPI + // information about a specific split instance : handles both split instances. // So that we do not need to worry about the differences. -/** Developer API */ +@DeveloperAPI class SplitInfo(val inputFormatClazz: Class[_], val hostLocation: String, val path: String, val length: Long, val underlyingSplit: Any) { override def toString(): String = { diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 8d2b752078a91..ecca565628cae 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -17,8 +17,8 @@ package org.apache.spark.scheduler -import org.apache.spark.storage.RDDInfo import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.storage.RDDInfo /** * Stores information about a stage to pass from the scheduler to SparkListeners. diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala index 515755a93c6e6..f7ca08bc4603a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala @@ -17,10 +17,12 @@ package org.apache.spark.scheduler +import org.apache.spark.annotations.DeveloperAPI + /** - * Developer API * Information about a running task attempt inside a TaskSet. */ +@DeveloperAPI class TaskInfo( val taskId: Long, val index: Int, diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala index d2d05a0b81bba..dfe447a294d39 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala @@ -17,7 +17,9 @@ package org.apache.spark.scheduler -/** Developer API */ +import org.apache.spark.annotations.DeveloperAPI + +@DeveloperAPI object TaskLocality extends Enumeration { // Process local is expected to be used ONLY within TaskSetManager for now. val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index cf0576713f4ad..8c818f98a0cda 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -21,6 +21,7 @@ import java.io._ import java.nio.ByteBuffer import org.apache.spark.SparkConf +import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.util.ByteBufferInputStream private[spark] class JavaSerializationStream(out: OutputStream, counterReset: Int) @@ -94,13 +95,13 @@ private[spark] class JavaSerializerInstance(counterReset: Int) extends Serialize } /** - * Developer API * A Spark serializer that uses Java's built-in serialization. * * Note that this serializer is not guaranteed to be wire-compatible across different versions of * Spark. It is intended to be used to serialize/de-serialize data within a single * Spark application. */ +@DeveloperAPI class JavaSerializer(conf: SparkConf) extends Serializer with Externalizable { private var counterReset = conf.getInt("spark.serializer.objectStreamReset", 10000) diff --git a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala index 015c4f62e110b..559c223d4e175 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -22,11 +22,11 @@ import java.nio.ByteBuffer import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream -import org.apache.spark.util.{ByteBufferInputStream, NextIterator} import org.apache.spark.SparkEnv +import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.util.{ByteBufferInputStream, NextIterator} /** - * Developer API * A serializer. Because some serialization libraries are not thread safe, this class is used to * create [[org.apache.spark.serializer.SerializerInstance]] objects that do the actual * serialization and are guaranteed to only be called from one thread at a time. @@ -41,6 +41,7 @@ import org.apache.spark.SparkEnv * Note that serializers are not required to be wire-compatible across different versions of Spark. * They are intended to be used to serialize/de-serialize data within a single Spark application. */ +@DeveloperAPI trait Serializer { def newInstance(): SerializerInstance } @@ -54,9 +55,9 @@ object Serializer { /** - * Developer API * An instance of a serializer, for use by one thread at a time. */ +@DeveloperAPI trait SerializerInstance { def serialize[T](t: T): ByteBuffer @@ -87,9 +88,9 @@ trait SerializerInstance { /** - * Developer API * A stream for writing serialized objects. */ +@DeveloperAPI trait SerializationStream { def writeObject[T](t: T): SerializationStream def flush(): Unit @@ -105,9 +106,9 @@ trait SerializationStream { /** - * Developer API * A stream for reading serialized objects. */ +@DeveloperAPI trait DeserializationStream { def readObject[T](): T def close(): Unit diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 63be6917d5131..182c4127a1952 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -21,6 +21,7 @@ import scala.collection.Map import scala.collection.mutable import org.apache.spark.SparkContext +import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.util.Utils private[spark] @@ -48,7 +49,7 @@ class StorageStatus( } -/** Developer API */ +@DeveloperAPI class RDDInfo(val id: Int, val name: String, val numPartitions: Int, val storageLevel: StorageLevel) extends Ordered[RDDInfo] { diff --git a/core/src/main/scala/org/apache/spark/util/MutablePair.scala b/core/src/main/scala/org/apache/spark/util/MutablePair.scala index 75ff7f68e51b7..da8b1c0f5a37e 100644 --- a/core/src/main/scala/org/apache/spark/util/MutablePair.scala +++ b/core/src/main/scala/org/apache/spark/util/MutablePair.scala @@ -17,14 +17,16 @@ package org.apache.spark.util +import org.apache.spark.annotations.DeveloperAPI + /** - * Developer API * A tuple of 2 elements. This can be used as an alternative to Scala's Tuple2 when we want to * minimize object allocation. * * @param _1 Element 1 of this MutablePair * @param _2 Element 2 of this MutablePair */ +@DeveloperAPI case class MutablePair[@specialized(Int, Long, Double, Char, Boolean/* , AnyRef */) T1, @specialized(Int, Long, Double, Char, Boolean/* , AnyRef */) T2] (var _1: T1, var _2: T2) diff --git a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala index 6ee1d96bbc894..4891789d9c4e3 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala @@ -19,8 +19,9 @@ package org.apache.spark.util.collection import java.util.{Arrays, Comparator} +import org.apache.spark.annotations.DeveloperAPI + /** - * Developer API * A simple open hash table optimized for the append-only use case, where keys * are never removed, but the value for each key may be changed. * @@ -30,6 +31,7 @@ import java.util.{Arrays, Comparator} * * TODO: Cache the hash values of each key? java.util.HashMap does that. */ +@DeveloperAPI class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] with Serializable { require(initialCapacity <= (1 << 29), "Can't make capacity bigger than 2^29 elements") diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index f255b258889c0..37c85697d2f5a 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -27,11 +27,11 @@ import com.google.common.io.ByteStreams import it.unimi.dsi.fastutil.io.FastBufferedInputStream import org.apache.spark.{Logging, SparkEnv} +import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{BlockId, BlockManager} /** - * Developer API * An append-only map that spills sorted content to disk when there is insufficient space for it * to grow. * @@ -56,6 +56,7 @@ import org.apache.spark.storage.{BlockId, BlockManager} * `spark.shuffle.safetyFraction` specifies an additional margin of safety as a fraction of * this threshold, in case map size estimation is not sufficiently accurate. */ +@DeveloperAPI class ExternalAppendOnlyMap[K, V, C]( createCombiner: V => C, mergeValue: (C, V) => C, diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala index 959fe44f5132a..b039d426161cf 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala @@ -19,14 +19,16 @@ package org.apache.spark.util.collection import scala.reflect.ClassTag +import org.apache.spark.annotations.DeveloperAPI + /** - * Developer API * A fast hash map implementation for nullable keys. This hash map supports insertions and updates, * but not deletions. This map is about 5X faster than java.util.HashMap, while using much less * space overhead. * * Under the hood, it uses our OpenHashSet implementation. */ +@DeveloperAPI class OpenHashMap[K >: Null : ClassTag, @specialized(Long, Int, Double) V: ClassTag]( initialCapacity: Int) extends Iterable[(K, V)] diff --git a/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala b/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala index e8adc28c2acda..b445318518c5b 100644 --- a/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala +++ b/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala @@ -17,10 +17,12 @@ package org.apache.spark.util.random +import org.apache.spark.annotations.DeveloperAPI + /** - * Developer API * A class with pseudorandom behavior. */ +@DeveloperAPI trait Pseudorandom { /** Set random seed. */ def setSeed(seed: Long) diff --git a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala index a999a607d1b31..bd621d336d481 100644 --- a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala +++ b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala @@ -22,8 +22,9 @@ import java.util.Random import cern.jet.random.Poisson import cern.jet.random.engine.DRand +import org.apache.spark.annotations.DeveloperAPI + /** - * Developer API * A pseudorandom sampler. It is possible to change the sampled item type. For example, we might * want to add weights for stratified sampling or importance sampling. Should only use * transformations that are tied to the sampler and cannot be applied after sampling. @@ -31,6 +32,7 @@ import cern.jet.random.engine.DRand * @tparam T item type * @tparam U sampled item type */ +@DeveloperAPI trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable { /** take a random sample */ @@ -41,7 +43,6 @@ trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable } /** - * Developer API * A sampler based on Bernoulli trials. * * @param lb lower bound of the acceptance range @@ -49,6 +50,7 @@ trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable * @param complement whether to use the complement of the range specified, default to false * @tparam T item type */ +@DeveloperAPI class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) (implicit random: Random = new XORShiftRandom) extends RandomSampler[T, T] { @@ -69,12 +71,12 @@ class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) } /** - * Developer API * A sampler based on values drawn from Poisson distribution. * * @param poisson a Poisson random number generator * @tparam T item type */ +@DeveloperAPI class PoissonSampler[T](mean: Double) (implicit var poisson: Poisson = new Poisson(mean, new DRand)) extends RandomSampler[T, T] { diff --git a/docs/js/api-docs.js b/docs/js/api-docs.js index ee63d611a18ec..3040910d0c9fd 100644 --- a/docs/js/api-docs.js +++ b/docs/js/api-docs.js @@ -1,5 +1,21 @@ /* Dynamically injected post-processing code for the API docs */ $(document).ready(function() { - console.log("Ready") + // Find annotations + var annotations = $("dt:contains('Annotations')").next("dd").children("span.name") + var alphaComponentElements = annotations.children("a[name='org.apache.spark.annotations.AlphaComponent']") + var developerAPIElements = annotations.children("a[name='org.apache.spark.annotations.DeveloperAPI']") + var experimentalElements = annotations.children("a[name='org.apache.spark.annotations.Experimental']") + + // Insert badges into DOM tree + var alphaComponentHTML = "ALPHA COMPONENT" + var developerAPIHTML = "Developer API" + var experimentalHTML = "Experimental" + alphaComponentElements.closest("div.fullcomment").prevAll("h4.signature").prepend(alphaComponentHTML) + alphaComponentElements.closest("div.fullcommenttop").prepend(alphaComponentHTML) + developerAPIElements.closest("div.fullcomment").prevAll("h4.signature").prepend(developerAPIHTML) + developerAPIElements.closest("div.fullcommenttop").prepend(developerAPIHTML) + experimentalElements.closest("div.fullcomment").prevAll("h4.signature").prepend(experimentalHTML) + experimentalElements.closest("div.fullcommenttop").prepend(experimentalHTML) }); + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 5cdae65e29721..18d89f1058f0a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -21,6 +21,7 @@ import scala.language.implicitConversions import scala.reflect.runtime.universe.TypeTag import org.apache.spark.SparkContext +import org.apache.spark.annotations.{AlphaComponent, Experimental} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.dsl @@ -31,13 +32,13 @@ import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.execution._ /** - * ALPHA COMPONENT * The entry point for running relational queries using Spark. Allows the creation of [[SchemaRDD]] * objects and the execution of SQL queries. * * @groupname userf Spark SQL Functions * @groupname Ungrouped Support functions for language integrated queries. */ +@AlphaComponent class SQLContext(@transient val sparkContext: SparkContext) extends Logging with dsl.ExpressionConversions @@ -61,11 +62,11 @@ class SQLContext(@transient val sparkContext: SparkContext) new this.QueryExecution { val logical = plan } /** - * Experimental * Allows catalyst LogicalPlans to be executed as a SchemaRDD. Note that the LogicalPlan * interface is considered internal, and thus not guranteed to be stable. As a result, using * them directly is not reccomended. */ + @Experimental implicit def logicalPlanToSparkQuery(plan: LogicalPlan): SchemaRDD = new SchemaRDD(this, plan) /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index b7ebce240578a..35e9bb70d070a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql +import org.apache.spark.annotations.{AlphaComponent, Experimental} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ @@ -26,7 +27,6 @@ import org.apache.spark.sql.catalyst.types.BooleanType import org.apache.spark.{Dependency, OneToOneDependency, Partition, TaskContext} /** - * ALPHA COMPONENT * An RDD of [[Row]] objects that has an associated schema. In addition to standard RDD functions, * SchemaRDDs can be used in relational queries, as shown in the examples below. * @@ -89,6 +89,7 @@ import org.apache.spark.{Dependency, OneToOneDependency, Partition, TaskContext} * @groupprio schema -1 * @groupname Ungrouped Base RDD Functions */ +@AlphaComponent class SchemaRDD( @transient val sqlContext: SQLContext, @transient val logicalPlan: LogicalPlan) @@ -240,7 +241,6 @@ class SchemaRDD( Filter(ScalaUdf(udf, BooleanType, Seq(UnresolvedAttribute(arg1.name))), logicalPlan)) /** - * Experimental * Filters tuples using a function over a `Dynamic` version of a given Row. DynamicRows use * scala's Dynamic trait to emulate an ORM of in a dynamically typed language. Since the type of * the column is not known at compile time, all attributes are converted to strings before @@ -252,17 +252,18 @@ class SchemaRDD( * * @group Query */ + @Experimental def where(dynamicUdf: (DynamicRow) => Boolean) = new SchemaRDD( sqlContext, Filter(ScalaUdf(dynamicUdf, BooleanType, Seq(WrapDynamic(logicalPlan.output))), logicalPlan)) /** - * Experimental * Returns a sampled version of the underlying dataset. * * @group Query */ + @Experimental def sample( fraction: Double, withReplacement: Boolean = true, @@ -270,7 +271,6 @@ class SchemaRDD( new SchemaRDD(sqlContext, Sample(fraction, withReplacement, seed, logicalPlan)) /** - * Experimental * Applies the given Generator, or table generating function, to this relation. * * @param generator A table generating function. The API for such functions is likely to change @@ -286,6 +286,7 @@ class SchemaRDD( * * @group Query */ + @Experimental def generate( generator: Generator, join: Boolean = false, @@ -294,7 +295,6 @@ class SchemaRDD( new SchemaRDD(sqlContext, Generate(generator, join, outer, None, logicalPlan)) /** - * Experimental * Adds the rows from this RDD to the specified table. Note in a standard [[SQLContext]] there is * no notion of persistent tables, and thus queries that contain this operator will fail to * optimize. When working with an extension of a SQLContext that has a persistent catalog, such @@ -302,6 +302,7 @@ class SchemaRDD( * * @group schema */ + @Experimental def insertInto(tableName: String, overwrite: Boolean = false) = new SchemaRDD( sqlContext, From f3954e0414590e5817a2d3f0517b191ee22b20ee Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 8 Apr 2014 15:35:23 -0700 Subject: [PATCH 24/31] Add identifier tags in comments to work around scaladocs bug The bug is that annotations and comments cannot both be displayed in the summary page. The workaround is adding a special pattern that we grep for as we post-process the DOM tree in JavaScript. Example: A @DeveloperAPI annotated class's comment must begin with ":: Developer API ::" --- .../scala/org/apache/spark/Aggregator.scala | 1 + .../scala/org/apache/spark/Dependency.scala | 5 +++ .../scala/org/apache/spark/FutureAction.scala | 3 ++ .../main/scala/org/apache/spark/Logging.scala | 1 + .../scala/org/apache/spark/SparkContext.scala | 5 +++ .../scala/org/apache/spark/SparkEnv.scala | 1 + .../scala/org/apache/spark/TaskContext.scala | 1 + .../org/apache/spark/TaskEndReason.scala | 4 ++ .../spark/broadcast/BroadcastFactory.scala | 3 +- .../apache/spark/executor/TaskMetrics.scala | 3 ++ .../apache/spark/io/CompressionCodec.scala | 3 ++ .../apache/spark/partial/BoundedDouble.scala | 1 + .../apache/spark/rdd/AsyncRDDActions.scala | 1 + .../org/apache/spark/rdd/CoGroupedRDD.scala | 1 + .../org/apache/spark/rdd/HadoopRDD.scala | 1 + .../org/apache/spark/rdd/NewHadoopRDD.scala | 1 + .../spark/rdd/PartitionPruningRDD.scala | 1 + .../main/scala/org/apache/spark/rdd/RDD.scala | 8 ++++ .../org/apache/spark/rdd/ShuffledRDD.scala | 1 + .../spark/scheduler/InputFormatInfo.scala | 1 + .../apache/spark/scheduler/JobLogger.scala | 2 +- .../apache/spark/scheduler/JobResult.scala | 1 + .../spark/scheduler/SparkListener.scala | 2 + .../apache/spark/scheduler/StageInfo.scala | 1 + .../org/apache/spark/scheduler/TaskInfo.scala | 1 + .../spark/serializer/JavaSerializer.scala | 1 + .../apache/spark/serializer/Serializer.scala | 4 ++ .../org/apache/spark/util/MutablePair.scala | 1 + .../spark/util/collection/AppendOnlyMap.scala | 1 + .../collection/ExternalAppendOnlyMap.scala | 1 + .../spark/util/collection/OpenHashMap.scala | 1 + .../spark/util/random/Pseudorandom.scala | 1 + .../spark/util/random/RandomSampler.scala | 3 ++ docs/_plugins/copy_api_dirs.rb | 9 +---- docs/css/api-docs.css | 14 +++++++ docs/js/api-docs.js | 38 +++++++++++-------- .../org/apache/spark/sql/SQLContext.scala | 2 + .../org/apache/spark/sql/SchemaRDD.scala | 5 +++ 38 files changed, 109 insertions(+), 25 deletions(-) create mode 100644 docs/css/api-docs.css diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index 43ab09004fabf..5bc9b880f4e94 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -21,6 +21,7 @@ import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.util.collection.{AppendOnlyMap, ExternalAppendOnlyMap} /** + * :: DeveloperAPI :: * A set of functions used to aggregate data. * * @param createCombiner function to create the initial value of the aggregation. diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index a37f4691dcbd0..73d1c40d523e6 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -22,6 +22,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer /** + * :: DeveloperAPI :: * Base class for dependencies. */ @DeveloperAPI @@ -29,6 +30,7 @@ abstract class Dependency[T](val rdd: RDD[T]) extends Serializable /** + * :: DeveloperAPI :: * Base class for dependencies where each partition of the parent RDD is used by at most one * partition of the child RDD. Narrow dependencies allow for pipelined execution. */ @@ -44,6 +46,7 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { /** + * :: DeveloperAPI :: * Represents a dependency on the output of a shuffle stage. * @param rdd the parent RDD * @param partitioner partitioner used to partition the shuffle output @@ -63,6 +66,7 @@ class ShuffleDependency[K, V]( /** + * :: DeveloperAPI :: * Represents a one-to-one dependency between partitions of the parent and child RDDs. */ @DeveloperAPI @@ -72,6 +76,7 @@ class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) { /** + * :: DeveloperAPI :: * Represents a one-to-one dependency between ranges of partitions in the parent and child RDDs. * @param rdd the parent RDD * @param inStart the start of the range in the parent RDD diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index e71f0e6c5d276..07c499f49d2b1 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -26,6 +26,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.{JobFailed, JobSucceeded, JobWaiter} /** + * :: Experimental :: * A future for the result of an action to support cancellation. This is an extension of the * Scala Future interface to support cancellation. */ @@ -86,6 +87,7 @@ trait FutureAction[T] extends Future[T] { /** + * :: Experimental :: * A [[FutureAction]] holding the result of an action that triggers a single job. Examples include * count, collect, reduce. */ @@ -151,6 +153,7 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: /** + * :: Experimental :: * A [[FutureAction]] for actions that could trigger multiple Spark jobs. Examples include take, * takeSample. Cancellation works by setting the cancelled flag to true and interrupting the * action thread if it is being blocked by a job. diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index ef6f44c9075e8..2c6b62dcc09e0 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -24,6 +24,7 @@ import org.slf4j.impl.StaticLoggerBinder import org.apache.spark.annotations.DeveloperAPI /** + * :: DeveloperAPI :: * Utility trait for classes that want to log data. Creates a SLF4J logger for the class and allows * logging messages at different levels using methods that only evaluate parameters lazily if the * log level is enabled. diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 8e6a68a18a267..d9aa8e614eb9b 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -49,6 +49,7 @@ import org.apache.spark.ui.SparkUI import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedHashMap, Utils} /** + * :: DeveloperAPI :: * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark * cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster. * @@ -65,6 +66,7 @@ class SparkContext(config: SparkConf) extends Logging { private[spark] var preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map() /** + * :: DeveloperAPI :: * Alternative constructor for setting preferred locations where Spark will create executors. * * @param preferredNodeLocationData used in YARN mode to select nodes to launch containers on. Ca @@ -716,6 +718,7 @@ class SparkContext(config: SparkConf) extends Logging { } /** + * :: DeveloperAPI :: * Register a listener to receive up-calls from events that happen during execution. */ @DeveloperAPI @@ -1028,6 +1031,7 @@ class SparkContext(config: SparkConf) extends Logging { } /** + * :: DeveloperAPI :: * Run a job that can return approximate results. */ @DeveloperAPI @@ -1046,6 +1050,7 @@ class SparkContext(config: SparkConf) extends Logging { } /** + * * Submit a job for execution and return a FutureJob holding the result. */ @Experimental diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 2850b19cd1503..5f63b6b5bd4f0 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -36,6 +36,7 @@ import org.apache.spark.storage._ import org.apache.spark.util.{AkkaUtils, Utils} /** + * :: DeveloperAPI :: * Holds all the runtime environment objects for a running Spark instance (either master or worker), * including the serializer, Akka actor system, block manager, map output tracker, etc. Currently * Spark code finds the SparkEnv through a thread-local variable, so each thread that accesses these diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 28ef01f4fe6fd..58e39abd6e0bd 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -23,6 +23,7 @@ import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.executor.TaskMetrics /** + * :: DeveloperAPI :: * Contextual information about a task which can be read or mutated during execution. */ @DeveloperAPI diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 160739bdfc6a9..cd73508711a1e 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -22,6 +22,7 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId /** + * :: DeveloperAPI :: * Various possible reasons why a task ended. The low-level TaskScheduler is supposed to retry * tasks several times for "ephemeral" failures, and only report back failures that require some * old stages to be resubmitted, such as shuffle map fetch failures. @@ -52,6 +53,7 @@ case class ExceptionFailure( extends TaskEndReason /** + * :: DeveloperAPI :: * The task finished successfully, but the result was lost from the executor's block manager before * it was fetched. */ @@ -62,6 +64,7 @@ case object TaskResultLost extends TaskEndReason case object TaskKilled extends TaskEndReason /** + * :: DeveloperAPI :: * The task failed because the executor that it was running on was lost. This may happen because * the task crashed the JVM. */ @@ -69,6 +72,7 @@ case object TaskKilled extends TaskEndReason case object ExecutorLostFailure extends TaskEndReason /** + * :: DeveloperAPI :: * We don't know why the task ended -- for example, because of a ClassNotFound exception when * deserializing the task result. */ diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala index d28860122b2da..67faa217c46bf 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala @@ -22,7 +22,8 @@ import org.apache.spark.SparkConf import org.apache.spark.annotations.DeveloperAPI /** - * An interface for all the broadcast implementations in Spark (to allow + * :: DeveloperAPI :: + * An interface for all the broadcast implementations in Spark (to allow * multiple broadcast implementations). SparkContext uses a user-specified * BroadcastFactory implementation to instantiate a particular broadcast for the * entire Spark job. diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index b7fe0c3daec8e..ed1b45f5d485b 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -21,6 +21,7 @@ import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.storage.{BlockId, BlockStatus} /** + * :: DeveloperAPI :: * Metrics tracked during the execution of a task. */ @DeveloperAPI @@ -88,6 +89,7 @@ private[spark] object TaskMetrics { /** + * :: DeveloperAPI :: * Metrics pertaining to shuffle data read in a given task. */ @DeveloperAPI @@ -126,6 +128,7 @@ class ShuffleReadMetrics extends Serializable { } /** + * :: DeveloperAPI :: * Metrics pertaining to shuffle data written in a given task. */ @DeveloperAPI diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index 3da1551643820..1f0134d0cdf3e 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -26,6 +26,7 @@ import org.apache.spark.SparkConf import org.apache.spark.annotations.DeveloperAPI /** + * :: DeveloperAPI :: * CompressionCodec allows the customization of choosing different compression implementations * to be used in block storage. * @@ -58,6 +59,7 @@ private[spark] object CompressionCodec { /** + * :: DeveloperAPI :: * LZF implementation of [[org.apache.spark.io.CompressionCodec]]. * * Note: The wire protocol for this codec is not guaranteed to be compatible across versions @@ -76,6 +78,7 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { /** + * :: DeveloperAPI :: * Snappy implementation of [[org.apache.spark.io.CompressionCodec]]. * Block size can be configured by spark.io.compression.snappy.block.size. * diff --git a/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala index dcd13554c84e1..ad11fccaf20a2 100644 --- a/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala +++ b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala @@ -20,6 +20,7 @@ package org.apache.spark.partial import org.apache.spark.annotations.Experimental /** + * :: Experimental :: * A Double value with error bars and associated confidence. */ @Experimental diff --git a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala index 594d7d2e029df..85ea8f42b39ee 100644 --- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala @@ -27,6 +27,7 @@ import org.apache.spark.{ComplexFutureAction, FutureAction, Logging} import org.apache.spark.annotations.Experimental /** + * :: Experimental :: * A set of asynchronous RDD actions available through an implicit conversion. * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions. */ diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index b3c42d014c0df..f8c3cda91781c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -52,6 +52,7 @@ private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep] } /** + * :: DeveloperAPI :: * A RDD that cogroups its parents. For each key k in parent RDDs, the resulting RDD contains a * tuple with the list of values for that key. * diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index c1beb2aa4116d..ea03b3b8e4861 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -71,6 +71,7 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp } /** + * :: DeveloperAPI :: * An RDD that provides core functionality for reading data stored in Hadoop (e.g., files in HDFS, * sources in HBase, or S3), using the older MapReduce API (`org.apache.hadoop.mapred`). * diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 944bb2e1f9305..510f0cfbd8be2 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -37,6 +37,7 @@ class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputS } /** + * :: DeveloperAPI :: * An RDD that provides core functionality for reading data stored in Hadoop (e.g., files in HDFS, * sources in HBase, or S3), using the new MapReduce API (`org.apache.hadoop.mapreduce`). * diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala index 3cbead2e183af..160236f943715 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala @@ -47,6 +47,7 @@ private[spark] class PruneDependency[T](rdd: RDD[T], @transient partitionFilterF /** + * :: DeveloperAPI :: * A RDD used to prune RDD partitions/partitions so we can avoid launching tasks on * all partitions. An example use case: If we know the RDD is partitioned by range, * and the execution DAG has a filter on the key, we can avoid launching tasks diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 3373b81356745..c56b67ef9b643 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -88,12 +88,14 @@ abstract class RDD[T: ClassTag]( // ======================================================================= /** + * :: DeveloperAPI :: * Implemented by subclasses to compute a given partition. */ @DeveloperAPI def compute(split: Partition, context: TaskContext): Iterator[T] /** + * :: DeveloperAPI :: * Implemented by subclasses to return the set of partitions in this RDD. This method will only * be called once, so it is safe to implement a time-consuming computation in it. */ @@ -101,6 +103,7 @@ abstract class RDD[T: ClassTag]( protected def getPartitions: Array[Partition] /** + * :: DeveloperAPI :: * Implemented by subclasses to return how this RDD depends on parent RDDs. This method will only * be called once, so it is safe to implement a time-consuming computation in it. */ @@ -108,6 +111,7 @@ abstract class RDD[T: ClassTag]( protected def getDependencies: Seq[Dependency[_]] = deps /** + * :: DeveloperAPI :: * Optionally overridden by subclasses to specify placement preferences. */ @DeveloperAPI @@ -522,6 +526,7 @@ abstract class RDD[T: ClassTag]( } /** + * :: DeveloperAPI :: * Return a new RDD by applying a function to each partition of this RDD. This is a variant of * mapPartitions that also passes the TaskContext into the closure. */ @@ -785,6 +790,7 @@ abstract class RDD[T: ClassTag]( def count(): Long = sc.runJob(this, Utils.getIteratorSize _).sum /** + * :: Experimental :: * Approximate version of count() that returns a potentially incomplete result * within a timeout, even if not all tasks have finished. */ @@ -832,6 +838,7 @@ abstract class RDD[T: ClassTag]( } /** + * :: Experimental :: * Approximate version of countByValue(). */ @Experimental @@ -855,6 +862,7 @@ abstract class RDD[T: ClassTag]( } /** + * :: Experimental :: * Return approximate number of distinct elements in the RDD. * * The accuracy of approximation can be controlled through the relative standard deviation diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala index b544e8b4b616e..e9971cdabdba6 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -29,6 +29,7 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { } /** + * :: DeveloperAPI :: * The resulting RDD from a shuffle (e.g. repartitioning of data). * @param prev the parent RDD. * @param part the partitioner used to partition the RDD diff --git a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala index 126dcbe7ed67b..6e02ea4ac7d7d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala @@ -31,6 +31,7 @@ import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.deploy.SparkHadoopUtil /** + * :: DeveloperAPI :: * Parses and holds information about inputFormat (and files) specified as a parameter. */ @DeveloperAPI diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 4a9f409868e94..fd458734ab09e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -29,6 +29,7 @@ import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.executor.TaskMetrics /** + * :: DeveloperAPI :: * A logger class to record runtime information for jobs in Spark. This class outputs one log file * for each Spark job, containing tasks start/stop and shuffle information. JobLogger is a subclass * of SparkListener, use addSparkListener to add JobLogger to a SparkContext after the SparkContext @@ -39,7 +40,6 @@ import org.apache.spark.executor.TaskMetrics * to log application information as SparkListenerEvents. To enable this functionality, set * spark.eventLog.enabled to true. */ - @DeveloperAPI @deprecated("Log application information by setting spark.eventLog.enabled.", "1.0.0") class JobLogger(val user: String, val logDirName: String) extends SparkListener with Logging { diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala index 31543464bdf1c..d8c443f6130ae 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler import org.apache.spark.annotations.DeveloperAPI /** + * :: DeveloperAPI :: * A result of a job in the DAGScheduler. */ @DeveloperAPI diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 657727d95e395..72ff68f97e0ab 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -80,6 +80,7 @@ private[spark] case object SparkListenerShutdown extends SparkListenerEvent /** + * :: DeveloperAPI :: * Interface for listening to events from the Spark scheduler. Note that this is an internal * interface which might change in different Spark releases. */ @@ -143,6 +144,7 @@ trait SparkListener { } /** + * :: DeveloperAPI :: * Simple SparkListener that logs a few summary statistics when each stage completes */ @DeveloperAPI diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index ecca565628cae..460f4109d536b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -21,6 +21,7 @@ import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.storage.RDDInfo /** + * :: DeveloperAPI :: * Stores information about a stage to pass from the scheduler to SparkListeners. */ @DeveloperAPI diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala index f7ca08bc4603a..04f891b6e72ed 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler import org.apache.spark.annotations.DeveloperAPI /** + * :: DeveloperAPI :: * Information about a running task attempt inside a TaskSet. */ @DeveloperAPI diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index 8c818f98a0cda..9880bd916a9bc 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -95,6 +95,7 @@ private[spark] class JavaSerializerInstance(counterReset: Int) extends Serialize } /** + * :: DeveloperAPI :: * A Spark serializer that uses Java's built-in serialization. * * Note that this serializer is not guaranteed to be wire-compatible across different versions of diff --git a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala index 559c223d4e175..cd19e45132ad4 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -27,6 +27,7 @@ import org.apache.spark.annotations.DeveloperAPI import org.apache.spark.util.{ByteBufferInputStream, NextIterator} /** + * :: DeveloperAPI :: * A serializer. Because some serialization libraries are not thread safe, this class is used to * create [[org.apache.spark.serializer.SerializerInstance]] objects that do the actual * serialization and are guaranteed to only be called from one thread at a time. @@ -55,6 +56,7 @@ object Serializer { /** + * :: DeveloperAPI :: * An instance of a serializer, for use by one thread at a time. */ @DeveloperAPI @@ -88,6 +90,7 @@ trait SerializerInstance { /** + * :: DeveloperAPI :: * A stream for writing serialized objects. */ @DeveloperAPI @@ -106,6 +109,7 @@ trait SerializationStream { /** + * :: DeveloperAPI :: * A stream for reading serialized objects. */ @DeveloperAPI diff --git a/core/src/main/scala/org/apache/spark/util/MutablePair.scala b/core/src/main/scala/org/apache/spark/util/MutablePair.scala index da8b1c0f5a37e..2b62e9e4e25e8 100644 --- a/core/src/main/scala/org/apache/spark/util/MutablePair.scala +++ b/core/src/main/scala/org/apache/spark/util/MutablePair.scala @@ -20,6 +20,7 @@ package org.apache.spark.util import org.apache.spark.annotations.DeveloperAPI /** + * :: DeveloperAPI :: * A tuple of 2 elements. This can be used as an alternative to Scala's Tuple2 when we want to * minimize object allocation. * diff --git a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala index 4891789d9c4e3..35d99f7e52d79 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala @@ -22,6 +22,7 @@ import java.util.{Arrays, Comparator} import org.apache.spark.annotations.DeveloperAPI /** + * :: DeveloperAPI :: * A simple open hash table optimized for the append-only use case, where keys * are never removed, but the value for each key may be changed. * diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 37c85697d2f5a..3e397f53aaaf3 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -32,6 +32,7 @@ import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{BlockId, BlockManager} /** + * :: DeveloperAPI :: * An append-only map that spills sorted content to disk when there is insufficient space for it * to grow. * diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala index b039d426161cf..418b6169c16ca 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala @@ -22,6 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.annotations.DeveloperAPI /** + * :: DeveloperAPI :: * A fast hash map implementation for nullable keys. This hash map supports insertions and updates, * but not deletions. This map is about 5X faster than java.util.HashMap, while using much less * space overhead. diff --git a/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala b/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala index b445318518c5b..77862518bd678 100644 --- a/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala +++ b/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala @@ -20,6 +20,7 @@ package org.apache.spark.util.random import org.apache.spark.annotations.DeveloperAPI /** + * :: DeveloperAPI :: * A class with pseudorandom behavior. */ @DeveloperAPI diff --git a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala index bd621d336d481..cbc15873f8cce 100644 --- a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala +++ b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala @@ -25,6 +25,7 @@ import cern.jet.random.engine.DRand import org.apache.spark.annotations.DeveloperAPI /** + * :: DeveloperAPI :: * A pseudorandom sampler. It is possible to change the sampled item type. For example, we might * want to add weights for stratified sampling or importance sampling. Should only use * transformations that are tied to the sampler and cannot be applied after sampling. @@ -43,6 +44,7 @@ trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable } /** + * :: DeveloperAPI :: * A sampler based on Bernoulli trials. * * @param lb lower bound of the acceptance range @@ -71,6 +73,7 @@ class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) } /** + * :: DeveloperAPI :: * A sampler based on values drawn from Poisson distribution. * * @param poisson a Poisson random number generator diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index 786d3ef6f09a9..df03ea5c620ae 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -57,14 +57,9 @@ File.open(js_file, 'a') { |f| f.write("\n" + js.join()) } # Append custom CSS + css = File.readlines("./css/api-docs.css") css_file = dest + "/lib/template.css" - extra_css = [ - "", - "/* Styles added by spark jekyll plug-in */", - ".developer {background-color: #44751E; float: right; text-transform: uppercase;}", - ".experimental {background-color: #257080; float: right; text-transform: uppercase;}", - ].join("\n") - File.open(css_file, 'a') { |f| f.write(extra_css) } + File.open(css_file, 'a') { |f| f.write("\n" + css.join()) } end # # Build Epydoc for Python diff --git a/docs/css/api-docs.css b/docs/css/api-docs.css new file mode 100644 index 0000000000000..cc5f37bbdb42e --- /dev/null +++ b/docs/css/api-docs.css @@ -0,0 +1,14 @@ +/* Dynamically injected style for the API docs */ + +.developer { + background-color: #44751E; +} + +.experimental { + background-color: #257080; +} + +.badge { + font-family: Arial, san-serif; + float: right; +} diff --git a/docs/js/api-docs.js b/docs/js/api-docs.js index 3040910d0c9fd..832f41958fc81 100644 --- a/docs/js/api-docs.js +++ b/docs/js/api-docs.js @@ -1,21 +1,27 @@ /* Dynamically injected post-processing code for the API docs */ $(document).ready(function() { - // Find annotations - var annotations = $("dt:contains('Annotations')").next("dd").children("span.name") - var alphaComponentElements = annotations.children("a[name='org.apache.spark.annotations.AlphaComponent']") - var developerAPIElements = annotations.children("a[name='org.apache.spark.annotations.DeveloperAPI']") - var experimentalElements = annotations.children("a[name='org.apache.spark.annotations.Experimental']") - - // Insert badges into DOM tree - var alphaComponentHTML = "ALPHA COMPONENT" - var developerAPIHTML = "Developer API" - var experimentalHTML = "Experimental" - alphaComponentElements.closest("div.fullcomment").prevAll("h4.signature").prepend(alphaComponentHTML) - alphaComponentElements.closest("div.fullcommenttop").prepend(alphaComponentHTML) - developerAPIElements.closest("div.fullcomment").prevAll("h4.signature").prepend(developerAPIHTML) - developerAPIElements.closest("div.fullcommenttop").prepend(developerAPIHTML) - experimentalElements.closest("div.fullcomment").prevAll("h4.signature").prepend(experimentalHTML) - experimentalElements.closest("div.fullcommenttop").prepend(experimentalHTML) + var annotations = $("dt:contains('Annotations')").next("dd").children("span.name"); + addBadges(annotations, "AlphaComponent", ":: AlphaComponent ::", "ALPHA COMPONENT"); + addBadges(annotations, "DeveloperAPI", ":: DeveloperAPI ::", "Developer API"); + addBadges(annotations, "Experimental", ":: Experimental ::", "Experimental"); }); +function addBadges(allAnnotations, name, tag, html) { + var fullName = "org.apache.spark.annotations." + name; + var annotations = allAnnotations.children("a[name='" + fullName + "']"); + var tags = $("p.comment:contains(" + tag + ")").add( + $("div.comment p:contains(" + tag + ")")); + + // Remove identifier tags from comments + tags.each(function(index) { + var oldHTML = $(this).html(); + var newHTML = oldHTML.replace(tag, ""); + $(this).html(newHTML); + }); + + // Add badges to all containers + tags.prevAll("h4.signature").prepend(html); + annotations.closest("div.fullcomment").prevAll("h4.signature").prepend(html); + annotations.closest("div.fullcommenttop").prepend(html); +} \ No newline at end of file diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 18d89f1058f0a..b8e4867f81a7f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.execution._ /** + * :: AlphaComponent :: * The entry point for running relational queries using Spark. Allows the creation of [[SchemaRDD]] * objects and the execution of SQL queries. * @@ -62,6 +63,7 @@ class SQLContext(@transient val sparkContext: SparkContext) new this.QueryExecution { val logical = plan } /** + * :: Experimental :: * Allows catalyst LogicalPlans to be executed as a SchemaRDD. Note that the LogicalPlan * interface is considered internal, and thus not guranteed to be stable. As a result, using * them directly is not reccomended. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 35e9bb70d070a..8eaddd5d0770e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.types.BooleanType import org.apache.spark.{Dependency, OneToOneDependency, Partition, TaskContext} /** + * :: AlphaComponent :: * An RDD of [[Row]] objects that has an associated schema. In addition to standard RDD functions, * SchemaRDDs can be used in relational queries, as shown in the examples below. * @@ -241,6 +242,7 @@ class SchemaRDD( Filter(ScalaUdf(udf, BooleanType, Seq(UnresolvedAttribute(arg1.name))), logicalPlan)) /** + * :: Experimental :: * Filters tuples using a function over a `Dynamic` version of a given Row. DynamicRows use * scala's Dynamic trait to emulate an ORM of in a dynamically typed language. Since the type of * the column is not known at compile time, all attributes are converted to strings before @@ -259,6 +261,7 @@ class SchemaRDD( Filter(ScalaUdf(dynamicUdf, BooleanType, Seq(WrapDynamic(logicalPlan.output))), logicalPlan)) /** + * :: Experimental :: * Returns a sampled version of the underlying dataset. * * @group Query @@ -271,6 +274,7 @@ class SchemaRDD( new SchemaRDD(sqlContext, Sample(fraction, withReplacement, seed, logicalPlan)) /** + * :: Experimental :: * Applies the given Generator, or table generating function, to this relation. * * @param generator A table generating function. The API for such functions is likely to change @@ -295,6 +299,7 @@ class SchemaRDD( new SchemaRDD(sqlContext, Generate(generator, join, outer, None, logicalPlan)) /** + * :: Experimental :: * Adds the rows from this RDD to the specified table. Note in a standard [[SQLContext]] there is * no notion of persistent tables, and thus queries that contain this operator will fail to * optimize. When working with an extension of a SQLContext that has a persistent catalog, such From 0d489081ca4f55fb48ea5b38d8b7784e944213e2 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 8 Apr 2014 15:45:03 -0700 Subject: [PATCH 25/31] Comments and new lines (minor) --- .../scala/org/apache/spark/SparkContext.scala | 1 - docs/_config.yml | 2 +- docs/_plugins/copy_api_dirs.rb | 30 +++++++++---------- docs/js/api-docs.js | 2 +- 4 files changed, 17 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index d9aa8e614eb9b..46f1980ffe942 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1050,7 +1050,6 @@ class SparkContext(config: SparkConf) extends Logging { } /** - * * Submit a job for execution and return a FutureJob holding the result. */ @Experimental diff --git a/docs/_config.yml b/docs/_config.yml index 5e69c5626f131..aa5a5adbc1743 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -1,5 +1,5 @@ pygments: true -markdown: rdiscount +markdown: kramdown # These allow the documentation to be updated with nerw releases # of Spark, Scala, and Mesos. diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index df03ea5c620ae..bbd56d2fd13bb 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -62,19 +62,19 @@ File.open(css_file, 'a') { |f| f.write("\n" + css.join()) } end -# # Build Epydoc for Python -# puts "Moving to python directory and building epydoc." -# cd("../python") -# puts `epydoc --config epydoc.conf` -# -# puts "Moving back into docs dir." -# cd("../docs") -# -# puts "echo making directory pyspark" -# mkdir_p "pyspark" -# -# puts "cp -r ../python/docs/. api/pyspark" -# cp_r("../python/docs/.", "api/pyspark") -# -# cd("..") + # Build Epydoc for Python + puts "Moving to python directory and building epydoc." + cd("../python") + puts `epydoc --config epydoc.conf` + + puts "Moving back into docs dir." + cd("../docs") + + puts "echo making directory pyspark" + mkdir_p "pyspark" + + puts "cp -r ../python/docs/. api/pyspark" + cp_r("../python/docs/.", "api/pyspark") + + cd("..") end diff --git a/docs/js/api-docs.js b/docs/js/api-docs.js index 832f41958fc81..1d234658e7ee9 100644 --- a/docs/js/api-docs.js +++ b/docs/js/api-docs.js @@ -24,4 +24,4 @@ function addBadges(allAnnotations, name, tag, html) { tags.prevAll("h4.signature").prepend(html); annotations.closest("div.fullcomment").prevAll("h4.signature").prepend(html); annotations.closest("div.fullcommenttop").prepend(html); -} \ No newline at end of file +} From c1bcb4177f3a544fcdde2bc2d05f98de48ebfcf2 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 8 Apr 2014 16:35:43 -0700 Subject: [PATCH 26/31] DeveloperAPI -> DeveloperApi --- .../scala/org/apache/spark/Aggregator.scala | 6 ++-- .../scala/org/apache/spark/Dependency.scala | 22 ++++++------ .../main/scala/org/apache/spark/Logging.scala | 6 ++-- .../apache/spark/SerializableWritable.scala | 4 +-- .../scala/org/apache/spark/SparkContext.scala | 18 +++++----- .../scala/org/apache/spark/SparkEnv.scala | 6 ++-- .../scala/org/apache/spark/TaskContext.scala | 6 ++-- .../org/apache/spark/TaskEndReason.scala | 28 +++++++-------- .../{DeveloperAPI.java => DeveloperApi.java} | 2 +- .../spark/broadcast/BroadcastFactory.scala | 6 ++-- .../apache/spark/executor/TaskMetrics.scala | 14 ++++---- .../apache/spark/io/CompressionCodec.scala | 14 ++++---- .../org/apache/spark/rdd/CoGroupedRDD.scala | 6 ++-- .../org/apache/spark/rdd/HadoopRDD.scala | 6 ++-- .../org/apache/spark/rdd/NewHadoopRDD.scala | 6 ++-- .../spark/rdd/PartitionPruningRDD.scala | 8 ++--- .../main/scala/org/apache/spark/rdd/RDD.scala | 22 ++++++------ .../org/apache/spark/rdd/ShuffledRDD.scala | 6 ++-- .../scala/org/apache/spark/rdd/UnionRDD.scala | 4 +-- .../spark/scheduler/InputFormatInfo.scala | 6 ++-- .../apache/spark/scheduler/JobLogger.scala | 6 ++-- .../apache/spark/scheduler/JobResult.scala | 10 +++--- .../spark/scheduler/SparkListener.scala | 34 +++++++++---------- .../apache/spark/scheduler/SplitInfo.scala | 4 +-- .../apache/spark/scheduler/StageInfo.scala | 6 ++-- .../org/apache/spark/scheduler/TaskInfo.scala | 6 ++-- .../apache/spark/scheduler/TaskLocality.scala | 4 +-- .../spark/serializer/JavaSerializer.scala | 6 ++-- .../apache/spark/serializer/Serializer.scala | 18 +++++----- .../apache/spark/storage/StorageUtils.scala | 4 +-- .../org/apache/spark/util/MutablePair.scala | 6 ++-- .../spark/util/collection/AppendOnlyMap.scala | 6 ++-- .../collection/ExternalAppendOnlyMap.scala | 6 ++-- .../spark/util/collection/OpenHashMap.scala | 6 ++-- .../spark/util/random/Pseudorandom.scala | 6 ++-- .../spark/util/random/RandomSampler.scala | 14 ++++---- docs/js/api-docs.js | 2 +- 37 files changed, 170 insertions(+), 170 deletions(-) rename core/src/main/scala/org/apache/spark/annotations/{DeveloperAPI.java => DeveloperApi.java} (96%) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index 5bc9b880f4e94..06d957a1b2cb8 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -17,18 +17,18 @@ package org.apache.spark -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.util.collection.{AppendOnlyMap, ExternalAppendOnlyMap} /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * A set of functions used to aggregate data. * * @param createCombiner function to create the initial value of the aggregation. * @param mergeValue function to merge a new value into the aggregation result. * @param mergeCombiners function to merge outputs from multiple mergeValue function. */ -@DeveloperAPI +@DeveloperApi case class Aggregator[K, V, C] ( createCombiner: V => C, mergeValue: (C, V) => C, diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 73d1c40d523e6..13f3dcd84d721 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -17,24 +17,24 @@ package org.apache.spark -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Base class for dependencies. */ -@DeveloperAPI +@DeveloperApi abstract class Dependency[T](val rdd: RDD[T]) extends Serializable /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Base class for dependencies where each partition of the parent RDD is used by at most one * partition of the child RDD. Narrow dependencies allow for pipelined execution. */ -@DeveloperAPI +@DeveloperApi abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { /** * Get the parent partitions for a child partition. @@ -46,7 +46,7 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Represents a dependency on the output of a shuffle stage. * @param rdd the parent RDD * @param partitioner partitioner used to partition the shuffle output @@ -54,7 +54,7 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { * the default serializer, as specified by `spark.serializer` config option, will * be used. */ -@DeveloperAPI +@DeveloperApi class ShuffleDependency[K, V]( @transient rdd: RDD[_ <: Product2[K, V]], val partitioner: Partitioner, @@ -66,24 +66,24 @@ class ShuffleDependency[K, V]( /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Represents a one-to-one dependency between partitions of the parent and child RDDs. */ -@DeveloperAPI +@DeveloperApi class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) { override def getParents(partitionId: Int) = List(partitionId) } /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Represents a one-to-one dependency between ranges of partitions in the parent and child RDDs. * @param rdd the parent RDD * @param inStart the start of the range in the parent RDD * @param outStart the start of the range in the child RDD * @param length the length of the range */ -@DeveloperAPI +@DeveloperApi class RangeDependency[T](rdd: RDD[T], inStart: Int, outStart: Int, length: Int) extends NarrowDependency[T](rdd) { diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index 2c6b62dcc09e0..ed33c7d2be88b 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -21,10 +21,10 @@ import org.apache.log4j.{LogManager, PropertyConfigurator} import org.slf4j.{Logger, LoggerFactory} import org.slf4j.impl.StaticLoggerBinder -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Utility trait for classes that want to log data. Creates a SLF4J logger for the class and allows * logging messages at different levels using methods that only evaluate parameters lazily if the * log level is enabled. @@ -32,7 +32,7 @@ import org.apache.spark.annotations.DeveloperAPI * NOTE: DO NOT USE this class outside of Spark. It is intended as an internal utility. * This will likely be changed or removed in future releases. */ -@DeveloperAPI +@DeveloperApi trait Logging { // Make the log field transient so that objects with Logging can // be serialized and used on another machine diff --git a/core/src/main/scala/org/apache/spark/SerializableWritable.scala b/core/src/main/scala/org/apache/spark/SerializableWritable.scala index 611be5ddf2e70..9c44340cacadb 100644 --- a/core/src/main/scala/org/apache/spark/SerializableWritable.scala +++ b/core/src/main/scala/org/apache/spark/SerializableWritable.scala @@ -23,9 +23,9 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.ObjectWritable import org.apache.hadoop.io.Writable -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi -@DeveloperAPI +@DeveloperApi class SerializableWritable[T <: Writable](@transient var t: T) extends Serializable { def value = t override def toString = t.toString diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 46f1980ffe942..6464b3bc4572f 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -35,7 +35,7 @@ import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHad import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} import org.apache.mesos.MesosNativeLibrary -import org.apache.spark.annotations.{DeveloperAPI, Experimental} +import org.apache.spark.annotations.{DeveloperApi, Experimental} import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} @@ -49,7 +49,7 @@ import org.apache.spark.ui.SparkUI import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedHashMap, Utils} /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark * cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster. * @@ -57,7 +57,7 @@ import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerTy * this config overrides the default configs as well as system properties. */ -@DeveloperAPI +@DeveloperApi class SparkContext(config: SparkConf) extends Logging { // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, @@ -66,14 +66,14 @@ class SparkContext(config: SparkConf) extends Logging { private[spark] var preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map() /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Alternative constructor for setting preferred locations where Spark will create executors. * * @param preferredNodeLocationData used in YARN mode to select nodes to launch containers on. Ca * be generated using [[org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations]] * from a list of input files or InputFormats for the application. */ - @DeveloperAPI + @DeveloperApi def this(config: SparkConf, preferredNodeLocationData: Map[String, Set[SplitInfo]]) = { this(config) this.preferredNodeLocationData = preferredNodeLocationData @@ -718,10 +718,10 @@ class SparkContext(config: SparkConf) extends Logging { } /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Register a listener to receive up-calls from events that happen during execution. */ - @DeveloperAPI + @DeveloperApi def addSparkListener(listener: SparkListener) { listenerBus.addListener(listener) } @@ -1031,10 +1031,10 @@ class SparkContext(config: SparkConf) extends Logging { } /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Run a job that can return approximate results. */ - @DeveloperAPI + @DeveloperApi def runApproximateJob[T, U, R]( rdd: RDD[T], func: (TaskContext, Iterator[T]) => U, diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 5f63b6b5bd4f0..e4e1f5e6378da 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -25,7 +25,7 @@ import scala.util.Properties import akka.actor._ import com.google.common.collect.MapMaker -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.api.python.PythonWorkerFactory import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.metrics.MetricsSystem @@ -36,14 +36,14 @@ import org.apache.spark.storage._ import org.apache.spark.util.{AkkaUtils, Utils} /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Holds all the runtime environment objects for a running Spark instance (either master or worker), * including the serializer, Akka actor system, block manager, map output tracker, etc. Currently * Spark code finds the SparkEnv through a thread-local variable, so each thread that accesses these * objects needs to have the right SparkEnv set. You can get the current environment with * SparkEnv.get (e.g. after creating a SparkContext) and set it with SparkEnv.set. */ -@DeveloperAPI +@DeveloperApi class SparkEnv ( val executorId: String, val actorSystem: ActorSystem, diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 58e39abd6e0bd..ec42f75598696 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -19,14 +19,14 @@ package org.apache.spark import scala.collection.mutable.ArrayBuffer -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.executor.TaskMetrics /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Contextual information about a task which can be read or mutated during execution. */ -@DeveloperAPI +@DeveloperApi class TaskContext( val stageId: Int, val partitionId: Int, diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index cd73508711a1e..01312a44714a4 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -17,26 +17,26 @@ package org.apache.spark -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Various possible reasons why a task ended. The low-level TaskScheduler is supposed to retry * tasks several times for "ephemeral" failures, and only report back failures that require some * old stages to be resubmitted, such as shuffle map fetch failures. */ -@DeveloperAPI +@DeveloperApi sealed trait TaskEndReason -@DeveloperAPI +@DeveloperApi case object Success extends TaskEndReason -@DeveloperAPI +@DeveloperApi case object Resubmitted extends TaskEndReason // Task was finished earlier but we've now lost it -@DeveloperAPI +@DeveloperApi case class FetchFailed( bmAddress: BlockManagerId, shuffleId: Int, @@ -44,7 +44,7 @@ case class FetchFailed( reduceId: Int) extends TaskEndReason -@DeveloperAPI +@DeveloperApi case class ExceptionFailure( className: String, description: String, @@ -53,28 +53,28 @@ case class ExceptionFailure( extends TaskEndReason /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * The task finished successfully, but the result was lost from the executor's block manager before * it was fetched. */ -@DeveloperAPI +@DeveloperApi case object TaskResultLost extends TaskEndReason -@DeveloperAPI +@DeveloperApi case object TaskKilled extends TaskEndReason /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * The task failed because the executor that it was running on was lost. This may happen because * the task crashed the JVM. */ -@DeveloperAPI +@DeveloperApi case object ExecutorLostFailure extends TaskEndReason /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * We don't know why the task ended -- for example, because of a ClassNotFound exception when * deserializing the task result. */ -@DeveloperAPI +@DeveloperApi case object UnknownReason extends TaskEndReason diff --git a/core/src/main/scala/org/apache/spark/annotations/DeveloperAPI.java b/core/src/main/scala/org/apache/spark/annotations/DeveloperApi.java similarity index 96% rename from core/src/main/scala/org/apache/spark/annotations/DeveloperAPI.java rename to core/src/main/scala/org/apache/spark/annotations/DeveloperApi.java index ae04a4a63b9fd..74c63ea949b3e 100644 --- a/core/src/main/scala/org/apache/spark/annotations/DeveloperAPI.java +++ b/core/src/main/scala/org/apache/spark/annotations/DeveloperApi.java @@ -22,4 +22,4 @@ @Retention(RetentionPolicy.SOURCE) @Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) -public @interface DeveloperAPI {} +public @interface DeveloperApi {} diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala index 67faa217c46bf..fd44abc37e16a 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala @@ -19,16 +19,16 @@ package org.apache.spark.broadcast import org.apache.spark.SecurityManager import org.apache.spark.SparkConf -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * An interface for all the broadcast implementations in Spark (to allow * multiple broadcast implementations). SparkContext uses a user-specified * BroadcastFactory implementation to instantiate a particular broadcast for the * entire Spark job. */ -@DeveloperAPI +@DeveloperApi trait BroadcastFactory { def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager): Unit def newBroadcast[T](value: T, isLocal: Boolean, id: Long): Broadcast[T] diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index ed1b45f5d485b..cbddcb3c34f7d 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,14 +17,14 @@ package org.apache.spark.executor -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.storage.{BlockId, BlockStatus} /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Metrics tracked during the execution of a task. */ -@DeveloperAPI +@DeveloperApi class TaskMetrics extends Serializable { /** * Host's name the task runs on @@ -89,10 +89,10 @@ private[spark] object TaskMetrics { /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Metrics pertaining to shuffle data read in a given task. */ -@DeveloperAPI +@DeveloperApi class ShuffleReadMetrics extends Serializable { /** * Absolute time when this task finished reading shuffle data @@ -128,10 +128,10 @@ class ShuffleReadMetrics extends Serializable { } /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Metrics pertaining to shuffle data written in a given task. */ -@DeveloperAPI +@DeveloperApi class ShuffleWriteMetrics extends Serializable { /** * Number of bytes written for the shuffle by this task diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index 1f0134d0cdf3e..a0df43bafc30f 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -23,10 +23,10 @@ import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} import org.apache.spark.SparkConf -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * CompressionCodec allows the customization of choosing different compression implementations * to be used in block storage. * @@ -34,7 +34,7 @@ import org.apache.spark.annotations.DeveloperAPI * This is intended for use as an internal compression utility within a single * Spark application. */ -@DeveloperAPI +@DeveloperApi trait CompressionCodec { def compressedOutputStream(s: OutputStream): OutputStream @@ -59,14 +59,14 @@ private[spark] object CompressionCodec { /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * LZF implementation of [[org.apache.spark.io.CompressionCodec]]. * * Note: The wire protocol for this codec is not guaranteed to be compatible across versions * of Spark. This is intended for use as an internal compression utility within a single Spark * application. */ -@DeveloperAPI +@DeveloperApi class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { override def compressedOutputStream(s: OutputStream): OutputStream = { @@ -78,7 +78,7 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Snappy implementation of [[org.apache.spark.io.CompressionCodec]]. * Block size can be configured by spark.io.compression.snappy.block.size. * @@ -86,7 +86,7 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { * of Spark. This is intended for use as an internal compression utility within a single Spark * application. */ -@DeveloperAPI +@DeveloperApi class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec { override def compressedOutputStream(s: OutputStream): OutputStream = { diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index f8c3cda91781c..9f0ce8fe919c7 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -23,7 +23,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.{InterruptibleIterator, Partition, Partitioner, SparkEnv, TaskContext} import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency} -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.util.collection.{ExternalAppendOnlyMap, AppendOnlyMap} import org.apache.spark.serializer.Serializer @@ -52,7 +52,7 @@ private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep] } /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * A RDD that cogroups its parents. For each key k in parent RDDs, the resulting RDD contains a * tuple with the list of values for that key. * @@ -62,7 +62,7 @@ private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep] * @param rdds parent RDDs. * @param part partitioner used to partition the shuffle output */ -@DeveloperAPI +@DeveloperApi class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: Partitioner) extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) { diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index ea03b3b8e4861..1fad40bcf8242 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -35,7 +35,7 @@ import org.apache.hadoop.mapred.TaskID import org.apache.hadoop.util.ReflectionUtils import org.apache.spark._ -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.NextIterator @@ -71,7 +71,7 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp } /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * An RDD that provides core functionality for reading data stored in Hadoop (e.g., files in HDFS, * sources in HBase, or S3), using the older MapReduce API (`org.apache.hadoop.mapred`). * @@ -89,7 +89,7 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp * @param valueClass Class of the value associated with the inputFormatClass. * @param minSplits Minimum number of Hadoop Splits (HadoopRDD partitions) to generate. */ -@DeveloperAPI +@DeveloperApi class HadoopRDD[K, V]( sc: SparkContext, broadcastedConf: Broadcast[SerializableWritable[Configuration]], diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 510f0cfbd8be2..bdfe9a41041e6 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ import org.apache.spark.{InterruptibleIterator, Logging, Partition, SerializableWritable, SparkContext, TaskContext} -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi private[spark] class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputSplit with Writable) @@ -37,7 +37,7 @@ class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputS } /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * An RDD that provides core functionality for reading data stored in Hadoop (e.g., files in HDFS, * sources in HBase, or S3), using the new MapReduce API (`org.apache.hadoop.mapreduce`). * @@ -50,7 +50,7 @@ class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputS * @param valueClass Class of the value associated with the inputFormatClass. * @param conf The Hadoop configuration. */ -@DeveloperAPI +@DeveloperApi class NewHadoopRDD[K, V]( sc : SparkContext, inputFormatClass: Class[_ <: InputFormat[K, V]], diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala index 160236f943715..5a42ba6a9d05a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala @@ -20,7 +20,7 @@ package org.apache.spark.rdd import scala.reflect.ClassTag import org.apache.spark.{NarrowDependency, Partition, TaskContext} -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi private[spark] class PartitionPruningRDDPartition(idx: Int, val parentSplit: Partition) extends Partition { @@ -47,13 +47,13 @@ private[spark] class PruneDependency[T](rdd: RDD[T], @transient partitionFilterF /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * A RDD used to prune RDD partitions/partitions so we can avoid launching tasks on * all partitions. An example use case: If we know the RDD is partitioned by range, * and the execution DAG has a filter on the key, we can avoid launching tasks * on partitions that don't have the range covering the key. */ -@DeveloperAPI +@DeveloperApi class PartitionPruningRDD[T: ClassTag]( @transient prev: RDD[T], @transient partitionFilterFunc: Int => Boolean) @@ -67,7 +67,7 @@ class PartitionPruningRDD[T: ClassTag]( } -@DeveloperAPI +@DeveloperApi object PartitionPruningRDD { /** diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index c56b67ef9b643..e9af8b358acaa 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -35,7 +35,7 @@ import org.apache.hadoop.mapred.TextOutputFormat import org.apache.spark._ import org.apache.spark.Partitioner._ import org.apache.spark.SparkContext._ -import org.apache.spark.annotations.{DeveloperAPI, Experimental} +import org.apache.spark.annotations.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaRDD import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.CountEvaluator @@ -88,33 +88,33 @@ abstract class RDD[T: ClassTag]( // ======================================================================= /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Implemented by subclasses to compute a given partition. */ - @DeveloperAPI + @DeveloperApi def compute(split: Partition, context: TaskContext): Iterator[T] /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Implemented by subclasses to return the set of partitions in this RDD. This method will only * be called once, so it is safe to implement a time-consuming computation in it. */ - @DeveloperAPI + @DeveloperApi protected def getPartitions: Array[Partition] /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Implemented by subclasses to return how this RDD depends on parent RDDs. This method will only * be called once, so it is safe to implement a time-consuming computation in it. */ - @DeveloperAPI + @DeveloperApi protected def getDependencies: Seq[Dependency[_]] = deps /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Optionally overridden by subclasses to specify placement preferences. */ - @DeveloperAPI + @DeveloperApi protected def getPreferredLocations(split: Partition): Seq[String] = Nil /** Optionally overridden by subclasses to specify how they are partitioned. */ @@ -526,11 +526,11 @@ abstract class RDD[T: ClassTag]( } /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Return a new RDD by applying a function to each partition of this RDD. This is a variant of * mapPartitions that also passes the TaskContext into the closure. */ - @DeveloperAPI + @DeveloperApi def mapPartitionsWithContext[U: ClassTag]( f: (TaskContext, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = { diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala index e9971cdabdba6..3661fe37f6f53 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -20,7 +20,7 @@ package org.apache.spark.rdd import scala.reflect.ClassTag import org.apache.spark.{Dependency, Partition, Partitioner, ShuffleDependency, SparkEnv, TaskContext} -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.serializer.Serializer private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { @@ -29,14 +29,14 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { } /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * The resulting RDD from a shuffle (e.g. repartitioning of data). * @param prev the parent RDD. * @param part the partitioner used to partition the RDD * @tparam K the key class. * @tparam V the value class. */ -@DeveloperAPI +@DeveloperApi class ShuffledRDD[K, V, P <: Product2[K, V] : ClassTag]( @transient var prev: RDD[P], part: Partitioner) diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala index f72400a7e8b35..0313a12867374 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -23,7 +23,7 @@ import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag import org.apache.spark.{Dependency, Partition, RangeDependency, SparkContext, TaskContext} -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi private[spark] class UnionPartition[T: ClassTag](idx: Int, rdd: RDD[T], splitIndex: Int) extends Partition { @@ -44,7 +44,7 @@ private[spark] class UnionPartition[T: ClassTag](idx: Int, rdd: RDD[T], splitInd } } -@DeveloperAPI +@DeveloperApi class UnionRDD[T: ClassTag]( sc: SparkContext, @transient var rdds: Seq[RDD[T]]) diff --git a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala index 6e02ea4ac7d7d..3fd2c4ff3b570 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala @@ -27,14 +27,14 @@ import org.apache.hadoop.mapreduce.Job import org.apache.hadoop.util.ReflectionUtils import org.apache.spark.Logging -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.deploy.SparkHadoopUtil /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Parses and holds information about inputFormat (and files) specified as a parameter. */ -@DeveloperAPI +@DeveloperApi class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Class[_], val path: String) extends Logging { diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index fd458734ab09e..30d898c991548 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -25,11 +25,11 @@ import java.util.concurrent.LinkedBlockingQueue import scala.collection.mutable.HashMap import org.apache.spark._ -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.executor.TaskMetrics /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * A logger class to record runtime information for jobs in Spark. This class outputs one log file * for each Spark job, containing tasks start/stop and shuffle information. JobLogger is a subclass * of SparkListener, use addSparkListener to add JobLogger to a SparkContext after the SparkContext @@ -40,7 +40,7 @@ import org.apache.spark.executor.TaskMetrics * to log application information as SparkListenerEvents. To enable this functionality, set * spark.eventLog.enabled to true. */ -@DeveloperAPI +@DeveloperApi @deprecated("Log application information by setting spark.eventLog.enabled.", "1.0.0") class JobLogger(val user: String, val logDirName: String) extends SparkListener with Logging { diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala index d8c443f6130ae..dc41effd59950 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala @@ -17,18 +17,18 @@ package org.apache.spark.scheduler -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * A result of a job in the DAGScheduler. */ -@DeveloperAPI +@DeveloperApi sealed trait JobResult -@DeveloperAPI +@DeveloperApi case object JobSucceeded extends JobResult // A failed stage ID of -1 means there is not a particular stage that caused the failure -@DeveloperAPI +@DeveloperApi case class JobFailed(exception: Exception, failedStageId: Int) extends JobResult diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 72ff68f97e0ab..50de7c2a9df8e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -23,28 +23,28 @@ import scala.collection.Map import scala.collection.mutable import org.apache.spark.{Logging, TaskEndReason} -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.{Distribution, Utils} -@DeveloperAPI +@DeveloperApi sealed trait SparkListenerEvent -@DeveloperAPI +@DeveloperApi case class SparkListenerStageSubmitted(stageInfo: StageInfo, properties: Properties = null) extends SparkListenerEvent -@DeveloperAPI +@DeveloperApi case class SparkListenerStageCompleted(stageInfo: StageInfo) extends SparkListenerEvent -@DeveloperAPI +@DeveloperApi case class SparkListenerTaskStart(stageId: Int, taskInfo: TaskInfo) extends SparkListenerEvent -@DeveloperAPI +@DeveloperApi case class SparkListenerTaskGettingResult(taskInfo: TaskInfo) extends SparkListenerEvent -@DeveloperAPI +@DeveloperApi case class SparkListenerTaskEnd( stageId: Int, taskType: String, @@ -53,26 +53,26 @@ case class SparkListenerTaskEnd( taskMetrics: TaskMetrics) extends SparkListenerEvent -@DeveloperAPI +@DeveloperApi case class SparkListenerJobStart(jobId: Int, stageIds: Seq[Int], properties: Properties = null) extends SparkListenerEvent -@DeveloperAPI +@DeveloperApi case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkListenerEvent -@DeveloperAPI +@DeveloperApi case class SparkListenerEnvironmentUpdate(environmentDetails: Map[String, Seq[(String, String)]]) extends SparkListenerEvent -@DeveloperAPI +@DeveloperApi case class SparkListenerBlockManagerAdded(blockManagerId: BlockManagerId, maxMem: Long) extends SparkListenerEvent -@DeveloperAPI +@DeveloperApi case class SparkListenerBlockManagerRemoved(blockManagerId: BlockManagerId) extends SparkListenerEvent -@DeveloperAPI +@DeveloperApi case class SparkListenerUnpersistRDD(rddId: Int) extends SparkListenerEvent /** An event used in the listener to shutdown the listener daemon thread. */ @@ -80,11 +80,11 @@ private[spark] case object SparkListenerShutdown extends SparkListenerEvent /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Interface for listening to events from the Spark scheduler. Note that this is an internal * interface which might change in different Spark releases. */ -@DeveloperAPI +@DeveloperApi trait SparkListener { /** * Called when a stage is completed, with information on the completed stage @@ -144,10 +144,10 @@ trait SparkListener { } /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Simple SparkListener that logs a few summary statistics when each stage completes */ -@DeveloperAPI +@DeveloperApi class StatsReportListener extends SparkListener with Logging { import org.apache.spark.scheduler.StatsReportListener._ diff --git a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala index 1c58b5103532c..e245ff8c25e0b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala @@ -19,11 +19,11 @@ package org.apache.spark.scheduler import collection.mutable.ArrayBuffer -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi // information about a specific split instance : handles both split instances. // So that we do not need to worry about the differences. -@DeveloperAPI +@DeveloperApi class SplitInfo(val inputFormatClazz: Class[_], val hostLocation: String, val path: String, val length: Long, val underlyingSplit: Any) { override def toString(): String = { diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 460f4109d536b..e2e287666834b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -17,14 +17,14 @@ package org.apache.spark.scheduler -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.storage.RDDInfo /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Stores information about a stage to pass from the scheduler to SparkListeners. */ -@DeveloperAPI +@DeveloperApi class StageInfo(val stageId: Int, val name: String, val numTasks: Int, val rddInfo: RDDInfo) { /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala index 04f891b6e72ed..a37ab5a338e4f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala @@ -17,13 +17,13 @@ package org.apache.spark.scheduler -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * Information about a running task attempt inside a TaskSet. */ -@DeveloperAPI +@DeveloperApi class TaskInfo( val taskId: Long, val index: Int, diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala index dfe447a294d39..eaf736fb20ea1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala @@ -17,9 +17,9 @@ package org.apache.spark.scheduler -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi -@DeveloperAPI +@DeveloperApi object TaskLocality extends Enumeration { // Process local is expected to be used ONLY within TaskSetManager for now. val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index 9880bd916a9bc..6c6463b1c8827 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -21,7 +21,7 @@ import java.io._ import java.nio.ByteBuffer import org.apache.spark.SparkConf -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.util.ByteBufferInputStream private[spark] class JavaSerializationStream(out: OutputStream, counterReset: Int) @@ -95,14 +95,14 @@ private[spark] class JavaSerializerInstance(counterReset: Int) extends Serialize } /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * A Spark serializer that uses Java's built-in serialization. * * Note that this serializer is not guaranteed to be wire-compatible across different versions of * Spark. It is intended to be used to serialize/de-serialize data within a single * Spark application. */ -@DeveloperAPI +@DeveloperApi class JavaSerializer(conf: SparkConf) extends Serializer with Externalizable { private var counterReset = conf.getInt("spark.serializer.objectStreamReset", 10000) diff --git a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala index cd19e45132ad4..d404035646065 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -23,11 +23,11 @@ import java.nio.ByteBuffer import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream import org.apache.spark.SparkEnv -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.util.{ByteBufferInputStream, NextIterator} /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * A serializer. Because some serialization libraries are not thread safe, this class is used to * create [[org.apache.spark.serializer.SerializerInstance]] objects that do the actual * serialization and are guaranteed to only be called from one thread at a time. @@ -42,7 +42,7 @@ import org.apache.spark.util.{ByteBufferInputStream, NextIterator} * Note that serializers are not required to be wire-compatible across different versions of Spark. * They are intended to be used to serialize/de-serialize data within a single Spark application. */ -@DeveloperAPI +@DeveloperApi trait Serializer { def newInstance(): SerializerInstance } @@ -56,10 +56,10 @@ object Serializer { /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * An instance of a serializer, for use by one thread at a time. */ -@DeveloperAPI +@DeveloperApi trait SerializerInstance { def serialize[T](t: T): ByteBuffer @@ -90,10 +90,10 @@ trait SerializerInstance { /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * A stream for writing serialized objects. */ -@DeveloperAPI +@DeveloperApi trait SerializationStream { def writeObject[T](t: T): SerializationStream def flush(): Unit @@ -109,10 +109,10 @@ trait SerializationStream { /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * A stream for reading serialized objects. */ -@DeveloperAPI +@DeveloperApi trait DeserializationStream { def readObject[T](): T def close(): Unit diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 182c4127a1952..d2df92ea7655d 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -21,7 +21,7 @@ import scala.collection.Map import scala.collection.mutable import org.apache.spark.SparkContext -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.util.Utils private[spark] @@ -49,7 +49,7 @@ class StorageStatus( } -@DeveloperAPI +@DeveloperApi class RDDInfo(val id: Int, val name: String, val numPartitions: Int, val storageLevel: StorageLevel) extends Ordered[RDDInfo] { diff --git a/core/src/main/scala/org/apache/spark/util/MutablePair.scala b/core/src/main/scala/org/apache/spark/util/MutablePair.scala index 2b62e9e4e25e8..a68859a1f7669 100644 --- a/core/src/main/scala/org/apache/spark/util/MutablePair.scala +++ b/core/src/main/scala/org/apache/spark/util/MutablePair.scala @@ -17,17 +17,17 @@ package org.apache.spark.util -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * A tuple of 2 elements. This can be used as an alternative to Scala's Tuple2 when we want to * minimize object allocation. * * @param _1 Element 1 of this MutablePair * @param _2 Element 2 of this MutablePair */ -@DeveloperAPI +@DeveloperApi case class MutablePair[@specialized(Int, Long, Double, Char, Boolean/* , AnyRef */) T1, @specialized(Int, Long, Double, Char, Boolean/* , AnyRef */) T2] (var _1: T1, var _2: T2) diff --git a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala index 35d99f7e52d79..f1fbfdd51896d 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala @@ -19,10 +19,10 @@ package org.apache.spark.util.collection import java.util.{Arrays, Comparator} -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * A simple open hash table optimized for the append-only use case, where keys * are never removed, but the value for each key may be changed. * @@ -32,7 +32,7 @@ import org.apache.spark.annotations.DeveloperAPI * * TODO: Cache the hash values of each key? java.util.HashMap does that. */ -@DeveloperAPI +@DeveloperApi class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] with Serializable { require(initialCapacity <= (1 << 29), "Can't make capacity bigger than 2^29 elements") diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 3e397f53aaaf3..abd3a6d61731b 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -27,12 +27,12 @@ import com.google.common.io.ByteStreams import it.unimi.dsi.fastutil.io.FastBufferedInputStream import org.apache.spark.{Logging, SparkEnv} -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{BlockId, BlockManager} /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * An append-only map that spills sorted content to disk when there is insufficient space for it * to grow. * @@ -57,7 +57,7 @@ import org.apache.spark.storage.{BlockId, BlockManager} * `spark.shuffle.safetyFraction` specifies an additional margin of safety as a fraction of * this threshold, in case map size estimation is not sufficiently accurate. */ -@DeveloperAPI +@DeveloperApi class ExternalAppendOnlyMap[K, V, C]( createCombiner: V => C, mergeValue: (C, V) => C, diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala index 418b6169c16ca..73011361b0676 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala @@ -19,17 +19,17 @@ package org.apache.spark.util.collection import scala.reflect.ClassTag -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * A fast hash map implementation for nullable keys. This hash map supports insertions and updates, * but not deletions. This map is about 5X faster than java.util.HashMap, while using much less * space overhead. * * Under the hood, it uses our OpenHashSet implementation. */ -@DeveloperAPI +@DeveloperApi class OpenHashMap[K >: Null : ClassTag, @specialized(Long, Int, Double) V: ClassTag]( initialCapacity: Int) extends Iterable[(K, V)] diff --git a/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala b/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala index 77862518bd678..745c38ef72143 100644 --- a/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala +++ b/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala @@ -17,13 +17,13 @@ package org.apache.spark.util.random -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * A class with pseudorandom behavior. */ -@DeveloperAPI +@DeveloperApi trait Pseudorandom { /** Set random seed. */ def setSeed(seed: Long) diff --git a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala index cbc15873f8cce..5306b89fddc86 100644 --- a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala +++ b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala @@ -22,10 +22,10 @@ import java.util.Random import cern.jet.random.Poisson import cern.jet.random.engine.DRand -import org.apache.spark.annotations.DeveloperAPI +import org.apache.spark.annotations.DeveloperApi /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * A pseudorandom sampler. It is possible to change the sampled item type. For example, we might * want to add weights for stratified sampling or importance sampling. Should only use * transformations that are tied to the sampler and cannot be applied after sampling. @@ -33,7 +33,7 @@ import org.apache.spark.annotations.DeveloperAPI * @tparam T item type * @tparam U sampled item type */ -@DeveloperAPI +@DeveloperApi trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable { /** take a random sample */ @@ -44,7 +44,7 @@ trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable } /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * A sampler based on Bernoulli trials. * * @param lb lower bound of the acceptance range @@ -52,7 +52,7 @@ trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable * @param complement whether to use the complement of the range specified, default to false * @tparam T item type */ -@DeveloperAPI +@DeveloperApi class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) (implicit random: Random = new XORShiftRandom) extends RandomSampler[T, T] { @@ -73,13 +73,13 @@ class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) } /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * A sampler based on values drawn from Poisson distribution. * * @param poisson a Poisson random number generator * @tparam T item type */ -@DeveloperAPI +@DeveloperApi class PoissonSampler[T](mean: Double) (implicit var poisson: Poisson = new Poisson(mean, new DRand)) extends RandomSampler[T, T] { diff --git a/docs/js/api-docs.js b/docs/js/api-docs.js index 1d234658e7ee9..2c3ca434ec46c 100644 --- a/docs/js/api-docs.js +++ b/docs/js/api-docs.js @@ -3,7 +3,7 @@ $(document).ready(function() { var annotations = $("dt:contains('Annotations')").next("dd").children("span.name"); addBadges(annotations, "AlphaComponent", ":: AlphaComponent ::", "ALPHA COMPONENT"); - addBadges(annotations, "DeveloperAPI", ":: DeveloperAPI ::", "Developer API"); + addBadges(annotations, "DeveloperApi", ":: DeveloperApi ::", "Developer API"); addBadges(annotations, "Experimental", ":: Experimental ::", "Experimental"); }); From 982a47333fddde4d0865369aa123e71cad910ebd Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 8 Apr 2014 17:50:34 -0700 Subject: [PATCH 27/31] Generalize jQuery matching for non Spark-core API docs --- docs/css/api-docs.css | 4 ++++ docs/js/api-docs.js | 15 +++++++-------- 2 files changed, 11 insertions(+), 8 deletions(-) diff --git a/docs/css/api-docs.css b/docs/css/api-docs.css index cc5f37bbdb42e..b2d1d7f869790 100644 --- a/docs/css/api-docs.css +++ b/docs/css/api-docs.css @@ -8,6 +8,10 @@ background-color: #257080; } +.alphaComponent { + background-color: #bb0000; +} + .badge { font-family: Arial, san-serif; float: right; diff --git a/docs/js/api-docs.js b/docs/js/api-docs.js index 2c3ca434ec46c..1414b6d0b81a1 100644 --- a/docs/js/api-docs.js +++ b/docs/js/api-docs.js @@ -2,16 +2,14 @@ $(document).ready(function() { var annotations = $("dt:contains('Annotations')").next("dd").children("span.name"); - addBadges(annotations, "AlphaComponent", ":: AlphaComponent ::", "ALPHA COMPONENT"); + addBadges(annotations, "AlphaComponent", ":: AlphaComponent ::", "Alpha Component"); addBadges(annotations, "DeveloperApi", ":: DeveloperApi ::", "Developer API"); addBadges(annotations, "Experimental", ":: Experimental ::", "Experimental"); }); function addBadges(allAnnotations, name, tag, html) { - var fullName = "org.apache.spark.annotations." + name; - var annotations = allAnnotations.children("a[name='" + fullName + "']"); - var tags = $("p.comment:contains(" + tag + ")").add( - $("div.comment p:contains(" + tag + ")")); + var annotations = allAnnotations.filter(":contains('" + name + "')") + var tags = $(".cmt:contains(" + tag + ")") // Remove identifier tags from comments tags.each(function(index) { @@ -21,7 +19,8 @@ function addBadges(allAnnotations, name, tag, html) { }); // Add badges to all containers - tags.prevAll("h4.signature").prepend(html); - annotations.closest("div.fullcomment").prevAll("h4.signature").prepend(html); - annotations.closest("div.fullcommenttop").prepend(html); + tags.prevAll("h4.signature") + .add(annotations.closest("div.fullcommenttop")) + .add(annotations.closest("div.fullcomment").prevAll("h4.signature")) + .prepend(html); } From 99b223a022f21d6a49b3da468e845a26ab08ad32 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 8 Apr 2014 17:45:18 -0700 Subject: [PATCH 28/31] Cleaning up annotations --- .../org/apache/spark/annotations/AlphaComponent.java | 5 +++-- .../org/apache/spark/annotations/DeveloperApi.java | 9 +++++++-- .../org/apache/spark/annotations/Experimental.java | 10 ++++++++-- 3 files changed, 18 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/annotations/AlphaComponent.java b/core/src/main/scala/org/apache/spark/annotations/AlphaComponent.java index d54767ab84c7e..61a4fb32ae6e6 100644 --- a/core/src/main/scala/org/apache/spark/annotations/AlphaComponent.java +++ b/core/src/main/scala/org/apache/spark/annotations/AlphaComponent.java @@ -19,7 +19,8 @@ import java.lang.annotation.*; -@Retention(RetentionPolicy.SOURCE) -@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, +/** A new component of Spark which may have unstable API's. */ +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) public @interface AlphaComponent {} diff --git a/core/src/main/scala/org/apache/spark/annotations/DeveloperApi.java b/core/src/main/scala/org/apache/spark/annotations/DeveloperApi.java index 74c63ea949b3e..808e8e4e2b153 100644 --- a/core/src/main/scala/org/apache/spark/annotations/DeveloperApi.java +++ b/core/src/main/scala/org/apache/spark/annotations/DeveloperApi.java @@ -19,7 +19,12 @@ import java.lang.annotation.*; -@Retention(RetentionPolicy.SOURCE) -@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, +/** + * A lower-level, unstable API intended for developers. + * + * Developer API's might change or be removed in minor versions of Spark. + */ +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) public @interface DeveloperApi {} diff --git a/core/src/main/scala/org/apache/spark/annotations/Experimental.java b/core/src/main/scala/org/apache/spark/annotations/Experimental.java index 58445cc2c1e86..643540f5330d8 100644 --- a/core/src/main/scala/org/apache/spark/annotations/Experimental.java +++ b/core/src/main/scala/org/apache/spark/annotations/Experimental.java @@ -19,7 +19,13 @@ import java.lang.annotation.*; -@Retention(RetentionPolicy.SOURCE) -@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, +/** + * An experimental user-facing API. + * + * Experimental API's might change or be removed in minor versions of Spark, or be adopted as + * first-class Spark API's. + */ +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) public @interface Experimental {} From 8d0c8733f0227dfa7592283955c05bcc562dd9cc Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 8 Apr 2014 18:12:36 -0700 Subject: [PATCH 29/31] Warning in SparkEnv --- core/src/main/scala/org/apache/spark/SparkEnv.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index e4e1f5e6378da..40a1d2d0a9614 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -42,6 +42,9 @@ import org.apache.spark.util.{AkkaUtils, Utils} * 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. + * + * NOTE: This is not intended for external use. This is exposed for Shark and may be made private + * in a future release. */ @DeveloperApi class SparkEnv ( From bfe7b52addf92adbfd269afe0f0c33986780bbbe Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 8 Apr 2014 20:09:06 -0700 Subject: [PATCH 30/31] Adding experimental for approximate counts --- .../apache/spark/api/java/JavaDoubleRDD.scala | 19 ++++++++++++++++--- .../apache/spark/api/java/JavaPairRDD.scala | 11 ++++++++--- .../apache/spark/api/java/JavaRDDLike.scala | 9 +++++++-- .../apache/spark/rdd/DoubleRDDFunctions.scala | 13 +++++++++++-- .../apache/spark/rdd/PairRDDFunctions.scala | 5 ++++- 5 files changed, 46 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala index f816bb43a5b44..ffb9425872c41 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala @@ -23,6 +23,7 @@ import scala.reflect.ClassTag import org.apache.spark.Partitioner import org.apache.spark.SparkContext.doubleRDDToDoubleRDDFunctions +import org.apache.spark.annotations.Experimental import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.RDD @@ -184,14 +185,26 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[JDouble, Ja def meanApprox(timeout: Long, confidence: JDouble): PartialResult[BoundedDouble] = srdd.meanApprox(timeout, confidence) - /** (Experimental) Approximate operation to return the mean within a timeout. */ + /** + * :: Experimental :: + * Approximate operation to return the mean within a timeout. + */ + @Experimental def meanApprox(timeout: Long): PartialResult[BoundedDouble] = srdd.meanApprox(timeout) - /** (Experimental) Approximate operation to return the sum within a timeout. */ + /** + * :: Experimental :: + * Approximate operation to return the sum within a timeout. + */ + @Experimental def sumApprox(timeout: Long, confidence: JDouble): PartialResult[BoundedDouble] = srdd.sumApprox(timeout, confidence) - /** (Experimental) Approximate operation to return the sum within a timeout. */ + /** + * :: Experimental :: + * Approximate operation to return the sum within a timeout. + */ + @Experimental def sumApprox(timeout: Long): PartialResult[BoundedDouble] = srdd.sumApprox(timeout) /** diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 9596dbaf75488..eb394acbba447 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -26,11 +26,12 @@ import com.google.common.base.Optional import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.{JobConf, OutputFormat} -import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat, Job} +import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} import org.apache.spark.{HashPartitioner, Partitioner} import org.apache.spark.Partitioner._ import org.apache.spark.SparkContext.rddToPairRDDFunctions +import org.apache.spark.annotations.Experimental import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2, PairFunction} import org.apache.spark.partial.{BoundedDouble, PartialResult} @@ -200,16 +201,20 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) def countByKey(): java.util.Map[K, Long] = mapAsJavaMap(rdd.countByKey()) /** - * (Experimental) Approximate version of countByKey that can return a partial result if it does + * :: Experimental :: + * Approximate version of countByKey that can return a partial result if it does * not finish within a timeout. */ + @Experimental def countByKeyApprox(timeout: Long): PartialResult[java.util.Map[K, BoundedDouble]] = rdd.countByKeyApprox(timeout).map(mapAsJavaMap) /** - * (Experimental) Approximate version of countByKey that can return a partial result if it does + * :: Experimental :: + * Approximate version of countByKey that can return a partial result if it does * not finish within a timeout. */ + @Experimental def countByKeyApprox(timeout: Long, confidence: Double = 0.95) : PartialResult[java.util.Map[K, BoundedDouble]] = rdd.countByKeyApprox(timeout, confidence).map(mapAsJavaMap) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index e03b8e78d5f52..e41356c2fc19d 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -26,6 +26,7 @@ import com.google.common.base.Optional import org.apache.hadoop.io.compress.CompressionCodec import org.apache.spark.{Partition, SparkContext, TaskContext} +import org.apache.spark.annotations.Experimental import org.apache.spark.api.java.JavaPairRDD._ import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2, _} @@ -331,16 +332,20 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def count(): Long = rdd.count() /** - * (Experimental) Approximate version of count() that returns a potentially incomplete result + * :: 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): PartialResult[BoundedDouble] = rdd.countApprox(timeout, confidence) /** - * (Experimental) Approximate version of count() that returns a potentially incomplete result + * :: 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): PartialResult[BoundedDouble] = rdd.countApprox(timeout) diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala index a7b6b3b5146ce..bb45473ace0ee 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -17,6 +17,7 @@ package org.apache.spark.rdd +import org.apache.spark.annotations.Experimental import org.apache.spark.{TaskContext, Logging} import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.MeanEvaluator @@ -63,14 +64,22 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { */ def sampleVariance(): Double = stats().sampleVariance - /** (Experimental) Approximate operation to return the mean within a timeout. */ + /** + * :: Experimental :: + * Approximate operation to return the mean within a timeout. + */ + @Experimental def meanApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { val processPartition = (ctx: TaskContext, ns: Iterator[Double]) => StatCounter(ns) val evaluator = new MeanEvaluator(self.partitions.size, confidence) self.context.runApproximateJob(self, processPartition, evaluator, timeout) } - /** (Experimental) Approximate operation to return the sum within a timeout. */ + /** + * :: Experimental :: + * Approximate operation to return the sum within a timeout. + */ + @Experimental def sumApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { val processPartition = (ctx: TaskContext, ns: Iterator[Double]) => StatCounter(ns) val evaluator = new SumEvaluator(self.partitions.size, confidence) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 14386ff5b9127..e816b213c802a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -39,6 +39,7 @@ RecordWriter => NewRecordWriter, SparkHadoopMapReduceUtil} import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat} import org.apache.spark._ +import org.apache.spark.annotations.Experimental import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.SparkHadoopWriter import org.apache.spark.Partitioner.defaultPartitioner @@ -201,9 +202,11 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) def countByKey(): Map[K, Long] = self.map(_._1).countByValue() /** - * (Experimental) Approximate version of countByKey that can return a partial result if it does + * :: Experimental :: + * Approximate version of countByKey that can return a partial result if it does * not finish within a timeout. */ + @Experimental def countByKeyApprox(timeout: Long, confidence: Double = 0.95) : PartialResult[Map[K, BoundedDouble]] = { self.map(_._1).countByValueApprox(timeout, confidence) From 042c8032ef0ee2f1bfbf55208f5ddb4a31630362 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 8 Apr 2014 22:36:55 -0700 Subject: [PATCH 31/31] spark.annotations -> spark.annotation --- core/src/main/scala/org/apache/spark/Aggregator.scala | 2 +- core/src/main/scala/org/apache/spark/Dependency.scala | 2 +- core/src/main/scala/org/apache/spark/FutureAction.scala | 2 +- core/src/main/scala/org/apache/spark/Logging.scala | 2 +- core/src/main/scala/org/apache/spark/SerializableWritable.scala | 2 +- core/src/main/scala/org/apache/spark/SparkContext.scala | 2 +- core/src/main/scala/org/apache/spark/SparkEnv.scala | 2 +- core/src/main/scala/org/apache/spark/TaskContext.scala | 2 +- core/src/main/scala/org/apache/spark/TaskEndReason.scala | 2 +- .../spark/{annotations => annotation}/AlphaComponent.java | 2 +- .../apache/spark/{annotations => annotation}/DeveloperApi.java | 2 +- .../apache/spark/{annotations => annotation}/Experimental.java | 2 +- .../main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala | 2 +- core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala | 2 +- core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala | 2 +- .../scala/org/apache/spark/broadcast/BroadcastFactory.scala | 2 +- core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala | 2 +- core/src/main/scala/org/apache/spark/io/CompressionCodec.scala | 2 +- .../src/main/scala/org/apache/spark/partial/BoundedDouble.scala | 2 +- .../src/main/scala/org/apache/spark/partial/PartialResult.scala | 2 +- core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala | 2 +- core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala | 2 +- .../main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala | 2 +- core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala | 2 +- core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala | 2 +- core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 2 +- .../main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala | 2 +- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 2 +- core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala | 2 +- core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala | 2 +- .../main/scala/org/apache/spark/scheduler/InputFormatInfo.scala | 2 +- core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala | 2 +- core/src/main/scala/org/apache/spark/scheduler/JobResult.scala | 2 +- .../main/scala/org/apache/spark/scheduler/SparkListener.scala | 2 +- core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala | 2 +- core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala | 2 +- core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala | 2 +- .../main/scala/org/apache/spark/scheduler/TaskLocality.scala | 2 +- .../main/scala/org/apache/spark/serializer/JavaSerializer.scala | 2 +- .../src/main/scala/org/apache/spark/serializer/Serializer.scala | 2 +- core/src/main/scala/org/apache/spark/storage/StorageUtils.scala | 2 +- core/src/main/scala/org/apache/spark/util/MutablePair.scala | 2 +- .../scala/org/apache/spark/util/collection/AppendOnlyMap.scala | 2 +- .../apache/spark/util/collection/ExternalAppendOnlyMap.scala | 2 +- .../scala/org/apache/spark/util/collection/OpenHashMap.scala | 2 +- .../main/scala/org/apache/spark/util/random/Pseudorandom.scala | 2 +- .../main/scala/org/apache/spark/util/random/RandomSampler.scala | 2 +- sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala | 2 +- sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala | 2 +- 49 files changed, 49 insertions(+), 49 deletions(-) rename core/src/main/scala/org/apache/spark/{annotations => annotation}/AlphaComponent.java (96%) rename core/src/main/scala/org/apache/spark/{annotations => annotation}/DeveloperApi.java (96%) rename core/src/main/scala/org/apache/spark/{annotations => annotation}/Experimental.java (97%) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index 06d957a1b2cb8..59fdf659c9e11 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -17,7 +17,7 @@ package org.apache.spark -import org.apache.spark.annotations.DeveloperApi +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.collection.{AppendOnlyMap, ExternalAppendOnlyMap} /** diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 13f3dcd84d721..b90223328c0f4 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -17,7 +17,7 @@ package org.apache.spark -import org.apache.spark.annotations.DeveloperApi +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index 07c499f49d2b1..62c3ff0056814 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -21,7 +21,7 @@ import scala.concurrent._ import scala.concurrent.duration.Duration import scala.util.Try -import org.apache.spark.annotations.Experimental +import org.apache.spark.annotation.Experimental import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.{JobFailed, JobSucceeded, JobWaiter} diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index ed33c7d2be88b..e5e15617acb10 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -21,7 +21,7 @@ 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.annotation.DeveloperApi /** * :: DeveloperApi :: diff --git a/core/src/main/scala/org/apache/spark/SerializableWritable.scala b/core/src/main/scala/org/apache/spark/SerializableWritable.scala index 9c44340cacadb..e50b9ac2291f9 100644 --- a/core/src/main/scala/org/apache/spark/SerializableWritable.scala +++ b/core/src/main/scala/org/apache/spark/SerializableWritable.scala @@ -23,7 +23,7 @@ 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.annotation.DeveloperApi @DeveloperApi class SerializableWritable[T <: Writable](@transient var t: T) extends Serializable { diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 6464b3bc4572f..ed30592728d71 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.annotation.{DeveloperApi, Experimental} import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 40a1d2d0a9614..78d57385ec988 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.annotation.DeveloperApi import org.apache.spark.api.python.PythonWorkerFactory import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.metrics.MetricsSystem diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index ec42f75598696..dc5a19ecd738e 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -19,7 +19,7 @@ package org.apache.spark import scala.collection.mutable.ArrayBuffer -import org.apache.spark.annotations.DeveloperApi +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics /** diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 01312a44714a4..a3074916d13e7 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -17,7 +17,7 @@ package org.apache.spark -import org.apache.spark.annotations.DeveloperApi +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId diff --git a/core/src/main/scala/org/apache/spark/annotations/AlphaComponent.java b/core/src/main/scala/org/apache/spark/annotation/AlphaComponent.java similarity index 96% rename from core/src/main/scala/org/apache/spark/annotations/AlphaComponent.java rename to core/src/main/scala/org/apache/spark/annotation/AlphaComponent.java index 61a4fb32ae6e6..af01fb7cfbd04 100644 --- a/core/src/main/scala/org/apache/spark/annotations/AlphaComponent.java +++ b/core/src/main/scala/org/apache/spark/annotation/AlphaComponent.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.annotations; +package org.apache.spark.annotation; import java.lang.annotation.*; diff --git a/core/src/main/scala/org/apache/spark/annotations/DeveloperApi.java b/core/src/main/scala/org/apache/spark/annotation/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/annotation/DeveloperApi.java index 808e8e4e2b153..5d546e7a63985 100644 --- a/core/src/main/scala/org/apache/spark/annotations/DeveloperApi.java +++ b/core/src/main/scala/org/apache/spark/annotation/DeveloperApi.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.annotations; +package org.apache.spark.annotation; import java.lang.annotation.*; diff --git a/core/src/main/scala/org/apache/spark/annotations/Experimental.java b/core/src/main/scala/org/apache/spark/annotation/Experimental.java similarity index 97% rename from core/src/main/scala/org/apache/spark/annotations/Experimental.java rename to core/src/main/scala/org/apache/spark/annotation/Experimental.java index 643540f5330d8..306b1418d8d0a 100644 --- a/core/src/main/scala/org/apache/spark/annotations/Experimental.java +++ b/core/src/main/scala/org/apache/spark/annotation/Experimental.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.annotations; +package org.apache.spark.annotation; import java.lang.annotation.*; diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala index ffb9425872c41..537f410b0ca26 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala @@ -23,7 +23,7 @@ import scala.reflect.ClassTag import org.apache.spark.Partitioner import org.apache.spark.SparkContext.doubleRDDToDoubleRDDFunctions -import org.apache.spark.annotations.Experimental +import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.RDD diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index eb394acbba447..e688ac2b39689 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -31,7 +31,7 @@ import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} import org.apache.spark.{HashPartitioner, Partitioner} import org.apache.spark.Partitioner._ import org.apache.spark.SparkContext.rddToPairRDDFunctions -import org.apache.spark.annotations.Experimental +import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2, PairFunction} import org.apache.spark.partial.{BoundedDouble, PartialResult} diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index e41356c2fc19d..13917dfcae4df 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -26,7 +26,7 @@ import com.google.common.base.Optional import org.apache.hadoop.io.compress.CompressionCodec import org.apache.spark.{Partition, SparkContext, TaskContext} -import org.apache.spark.annotations.Experimental +import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaPairRDD._ import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2, _} 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 fd44abc37e16a..f540ff1a6b2c2 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala @@ -19,7 +19,7 @@ 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.annotation.DeveloperApi /** * :: DeveloperApi :: 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 cbddcb3c34f7d..e4f02a4be0b97 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,7 +17,7 @@ package org.apache.spark.executor -import org.apache.spark.annotations.DeveloperApi +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.storage.{BlockId, BlockStatus} /** 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 a0df43bafc30f..e1a5ee316bb69 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -23,7 +23,7 @@ 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.annotation.DeveloperApi /** * :: DeveloperApi :: 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 ad11fccaf20a2..aed0353344427 100644 --- a/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala +++ b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala @@ -17,7 +17,7 @@ package org.apache.spark.partial -import org.apache.spark.annotations.Experimental +import org.apache.spark.annotation.Experimental /** * :: Experimental :: 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 6d5f4051c11f4..eade07fbcbe37 100644 --- a/core/src/main/scala/org/apache/spark/partial/PartialResult.scala +++ b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala @@ -17,7 +17,7 @@ package org.apache.spark.partial -import org.apache.spark.annotations.Experimental +import org.apache.spark.annotation.Experimental @Experimental class PartialResult[R](initialVal: R, isFinal: Boolean) { 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 85ea8f42b39ee..aed951a40b40c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala @@ -24,7 +24,7 @@ import scala.concurrent.ExecutionContext.Implicits.global import scala.reflect.ClassTag import org.apache.spark.{ComplexFutureAction, FutureAction, Logging} -import org.apache.spark.annotations.Experimental +import org.apache.spark.annotation.Experimental /** * :: Experimental :: 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 9f0ce8fe919c7..c6e79557f08a1 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.annotation.DeveloperApi import org.apache.spark.util.collection.{ExternalAppendOnlyMap, AppendOnlyMap} import org.apache.spark.serializer.Serializer diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala index bb45473ace0ee..2306c9736b334 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import org.apache.spark.annotations.Experimental +import org.apache.spark.annotation.Experimental import org.apache.spark.{TaskContext, Logging} import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.MeanEvaluator 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 1fad40bcf8242..6811e1abb8b70 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.annotation.DeveloperApi import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.NextIterator 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 bdfe9a41041e6..2d8dfa5a1645a 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.annotation.DeveloperApi private[spark] class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputSplit with Writable) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index e816b213c802a..db47450122bd4 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -39,7 +39,7 @@ RecordWriter => NewRecordWriter, SparkHadoopMapReduceUtil} import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat} import org.apache.spark._ -import org.apache.spark.annotations.Experimental +import org.apache.spark.annotation.Experimental import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.SparkHadoopWriter import org.apache.spark.Partitioner.defaultPartitioner 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 5a42ba6a9d05a..f781a8d776f2a 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.annotation.DeveloperApi private[spark] class PartitionPruningRDDPartition(idx: Int, val parentSplit: Partition) extends Partition { 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 e9af8b358acaa..a9fb1cb7f9b2e 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.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaRDD import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.CountEvaluator 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 3661fe37f6f53..802b0bdfb2d59 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.annotation.DeveloperApi import org.apache.spark.serializer.Serializer private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { 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 0313a12867374..21c6e07d69f90 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.annotation.DeveloperApi private[spark] class UnionPartition[T: ClassTag](idx: Int, rdd: RDD[T], splitIndex: Int) extends Partition { 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 3fd2c4ff3b570..cf12a98741abe 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala @@ -27,7 +27,7 @@ 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.annotation.DeveloperApi import org.apache.spark.deploy.SparkHadoopUtil /** 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 30d898c991548..757b2992d07ab 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -25,7 +25,7 @@ import java.util.concurrent.LinkedBlockingQueue import scala.collection.mutable.HashMap import org.apache.spark._ -import org.apache.spark.annotations.DeveloperApi +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics /** 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 dc41effd59950..77bb9fe0635b1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler -import org.apache.spark.annotations.DeveloperApi +import org.apache.spark.annotation.DeveloperApi /** * :: DeveloperApi :: diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 50de7c2a9df8e..85c8e86445100 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -23,7 +23,7 @@ import scala.collection.Map import scala.collection.mutable import org.apache.spark.{Logging, TaskEndReason} -import org.apache.spark.annotations.DeveloperApi +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.{Distribution, Utils} 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 e245ff8c25e0b..b85eabd6bbdbc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler import collection.mutable.ArrayBuffer -import org.apache.spark.annotations.DeveloperApi +import org.apache.spark.annotation.DeveloperApi // information about a specific split instance : handles both split instances. // So that we do not need to worry about the differences. 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 e2e287666834b..9b661c380d6c4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler -import org.apache.spark.annotations.DeveloperApi +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.storage.RDDInfo /** 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 a37ab5a338e4f..4c62e4dc0bac8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler -import org.apache.spark.annotations.DeveloperApi +import org.apache.spark.annotation.DeveloperApi /** * :: DeveloperApi :: 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 eaf736fb20ea1..eb920ab0c0b67 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,7 @@ package org.apache.spark.scheduler -import org.apache.spark.annotations.DeveloperApi +import org.apache.spark.annotation.DeveloperApi @DeveloperApi object TaskLocality extends Enumeration { 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 6c6463b1c8827..5e5883554fcc1 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.annotation.DeveloperApi import org.apache.spark.util.ByteBufferInputStream private[spark] class JavaSerializationStream(out: OutputStream, counterReset: Int) 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 d404035646065..9f04dc6e427c0 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -23,7 +23,7 @@ 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.annotation.DeveloperApi import org.apache.spark.util.{ByteBufferInputStream, NextIterator} /** 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 d2df92ea7655d..8d8c8c217e69f 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.annotation.DeveloperApi import org.apache.spark.util.Utils private[spark] 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 a68859a1f7669..74fa77b68de0b 100644 --- a/core/src/main/scala/org/apache/spark/util/MutablePair.scala +++ b/core/src/main/scala/org/apache/spark/util/MutablePair.scala @@ -17,7 +17,7 @@ package org.apache.spark.util -import org.apache.spark.annotations.DeveloperApi +import org.apache.spark.annotation.DeveloperApi /** * :: DeveloperApi :: 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 f1fbfdd51896d..025492b177a77 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,7 +19,7 @@ package org.apache.spark.util.collection import java.util.{Arrays, Comparator} -import org.apache.spark.annotations.DeveloperApi +import org.apache.spark.annotation.DeveloperApi /** * :: DeveloperApi :: 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 abd3a6d61731b..dd01ae821f705 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,7 +27,7 @@ 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.annotation.DeveloperApi import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{BlockId, BlockManager} 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 73011361b0676..62f99f3981793 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,7 +19,7 @@ package org.apache.spark.util.collection import scala.reflect.ClassTag -import org.apache.spark.annotations.DeveloperApi +import org.apache.spark.annotation.DeveloperApi /** * :: DeveloperApi :: 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 745c38ef72143..70f3dd62b9b19 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,7 +17,7 @@ package org.apache.spark.util.random -import org.apache.spark.annotations.DeveloperApi +import org.apache.spark.annotation.DeveloperApi /** * :: DeveloperApi :: diff --git a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala index 5306b89fddc86..37a6b04f5200f 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,7 +22,7 @@ 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.annotation.DeveloperApi /** * :: DeveloperApi :: 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 b8e4867f81a7f..68d1eedbeae22 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,7 +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.annotation.{AlphaComponent, Experimental} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.dsl 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 8eaddd5d0770e..f6737b89d65ed 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,7 +17,7 @@ package org.apache.spark.sql -import org.apache.spark.annotations.{AlphaComponent, Experimental} +import org.apache.spark.annotation.{AlphaComponent, Experimental} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._