diff options
author | Patrick Wendell <pwendell@gmail.com> | 2014-04-09 01:14:46 -0700 |
---|---|---|
committer | Patrick Wendell <pwendell@gmail.com> | 2014-04-09 01:14:46 -0700 |
commit | 87bd1f9ef7d547ee54a8a83214b45462e0751efb (patch) | |
tree | f815ba0464d366300b9bf3833ad3f5fdea2ca9eb /sql/core | |
parent | 9689b663a2a4947ad60795321c770052f3c637f1 (diff) | |
download | spark-87bd1f9ef7d547ee54a8a83214b45462e0751efb.tar.gz spark-87bd1f9ef7d547ee54a8a83214b45462e0751efb.tar.bz2 spark-87bd1f9ef7d547ee54a8a83214b45462e0751efb.zip |
SPARK-1093: Annotate developer and experimental API's
This patch marks some existing classes as private[spark] and adds two types of API annotations:
- `EXPERIMENTAL API` = experimental user-facing module
- `DEVELOPER API - UNSTABLE` = developer-facing API that might change
There is some discussion of the different mechanisms for doing this here:
https://issues.apache.org/jira/browse/SPARK-1081
I was pretty aggressive with marking things private. Keep in mind that if we want to open something up in the future we can, but we can never reduce visibility.
A few notes here:
- In the past we've been inconsistent with the visiblity of the X-RDD classes. This patch marks them private whenever there is an existing function in RDD that can directly creat them (e.g. CoalescedRDD and rdd.coalesce()). One trade-off here is users can't subclass them.
- Noted that compression and serialization formats don't have to be wire compatible across versions.
- Compression codecs and serialization formats are semi-private as users typically don't instantiate them directly.
- Metrics sources are made private - user only interacts with them through Spark's reflection
Author: Patrick Wendell <pwendell@gmail.com>
Author: Andrew Or <andrewor14@gmail.com>
Closes #274 from pwendell/private-apis and squashes the following commits:
44179e4 [Patrick Wendell] Merge remote-tracking branch 'apache-github/master' into private-apis
042c803 [Patrick Wendell] spark.annotations -> spark.annotation
bfe7b52 [Patrick Wendell] Adding experimental for approximate counts
8d0c873 [Patrick Wendell] Warning in SparkEnv
99b223a [Patrick Wendell] Cleaning up annotations
e849f64 [Patrick Wendell] Merge pull request #2 from andrewor14/annotations
982a473 [Andrew Or] Generalize jQuery matching for non Spark-core API docs
a01c076 [Patrick Wendell] Merge pull request #1 from andrewor14/annotations
c1bcb41 [Andrew Or] DeveloperAPI -> DeveloperApi
0d48908 [Andrew Or] Comments and new lines (minor)
f3954e0 [Andrew Or] Add identifier tags in comments to work around scaladocs bug
99192ef [Andrew Or] Dynamically add badges based on annotations
824011b [Andrew Or] Add support for injecting arbitrary JavaScript to API docs
037755c [Patrick Wendell] Some changes after working with andrew or
f7d124f [Patrick Wendell] Small fixes
c318b24 [Patrick Wendell] Use CSS styles
e4c76b9 [Patrick Wendell] Logging
f390b13 [Patrick Wendell] Better visibility for workaround constructors
d6b0afd [Patrick Wendell] Small chang to existing constructor
403ba52 [Patrick Wendell] Style fix
870a7ba [Patrick Wendell] Work around for SI-8479
7fb13b2 [Patrick Wendell] Changes to UnionRDD and EmptyRDD
4a9e90c [Patrick Wendell] EXPERIMENTAL API --> EXPERIMENTAL
c581dce [Patrick Wendell] Changes after building against Shark.
8452309 [Patrick Wendell] Style fixes
1ed27d2 [Patrick Wendell] Formatting and coloring of badges
cd7a465 [Patrick Wendell] Code review feedback
2f706f1 [Patrick Wendell] Don't use floats
542a736 [Patrick Wendell] Small fixes
cf23ec6 [Patrick Wendell] Marking GraphX as alpha
d86818e [Patrick Wendell] Another naming change
5a76ed6 [Patrick Wendell] More visiblity clean-up
42c1f09 [Patrick Wendell] Using better labels
9d48cbf [Patrick Wendell] Initial pass
Diffstat (limited to 'sql/core')
-rw-r--r-- | sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala | 9 | ||||
-rw-r--r-- | sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala | 21 |
2 files changed, 16 insertions, 14 deletions
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 3193787680..d3d4c56baf 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.annotation.{AlphaComponent, Experimental} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.dsl @@ -32,14 +33,14 @@ import org.apache.spark.sql.columnar.InMemoryColumnarTableScan import org.apache.spark.sql.execution._ /** - * <span class="badge" style="float: right; background-color: darkblue;">ALPHA COMPONENT</span> - * + * :: AlphaComponent :: * 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 @@ -63,12 +64,12 @@ class SQLContext(@transient val sparkContext: SparkContext) new this.QueryExecution { val logical = plan } /** - * <span class="badge badge-red" style="float: right;">EXPERIMENTAL</span> - * + * :: 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 fc95781448..16da7fd92b 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql import org.apache.spark.{Dependency, OneToOneDependency, Partition, TaskContext} +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._ @@ -26,8 +27,7 @@ import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} import org.apache.spark.sql.catalyst.types.BooleanType /** - * <span class="badge" style="float: right; background-color: darkblue;">ALPHA COMPONENT</span> - * + * :: 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. * @@ -90,6 +90,7 @@ import org.apache.spark.sql.catalyst.types.BooleanType * @groupprio schema -1 * @groupname Ungrouped Base RDD Functions */ +@AlphaComponent class SchemaRDD( @transient val sqlContext: SQLContext, @transient protected[spark] val logicalPlan: LogicalPlan) @@ -228,8 +229,7 @@ class SchemaRDD( Filter(ScalaUdf(udf, BooleanType, Seq(UnresolvedAttribute(arg1.name))), logicalPlan)) /** - * <span class="badge badge-red" style="float: right;">EXPERIMENTAL</span> - * + * :: 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 @@ -241,18 +241,19 @@ class SchemaRDD( * * @group Query */ + @Experimental def where(dynamicUdf: (DynamicRow) => Boolean) = new SchemaRDD( sqlContext, Filter(ScalaUdf(dynamicUdf, BooleanType, Seq(WrapDynamic(logicalPlan.output))), logicalPlan)) /** - * <span class="badge badge-red" style="float: right;">EXPERIMENTAL</span> - * + * :: Experimental :: * Returns a sampled version of the underlying dataset. * * @group Query */ + @Experimental def sample( fraction: Double, withReplacement: Boolean = true, @@ -260,8 +261,7 @@ class SchemaRDD( new SchemaRDD(sqlContext, Sample(fraction, withReplacement, seed, logicalPlan)) /** - * <span class="badge badge-red" style="float: right;">EXPERIMENTAL</span> - * + * :: 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 @@ -277,6 +277,7 @@ class SchemaRDD( * * @group Query */ + @Experimental def generate( generator: Generator, join: Boolean = false, @@ -285,8 +286,7 @@ class SchemaRDD( new SchemaRDD(sqlContext, Generate(generator, join, outer, None, logicalPlan)) /** - * <span class="badge badge-red" style="float: right;">EXPERIMENTAL</span> - * + * :: 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 @@ -294,6 +294,7 @@ class SchemaRDD( * * @group schema */ + @Experimental def insertInto(tableName: String, overwrite: Boolean = false) = new SchemaRDD( sqlContext, |