aboutsummaryrefslogtreecommitdiff
path: root/sql
diff options
context:
space:
mode:
authorReynold Xin <rxin@databricks.com>2015-03-24 16:03:55 -0700
committerReynold Xin <rxin@databricks.com>2015-03-24 16:03:55 -0700
commit73348012d4ce6c9db85dfb48d51026efe5051c73 (patch)
tree72686e324e67d0bb395e3840d655cabb522eda6a /sql
parent7215aa745590a3eec9c1ff35d28194235a550db7 (diff)
downloadspark-73348012d4ce6c9db85dfb48d51026efe5051c73.tar.gz
spark-73348012d4ce6c9db85dfb48d51026efe5051c73.tar.bz2
spark-73348012d4ce6c9db85dfb48d51026efe5051c73.zip
[SPARK-6428][SQL] Added explicit types for all public methods in catalyst
I think after this PR, we can finally turn the rule on. There are still some smaller ones that need to be fixed, but those are easier. Author: Reynold Xin <rxin@databricks.com> Closes #5162 from rxin/catalyst-explicit-types and squashes the following commits: e7eac03 [Reynold Xin] [SPARK-6428][SQL] Added explicit types for all public methods in catalyst.
Diffstat (limited to 'sql')
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/AbstractSparkSQLParser.scala8
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala22
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala14
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala78
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala202
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala4
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala24
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala10
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala14
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala52
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala13
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala4
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala7
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala80
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala165
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala66
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala26
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala12
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala6
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala13
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala51
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala21
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala80
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala18
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala35
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala37
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala14
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala6
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala46
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala11
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala24
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala27
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala4
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala6
40 files changed, 626 insertions, 586 deletions
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/AbstractSparkSQLParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/AbstractSparkSQLParser.scala
index 366be00473..3823584287 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/AbstractSparkSQLParser.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/AbstractSparkSQLParser.scala
@@ -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
@@ -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
}
@@ -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 */
@@ -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
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 13d2ae4c6f..44eceb0b37 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -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 {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
index 9e6e2912e0..5eb7dff0ce 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
@@ -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()
}
@@ -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))
@@ -205,15 +205,15 @@ 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
}
@@ -221,11 +221,11 @@ object EmptyCatalog extends Catalog {
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
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
index 425e1e41cb..40472a1cbb 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
@@ -33,7 +33,7 @@ class CheckAnalysis {
*/
val extendedCheckRules: Seq[LogicalPlan => Unit] = Nil
- def failAnalysis(msg: String) = {
+ def failAnalysis(msg: String): Nothing = {
throw new AnalysisException(msg)
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
index 9f334f6d42..c43ea55899 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
@@ -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)
}
@@ -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)
}
@@ -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
}
/**
@@ -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)
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala
index a7d3a8ee7d..c61c395cb4 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala
@@ -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)
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
index ad5172c034..300e9ba187 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
@@ -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
@@ -38,9 +39,10 @@ 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
}
@@ -48,16 +50,16 @@ case class UnresolvedRelation(
* 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 =
@@ -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(",")})"
}
/**
@@ -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 =
@@ -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("") + "*"
}
/**
@@ -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}")
@@ -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"
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
index 7f5f617812..145f062dd6 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
@@ -61,60 +61,60 @@ package object dsl {
trait ImplicitOperators {
def expr: Expression
- def unary_- = UnaryMinus(expr)
- def unary_! = Not(expr)
- def unary_~ = BitwiseNot(expr)
-
- def + (other: Expression) = Add(expr, other)
- def - (other: Expression) = Subtract(expr, other)
- def * (other: Expression) = Multiply(expr, other)
- def / (other: Expression) = Divide(expr, other)
- def % (other: Expression) = Remainder(expr, other)
- def & (other: Expression) = BitwiseAnd(expr, other)
- def | (other: Expression) = BitwiseOr(expr, other)
- def ^ (other: Expression) = BitwiseXor(expr, other)
-
- def && (other: Expression) = And(expr, other)
- def || (other: Expression) = Or(expr, other)
-
- def < (other: Expression) = LessThan(expr, other)
- def <= (other: Expression) = LessThanOrEqual(expr, other)
- def > (other: Expression) = GreaterThan(expr, other)
- def >= (other: Expression) = GreaterThanOrEqual(expr, other)
- def === (other: Expression) = EqualTo(expr, other)
- def <=> (other: Expression) = EqualNullSafe(expr, other)
- def !== (other: Expression) = Not(EqualTo(expr, other))
-
- def in(list: Expression*) = In(expr, list)
-
- def like(other: Expression) = Like(expr, other)
- def rlike(other: Expression) = RLike(expr, other)
- def contains(other: Expression) = Contains(expr, other)
- def startsWith(other: Expression) = StartsWith(expr, other)
- def endsWith(other: Expression) = EndsWith(expr, other)
- def substr(pos: Expression, len: Expression = Literal(Int.MaxValue)) =
+ def unary_- : Expression= UnaryMinus(expr)
+ def unary_! : Predicate = Not(expr)
+ def unary_~ : Expression = BitwiseNot(expr)
+
+ def + (other: Expression): Expression = Add(expr, other)
+ def - (other: Expression): Expression = Subtract(expr, other)
+ def * (other: Expression): Expression = Multiply(expr, other)
+ def / (other: Expression): Expression = Divide(expr, other)
+ def % (other: Expression): Expression = Remainder(expr, other)
+ def & (other: Expression): Expression = BitwiseAnd(expr, other)
+ def | (other: Expression): Expression = BitwiseOr(expr, other)
+ def ^ (other: Expression): Expression = BitwiseXor(expr, other)
+
+ def && (other: Expression): Predicate = And(expr, other)
+ def || (other: Expression): Predicate = Or(expr, other)
+
+ def < (other: Expression): Predicate = LessThan(expr, other)
+ def <= (other: Expression): Predicate = LessThanOrEqual(expr, other)
+ def > (other: Expression): Predicate = GreaterThan(expr, other)
+ def >= (other: Expression): Predicate = GreaterThanOrEqual(expr, other)
+ def === (other: Expression): Predicate = EqualTo(expr, other)
+ def <=> (other: Expression): Predicate = EqualNullSafe(expr, other)
+ def !== (other: Expression): Predicate = Not(EqualTo(expr, other))
+
+ def in(list: Expression*): Expression = In(expr, list)
+
+ def like(other: Expression): Expression = Like(expr, other)
+ def rlike(other: Expression): Expression = RLike(expr, other)
+ def contains(other: Expression): Expression = Contains(expr, other)
+ def startsWith(other: Expression): Expression = StartsWith(expr, other)
+ def endsWith(other: Expression): Expression = EndsWith(expr, other)
+ def substr(pos: Expression, len: Expression = Literal(Int.MaxValue)): Expression =
Substring(expr, pos, len)
- def substring(pos: Expression, len: Expression = Literal(Int.MaxValue)) =
+ def substring(pos: Expression, len: Expression = Literal(Int.MaxValue)): Expression =
Substring(expr, pos, len)
- def isNull = IsNull(expr)
- def isNotNull = IsNotNull(expr)
+ def isNull: Predicate = IsNull(expr)
+ def isNotNull: Predicate = IsNotNull(expr)
- def getItem(ordinal: Expression) = GetItem(expr, ordinal)
- def getField(fieldName: String) = UnresolvedGetField(expr, fieldName)
+ def getItem(ordinal: Expression): Expression = GetItem(expr, ordinal)
+ def getField(fieldName: String): UnresolvedGetField = UnresolvedGetField(expr, fieldName)
- def cast(to: DataType) = Cast(expr, to)
+ def cast(to: DataType): Expression = Cast(expr, to)
- def asc = SortOrder(expr, Ascending)
- def desc = SortOrder(expr, Descending)
+ def asc: SortOrder = SortOrder(expr, Ascending)
+ def desc: SortOrder = SortOrder(expr, Descending)
- def as(alias: String) = Alias(expr, alias)()
- def as(alias: Symbol) = Alias(expr, alias.name)()
+ def as(alias: String): NamedExpression = Alias(expr, alias)()
+ def as(alias: Symbol): NamedExpression = Alias(expr, alias.name)()
}
trait ExpressionConversions {
implicit class DslExpression(e: Expression) extends ImplicitOperators {
- def expr = e
+ def expr: Expression = e
}
implicit def booleanToLiteral(b: Boolean): Literal = Literal(b)
@@ -144,94 +144,100 @@ package object dsl {
}
}
- def sum(e: Expression) = Sum(e)
- def sumDistinct(e: Expression) = SumDistinct(e)
- def count(e: Expression) = Count(e)
- def countDistinct(e: Expression*) = CountDistinct(e)
- def approxCountDistinct(e: Expression, rsd: Double = 0.05) = ApproxCountDistinct(e, rsd)
- def avg(e: Expression) = Average(e)
- def first(e: Expression) = First(e)
- def last(e: Expression) = Last(e)
- def min(e: Expression) = Min(e)
- def max(e: Expression) = Max(e)
- def upper(e: Expression) = Upper(e)
- def lower(e: Expression) = Lower(e)
- def sqrt(e: Expression) = Sqrt(e)
- def abs(e: Expression) = Abs(e)
-
- implicit class DslSymbol(sym: Symbol) extends ImplicitAttribute { def s = sym.name }
+ def sum(e: Expression): Expression = Sum(e)
+ def sumDistinct(e: Expression): Expression = SumDistinct(e)
+ def count(e: Expression): Expression = Count(e)
+ def countDistinct(e: Expression*): Expression = CountDistinct(e)
+ def approxCountDistinct(e: Expression, rsd: Double = 0.05): Expression =
+ ApproxCountDistinct(e, rsd)
+ def avg(e: Expression): Expression = Average(e)
+ def first(e: Expression): Expression = First(e)
+ def last(e: Expression): Expression = Last(e)
+ def min(e: Expression): Expression = Min(e)
+ def max(e: Expression): Expression = Max(e)
+ def upper(e: Expression): Expression = Upper(e)
+ def lower(e: Expression): Expression = Lower(e)
+ def sqrt(e: Expression): Expression = Sqrt(e)
+ def abs(e: Expression): Expression = Abs(e)
+
+ implicit class DslSymbol(sym: Symbol) extends ImplicitAttribute { def s: String = sym.name }
// TODO more implicit class for literal?
implicit class DslString(val s: String) extends ImplicitOperators {
override def expr: Expression = Literal(s)
- def attr = analysis.UnresolvedAttribute(s)
+ def attr: UnresolvedAttribute = analysis.UnresolvedAttribute(s)
}
abstract class ImplicitAttribute extends ImplicitOperators {
def s: String
- def expr = attr
- def attr = analysis.UnresolvedAttribute(s)
+ def expr: UnresolvedAttribute = attr
+ def attr: UnresolvedAttribute = analysis.UnresolvedAttribute(s)
/** Creates a new AttributeReference of type boolean */
- def boolean = AttributeReference(s, BooleanType, nullable = true)()
+ def boolean: AttributeReference = AttributeReference(s, BooleanType, nullable = true)()
/** Creates a new AttributeReference of type byte */
- def byte = AttributeReference(s, ByteType, nullable = true)()
+ def byte: AttributeReference = AttributeReference(s, ByteType, nullable = true)()
/** Creates a new AttributeReference of type short */
- def short = AttributeReference(s, ShortType, nullable = true)()
+ def short: AttributeReference = AttributeReference(s, ShortType, nullable = true)()
/** Creates a new AttributeReference of type int */
- def int = AttributeReference(s, IntegerType, nullable = true)()
+ def int: AttributeReference = AttributeReference(s, IntegerType, nullable = true)()
/** Creates a new AttributeReference of type long */
- def long = AttributeReference(s, LongType, nullable = true)()
+ def long: AttributeReference = AttributeReference(s, LongType, nullable = true)()
/** Creates a new AttributeReference of type float */
- def float = AttributeReference(s, FloatType, nullable = true)()
+ def float: AttributeReference = AttributeReference(s, FloatType, nullable = true)()
/** Creates a new AttributeReference of type double */
- def double = AttributeReference(s, DoubleType, nullable = true)()
+ def double: AttributeReference = AttributeReference(s, DoubleType, nullable = true)()
/** Creates a new AttributeReference of type string */
- def string = AttributeReference(s, StringType, nullable = true)()
+ def string: AttributeReference = AttributeReference(s, StringType, nullable = true)()
/** Creates a new AttributeReference of type date */
- def date = AttributeReference(s, DateType, nullable = true)()
+ def date: AttributeReference = AttributeReference(s, DateType, nullable = true)()
/** Creates a new AttributeReference of type decimal */
- def decimal = AttributeReference(s, DecimalType.Unlimited, nullable = true)()
+ def decimal: AttributeReference =
+ AttributeReference(s, DecimalType.Unlimited, nullable = true)()
/** Creates a new AttributeReference of type decimal */
- def decimal(precision: Int, scale: Int) =
+ def decimal(precision: Int, scale: Int): AttributeReference =
AttributeReference(s, DecimalType(precision, scale), nullable = true)()
/** Creates a new AttributeReference of type timestamp */
- def timestamp = AttributeReference(s, TimestampType, nullable = true)()
+ def timestamp: AttributeReference = AttributeReference(s, TimestampType, nullable = true)()
/** Creates a new AttributeReference of type binary */
- def binary = AttributeReference(s, BinaryType, nullable = true)()
+ def binary: AttributeReference = AttributeReference(s, BinaryType, nullable = true)()
/** Creates a new AttributeReference of type array */
- def array(dataType: DataType) = AttributeReference(s, ArrayType(dataType), nullable = true)()
+ def array(dataType: DataType): AttributeReference =
+ AttributeReference(s, ArrayType(dataType), nullable = true)()
/** Creates a new AttributeReference of type map */
def map(keyType: DataType, valueType: DataType): AttributeReference =
map(MapType(keyType, valueType))
- def map(mapType: MapType) = AttributeReference(s, mapType, nullable = true)()
+
+ def map(mapType: MapType): AttributeReference =
+ AttributeReference(s, mapType, nullable = true)()
/** Creates a new AttributeReference of type struct */
def struct(fields: StructField*): AttributeReference = struct(StructType(fields))
- def struct(structType: StructType) = AttributeReference(s, structType, nullable = true)()
+ def struct(structType: StructType): AttributeReference =
+ AttributeReference(s, structType, nullable = true)()
}
implicit class DslAttribute(a: AttributeReference) {
- def notNull = a.withNullability(false)
- def nullable = a.withNullability(true)
+ def notNull: AttributeReference = a.withNullability(false)
+ def nullable: AttributeReference = a.withNullability(true)
// Protobuf terminology
- def required = a.withNullability(false)
+ def required: AttributeReference = a.withNullability(false)
- def at(ordinal: Int) = BoundReference(ordinal, a.dataType, a.nullable)
+ def at(ordinal: Int): BoundReference = BoundReference(ordinal, a.dataType, a.nullable)
}
}
@@ -241,23 +247,23 @@ package object dsl {
abstract class LogicalPlanFunctions {
def logicalPlan: LogicalPlan
- def select(exprs: NamedExpression*) = Project(exprs, logicalPlan)
+ def select(exprs: NamedExpression*): LogicalPlan = Project(exprs, logicalPlan)
- def where(condition: Expression) = Filter(condition, logicalPlan)
+ def where(condition: Expression): LogicalPlan = Filter(condition, logicalPlan)
- def limit(limitExpr: Expression) = Limit(limitExpr, logicalPlan)
+ def limit(limitExpr: Expression): LogicalPlan = Limit(limitExpr, logicalPlan)
def join(
otherPlan: LogicalPlan,
joinType: JoinType = Inner,
- condition: Option[Expression] = None) =
+ condition: Option[Expression] = None): LogicalPlan =
Join(logicalPlan, otherPlan, joinType, condition)
- def orderBy(sortExprs: SortOrder*) = Sort(sortExprs, true, logicalPlan)
+ def orderBy(sortExprs: SortOrder*): LogicalPlan = Sort(sortExprs, true, logicalPlan)
- def sortBy(sortExprs: SortOrder*) = Sort(sortExprs, false, logicalPlan)
+ def sortBy(sortExprs: SortOrder*): LogicalPlan = Sort(sortExprs, false, logicalPlan)
- def groupBy(groupingExprs: Expression*)(aggregateExprs: Expression*) = {
+ def groupBy(groupingExprs: Expression*)(aggregateExprs: Expression*): LogicalPlan = {
val aliasedExprs = aggregateExprs.map {
case ne: NamedExpression => ne
case e => Alias(e, e.toString)()
@@ -265,27 +271,27 @@ package object dsl {
Aggregate(groupingExprs, aliasedExprs, logicalPlan)
}
- def subquery(alias: Symbol) = Subquery(alias.name, logicalPlan)
+ def subquery(alias: Symbol): LogicalPlan = Subquery(alias.name, logicalPlan)
- def unionAll(otherPlan: LogicalPlan) = Union(logicalPlan, otherPlan)
+ def unionAll(otherPlan: LogicalPlan): LogicalPlan = Union(logicalPlan, otherPlan)
- def sfilter[T1](arg1: Symbol)(udf: (T1) => Boolean) =
+ def sfilter[T1](arg1: Symbol)(udf: (T1) => Boolean): LogicalPlan =
Filter(ScalaUdf(udf, BooleanType, Seq(UnresolvedAttribute(arg1.name))), logicalPlan)
def sample(
fraction: Double,
withReplacement: Boolean = true,
- seed: Int = (math.random * 1000).toInt) =
+ seed: Int = (math.random * 1000).toInt): LogicalPlan =
Sample(fraction, withReplacement, seed, logicalPlan)
def generate(
generator: Generator,
join: Boolean = false,
outer: Boolean = false,
- alias: Option[String] = None) =
+ alias: Option[String] = None): LogicalPlan =
Generate(generator, join, outer, None, logicalPlan)
- def insertInto(tableName: String, overwrite: Boolean = false) =
+ def insertInto(tableName: String, overwrite: Boolean = false): LogicalPlan =
InsertIntoTable(
analysis.UnresolvedRelation(Seq(tableName)), Map.empty, logicalPlan, overwrite)
@@ -294,12 +300,14 @@ package object dsl {
object plans { // scalastyle:ignore
implicit class DslLogicalPlan(val logicalPlan: LogicalPlan) extends LogicalPlanFunctions {
- def writeToFile(path: String) = WriteToFile(path, logicalPlan)
+ def writeToFile(path: String): LogicalPlan = WriteToFile(path, logicalPlan)
}
}
case class ScalaUdfBuilder[T: TypeTag](f: AnyRef) {
- def call(args: Expression*) = ScalaUdf(f, ScalaReflection.schemaFor(typeTag[T]).dataType, args)
+ def call(args: Expression*): ScalaUdf = {
+ ScalaUdf(f, ScalaReflection.schemaFor(typeTag[T]).dataType, args)
+ }
}
// scalastyle:off
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala
index 82e760b6c6..96a11e352e 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala
@@ -23,7 +23,9 @@ package org.apache.spark.sql.catalyst.expressions
* of the name, or the expected nullability).
*/
object AttributeMap {
- def apply[A](kvs: Seq[(Attribute, A)]) = new AttributeMap(kvs.map(kv => (kv._1.exprId, kv)).toMap)
+ def apply[A](kvs: Seq[(Attribute, A)]): AttributeMap[A] = {
+ new AttributeMap(kvs.map(kv => (kv._1.exprId, kv)).toMap)
+ }
}
class AttributeMap[A](baseMap: Map[ExprId, (Attribute, A)])
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala
index adaeab0b5c..f9ae85a5cf 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala
@@ -19,27 +19,27 @@ package org.apache.spark.sql.catalyst.expressions
protected class AttributeEquals(val a: Attribute) {
- override def hashCode() = a match {
+ override def hashCode(): Int = a match {
case ar: AttributeReference => ar.exprId.hashCode()
case a => a.hashCode()
}
- override def equals(other: Any) = (a, other.asInstanceOf[AttributeEquals].a) match {
+ override def equals(other: Any): Boolean = (a, other.asInstanceOf[AttributeEquals].a) match {
case (a1: AttributeReference, a2: AttributeReference) => a1.exprId == a2.exprId
case (a1, a2) => a1 == a2
}
}
object AttributeSet {
- def apply(a: Attribute) =
- new AttributeSet(Set(new AttributeEquals(a)))
+ def apply(a: Attribute): AttributeSet = new AttributeSet(Set(new AttributeEquals(a)))
/** Constructs a new [[AttributeSet]] given a sequence of [[Expression Expressions]]. */
- def apply(baseSet: Seq[Expression]) =
+ def apply(baseSet: Seq[Expression]): AttributeSet = {
new AttributeSet(
baseSet
.flatMap(_.references)
.map(new AttributeEquals(_)).toSet)
+ }
}
/**
@@ -57,7 +57,7 @@ class AttributeSet private (val baseSet: Set[AttributeEquals])
extends Traversable[Attribute] with Serializable {
/** Returns true if the members of this AttributeSet and other are the same. */
- override def equals(other: Any) = other match {
+ override def equals(other: Any): Boolean = other match {
case otherSet: AttributeSet => baseSet.map(_.a).forall(otherSet.contains)
case _ => false
}
@@ -81,32 +81,34 @@ class AttributeSet private (val baseSet: Set[AttributeEquals])
* Returns true if the [[Attribute Attributes]] in this set are a subset of the Attributes in
* `other`.
*/
- def subsetOf(other: AttributeSet) = baseSet.subsetOf(other.baseSet)
+ def subsetOf(other: AttributeSet): Boolean = baseSet.subsetOf(other.baseSet)
/**
* Returns a new [[AttributeSet]] that does not contain any of the [[Attribute Attributes]] found
* in `other`.
*/
- def --(other: Traversable[NamedExpression]) =
+ def --(other: Traversable[NamedExpression]): AttributeSet =
new AttributeSet(baseSet -- other.map(a => new AttributeEquals(a.toAttribute)))
/**
* Returns a new [[AttributeSet]] that contains all of the [[Attribute Attributes]] found
* in `other`.
*/
- def ++(other: AttributeSet) = new AttributeSet(baseSet ++ other.baseSet)
+ def ++(other: AttributeSet): AttributeSet = new AttributeSet(baseSet ++ other.baseSet)
/**
* Returns a new [[AttributeSet]] contain only the [[Attribute Attributes]] where `f` evaluates to
* true.
*/
- override def filter(f: Attribute => Boolean) = new AttributeSet(baseSet.filter(ae => f(ae.a)))
+ override def filter(f: Attribute => Boolean): AttributeSet =
+ new AttributeSet(baseSet.filter(ae => f(ae.a)))
/**
* Returns a new [[AttributeSet]] that only contains [[Attribute Attributes]] that are found in
* `this` and `other`.
*/
- def intersect(other: AttributeSet) = new AttributeSet(baseSet.intersect(other.baseSet))
+ def intersect(other: AttributeSet): AttributeSet =
+ new AttributeSet(baseSet.intersect(other.baseSet))
override def foreach[U](f: (Attribute) => U): Unit = baseSet.map(_.a).foreach(f)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala
index 76a9f08dea..2225621dba 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala
@@ -32,7 +32,7 @@ case class BoundReference(ordinal: Int, dataType: DataType, nullable: Boolean)
type EvaluatedType = Any
- override def toString = s"input[$ordinal]"
+ override def toString: String = s"input[$ordinal]"
override def eval(input: Row): Any = input(ordinal)
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
index b1bc858478..9bde74ac22 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
@@ -29,9 +29,9 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w
override lazy val resolved = childrenResolved && resolve(child.dataType, dataType)
- override def foldable = child.foldable
+ override def foldable: Boolean = child.foldable
- override def nullable = forceNullable(child.dataType, dataType) || child.nullable
+ override def nullable: Boolean = forceNullable(child.dataType, dataType) || child.nullable
private[this] def forceNullable(from: DataType, to: DataType) = (from, to) match {
case (StringType, _: NumericType) => true
@@ -103,7 +103,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w
}
}
- override def toString = s"CAST($child, $dataType)"
+ override def toString: String = s"CAST($child, $dataType)"
type EvaluatedType = Any
@@ -430,14 +430,14 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w
object Cast {
// `SimpleDateFormat` is not thread-safe.
private[sql] val threadLocalTimestampFormat = new ThreadLocal[DateFormat] {
- override def initialValue() = {
+ override def initialValue(): SimpleDateFormat = {
new SimpleDateFormat("yyyy-MM-dd HH:mm:ss")
}
}
// `SimpleDateFormat` is not thread-safe.
private[sql] val threadLocalDateFormat = new ThreadLocal[DateFormat] {
- override def initialValue() = {
+ override def initialValue(): SimpleDateFormat = {
new SimpleDateFormat("yyyy-MM-dd")
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
index 6ad39b8372..4e3bbc06a5 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
@@ -65,7 +65,7 @@ abstract class Expression extends TreeNode[Expression] {
* Returns true if all the children of this expression have been resolved to a specific schema
* and false if any still contains any unresolved placeholders.
*/
- def childrenResolved = !children.exists(!_.resolved)
+ def childrenResolved: Boolean = !children.exists(!_.resolved)
/**
* Returns a string representation of this expression that does not have developer centric
@@ -84,9 +84,9 @@ abstract class BinaryExpression extends Expression with trees.BinaryNode[Express
def symbol: String
- override def foldable = left.foldable && right.foldable
+ override def foldable: Boolean = left.foldable && right.foldable
- override def toString = s"($left $symbol $right)"
+ override def toString: String = s"($left $symbol $right)"
}
abstract class LeafExpression extends Expression with trees.LeafNode[Expression] {
@@ -104,8 +104,8 @@ abstract class UnaryExpression extends Expression with trees.UnaryNode[Expressio
case class GroupExpression(children: Seq[Expression]) extends Expression {
self: Product =>
type EvaluatedType = Seq[Any]
- override def eval(input: Row): EvaluatedType = ???
- override def nullable = false
- override def foldable = false
- override def dataType = ???
+ override def eval(input: Row): EvaluatedType = throw new UnsupportedOperationException
+ override def nullable: Boolean = false
+ override def foldable: Boolean = false
+ override def dataType: DataType = throw new UnsupportedOperationException
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala
index db5d897ee5..c2866cd955 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala
@@ -40,7 +40,7 @@ class InterpretedProjection(expressions: Seq[Expression]) extends Projection {
new GenericRow(outputArray)
}
- override def toString = s"Row => [${exprArray.mkString(",")}]"
+ override def toString: String = s"Row => [${exprArray.mkString(",")}]"
}
/**
@@ -107,12 +107,12 @@ class JoinedRow extends Row {
override def toSeq: Seq[Any] = row1.toSeq ++ row2.toSeq
- override def length = row1.length + row2.length
+ override def length: Int = row1.length + row2.length
- override def apply(i: Int) =
+ override def apply(i: Int): Any =
if (i < row1.length) row1(i) else row2(i - row1.length)
- override def isNullAt(i: Int) =
+ override def isNullAt(i: Int): Boolean =
if (i < row1.length) row1.isNullAt(i) else row2.isNullAt(i - row1.length)
override def getInt(i: Int): Int =
@@ -142,7 +142,7 @@ class JoinedRow extends Row {
override def getAs[T](i: Int): T =
if (i < row1.length) row1.getAs[T](i) else row2.getAs[T](i - row1.length)
- override def copy() = {
+ override def copy(): Row = {
val totalSize = row1.length + row2.length
val copiedValues = new Array[Any](totalSize)
var i = 0
@@ -153,7 +153,7 @@ class JoinedRow extends Row {
new GenericRow(copiedValues)
}
- override def toString() = {
+ override def toString: String = {
// Make sure toString never throws NullPointerException.
if ((row1 eq null) && (row2 eq null)) {
"[ empty row ]"
@@ -207,12 +207,12 @@ class JoinedRow2 extends Row {
override def toSeq: Seq[Any] = row1.toSeq ++ row2.toSeq
- override def length = row1.length + row2.length
+ override def length: Int = row1.length + row2.length
- override def apply(i: Int) =
+ override def apply(i: Int): Any =
if (i < row1.length) row1(i) else row2(i - row1.length)
- override def isNullAt(i: Int) =
+ override def isNullAt(i: Int): Boolean =
if (i < row1.length) row1.isNullAt(i) else row2.isNullAt(i - row1.length)
override def getInt(i: Int): Int =
@@ -242,7 +242,7 @@ class JoinedRow2 extends Row {
override def getAs[T](i: Int): T =
if (i < row1.length) row1.getAs[T](i) else row2.getAs[T](i - row1.length)
- override def copy() = {
+ override def copy(): Row = {
val totalSize = row1.length + row2.length
val copiedValues = new Array[Any](totalSize)
var i = 0
@@ -253,7 +253,7 @@ class JoinedRow2 extends Row {
new GenericRow(copiedValues)
}
- override def toString() = {
+ override def toString: String = {
// Make sure toString never throws NullPointerException.
if ((row1 eq null) && (row2 eq null)) {
"[ empty row ]"
@@ -301,12 +301,12 @@ class JoinedRow3 extends Row {
override def toSeq: Seq[Any] = row1.toSeq ++ row2.toSeq
- override def length = row1.length + row2.length
+ override def length: Int = row1.length + row2.length
- override def apply(i: Int) =
+ override def apply(i: Int): Any =
if (i < row1.length) row1(i) else row2(i - row1.length)
- override def isNullAt(i: Int) =
+ override def isNullAt(i: Int): Boolean =
if (i < row1.length) row1.isNullAt(i) else row2.isNullAt(i - row1.length)
override def getInt(i: Int): Int =
@@ -336,7 +336,7 @@ class JoinedRow3 extends Row {
override def getAs[T](i: Int): T =
if (i < row1.length) row1.getAs[T](i) else row2.getAs[T](i - row1.length)
- override def copy() = {
+ override def copy(): Row = {
val totalSize = row1.length + row2.length
val copiedValues = new Array[Any](totalSize)
var i = 0
@@ -347,7 +347,7 @@ class JoinedRow3 extends Row {
new GenericRow(copiedValues)
}
- override def toString() = {
+ override def toString: String = {
// Make sure toString never throws NullPointerException.
if ((row1 eq null) && (row2 eq null)) {
"[ empty row ]"
@@ -395,12 +395,12 @@ class JoinedRow4 extends Row {
override def toSeq: Seq[Any] = row1.toSeq ++ row2.toSeq
- override def length = row1.length + row2.length
+ override def length: Int = row1.length + row2.length
- override def apply(i: Int) =
+ override def apply(i: Int): Any =
if (i < row1.length) row1(i) else row2(i - row1.length)
- override def isNullAt(i: Int) =
+ override def isNullAt(i: Int): Boolean =
if (i < row1.length) row1.isNullAt(i) else row2.isNullAt(i - row1.length)
override def getInt(i: Int): Int =
@@ -430,7 +430,7 @@ class JoinedRow4 extends Row {
override def getAs[T](i: Int): T =
if (i < row1.length) row1.getAs[T](i) else row2.getAs[T](i - row1.length)
- override def copy() = {
+ override def copy(): Row = {
val totalSize = row1.length + row2.length
val copiedValues = new Array[Any](totalSize)
var i = 0
@@ -441,7 +441,7 @@ class JoinedRow4 extends Row {
new GenericRow(copiedValues)
}
- override def toString() = {
+ override def toString: String = {
// Make sure toString never throws NullPointerException.
if ((row1 eq null) && (row2 eq null)) {
"[ empty row ]"
@@ -489,12 +489,12 @@ class JoinedRow5 extends Row {
override def toSeq: Seq[Any] = row1.toSeq ++ row2.toSeq
- override def length = row1.length + row2.length
+ override def length: Int = row1.length + row2.length
- override def apply(i: Int) =
+ override def apply(i: Int): Any =
if (i < row1.length) row1(i) else row2(i - row1.length)
- override def isNullAt(i: Int) =
+ override def isNullAt(i: Int): Boolean =
if (i < row1.length) row1.isNullAt(i) else row2.isNullAt(i - row1.length)
override def getInt(i: Int): Int =
@@ -524,7 +524,7 @@ class JoinedRow5 extends Row {
override def getAs[T](i: Int): T =
if (i < row1.length) row1.getAs[T](i) else row2.getAs[T](i - row1.length)
- override def copy() = {
+ override def copy(): Row = {
val totalSize = row1.length + row2.length
val copiedValues = new Array[Any](totalSize)
var i = 0
@@ -535,7 +535,7 @@ class JoinedRow5 extends Row {
new GenericRow(copiedValues)
}
- override def toString() = {
+ override def toString: String = {
// Make sure toString never throws NullPointerException.
if ((row1 eq null) && (row2 eq null)) {
"[ empty row ]"
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala
index b2c6d30290..f5fea3f015 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala
@@ -18,16 +18,19 @@
package org.apache.spark.sql.catalyst.expressions
import java.util.Random
-import org.apache.spark.sql.types.DoubleType
+
+import org.apache.spark.sql.types.{DataType, DoubleType}
case object Rand extends LeafExpression {
- override def dataType = DoubleType
- override def nullable = false
+ override def dataType: DataType = DoubleType
+ override def nullable: Boolean = false
private[this] lazy val rand = new Random
- override def eval(input: Row = null) = rand.nextDouble().asInstanceOf[EvaluatedType]
+ override def eval(input: Row = null): EvaluatedType = {
+ rand.nextDouble().asInstanceOf[EvaluatedType]
+ }
- override def toString = "RAND()"
+ override def toString: String = "RAND()"
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala
index 8a36c68107..1fd5ce342b 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala
@@ -29,9 +29,9 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi
type EvaluatedType = Any
- def nullable = true
+ override def nullable: Boolean = true
- override def toString = s"scalaUDF(${children.mkString(",")})"
+ override def toString: String = s"scalaUDF(${children.mkString(",")})"
// scalastyle:off
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala
index d00b2ac097..83074eb1e6 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala
@@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.errors.TreeNodeException
import org.apache.spark.sql.catalyst.trees
+import org.apache.spark.sql.types.DataType
abstract sealed class SortDirection
case object Ascending extends SortDirection
@@ -31,12 +32,12 @@ case object Descending extends SortDirection
case class SortOrder(child: Expression, direction: SortDirection) extends Expression
with trees.UnaryNode[Expression] {
- override def dataType = child.dataType
- override def nullable = child.nullable
+ override def dataType: DataType = child.dataType
+ override def nullable: Boolean = child.nullable
// SortOrder itself is never evaluated.
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 ${if (direction == Ascending) "ASC" else "DESC"}"
+ override def toString: String = s"$child ${if (direction == Ascending) "ASC" else "DESC"}"
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala
index 21d714c9a8..47b6f358ed 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala
@@ -62,126 +62,126 @@ abstract class MutableValue extends Serializable {
var isNull: Boolean = true
def boxed: Any
def update(v: Any)
- def copy(): this.type
+ def copy(): MutableValue
}
final class MutableInt extends MutableValue {
var value: Int = 0
- def boxed = if (isNull) null else value
- def update(v: Any) = value = {
+ override def boxed: Any = if (isNull) null else value
+ override def update(v: Any): Unit = {
isNull = false
- v.asInstanceOf[Int]
+ value = v.asInstanceOf[Int]
}
- def copy() = {
+ override def copy(): MutableInt = {
val newCopy = new MutableInt
newCopy.isNull = isNull
newCopy.value = value
- newCopy.asInstanceOf[this.type]
+ newCopy.asInstanceOf[MutableInt]
}
}
final class MutableFloat extends MutableValue {
var value: Float = 0
- def boxed = if (isNull) null else value
- def update(v: Any) = value = {
+ override def boxed: Any = if (isNull) null else value
+ override def update(v: Any): Unit = {
isNull = false
- v.asInstanceOf[Float]
+ value = v.asInstanceOf[Float]
}
- def copy() = {
+ override def copy(): MutableFloat = {
val newCopy = new MutableFloat
newCopy.isNull = isNull
newCopy.value = value
- newCopy.asInstanceOf[this.type]
+ newCopy.asInstanceOf[MutableFloat]
}
}
final class MutableBoolean extends MutableValue {
var value: Boolean = false
- def boxed = if (isNull) null else value
- def update(v: Any) = value = {
+ override def boxed: Any = if (isNull) null else value
+ override def update(v: Any): Unit = {
isNull = false
- v.asInstanceOf[Boolean]
+ value = v.asInstanceOf[Boolean]
}
- def copy() = {
+ override def copy(): MutableBoolean = {
val newCopy = new MutableBoolean
newCopy.isNull = isNull
newCopy.value = value
- newCopy.asInstanceOf[this.type]
+ newCopy.asInstanceOf[MutableBoolean]
}
}
final class MutableDouble extends MutableValue {
var value: Double = 0
- def boxed = if (isNull) null else value
- def update(v: Any) = value = {
+ override def boxed: Any = if (isNull) null else value
+ override def update(v: Any): Unit = {
isNull = false
- v.asInstanceOf[Double]
+ value = v.asInstanceOf[Double]
}
- def copy() = {
+ override def copy(): MutableDouble = {
val newCopy = new MutableDouble
newCopy.isNull = isNull
newCopy.value = value
- newCopy.asInstanceOf[this.type]
+ newCopy.asInstanceOf[MutableDouble]
}
}
final class MutableShort extends MutableValue {
var value: Short = 0
- def boxed = if (isNull) null else value
- def update(v: Any) = value = {
+ override def boxed: Any = if (isNull) null else value
+ override def update(v: Any): Unit = value = {
isNull = false
v.asInstanceOf[Short]
}
- def copy() = {
+ override def copy(): MutableShort = {
val newCopy = new MutableShort
newCopy.isNull = isNull
newCopy.value = value
- newCopy.asInstanceOf[this.type]
+ newCopy.asInstanceOf[MutableShort]
}
}
final class MutableLong extends MutableValue {
var value: Long = 0
- def boxed = if (isNull) null else value
- def update(v: Any) = value = {
+ override def boxed: Any = if (isNull) null else value
+ override def update(v: Any): Unit = value = {
isNull = false
v.asInstanceOf[Long]
}
- def copy() = {
+ override def copy(): MutableLong = {
val newCopy = new MutableLong
newCopy.isNull = isNull
newCopy.value = value
- newCopy.asInstanceOf[this.type]
+ newCopy.asInstanceOf[MutableLong]
}
}
final class MutableByte extends MutableValue {
var value: Byte = 0
- def boxed = if (isNull) null else value
- def update(v: Any) = value = {
+ override def boxed: Any = if (isNull) null else value
+ override def update(v: Any): Unit = value = {
isNull = false
v.asInstanceOf[Byte]
}
- def copy() = {
+ override def copy(): MutableByte = {
val newCopy = new MutableByte
newCopy.isNull = isNull
newCopy.value = value
- newCopy.asInstanceOf[this.type]
+ newCopy.asInstanceOf[MutableByte]
}
}
final class MutableAny extends MutableValue {
var value: Any = _
- def boxed = if (isNull) null else value
- def update(v: Any) = value = {
+ override def boxed: Any = if (isNull) null else value
+ override def update(v: Any): Unit = {
isNull = false
- v.asInstanceOf[Any]
+ value = v.asInstanceOf[Any]
}
- def copy() = {
+ override def copy(): MutableAny = {
val newCopy = new MutableAny
newCopy.isNull = isNull
newCopy.value = value
- newCopy.asInstanceOf[this.type]
+ newCopy.asInstanceOf[MutableAny]
}
}
@@ -234,9 +234,9 @@ final class SpecificMutableRow(val values: Array[MutableValue]) extends MutableR
if (value == null) setNullAt(ordinal) else values(ordinal).update(value)
}
- override def setString(ordinal: Int, value: String) = update(ordinal, value)
+ override def setString(ordinal: Int, value: String): Unit = update(ordinal, value)
- override def getString(ordinal: Int) = apply(ordinal).asInstanceOf[String]
+ override def getString(ordinal: Int): String = apply(ordinal).asInstanceOf[String]
override def setInt(ordinal: Int, value: Int): Unit = {
val currentValue = values(ordinal).asInstanceOf[MutableInt]
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala
index 5297d1e312..30da4faa3f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala
@@ -79,27 +79,29 @@ abstract class AggregateFunction
/** Base should return the generic aggregate expression that this function is computing */
val base: AggregateExpression
- override def nullable = base.nullable
- override def dataType = base.dataType
+ override def nullable: Boolean = base.nullable
+ override def dataType: DataType = base.dataType
def update(input: Row): Unit
// Do we really need this?
- override def newInstance() = makeCopy(productIterator.map { case a: AnyRef => a }.toArray)
+ override def newInstance(): AggregateFunction = {
+ makeCopy(productIterator.map { case a: AnyRef => a }.toArray)
+ }
}
case class Min(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] {
- override def nullable = true
- override def dataType = child.dataType
- override def toString = s"MIN($child)"
+ override def nullable: Boolean = true
+ override def dataType: DataType = child.dataType
+ override def toString: String = s"MIN($child)"
override def asPartial: SplitEvaluation = {
val partialMin = Alias(Min(child), "PartialMin")()
SplitEvaluation(Min(partialMin.toAttribute), partialMin :: Nil)
}
- override def newInstance() = new MinFunction(child, this)
+ override def newInstance(): MinFunction = new MinFunction(child, this)
}
case class MinFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction {
@@ -121,16 +123,16 @@ case class MinFunction(expr: Expression, base: AggregateExpression) extends Aggr
case class Max(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] {
- override def nullable = true
- override def dataType = child.dataType
- override def toString = s"MAX($child)"
+ override def nullable: Boolean = true
+ override def dataType: DataType = child.dataType
+ override def toString: String = s"MAX($child)"
override def asPartial: SplitEvaluation = {
val partialMax = Alias(Max(child), "PartialMax")()
SplitEvaluation(Max(partialMax.toAttribute), partialMax :: Nil)
}
- override def newInstance() = new MaxFunction(child, this)
+ override def newInstance(): MaxFunction = new MaxFunction(child, this)
}
case class MaxFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction {
@@ -152,29 +154,29 @@ case class MaxFunction(expr: Expression, base: AggregateExpression) extends Aggr
case class Count(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] {
- override def nullable = false
- override def dataType = LongType
- override def toString = s"COUNT($child)"
+ override def nullable: Boolean = false
+ override def dataType: LongType.type = LongType
+ override def toString: String = s"COUNT($child)"
override def asPartial: SplitEvaluation = {
val partialCount = Alias(Count(child), "PartialCount")()
SplitEvaluation(Coalesce(Seq(Sum(partialCount.toAttribute), Literal(0L))), partialCount :: Nil)
}
- override def newInstance() = new CountFunction(child, this)
+ override def newInstance(): CountFunction = new CountFunction(child, this)
}
case class CountDistinct(expressions: Seq[Expression]) extends PartialAggregate {
def this() = this(null)
- override def children = expressions
+ override def children: Seq[Expression] = expressions
- override def nullable = false
- override def dataType = LongType
- override def toString = s"COUNT(DISTINCT ${expressions.mkString(",")})"
- override def newInstance() = new CountDistinctFunction(expressions, this)
+ override def nullable: Boolean = false
+ override def dataType: DataType = LongType
+ override def toString: String = s"COUNT(DISTINCT ${expressions.mkString(",")})"
+ override def newInstance(): CountDistinctFunction = new CountDistinctFunction(expressions, this)
- override def asPartial = {
+ override def asPartial: SplitEvaluation = {
val partialSet = Alias(CollectHashSet(expressions), "partialSets")()
SplitEvaluation(
CombineSetsAndCount(partialSet.toAttribute),
@@ -185,11 +187,11 @@ case class CountDistinct(expressions: Seq[Expression]) extends PartialAggregate
case class CollectHashSet(expressions: Seq[Expression]) extends AggregateExpression {
def this() = this(null)
- override def children = expressions
- override def nullable = false
- override def dataType = ArrayType(expressions.head.dataType)
- override def toString = s"AddToHashSet(${expressions.mkString(",")})"
- override def newInstance() = new CollectHashSetFunction(expressions, this)
+ override def children: Seq[Expression] = expressions
+ override def nullable: Boolean = false
+ override def dataType: ArrayType = ArrayType(expressions.head.dataType)
+ override def toString: String = s"AddToHashSet(${expressions.mkString(",")})"
+ override def newInstance(): CollectHashSetFunction = new CollectHashSetFunction(expressions, this)
}
case class CollectHashSetFunction(
@@ -219,11 +221,13 @@ case class CollectHashSetFunction(
case class CombineSetsAndCount(inputSet: Expression) extends AggregateExpression {
def this() = this(null)
- override def children = inputSet :: Nil
- override def nullable = false
- override def dataType = LongType
- override def toString = s"CombineAndCount($inputSet)"
- override def newInstance() = new CombineSetsAndCountFunction(inputSet, this)
+ override def children: Seq[Expression] = inputSet :: Nil
+ override def nullable: Boolean = false
+ override def dataType: DataType = LongType
+ override def toString: String = s"CombineAndCount($inputSet)"
+ override def newInstance(): CombineSetsAndCountFunction = {
+ new CombineSetsAndCountFunction(inputSet, this)
+ }
}
case class CombineSetsAndCountFunction(
@@ -249,27 +253,31 @@ case class CombineSetsAndCountFunction(
case class ApproxCountDistinctPartition(child: Expression, relativeSD: Double)
extends AggregateExpression with trees.UnaryNode[Expression] {
- override def nullable = false
- override def dataType = child.dataType
- override def toString = s"APPROXIMATE COUNT(DISTINCT $child)"
- override def newInstance() = new ApproxCountDistinctPartitionFunction(child, this, relativeSD)
+ override def nullable: Boolean = false
+ override def dataType: DataType = child.dataType
+ override def toString: String = s"APPROXIMATE COUNT(DISTINCT $child)"
+ override def newInstance(): ApproxCountDistinctPartitionFunction = {
+ new ApproxCountDistinctPartitionFunction(child, this, relativeSD)
+ }
}
case class ApproxCountDistinctMerge(child: Expression, relativeSD: Double)
extends AggregateExpression with trees.UnaryNode[Expression] {
- override def nullable = false
- override def dataType = LongType
- override def toString = s"APPROXIMATE COUNT(DISTINCT $child)"
- override def newInstance() = new ApproxCountDistinctMergeFunction(child, this, relativeSD)
+ override def nullable: Boolean = false
+ override def dataType: LongType.type = LongType
+ override def toString: String = s"APPROXIMATE COUNT(DISTINCT $child)"
+ override def newInstance(): ApproxCountDistinctMergeFunction = {
+ new ApproxCountDistinctMergeFunction(child, this, relativeSD)
+ }
}
case class ApproxCountDistinct(child: Expression, relativeSD: Double = 0.05)
extends PartialAggregate with trees.UnaryNode[Expression] {
- override def nullable = false
- override def dataType = LongType
- override def toString = s"APPROXIMATE COUNT(DISTINCT $child)"
+ override def nullable: Boolean = false
+ override def dataType: LongType.type = LongType
+ override def toString: String = s"APPROXIMATE COUNT(DISTINCT $child)"
override def asPartial: SplitEvaluation = {
val partialCount =
@@ -280,14 +288,14 @@ case class ApproxCountDistinct(child: Expression, relativeSD: Double = 0.05)
partialCount :: Nil)
}
- override def newInstance() = new CountDistinctFunction(child :: Nil, this)
+ override def newInstance(): CountDistinctFunction = new CountDistinctFunction(child :: Nil, this)
}
case class Average(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] {
- override def nullable = true
+ override def nullable: Boolean = true
- override def dataType = child.dataType match {
+ override def dataType: DataType = child.dataType match {
case DecimalType.Fixed(precision, scale) =>
DecimalType(precision + 4, scale + 4) // Add 4 digits after decimal point, like Hive
case DecimalType.Unlimited =>
@@ -296,7 +304,7 @@ case class Average(child: Expression) extends PartialAggregate with trees.UnaryN
DoubleType
}
- override def toString = s"AVG($child)"
+ override def toString: String = s"AVG($child)"
override def asPartial: SplitEvaluation = {
child.dataType match {
@@ -323,14 +331,14 @@ case class Average(child: Expression) extends PartialAggregate with trees.UnaryN
}
}
- override def newInstance() = new AverageFunction(child, this)
+ override def newInstance(): AverageFunction = new AverageFunction(child, this)
}
case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] {
- override def nullable = true
+ override def nullable: Boolean = true
- override def dataType = child.dataType match {
+ override def dataType: DataType = child.dataType match {
case DecimalType.Fixed(precision, scale) =>
DecimalType(precision + 10, scale) // Add 10 digits left of decimal point, like Hive
case DecimalType.Unlimited =>
@@ -339,7 +347,7 @@ case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[
child.dataType
}
- override def toString = s"SUM($child)"
+ override def toString: String = s"SUM($child)"
override def asPartial: SplitEvaluation = {
child.dataType match {
@@ -357,7 +365,7 @@ case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[
}
}
- override def newInstance() = new SumFunction(child, this)
+ override def newInstance(): SumFunction = new SumFunction(child, this)
}
/**
@@ -377,19 +385,19 @@ case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[
case class CombineSum(child: Expression) extends AggregateExpression {
def this() = this(null)
- override def children = child :: Nil
- override def nullable = true
- override def dataType = child.dataType
- override def toString = s"CombineSum($child)"
- override def newInstance() = new CombineSumFunction(child, this)
+ override def children: Seq[Expression] = child :: Nil
+ override def nullable: Boolean = true
+ override def dataType: DataType = child.dataType
+ override def toString: String = s"CombineSum($child)"
+ override def newInstance(): CombineSumFunction = new CombineSumFunction(child, this)
}
case class SumDistinct(child: Expression)
extends PartialAggregate with trees.UnaryNode[Expression] {
def this() = this(null)
- override def nullable = true
- override def dataType = child.dataType match {
+ override def nullable: Boolean = true
+ override def dataType: DataType = child.dataType match {
case DecimalType.Fixed(precision, scale) =>
DecimalType(precision + 10, scale) // Add 10 digits left of decimal point, like Hive
case DecimalType.Unlimited =>
@@ -397,10 +405,10 @@ case class SumDistinct(child: Expression)
case _ =>
child.dataType
}
- override def toString = s"SUM(DISTINCT ${child})"
- override def newInstance() = new SumDistinctFunction(child, this)
+ override def toString: String = s"SUM(DISTINCT $child)"
+ override def newInstance(): SumDistinctFunction = new SumDistinctFunction(child, this)
- override def asPartial = {
+ override def asPartial: SplitEvaluation = {
val partialSet = Alias(CollectHashSet(child :: Nil), "partialSets")()
SplitEvaluation(
CombineSetsAndSum(partialSet.toAttribute, this),
@@ -411,11 +419,13 @@ case class SumDistinct(child: Expression)
case class CombineSetsAndSum(inputSet: Expression, base: Expression) extends AggregateExpression {
def this() = this(null, null)
- override def children = inputSet :: Nil
- override def nullable = true
- override def dataType = base.dataType
- override def toString = s"CombineAndSum($inputSet)"
- override def newInstance() = new CombineSetsAndSumFunction(inputSet, this)
+ override def children: Seq[Expression] = inputSet :: Nil
+ override def nullable: Boolean = true
+ override def dataType: DataType = base.dataType
+ override def toString: String = s"CombineAndSum($inputSet)"
+ override def newInstance(): CombineSetsAndSumFunction = {
+ new CombineSetsAndSumFunction(inputSet, this)
+ }
}
case class CombineSetsAndSumFunction(
@@ -449,9 +459,9 @@ case class CombineSetsAndSumFunction(
}
case class First(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] {
- override def nullable = true
- override def dataType = child.dataType
- override def toString = s"FIRST($child)"
+ override def nullable: Boolean = true
+ override def dataType: DataType = child.dataType
+ override def toString: String = s"FIRST($child)"
override def asPartial: SplitEvaluation = {
val partialFirst = Alias(First(child), "PartialFirst")()
@@ -459,14 +469,14 @@ case class First(child: Expression) extends PartialAggregate with trees.UnaryNod
First(partialFirst.toAttribute),
partialFirst :: Nil)
}
- override def newInstance() = new FirstFunction(child, this)
+ override def newInstance(): FirstFunction = new FirstFunction(child, this)
}
case class Last(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] {
- override def references = child.references
- override def nullable = true
- override def dataType = child.dataType
- override def toString = s"LAST($child)"
+ override def references: AttributeSet = child.references
+ override def nullable: Boolean = true
+ override def dataType: DataType = child.dataType
+ override def toString: String = s"LAST($child)"
override def asPartial: SplitEvaluation = {
val partialLast = Alias(Last(child), "PartialLast")()
@@ -474,7 +484,7 @@ case class Last(child: Expression) extends PartialAggregate with trees.UnaryNode
Last(partialLast.toAttribute),
partialLast :: Nil)
}
- override def newInstance() = new LastFunction(child, this)
+ override def newInstance(): LastFunction = new LastFunction(child, this)
}
case class AverageFunction(expr: Expression, base: AggregateExpression)
@@ -713,6 +723,7 @@ case class LastFunction(expr: Expression, base: AggregateExpression) extends Agg
result = input
}
- override def eval(input: Row): Any = if (result != null) expr.eval(result.asInstanceOf[Row])
- else null
+ override def eval(input: Row): Any = {
+ if (result != null) expr.eval(result.asInstanceOf[Row]) else null
+ }
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
index 00b0d3c683..1f6526ef66 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
@@ -24,10 +24,10 @@ import org.apache.spark.sql.types._
case class UnaryMinus(child: Expression) extends UnaryExpression {
type EvaluatedType = Any
- def dataType = child.dataType
- override def foldable = child.foldable
- def nullable = child.nullable
- override def toString = s"-$child"
+ override def dataType: DataType = child.dataType
+ override def foldable: Boolean = child.foldable
+ override def nullable: Boolean = child.nullable
+ override def toString: String = s"-$child"
lazy val numeric = dataType match {
case n: NumericType => n.numeric.asInstanceOf[Numeric[Any]]
@@ -47,10 +47,10 @@ case class UnaryMinus(child: Expression) extends UnaryExpression {
case class Sqrt(child: Expression) extends UnaryExpression {
type EvaluatedType = Any
- def dataType = DoubleType
- override def foldable = child.foldable
- def nullable = true
- override def toString = s"SQRT($child)"
+ override def dataType: DataType = DoubleType
+ override def foldable: Boolean = child.foldable
+ override def nullable: Boolean = true
+ override def toString: String = s"SQRT($child)"
lazy val numeric = child.dataType match {
case n: NumericType => n.numeric.asInstanceOf[Numeric[Any]]
@@ -74,14 +74,14 @@ abstract class BinaryArithmetic extends BinaryExpression {
type EvaluatedType = Any
- def nullable = left.nullable || right.nullable
+ def nullable: Boolean = left.nullable || right.nullable
override lazy val resolved =
left.resolved && right.resolved &&
left.dataType == right.dataType &&
!DecimalType.isFixed(left.dataType)
- def dataType = {
+ def dataType: DataType = {
if (!resolved) {
throw new UnresolvedException(this,
s"datatype. Can not resolve due to differing types ${left.dataType}, ${right.dataType}")
@@ -108,7 +108,7 @@ abstract class BinaryArithmetic extends BinaryExpression {
}
case class Add(left: Expression, right: Expression) extends BinaryArithmetic {
- def symbol = "+"
+ override def symbol: String = "+"
lazy val numeric = dataType match {
case n: NumericType => n.numeric.asInstanceOf[Numeric[Any]]
@@ -131,7 +131,7 @@ case class Add(left: Expression, right: Expression) extends BinaryArithmetic {
}
case class Subtract(left: Expression, right: Expression) extends BinaryArithmetic {
- def symbol = "-"
+ override def symbol: String = "-"
lazy val numeric = dataType match {
case n: NumericType => n.numeric.asInstanceOf[Numeric[Any]]
@@ -154,7 +154,7 @@ case class Subtract(left: Expression, right: Expression) extends BinaryArithmeti
}
case class Multiply(left: Expression, right: Expression) extends BinaryArithmetic {
- def symbol = "*"
+ override def symbol: String = "*"
lazy val numeric = dataType match {
case n: NumericType => n.numeric.asInstanceOf[Numeric[Any]]
@@ -177,9 +177,9 @@ case class Multiply(left: Expression, right: Expression) extends BinaryArithmeti
}
case class Divide(left: Expression, right: Expression) extends BinaryArithmetic {
- def symbol = "/"
+ override def symbol: String = "/"
- override def nullable = true
+ override def nullable: Boolean = true
lazy val div: (Any, Any) => Any = dataType match {
case ft: FractionalType => ft.fractional.asInstanceOf[Fractional[Any]].div
@@ -203,9 +203,9 @@ case class Divide(left: Expression, right: Expression) extends BinaryArithmetic
}
case class Remainder(left: Expression, right: Expression) extends BinaryArithmetic {
- def symbol = "%"
+ override def symbol: String = "%"
- override def nullable = true
+ override def nullable: Boolean = true
lazy val integral = dataType match {
case i: IntegralType => i.integral.asInstanceOf[Integral[Any]]
@@ -232,7 +232,7 @@ case class Remainder(left: Expression, right: Expression) extends BinaryArithmet
* A function that calculates bitwise and(&) of two numbers.
*/
case class BitwiseAnd(left: Expression, right: Expression) extends BinaryArithmetic {
- def symbol = "&"
+ override def symbol: String = "&"
lazy val and: (Any, Any) => Any = dataType match {
case ByteType =>
@@ -253,7 +253,7 @@ case class BitwiseAnd(left: Expression, right: Expression) extends BinaryArithme
* A function that calculates bitwise or(|) of two numbers.
*/
case class BitwiseOr(left: Expression, right: Expression) extends BinaryArithmetic {
- def symbol = "|"
+ override def symbol: String = "|"
lazy val or: (Any, Any) => Any = dataType match {
case ByteType =>
@@ -274,7 +274,7 @@ case class BitwiseOr(left: Expression, right: Expression) extends BinaryArithmet
* A function that calculates bitwise xor(^) of two numbers.
*/
case class BitwiseXor(left: Expression, right: Expression) extends BinaryArithmetic {
- def symbol = "^"
+ override def symbol: String = "^"
lazy val xor: (Any, Any) => Any = dataType match {
case ByteType =>
@@ -297,10 +297,10 @@ case class BitwiseXor(left: Expression, right: Expression) extends BinaryArithme
case class BitwiseNot(child: Expression) extends UnaryExpression {
type EvaluatedType = Any
- def dataType = child.dataType
- override def foldable = child.foldable
- def nullable = child.nullable
- override def toString = s"~$child"
+ override def dataType: DataType = child.dataType
+ override def foldable: Boolean = child.foldable
+ override def nullable: Boolean = child.nullable
+ override def toString: String = s"~$child"
lazy val not: (Any) => Any = dataType match {
case ByteType =>
@@ -327,17 +327,17 @@ case class BitwiseNot(child: Expression) extends UnaryExpression {
case class MaxOf(left: Expression, right: Expression) extends Expression {
type EvaluatedType = Any
- override def foldable = left.foldable && right.foldable
+ override def foldable: Boolean = left.foldable && right.foldable
- override def nullable = left.nullable && right.nullable
+ override def nullable: Boolean = left.nullable && right.nullable
- override def children = left :: right :: Nil
+ override def children: Seq[Expression] = left :: right :: Nil
override lazy val resolved =
left.resolved && right.resolved &&
left.dataType == right.dataType
- override def dataType = {
+ override def dataType: DataType = {
if (!resolved) {
throw new UnresolvedException(this,
s"datatype. Can not resolve due to differing types ${left.dataType}, ${right.dataType}")
@@ -366,7 +366,7 @@ case class MaxOf(left: Expression, right: Expression) extends Expression {
}
}
- override def toString = s"MaxOf($left, $right)"
+ override def toString: String = s"MaxOf($left, $right)"
}
/**
@@ -375,10 +375,10 @@ case class MaxOf(left: Expression, right: Expression) extends Expression {
case class Abs(child: Expression) extends UnaryExpression {
type EvaluatedType = Any
- def dataType = child.dataType
- override def foldable = child.foldable
- def nullable = child.nullable
- override def toString = s"Abs($child)"
+ override def dataType: DataType = child.dataType
+ override def foldable: Boolean = child.foldable
+ override def nullable: Boolean = child.nullable
+ override def toString: String = s"Abs($child)"
lazy val numeric = dataType match {
case n: NumericType => n.numeric.asInstanceOf[Numeric[Any]]
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
index e48b8cde20..d1abf3c0b6 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
@@ -91,7 +91,7 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin
val startTime = System.nanoTime()
val result = create(in)
val endTime = System.nanoTime()
- def timeMs = (endTime - startTime).toDouble / 1000000
+ def timeMs: Double = (endTime - startTime).toDouble / 1000000
logInfo(s"Code generated expression $in in $timeMs ms")
result
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala
index 68051a2a20..3fd78db297 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala
@@ -27,12 +27,12 @@ import org.apache.spark.sql.types._
case class GetItem(child: Expression, ordinal: Expression) extends Expression {
type EvaluatedType = Any
- val children = child :: ordinal :: Nil
+ val children: Seq[Expression] = child :: ordinal :: Nil
/** `Null` is returned for invalid ordinals. */
- override def nullable = true
- override def foldable = child.foldable && ordinal.foldable
+ override def nullable: Boolean = true
+ override def foldable: Boolean = child.foldable && ordinal.foldable
- def dataType = child.dataType match {
+ override def dataType: DataType = child.dataType match {
case ArrayType(dt, _) => dt
case MapType(_, vt, _) => vt
}
@@ -40,7 +40,7 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression {
childrenResolved &&
(child.dataType.isInstanceOf[ArrayType] || child.dataType.isInstanceOf[MapType])
- override def toString = s"$child[$ordinal]"
+ override def toString: String = s"$child[$ordinal]"
override def eval(input: Row): Any = {
val value = child.eval(input)
@@ -75,8 +75,8 @@ trait GetField extends UnaryExpression {
self: Product =>
type EvaluatedType = Any
- override def foldable = child.foldable
- override def toString = s"$child.${field.name}"
+ override def foldable: Boolean = child.foldable
+ override def toString: String = s"$child.${field.name}"
def field: StructField
}
@@ -86,8 +86,8 @@ trait GetField extends UnaryExpression {
*/
case class StructGetField(child: Expression, field: StructField, ordinal: Int) extends GetField {
- def dataType = field.dataType
- override def nullable = child.nullable || field.nullable
+ override def dataType: DataType = field.dataType
+ override def nullable: Boolean = child.nullable || field.nullable
override def eval(input: Row): Any = {
val baseValue = child.eval(input).asInstanceOf[Row]
@@ -101,8 +101,8 @@ case class StructGetField(child: Expression, field: StructField, ordinal: Int) e
case class ArrayGetField(child: Expression, field: StructField, ordinal: Int, containsNull: Boolean)
extends GetField {
- def dataType = ArrayType(field.dataType, containsNull)
- override def nullable = child.nullable
+ override def dataType: DataType = ArrayType(field.dataType, containsNull)
+ override def nullable: Boolean = child.nullable
override def eval(input: Row): Any = {
val baseValue = child.eval(input).asInstanceOf[Seq[Row]]
@@ -120,7 +120,7 @@ case class ArrayGetField(child: Expression, field: StructField, ordinal: Int, co
case class CreateArray(children: Seq[Expression]) extends Expression {
override type EvaluatedType = Any
- override def foldable = !children.exists(!_.foldable)
+ override def foldable: Boolean = !children.exists(!_.foldable)
lazy val childTypes = children.map(_.dataType).distinct
@@ -140,5 +140,5 @@ case class CreateArray(children: Seq[Expression]) extends Expression {
children.map(_.eval(input))
}
- override def toString = s"Array(${children.mkString(",")})"
+ override def toString: String = s"Array(${children.mkString(",")})"
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala
index 83d8c1d42b..adb94df7d1 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala
@@ -24,9 +24,9 @@ case class UnscaledValue(child: Expression) extends UnaryExpression {
override type EvaluatedType = Any
override def dataType: DataType = LongType
- override def foldable = child.foldable
- def nullable = child.nullable
- override def toString = s"UnscaledValue($child)"
+ override def foldable: Boolean = child.foldable
+ override def nullable: Boolean = child.nullable
+ override def toString: String = s"UnscaledValue($child)"
override def eval(input: Row): Any = {
val childResult = child.eval(input)
@@ -43,9 +43,9 @@ case class MakeDecimal(child: Expression, precision: Int, scale: Int) extends Un
override type EvaluatedType = Decimal
override def dataType: DataType = DecimalType(precision, scale)
- override def foldable = child.foldable
- def nullable = child.nullable
- override def toString = s"MakeDecimal($child,$precision,$scale)"
+ override def foldable: Boolean = child.foldable
+ override def nullable: Boolean = child.nullable
+ override def toString: String = s"MakeDecimal($child,$precision,$scale)"
override def eval(input: Row): Decimal = {
val childResult = child.eval(input)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
index 0983d274de..860b72fad3 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
@@ -45,7 +45,7 @@ abstract class Generator extends Expression {
override lazy val dataType =
ArrayType(StructType(output.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata))))
- override def nullable = false
+ override def nullable: Boolean = false
/**
* Should be overridden by specific generators. Called only once for each instance to ensure
@@ -89,7 +89,7 @@ case class UserDefinedGenerator(
function(inputRow(input))
}
- override def toString = s"UserDefinedGenerator(${children.mkString(",")})"
+ override def toString: String = s"UserDefinedGenerator(${children.mkString(",")})"
}
/**
@@ -130,5 +130,5 @@ case class Explode(attributeNames: Seq[String], child: Expression)
}
}
- override def toString() = s"explode($child)"
+ override def toString: String = s"explode($child)"
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
index 9ff66563c8..19f3fc9c22 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
@@ -64,14 +64,13 @@ object IntegerLiteral {
case class Literal(value: Any, dataType: DataType) extends LeafExpression {
- override def foldable = true
- def nullable = value == null
+ override def foldable: Boolean = true
+ override def nullable: Boolean = value == null
-
- override def toString = if (value != null) value.toString else "null"
+ override def toString: String = if (value != null) value.toString else "null"
type EvaluatedType = Any
- override def eval(input: Row):Any = value
+ override def eval(input: Row): Any = value
}
// TODO: Specialize
@@ -79,9 +78,9 @@ case class MutableLiteral(var value: Any, dataType: DataType, nullable: Boolean
extends LeafExpression {
type EvaluatedType = Any
- def update(expression: Expression, input: Row) = {
+ def update(expression: Expression, input: Row): Unit = {
value = expression.eval(input)
}
- override def eval(input: Row) = value
+ override def eval(input: Row): Any = value
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala
index 08361d043b..bcbcbeb31c 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala
@@ -20,11 +20,12 @@ package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.trees
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
import org.apache.spark.sql.catalyst.errors.TreeNodeException
+import org.apache.spark.sql.catalyst.trees.LeafNode
import org.apache.spark.sql.types._
object NamedExpression {
private val curId = new java.util.concurrent.atomic.AtomicLong()
- def newExprId = ExprId(curId.getAndIncrement())
+ def newExprId: ExprId = ExprId(curId.getAndIncrement())
def unapply(expr: NamedExpression): Option[(String, DataType)] = Some(expr.name, expr.dataType)
}
@@ -79,13 +80,13 @@ abstract class NamedExpression extends Expression {
abstract class Attribute extends NamedExpression {
self: Product =>
- override def references = AttributeSet(this)
+ override def references: AttributeSet = AttributeSet(this)
def withNullability(newNullability: Boolean): Attribute
def withQualifiers(newQualifiers: Seq[String]): Attribute
def withName(newName: String): Attribute
- def toAttribute = this
+ def toAttribute: Attribute = this
def newInstance(): Attribute
}
@@ -112,10 +113,10 @@ case class Alias(child: Expression, name: String)(
override type EvaluatedType = Any
- override def eval(input: Row) = child.eval(input)
+ override def eval(input: Row): Any = child.eval(input)
- override def dataType = child.dataType
- override def nullable = child.nullable
+ override def dataType: DataType = child.dataType
+ override def nullable: Boolean = child.nullable
override def metadata: Metadata = {
explicitMetadata.getOrElse {
child match {
@@ -125,7 +126,7 @@ case class Alias(child: Expression, name: String)(
}
}
- override def toAttribute = {
+ override def toAttribute: Attribute = {
if (resolved) {
AttributeReference(name, child.dataType, child.nullable, metadata)(exprId, qualifiers)
} else {
@@ -135,7 +136,9 @@ case class Alias(child: Expression, name: String)(
override def toString: String = s"$child AS $name#${exprId.id}$typeSuffix"
- override protected final def otherCopyArgs = exprId :: qualifiers :: explicitMetadata :: Nil
+ override protected final def otherCopyArgs: Seq[AnyRef] = {
+ exprId :: qualifiers :: explicitMetadata :: Nil
+ }
override def equals(other: Any): Boolean = other match {
case a: Alias =>
@@ -166,7 +169,7 @@ case class AttributeReference(
val exprId: ExprId = NamedExpression.newExprId,
val qualifiers: Seq[String] = Nil) extends Attribute with trees.LeafNode[Expression] {
- override def equals(other: Any) = other match {
+ override def equals(other: Any): Boolean = other match {
case ar: AttributeReference => name == ar.name && exprId == ar.exprId && dataType == ar.dataType
case _ => false
}
@@ -180,7 +183,7 @@ case class AttributeReference(
h
}
- override def newInstance() =
+ override def newInstance(): AttributeReference =
AttributeReference(name, dataType, nullable, metadata)(qualifiers = qualifiers)
/**
@@ -205,7 +208,7 @@ case class AttributeReference(
/**
* Returns a copy of this [[AttributeReference]] with new qualifiers.
*/
- override def withQualifiers(newQualifiers: Seq[String]) = {
+ override def withQualifiers(newQualifiers: Seq[String]): AttributeReference = {
if (newQualifiers.toSet == qualifiers.toSet) {
this
} else {
@@ -227,20 +230,22 @@ case class AttributeReference(
case class PrettyAttribute(name: String) extends Attribute with trees.LeafNode[Expression] {
type EvaluatedType = Any
- override def toString = name
-
- override def withNullability(newNullability: Boolean): Attribute = ???
- override def newInstance(): Attribute = ???
- override def withQualifiers(newQualifiers: Seq[String]): Attribute = ???
- override def withName(newName: String): Attribute = ???
- override def qualifiers: Seq[String] = ???
- override def exprId: ExprId = ???
- override def eval(input: Row): EvaluatedType = ???
- override def nullable: Boolean = ???
+ override def toString: String = name
+
+ override def withNullability(newNullability: Boolean): Attribute =
+ throw new UnsupportedOperationException
+ override def newInstance(): Attribute = throw new UnsupportedOperationException
+ override def withQualifiers(newQualifiers: Seq[String]): Attribute =
+ throw new UnsupportedOperationException
+ override def withName(newName: String): Attribute = throw new UnsupportedOperationException
+ override def qualifiers: Seq[String] = throw new UnsupportedOperationException
+ override def exprId: ExprId = throw new UnsupportedOperationException
+ override def eval(input: Row): EvaluatedType = throw new UnsupportedOperationException
+ override def nullable: Boolean = throw new UnsupportedOperationException
override def dataType: DataType = NullType
}
object VirtualColumn {
- val groupingIdName = "grouping__id"
- def newGroupingId = AttributeReference(groupingIdName, IntegerType, false)()
+ val groupingIdName: String = "grouping__id"
+ def newGroupingId: AttributeReference = AttributeReference(groupingIdName, IntegerType, false)()
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala
index 08b982bc67..d1f3d4f4ee 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala
@@ -19,22 +19,23 @@ package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.trees
import org.apache.spark.sql.catalyst.analysis.UnresolvedException
+import org.apache.spark.sql.types.DataType
case class Coalesce(children: Seq[Expression]) extends Expression {
type EvaluatedType = Any
/** Coalesce is nullable if all of its children are nullable, or if it has no children. */
- def nullable = !children.exists(!_.nullable)
+ override def nullable: Boolean = !children.exists(!_.nullable)
// Coalesce is foldable if all children are foldable.
- override def foldable = !children.exists(!_.foldable)
+ override def foldable: Boolean = !children.exists(!_.foldable)
// Only resolved if all the children are of the same type.
override lazy val resolved = childrenResolved && (children.map(_.dataType).distinct.size == 1)
- override def toString = s"Coalesce(${children.mkString(",")})"
+ override def toString: String = s"Coalesce(${children.mkString(",")})"
- def dataType = if (resolved) {
+ def dataType: DataType = if (resolved) {
children.head.dataType
} else {
val childTypes = children.map(c => s"$c: ${c.dataType}").mkString(", ")
@@ -54,20 +55,20 @@ case class Coalesce(children: Seq[Expression]) extends Expression {
}
case class IsNull(child: Expression) extends Predicate with trees.UnaryNode[Expression] {
- override def foldable = child.foldable
- def nullable = false
+ override def foldable: Boolean = child.foldable
+ override def nullable: Boolean = false
override def eval(input: Row): Any = {
child.eval(input) == null
}
- override def toString = s"IS NULL $child"
+ override def toString: String = s"IS NULL $child"
}
case class IsNotNull(child: Expression) extends Predicate with trees.UnaryNode[Expression] {
- override def foldable = child.foldable
- def nullable = false
- override def toString = s"IS NOT NULL $child"
+ override def foldable: Boolean = child.foldable
+ override def nullable: Boolean = false
+ override def toString: String = s"IS NOT NULL $child"
override def eval(input: Row): Any = {
child.eval(input) != null
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
index 0024ef92c0..7e47cb3fff 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.analysis.UnresolvedException
import org.apache.spark.sql.catalyst.errors.TreeNodeException
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.types.{BinaryType, BooleanType, NativeType}
+import org.apache.spark.sql.types.{DataType, BinaryType, BooleanType, NativeType}
object InterpretedPredicate {
def apply(expression: Expression, inputSchema: Seq[Attribute]): (Row => Boolean) =
@@ -34,7 +34,7 @@ object InterpretedPredicate {
trait Predicate extends Expression {
self: Product =>
- def dataType = BooleanType
+ override def dataType: DataType = BooleanType
type EvaluatedType = Any
}
@@ -72,13 +72,13 @@ trait PredicateHelper {
abstract class BinaryPredicate extends BinaryExpression with Predicate {
self: Product =>
- def nullable = left.nullable || right.nullable
+ override def nullable: Boolean = left.nullable || right.nullable
}
case class Not(child: Expression) extends UnaryExpression with Predicate {
- override def foldable = child.foldable
- def nullable = child.nullable
- override def toString = s"NOT $child"
+ override def foldable: Boolean = child.foldable
+ override def nullable: Boolean = child.nullable
+ override def toString: String = s"NOT $child"
override def eval(input: Row): Any = {
child.eval(input) match {
@@ -92,10 +92,10 @@ case class Not(child: Expression) extends UnaryExpression with Predicate {
* Evaluates to `true` if `list` contains `value`.
*/
case class In(value: Expression, list: Seq[Expression]) extends Predicate {
- def children = value +: list
+ override def children: Seq[Expression] = value +: list
- def nullable = true // TODO: Figure out correct nullability semantics of IN.
- override def toString = s"$value IN ${list.mkString("(", ",", ")")}"
+ override def nullable: Boolean = true // TODO: Figure out correct nullability semantics of IN.
+ override def toString: String = s"$value IN ${list.mkString("(", ",", ")")}"
override def eval(input: Row): Any = {
val evaluatedValue = value.eval(input)
@@ -110,10 +110,10 @@ case class In(value: Expression, list: Seq[Expression]) extends Predicate {
case class InSet(value: Expression, hset: Set[Any])
extends Predicate {
- def children = value :: Nil
+ override def children: Seq[Expression] = value :: Nil
- def nullable = true // TODO: Figure out correct nullability semantics of IN.
- override def toString = s"$value INSET ${hset.mkString("(", ",", ")")}"
+ override def nullable: Boolean = true // TODO: Figure out correct nullability semantics of IN.
+ override def toString: String = s"$value INSET ${hset.mkString("(", ",", ")")}"
override def eval(input: Row): Any = {
hset.contains(value.eval(input))
@@ -121,7 +121,7 @@ case class InSet(value: Expression, hset: Set[Any])
}
case class And(left: Expression, right: Expression) extends BinaryPredicate {
- def symbol = "&&"
+ override def symbol: String = "&&"
override def eval(input: Row): Any = {
val l = left.eval(input)
@@ -143,7 +143,7 @@ case class And(left: Expression, right: Expression) extends BinaryPredicate {
}
case class Or(left: Expression, right: Expression) extends BinaryPredicate {
- def symbol = "||"
+ override def symbol: String = "||"
override def eval(input: Row): Any = {
val l = left.eval(input)
@@ -169,7 +169,8 @@ abstract class BinaryComparison extends BinaryPredicate {
}
case class EqualTo(left: Expression, right: Expression) extends BinaryComparison {
- def symbol = "="
+ override def symbol: String = "="
+
override def eval(input: Row): Any = {
val l = left.eval(input)
if (l == null) {
@@ -185,8 +186,10 @@ case class EqualTo(left: Expression, right: Expression) extends BinaryComparison
}
case class EqualNullSafe(left: Expression, right: Expression) extends BinaryComparison {
- def symbol = "<=>"
- override def nullable = false
+ override def symbol: String = "<=>"
+
+ override def nullable: Boolean = false
+
override def eval(input: Row): Any = {
val l = left.eval(input)
val r = right.eval(input)
@@ -201,9 +204,9 @@ case class EqualNullSafe(left: Expression, right: Expression) extends BinaryComp
}
case class LessThan(left: Expression, right: Expression) extends BinaryComparison {
- def symbol = "<"
+ override def symbol: String = "<"
- lazy val ordering = {
+ lazy val ordering: Ordering[Any] = {
if (left.dataType != right.dataType) {
throw new TreeNodeException(this,
s"Types do not match ${left.dataType} != ${right.dataType}")
@@ -216,7 +219,7 @@ case class LessThan(left: Expression, right: Expression) extends BinaryCompariso
override def eval(input: Row): Any = {
val evalE1 = left.eval(input)
- if(evalE1 == null) {
+ if (evalE1 == null) {
null
} else {
val evalE2 = right.eval(input)
@@ -230,9 +233,9 @@ case class LessThan(left: Expression, right: Expression) extends BinaryCompariso
}
case class LessThanOrEqual(left: Expression, right: Expression) extends BinaryComparison {
- def symbol = "<="
+ override def symbol: String = "<="
- lazy val ordering = {
+ lazy val ordering: Ordering[Any] = {
if (left.dataType != right.dataType) {
throw new TreeNodeException(this,
s"Types do not match ${left.dataType} != ${right.dataType}")
@@ -245,7 +248,7 @@ case class LessThanOrEqual(left: Expression, right: Expression) extends BinaryCo
override def eval(input: Row): Any = {
val evalE1 = left.eval(input)
- if(evalE1 == null) {
+ if (evalE1 == null) {
null
} else {
val evalE2 = right.eval(input)
@@ -259,9 +262,9 @@ case class LessThanOrEqual(left: Expression, right: Expression) extends BinaryCo
}
case class GreaterThan(left: Expression, right: Expression) extends BinaryComparison {
- def symbol = ">"
+ override def symbol: String = ">"
- lazy val ordering = {
+ lazy val ordering: Ordering[Any] = {
if (left.dataType != right.dataType) {
throw new TreeNodeException(this,
s"Types do not match ${left.dataType} != ${right.dataType}")
@@ -288,9 +291,9 @@ case class GreaterThan(left: Expression, right: Expression) extends BinaryCompar
}
case class GreaterThanOrEqual(left: Expression, right: Expression) extends BinaryComparison {
- def symbol = ">="
+ override def symbol: String = ">="
- lazy val ordering = {
+ lazy val ordering: Ordering[Any] = {
if (left.dataType != right.dataType) {
throw new TreeNodeException(this,
s"Types do not match ${left.dataType} != ${right.dataType}")
@@ -303,7 +306,7 @@ case class GreaterThanOrEqual(left: Expression, right: Expression) extends Binar
override def eval(input: Row): Any = {
val evalE1 = left.eval(input)
- if(evalE1 == null) {
+ if (evalE1 == null) {
null
} else {
val evalE2 = right.eval(input)
@@ -317,13 +320,13 @@ case class GreaterThanOrEqual(left: Expression, right: Expression) extends Binar
}
case class If(predicate: Expression, trueValue: Expression, falseValue: Expression)
- extends Expression {
+ extends Expression {
- def children = predicate :: trueValue :: falseValue :: Nil
- override def nullable = trueValue.nullable || falseValue.nullable
+ override def children: Seq[Expression] = predicate :: trueValue :: falseValue :: Nil
+ override def nullable: Boolean = trueValue.nullable || falseValue.nullable
override lazy val resolved = childrenResolved && trueValue.dataType == falseValue.dataType
- def dataType = {
+ override def dataType: DataType = {
if (!resolved) {
throw new UnresolvedException(
this,
@@ -342,7 +345,7 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi
}
}
- override def toString = s"if ($predicate) $trueValue else $falseValue"
+ override def toString: String = s"if ($predicate) $trueValue else $falseValue"
}
// scalastyle:off
@@ -362,9 +365,10 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi
// scalastyle:on
case class CaseWhen(branches: Seq[Expression]) extends Expression {
type EvaluatedType = Any
- def children = branches
- def dataType = {
+ override def children: Seq[Expression] = branches
+
+ override def dataType: DataType = {
if (!resolved) {
throw new UnresolvedException(this, "cannot resolve due to differing types in some branches")
}
@@ -379,12 +383,12 @@ case class CaseWhen(branches: Seq[Expression]) extends Expression {
@transient private[this] lazy val elseValue =
if (branches.length % 2 == 0) None else Option(branches.last)
- override def nullable = {
+ override def nullable: Boolean = {
// If no value is nullable and no elseValue is provided, the whole statement defaults to null.
values.exists(_.nullable) || (elseValue.map(_.nullable).getOrElse(true))
}
- override lazy val resolved = {
+ override lazy val resolved: Boolean = {
if (!childrenResolved) {
false
} else {
@@ -415,7 +419,7 @@ case class CaseWhen(branches: Seq[Expression]) extends Expression {
res
}
- override def toString = {
+ override def toString: String = {
"CASE" + branches.sliding(2, 2).map {
case Seq(cond, value) => s" WHEN $cond THEN $value"
case Seq(elseValue) => s" ELSE $elseValue"
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala
index f03d6f71a9..8bba26bc4c 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala
@@ -44,8 +44,8 @@ trait MutableRow extends Row {
*/
object EmptyRow extends Row {
override def apply(i: Int): Any = throw new UnsupportedOperationException
- override def toSeq = Seq.empty
- override def length = 0
+ override def toSeq: Seq[Any] = Seq.empty
+ override def length: Int = 0
override def isNullAt(i: Int): Boolean = throw new UnsupportedOperationException
override def getInt(i: Int): Int = throw new UnsupportedOperationException
override def getLong(i: Int): Long = throw new UnsupportedOperationException
@@ -56,7 +56,7 @@ object EmptyRow extends Row {
override def getByte(i: Int): Byte = throw new UnsupportedOperationException
override def getString(i: Int): String = throw new UnsupportedOperationException
override def getAs[T](i: Int): T = throw new UnsupportedOperationException
- def copy() = this
+ override def copy(): Row = this
}
/**
@@ -70,13 +70,13 @@ class GenericRow(protected[sql] val values: Array[Any]) extends Row {
def this(size: Int) = this(new Array[Any](size))
- override def toSeq = values.toSeq
+ override def toSeq: Seq[Any] = values.toSeq
- override def length = values.length
+ override def length: Int = values.length
- override def apply(i: Int) = values(i)
+ override def apply(i: Int): Any = values(i)
- override def isNullAt(i: Int) = values(i) == null
+ override def isNullAt(i: Int): Boolean = values(i) == null
override def getInt(i: Int): Int = {
if (values(i) == null) sys.error("Failed to check null bit for primitive int value.")
@@ -167,7 +167,7 @@ class GenericRow(protected[sql] val values: Array[Any]) extends Row {
case _ => false
}
- def copy() = this
+ override def copy(): Row = this
}
class GenericRowWithSchema(values: Array[Any], override val schema: StructType)
@@ -194,7 +194,7 @@ class GenericMutableRow(v: Array[Any]) extends GenericRow(v) with MutableRow {
override def update(ordinal: Int, value: Any): Unit = { values(ordinal) = value }
- override def copy() = new GenericRow(values.clone())
+ override def copy(): Row = new GenericRow(values.clone())
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala
index 3a5bdca1f0..35faa00782 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala
@@ -26,17 +26,17 @@ import org.apache.spark.util.collection.OpenHashSet
case class NewSet(elementType: DataType) extends LeafExpression {
type EvaluatedType = Any
- def nullable = false
+ override def nullable: Boolean = false
// We are currently only using these Expressions internally for aggregation. However, if we ever
// expose these to users we'll want to create a proper type instead of hijacking ArrayType.
- def dataType = ArrayType(elementType)
+ override def dataType: DataType = ArrayType(elementType)
- def eval(input: Row): Any = {
+ override def eval(input: Row): Any = {
new OpenHashSet[Any]()
}
- override def toString = s"new Set($dataType)"
+ override def toString: String = s"new Set($dataType)"
}
/**
@@ -46,12 +46,13 @@ case class NewSet(elementType: DataType) extends LeafExpression {
case class AddItemToSet(item: Expression, set: Expression) extends Expression {
type EvaluatedType = Any
- def children = item :: set :: Nil
+ override def children: Seq[Expression] = item :: set :: Nil
- def nullable = set.nullable
+ override def nullable: Boolean = set.nullable
- def dataType = set.dataType
- def eval(input: Row): Any = {
+ override def dataType: DataType = set.dataType
+
+ override def eval(input: Row): Any = {
val itemEval = item.eval(input)
val setEval = set.eval(input).asInstanceOf[OpenHashSet[Any]]
@@ -67,7 +68,7 @@ case class AddItemToSet(item: Expression, set: Expression) extends Expression {
}
}
- override def toString = s"$set += $item"
+ override def toString: String = s"$set += $item"
}
/**
@@ -77,13 +78,13 @@ case class AddItemToSet(item: Expression, set: Expression) extends Expression {
case class CombineSets(left: Expression, right: Expression) extends BinaryExpression {
type EvaluatedType = Any
- def nullable = left.nullable || right.nullable
+ override def nullable: Boolean = left.nullable || right.nullable
- def dataType = left.dataType
+ override def dataType: DataType = left.dataType
- def symbol = "++="
+ override def symbol: String = "++="
- def eval(input: Row): Any = {
+ override def eval(input: Row): Any = {
val leftEval = left.eval(input).asInstanceOf[OpenHashSet[Any]]
if(leftEval != null) {
val rightEval = right.eval(input).asInstanceOf[OpenHashSet[Any]]
@@ -109,16 +110,16 @@ case class CombineSets(left: Expression, right: Expression) extends BinaryExpres
case class CountSet(child: Expression) extends UnaryExpression {
type EvaluatedType = Any
- def nullable = child.nullable
+ override def nullable: Boolean = child.nullable
- def dataType = LongType
+ override def dataType: DataType = LongType
- def eval(input: Row): Any = {
+ override def eval(input: Row): Any = {
val childEval = child.eval(input).asInstanceOf[OpenHashSet[Any]]
if (childEval != null) {
childEval.size.toLong
}
}
- override def toString = s"$child.count()"
+ override def toString: String = s"$child.count()"
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
index f85ee0a9bb..3cdca4e9dd 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
@@ -33,8 +33,8 @@ trait StringRegexExpression {
def escape(v: String): String
def matches(regex: Pattern, str: String): Boolean
- def nullable: Boolean = left.nullable || right.nullable
- def dataType: DataType = BooleanType
+ override def nullable: Boolean = left.nullable || right.nullable
+ override def dataType: DataType = BooleanType
// try cache the pattern for Literal
private lazy val cache: Pattern = right match {
@@ -98,11 +98,11 @@ trait CaseConversionExpression {
case class Like(left: Expression, right: Expression)
extends BinaryExpression with StringRegexExpression {
- def symbol = "LIKE"
+ override def symbol: String = "LIKE"
// replace the _ with .{1} exactly match 1 time of any character
// replace the % with .*, match 0 or more times with any character
- override def escape(v: String) =
+ override def escape(v: String): String =
if (!v.isEmpty) {
"(?s)" + (' ' +: v.init).zip(v).flatMap {
case (prev, '\\') => ""
@@ -129,7 +129,7 @@ case class Like(left: Expression, right: Expression)
case class RLike(left: Expression, right: Expression)
extends BinaryExpression with StringRegexExpression {
- def symbol = "RLIKE"
+ override def symbol: String = "RLIKE"
override def escape(v: String): String = v
override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).find(0)
}
@@ -141,7 +141,7 @@ case class Upper(child: Expression) extends UnaryExpression with CaseConversionE
override def convert(v: String): String = v.toUpperCase()
- override def toString() = s"Upper($child)"
+ override def toString: String = s"Upper($child)"
}
/**
@@ -151,7 +151,7 @@ case class Lower(child: Expression) extends UnaryExpression with CaseConversionE
override def convert(v: String): String = v.toLowerCase()
- override def toString() = s"Lower($child)"
+ override def toString: String = s"Lower($child)"
}
/** A base trait for functions that compare two strings, returning a boolean. */
@@ -160,7 +160,7 @@ trait StringComparison {
type EvaluatedType = Any
- def nullable: Boolean = left.nullable || right.nullable
+ override def nullable: Boolean = left.nullable || right.nullable
override def dataType: DataType = BooleanType
def compare(l: String, r: String): Boolean
@@ -175,9 +175,9 @@ trait StringComparison {
}
}
- def symbol: String = nodeName
+ override def symbol: String = nodeName
- override def toString() = s"$nodeName($left, $right)"
+ override def toString: String = s"$nodeName($left, $right)"
}
/**
@@ -185,7 +185,7 @@ trait StringComparison {
*/
case class Contains(left: Expression, right: Expression)
extends BinaryExpression with StringComparison {
- override def compare(l: String, r: String) = l.contains(r)
+ override def compare(l: String, r: String): Boolean = l.contains(r)
}
/**
@@ -193,7 +193,7 @@ case class Contains(left: Expression, right: Expression)
*/
case class StartsWith(left: Expression, right: Expression)
extends BinaryExpression with StringComparison {
- def compare(l: String, r: String) = l.startsWith(r)
+ override def compare(l: String, r: String): Boolean = l.startsWith(r)
}
/**
@@ -201,7 +201,7 @@ case class StartsWith(left: Expression, right: Expression)
*/
case class EndsWith(left: Expression, right: Expression)
extends BinaryExpression with StringComparison {
- def compare(l: String, r: String) = l.endsWith(r)
+ override def compare(l: String, r: String): Boolean = l.endsWith(r)
}
/**
@@ -212,17 +212,17 @@ case class Substring(str: Expression, pos: Expression, len: Expression) extends
type EvaluatedType = Any
- override def foldable = str.foldable && pos.foldable && len.foldable
+ override def foldable: Boolean = str.foldable && pos.foldable && len.foldable
- def nullable: Boolean = str.nullable || pos.nullable || len.nullable
- def dataType: DataType = {
+ override def nullable: Boolean = str.nullable || pos.nullable || len.nullable
+ override def dataType: DataType = {
if (!resolved) {
throw new UnresolvedException(this, s"Cannot resolve since $children are not resolved")
}
if (str.dataType == BinaryType) str.dataType else StringType
}
- override def children = str :: pos :: len :: Nil
+ override def children: Seq[Expression] = str :: pos :: len :: Nil
@inline
def slice[T, C <: Any](str: C, startPos: Int, sliceLen: Int)
@@ -267,7 +267,8 @@ case class Substring(str: Expression, pos: Expression, len: Expression) extends
}
}
- override def toString = len match {
+ override def toString: String = len match {
+ // TODO: This is broken because max is not an integer value.
case max if max == Integer.MAX_VALUE => s"SUBSTR($str, $pos)"
case _ => s"SUBSTR($str, $pos, $len)"
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
index 74edaacc4f..c23d3b6188 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
@@ -141,7 +141,7 @@ object ColumnPruning extends Rule[LogicalPlan] {
condition.map(_.references).getOrElse(AttributeSet(Seq.empty))
/** Applies a projection only when the child is producing unnecessary attributes */
- def pruneJoinChild(c: LogicalPlan) = prunedChild(c, allReferences)
+ def pruneJoinChild(c: LogicalPlan): LogicalPlan = prunedChild(c, allReferences)
Project(projectList, Join(pruneJoinChild(left), pruneJoinChild(right), joinType, condition))
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala
index b4c445b3ba..9c8c643f7d 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala
@@ -91,16 +91,18 @@ object PhysicalOperation extends PredicateHelper {
(None, Nil, other, Map.empty)
}
- def collectAliases(fields: Seq[Expression]) = fields.collect {
+ def collectAliases(fields: Seq[Expression]): Map[Attribute, Expression] = fields.collect {
case a @ Alias(child, _) => a.toAttribute.asInstanceOf[Attribute] -> child
}.toMap
- def substitute(aliases: Map[Attribute, Expression])(expr: Expression) = expr.transform {
- case a @ Alias(ref: AttributeReference, name) =>
- aliases.get(ref).map(Alias(_, name)(a.exprId, a.qualifiers)).getOrElse(a)
+ def substitute(aliases: Map[Attribute, Expression])(expr: Expression): Expression = {
+ expr.transform {
+ case a @ Alias(ref: AttributeReference, name) =>
+ aliases.get(ref).map(Alias(_, name)(a.exprId, a.qualifiers)).getOrElse(a)
- case a: AttributeReference =>
- aliases.get(a).map(Alias(_, a.name)(a.exprId, a.qualifiers)).getOrElse(a)
+ case a: AttributeReference =>
+ aliases.get(a).map(Alias(_, a.name)(a.exprId, a.qualifiers)).getOrElse(a)
+ }
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
index bd9291e9ba..02f7c26a8a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
@@ -71,7 +71,7 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy
def transformExpressionsDown(rule: PartialFunction[Expression, Expression]): this.type = {
var changed = false
- @inline def transformExpressionDown(e: Expression) = {
+ @inline def transformExpressionDown(e: Expression): Expression = {
val newE = e.transformDown(rule)
if (newE.fastEquals(e)) {
e
@@ -104,7 +104,7 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy
def transformExpressionsUp(rule: PartialFunction[Expression, Expression]): this.type = {
var changed = false
- @inline def transformExpressionUp(e: Expression) = {
+ @inline def transformExpressionUp(e: Expression): Expression = {
val newE = e.transformUp(rule)
if (newE.fastEquals(e)) {
e
@@ -165,5 +165,5 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy
*/
protected def statePrefix = if (missingInput.nonEmpty && children.nonEmpty) "!" else ""
- override def simpleString = statePrefix + super.simpleString
+ override def simpleString: String = statePrefix + super.simpleString
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
index 384fe53a68..4d9e41a2b5 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
@@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.types._
case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends UnaryNode {
- def output = projectList.map(_.toAttribute)
+ override def output: Seq[Attribute] = projectList.map(_.toAttribute)
override lazy val resolved: Boolean = {
val containsAggregatesOrGenerators = projectList.exists ( _.collect {
@@ -66,19 +66,19 @@ case class Generate(
}
}
- override def output =
+ override def output: Seq[Attribute] =
if (join) child.output ++ generatorOutput else generatorOutput
}
case class Filter(condition: Expression, child: LogicalPlan) extends UnaryNode {
- override def output = child.output
+ override def output: Seq[Attribute] = child.output
}
case class Union(left: LogicalPlan, right: LogicalPlan) extends BinaryNode {
// TODO: These aren't really the same attributes as nullability etc might change.
- override def output = left.output
+ override def output: Seq[Attribute] = left.output
- override lazy val resolved =
+ override lazy val resolved: Boolean =
childrenResolved &&
!left.output.zip(right.output).exists { case (l,r) => l.dataType != r.dataType }
@@ -94,7 +94,7 @@ case class Join(
joinType: JoinType,
condition: Option[Expression]) extends BinaryNode {
- override def output = {
+ override def output: Seq[Attribute] = {
joinType match {
case LeftSemi =>
left.output
@@ -109,7 +109,7 @@ case class Join(
}
}
- def selfJoinResolved = left.outputSet.intersect(right.outputSet).isEmpty
+ private def selfJoinResolved: Boolean = left.outputSet.intersect(right.outputSet).isEmpty
// Joins are only resolved if they don't introduce ambiguious expression ids.
override lazy val resolved: Boolean = {
@@ -118,7 +118,7 @@ case class Join(
}
case class Except(left: LogicalPlan, right: LogicalPlan) extends BinaryNode {
- def output = left.output
+ override def output: Seq[Attribute] = left.output
}
case class InsertIntoTable(
@@ -128,10 +128,10 @@ case class InsertIntoTable(
overwrite: Boolean)
extends LogicalPlan {
- override def children = child :: Nil
- override def output = child.output
+ override def children: Seq[LogicalPlan] = child :: Nil
+ override def output: Seq[Attribute] = child.output
- override lazy val resolved = childrenResolved && child.output.zip(table.output).forall {
+ override lazy val resolved: Boolean = childrenResolved && child.output.zip(table.output).forall {
case (childAttr, tableAttr) =>
DataType.equalsIgnoreCompatibleNullability(childAttr.dataType, tableAttr.dataType)
}
@@ -143,14 +143,14 @@ case class CreateTableAsSelect[T](
child: LogicalPlan,
allowExisting: Boolean,
desc: Option[T] = None) extends UnaryNode {
- override def output = Seq.empty[Attribute]
- override lazy val resolved = databaseName != None && childrenResolved
+ override def output: Seq[Attribute] = Seq.empty[Attribute]
+ override lazy val resolved: Boolean = databaseName != None && childrenResolved
}
case class WriteToFile(
path: String,
child: LogicalPlan) extends UnaryNode {
- override def output = child.output
+ override def output: Seq[Attribute] = child.output
}
/**
@@ -163,7 +163,7 @@ case class Sort(
order: Seq[SortOrder],
global: Boolean,
child: LogicalPlan) extends UnaryNode {
- override def output = child.output
+ override def output: Seq[Attribute] = child.output
}
case class Aggregate(
@@ -172,7 +172,7 @@ case class Aggregate(
child: LogicalPlan)
extends UnaryNode {
- override def output = aggregateExpressions.map(_.toAttribute)
+ override def output: Seq[Attribute] = aggregateExpressions.map(_.toAttribute)
}
/**
@@ -199,7 +199,7 @@ trait GroupingAnalytics extends UnaryNode {
def groupByExprs: Seq[Expression]
def aggregations: Seq[NamedExpression]
- override def output = aggregations.map(_.toAttribute)
+ override def output: Seq[Attribute] = aggregations.map(_.toAttribute)
}
/**
@@ -264,7 +264,7 @@ case class Rollup(
gid: AttributeReference = VirtualColumn.newGroupingId) extends GroupingAnalytics
case class Limit(limitExpr: Expression, child: LogicalPlan) extends UnaryNode {
- override def output = child.output
+ override def output: Seq[Attribute] = child.output
override lazy val statistics: Statistics = {
val limit = limitExpr.eval(null).asInstanceOf[Int]
@@ -274,21 +274,21 @@ case class Limit(limitExpr: Expression, child: LogicalPlan) extends UnaryNode {
}
case class Subquery(alias: String, child: LogicalPlan) extends UnaryNode {
- override def output = child.output.map(_.withQualifiers(alias :: Nil))
+ override def output: Seq[Attribute] = child.output.map(_.withQualifiers(alias :: Nil))
}
case class Sample(fraction: Double, withReplacement: Boolean, seed: Long, child: LogicalPlan)
extends UnaryNode {
- override def output = child.output
+ override def output: Seq[Attribute] = child.output
}
case class Distinct(child: LogicalPlan) extends UnaryNode {
- override def output = child.output
+ override def output: Seq[Attribute] = child.output
}
case object NoRelation extends LeafNode {
- override def output = Nil
+ override def output: Seq[Attribute] = Nil
/**
* Computes [[Statistics]] for this plan. The default implementation assumes the output
@@ -301,5 +301,5 @@ case object NoRelation extends LeafNode {
}
case class Intersect(left: LogicalPlan, right: LogicalPlan) extends BinaryNode {
- override def output = left.output
+ override def output: Seq[Attribute] = left.output
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala
index 72b0c5c8e7..e737418d9c 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala
@@ -17,7 +17,7 @@
package org.apache.spark.sql.catalyst.plans.logical
-import org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder}
+import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, SortOrder}
/**
* Performs a physical redistribution of the data. Used when the consumer of the query
@@ -26,14 +26,11 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder}
abstract class RedistributeData extends UnaryNode {
self: Product =>
- def output = child.output
+ override def output: Seq[Attribute] = child.output
}
case class SortPartitions(sortExpressions: Seq[SortOrder], child: LogicalPlan)
- extends RedistributeData {
-}
+ extends RedistributeData
case class Repartition(partitionExpressions: Seq[Expression], child: LogicalPlan)
- extends RedistributeData {
-}
-
+ extends RedistributeData
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
index 3c3d7a3119..288c11f69f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.plans.physical
import org.apache.spark.sql.catalyst.errors.TreeNodeException
import org.apache.spark.sql.catalyst.expressions.{Expression, Row, SortOrder}
-import org.apache.spark.sql.types.IntegerType
+import org.apache.spark.sql.types.{DataType, IntegerType}
/**
* Specifies how tuples that share common expressions will be distributed when a query is executed
@@ -72,7 +72,7 @@ case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution {
"a single partition.")
// TODO: This is not really valid...
- def clustering = ordering.map(_.child).toSet
+ def clustering: Set[Expression] = ordering.map(_.child).toSet
}
sealed trait Partitioning {
@@ -113,7 +113,7 @@ case object SinglePartition extends Partitioning {
override def satisfies(required: Distribution): Boolean = true
- override def compatibleWith(other: Partitioning) = other match {
+ override def compatibleWith(other: Partitioning): Boolean = other match {
case SinglePartition => true
case _ => false
}
@@ -124,7 +124,7 @@ case object BroadcastPartitioning extends Partitioning {
override def satisfies(required: Distribution): Boolean = true
- override def compatibleWith(other: Partitioning) = other match {
+ override def compatibleWith(other: Partitioning): Boolean = other match {
case SinglePartition => true
case _ => false
}
@@ -139,9 +139,9 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int)
extends Expression
with Partitioning {
- override def children = expressions
- override def nullable = false
- override def dataType = IntegerType
+ override def children: Seq[Expression] = expressions
+ override def nullable: Boolean = false
+ override def dataType: DataType = IntegerType
private[this] lazy val clusteringSet = expressions.toSet
@@ -152,7 +152,7 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int)
case _ => false
}
- override def compatibleWith(other: Partitioning) = other match {
+ override def compatibleWith(other: Partitioning): Boolean = other match {
case BroadcastPartitioning => true
case h: HashPartitioning if h == this => true
case _ => false
@@ -178,9 +178,9 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int)
extends Expression
with Partitioning {
- override def children = ordering
- override def nullable = false
- override def dataType = IntegerType
+ override def children: Seq[SortOrder] = ordering
+ override def nullable: Boolean = false
+ override def dataType: DataType = IntegerType
private[this] lazy val clusteringSet = ordering.map(_.child).toSet
@@ -194,7 +194,7 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int)
case _ => false
}
- override def compatibleWith(other: Partitioning) = other match {
+ override def compatibleWith(other: Partitioning): Boolean = other match {
case BroadcastPartitioning => true
case r: RangePartitioning if r == this => true
case _ => false
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
index 0ae9f6b296..a2df51e598 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
@@ -36,12 +36,12 @@ object CurrentOrigin {
override def initialValue: Origin = Origin()
}
- def get = value.get()
- def set(o: Origin) = value.set(o)
+ def get: Origin = value.get()
+ def set(o: Origin): Unit = value.set(o)
- def reset() = value.set(Origin())
+ def reset(): Unit = value.set(Origin())
- def setPosition(line: Int, start: Int) = {
+ def setPosition(line: Int, start: Int): Unit = {
value.set(
value.get.copy(line = Some(line), startPosition = Some(start)))
}
@@ -57,7 +57,7 @@ object CurrentOrigin {
abstract class TreeNode[BaseType <: TreeNode[BaseType]] {
self: BaseType with Product =>
- val origin = CurrentOrigin.get
+ val origin: Origin = CurrentOrigin.get
/** Returns a Seq of the children of this node */
def children: Seq[BaseType]
@@ -340,12 +340,12 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] {
}
/** Returns the name of this type of TreeNode. Defaults to the class name. */
- def nodeName = getClass.getSimpleName
+ def nodeName: String = getClass.getSimpleName
/**
* The arguments that should be included in the arg string. Defaults to the `productIterator`.
*/
- protected def stringArgs = productIterator
+ protected def stringArgs: Iterator[Any] = productIterator
/** Returns a string representing the arguments to this node, minus any children */
def argString: String = productIterator.flatMap {
@@ -357,18 +357,18 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] {
}.mkString(", ")
/** String representation of this node without any children */
- def simpleString = s"$nodeName $argString".trim
+ def simpleString: String = s"$nodeName $argString".trim
override def toString: String = treeString
/** Returns a string representation of the nodes in this tree */
- def treeString = generateTreeString(0, new StringBuilder).toString
+ def treeString: String = generateTreeString(0, new StringBuilder).toString
/**
* Returns a string representation of the nodes in this tree, where each operator is numbered.
* The numbers can be used with [[trees.TreeNode.apply apply]] to easily access specific subtrees.
*/
- def numberedTreeString =
+ def numberedTreeString: String =
treeString.split("\n").zipWithIndex.map { case (line, i) => f"$i%02d $line" }.mkString("\n")
/**
@@ -420,14 +420,14 @@ trait BinaryNode[BaseType <: TreeNode[BaseType]] {
def left: BaseType
def right: BaseType
- def children = Seq(left, right)
+ def children: Seq[BaseType] = Seq(left, right)
}
/**
* A [[TreeNode]] with no children.
*/
trait LeafNode[BaseType <: TreeNode[BaseType]] {
- def children = Nil
+ def children: Seq[BaseType] = Nil
}
/**
@@ -435,6 +435,5 @@ trait LeafNode[BaseType <: TreeNode[BaseType]] {
*/
trait UnaryNode[BaseType <: TreeNode[BaseType]] {
def child: BaseType
- def children = child :: Nil
+ def children: Seq[BaseType] = child :: Nil
}
-
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala
index 79a8e06d4b..ea6aa1850d 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala
@@ -41,11 +41,11 @@ package object trees extends Logging {
* A [[TreeNode]] companion for reference equality for Hash based Collection.
*/
class TreeNodeRef(val obj: TreeNode[_]) {
- override def equals(o: Any) = o match {
+ override def equals(o: Any): Boolean = o match {
case that: TreeNodeRef => that.obj.eq(obj)
case _ => false
}
- override def hashCode = if (obj == null) 0 else obj.hashCode
+ override def hashCode: Int = if (obj == null) 0 else obj.hashCode
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala
index feed50f9a2..c86214a2aa 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala
@@ -23,7 +23,7 @@ import org.apache.spark.util.Utils
package object util {
- def fileToString(file: File, encoding: String = "UTF-8") = {
+ def fileToString(file: File, encoding: String = "UTF-8"): String = {
val inStream = new FileInputStream(file)
val outStream = new ByteArrayOutputStream
try {
@@ -45,7 +45,7 @@ package object util {
def resourceToString(
resource:String,
encoding: String = "UTF-8",
- classLoader: ClassLoader = Utils.getSparkClassLoader) = {
+ classLoader: ClassLoader = Utils.getSparkClassLoader): String = {
val inStream = classLoader.getResourceAsStream(resource)
val outStream = new ByteArrayOutputStream
try {
@@ -93,7 +93,7 @@ package object util {
new String(out.toByteArray)
}
- def stringOrNull(a: AnyRef) = if (a == null) null else a.toString
+ def stringOrNull(a: AnyRef): String = if (a == null) null else a.toString
def benchmark[A](f: => A): A = {
val startTime = System.nanoTime()