A few minor Spark SQL Scaladoc fixes.
Author: Reynold Xin <rxin@apache.org> Closes #1139 from rxin/sparksqldoc and squashes the following commits: c3049d8 [Reynold Xin] Fixed line length. 66dc72c [Reynold Xin] A few minor Spark SQL Scaladoc fixes.
This commit is contained in:
parent
f14b00a9c6
commit
5464e79175
|
@ -33,14 +33,11 @@ abstract class Expression extends TreeNode[Expression] {
|
|||
* executed.
|
||||
*
|
||||
* The following conditions are used to determine suitability for constant folding:
|
||||
* - A [[expressions.Coalesce Coalesce]] is foldable if all of its children are foldable
|
||||
* - A [[expressions.BinaryExpression BinaryExpression]] is foldable if its both left and right
|
||||
* child are foldable
|
||||
* - A [[expressions.Not Not]], [[expressions.IsNull IsNull]], or
|
||||
* [[expressions.IsNotNull IsNotNull]] is foldable if its child is foldable.
|
||||
* - A [[expressions.Literal]] is foldable.
|
||||
* - A [[expressions.Cast Cast]] or [[expressions.UnaryMinus UnaryMinus]] is foldable if its
|
||||
* child is foldable.
|
||||
* - A [[Coalesce]] is foldable if all of its children are foldable
|
||||
* - A [[BinaryExpression]] is foldable if its both left and right child are foldable
|
||||
* - A [[Not]], [[IsNull]], or [[IsNotNull]] is foldable if its child is foldable
|
||||
* - A [[Literal]] is foldable
|
||||
* - A [[Cast]] or [[UnaryMinus]] is foldable if its child is foldable
|
||||
*/
|
||||
def foldable: Boolean = false
|
||||
def nullable: Boolean
|
||||
|
@ -58,7 +55,7 @@ abstract class Expression extends TreeNode[Expression] {
|
|||
lazy val resolved: Boolean = childrenResolved
|
||||
|
||||
/**
|
||||
* Returns the [[types.DataType DataType]] of the result of evaluating this expression. It is
|
||||
* Returns the [[DataType]] of the result of evaluating this expression. It is
|
||||
* invalid to query the dataType of an unresolved expression (i.e., when `resolved` == false).
|
||||
*/
|
||||
def dataType: DataType
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
package org.apache.spark.sql.catalyst.plans
|
||||
|
||||
import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression}
|
||||
import org.apache.spark.sql.catalyst.plans
|
||||
import org.apache.spark.sql.catalyst.trees.TreeNode
|
||||
import org.apache.spark.sql.catalyst.types.{ArrayType, DataType, StructField, StructType}
|
||||
|
||||
|
|
|
@ -41,19 +41,19 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] {
|
|||
/**
|
||||
* Returns true if this expression and all its children have been resolved to a specific schema
|
||||
* and false if it is still contains any unresolved placeholders. Implementations of LogicalPlan
|
||||
* can override this (e.g. [[catalyst.analysis.UnresolvedRelation UnresolvedRelation]] should
|
||||
* return `false`).
|
||||
* can override this (e.g.
|
||||
* [[org.apache.spark.sql.catalyst.analysis.UnresolvedRelation UnresolvedRelation]]
|
||||
* should return `false`).
|
||||
*/
|
||||
lazy val resolved: Boolean = !expressions.exists(!_.resolved) && childrenResolved
|
||||
|
||||
/**
|
||||
* Returns true if all its children of this query plan have been resolved.
|
||||
*/
|
||||
def childrenResolved = !children.exists(!_.resolved)
|
||||
def childrenResolved: Boolean = !children.exists(!_.resolved)
|
||||
|
||||
/**
|
||||
* Optionally resolves the given string to a
|
||||
* [[catalyst.expressions.NamedExpression NamedExpression]]. The attribute is expressed as
|
||||
* Optionally resolves the given string to a [[NamedExpression]]. The attribute is expressed as
|
||||
* as string in the following form: `[scope].AttributeName.[nested].[fields]...`.
|
||||
*/
|
||||
def resolve(name: String): Option[NamedExpression] = {
|
||||
|
@ -93,7 +93,7 @@ abstract class LeafNode extends LogicalPlan with trees.LeafNode[LogicalPlan] {
|
|||
self: Product =>
|
||||
|
||||
// Leaf nodes by definition cannot reference any input attributes.
|
||||
def references = Set.empty
|
||||
override def references = Set.empty
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -27,7 +27,7 @@ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extend
|
|||
}
|
||||
|
||||
/**
|
||||
* Applies a [[catalyst.expressions.Generator Generator]] to a stream of input rows, combining the
|
||||
* 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
|
||||
* their output.
|
||||
|
@ -46,32 +46,32 @@ case class Generate(
|
|||
child: LogicalPlan)
|
||||
extends UnaryNode {
|
||||
|
||||
protected def generatorOutput =
|
||||
protected def generatorOutput: Seq[Attribute] =
|
||||
alias
|
||||
.map(a => generator.output.map(_.withQualifiers(a :: Nil)))
|
||||
.getOrElse(generator.output)
|
||||
|
||||
def output =
|
||||
override def output =
|
||||
if (join) child.output ++ generatorOutput else generatorOutput
|
||||
|
||||
def references =
|
||||
override def references =
|
||||
if (join) child.outputSet else generator.references
|
||||
}
|
||||
|
||||
case class Filter(condition: Expression, child: LogicalPlan) extends UnaryNode {
|
||||
def output = child.output
|
||||
def references = condition.references
|
||||
override def output = child.output
|
||||
override def references = condition.references
|
||||
}
|
||||
|
||||
case class Union(left: LogicalPlan, right: LogicalPlan) extends BinaryNode {
|
||||
// TODO: These aren't really the same attributes as nullability etc might change.
|
||||
def output = left.output
|
||||
override def output = left.output
|
||||
|
||||
override lazy val resolved =
|
||||
childrenResolved &&
|
||||
!left.output.zip(right.output).exists { case (l,r) => l.dataType != r.dataType }
|
||||
|
||||
def references = Set.empty
|
||||
override def references = Set.empty
|
||||
}
|
||||
|
||||
case class Join(
|
||||
|
@ -80,8 +80,8 @@ case class Join(
|
|||
joinType: JoinType,
|
||||
condition: Option[Expression]) extends BinaryNode {
|
||||
|
||||
def references = condition.map(_.references).getOrElse(Set.empty)
|
||||
def output = joinType match {
|
||||
override def references = condition.map(_.references).getOrElse(Set.empty)
|
||||
override def output = joinType match {
|
||||
case LeftSemi =>
|
||||
left.output
|
||||
case _ =>
|
||||
|
@ -96,9 +96,9 @@ case class InsertIntoTable(
|
|||
overwrite: Boolean)
|
||||
extends LogicalPlan {
|
||||
// The table being inserted into is a child for the purposes of transformations.
|
||||
def children = table :: child :: Nil
|
||||
def references = Set.empty
|
||||
def output = child.output
|
||||
override def children = table :: child :: Nil
|
||||
override def references = Set.empty
|
||||
override def output = child.output
|
||||
|
||||
override lazy val resolved = childrenResolved && child.output.zip(table.output).forall {
|
||||
case (childAttr, tableAttr) => childAttr.dataType == tableAttr.dataType
|
||||
|
@ -109,20 +109,20 @@ case class InsertIntoCreatedTable(
|
|||
databaseName: Option[String],
|
||||
tableName: String,
|
||||
child: LogicalPlan) extends UnaryNode {
|
||||
def references = Set.empty
|
||||
def output = child.output
|
||||
override def references = Set.empty
|
||||
override def output = child.output
|
||||
}
|
||||
|
||||
case class WriteToFile(
|
||||
path: String,
|
||||
child: LogicalPlan) extends UnaryNode {
|
||||
def references = Set.empty
|
||||
def output = child.output
|
||||
override def references = Set.empty
|
||||
override def output = child.output
|
||||
}
|
||||
|
||||
case class Sort(order: Seq[SortOrder], child: LogicalPlan) extends UnaryNode {
|
||||
def output = child.output
|
||||
def references = order.flatMap(_.references).toSet
|
||||
override def output = child.output
|
||||
override def references = order.flatMap(_.references).toSet
|
||||
}
|
||||
|
||||
case class Aggregate(
|
||||
|
@ -131,18 +131,19 @@ case class Aggregate(
|
|||
child: LogicalPlan)
|
||||
extends UnaryNode {
|
||||
|
||||
def output = aggregateExpressions.map(_.toAttribute)
|
||||
def references = (groupingExpressions ++ aggregateExpressions).flatMap(_.references).toSet
|
||||
override def output = aggregateExpressions.map(_.toAttribute)
|
||||
override def references =
|
||||
(groupingExpressions ++ aggregateExpressions).flatMap(_.references).toSet
|
||||
}
|
||||
|
||||
case class Limit(limitExpr: Expression, child: LogicalPlan) extends UnaryNode {
|
||||
def output = child.output
|
||||
def references = limitExpr.references
|
||||
override def output = child.output
|
||||
override def references = limitExpr.references
|
||||
}
|
||||
|
||||
case class Subquery(alias: String, child: LogicalPlan) extends UnaryNode {
|
||||
def output = child.output.map(_.withQualifiers(alias :: Nil))
|
||||
def references = Set.empty
|
||||
override def output = child.output.map(_.withQualifiers(alias :: Nil))
|
||||
override def references = Set.empty
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -159,7 +160,7 @@ case class LowerCaseSchema(child: LogicalPlan) extends UnaryNode {
|
|||
case otherType => otherType
|
||||
}
|
||||
|
||||
val output = child.output.map {
|
||||
override val output = child.output.map {
|
||||
case a: AttributeReference =>
|
||||
AttributeReference(
|
||||
a.name.toLowerCase,
|
||||
|
@ -170,21 +171,21 @@ case class LowerCaseSchema(child: LogicalPlan) extends UnaryNode {
|
|||
case other => other
|
||||
}
|
||||
|
||||
def references = Set.empty
|
||||
override def references = Set.empty
|
||||
}
|
||||
|
||||
case class Sample(fraction: Double, withReplacement: Boolean, seed: Long, child: LogicalPlan)
|
||||
extends UnaryNode {
|
||||
|
||||
def output = child.output
|
||||
def references = Set.empty
|
||||
override def output = child.output
|
||||
override def references = Set.empty
|
||||
}
|
||||
|
||||
case class Distinct(child: LogicalPlan) extends UnaryNode {
|
||||
def output = child.output
|
||||
def references = child.outputSet
|
||||
override def output = child.output
|
||||
override def references = child.outputSet
|
||||
}
|
||||
|
||||
case object NoRelation extends LeafNode {
|
||||
def output = Nil
|
||||
override def output = Nil
|
||||
}
|
||||
|
|
|
@ -46,7 +46,7 @@ case object AllTuples extends Distribution
|
|||
|
||||
/**
|
||||
* Represents data where tuples that share the same values for the `clustering`
|
||||
* [[catalyst.expressions.Expression Expressions]] will be co-located. Based on the context, this
|
||||
* [[Expression Expressions]] will be co-located. Based on the context, this
|
||||
* can mean such tuples are either co-located in the same partition or they will be contiguous
|
||||
* within a single partition.
|
||||
*/
|
||||
|
@ -60,7 +60,7 @@ case class ClusteredDistribution(clustering: Seq[Expression]) extends Distributi
|
|||
|
||||
/**
|
||||
* Represents data where tuples have been ordered according to the `ordering`
|
||||
* [[catalyst.expressions.Expression Expressions]]. This is a strictly stronger guarantee than
|
||||
* [[Expression Expressions]]. This is a strictly stronger guarantee than
|
||||
* [[ClusteredDistribution]] as an ordering will ensure that tuples that share the same value for
|
||||
* the ordering expressions are contiguous and will never be split across partitions.
|
||||
*/
|
||||
|
@ -79,19 +79,17 @@ sealed trait Partitioning {
|
|||
val numPartitions: Int
|
||||
|
||||
/**
|
||||
* Returns true iff the guarantees made by this
|
||||
* [[catalyst.plans.physical.Partitioning Partitioning]] are sufficient to satisfy
|
||||
* the partitioning scheme mandated by the `required`
|
||||
* [[catalyst.plans.physical.Distribution Distribution]], i.e. the current dataset does not
|
||||
* need to be re-partitioned for the `required` Distribution (it is possible that tuples within
|
||||
* a partition need to be reorganized).
|
||||
* Returns true iff the guarantees made by this [[Partitioning]] are sufficient
|
||||
* to satisfy the partitioning scheme mandated by the `required` [[Distribution]],
|
||||
* i.e. the current dataset does not need to be re-partitioned for the `required`
|
||||
* Distribution (it is possible that tuples within a partition need to be reorganized).
|
||||
*/
|
||||
def satisfies(required: Distribution): Boolean
|
||||
|
||||
/**
|
||||
* Returns true iff all distribution guarantees made by this partitioning can also be made
|
||||
* for the `other` specified partitioning.
|
||||
* For example, two [[catalyst.plans.physical.HashPartitioning HashPartitioning]]s are
|
||||
* For example, two [[HashPartitioning HashPartitioning]]s are
|
||||
* only compatible if the `numPartitions` of them is the same.
|
||||
*/
|
||||
def compatibleWith(other: Partitioning): Boolean
|
||||
|
|
|
@ -20,7 +20,7 @@ package org.apache.spark.sql.execution
|
|||
import org.apache.spark.annotation.DeveloperApi
|
||||
import org.apache.spark.{HashPartitioner, RangePartitioner, SparkConf}
|
||||
import org.apache.spark.rdd.ShuffledRDD
|
||||
import org.apache.spark.sql.{SQLConf, SQLContext, Row}
|
||||
import org.apache.spark.sql.{SQLContext, Row}
|
||||
import org.apache.spark.sql.catalyst.errors.attachTree
|
||||
import org.apache.spark.sql.catalyst.expressions.{NoBind, MutableProjection, RowOrdering}
|
||||
import org.apache.spark.sql.catalyst.plans.physical._
|
||||
|
@ -82,9 +82,10 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una
|
|||
}
|
||||
|
||||
/**
|
||||
* Ensures that the [[catalyst.plans.physical.Partitioning Partitioning]] of input data meets the
|
||||
* [[catalyst.plans.physical.Distribution Distribution]] requirements for each operator by inserting
|
||||
* [[Exchange]] Operators where required.
|
||||
* Ensures that the [[org.apache.spark.sql.catalyst.plans.physical.Partitioning Partitioning]]
|
||||
* of input data meets the
|
||||
* [[org.apache.spark.sql.catalyst.plans.physical.Distribution Distribution]] requirements for
|
||||
* each operator by inserting [[Exchange]] Operators where required.
|
||||
*/
|
||||
private[sql] case class AddExchange(sqlContext: SQLContext) extends Rule[SparkPlan] {
|
||||
// TODO: Determine the number of partitions.
|
||||
|
|
Loading…
Reference in a new issue