Skip to content
Snippets Groups Projects
Commit 2b5e31c7 authored by Reynold Xin's avatar Reynold Xin
Browse files

[SPARK-11113] [SQL] Remove DeveloperApi annotation from private classes.

o.a.s.sql.catalyst and o.a.s.sql.execution are supposed to be private.

Author: Reynold Xin <rxin@databricks.com>

Closes #9121 from rxin/SPARK-11113.
parent 56d7da14
No related branches found
No related tags found
No related merge requests found
Showing
with 21 additions and 118 deletions
......@@ -17,7 +17,6 @@
package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.sql.catalyst.rules
import org.apache.spark.util.Utils
......@@ -40,10 +39,8 @@ package object codegen {
}
/**
* :: DeveloperApi ::
* Dumps the bytecode from a class to the screen using javap.
*/
@DeveloperApi
object DumpByteCode {
import scala.sys.process._
val dumpDirectory = Utils.createTempDir()
......
......@@ -19,7 +19,6 @@ package org.apache.spark.sql.execution
import java.util.HashMap
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.errors._
......@@ -28,7 +27,6 @@ import org.apache.spark.sql.catalyst.plans.physical._
import org.apache.spark.sql.execution.metric.SQLMetrics
/**
* :: DeveloperApi ::
* Groups input data by `groupingExpressions` and computes the `aggregateExpressions` for each
* group.
*
......@@ -38,7 +36,6 @@ import org.apache.spark.sql.execution.metric.SQLMetrics
* @param aggregateExpressions expressions that are computed for each group.
* @param child the input data source.
*/
@DeveloperApi
case class Aggregate(
partial: Boolean,
groupingExpressions: Seq[Expression],
......
......@@ -19,7 +19,7 @@ package org.apache.spark.sql.execution
import java.util.Random
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark._
import org.apache.spark.rdd.RDD
import org.apache.spark.serializer.Serializer
import org.apache.spark.shuffle.hash.HashShuffleManager
......@@ -33,13 +33,10 @@ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjectio
import org.apache.spark.sql.catalyst.plans.physical._
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.util.MutablePair
import org.apache.spark._
/**
* :: DeveloperApi ::
* Performs a shuffle that will result in the desired `newPartitioning`.
*/
@DeveloperApi
case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends UnaryNode {
override def nodeName: String = if (tungstenMode) "TungstenExchange" else "Exchange"
......
......@@ -17,7 +17,6 @@
package org.apache.spark.sql.execution
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.{InternalRow, CatalystTypeConverters}
import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
......@@ -27,10 +26,7 @@ import org.apache.spark.sql.sources.BaseRelation
import org.apache.spark.sql.types.DataType
import org.apache.spark.sql.{Row, SQLContext}
/**
* :: DeveloperApi ::
*/
@DeveloperApi
object RDDConversions {
def productToRowRdd[A <: Product](data: RDD[A], outputTypes: Seq[DataType]): RDD[InternalRow] = {
data.mapPartitions { iterator =>
......
......@@ -17,7 +17,6 @@
package org.apache.spark.sql.execution
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.errors._
......@@ -32,7 +31,6 @@ import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartit
* @param output The output Schema
* @param child Child operator
*/
@DeveloperApi
case class Expand(
projections: Seq[Seq[Expression]],
output: Seq[Attribute],
......
......@@ -17,7 +17,6 @@
package org.apache.spark.sql.execution
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
......@@ -35,7 +34,6 @@ private[execution] sealed case class LazyIterator(func: () => TraversableOnce[In
}
/**
* :: DeveloperApi ::
* Applies a [[Generator]] to a stream of input rows, combining the
* output of each into a new stream of rows. This operation is similar to a `flatMap` in functional
* programming with one important additional feature, which allows the input rows to be joined with
......@@ -48,7 +46,6 @@ private[execution] sealed case class LazyIterator(func: () => TraversableOnce[In
* @param output the output attributes of this node, which constructed in analysis phase,
* and we can not change it, as the parent node bound with it already.
*/
@DeveloperApi
case class Generate(
generator: Generator,
join: Boolean,
......
......@@ -18,8 +18,7 @@
package org.apache.spark.sql.execution
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.{InternalRow, CatalystTypeConverters}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.Attribute
......
......@@ -17,18 +17,15 @@
package org.apache.spark.sql.execution
import org.apache.spark.annotation.{Experimental, DeveloperApi}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.{InternalRow, optimizer}
import org.apache.spark.sql.{SQLContext, Row}
import org.apache.spark.sql.SQLContext
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
/**
* :: DeveloperApi ::
* The primary workflow for executing relational queries using Spark. Designed to allow easy
* access to the intermediate phases of query execution for developers.
*/
@DeveloperApi
class QueryExecution(val sqlContext: SQLContext, val logical: LogicalPlan) {
val analyzer = sqlContext.analyzer
val optimizer = sqlContext.optimizer
......
......@@ -21,7 +21,6 @@ import org.apache.spark._
import org.apache.spark.rdd.RDD
import org.apache.spark.serializer.Serializer
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.types.DataType
private class ShuffledRowRDDPartition(val idx: Int) extends Partition {
override val index: Int = idx
......
......@@ -22,7 +22,6 @@ import java.util.concurrent.atomic.AtomicBoolean
import scala.collection.mutable.ArrayBuffer
import org.apache.spark.Logging
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.{RDD, RDDOperationScope}
import org.apache.spark.sql.SQLContext
import org.apache.spark.sql.catalyst.InternalRow
......@@ -32,7 +31,7 @@ import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.expressions.codegen._
import org.apache.spark.sql.catalyst.plans.QueryPlan
import org.apache.spark.sql.catalyst.plans.physical._
import org.apache.spark.sql.execution.metric.{LongSQLMetric, SQLMetric, SQLMetrics}
import org.apache.spark.sql.execution.metric.{LongSQLMetric, SQLMetric}
import org.apache.spark.sql.types.DataType
object SparkPlan {
......@@ -40,9 +39,8 @@ object SparkPlan {
}
/**
* :: DeveloperApi ::
* The base class for physical operators.
*/
@DeveloperApi
abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializable {
/**
......
......@@ -17,19 +17,14 @@
package org.apache.spark.sql.execution
import java.util
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.physical._
import org.apache.spark.sql.types.IntegerType
import org.apache.spark.rdd.RDD
import org.apache.spark.util.collection.CompactBuffer
import scala.collection.mutable
/**
* :: DeveloperApi ::
* This class calculates and outputs (windowed) aggregates over the rows in a single (sorted)
* partition. The aggregates are calculated for each row in the group. Special processing
* instructions, frames, are used to calculate these aggregates. Frames are processed in the order
......@@ -76,7 +71,6 @@ import scala.collection.mutable
* Entire Partition, Sliding, Growing & Shrinking. Boundary evaluation is also delegated to a pair
* of specialized classes: [[RowBoundOrdering]] & [[RangeBoundOrdering]].
*/
@DeveloperApi
case class Window(
projectList: Seq[Attribute],
windowExpression: Seq[NamedExpression],
......@@ -229,7 +223,7 @@ case class Window(
// function result buffer.
val framedWindowExprs = windowExprs.groupBy(_.windowSpec.frameSpecification)
val factories = Array.ofDim[() => WindowFunctionFrame](framedWindowExprs.size)
val unboundExpressions = mutable.Buffer.empty[Expression]
val unboundExpressions = scala.collection.mutable.Buffer.empty[Expression]
framedWindowExprs.zipWithIndex.foreach {
case ((frame, unboundFrameExpressions), index) =>
// Track the ordinal.
......@@ -529,7 +523,7 @@ private[execution] final class SlidingWindowFunctionFrame(
private[this] var inputLowIndex = 0
/** Buffer used for storing prepared input for the window functions. */
private[this] val buffer = new util.ArrayDeque[Array[AnyRef]]
private[this] val buffer = new java.util.ArrayDeque[Array[AnyRef]]
/** Index of the row we are currently writing. */
private[this] var outputIndex = 0
......
......@@ -17,7 +17,6 @@
package org.apache.spark.sql.execution
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.{PartitionwiseSampledRDD, RDD, ShuffledRDD}
import org.apache.spark.shuffle.sort.SortShuffleManager
import org.apache.spark.sql.catalyst.InternalRow
......@@ -28,10 +27,7 @@ import org.apache.spark.util.MutablePair
import org.apache.spark.util.random.PoissonSampler
import org.apache.spark.{HashPartitioner, SparkEnv}
/**
* :: DeveloperApi ::
*/
@DeveloperApi
case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends UnaryNode {
override def output: Seq[Attribute] = projectList.map(_.toAttribute)
......@@ -90,10 +86,6 @@ case class TungstenProject(projectList: Seq[NamedExpression], child: SparkPlan)
}
/**
* :: DeveloperApi ::
*/
@DeveloperApi
case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode {
override def output: Seq[Attribute] = child.output
......@@ -125,8 +117,8 @@ case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode {
}
/**
* :: DeveloperApi ::
* Sample the dataset.
*
* @param lowerBound Lower-bound of the sampling probability (usually 0.0)
* @param upperBound Upper-bound of the sampling probability. The expected fraction sampled
* will be ub - lb.
......@@ -134,7 +126,6 @@ case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode {
* @param seed the random seed
* @param child the SparkPlan
*/
@DeveloperApi
case class Sample(
lowerBound: Double,
upperBound: Double,
......@@ -165,9 +156,8 @@ case class Sample(
}
/**
* :: DeveloperApi ::
* Union two plans, without a distinct. This is UNION ALL in SQL.
*/
@DeveloperApi
case class Union(children: Seq[SparkPlan]) extends SparkPlan {
// TODO: attributes output by union should be distinct for nullability purposes
override def output: Seq[Attribute] = children.head.output
......@@ -179,14 +169,12 @@ case class Union(children: Seq[SparkPlan]) extends SparkPlan {
}
/**
* :: DeveloperApi ::
* Take the first limit elements. Note that the implementation is different depending on whether
* this is a terminal operator or not. If it is terminal and is invoked using executeCollect,
* this operator uses something similar to Spark's take method on the Spark driver. If it is not
* terminal or is invoked using execute, we first take the limit on each partition, and then
* repartition all the data to a single partition to compute the global limit.
*/
@DeveloperApi
case class Limit(limit: Int, child: SparkPlan)
extends UnaryNode {
// TODO: Implement a partition local limit, and use a strategy to generate the proper limit plan:
......@@ -219,14 +207,12 @@ case class Limit(limit: Int, child: SparkPlan)
}
/**
* :: DeveloperApi ::
* Take the first limit elements as defined by the sortOrder, and do projection if needed.
* This is logically equivalent to having a [[Limit]] operator after a [[Sort]] operator,
* or having a [[Project]] operator between them.
* This could have been named TopK, but Spark's top operator does the opposite in ordering
* so we name it TakeOrdered to avoid confusion.
*/
@DeveloperApi
case class TakeOrderedAndProject(
limit: Int,
sortOrder: Seq[SortOrder],
......@@ -271,13 +257,11 @@ case class TakeOrderedAndProject(
}
/**
* :: DeveloperApi ::
* Return a new RDD that has exactly `numPartitions` partitions.
* Similar to coalesce defined on an [[RDD]], this operation results in a narrow dependency, e.g.
* if you go from 1000 partitions to 100 partitions, there will not be a shuffle, instead each of
* the 100 new partitions will claim 10 of the current partitions.
*/
@DeveloperApi
case class Coalesce(numPartitions: Int, child: SparkPlan) extends UnaryNode {
override def output: Seq[Attribute] = child.output
......@@ -294,11 +278,9 @@ case class Coalesce(numPartitions: Int, child: SparkPlan) extends UnaryNode {
}
/**
* :: DeveloperApi ::
* Returns a table with the elements from left that are not in right using
* the built-in spark subtract function.
*/
@DeveloperApi
case class Except(left: SparkPlan, right: SparkPlan) extends BinaryNode {
override def output: Seq[Attribute] = left.output
......@@ -308,11 +290,9 @@ case class Except(left: SparkPlan, right: SparkPlan) extends BinaryNode {
}
/**
* :: DeveloperApi ::
* Returns the rows in left that also appear in right using the built in spark
* intersection function.
*/
@DeveloperApi
case class Intersect(left: SparkPlan, right: SparkPlan) extends BinaryNode {
override def output: Seq[Attribute] = children.head.output
......@@ -322,12 +302,10 @@ case class Intersect(left: SparkPlan, right: SparkPlan) extends BinaryNode {
}
/**
* :: DeveloperApi ::
* A plan node that does nothing but lie about the output of its child. Used to spice a
* (hopefully structurally equivalent) tree from a different optimization sequence into an already
* resolved tree.
*/
@DeveloperApi
case class OutputFaker(output: Seq[Attribute], child: SparkPlan) extends SparkPlan {
def children: Seq[SparkPlan] = child :: Nil
......
......@@ -20,11 +20,10 @@ package org.apache.spark.sql.execution
import java.util.NoSuchElementException
import org.apache.spark.Logging
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.{InternalRow, CatalystTypeConverters}
import org.apache.spark.sql.catalyst.errors.TreeNodeException
import org.apache.spark.sql.catalyst.expressions.{ExpressionDescription, Expression, Attribute, AttributeReference}
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
import org.apache.spark.sql.catalyst.plans.logical
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.types._
......@@ -74,10 +73,7 @@ private[sql] case class ExecutedCommand(cmd: RunnableCommand) extends SparkPlan
override def argString: String = cmd.toString
}
/**
* :: DeveloperApi ::
*/
@DeveloperApi
case class SetCommand(kv: Option[(String, Option[String])]) extends RunnableCommand with Logging {
private def keyValueOutput: Seq[Attribute] = {
......@@ -180,10 +176,7 @@ case class SetCommand(kv: Option[(String, Option[String])]) extends RunnableComm
*
* Note that this command takes in a logical plan, runs the optimizer on the logical plan
* (but do NOT actually execute it).
*
* :: DeveloperApi ::
*/
@DeveloperApi
case class ExplainCommand(
logicalPlan: LogicalPlan,
override val output: Seq[Attribute] =
......@@ -203,10 +196,7 @@ case class ExplainCommand(
}
}
/**
* :: DeveloperApi ::
*/
@DeveloperApi
case class CacheTableCommand(
tableName: String,
plan: Option[LogicalPlan],
......@@ -231,10 +221,6 @@ case class CacheTableCommand(
}
/**
* :: DeveloperApi ::
*/
@DeveloperApi
case class UncacheTableCommand(tableName: String) extends RunnableCommand {
override def run(sqlContext: SQLContext): Seq[Row] = {
......@@ -246,10 +232,8 @@ case class UncacheTableCommand(tableName: String) extends RunnableCommand {
}
/**
* :: DeveloperApi ::
* Clear all cached data from the in-memory cache.
*/
@DeveloperApi
case object ClearCacheCommand extends RunnableCommand {
override def run(sqlContext: SQLContext): Seq[Row] = {
......@@ -260,10 +244,7 @@ case object ClearCacheCommand extends RunnableCommand {
override def output: Seq[Attribute] = Seq.empty
}
/**
* :: DeveloperApi ::
*/
@DeveloperApi
case class DescribeCommand(
child: SparkPlan,
override val output: Seq[Attribute],
......@@ -286,9 +267,7 @@ case class DescribeCommand(
* {{{
* SHOW TABLES [IN databaseName]
* }}}
* :: DeveloperApi ::
*/
@DeveloperApi
case class ShowTablesCommand(databaseName: Option[String]) extends RunnableCommand {
// The result of SHOW TABLES has two columns, tableName and isTemporary.
......
......@@ -20,7 +20,6 @@ package org.apache.spark.sql.execution.joins
import scala.concurrent._
import scala.concurrent.duration._
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.Expression
......@@ -31,13 +30,11 @@ import org.apache.spark.util.ThreadUtils
import org.apache.spark.{InternalAccumulator, TaskContext}
/**
* :: DeveloperApi ::
* Performs an inner hash join of two child relations. When the output RDD of this operator is
* being constructed, a Spark job is asynchronously started to calculate the values for the
* broadcasted relation. This data is then placed in a Spark broadcast variable. The streamed
* relation is not shuffled.
*/
@DeveloperApi
case class BroadcastHashJoin(
leftKeys: Seq[Expression],
rightKeys: Seq[Expression],
......
......@@ -20,7 +20,6 @@ package org.apache.spark.sql.execution.joins
import scala.concurrent._
import scala.concurrent.duration._
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
......@@ -31,13 +30,11 @@ import org.apache.spark.sql.execution.metric.SQLMetrics
import org.apache.spark.{InternalAccumulator, TaskContext}
/**
* :: DeveloperApi ::
* Performs a outer hash join for two child relations. When the output RDD of this operator is
* being constructed, a Spark job is asynchronously started to calculate the values for the
* broadcasted relation. This data is then placed in a Spark broadcast variable. The streamed
* relation is not shuffled.
*/
@DeveloperApi
case class BroadcastHashOuterJoin(
leftKeys: Seq[Expression],
rightKeys: Seq[Expression],
......
......@@ -18,7 +18,6 @@
package org.apache.spark.sql.execution.joins
import org.apache.spark.{InternalAccumulator, TaskContext}
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
......@@ -26,11 +25,9 @@ import org.apache.spark.sql.execution.{BinaryNode, SparkPlan}
import org.apache.spark.sql.execution.metric.SQLMetrics
/**
* :: DeveloperApi ::
* Build the right table's join keys into a HashSet, and iteratively go through the left
* table, to find the if join keys are in the Hash set.
*/
@DeveloperApi
case class BroadcastLeftSemiJoinHash(
leftKeys: Seq[Expression],
rightKeys: Seq[Expression],
......
......@@ -17,7 +17,6 @@
package org.apache.spark.sql.execution.joins
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
......@@ -27,10 +26,7 @@ import org.apache.spark.sql.execution.{BinaryNode, SparkPlan}
import org.apache.spark.sql.execution.metric.SQLMetrics
import org.apache.spark.util.collection.CompactBuffer
/**
* :: DeveloperApi ::
*/
@DeveloperApi
case class BroadcastNestedLoopJoin(
left: SparkPlan,
right: SparkPlan,
......
......@@ -17,17 +17,13 @@
package org.apache.spark.sql.execution.joins
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Attribute, JoinedRow}
import org.apache.spark.sql.execution.{BinaryNode, SparkPlan}
import org.apache.spark.sql.execution.metric.SQLMetrics
/**
* :: DeveloperApi ::
*/
@DeveloperApi
case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNode {
override def output: Seq[Attribute] = left.output ++ right.output
......
......@@ -17,9 +17,6 @@
package org.apache.spark.sql.execution.joins
import java.util.{HashMap => JavaHashMap}
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans._
......@@ -27,7 +24,7 @@ import org.apache.spark.sql.execution.SparkPlan
import org.apache.spark.sql.execution.metric.LongSQLMetric
import org.apache.spark.util.collection.CompactBuffer
@DeveloperApi
trait HashOuterJoin {
self: SparkPlan =>
......@@ -230,8 +227,8 @@ trait HashOuterJoin {
protected[this] def buildHashTable(
iter: Iterator[InternalRow],
numIterRows: LongSQLMetric,
keyGenerator: Projection): JavaHashMap[InternalRow, CompactBuffer[InternalRow]] = {
val hashTable = new JavaHashMap[InternalRow, CompactBuffer[InternalRow]]()
keyGenerator: Projection): java.util.HashMap[InternalRow, CompactBuffer[InternalRow]] = {
val hashTable = new java.util.HashMap[InternalRow, CompactBuffer[InternalRow]]()
while (iter.hasNext) {
val currentRow = iter.next()
numIterRows += 1
......
......@@ -17,7 +17,6 @@
package org.apache.spark.sql.execution.joins
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
......@@ -26,11 +25,9 @@ import org.apache.spark.sql.execution.{BinaryNode, SparkPlan}
import org.apache.spark.sql.execution.metric.SQLMetrics
/**
* :: DeveloperApi ::
* Using BroadcastNestedLoopJoin to calculate left semi join result when there's no join keys
* for hash join.
*/
@DeveloperApi
case class LeftSemiJoinBNL(
streamed: SparkPlan, broadcast: SparkPlan, condition: Option[Expression])
extends BinaryNode {
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment