Skip to content

Commit 83f2a2f

Browse files
committed
[sql] Rename Expression.apply to eval for better readability.
Also used this opportunity to add a bunch of override's and made some members private. Author: Reynold Xin <[email protected]> Closes #340 from rxin/eval and squashes the following commits: a7c7ca7 [Reynold Xin] Fixed conflicts in merge. 9069de6 [Reynold Xin] Merge branch 'master' into eval 3ccc313 [Reynold Xin] Merge branch 'master' into eval 1a47e10 [Reynold Xin] Renamed apply to eval for generators and added a bunch of override's. ea061de [Reynold Xin] Rename Expression.apply to eval for better readability.
1 parent a3c51c6 commit 83f2a2f

File tree

24 files changed

+156
-159
lines changed

24 files changed

+156
-159
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -45,7 +45,7 @@ case class BoundReference(ordinal: Int, baseReference: Attribute)
4545

4646
override def toString = s"$baseReference:$ordinal"
4747

48-
override def apply(input: Row): Any = input(ordinal)
48+
override def eval(input: Row): Any = input(ordinal)
4949
}
5050

5151
/**

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -185,8 +185,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression {
185185
case DoubleType => castToDouble
186186
}
187187

188-
override def apply(input: Row): Any = {
189-
val evaluated = child.apply(input)
188+
override def eval(input: Row): Any = {
189+
val evaluated = child.eval(input)
190190
if (evaluated == null) {
191191
null
192192
} else {

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala

Lines changed: 13 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -17,8 +17,8 @@
1717

1818
package org.apache.spark.sql.catalyst.expressions
1919

20-
import org.apache.spark.sql.catalyst.trees
2120
import org.apache.spark.sql.catalyst.errors.TreeNodeException
21+
import org.apache.spark.sql.catalyst.trees
2222
import org.apache.spark.sql.catalyst.trees.TreeNode
2323
import org.apache.spark.sql.catalyst.types.{DataType, FractionalType, IntegralType, NumericType, NativeType}
2424

@@ -50,7 +50,7 @@ abstract class Expression extends TreeNode[Expression] {
5050
def references: Set[Attribute]
5151

5252
/** Returns the result of evaluating this expression on a given input Row */
53-
def apply(input: Row = null): EvaluatedType =
53+
def eval(input: Row = null): EvaluatedType =
5454
throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}")
5555

