diff options
author | Sameer Agarwal <sameer@databricks.com> | 2016-04-18 14:03:40 -0700 |
---|---|---|
committer | Reynold Xin <rxin@databricks.com> | 2016-04-18 14:03:40 -0700 |
commit | 8bd8121329cb1bb137e935dec124aa23f0fcf8c5 (patch) | |
tree | eb305c65cf776ad658515c56e4ed1cfddb5b6783 /sql | |
parent | 6fc1e72d9b70615bd91b598084406eb1893d6706 (diff) | |
download | spark-8bd8121329cb1bb137e935dec124aa23f0fcf8c5.tar.gz spark-8bd8121329cb1bb137e935dec124aa23f0fcf8c5.tar.bz2 spark-8bd8121329cb1bb137e935dec124aa23f0fcf8c5.zip |
[SPARK-14710][SQL] Rename gen/genCode to genCode/doGenCode to better reflect the semantics
## What changes were proposed in this pull request?
Per rxin's suggestions, this patch renames `s/gen/genCode` and `s/genCode/doGenCode` to better reflect the semantics of these 2 function calls.
## How was this patch tested?
N/A (refactoring only)
Author: Sameer Agarwal <sameer@databricks.com>
Closes #12475 from sameeragarwal/gencode.
Diffstat (limited to 'sql')
44 files changed, 305 insertions, 298 deletions
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 4ec43aba02..90b7b60b1c 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 @@ -153,7 +153,7 @@ case class UnresolvedGenerator(name: String, children: Seq[Expression]) extends override def eval(input: InternalRow = null): TraversableOnce[InternalRow] = throw new UnsupportedOperationException(s"Cannot evaluate expression: $this") - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = throw new UnsupportedOperationException(s"Cannot evaluate expression: $this") override def terminate(): TraversableOnce[InternalRow] = 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 c1fd23f28d..cf23884c44 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 @@ -58,7 +58,7 @@ case class BoundReference(ordinal: Int, dataType: DataType, nullable: Boolean) } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val javaType = ctx.javaType(dataType) val value = ctx.getValue(ctx.INPUT_ROW, dataType, ordinal.toString) if (ctx.currentVars != null && ctx.currentVars(ordinal) != null) { 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 0f8876a9e6..ffb100ee54 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 @@ -446,8 +446,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w protected override def nullSafeEval(input: Any): Any = cast(input) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val eval = child.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val eval = child.genCode(ctx) val nullSafeCast = nullSafeCastFunction(child.dataType, dataType, ctx) eval.code + castCode(ctx, eval.value, eval.isNull, ev.value, ev.isNull, dataType, nullSafeCast) 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 718bb4b118..11e3fd78d0 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 @@ -86,23 +86,23 @@ abstract class Expression extends TreeNode[Expression] { def eval(input: InternalRow = null): Any /** - * Returns an [[ExprCode]], which contains Java source code that - * can be used to generate the result of evaluating the expression on an input row. + * Returns an [[ExprCode]], that contains the Java source code to generate the result of + * evaluating the expression on an input row. * * @param ctx a [[CodegenContext]] * @return [[ExprCode]] */ - def gen(ctx: CodegenContext): ExprCode = { + def genCode(ctx: CodegenContext): ExprCode = { ctx.subExprEliminationExprs.get(this).map { subExprState => - // This expression is repeated meaning the code to evaluated has already been added - // as a function and called in advance. Just use it. + // This expression is repeated which means that the code to evaluate it has already been added + // as a function before. In that case, we just re-use it. val code = s"/* ${toCommentSafeString(this.toString)} */" ExprCode(code, subExprState.isNull, subExprState.value) }.getOrElse { val isNull = ctx.freshName("isNull") val value = ctx.freshName("value") val ve = ExprCode("", isNull, value) - ve.code = genCode(ctx, ve) + ve.code = doGenCode(ctx, ve) if (ve.code != "") { // Add `this` in the comment. ve.copy(s"/* ${toCommentSafeString(this.toString)} */\n" + ve.code.trim) @@ -121,7 +121,7 @@ abstract class Expression extends TreeNode[Expression] { * @param ev an [[ExprCode]] with unique terms. * @return Java source code */ - protected def genCode(ctx: CodegenContext, ev: ExprCode): String + protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String /** * Returns `true` if this expression and all its children have been resolved to a specific schema @@ -216,7 +216,7 @@ trait Unevaluable extends Expression { final override def eval(input: InternalRow = null): Any = throw new UnsupportedOperationException(s"Cannot evaluate expression: $this") - final override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = + final override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = throw new UnsupportedOperationException(s"Cannot evaluate expression: $this") } @@ -333,7 +333,7 @@ abstract class UnaryExpression extends Expression { ctx: CodegenContext, ev: ExprCode, f: String => String): String = { - val childGen = child.gen(ctx) + val childGen = child.genCode(ctx) val resultCode = f(childGen.value) if (nullable) { @@ -424,8 +424,8 @@ abstract class BinaryExpression extends Expression { ctx: CodegenContext, ev: ExprCode, f: (String, String) => String): String = { - val leftGen = left.gen(ctx) - val rightGen = right.gen(ctx) + val leftGen = left.genCode(ctx) + val rightGen = right.genCode(ctx) val resultCode = f(leftGen.value, rightGen.value) if (nullable) { @@ -566,9 +566,9 @@ abstract class TernaryExpression extends Expression { ctx: CodegenContext, ev: ExprCode, f: (String, String, String) => String): String = { - val leftGen = children(0).gen(ctx) - val midGen = children(1).gen(ctx) - val rightGen = children(2).gen(ctx) + val leftGen = children(0).genCode(ctx) + val midGen = children(1).genCode(ctx) + val rightGen = children(2).genCode(ctx) val resultCode = f(leftGen.value, midGen.value, rightGen.value) if (nullable) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InputFileName.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InputFileName.scala index 2ed6fc0d38..144efb751b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InputFileName.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InputFileName.scala @@ -43,7 +43,7 @@ case class InputFileName() extends LeafExpression with Nondeterministic { InputFileNameHolder.getInputFileName() } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { ev.isNull = "false" s"final ${ctx.javaType(dataType)} ${ev.value} = " + "org.apache.spark.rdd.InputFileNameHolder.getInputFileName();" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala index 5d28f8fbde..9d3e80cad6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala @@ -65,7 +65,7 @@ private[sql] case class MonotonicallyIncreasingID() extends LeafExpression with partitionMask + currentCount } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val countTerm = ctx.freshName("count") val partitionMaskTerm = ctx.freshName("partitionMask") ctx.addMutableState(ctx.JAVA_LONG, countTerm, s"$countTerm = 0L;") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ReferenceToExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ReferenceToExpressions.scala index 22645c952e..98710f8d78 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ReferenceToExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ReferenceToExpressions.scala @@ -59,15 +59,15 @@ case class ReferenceToExpressions(result: Expression, children: Seq[Expression]) result.eval(projection(input)) } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val childrenGen = children.map(_.gen(ctx)) + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val childrenGen = children.map(_.genCode(ctx)) val childrenVars = childrenGen.zip(children).map { case (childGen, child) => LambdaVariable(childGen.value, childGen.isNull, child.dataType) } val resultGen = result.transform { case b: BoundReference => childrenVars(b.ordinal) - }.gen(ctx) + }.genCode(ctx) ev.value = resultGen.value ev.isNull = resultGen.isNull 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 500ff447a9..1b19cdbadd 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 @@ -989,7 +989,7 @@ case class ScalaUDF( converterTerm } - override def genCode( + override def doGenCode( ctx: CodegenContext, ev: ExprCode): String = { @@ -1024,7 +1024,7 @@ case class ScalaUDF( s"[$funcExpressionIdx]).userDefinedFunc());") // codegen for children expressions - val evals = children.map(_.gen(ctx)) + val evals = children.map(_.genCode(ctx)) // Generate the codes for expressions and calling user-defined function // We need to get the boxedType of dataType's javaType here. Because for the dataType 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 b739361937..beced2c646 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 @@ -70,8 +70,8 @@ case class SortPrefix(child: SortOrder) extends UnaryExpression { override def eval(input: InternalRow): Any = throw new UnsupportedOperationException - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val childCode = child.child.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val childCode = child.child.genCode(ctx) val input = childCode.value val BinaryPrefixCmp = classOf[BinaryPrefixComparator].getName val DoublePrefixCmp = classOf[DoublePrefixComparator].getName diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala index 377f08eb10..8ca168a85b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala @@ -44,7 +44,7 @@ private[sql] case class SparkPartitionID() extends LeafExpression with Nondeterm override protected def evalInternal(input: InternalRow): Int = partitionId - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val idTerm = ctx.freshName("partitionId") ctx.addMutableState(ctx.JAVA_INT, idTerm, s"$idTerm = org.apache.spark.TaskContext.getPartitionId();") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala index daf3de95dd..46cbd12496 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala @@ -158,8 +158,8 @@ object TimeWindow { case class PreciseTimestamp(child: Expression) extends UnaryExpression with ExpectsInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) override def dataType: DataType = LongType - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val eval = child.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val eval = child.genCode(ctx) eval.code + s"""boolean ${ev.isNull} = ${eval.isNull}; |${ctx.javaType(dataType)} ${ev.value} = ${eval.value}; 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 f3d42fc0b2..25806c547b 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 @@ -36,7 +36,7 @@ case class UnaryMinus(child: Expression) extends UnaryExpression private lazy val numeric = TypeUtils.getNumeric(dataType) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = dataType match { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = dataType match { case dt: DecimalType => defineCodeGen(ctx, ev, c => s"$c.unary_$$minus()") case dt: NumericType => nullSafeCodeGen(ctx, ev, eval => { val originValue = ctx.freshName("origin") @@ -70,7 +70,7 @@ case class UnaryPositive(child: Expression) override def dataType: DataType = child.dataType - override def genCode(ctx: CodegenContext, ev: ExprCode): String = + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = defineCodeGen(ctx, ev, c => c) protected override def nullSafeEval(input: Any): Any = input @@ -93,7 +93,7 @@ case class Abs(child: Expression) private lazy val numeric = TypeUtils.getNumeric(dataType) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = dataType match { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = dataType match { case dt: DecimalType => defineCodeGen(ctx, ev, c => s"$c.abs()") case dt: NumericType => @@ -113,7 +113,7 @@ abstract class BinaryArithmetic extends BinaryOperator { def decimalMethod: String = sys.error("BinaryArithmetics must override either decimalMethod or genCode") - override def genCode(ctx: CodegenContext, ev: ExprCode): String = dataType match { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = dataType match { case dt: DecimalType => defineCodeGen(ctx, ev, (eval1, eval2) => s"$eval1.$decimalMethod($eval2)") // byte and short are casted into int when add, minus, times or divide @@ -147,7 +147,7 @@ case class Add(left: Expression, right: Expression) extends BinaryArithmetic wit } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = dataType match { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = dataType match { case dt: DecimalType => defineCodeGen(ctx, ev, (eval1, eval2) => s"$eval1.$$plus($eval2)") case ByteType | ShortType => @@ -179,7 +179,7 @@ case class Subtract(left: Expression, right: Expression) } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = dataType match { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = dataType match { case dt: DecimalType => defineCodeGen(ctx, ev, (eval1, eval2) => s"$eval1.$$minus($eval2)") case ByteType | ShortType => @@ -241,9 +241,9 @@ case class Divide(left: Expression, right: Expression) /** * Special case handling due to division by 0 => null. */ - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val eval1 = left.gen(ctx) - val eval2 = right.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val eval1 = left.genCode(ctx) + val eval2 = right.genCode(ctx) val isZero = if (dataType.isInstanceOf[DecimalType]) { s"${eval2.value}.isZero()" } else { @@ -320,9 +320,9 @@ case class Remainder(left: Expression, right: Expression) /** * Special case handling for x % 0 ==> null. */ - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val eval1 = left.gen(ctx) - val eval2 = right.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val eval1 = left.genCode(ctx) + val eval2 = right.genCode(ctx) val isZero = if (dataType.isInstanceOf[DecimalType]) { s"${eval2.value}.isZero()" } else { @@ -393,9 +393,9 @@ case class MaxOf(left: Expression, right: Expression) } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val eval1 = left.gen(ctx) - val eval2 = right.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val eval1 = left.genCode(ctx) + val eval2 = right.genCode(ctx) val compCode = ctx.genComp(dataType, eval1.value, eval2.value) eval1.code + eval2.code + s""" @@ -449,9 +449,9 @@ case class MinOf(left: Expression, right: Expression) } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val eval1 = left.gen(ctx) - val eval2 = right.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val eval1 = left.genCode(ctx) + val eval2 = right.genCode(ctx) val compCode = ctx.genComp(dataType, eval1.value, eval2.value) eval1.code + eval2.code + s""" @@ -503,7 +503,7 @@ case class Pmod(left: Expression, right: Expression) extends BinaryArithmetic wi case _: DecimalType => pmod(left.asInstanceOf[Decimal], right.asInstanceOf[Decimal]) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (eval1, eval2) => { dataType match { case dt: DecimalType => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwiseExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwiseExpressions.scala index a7e1cd66f2..8fd8a9bd4e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwiseExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwiseExpressions.scala @@ -130,7 +130,7 @@ case class BitwiseNot(child: Expression) extends UnaryExpression with ExpectsInp ((evalE: Long) => ~evalE).asInstanceOf[(Any) => Any] } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, c => s"(${ctx.javaType(dataType)}) ~($c)") } 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 f43626ca81..38ac13b208 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 @@ -526,7 +526,7 @@ class CodegenContext { val value = s"${fnName}Value" // Generate the code for this expression tree and wrap it in a function. - val code = expr.gen(this) + val code = expr.genCode(this) val fn = s""" |private void $fnName(InternalRow $INPUT_ROW) { @@ -572,7 +572,7 @@ class CodegenContext { def generateExpressions(expressions: Seq[Expression], doSubexpressionElimination: Boolean = false): Seq[ExprCode] = { if (doSubexpressionElimination) subexpressionElimination(expressions) - expressions.map(e => e.gen(this)) + expressions.map(e => e.genCode(this)) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenFallback.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenFallback.scala index 1365ee4b55..1e446c498d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenFallback.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenFallback.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.util.toCommentSafeString */ trait CodegenFallback extends Expression { - protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { foreach { case n: Nondeterministic => n.setInitialValues() case _ => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala index 908c32de4d..5635c91830 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala @@ -70,7 +70,7 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[InternalR */ def genComparisons(ctx: CodegenContext, ordering: Seq[SortOrder]): String = { val comparisons = ordering.map { order => - val eval = order.child.gen(ctx) + val eval = order.child.genCode(ctx) val asc = order.direction == Ascending val isNullA = ctx.freshName("isNullA") val primitiveA = ctx.freshName("primitiveA") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala index 58065d956f..dd8e2a289a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala @@ -39,7 +39,7 @@ object GeneratePredicate extends CodeGenerator[Expression, (InternalRow) => Bool protected def create(predicate: Expression): ((InternalRow) => Boolean) = { val ctx = newCodeGenContext() - val eval = predicate.gen(ctx) + val eval = predicate.genCode(ctx) val code = s""" public SpecificPredicate generate(Object[] references) { return new SpecificPredicate(references); diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala index cf73e36d22..7be57aca33 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala @@ -141,7 +141,7 @@ object GenerateSafeProjection extends CodeGenerator[Seq[Expression], Projection] val expressionCodes = expressions.zipWithIndex.map { case (NoOp, _) => "" case (e, i) => - val evaluationCode = e.gen(ctx) + val evaluationCode = e.genCode(ctx) val converter = convertToSafe(ctx, evaluationCode.value, e.dataType) evaluationCode.code + s""" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index ab790cf372..8cb691c9b1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -37,7 +37,7 @@ case class Size(child: Expression) extends UnaryExpression with ExpectsInputType case _: MapType => value.asInstanceOf[MapData].numElements() } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, c => s"${ev.value} = ($c).numElements();") } } @@ -180,7 +180,7 @@ case class ArrayContains(left: Expression, right: Expression) } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (arr, value) => { val i = ctx.freshName("i") val getValue = ctx.getValue(arr, right.dataType, i) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala index 74de4a776d..a7a59d8784 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala @@ -48,7 +48,7 @@ case class CreateArray(children: Seq[Expression]) extends Expression { new GenericArrayData(children.map(_.eval(input)).toArray) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val arrayClass = classOf[GenericArrayData].getName val values = ctx.freshName("values") s""" @@ -56,7 +56,7 @@ case class CreateArray(children: Seq[Expression]) extends Expression { final Object[] $values = new Object[${children.size}]; """ + children.zipWithIndex.map { case (e, i) => - val eval = e.gen(ctx) + val eval = e.genCode(ctx) eval.code + s""" if (${eval.isNull}) { $values[$i] = null; @@ -115,7 +115,7 @@ case class CreateMap(children: Seq[Expression]) extends Expression { new ArrayBasedMapData(new GenericArrayData(keyArray), new GenericArrayData(valueArray)) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val arrayClass = classOf[GenericArrayData].getName val mapClass = classOf[ArrayBasedMapData].getName val keyArray = ctx.freshName("keyArray") @@ -128,7 +128,7 @@ case class CreateMap(children: Seq[Expression]) extends Expression { final Object[] $valueArray = new Object[${values.size}]; """ + keys.zipWithIndex.map { case (key, i) => - val eval = key.gen(ctx) + val eval = key.genCode(ctx) s""" ${eval.code} if (${eval.isNull}) { @@ -139,7 +139,7 @@ case class CreateMap(children: Seq[Expression]) extends Expression { """ }.mkString("\n") + values.zipWithIndex.map { case (value, i) => - val eval = value.gen(ctx) + val eval = value.genCode(ctx) s""" ${eval.code} if (${eval.isNull}) { @@ -181,7 +181,7 @@ case class CreateStruct(children: Seq[Expression]) extends Expression { InternalRow(children.map(_.eval(input)): _*) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val rowClass = classOf[GenericInternalRow].getName val values = ctx.freshName("values") s""" @@ -189,7 +189,7 @@ case class CreateStruct(children: Seq[Expression]) extends Expression { final Object[] $values = new Object[${children.size}]; """ + children.zipWithIndex.map { case (e, i) => - val eval = e.gen(ctx) + val eval = e.genCode(ctx) eval.code + s""" if (${eval.isNull}) { $values[$i] = null; @@ -262,7 +262,7 @@ case class CreateNamedStruct(children: Seq[Expression]) extends Expression { InternalRow(valExprs.map(_.eval(input)): _*) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val rowClass = classOf[GenericInternalRow].getName val values = ctx.freshName("values") s""" @@ -270,7 +270,7 @@ case class CreateNamedStruct(children: Seq[Expression]) extends Expression { final Object[] $values = new Object[${valExprs.size}]; """ + valExprs.zipWithIndex.map { case (e, i) => - val eval = e.gen(ctx) + val eval = e.genCode(ctx) eval.code + s""" if (${eval.isNull}) { $values[$i] = null; @@ -314,7 +314,7 @@ case class CreateStructUnsafe(children: Seq[Expression]) extends Expression { InternalRow(children.map(_.eval(input)): _*) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val eval = GenerateUnsafeProjection.createCode(ctx, children) ev.isNull = eval.isNull ev.value = eval.value @@ -354,7 +354,7 @@ case class CreateNamedStructUnsafe(children: Seq[Expression]) extends Expression InternalRow(valExprs.map(_.eval(input)): _*) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val eval = GenerateUnsafeProjection.createCode(ctx, valExprs) ev.isNull = eval.isNull ev.value = eval.value diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala index c06dcc9867..b5ff9f55d5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala @@ -122,7 +122,7 @@ case class GetStructField(child: Expression, ordinal: Int, name: Option[String] protected override def nullSafeEval(input: Any): Any = input.asInstanceOf[InternalRow].get(ordinal, childSchema(ordinal).dataType) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, eval => { if (nullable) { s""" @@ -179,7 +179,7 @@ case class GetArrayStructFields( new GenericArrayData(result) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val arrayClass = classOf[GenericArrayData].getName nullSafeCodeGen(ctx, ev, eval => { val n = ctx.freshName("n") @@ -239,7 +239,7 @@ case class GetArrayItem(child: Expression, ordinal: Expression) } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (eval1, eval2) => { val index = ctx.freshName("index") s""" @@ -302,7 +302,7 @@ case class GetMapValue(child: Expression, key: Expression) } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val index = ctx.freshName("index") val length = ctx.freshName("length") val keys = ctx.freshName("keys") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala index ae6a94842f..a4c800a26c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala @@ -55,10 +55,10 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val condEval = predicate.gen(ctx) - val trueEval = trueValue.gen(ctx) - val falseEval = falseValue.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val condEval = predicate.genCode(ctx) + val trueEval = trueValue.genCode(ctx) + val falseEval = falseValue.genCode(ctx) s""" ${condEval.code} @@ -147,11 +147,11 @@ case class CaseWhen(branches: Seq[(Expression, Expression)], elseValue: Option[E branches.length < CaseWhen.MAX_NUM_CASES_FOR_CODEGEN } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { if (!shouldCodegen) { // Fallback to interpreted mode if there are too many branches, as it may reach the // 64K limit (limit on bytecode size for a single function). - return super[CodegenFallback].genCode(ctx, ev) + return super[CodegenFallback].doGenCode(ctx, ev) } // Generate code that looks like: // @@ -172,8 +172,8 @@ case class CaseWhen(branches: Seq[(Expression, Expression)], elseValue: Option[E // } // } val cases = branches.map { case (condExpr, valueExpr) => - val cond = condExpr.gen(ctx) - val res = valueExpr.gen(ctx) + val cond = condExpr.genCode(ctx) + val res = valueExpr.genCode(ctx) s""" ${cond.code} if (!${cond.isNull} && ${cond.value}) { @@ -187,7 +187,7 @@ case class CaseWhen(branches: Seq[(Expression, Expression)], elseValue: Option[E var generatedCode = cases.mkString("", "\nelse {\n", "\nelse {\n") elseValue.foreach { elseExpr => - val res = elseExpr.gen(ctx) + val res = elseExpr.genCode(ctx) generatedCode += s""" ${res.code} @@ -230,6 +230,7 @@ object CaseWhen { /** * A factory method to facilitate the creation of this expression when used in parsers. + * * @param branches Expressions at even position are the branch conditions, and expressions at odd * position are branch values. */ @@ -297,8 +298,8 @@ case class Least(children: Seq[Expression]) extends Expression { }) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val evalChildren = children.map(_.gen(ctx)) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val evalChildren = children.map(_.genCode(ctx)) val first = evalChildren(0) val rest = evalChildren.drop(1) def updateEval(eval: ExprCode): String = { @@ -358,8 +359,8 @@ case class Greatest(children: Seq[Expression]) extends Expression { }) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val evalChildren = children.map(_.gen(ctx)) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val evalChildren = children.map(_.genCode(ctx)) val first = evalChildren(0) val rest = evalChildren.drop(1) def updateEval(eval: ExprCode): String = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 9135753041..18649a39cb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -91,7 +91,7 @@ case class DateAdd(startDate: Expression, days: Expression) start.asInstanceOf[Int] + d.asInstanceOf[Int] } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (sd, d) => { s"""${ev.value} = $sd + $d;""" }) @@ -119,7 +119,7 @@ case class DateSub(startDate: Expression, days: Expression) start.asInstanceOf[Int] - d.asInstanceOf[Int] } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (sd, d) => { s"""${ev.value} = $sd - $d;""" }) @@ -141,7 +141,7 @@ case class Hour(child: Expression) extends UnaryExpression with ImplicitCastInpu DateTimeUtils.getHours(timestamp.asInstanceOf[Long]) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, c => s"$dtu.getHours($c)") } @@ -160,7 +160,7 @@ case class Minute(child: Expression) extends UnaryExpression with ImplicitCastIn DateTimeUtils.getMinutes(timestamp.asInstanceOf[Long]) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, c => s"$dtu.getMinutes($c)") } @@ -179,7 +179,7 @@ case class Second(child: Expression) extends UnaryExpression with ImplicitCastIn DateTimeUtils.getSeconds(timestamp.asInstanceOf[Long]) } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, c => s"$dtu.getSeconds($c)") } @@ -198,7 +198,7 @@ case class DayOfYear(child: Expression) extends UnaryExpression with ImplicitCas DateTimeUtils.getDayInYear(date.asInstanceOf[Int]) } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, c => s"$dtu.getDayInYear($c)") } @@ -217,7 +217,7 @@ case class Year(child: Expression) extends UnaryExpression with ImplicitCastInpu DateTimeUtils.getYear(date.asInstanceOf[Int]) } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, c => s"$dtu.getYear($c)") } @@ -235,7 +235,7 @@ case class Quarter(child: Expression) extends UnaryExpression with ImplicitCastI DateTimeUtils.getQuarter(date.asInstanceOf[Int]) } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, c => s"$dtu.getQuarter($c)") } @@ -254,7 +254,7 @@ case class Month(child: Expression) extends UnaryExpression with ImplicitCastInp DateTimeUtils.getMonth(date.asInstanceOf[Int]) } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, c => s"$dtu.getMonth($c)") } @@ -273,7 +273,7 @@ case class DayOfMonth(child: Expression) extends UnaryExpression with ImplicitCa DateTimeUtils.getDayOfMonth(date.asInstanceOf[Int]) } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, c => s"$dtu.getDayOfMonth($c)") } @@ -300,7 +300,7 @@ case class WeekOfYear(child: Expression) extends UnaryExpression with ImplicitCa c.get(Calendar.WEEK_OF_YEAR) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, time => { val cal = classOf[Calendar].getName val c = ctx.freshName("cal") @@ -335,7 +335,7 @@ case class DateFormatClass(left: Expression, right: Expression) extends BinaryEx UTF8String.fromString(sdf.format(new java.util.Date(timestamp.asInstanceOf[Long] / 1000))) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val sdf = classOf[SimpleDateFormat].getName defineCodeGen(ctx, ev, (timestamp, format) => { s"""UTF8String.fromString((new $sdf($format.toString())) @@ -430,7 +430,7 @@ abstract class UnixTime extends BinaryExpression with ExpectsInputTypes { } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { left.dataType match { case StringType if right.foldable => val sdf = classOf[SimpleDateFormat].getName @@ -442,7 +442,7 @@ abstract class UnixTime extends BinaryExpression with ExpectsInputTypes { ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)}; """ } else { - val eval1 = left.gen(ctx) + val eval1 = left.genCode(ctx) s""" ${eval1.code} boolean ${ev.isNull} = ${eval1.isNull}; @@ -471,7 +471,7 @@ abstract class UnixTime extends BinaryExpression with ExpectsInputTypes { """ }) case TimestampType => - val eval1 = left.gen(ctx) + val eval1 = left.genCode(ctx) s""" ${eval1.code} boolean ${ev.isNull} = ${eval1.isNull}; @@ -482,7 +482,7 @@ abstract class UnixTime extends BinaryExpression with ExpectsInputTypes { """ case DateType => val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") - val eval1 = left.gen(ctx) + val eval1 = left.genCode(ctx) s""" ${eval1.code} boolean ${ev.isNull} = ${eval1.isNull}; @@ -550,7 +550,7 @@ case class FromUnixTime(sec: Expression, format: Expression) } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val sdf = classOf[SimpleDateFormat].getName if (format.foldable) { if (constFormat == null) { @@ -559,7 +559,7 @@ case class FromUnixTime(sec: Expression, format: Expression) ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)}; """ } else { - val t = left.gen(ctx) + val t = left.genCode(ctx) s""" ${t.code} boolean ${ev.isNull} = ${t.isNull}; @@ -605,7 +605,7 @@ case class LastDay(startDate: Expression) extends UnaryExpression with ImplicitC DateTimeUtils.getLastDayOfMonth(date.asInstanceOf[Int]) } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, sd => s"$dtu.getLastDayOfMonth($sd)") } @@ -646,7 +646,7 @@ case class NextDay(startDate: Expression, dayOfWeek: Expression) } } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (sd, dowS) => { val dateTimeUtilClass = DateTimeUtils.getClass.getName.stripSuffix("$") val dayOfWeekTerm = ctx.freshName("dayOfWeek") @@ -698,7 +698,7 @@ case class TimeAdd(start: Expression, interval: Expression) start.asInstanceOf[Long], itvl.months, itvl.microseconds) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, (sd, i) => { s"""$dtu.timestampAddInterval($sd, $i.months, $i.microseconds)""" @@ -725,7 +725,7 @@ case class FromUTCTimestamp(left: Expression, right: Expression) timezone.asInstanceOf[UTF8String].toString) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") if (right.foldable) { val tz = right.eval() @@ -738,7 +738,7 @@ case class FromUTCTimestamp(left: Expression, right: Expression) val tzTerm = ctx.freshName("tz") val tzClass = classOf[TimeZone].getName ctx.addMutableState(tzClass, tzTerm, s"""$tzTerm = $tzClass.getTimeZone("$tz");""") - val eval = left.gen(ctx) + val eval = left.genCode(ctx) s""" |${eval.code} |boolean ${ev.isNull} = ${eval.isNull}; @@ -777,7 +777,7 @@ case class TimeSub(start: Expression, interval: Expression) start.asInstanceOf[Long], 0 - itvl.months, 0 - itvl.microseconds) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, (sd, i) => { s"""$dtu.timestampAddInterval($sd, 0 - $i.months, 0 - $i.microseconds)""" @@ -805,7 +805,7 @@ case class AddMonths(startDate: Expression, numMonths: Expression) DateTimeUtils.dateAddMonths(start.asInstanceOf[Int], months.asInstanceOf[Int]) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, (sd, m) => { s"""$dtu.dateAddMonths($sd, $m)""" @@ -835,7 +835,7 @@ case class MonthsBetween(date1: Expression, date2: Expression) DateTimeUtils.monthsBetween(t1.asInstanceOf[Long], t2.asInstanceOf[Long]) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, (l, r) => { s"""$dtu.monthsBetween($l, $r)""" @@ -864,7 +864,7 @@ case class ToUTCTimestamp(left: Expression, right: Expression) timezone.asInstanceOf[UTF8String].toString) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") if (right.foldable) { val tz = right.eval() @@ -877,7 +877,7 @@ case class ToUTCTimestamp(left: Expression, right: Expression) val tzTerm = ctx.freshName("tz") val tzClass = classOf[TimeZone].getName ctx.addMutableState(tzClass, tzTerm, s"""$tzTerm = $tzClass.getTimeZone("$tz");""") - val eval = left.gen(ctx) + val eval = left.genCode(ctx) s""" |${eval.code} |boolean ${ev.isNull} = ${eval.isNull}; @@ -912,7 +912,7 @@ case class ToDate(child: Expression) extends UnaryExpression with ImplicitCastIn override def eval(input: InternalRow): Any = child.eval(input) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, d => d) } @@ -959,7 +959,7 @@ case class TruncDate(date: Expression, format: Expression) } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") if (format.foldable) { @@ -969,7 +969,7 @@ case class TruncDate(date: Expression, format: Expression) ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)}; """ } else { - val d = date.gen(ctx) + val d = date.genCode(ctx) s""" ${d.code} boolean ${ev.isNull} = ${d.isNull}; @@ -1013,7 +1013,7 @@ case class DateDiff(endDate: Expression, startDate: Expression) end.asInstanceOf[Int] - start.asInstanceOf[Int] } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (end, start) => s"$end - $start") } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala index 74e86f40c0..5629ee1a14 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala @@ -34,7 +34,7 @@ case class UnscaledValue(child: Expression) extends UnaryExpression { protected override def nullSafeEval(input: Any): Any = input.asInstanceOf[Decimal].toUnscaledLong - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, c => s"$c.toUnscaledLong()") } } @@ -53,7 +53,7 @@ case class MakeDecimal(child: Expression, precision: Int, scale: Int) extends Un protected override def nullSafeEval(input: Any): Any = Decimal(input.asInstanceOf[Long], precision, scale) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, eval => { s""" ${ev.value} = (new Decimal()).setOrNull($eval, $precision, $scale); @@ -70,8 +70,8 @@ case class MakeDecimal(child: Expression, precision: Int, scale: Int) extends Un case class PromotePrecision(child: Expression) extends UnaryExpression { override def dataType: DataType = child.dataType override def eval(input: InternalRow): Any = child.eval(input) - override def gen(ctx: CodegenContext): ExprCode = child.gen(ctx) - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = "" + override def genCode(ctx: CodegenContext): ExprCode = child.genCode(ctx) + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = "" override def prettyName: String = "promote_precision" override def sql: String = child.sql } @@ -93,7 +93,7 @@ case class CheckOverflow(child: Expression, dataType: DecimalType) extends Unary } } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, eval => { val tmp = ctx.freshName("tmp") s""" 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 7fd4bc3066..bdadbfbbb0 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 @@ -191,7 +191,7 @@ case class Literal protected (value: Any, dataType: DataType) override def eval(input: InternalRow): Any = value - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { // change the isNull and primitive to consts, to inline them if (value == null) { ev.isNull = "true" @@ -205,7 +205,7 @@ case class Literal protected (value: Any, dataType: DataType) case FloatType => val v = value.asInstanceOf[Float] if (v.isNaN || v.isInfinite) { - super[CodegenFallback].genCode(ctx, ev) + super[CodegenFallback].doGenCode(ctx, ev) } else { ev.isNull = "false" ev.value = s"${value}f" @@ -214,7 +214,7 @@ case class Literal protected (value: Any, dataType: DataType) case DoubleType => val v = value.asInstanceOf[Double] if (v.isNaN || v.isInfinite) { - super[CodegenFallback].genCode(ctx, ev) + super[CodegenFallback].doGenCode(ctx, ev) } else { ev.isNull = "false" ev.value = s"${value}D" @@ -234,7 +234,7 @@ case class Literal protected (value: Any, dataType: DataType) "" // eval() version may be faster for non-primitive types case other => - super[CodegenFallback].genCode(ctx, ev) + super[CodegenFallback].doGenCode(ctx, ev) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala index 9e190289b7..231382e6bb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala @@ -70,7 +70,7 @@ abstract class UnaryMathExpression(val f: Double => Double, name: String) // name of function in java.lang.Math def funcName: String = name.toLowerCase - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, c => s"java.lang.Math.${funcName}($c)") } } @@ -88,7 +88,7 @@ abstract class UnaryLogExpression(f: Double => Double, name: String) if (d <= yAsymptote) null else f(d) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, c => s""" if ($c <= $yAsymptote) { @@ -123,7 +123,7 @@ abstract class BinaryMathExpression(f: (Double, Double) => Double, name: String) f(input1.asInstanceOf[Double], input2.asInstanceOf[Double]) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (c1, c2) => s"java.lang.Math.${name.toLowerCase}($c1, $c2)") } } @@ -197,7 +197,7 @@ case class Ceil(child: Expression) extends UnaryMathExpression(math.ceil, "CEIL" case DecimalType.Fixed(precision, scale) => input.asInstanceOf[Decimal].ceil } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { child.dataType match { case DecimalType.Fixed(_, 0) => defineCodeGen(ctx, ev, c => s"$c") case DecimalType.Fixed(precision, scale) => @@ -242,7 +242,7 @@ case class Conv(numExpr: Expression, fromBaseExpr: Expression, toBaseExpr: Expre toBase.asInstanceOf[Int]) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val numconv = NumberConverter.getClass.getName.stripSuffix("$") nullSafeCodeGen(ctx, ev, (num, from, to) => s""" @@ -284,7 +284,7 @@ case class Floor(child: Expression) extends UnaryMathExpression(math.floor, "FLO case DecimalType.Fixed(precision, scale) => input.asInstanceOf[Decimal].floor } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { child.dataType match { case DecimalType.Fixed(_, 0) => defineCodeGen(ctx, ev, c => s"$c") case DecimalType.Fixed(precision, scale) => @@ -346,7 +346,7 @@ case class Factorial(child: Expression) extends UnaryExpression with ImplicitCas } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, eval => { s""" if ($eval > 20 || $eval < 0) { @@ -370,7 +370,7 @@ case class Log(child: Expression) extends UnaryLogExpression(math.log, "LOG") extended = "> SELECT _FUNC_(2);\n 1.0") case class Log2(child: Expression) extends UnaryLogExpression((x: Double) => math.log(x) / math.log(2), "LOG2") { - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, c => s""" if ($c <= $yAsymptote) { @@ -458,7 +458,7 @@ case class Bin(child: Expression) protected override def nullSafeEval(input: Any): Any = UTF8String.fromString(jl.Long.toBinaryString(input.asInstanceOf[Long])) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (c) => s"UTF8String.fromString(java.lang.Long.toBinaryString($c))") } @@ -556,7 +556,7 @@ case class Hex(child: Expression) extends UnaryExpression with ImplicitCastInput case StringType => Hex.hex(num.asInstanceOf[UTF8String].getBytes) } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (c) => { val hex = Hex.getClass.getName.stripSuffix("$") s"${ev.value} = " + (child.dataType match { @@ -584,7 +584,7 @@ case class Unhex(child: Expression) extends UnaryExpression with ImplicitCastInp protected override def nullSafeEval(num: Any): Any = Hex.unhex(num.asInstanceOf[UTF8String].getBytes) - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (c) => { val hex = Hex.getClass.getName.stripSuffix("$") s""" @@ -613,7 +613,7 @@ case class Atan2(left: Expression, right: Expression) math.atan2(input1.asInstanceOf[Double] + 0.0, input2.asInstanceOf[Double] + 0.0) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (c1, c2) => s"java.lang.Math.atan2($c1 + 0.0, $c2 + 0.0)") } } @@ -623,7 +623,7 @@ case class Atan2(left: Expression, right: Expression) extended = "> SELECT _FUNC_(2, 3);\n 8.0") case class Pow(left: Expression, right: Expression) extends BinaryMathExpression(math.pow, "POWER") { - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (c1, c2) => s"java.lang.Math.pow($c1, $c2)") } } @@ -653,7 +653,7 @@ case class ShiftLeft(left: Expression, right: Expression) } } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (left, right) => s"$left << $right") } } @@ -683,7 +683,7 @@ case class ShiftRight(left: Expression, right: Expression) } } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (left, right) => s"$left >> $right") } } @@ -713,7 +713,7 @@ case class ShiftRightUnsigned(left: Expression, right: Expression) } } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (left, right) => s"$left >>> $right") } } @@ -753,7 +753,7 @@ case class Logarithm(left: Expression, right: Expression) if (dLeft <= 0.0 || dRight <= 0.0) null else math.log(dRight) / math.log(dLeft) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { if (left.isInstanceOf[EulerNumber]) { nullSafeCodeGen(ctx, ev, (c1, c2) => s""" @@ -874,8 +874,8 @@ abstract class RoundBase(child: Expression, scale: Expression, } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val ce = child.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val ce = child.genCode(ctx) val evaluationCode = child.dataType match { case _: DecimalType => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala index f2f0c2d698..8bef2524cc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala @@ -49,7 +49,7 @@ case class Md5(child: Expression) extends UnaryExpression with ImplicitCastInput protected override def nullSafeEval(input: Any): Any = UTF8String.fromString(DigestUtils.md5Hex(input.asInstanceOf[Array[Byte]])) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, c => s"UTF8String.fromString(org.apache.commons.codec.digest.DigestUtils.md5Hex($c))") } @@ -102,7 +102,7 @@ case class Sha2(left: Expression, right: Expression) } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val digestUtils = "org.apache.commons.codec.digest.DigestUtils" nullSafeCodeGen(ctx, ev, (eval1, eval2) => { s""" @@ -147,7 +147,7 @@ case class Sha1(child: Expression) extends UnaryExpression with ImplicitCastInpu protected override def nullSafeEval(input: Any): Any = UTF8String.fromString(DigestUtils.sha1Hex(input.asInstanceOf[Array[Byte]])) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, c => s"UTF8String.fromString(org.apache.commons.codec.digest.DigestUtils.sha1Hex($c))" ) @@ -173,7 +173,7 @@ case class Crc32(child: Expression) extends UnaryExpression with ImplicitCastInp checksum.getValue } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val CRC32 = "java.util.zip.CRC32" nullSafeCodeGen(ctx, ev, value => { s""" @@ -244,10 +244,10 @@ abstract class HashExpression[E] extends Expression { protected def computeHash(value: Any, dataType: DataType, seed: E): E - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { ev.isNull = "false" val childrenHash = children.map { child => - val childGen = child.gen(ctx) + val childGen = child.genCode(ctx) childGen.code + ctx.nullSafeExec(child.nullable, childGen.isNull) { computeHash(childGen.value, child.dataType, ev.value, ctx) } @@ -477,7 +477,7 @@ case class PrintToStderr(child: Expression) extends UnaryExpression { protected override def nullSafeEval(input: Any): Any = input - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, c => s""" | System.err.println("Result of ${child.simpleString} is " + $c); @@ -510,8 +510,8 @@ case class AssertTrue(child: Expression) extends UnaryExpression with ImplicitCa } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val eval = child.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val eval = child.genCode(ctx) ev.isNull = "true" ev.value = "null" s"""${eval.code} 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 78310fb2f1..b0434674c6 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 @@ -142,8 +142,8 @@ case class Alias(child: Expression, name: String)( override def eval(input: InternalRow): Any = child.eval(input) /** Just a simple passthrough for code generation. */ - override def gen(ctx: CodegenContext): ExprCode = child.gen(ctx) - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = "" + override def genCode(ctx: CodegenContext): ExprCode = child.genCode(ctx) + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = "" override def dataType: DataType = child.dataType override def nullable: Boolean = child.nullable diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala index 6a45249943..d9c06e3b99 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala @@ -64,17 +64,17 @@ case class Coalesce(children: Seq[Expression]) extends Expression { result } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val first = children(0) val rest = children.drop(1) - val firstEval = first.gen(ctx) + val firstEval = first.genCode(ctx) s""" ${firstEval.code} boolean ${ev.isNull} = ${firstEval.isNull}; ${ctx.javaType(dataType)} ${ev.value} = ${firstEval.value}; """ + rest.map { e => - val eval = e.gen(ctx) + val eval = e.genCode(ctx) s""" if (${ev.isNull}) { ${eval.code} @@ -113,8 +113,8 @@ case class IsNaN(child: Expression) extends UnaryExpression } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val eval = child.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val eval = child.genCode(ctx) child.dataType match { case DoubleType | FloatType => s""" @@ -155,9 +155,9 @@ case class NaNvl(left: Expression, right: Expression) } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val leftGen = left.gen(ctx) - val rightGen = right.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val leftGen = left.genCode(ctx) + val rightGen = right.genCode(ctx) left.dataType match { case DoubleType | FloatType => s""" @@ -196,8 +196,8 @@ case class IsNull(child: Expression) extends UnaryExpression with Predicate { child.eval(input) == null } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val eval = child.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val eval = child.genCode(ctx) ev.isNull = "false" ev.value = eval.isNull eval.code @@ -219,8 +219,8 @@ case class IsNotNull(child: Expression) extends UnaryExpression with Predicate { child.eval(input) != null } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val eval = child.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val eval = child.genCode(ctx) ev.isNull = "false" ev.value = s"(!(${eval.isNull}))" eval.code @@ -259,10 +259,10 @@ case class AtLeastNNonNulls(n: Int, children: Seq[Expression]) extends Predicate numNonNulls >= n } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val nonnull = ctx.freshName("nonnull") val code = children.map { e => - val eval = e.gen(ctx) + val eval = e.genCode(ctx) e.dataType match { case DoubleType | FloatType => s""" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala index 26b1ff39b3..f5f102a578 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala @@ -59,9 +59,9 @@ case class StaticInvoke( override def eval(input: InternalRow): Any = throw new UnsupportedOperationException("Only code-generated evaluation is supported.") - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val javaType = ctx.javaType(dataType) - val argGen = arguments.map(_.gen(ctx)) + val argGen = arguments.map(_.genCode(ctx)) val argString = argGen.map(_.value).mkString(", ") if (propagateNull) { @@ -148,10 +148,10 @@ case class Invoke( case _ => identity[String] _ } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val javaType = ctx.javaType(dataType) - val obj = targetObject.gen(ctx) - val argGen = arguments.map(_.gen(ctx)) + val obj = targetObject.genCode(ctx) + val argGen = arguments.map(_.genCode(ctx)) val argString = argGen.map(_.value).mkString(", ") // If the function can return null, we do an extra check to make sure our null bit is still set @@ -239,12 +239,12 @@ case class NewInstance( override def eval(input: InternalRow): Any = throw new UnsupportedOperationException("Only code-generated evaluation is supported.") - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val javaType = ctx.javaType(dataType) - val argGen = arguments.map(_.gen(ctx)) + val argGen = arguments.map(_.genCode(ctx)) val argString = argGen.map(_.value).mkString(", ") - val outer = outerPointer.map(func => Literal.fromObject(func()).gen(ctx)) + val outer = outerPointer.map(func => Literal.fromObject(func()).genCode(ctx)) val setup = s""" @@ -302,9 +302,9 @@ case class UnwrapOption( override def eval(input: InternalRow): Any = throw new UnsupportedOperationException("Only code-generated evaluation is supported") - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val javaType = ctx.javaType(dataType) - val inputObject = child.gen(ctx) + val inputObject = child.genCode(ctx) s""" ${inputObject.code} @@ -319,6 +319,7 @@ case class UnwrapOption( /** * Converts the result of evaluating `child` into an option, checking both the isNull bit and * (in the case of reference types) equality with null. + * * @param child The expression to evaluate and wrap. * @param optType The type of this option. */ @@ -334,8 +335,8 @@ case class WrapOption(child: Expression, optType: DataType) override def eval(input: InternalRow): Any = throw new UnsupportedOperationException("Only code-generated evaluation is supported") - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val inputObject = child.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val inputObject = child.genCode(ctx) s""" ${inputObject.code} @@ -357,7 +358,7 @@ case class LambdaVariable(value: String, isNull: String, dataType: DataType) ext override def nullable: Boolean = true - override def gen(ctx: CodegenContext): ExprCode = { + override def genCode(ctx: CodegenContext): ExprCode = { ExprCode(code = "", value = value, isNull = isNull) } } @@ -443,13 +444,13 @@ case class MapObjects private( override def dataType: DataType = ArrayType(lambdaFunction.dataType) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val javaType = ctx.javaType(dataType) val elementJavaType = ctx.javaType(loopVar.dataType) ctx.addMutableState("boolean", loopVar.isNull, "") ctx.addMutableState(elementJavaType, loopVar.value, "") - val genInputData = inputData.gen(ctx) - val genFunction = lambdaFunction.gen(ctx) + val genInputData = inputData.genCode(ctx) + val genFunction = lambdaFunction.genCode(ctx) val dataLength = ctx.freshName("dataLength") val convertedArray = ctx.freshName("convertedArray") val loopIndex = ctx.freshName("loopIndex") @@ -523,13 +524,13 @@ case class CreateExternalRow(children: Seq[Expression], schema: StructType) override def eval(input: InternalRow): Any = throw new UnsupportedOperationException("Only code-generated evaluation is supported") - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val rowClass = classOf[GenericRowWithSchema].getName val values = ctx.freshName("values") ctx.addMutableState("Object[]", values, "") val childrenCodes = children.zipWithIndex.map { case (e, i) => - val eval = e.gen(ctx) + val eval = e.genCode(ctx) eval.code + s""" if (${eval.isNull}) { $values[$i] = null; @@ -551,6 +552,7 @@ case class CreateExternalRow(children: Seq[Expression], schema: StructType) /** * Serializes an input object using a generic serializer (Kryo or Java). + * * @param kryo if true, use Kryo. Otherwise, use Java. */ case class EncodeUsingSerializer(child: Expression, kryo: Boolean) @@ -559,7 +561,7 @@ case class EncodeUsingSerializer(child: Expression, kryo: Boolean) override def eval(input: InternalRow): Any = throw new UnsupportedOperationException("Only code-generated evaluation is supported") - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { // Code to initialize the serializer. val serializer = ctx.freshName("serializer") val (serializerClass, serializerInstanceClass) = { @@ -576,7 +578,7 @@ case class EncodeUsingSerializer(child: Expression, kryo: Boolean) s"$serializer = ($serializerInstanceClass) new $serializerClass($sparkConf).newInstance();") // Code to serialize. - val input = child.gen(ctx) + val input = child.genCode(ctx) s""" ${input.code} final boolean ${ev.isNull} = ${input.isNull}; @@ -593,12 +595,13 @@ case class EncodeUsingSerializer(child: Expression, kryo: Boolean) /** * Serializes an input object using a generic serializer (Kryo or Java). Note that the ClassTag * is not an implicit parameter because TreeNode cannot copy implicit parameters. + * * @param kryo if true, use Kryo. Otherwise, use Java. */ case class DecodeUsingSerializer[T](child: Expression, tag: ClassTag[T], kryo: Boolean) extends UnaryExpression with NonSQLExpression { - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { // Code to initialize the serializer. val serializer = ctx.freshName("serializer") val (serializerClass, serializerInstanceClass) = { @@ -615,7 +618,7 @@ case class DecodeUsingSerializer[T](child: Expression, tag: ClassTag[T], kryo: B s"$serializer = ($serializerInstanceClass) new $serializerClass($sparkConf).newInstance();") // Code to serialize. - val input = child.gen(ctx) + val input = child.genCode(ctx) s""" ${input.code} final boolean ${ev.isNull} = ${input.isNull}; @@ -643,12 +646,12 @@ case class InitializeJavaBean(beanInstance: Expression, setters: Map[String, Exp override def eval(input: InternalRow): Any = throw new UnsupportedOperationException("Only code-generated evaluation is supported.") - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val instanceGen = beanInstance.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val instanceGen = beanInstance.genCode(ctx) val initialize = setters.map { case (setterMethod, fieldValue) => - val fieldGen = fieldValue.gen(ctx) + val fieldGen = fieldValue.genCode(ctx) s""" ${fieldGen.code} ${instanceGen.value}.$setterMethod(${fieldGen.value}); @@ -685,8 +688,8 @@ case class AssertNotNull(child: Expression, walkedTypePath: Seq[String]) override def eval(input: InternalRow): Any = throw new UnsupportedOperationException("Only code-generated evaluation is supported.") - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val childGen = child.gen(ctx) + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val childGen = child.genCode(ctx) val errMsg = "Null value appeared in non-nullable field:" + walkedTypePath.mkString("\n", "\n", "\n") + 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 38f1210a4e..b15a77a8e7 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 @@ -99,7 +99,7 @@ case class Not(child: Expression) protected override def nullSafeEval(input: Any): Any = !input.asInstanceOf[Boolean] - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, c => s"!($c)") } @@ -157,9 +157,9 @@ case class In(value: Expression, list: Seq[Expression]) extends Predicate } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val valueGen = value.gen(ctx) - val listGen = list.map(_.gen(ctx)) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val valueGen = value.genCode(ctx) + val listGen = list.map(_.genCode(ctx)) val listCode = listGen.map(x => s""" if (!${ev.value}) { @@ -216,10 +216,10 @@ case class InSet(child: Expression, hset: Set[Any]) extends UnaryExpression with def getHSet(): Set[Any] = hset - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val setName = classOf[Set[Any]].getName val InSetName = classOf[InSet].getName - val childGen = child.gen(ctx) + val childGen = child.genCode(ctx) ctx.references += this val hsetTerm = ctx.freshName("hset") val hasNullTerm = ctx.freshName("hasNull") @@ -274,9 +274,9 @@ case class And(left: Expression, right: Expression) extends BinaryOperator with } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val eval1 = left.gen(ctx) - val eval2 = right.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val eval1 = left.genCode(ctx) + val eval2 = right.genCode(ctx) // The result should be `false`, if any of them is `false` whenever the other is null or not. if (!left.nullable && !right.nullable) { @@ -339,9 +339,9 @@ case class Or(left: Expression, right: Expression) extends BinaryOperator with P } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val eval1 = left.gen(ctx) - val eval2 = right.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val eval1 = left.genCode(ctx) + val eval2 = right.genCode(ctx) // The result should be `true`, if any of them is `true` whenever the other is null or not. if (!left.nullable && !right.nullable) { @@ -379,7 +379,7 @@ case class Or(left: Expression, right: Expression) extends BinaryOperator with P abstract class BinaryComparison extends BinaryOperator with Predicate { - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { if (ctx.isPrimitiveType(left.dataType) && left.dataType != BooleanType // java boolean doesn't support > or < operator && left.dataType != FloatType @@ -428,7 +428,7 @@ case class EqualTo(left: Expression, right: Expression) } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (c1, c2) => ctx.genEqual(left.dataType, c1, c2)) } } @@ -464,9 +464,9 @@ case class EqualNullSafe(left: Expression, right: Expression) extends BinaryComp } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val eval1 = left.gen(ctx) - val eval2 = right.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val eval1 = left.genCode(ctx) + val eval2 = right.genCode(ctx) val equalCode = ctx.genEqual(left.dataType, eval1.value, eval2.value) ev.isNull = "false" eval1.code + eval2.code + s""" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala index 1ec092a5be..1eed24dd1e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala @@ -67,7 +67,7 @@ case class Rand(seed: Long) extends RDG { case _ => throw new AnalysisException("Input argument to rand must be an integer literal.") }) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val rngTerm = ctx.freshName("rng") val className = classOf[XORShiftRandom].getName ctx.addMutableState(className, rngTerm, @@ -92,7 +92,7 @@ case class Randn(seed: Long) extends RDG { case _ => throw new AnalysisException("Input argument to randn must be an integer literal.") }) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val rngTerm = ctx.freshName("rng") val className = classOf[XORShiftRandom].getName ctx.addMutableState(className, rngTerm, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala index 85a5429263..4f5b85d7f4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala @@ -78,7 +78,7 @@ case class Like(left: Expression, right: Expression) override def toString: String = s"$left LIKE $right" - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val patternClass = classOf[Pattern].getName val escapeFunc = StringUtils.getClass.getName.stripSuffix("$") + ".escapeLikeRegex" val pattern = ctx.freshName("pattern") @@ -92,7 +92,7 @@ case class Like(left: Expression, right: Expression) s"""$pattern = ${patternClass}.compile("$regexStr");""") // We don't use nullSafeCodeGen here because we don't want to re-evaluate right again. - val eval = left.gen(ctx) + val eval = left.genCode(ctx) s""" ${eval.code} boolean ${ev.isNull} = ${eval.isNull}; @@ -128,7 +128,7 @@ case class RLike(left: Expression, right: Expression) override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).find(0) override def toString: String = s"$left RLIKE $right" - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val patternClass = classOf[Pattern].getName val pattern = ctx.freshName("pattern") @@ -141,7 +141,7 @@ case class RLike(left: Expression, right: Expression) s"""$pattern = ${patternClass}.compile("$regexStr");""") // We don't use nullSafeCodeGen here because we don't want to re-evaluate right again. - val eval = left.gen(ctx) + val eval = left.genCode(ctx) s""" ${eval.code} boolean ${ev.isNull} = ${eval.isNull}; @@ -188,7 +188,7 @@ case class StringSplit(str: Expression, pattern: Expression) new GenericArrayData(strings.asInstanceOf[Array[Any]]) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val arrayClass = classOf[GenericArrayData].getName nullSafeCodeGen(ctx, ev, (str, pattern) => // Array in java is covariant, so we don't need to cast UTF8String[] to Object[]. @@ -247,7 +247,7 @@ case class RegExpReplace(subject: Expression, regexp: Expression, rep: Expressio override def children: Seq[Expression] = subject :: regexp :: rep :: Nil override def prettyName: String = "regexp_replace" - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val termLastRegex = ctx.freshName("lastRegex") val termPattern = ctx.freshName("pattern") @@ -330,7 +330,7 @@ case class RegExpExtract(subject: Expression, regexp: Expression, idx: Expressio override def children: Seq[Expression] = subject :: regexp :: idx :: Nil override def prettyName: String = "regexp_extract" - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val termLastRegex = ctx.freshName("lastRegex") val termPattern = ctx.freshName("pattern") val classNamePattern = classOf[Pattern].getCanonicalName diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala index a17482697d..8c15357360 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala @@ -51,8 +51,8 @@ case class Concat(children: Seq[Expression]) extends Expression with ImplicitCas UTF8String.concat(inputs : _*) } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val evals = children.map(_.gen(ctx)) + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val evals = children.map(_.genCode(ctx)) val inputs = evals.map { eval => s"${eval.isNull} ? null : ${eval.value}" }.mkString(", ") @@ -106,10 +106,10 @@ case class ConcatWs(children: Seq[Expression]) UTF8String.concatWs(flatInputs.head, flatInputs.tail : _*) } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { if (children.forall(_.dataType == StringType)) { // All children are strings. In that case we can construct a fixed size array. - val evals = children.map(_.gen(ctx)) + val evals = children.map(_.genCode(ctx)) val inputs = evals.map { eval => s"${eval.isNull} ? (UTF8String) null : ${eval.value}" @@ -124,7 +124,7 @@ case class ConcatWs(children: Seq[Expression]) val varargNum = ctx.freshName("varargNum") val idxInVararg = ctx.freshName("idxInVararg") - val evals = children.map(_.gen(ctx)) + val evals = children.map(_.genCode(ctx)) val (varargCount, varargBuild) = children.tail.zip(evals.tail).map { case (child, eval) => child.dataType match { case StringType => @@ -185,7 +185,7 @@ case class Upper(child: Expression) override def convert(v: UTF8String): UTF8String = v.toUpperCase - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, c => s"($c).toUpperCase()") } } @@ -200,7 +200,7 @@ case class Lower(child: Expression) extends UnaryExpression with String2StringEx override def convert(v: UTF8String): UTF8String = v.toLowerCase - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, c => s"($c).toLowerCase()") } } @@ -225,7 +225,7 @@ trait StringPredicate extends Predicate with ImplicitCastInputTypes { case class Contains(left: Expression, right: Expression) extends BinaryExpression with StringPredicate { override def compare(l: UTF8String, r: UTF8String): Boolean = l.contains(r) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (c1, c2) => s"($c1).contains($c2)") } } @@ -236,7 +236,7 @@ case class Contains(left: Expression, right: Expression) case class StartsWith(left: Expression, right: Expression) extends BinaryExpression with StringPredicate { override def compare(l: UTF8String, r: UTF8String): Boolean = l.startsWith(r) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (c1, c2) => s"($c1).startsWith($c2)") } } @@ -247,7 +247,7 @@ case class StartsWith(left: Expression, right: Expression) case class EndsWith(left: Expression, right: Expression) extends BinaryExpression with StringPredicate { override def compare(l: UTF8String, r: UTF8String): Boolean = l.endsWith(r) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (c1, c2) => s"($c1).endsWith($c2)") } } @@ -298,7 +298,7 @@ case class StringTranslate(srcExpr: Expression, matchingExpr: Expression, replac srcEval.asInstanceOf[UTF8String].translate(dict) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val termLastMatching = ctx.freshName("lastMatching") val termLastReplace = ctx.freshName("lastReplace") val termDict = ctx.freshName("dict") @@ -351,7 +351,7 @@ case class FindInSet(left: Expression, right: Expression) extends BinaryExpressi override protected def nullSafeEval(word: Any, set: Any): Any = set.asInstanceOf[UTF8String].findInSet(word.asInstanceOf[UTF8String]) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (word, set) => s"${ev.value} = $set.findInSet($word);" ) @@ -375,7 +375,7 @@ case class StringTrim(child: Expression) override def prettyName: String = "trim" - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, c => s"($c).trim()") } } @@ -393,7 +393,7 @@ case class StringTrimLeft(child: Expression) override def prettyName: String = "ltrim" - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, c => s"($c).trimLeft()") } } @@ -411,7 +411,7 @@ case class StringTrimRight(child: Expression) override def prettyName: String = "rtrim" - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, c => s"($c).trimRight()") } } @@ -440,7 +440,7 @@ case class StringInstr(str: Expression, substr: Expression) override def prettyName: String = "instr" - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (l, r) => s"($l).indexOf($r, 0) + 1") } @@ -475,7 +475,7 @@ case class SubstringIndex(strExpr: Expression, delimExpr: Expression, countExpr: count.asInstanceOf[Int]) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (str, delim, count) => s"$str.subStringIndex($delim, $count)") } } @@ -524,10 +524,10 @@ case class StringLocate(substr: Expression, str: Expression, start: Expression) } } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val substrGen = substr.gen(ctx) - val strGen = str.gen(ctx) - val startGen = start.gen(ctx) + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val substrGen = substr.genCode(ctx) + val strGen = str.genCode(ctx) + val startGen = start.genCode(ctx) s""" int ${ev.value} = 0; boolean ${ev.isNull} = false; @@ -571,7 +571,7 @@ case class StringLPad(str: Expression, len: Expression, pad: Expression) str.asInstanceOf[UTF8String].lpad(len.asInstanceOf[Int], pad.asInstanceOf[UTF8String]) } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (str, len, pad) => s"$str.lpad($len, $pad)") } @@ -597,7 +597,7 @@ case class StringRPad(str: Expression, len: Expression, pad: Expression) str.asInstanceOf[UTF8String].rpad(len.asInstanceOf[Int], pad.asInstanceOf[UTF8String]) } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (str, len, pad) => s"$str.rpad($len, $pad)") } @@ -638,10 +638,10 @@ case class FormatString(children: Expression*) extends Expression with ImplicitC } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val pattern = children.head.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val pattern = children.head.genCode(ctx) - val argListGen = children.tail.map(x => (x.dataType, x.gen(ctx))) + val argListGen = children.tail.map(x => (x.dataType, x.genCode(ctx))) val argListCode = argListGen.map(_._2.code + "\n") val argListString = argListGen.foldLeft("")((s, v) => { @@ -694,7 +694,7 @@ case class InitCap(child: Expression) extends UnaryExpression with ImplicitCastI override def nullSafeEval(string: Any): Any = { string.asInstanceOf[UTF8String].toLowerCase.toTitleCase } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, str => s"$str.toLowerCase().toTitleCase()") } } @@ -719,7 +719,7 @@ case class StringRepeat(str: Expression, times: Expression) override def prettyName: String = "repeat" - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (l, r) => s"($l).repeat($r)") } } @@ -735,7 +735,7 @@ case class StringReverse(child: Expression) extends UnaryExpression with String2 override def prettyName: String = "reverse" - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, c => s"($c).reverse()") } } @@ -757,7 +757,7 @@ case class StringSpace(child: Expression) UTF8String.blankString(if (length < 0) 0 else length) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (length) => s"""${ev.value} = UTF8String.blankString(($length < 0) ? 0 : $length);""") } @@ -799,7 +799,7 @@ case class Substring(str: Expression, pos: Expression, len: Expression) } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (string, pos, len) => { str.dataType match { @@ -825,7 +825,7 @@ case class Length(child: Expression) extends UnaryExpression with ExpectsInputTy case BinaryType => value.asInstanceOf[Array[Byte]].length } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { child.dataType match { case StringType => defineCodeGen(ctx, ev, c => s"($c).numChars()") case BinaryType => defineCodeGen(ctx, ev, c => s"($c).length") @@ -848,7 +848,7 @@ case class Levenshtein(left: Expression, right: Expression) extends BinaryExpres protected override def nullSafeEval(leftValue: Any, rightValue: Any): Any = leftValue.asInstanceOf[UTF8String].levenshteinDistance(rightValue.asInstanceOf[UTF8String]) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (left, right) => s"${ev.value} = $left.levenshteinDistance($right);") } @@ -868,7 +868,7 @@ case class SoundEx(child: Expression) extends UnaryExpression with ExpectsInputT override def nullSafeEval(input: Any): Any = input.asInstanceOf[UTF8String].soundex() - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, c => s"$c.soundex()") } } @@ -894,7 +894,7 @@ case class Ascii(child: Expression) extends UnaryExpression with ImplicitCastInp } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (child) => { val bytes = ctx.freshName("bytes") s""" @@ -924,7 +924,7 @@ case class Base64(child: Expression) extends UnaryExpression with ImplicitCastIn bytes.asInstanceOf[Array[Byte]])) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (child) => { s"""${ev.value} = UTF8String.fromBytes( org.apache.commons.codec.binary.Base64.encodeBase64($child)); @@ -945,7 +945,7 @@ case class UnBase64(child: Expression) extends UnaryExpression with ImplicitCast protected override def nullSafeEval(string: Any): Any = org.apache.commons.codec.binary.Base64.decodeBase64(string.asInstanceOf[UTF8String].toString) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (child) => { s""" ${ev.value} = org.apache.commons.codec.binary.Base64.decodeBase64($child.toString()); @@ -973,7 +973,7 @@ case class Decode(bin: Expression, charset: Expression) UTF8String.fromString(new String(input1.asInstanceOf[Array[Byte]], fromCharset)) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (bytes, charset) => s""" try { @@ -1005,7 +1005,7 @@ case class Encode(value: Expression, charset: Expression) input1.asInstanceOf[UTF8String].toString.getBytes(toCharset) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (string, charset) => s""" try { @@ -1088,7 +1088,7 @@ case class FormatNumber(x: Expression, d: Expression) } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (num, d) => { def typeHelper(p: String): String = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NonFoldableLiteral.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NonFoldableLiteral.scala index ff34b1e37b..de410b86ea 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NonFoldableLiteral.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NonFoldableLiteral.scala @@ -35,8 +35,8 @@ case class NonFoldableLiteral(value: Any, dataType: DataType) extends LeafExpres override def eval(input: InternalRow): Any = value - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - Literal.create(value, dataType).genCode(ctx, ev) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + Literal.create(value, dataType).doGenCode(ctx, ev) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index 392c48fb7b..3dc2aa33df 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -192,7 +192,7 @@ private[sql] case class RowDataSourceScan( val row = ctx.freshName("row") ctx.INPUT_ROW = row ctx.currentVars = null - val columnsRowInput = exprRows.map(_.gen(ctx)) + val columnsRowInput = exprRows.map(_.genCode(ctx)) val inputRow = if (outputUnsafeRows) row else null s""" |while ($input.hasNext()) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Expand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Expand.scala index bd23b7e3ad..cc0382e5d4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Expand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Expand.scala @@ -149,7 +149,7 @@ case class Expand( val firstExpr = projections.head(col) if (sameOutput(col)) { // This column is the same across all output rows. Just generate code for it here. - BindReferences.bindReference(firstExpr, child.output).gen(ctx) + BindReferences.bindReference(firstExpr, child.output).genCode(ctx) } else { val isNull = ctx.freshName("isNull") val value = ctx.freshName("value") @@ -166,7 +166,7 @@ case class Expand( var updateCode = "" for (col <- exprs.indices) { if (!sameOutput(col)) { - val ev = BindReferences.bindReference(exprs(col), child.output).gen(ctx) + val ev = BindReferences.bindReference(exprs(col), child.output).genCode(ctx) updateCode += s""" |${ev.code} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala index 29acc38ab3..12d08c8c45 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala @@ -118,7 +118,7 @@ trait CodegenSupport extends SparkPlan { ctx.currentVars = null ctx.INPUT_ROW = row output.zipWithIndex.map { case (attr, i) => - BoundReference(i, attr.dataType, attr.nullable).gen(ctx) + BoundReference(i, attr.dataType, attr.nullable).genCode(ctx) } } else { assert(outputVars != null) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala index f585759e58..d819a65993 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala @@ -165,7 +165,7 @@ case class TungstenAggregate( ctx.addMutableState("boolean", isNull, "") ctx.addMutableState(ctx.javaType(e.dataType), value, "") // The initial expression should not access any column - val ev = e.gen(ctx) + val ev = e.genCode(ctx) val initVars = s""" | $isNull = ${ev.isNull}; | $value = ${ev.value}; @@ -179,13 +179,13 @@ case class TungstenAggregate( // evaluate aggregate results ctx.currentVars = bufVars val aggResults = functions.map(_.evaluateExpression).map { e => - BindReferences.bindReference(e, aggregateBufferAttributes).gen(ctx) + BindReferences.bindReference(e, aggregateBufferAttributes).genCode(ctx) } val evaluateAggResults = evaluateVariables(aggResults) // evaluate result expressions ctx.currentVars = aggResults val resultVars = resultExpressions.map { e => - BindReferences.bindReference(e, aggregateAttributes).gen(ctx) + BindReferences.bindReference(e, aggregateAttributes).genCode(ctx) } (resultVars, s""" |$evaluateAggResults @@ -196,7 +196,7 @@ case class TungstenAggregate( (bufVars, "") } else { // no aggregate function, the result should be literals - val resultVars = resultExpressions.map(_.gen(ctx)) + val resultVars = resultExpressions.map(_.genCode(ctx)) (resultVars, evaluateVariables(resultVars)) } @@ -240,7 +240,7 @@ case class TungstenAggregate( } ctx.currentVars = bufVars ++ input // TODO: support subexpression elimination - val aggVals = updateExpr.map(BindReferences.bindReference(_, inputAttrs).gen(ctx)) + val aggVals = updateExpr.map(BindReferences.bindReference(_, inputAttrs).genCode(ctx)) // aggregate buffer should be updated atomic val updates = aggVals.zipWithIndex.map { case (ev, i) => s""" @@ -394,25 +394,25 @@ case class TungstenAggregate( ctx.currentVars = null ctx.INPUT_ROW = keyTerm val keyVars = groupingExpressions.zipWithIndex.map { case (e, i) => - BoundReference(i, e.dataType, e.nullable).gen(ctx) + BoundReference(i, e.dataType, e.nullable).genCode(ctx) } val evaluateKeyVars = evaluateVariables(keyVars) ctx.INPUT_ROW = bufferTerm val bufferVars = aggregateBufferAttributes.zipWithIndex.map { case (e, i) => - BoundReference(i, e.dataType, e.nullable).gen(ctx) + BoundReference(i, e.dataType, e.nullable).genCode(ctx) } val evaluateBufferVars = evaluateVariables(bufferVars) // evaluate the aggregation result ctx.currentVars = bufferVars val aggResults = declFunctions.map(_.evaluateExpression).map { e => - BindReferences.bindReference(e, aggregateBufferAttributes).gen(ctx) + BindReferences.bindReference(e, aggregateBufferAttributes).genCode(ctx) } val evaluateAggResults = evaluateVariables(aggResults) // generate the final result ctx.currentVars = keyVars ++ aggResults val inputAttrs = groupingAttributes ++ aggregateAttributes val resultVars = resultExpressions.map { e => - BindReferences.bindReference(e, inputAttrs).gen(ctx) + BindReferences.bindReference(e, inputAttrs).genCode(ctx) } s""" $evaluateKeyVars @@ -437,7 +437,7 @@ case class TungstenAggregate( ctx.INPUT_ROW = keyTerm ctx.currentVars = null val eval = resultExpressions.map{ e => - BindReferences.bindReference(e, groupingAttributes).gen(ctx) + BindReferences.bindReference(e, groupingAttributes).genCode(ctx) } consume(ctx, eval) } @@ -576,7 +576,7 @@ case class TungstenAggregate( // generate hash code for key val hashExpr = Murmur3Hash(groupingExpressions, 42) ctx.currentVars = input - val hashEval = BindReferences.bindReference(hashExpr, child.output).gen(ctx) + val hashEval = BindReferences.bindReference(hashExpr, child.output).genCode(ctx) val inputAttr = aggregateBufferAttributes ++ child.output ctx.currentVars = new Array[ExprCode](aggregateBufferAttributes.length) ++ input @@ -613,7 +613,8 @@ case class TungstenAggregate( val updateRowInVectorizedHashMap: Option[String] = { if (isVectorizedHashMapEnabled) { ctx.INPUT_ROW = vectorizedRowBuffer - val vectorizedRowEvals = updateExpr.map(BindReferences.bindReference(_, inputAttr).gen(ctx)) + val vectorizedRowEvals = + updateExpr.map(BindReferences.bindReference(_, inputAttr).genCode(ctx)) val updateVectorizedRow = vectorizedRowEvals.zipWithIndex.map { case (ev, i) => val dt = updateExpr(i).dataType ctx.updateColumn(vectorizedRowBuffer, dt, i, ev, updateExpr(i).nullable) @@ -663,7 +664,7 @@ case class TungstenAggregate( val updateRowInUnsafeRowMap: String = { ctx.INPUT_ROW = unsafeRowBuffer val unsafeRowBufferEvals = - updateExpr.map(BindReferences.bindReference(_, inputAttr).gen(ctx)) + updateExpr.map(BindReferences.bindReference(_, inputAttr).genCode(ctx)) val updateUnsafeRowBuffer = unsafeRowBufferEvals.zipWithIndex.map { case (ev, i) => val dt = updateExpr(i).dataType ctx.updateColumn(unsafeRowBuffer, dt, i, ev, updateExpr(i).nullable) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 344aaff348..c689fc3fbb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -53,7 +53,7 @@ case class Project(projectList: Seq[NamedExpression], child: SparkPlan) val exprs = projectList.map(x => ExpressionCanonicalizer.execute(BindReferences.bindReference(x, child.output))) ctx.currentVars = input - val resultVars = exprs.map(_.gen(ctx)) + val resultVars = exprs.map(_.genCode(ctx)) // Evaluation of non-deterministic expressions can't be deferred. val nonDeterministicAttrs = projectList.filterNot(_.deterministic).map(_.toAttribute) s""" @@ -122,7 +122,7 @@ case class Filter(condition: Expression, child: SparkPlan) val evaluated = evaluateRequiredVariables(child.output, in, c.references) // Generate the code for the predicate. - val ev = ExpressionCanonicalizer.execute(bound).gen(ctx) + val ev = ExpressionCanonicalizer.execute(bound).genCode(ctx) val nullCheck = if (bound.nullable) { s"${ev.isNull} || " } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala index a8f854136c..b94b0d26b2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala @@ -118,7 +118,7 @@ case class BroadcastHashJoin( ctx.currentVars = input if (streamedKeys.length == 1 && streamedKeys.head.dataType == LongType) { // generate the join key as Long - val ev = streamedKeys.head.gen(ctx) + val ev = streamedKeys.head.genCode(ctx) (ev, ev.isNull) } else { // generate the join key as UnsafeRow @@ -134,7 +134,7 @@ case class BroadcastHashJoin( ctx.currentVars = null ctx.INPUT_ROW = matched buildPlan.output.zipWithIndex.map { case (a, i) => - val ev = BoundReference(i, a.dataType, a.nullable).gen(ctx) + val ev = BoundReference(i, a.dataType, a.nullable).genCode(ctx) if (joinType == Inner) { ev } else { @@ -170,7 +170,8 @@ case class BroadcastHashJoin( val eval = evaluateRequiredVariables(buildPlan.output, buildVars, expr.references) // filter the output via condition ctx.currentVars = input ++ buildVars - val ev = BindReferences.bindReference(expr, streamedPlan.output ++ buildPlan.output).gen(ctx) + val ev = + BindReferences.bindReference(expr, streamedPlan.output ++ buildPlan.output).genCode(ctx) s""" |$eval |${ev.code} @@ -244,7 +245,8 @@ case class BroadcastHashJoin( // evaluate the variables from build side that used by condition val eval = evaluateRequiredVariables(buildPlan.output, buildVars, expr.references) ctx.currentVars = input ++ buildVars - val ev = BindReferences.bindReference(expr, streamedPlan.output ++ buildPlan.output).gen(ctx) + val ev = + BindReferences.bindReference(expr, streamedPlan.output ++ buildPlan.output).genCode(ctx) s""" |boolean $conditionPassed = true; |${eval.trim} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala index 0e7b2f2f31..443a7b43b6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala @@ -226,7 +226,7 @@ case class SortMergeJoin( keys: Seq[Expression], input: Seq[Attribute]): Seq[ExprCode] = { ctx.INPUT_ROW = row - keys.map(BindReferences.bindReference(_, input).gen(ctx)) + keys.map(BindReferences.bindReference(_, input).genCode(ctx)) } private def copyKeys(ctx: CodegenContext, vars: Seq[ExprCode]): Seq[ExprCode] = { @@ -376,7 +376,7 @@ case class SortMergeJoin( private def createRightVar(ctx: CodegenContext, rightRow: String): Seq[ExprCode] = { ctx.INPUT_ROW = rightRow right.output.zipWithIndex.map { case (a, i) => - BoundReference(i, a.dataType, a.nullable).gen(ctx) + BoundReference(i, a.dataType, a.nullable).genCode(ctx) } } @@ -427,7 +427,7 @@ case class SortMergeJoin( val (rightBefore, rightAfter) = splitVarsByCondition(right.output, rightVars) // Generate code for condition ctx.currentVars = leftVars ++ rightVars - val cond = BindReferences.bindReference(condition.get, output).gen(ctx) + val cond = BindReferences.bindReference(condition.get, output).genCode(ctx) // evaluate the columns those used by condition before loop val before = s""" |boolean $loaded = false; diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index d2ab18ef0e..784b1e8c26 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -48,7 +48,7 @@ case class DeserializeToObject( val bound = ExpressionCanonicalizer.execute( BindReferences.bindReference(deserializer, child.output)) ctx.currentVars = input - val resultVars = bound.gen(ctx) :: Nil + val resultVars = bound.genCode(ctx) :: Nil consume(ctx, resultVars) } @@ -82,7 +82,7 @@ case class SerializeFromObject( ExpressionCanonicalizer.execute(BindReferences.bindReference(expr, child.output)) } ctx.currentVars = input - val resultVars = bound.map(_.gen(ctx)) + val resultVars = bound.map(_.genCode(ctx)) consume(ctx, resultVars) } @@ -173,13 +173,13 @@ case class MapElements( val bound = ExpressionCanonicalizer.execute( BindReferences.bindReference(callFunc, child.output)) ctx.currentVars = input - val evaluated = bound.gen(ctx) + val evaluated = bound.genCode(ctx) val resultObj = LambdaVariable(evaluated.value, evaluated.isNull, resultObjType) val outputFields = serializer.map(_ transform { case _: BoundReference => resultObj }) - val resultVars = outputFields.map(_.gen(ctx)) + val resultVars = outputFields.map(_.genCode(ctx)) s""" ${evaluated.code} ${consume(ctx, resultVars)} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index 4b3091ba22..03defc121c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -54,8 +54,8 @@ case class ScalarSubquery( override def eval(input: InternalRow): Any = result - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - Literal.create(result, dataType).genCode(ctx, ev) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + Literal.create(result, dataType).doGenCode(ctx, ev) } } |