Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ import scala.util.parsing.input.CharArrayReader.EofCh
import org.apache.spark.sql.catalyst.plans.logical._

private[sql] object KeywordNormalizer {
def apply(str: String) = str.toLowerCase()
def apply(str: String): String = str.toLowerCase()
}

private[sql] abstract class AbstractSparkSQLParser
Expand All @@ -42,7 +42,7 @@ private[sql] abstract class AbstractSparkSQLParser
}

protected case class Keyword(str: String) {
def normalize = KeywordNormalizer(str)
def normalize: String = KeywordNormalizer(str)
def parser: Parser[String] = normalize
}

Expand Down Expand Up @@ -81,7 +81,7 @@ private[sql] abstract class AbstractSparkSQLParser

class SqlLexical extends StdLexical {
case class FloatLit(chars: String) extends Token {
override def toString = chars
override def toString: String = chars
}

/* This is a work around to support the lazy setting */
Expand Down Expand Up @@ -120,7 +120,7 @@ class SqlLexical extends StdLexical {
| failure("illegal character")
)

override def identChar = letter | elem('_')
override def identChar: Parser[Elem] = letter | elem('_')

override def whitespace: Parser[Any] =
( whitespaceChar
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -168,7 +168,7 @@ class Analyzer(catalog: Catalog,
* Replaces [[UnresolvedRelation]]s with concrete relations from the catalog.
*/
object ResolveRelations extends Rule[LogicalPlan] {
def getTable(u: UnresolvedRelation) = {
def getTable(u: UnresolvedRelation): LogicalPlan = {
try {
catalog.lookupRelation(u.tableIdentifier, u.alias)
} catch {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -86,12 +86,12 @@ class SimpleCatalog(val caseSensitive: Boolean) extends Catalog {
tables += ((getDbTableName(tableIdent), plan))
}

override def unregisterTable(tableIdentifier: Seq[String]) = {
override def unregisterTable(tableIdentifier: Seq[String]): Unit = {
val tableIdent = processTableIdentifier(tableIdentifier)
tables -= getDbTableName(tableIdent)
}

override def unregisterAllTables() = {
override def unregisterAllTables(): Unit = {
tables.clear()
}

Expand Down Expand Up @@ -147,8 +147,8 @@ trait OverrideCatalog extends Catalog {
}

abstract override def lookupRelation(
tableIdentifier: Seq[String],
alias: Option[String] = None): LogicalPlan = {
tableIdentifier: Seq[String],
alias: Option[String] = None): LogicalPlan = {
val tableIdent = processTableIdentifier(tableIdentifier)
val overriddenTable = overrides.get(getDBTable(tableIdent))
val tableWithQualifers = overriddenTable.map(r => Subquery(tableIdent.last, r))
Expand Down Expand Up @@ -205,27 +205,27 @@ trait OverrideCatalog extends Catalog {
*/
object EmptyCatalog extends Catalog {

val caseSensitive: Boolean = true
override val caseSensitive: Boolean = true

def tableExists(tableIdentifier: Seq[String]): Boolean = {
override def tableExists(tableIdentifier: Seq[String]): Boolean = {
throw new UnsupportedOperationException
}

def lookupRelation(
tableIdentifier: Seq[String],
alias: Option[String] = None) = {
override def lookupRelation(
tableIdentifier: Seq[String],
alias: Option[String] = None): LogicalPlan = {
throw new UnsupportedOperationException
}

override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] = {
throw new UnsupportedOperationException
}

def registerTable(tableIdentifier: Seq[String], plan: LogicalPlan): Unit = {
override def registerTable(tableIdentifier: Seq[String], plan: LogicalPlan): Unit = {
throw new UnsupportedOperationException
}

def unregisterTable(tableIdentifier: Seq[String]): Unit = {
override def unregisterTable(tableIdentifier: Seq[String]): Unit = {
throw new UnsupportedOperationException
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ class CheckAnalysis {
*/
val extendedCheckRules: Seq[LogicalPlan => Unit] = Nil

def failAnalysis(msg: String) = {
def failAnalysis(msg: String): Nothing = {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Unit

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Unit won't actually compile because we use this as the last statement in transformExpressionUp and the compiler expects this function to return something.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

oh, no this is correct...

throw new AnalysisException(msg)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@ trait OverrideFunctionRegistry extends FunctionRegistry {

val functionBuilders = StringKeyHashMap[FunctionBuilder](caseSensitive)

def registerFunction(name: String, builder: FunctionBuilder) = {
override def registerFunction(name: String, builder: FunctionBuilder): Unit = {
functionBuilders.put(name, builder)
}

Expand All @@ -47,7 +47,7 @@ trait OverrideFunctionRegistry extends FunctionRegistry {
class SimpleFunctionRegistry(val caseSensitive: Boolean) extends FunctionRegistry {
val functionBuilders = StringKeyHashMap[FunctionBuilder](caseSensitive)

def registerFunction(name: String, builder: FunctionBuilder) = {
override def registerFunction(name: String, builder: FunctionBuilder): Unit = {
functionBuilders.put(name, builder)
}

Expand All @@ -61,13 +61,15 @@ class SimpleFunctionRegistry(val caseSensitive: Boolean) extends FunctionRegistr
* functions are already filled in and the analyser needs only to resolve attribute references.
*/
object EmptyFunctionRegistry extends FunctionRegistry {
def registerFunction(name: String, builder: FunctionBuilder) = ???
override def registerFunction(name: String, builder: FunctionBuilder): Unit = {
throw new UnsupportedOperationException
}

def lookupFunction(name: String, children: Seq[Expression]): Expression = {
override def lookupFunction(name: String, children: Seq[Expression]): Expression = {
throw new UnsupportedOperationException
}

def caseSensitive: Boolean = ???
override def caseSensitive: Boolean = throw new UnsupportedOperationException
}

/**
Expand All @@ -76,7 +78,7 @@ object EmptyFunctionRegistry extends FunctionRegistry {
* TODO move this into util folder?
*/
object StringKeyHashMap {
def apply[T](caseSensitive: Boolean) = caseSensitive match {
def apply[T](caseSensitive: Boolean): StringKeyHashMap[T] = caseSensitive match {
case false => new StringKeyHashMap[T](_.toLowerCase)
case true => new StringKeyHashMap[T](identity)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ package object analysis {

implicit class AnalysisErrorAt(t: TreeNode[_]) {
/** Fails the analysis at the point where a specific tree node was parsed. */
def failAnalysis(msg: String) = {
def failAnalysis(msg: String): Nothing = {
throw new AnalysisException(msg, t.origin.line, t.origin.startPosition)
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import org.apache.spark.sql.catalyst.errors.TreeNodeException
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical.LeafNode
import org.apache.spark.sql.catalyst.trees.TreeNode
import org.apache.spark.sql.types.DataType

/**
* Thrown when an invalid attempt is made to access a property of a tree that has yet to be fully
Expand All @@ -38,26 +39,27 @@ case class UnresolvedRelation(
alias: Option[String] = None) extends LeafNode {

/** Returns a `.` separated name for this relation. */
def tableName = tableIdentifier.mkString(".")
def tableName: String = tableIdentifier.mkString(".")

override def output: Seq[Attribute] = Nil

override def output = Nil
override lazy val resolved = false
}

/**
* Holds the name of an attribute that has yet to be resolved.
*/
case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNode[Expression] {
override def exprId = throw new UnresolvedException(this, "exprId")
override def dataType = throw new UnresolvedException(this, "dataType")
override def nullable = throw new UnresolvedException(this, "nullable")
override def qualifiers = throw new UnresolvedException(this, "qualifiers")
override def exprId: ExprId = throw new UnresolvedException(this, "exprId")
override def dataType: DataType = throw new UnresolvedException(this, "dataType")
override def nullable: Boolean = throw new UnresolvedException(this, "nullable")
override def qualifiers: Seq[String] = throw new UnresolvedException(this, "qualifiers")
override lazy val resolved = false

override def newInstance() = this
override def withNullability(newNullability: Boolean) = this
override def withQualifiers(newQualifiers: Seq[String]) = this
override def withName(newName: String) = UnresolvedAttribute(name)
override def newInstance(): UnresolvedAttribute = this
override def withNullability(newNullability: Boolean): UnresolvedAttribute = this
override def withQualifiers(newQualifiers: Seq[String]): UnresolvedAttribute = this
override def withName(newName: String): UnresolvedAttribute = UnresolvedAttribute(name)

// Unresolved attributes are transient at compile time and don't get evaluated during execution.
override def eval(input: Row = null): EvaluatedType =
Expand All @@ -67,16 +69,16 @@ case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNo
}

case class UnresolvedFunction(name: String, children: Seq[Expression]) extends Expression {
override def dataType = throw new UnresolvedException(this, "dataType")
override def foldable = throw new UnresolvedException(this, "foldable")
override def nullable = throw new UnresolvedException(this, "nullable")
override def dataType: DataType = throw new UnresolvedException(this, "dataType")
override def foldable: Boolean = throw new UnresolvedException(this, "foldable")
override def nullable: Boolean = throw new UnresolvedException(this, "nullable")
override lazy val resolved = false

// Unresolved functions are transient at compile time and don't get evaluated during execution.
override def eval(input: Row = null): EvaluatedType =
throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}")

override def toString = s"'$name(${children.mkString(",")})"
override def toString: String = s"'$name(${children.mkString(",")})"
}

/**
Expand All @@ -86,17 +88,17 @@ case class UnresolvedFunction(name: String, children: Seq[Expression]) extends E
trait Star extends Attribute with trees.LeafNode[Expression] {
self: Product =>

override def name = throw new UnresolvedException(this, "name")
override def exprId = throw new UnresolvedException(this, "exprId")
override def dataType = throw new UnresolvedException(this, "dataType")
override def nullable = throw new UnresolvedException(this, "nullable")
override def qualifiers = throw new UnresolvedException(this, "qualifiers")
override def name: String = throw new UnresolvedException(this, "name")
override def exprId: ExprId = throw new UnresolvedException(this, "exprId")
override def dataType: DataType = throw new UnresolvedException(this, "dataType")
override def nullable: Boolean = throw new UnresolvedException(this, "nullable")
override def qualifiers: Seq[String] = throw new UnresolvedException(this, "qualifiers")
override lazy val resolved = false

override def newInstance() = this
override def withNullability(newNullability: Boolean) = this
override def withQualifiers(newQualifiers: Seq[String]) = this
override def withName(newName: String) = this
override def newInstance(): Star = this
override def withNullability(newNullability: Boolean): Star = this
override def withQualifiers(newQualifiers: Seq[String]): Star = this
override def withName(newName: String): Star = this

// Star gets expanded at runtime so we never evaluate a Star.
override def eval(input: Row = null): EvaluatedType =
Expand Down Expand Up @@ -129,7 +131,7 @@ case class UnresolvedStar(table: Option[String]) extends Star {
}
}

override def toString = table.map(_ + ".").getOrElse("") + "*"
override def toString: String = table.map(_ + ".").getOrElse("") + "*"
}

/**
Expand All @@ -144,25 +146,25 @@ case class UnresolvedStar(table: Option[String]) extends Star {
case class MultiAlias(child: Expression, names: Seq[String])
extends Attribute with trees.UnaryNode[Expression] {

override def name = throw new UnresolvedException(this, "name")
override def name: String = throw new UnresolvedException(this, "name")

override def exprId = throw new UnresolvedException(this, "exprId")
override def exprId: ExprId = throw new UnresolvedException(this, "exprId")

override def dataType = throw new UnresolvedException(this, "dataType")
override def dataType: DataType = throw new UnresolvedException(this, "dataType")

override def nullable = throw new UnresolvedException(this, "nullable")
override def nullable: Boolean = throw new UnresolvedException(this, "nullable")

override def qualifiers = throw new UnresolvedException(this, "qualifiers")
override def qualifiers: Seq[String] = throw new UnresolvedException(this, "qualifiers")

override lazy val resolved = false

override def newInstance() = this
override def newInstance(): MultiAlias = this

override def withNullability(newNullability: Boolean) = this
override def withNullability(newNullability: Boolean): MultiAlias = this

override def withQualifiers(newQualifiers: Seq[String]) = this
override def withQualifiers(newQualifiers: Seq[String]): MultiAlias = this

override def withName(newName: String) = this
override def withName(newName: String): MultiAlias = this

override def eval(input: Row = null): EvaluatedType =
throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}")
Expand All @@ -179,17 +181,17 @@ case class MultiAlias(child: Expression, names: Seq[String])
*/
case class ResolvedStar(expressions: Seq[NamedExpression]) extends Star {
override def expand(input: Seq[Attribute], resolver: Resolver): Seq[NamedExpression] = expressions
override def toString = expressions.mkString("ResolvedStar(", ", ", ")")
override def toString: String = expressions.mkString("ResolvedStar(", ", ", ")")
}

case class UnresolvedGetField(child: Expression, fieldName: String) extends UnaryExpression {
override def dataType = throw new UnresolvedException(this, "dataType")
override def foldable = throw new UnresolvedException(this, "foldable")
override def nullable = throw new UnresolvedException(this, "nullable")
override def dataType: DataType = throw new UnresolvedException(this, "dataType")
override def foldable: Boolean = throw new UnresolvedException(this, "foldable")
override def nullable: Boolean = throw new UnresolvedException(this, "nullable")
override lazy val resolved = false

override def eval(input: Row = null): EvaluatedType =
throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}")

override def toString = s"$child.$fieldName"
override def toString: String = s"$child.$fieldName"
}
Loading