5656
/**
@@ -73,7 +73,7 @@ abstract class Expression extends TreeNode[Expression] {
7373
*/
7474
@inline
7575
def n1(e: Expression, i: Row, f: ((Numeric[Any], Any) => Any)): Any = {
76-
val evalE = e.apply(i)
76+
val evalE = e.eval(i)
7777
if (evalE == null) {
7878
null
7979
} else {
@@ -102,11 +102,11 @@ abstract class Expression extends TreeNode[Expression] {
102102
throw new TreeNodeException(this, s"Types do not match ${e1.dataType} != ${e2.dataType}")
103103
}
104104

105-
val evalE1 = e1.apply(i)
105+
val evalE1 = e1.eval(i)
106106
if(evalE1 == null) {
107107
null
108108
} else {
109-
val evalE2 = e2.apply(i)
109+
val evalE2 = e2.eval(i)
110110
if (evalE2 == null) {
111111
null
112112
} else {
@@ -135,11 +135,11 @@ abstract class Expression extends TreeNode[Expression] {
135135
throw new TreeNodeException(this, s"Types do not match ${e1.dataType} != ${e2.dataType}")
136136
}
137137

138-
val evalE1 = e1.apply(i: Row)
138+
val evalE1 = e1.eval(i: Row)
139139
if(evalE1 == null) {
140140
null
141141
} else {
142-
val evalE2 = e2.apply(i: Row)
142+
val evalE2 = e2.eval(i: Row)
143143
if (evalE2 == null) {
144144
null
145145
} else {
@@ -168,11 +168,11 @@ abstract class Expression extends TreeNode[Expression] {
168168
throw new TreeNodeException(this, s"Types do not match ${e1.dataType} != ${e2.dataType}")
169169
}
170170

171-
val evalE1 = e1.apply(i)
171+
val evalE1 = e1.eval(i)
172172
if(evalE1 == null) {
173173
null
174174
} else {
175-
val evalE2 = e2.apply(i)
175+
val evalE2 = e2.eval(i)
176176
if (evalE2 == null) {
177177
null
178178
} else {
@@ -205,11 +205,11 @@ abstract class Expression extends TreeNode[Expression] {
205205
throw new TreeNodeException(this, s"Types do not match ${e1.dataType} != ${e2.dataType}")
206206
}
207207

208-
val evalE1 = e1.apply(i)
208+
val evalE1 = e1.eval(i)
209209
if(evalE1 == null) {
210210
null
211211
} else {
212-
val evalE2 = e2.apply(i)
212+
val evalE2 = e2.eval(i)
213213
if (evalE2 == null) {
214214
null
215215
} else {
@@ -231,7 +231,7 @@ abstract class BinaryExpression extends Expression with trees.BinaryNode[Express
231231

232232
override def foldable = left.foldable && right.foldable
233233

234-
def references = left.references ++ right.references
234+
override def references = left.references ++ right.references
235235

236236
override def toString = s"($left $symbol $right)"
237237
}
@@ -243,5 +243,5 @@ abstract class LeafExpression extends Expression with trees.LeafNode[Expression]
243243
abstract class UnaryExpression extends Expression with trees.UnaryNode[Expression] {
244244
self: Product =>
245245

246-
def references = child.references
246+
override def references = child.references
247247
}

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -27,11 +27,12 @@ class Projection(expressions: Seq[Expression]) extends (Row => Row) {
2727
this(expressions.map(BindReferences.bindReference(_, inputSchema)))
2828

2929
protected val exprArray = expressions.toArray
30+
3031
def apply(input: Row): Row = {
3132
val outputArray = new Array[Any](exprArray.length)
3233
var i = 0
3334
while (i < exprArray.length) {
34-
outputArray(i) = exprArray(i).apply(input)
35+
outputArray(i) = exprArray(i).eval(input)
3536
i += 1
3637
}
3738
new GenericRow(outputArray)
@@ -58,7 +59,7 @@ case class MutableProjection(expressions: Seq[Expression]) extends (Row => Row)
5859
def apply(input: Row): Row = {
5960
var i = 0
6061
while (i < exprArray.length) {
61-
mutableRow(i) = exprArray(i).apply(input)
62+
mutableRow(i) = exprArray(i).eval(input)
6263
i += 1
6364
}
6465
mutableRow

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -212,8 +212,8 @@ class RowOrdering(ordering: Seq[SortOrder]) extends Ordering[Row] {
212212
var i = 0
213213
while (i < ordering.size) {
214214
val order = ordering(i)
215-
val left = order.child.apply(a)
216-
val right = order.child.apply(b)
215+
val left = order.child.eval(a)
216+
val right = order.child.eval(b)
217217

218218
if (left == null && right == null) {
219219
// Both null, continue looking.

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -27,13 +27,13 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi
2727
def references = children.flatMap(_.references).toSet
2828
def nullable = true
2929

30-
override def apply(input: Row): Any = {
30+
override def eval(input: Row): Any = {
3131
children.size match {
32-
case 1 => function.asInstanceOf[(Any) => Any](children(0).apply(input))
32+
case 1 => function.asInstanceOf[(Any) => Any](children(0).eval(input))
3333
case 2 =>
3434
function.asInstanceOf[(Any, Any) => Any](
35-
children(0).apply(input),
36-
children(1).apply(input))
35+
children(0).eval(input),
36+
children(1).eval(input))
3737
}
3838
}
3939
}

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -30,7 +30,7 @@ case class WrapDynamic(children: Seq[Attribute]) extends Expression {
3030
def references = children.toSet
3131
def dataType = DynamicType
3232

33-
override def apply(input: Row): DynamicRow = input match {
33+
override def eval(input: Row): DynamicRow = input match {
3434
// Avoid copy for generic rows.
3535
case g: GenericRow => new DynamicRow(children, g.values)
3636
case otherRowType => new DynamicRow(children, otherRowType.toArray)

0 commit comments

Comments
 (0)