Skip to content

Commit

Permalink
Code review comments from davies.
Browse files Browse the repository at this point in the history
  • Loading branch information
nongli committed Nov 6, 2015
1 parent 2feafbc commit e65def9
Show file tree
Hide file tree
Showing 3 changed files with 21 additions and 25 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,7 @@ class EquivalentExpressions {
* is found. That is, if `expr` has already been added, its children are not added.
* If ignoreLeaf is true, leaf nodes are ignored.
*/
def addExprTree(root: Expression, ignoreLeaf: Boolean): Unit = {
def addExprTree(root: Expression, ignoreLeaf: Boolean = true): Unit = {
val skip = root.isInstanceOf[LeafExpression] && ignoreLeaf
if (!skip && root.deterministic && !addExpr(root)) {
root.children.foreach(addExprTree(_, ignoreLeaf))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -109,17 +109,16 @@ class CodeGenContext {

// State used for subexpression elimination.
case class SubExprEliminationState(
val isLoaded: String, code: GeneratedExpressionCode, val fnName: String, val dt: DataType)

// All the subexpr elimination states. There is one of these states for each group of common
// subexpressions.
val subExprEliminationStates: mutable.ArrayBuffer[SubExprEliminationState] =
mutable.ArrayBuffer.empty[SubExprEliminationState]
val isLoaded: String, code: GeneratedExpressionCode, val fnName: String)

// Foreach expression that is participating in subexpression elimination, the state to use.
val subExprEliminationExprs: mutable.HashMap[Expression, SubExprEliminationState] =
mutable.HashMap[Expression, SubExprEliminationState]()

// The collection of isLoaded variables that need to be reset on each row.
val subExprIsLoadedVariables: mutable.ArrayBuffer[String] =
mutable.ArrayBuffer.empty[String]

final val JAVA_BOOLEAN = "boolean"
final val JAVA_BYTE = "byte"
final val JAVA_SHORT = "short"
Expand Down Expand Up @@ -353,7 +352,7 @@ class CodeGenContext {
*/
private def subexpressionElimination(expressions: Seq[Expression]) = {
// Add each expression tree and compute the common subexpressions.
expressions.foreach(equivalentExpressions.addExprTree(_, true))
expressions.foreach(equivalentExpressions.addExprTree(_))

// Get all the exprs that appear at least twice and set up the state for subexpression
// elimination.
Expand Down Expand Up @@ -400,8 +399,18 @@ class CodeGenContext {
// 2. Less code.
// Currently, we will do this for all non-leaf only expression trees (i.e. expr trees with
// at least two nodes) as the cost of doing it is expected to be low.
val state = SubExprEliminationState(isLoaded, code, fnName, expr.dataType)
subExprEliminationStates += state

// Maintain the loaded value and isNull as member variables. This is necessary if the codegen
// function is split across multiple functions.
// TODO: maintaining this as a local variable probably allows the compiler to do better
// optimizations.
addMutableState("boolean", isLoaded, s"$isLoaded = false;")
addMutableState("boolean", isNull, s"$isNull = false;")
addMutableState(javaType(expr.dataType), primitive,
s"$primitive = ${defaultValue(expr.dataType)};")
subExprIsLoadedVariables += isLoaded

val state = SubExprEliminationState(isLoaded, code, fnName)
e.foreach(subExprEliminationExprs.put(_, state))
})
}
Expand Down Expand Up @@ -440,18 +449,7 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin
protected def declareMutableStates(ctx: CodeGenContext): String = {
ctx.mutableStates.map { case (javaType, variableName, _) =>
s"private $javaType $variableName;"
}.mkString("\n") + "\n" +
// Maintain the loaded value and isNull as member variables. This is necessary if the codegen
// function is split across multiple functions.
// TODO: maintaining this as a local variable probably allows the compiler to do better
// optimizations.
ctx.subExprEliminationStates.map { s => {
s"""
| private boolean ${s.isLoaded} = false;
| private boolean ${s.code.isNull};
| private ${ctx.javaType(s.dt)} ${s.code.value} = ${ctx.defaultValue(s.dt)};
""".stripMargin
}}.mkString("\n").trim
}.mkString("\n")
}

protected def initMutableStates(ctx: CodeGenContext): String = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -287,9 +287,7 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro
ctx.addMutableState(holderClass, bufferHolder, s"this.$bufferHolder = new $holderClass();")

// Reset the isLoaded flag for each row.
val subexprReset = ctx.subExprEliminationStates.map(s => {
s"${s.isLoaded} = false;"
}).mkString("\n")
val subexprReset = ctx.subExprIsLoadedVariables.map { v => s"${v} = false;" }.mkString("\n")

val code =
s"""
Expand Down

0 comments on commit e65def9

Please sign in to comment.