Skip to content

Commit

Permalink
Fixed relative package imports for package catalyst
Browse files Browse the repository at this point in the history
  • Loading branch information
liancheng committed Mar 22, 2014
1 parent 646e554 commit 3dcbbbd
Show file tree
Hide file tree
Showing 47 changed files with 137 additions and 150 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -17,29 +17,26 @@

package org.apache.spark.sql.catalyst

import scala.util.matching.Regex
import scala.util.parsing.combinator._
import scala.util.parsing.combinator.lexical.StdLexical
import scala.util.parsing.combinator.syntactical.StandardTokenParsers
import scala.util.parsing.input.CharArrayReader.EofCh
import lexical._
import syntactical._
import token._

import analysis._
import expressions._
import plans._
import plans.logical._
import types._
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.types._

/**
* A very simple SQL parser. Based loosly on:
* A very simple SQL parser. Based loosely on:
* https://github.com/stephentu/scala-sql-parser/blob/master/src/main/scala/parser.scala
*
* Limitations:
* - Only supports a very limited subset of SQL.
* - Keywords must be capital.
*
* This is currently included mostly for illustrative purposes. Users wanting more complete support
* for a SQL like language should checkout the HiveQL support in the sql/hive subproject.
* for a SQL like language should checkout the HiveQL support in the sql/hive sub-project.
*/
class SqlParser extends StandardTokenParsers {

Expand Down Expand Up @@ -196,7 +193,7 @@ class SqlParser extends StandardTokenParsers {

protected lazy val from: Parser[LogicalPlan] = FROM ~> relations

// Based very loosly on the MySQL Grammar.
// Based very loosely on the MySQL Grammar.
// http://dev.mysql.com/doc/refman/5.0/en/join.html
protected lazy val relations: Parser[LogicalPlan] =
relation ~ "," ~ relation ^^ { case r1 ~ _ ~ r2 => Join(r1, r2, Inner, None) } |
Expand Down Expand Up @@ -261,9 +258,9 @@ class SqlParser extends StandardTokenParsers {
andExpression * (OR ^^^ { (e1: Expression, e2: Expression) => Or(e1,e2) })

protected lazy val andExpression: Parser[Expression] =
comparisionExpression * (AND ^^^ { (e1: Expression, e2: Expression) => And(e1,e2) })
comparisonExpression * (AND ^^^ { (e1: Expression, e2: Expression) => And(e1,e2) })

protected lazy val comparisionExpression: Parser[Expression] =
protected lazy val comparisonExpression: Parser[Expression] =
termExpression ~ "=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => Equals(e1, e2) } |
termExpression ~ "<" ~ termExpression ^^ { case e1 ~ _ ~ e2 => LessThan(e1, e2) } |
termExpression ~ "<=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => LessThanOrEqual(e1, e2) } |
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,10 @@ package org.apache.spark.sql
package catalyst
package analysis

import expressions._
import plans.logical._
import rules._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules._


/**
* A trivial [[Analyzer]] with an [[EmptyCatalog]] and [[EmptyFunctionRegistry]]. Used for testing
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,11 @@ package org.apache.spark.sql
package catalyst
package analysis

import plans.logical.{LogicalPlan, Subquery}
import scala.collection.mutable

import org.apache.spark.sql.catalyst.plans.logical.{Subquery, LogicalPlan}


/**
* An interface for looking up relations by name. Used by an [[Analyzer]].
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ package org.apache.spark.sql
package catalyst
package analysis

import expressions._
import org.apache.spark.sql.catalyst.expressions.Expression

/** A catalog for looking up user defined functions, used by an [[Analyzer]]. */
trait FunctionRegistry {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,10 @@ package org.apache.spark.sql
package catalyst
package analysis

import expressions._
import plans.logical._
import rules._
import types._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, Union}
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.catalyst.types._

/**
* A collection of [[catalyst.rules.Rule Rules]] that can be used to coerce differing types that
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,14 +18,14 @@
package org.apache.spark.sql.catalyst
package analysis

import plans.logical.LogicalPlan
import rules._
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan

/**
* A trait that should be mixed into query operators where an single instance might appear multiple
* times in a logical query plan. It is invalid to have multiple copies of the same attribute
* produced by distinct operators in a query tree as this breaks the gurantee that expression
* ids, which are used to differentate attributes, are unique.
* produced by distinct operators in a query tree as this breaks the guarantee that expression
* ids, which are used to differentiate attributes, are unique.
*
* Before analysis, all operators that include this trait will be asked to produce a new version
* of itself with globally unique expression ids.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@
* limitations under the License.
*/

package org.apache.spark.sql
package catalyst

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,9 @@ package org.apache.spark.sql
package catalyst
package analysis

import expressions._
import plans.logical.BaseRelation
import trees.TreeNode
import org.apache.spark.sql.catalyst.expressions.{Alias, NamedExpression, Expression, Attribute}
import org.apache.spark.sql.catalyst.plans.logical.BaseRelation
import org.apache.spark.sql.catalyst.trees.TreeNode

/**
* Thrown when an invalid attempt is made to access a property of a tree that has yet to be fully
Expand Down Expand Up @@ -95,7 +95,7 @@ case class Star(
// If there is no table specified, use all input attributes.
case None => input
// If there is a table, pick out attributes that are part of this table.
case Some(table) => input.filter(_.qualifiers contains table)
case Some(t) => input.filter(_.qualifiers contains t)
}
val mappedAttributes = expandedAttributes.map(mapFunction).zip(input).map {
case (n: NamedExpression, _) => n
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,13 +19,12 @@ package org.apache.spark.sql
package catalyst

import scala.language.implicitConversions
import scala.reflect.runtime.universe.TypeTag

import analysis.UnresolvedAttribute
import expressions._
import plans._
import plans.logical._
import types._
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.plans.{Inner, JoinType}
import org.apache.spark.sql.catalyst.types._

/**
* Provides experimental support for generating catalyst schemas for scala objects.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
package org.apache.spark.sql
package catalyst

import trees._
import org.apache.spark.sql.catalyst.trees.TreeNode

/**
* Functions for attaching and retrieving trees that are associated with errors.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,9 @@ package org.apache.spark.sql
package catalyst
package expressions

import rules._
import errors._

import catalyst.plans.QueryPlan
import org.apache.spark.sql.catalyst.errors.attachTree
import org.apache.spark.sql.catalyst.plans.QueryPlan
import org.apache.spark.sql.catalyst.rules.Rule

/**
* A bound reference points to a specific slot in the input tuple, allowing the actual value
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ package org.apache.spark.sql
package catalyst
package expressions

import types._
import org.apache.spark.sql.catalyst.types._

/** Cast the child expression to the target data type. */
case class Cast(child: Expression, dataType: DataType) extends UnaryExpression {
Expand All @@ -40,7 +40,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression {
case (StringType, ShortType) => a: Any => castOrNull(a, _.toShort)
case (StringType, ByteType) => a: Any => castOrNull(a, _.toByte)
case (StringType, DecimalType) => a: Any => castOrNull(a, BigDecimal(_))
case (BooleanType, ByteType) => a: Any => a match {
case (BooleanType, ByteType) => {
case null => null
case true => 1.toByte
case false => 0.toByte
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,9 @@ package org.apache.spark.sql
package catalyst
package expressions

import errors._
import trees._
import types._
import org.apache.spark.sql.catalyst.trees.TreeNode
import org.apache.spark.sql.catalyst.types.{IntegralType, FractionalType, NumericType, DataType}
import org.apache.spark.sql.catalyst.errors.TreeNodeException

abstract class Expression extends TreeNode[Expression] {
self: Product =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ package org.apache.spark.sql
package catalyst
package expressions

import types.DoubleType
import org.apache.spark.sql.catalyst.types.DoubleType

case object Rand extends LeafExpression {
def dataType = DoubleType
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ package org.apache.spark.sql
package catalyst
package expressions

import types._
import org.apache.spark.sql.catalyst.types.NativeType

/**
* Represents one row of output from a relational operator. Allows both generic access by ordinal,
Expand Down Expand Up @@ -209,6 +209,6 @@ class RowOrdering(ordering: Seq[SortOrder]) extends Ordering[Row] {
}
i += 1
}
return 0
0
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ package org.apache.spark.sql
package catalyst
package expressions

import types._
import org.apache.spark.sql.catalyst.types.DataType

case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expression])
extends Expression {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ package expressions

import scala.language.dynamics

import types._
import org.apache.spark.sql.catalyst.types.DataType

case object DynamicType extends DataType

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ package org.apache.spark.sql
package catalyst
package expressions

import catalyst.types._
import org.apache.spark.sql.catalyst.types._

abstract class AggregateExpression extends Expression {
self: Product =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,9 @@ package org.apache.spark.sql
package catalyst
package expressions

import catalyst.analysis.UnresolvedException
import catalyst.types._
import org.apache.spark.sql.catalyst.analysis.UnresolvedException
import org.apache.spark.sql.catalyst.types._


case class UnaryMinus(child: Expression) extends UnaryExpression {
type EvaluatedType = Any
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ package org.apache.spark.sql
package catalyst
package expressions

import types._
import org.apache.spark.sql.catalyst.types._

/**
* Returns the item at `ordinal` in the Array `child` or the Key `ordinal` in Map `child`.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ package org.apache.spark.sql
package catalyst
package expressions

import catalyst.types._
import org.apache.spark.sql.catalyst.types._

/**
* An expression that produces zero or more rows given a single input row.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ package org.apache.spark.sql
package catalyst
package expressions

import types._
import org.apache.spark.sql.catalyst.types._

object Literal {
def apply(v: Any): Literal = v match {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,8 @@ package org.apache.spark.sql
package catalyst
package expressions

import catalyst.analysis.UnresolvedAttribute
import types._
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
import org.apache.spark.sql.catalyst.types._

object NamedExpression {
private val curId = new java.util.concurrent.atomic.AtomicLong()
Expand All @@ -30,7 +30,7 @@ object NamedExpression {
/**
* A globally (within this JVM) id for a given named expression.
* Used to identify with attribute output by a relation is being
* referenced in a subsuqent computation.
* referenced in a subsequent computation.
*/
case class ExprId(id: Long)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ package org.apache.spark.sql
package catalyst
package expressions

import catalyst.analysis.UnresolvedException
import org.apache.spark.sql.catalyst.analysis.UnresolvedException

case class Coalesce(children: Seq[Expression]) extends Expression {
type EvaluatedType = Any
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,8 @@ package org.apache.spark.sql
package catalyst
package expressions

import types._
import catalyst.analysis.UnresolvedException
import org.apache.spark.sql.catalyst.types.{StringType, BooleanType}
import org.apache.spark.sql.catalyst.analysis.UnresolvedException

trait Predicate extends Expression {
self: Product =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ package org.apache.spark.sql
package catalyst
package expressions

import catalyst.types.BooleanType
import org.apache.spark.sql.catalyst.types.BooleanType

case class Like(left: Expression, right: Expression) extends BinaryExpression {
def dataType = BooleanType
Expand Down
Loading

0 comments on commit 3dcbbbd

Please sign in to comment.