Repository: spark
Updated Branches:
  refs/heads/master 269cd5359 -> 2ce37b50f


http://git-wip-us.apache.org/repos/asf/spark/blob/2ce37b50/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala
----------------------------------------------------------------------
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 d8dc086..2c2cf3d 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
@@ -1128,15 +1128,16 @@ abstract class RoundBase(child: Expression, scale: 
Expression,
           }"""
     }
 
+    val javaType = CodeGenerator.javaType(dataType)
     if (scaleV == null) { // if scale is null, no need to eval its child at all
       ev.copy(code = s"""
         boolean ${ev.isNull} = true;
-        ${ctx.javaType(dataType)} ${ev.value} = 
${ctx.defaultValue(dataType)};""")
+        $javaType ${ev.value} = ${CodeGenerator.defaultValue(dataType)};""")
     } else {
       ev.copy(code = s"""
         ${ce.code}
         boolean ${ev.isNull} = ${ce.isNull};
-        ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)};
+        $javaType ${ev.value} = ${CodeGenerator.defaultValue(dataType)};
         if (!${ev.isNull}) {
           $evaluationCode
         }""")

http://git-wip-us.apache.org/repos/asf/spark/blob/2ce37b50/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala
----------------------------------------------------------------------
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 470d5da..b35fa72 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
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions
 
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
-import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
CodeGenerator, ExprCode}
 import org.apache.spark.sql.catalyst.util.TypeUtils
 import org.apache.spark.sql.types._
 
@@ -72,7 +72,7 @@ case class Coalesce(children: Seq[Expression]) extends 
Expression {
   }
 
   override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
-    ev.isNull = ctx.addMutableState(ctx.JAVA_BOOLEAN, ev.isNull)
+    ev.isNull = ctx.addMutableState(CodeGenerator.JAVA_BOOLEAN, ev.isNull)
 
     // all the evals are meant to be in a do { ... } while (false); loop
     val evals = children.map { e =>
@@ -87,14 +87,14 @@ case class Coalesce(children: Seq[Expression]) extends 
Expression {
        """.stripMargin
     }
 
-    val resultType = ctx.javaType(dataType)
+    val resultType = CodeGenerator.javaType(dataType)
     val codes = ctx.splitExpressionsWithCurrentInputs(
       expressions = evals,
       funcName = "coalesce",
       returnType = resultType,
       makeSplitFunction = func =>
         s"""
-           |$resultType ${ev.value} = ${ctx.defaultValue(dataType)};
+           |$resultType ${ev.value} = ${CodeGenerator.defaultValue(dataType)};
            |do {
            |  $func
            |} while (false);
@@ -113,7 +113,7 @@ case class Coalesce(children: Seq[Expression]) extends 
Expression {
     ev.copy(code =
       s"""
          |${ev.isNull} = true;
-         |$resultType ${ev.value} = ${ctx.defaultValue(dataType)};
+         |$resultType ${ev.value} = ${CodeGenerator.defaultValue(dataType)};
          |do {
          |  $codes
          |} while (false);
@@ -234,7 +234,7 @@ case class IsNaN(child: Expression) extends UnaryExpression
       case DoubleType | FloatType =>
         ev.copy(code = s"""
           ${eval.code}
-          ${ctx.javaType(dataType)} ${ev.value} = 
${ctx.defaultValue(dataType)};
+          ${CodeGenerator.javaType(dataType)} ${ev.value} = 
${CodeGenerator.defaultValue(dataType)};
           ${ev.value} = !${eval.isNull} && Double.isNaN(${eval.value});""", 
isNull = "false")
     }
   }
@@ -281,7 +281,7 @@ case class NaNvl(left: Expression, right: Expression)
         ev.copy(code = s"""
           ${leftGen.code}
           boolean ${ev.isNull} = false;
-          ${ctx.javaType(dataType)} ${ev.value} = 
${ctx.defaultValue(dataType)};
+          ${CodeGenerator.javaType(dataType)} ${ev.value} = 
${CodeGenerator.defaultValue(dataType)};
           if (${leftGen.isNull}) {
             ${ev.isNull} = true;
           } else {
@@ -416,8 +416,8 @@ case class AtLeastNNonNulls(n: Int, children: 
Seq[Expression]) extends Predicate
     val codes = ctx.splitExpressionsWithCurrentInputs(
       expressions = evals,
       funcName = "atLeastNNonNulls",
-      extraArguments = (ctx.JAVA_INT, nonnull) :: Nil,
-      returnType = ctx.JAVA_INT,
+      extraArguments = (CodeGenerator.JAVA_INT, nonnull) :: Nil,
+      returnType = CodeGenerator.JAVA_INT,
       makeSplitFunction = body =>
         s"""
            |do {
@@ -436,11 +436,11 @@ case class AtLeastNNonNulls(n: Int, children: 
Seq[Expression]) extends Predicate
 
     ev.copy(code =
       s"""
-         |${ctx.JAVA_INT} $nonnull = 0;
+         |${CodeGenerator.JAVA_INT} $nonnull = 0;
          |do {
          |  $codes
          |} while (false);
-         |${ctx.JAVA_BOOLEAN} ${ev.value} = $nonnull >= $n;
+         |${CodeGenerator.JAVA_BOOLEAN} ${ev.value} = $nonnull >= $n;
        """.stripMargin, isNull = "false")
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/2ce37b50/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
----------------------------------------------------------------------
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
index 64da9bb..80618af 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
@@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.ScalaReflection.universe.TermName
 import org.apache.spark.sql.catalyst.encoders.RowEncoder
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
CodeGenerator, ExprCode}
 import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, 
GenericArrayData}
 import org.apache.spark.sql.types._
 
@@ -62,13 +62,13 @@ trait InvokeLike extends Expression with NonSQLExpression {
   def prepareArguments(ctx: CodegenContext): (String, String, String) = {
 
     val resultIsNull = if (needNullCheck) {
-      val resultIsNull = ctx.addMutableState(ctx.JAVA_BOOLEAN, "resultIsNull")
+      val resultIsNull = ctx.addMutableState(CodeGenerator.JAVA_BOOLEAN, 
"resultIsNull")
       resultIsNull
     } else {
       "false"
     }
     val argValues = arguments.map { e =>
-      val argValue = ctx.addMutableState(ctx.javaType(e.dataType), "argValue")
+      val argValue = ctx.addMutableState(CodeGenerator.javaType(e.dataType), 
"argValue")
       argValue
     }
 
@@ -137,7 +137,7 @@ case class StaticInvoke(
     throw new UnsupportedOperationException("Only code-generated evaluation is 
supported.")
 
   override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
-    val javaType = ctx.javaType(dataType)
+    val javaType = CodeGenerator.javaType(dataType)
 
     val (argCode, argString, resultIsNull) = prepareArguments(ctx)
 
@@ -151,7 +151,7 @@ case class StaticInvoke(
     }
 
     val evaluate = if (returnNullable) {
-      if (ctx.defaultValue(dataType) == "null") {
+      if (CodeGenerator.defaultValue(dataType) == "null") {
         s"""
           ${ev.value} = $callFunc;
           ${ev.isNull} = ${ev.value} == null;
@@ -159,7 +159,7 @@ case class StaticInvoke(
       } else {
         val boxedResult = ctx.freshName("boxedResult")
         s"""
-          ${ctx.boxedType(dataType)} $boxedResult = $callFunc;
+          ${CodeGenerator.boxedType(dataType)} $boxedResult = $callFunc;
           ${ev.isNull} = $boxedResult == null;
           if (!${ev.isNull}) {
             ${ev.value} = $boxedResult;
@@ -173,7 +173,7 @@ case class StaticInvoke(
     val code = s"""
       $argCode
       $prepareIsNull
-      $javaType ${ev.value} = ${ctx.defaultValue(dataType)};
+      $javaType ${ev.value} = ${CodeGenerator.defaultValue(dataType)};
       if (!$resultIsNull) {
         $evaluate
       }
@@ -228,7 +228,7 @@ case class Invoke(
   }
 
   override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
-    val javaType = ctx.javaType(dataType)
+    val javaType = CodeGenerator.javaType(dataType)
     val obj = targetObject.genCode(ctx)
 
     val (argCode, argString, resultIsNull) = prepareArguments(ctx)
@@ -255,11 +255,11 @@ case class Invoke(
       // If the function can return null, we do an extra check to make sure 
our null bit is still
       // set correctly.
       val assignResult = if (!returnNullable) {
-        s"${ev.value} = (${ctx.boxedType(javaType)}) $funcResult;"
+        s"${ev.value} = (${CodeGenerator.boxedType(javaType)}) $funcResult;"
       } else {
         s"""
           if ($funcResult != null) {
-            ${ev.value} = (${ctx.boxedType(javaType)}) $funcResult;
+            ${ev.value} = (${CodeGenerator.boxedType(javaType)}) $funcResult;
           } else {
             ${ev.isNull} = true;
           }
@@ -275,7 +275,7 @@ case class Invoke(
     val code = s"""
       ${obj.code}
       boolean ${ev.isNull} = true;
-      $javaType ${ev.value} = ${ctx.defaultValue(dataType)};
+      $javaType ${ev.value} = ${CodeGenerator.defaultValue(dataType)};
       if (!${obj.isNull}) {
         $argCode
         ${ev.isNull} = $resultIsNull;
@@ -341,7 +341,7 @@ case class NewInstance(
     throw new UnsupportedOperationException("Only code-generated evaluation is 
supported.")
 
   override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
-    val javaType = ctx.javaType(dataType)
+    val javaType = CodeGenerator.javaType(dataType)
 
     val (argCode, argString, resultIsNull) = prepareArguments(ctx)
 
@@ -358,7 +358,8 @@ case class NewInstance(
     val code = s"""
       $argCode
       ${outer.map(_.code).getOrElse("")}
-      final $javaType ${ev.value} = ${ev.isNull} ? 
${ctx.defaultValue(javaType)} : $constructorCall;
+      final $javaType ${ev.value} = ${ev.isNull} ?
+        ${CodeGenerator.defaultValue(dataType)} : $constructorCall;
     """
     ev.copy(code = code)
   }
@@ -385,15 +386,15 @@ case class UnwrapOption(
     throw new UnsupportedOperationException("Only code-generated evaluation is 
supported")
 
   override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
-    val javaType = ctx.javaType(dataType)
+    val javaType = CodeGenerator.javaType(dataType)
     val inputObject = child.genCode(ctx)
 
     val code = s"""
       ${inputObject.code}
 
       final boolean ${ev.isNull} = ${inputObject.isNull} || 
${inputObject.value}.isEmpty();
-      $javaType ${ev.value} = ${ev.isNull} ?
-        ${ctx.defaultValue(javaType)} : (${ctx.boxedType(javaType)}) 
${inputObject.value}.get();
+      $javaType ${ev.value} = ${ev.isNull} ? 
${CodeGenerator.defaultValue(dataType)} :
+        (${CodeGenerator.boxedType(javaType)}) ${inputObject.value}.get();
     """
     ev.copy(code = code)
   }
@@ -546,7 +547,7 @@ case class MapObjects private(
       ArrayType(lambdaFunction.dataType, containsNull = 
lambdaFunction.nullable))
 
   override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
-    val elementJavaType = ctx.javaType(loopVarDataType)
+    val elementJavaType = CodeGenerator.javaType(loopVarDataType)
     ctx.addMutableState(elementJavaType, loopValue, forceInline = true, 
useFreshName = false)
     val genInputData = inputData.genCode(ctx)
     val genFunction = lambdaFunction.genCode(ctx)
@@ -554,7 +555,7 @@ case class MapObjects private(
     val convertedArray = ctx.freshName("convertedArray")
     val loopIndex = ctx.freshName("loopIndex")
 
-    val convertedType = ctx.boxedType(lambdaFunction.dataType)
+    val convertedType = CodeGenerator.boxedType(lambdaFunction.dataType)
 
     // Because of the way Java defines nested arrays, we have to handle the 
syntax specially.
     // Specifically, we have to insert the [$dataLength] in between the type 
and any extra nested
@@ -621,7 +622,7 @@ case class MapObjects private(
         (
           s"${genInputData.value}.numElements()",
           "",
-          ctx.getValue(genInputData.value, et, loopIndex)
+          CodeGenerator.getValue(genInputData.value, et, loopIndex)
         )
       case ObjectType(cls) if cls == classOf[Object] =>
         val it = ctx.freshName("it")
@@ -643,7 +644,8 @@ case class MapObjects private(
     }
 
     val loopNullCheck = if (loopIsNull != "false") {
-      ctx.addMutableState(ctx.JAVA_BOOLEAN, loopIsNull, forceInline = true, 
useFreshName = false)
+      ctx.addMutableState(
+        CodeGenerator.JAVA_BOOLEAN, loopIsNull, forceInline = true, 
useFreshName = false)
       inputDataType match {
         case _: ArrayType => s"$loopIsNull = 
${genInputData.value}.isNullAt($loopIndex);"
         case _ => s"$loopIsNull = $loopValue == null;"
@@ -695,7 +697,7 @@ case class MapObjects private(
 
     val code = s"""
       ${genInputData.code}
-      ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)};
+      ${CodeGenerator.javaType(dataType)} ${ev.value} = 
${CodeGenerator.defaultValue(dataType)};
 
       if (!${genInputData.isNull}) {
         $determineCollectionType
@@ -806,10 +808,10 @@ case class CatalystToExternalMap private(
     }
 
     val mapType = inputDataType(inputData.dataType).asInstanceOf[MapType]
-    val keyElementJavaType = ctx.javaType(mapType.keyType)
+    val keyElementJavaType = CodeGenerator.javaType(mapType.keyType)
     ctx.addMutableState(keyElementJavaType, keyLoopValue, forceInline = true, 
useFreshName = false)
     val genKeyFunction = keyLambdaFunction.genCode(ctx)
-    val valueElementJavaType = ctx.javaType(mapType.valueType)
+    val valueElementJavaType = CodeGenerator.javaType(mapType.valueType)
     ctx.addMutableState(valueElementJavaType, valueLoopValue, forceInline = 
true,
       useFreshName = false)
     val genValueFunction = valueLambdaFunction.genCode(ctx)
@@ -825,10 +827,11 @@ case class CatalystToExternalMap private(
     val valueArray = ctx.freshName("valueArray")
     val getKeyArray =
       s"${classOf[ArrayData].getName} $keyArray = 
${genInputData.value}.keyArray();"
-    val getKeyLoopVar = ctx.getValue(keyArray, inputDataType(mapType.keyType), 
loopIndex)
+    val getKeyLoopVar = CodeGenerator.getValue(keyArray, 
inputDataType(mapType.keyType), loopIndex)
     val getValueArray =
       s"${classOf[ArrayData].getName} $valueArray = 
${genInputData.value}.valueArray();"
-    val getValueLoopVar = ctx.getValue(valueArray, 
inputDataType(mapType.valueType), loopIndex)
+    val getValueLoopVar = CodeGenerator.getValue(
+      valueArray, inputDataType(mapType.valueType), loopIndex)
 
     // Make a copy of the data if it's unsafe-backed
     def makeCopyIfInstanceOf(clazz: Class[_ <: Any], value: String) =
@@ -844,7 +847,7 @@ case class CatalystToExternalMap private(
     val genValueFunctionValue = genFunctionValue(valueLambdaFunction, 
genValueFunction)
 
     val valueLoopNullCheck = if (valueLoopIsNull != "false") {
-      ctx.addMutableState(ctx.JAVA_BOOLEAN, valueLoopIsNull, forceInline = 
true,
+      ctx.addMutableState(CodeGenerator.JAVA_BOOLEAN, valueLoopIsNull, 
forceInline = true,
         useFreshName = false)
       s"$valueLoopIsNull = $valueArray.isNullAt($loopIndex);"
     } else {
@@ -873,7 +876,7 @@ case class CatalystToExternalMap private(
 
     val code = s"""
       ${genInputData.code}
-      ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)};
+      ${CodeGenerator.javaType(dataType)} ${ev.value} = 
${CodeGenerator.defaultValue(dataType)};
 
       if (!${genInputData.isNull}) {
         int $dataLength = $getLength;
@@ -993,8 +996,8 @@ case class ExternalMapToCatalyst private(
     val entry = ctx.freshName("entry")
     val entries = ctx.freshName("entries")
 
-    val keyElementJavaType = ctx.javaType(keyType)
-    val valueElementJavaType = ctx.javaType(valueType)
+    val keyElementJavaType = CodeGenerator.javaType(keyType)
+    val valueElementJavaType = CodeGenerator.javaType(valueType)
     ctx.addMutableState(keyElementJavaType, key, forceInline = true, 
useFreshName = false)
     ctx.addMutableState(valueElementJavaType, value, forceInline = true, 
useFreshName = false)
 
@@ -1009,8 +1012,8 @@ case class ExternalMapToCatalyst private(
         val defineKeyValue =
           s"""
             final $javaMapEntryCls $entry = ($javaMapEntryCls) $entries.next();
-            $key = (${ctx.boxedType(keyType)}) $entry.getKey();
-            $value = (${ctx.boxedType(valueType)}) $entry.getValue();
+            $key = (${CodeGenerator.boxedType(keyType)}) $entry.getKey();
+            $value = (${CodeGenerator.boxedType(valueType)}) $entry.getValue();
           """
 
         defineEntries -> defineKeyValue
@@ -1024,22 +1027,24 @@ case class ExternalMapToCatalyst private(
         val defineKeyValue =
           s"""
             final $scalaMapEntryCls $entry = ($scalaMapEntryCls) 
$entries.next();
-            $key = (${ctx.boxedType(keyType)}) $entry._1();
-            $value = (${ctx.boxedType(valueType)}) $entry._2();
+            $key = (${CodeGenerator.boxedType(keyType)}) $entry._1();
+            $value = (${CodeGenerator.boxedType(valueType)}) $entry._2();
           """
 
         defineEntries -> defineKeyValue
     }
 
     val keyNullCheck = if (keyIsNull != "false") {
-      ctx.addMutableState(ctx.JAVA_BOOLEAN, keyIsNull, forceInline = true, 
useFreshName = false)
+      ctx.addMutableState(
+        CodeGenerator.JAVA_BOOLEAN, keyIsNull, forceInline = true, 
useFreshName = false)
       s"$keyIsNull = $key == null;"
     } else {
       ""
     }
 
     val valueNullCheck = if (valueIsNull != "false") {
-      ctx.addMutableState(ctx.JAVA_BOOLEAN, valueIsNull, forceInline = true, 
useFreshName = false)
+      ctx.addMutableState(
+        CodeGenerator.JAVA_BOOLEAN, valueIsNull, forceInline = true, 
useFreshName = false)
       s"$valueIsNull = $value == null;"
     } else {
       ""
@@ -1047,12 +1052,12 @@ case class ExternalMapToCatalyst private(
 
     val arrayCls = classOf[GenericArrayData].getName
     val mapCls = classOf[ArrayBasedMapData].getName
-    val convertedKeyType = ctx.boxedType(keyConverter.dataType)
-    val convertedValueType = ctx.boxedType(valueConverter.dataType)
+    val convertedKeyType = CodeGenerator.boxedType(keyConverter.dataType)
+    val convertedValueType = CodeGenerator.boxedType(valueConverter.dataType)
     val code =
       s"""
         ${inputMap.code}
-        ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)};
+        ${CodeGenerator.javaType(dataType)} ${ev.value} = 
${CodeGenerator.defaultValue(dataType)};
         if (!${inputMap.isNull}) {
           final int $length = ${inputMap.value}.size();
           final Object[] $convertedKeys = new Object[$length];
@@ -1174,12 +1179,13 @@ case class EncodeUsingSerializer(child: Expression, 
kryo: Boolean)
 
     // Code to serialize.
     val input = child.genCode(ctx)
-    val javaType = ctx.javaType(dataType)
+    val javaType = CodeGenerator.javaType(dataType)
     val serialize = s"$serializer.serialize(${input.value}, null).array()"
 
     val code = s"""
       ${input.code}
-      final $javaType ${ev.value} = ${input.isNull} ? 
${ctx.defaultValue(javaType)} : $serialize;
+      final $javaType ${ev.value} =
+        ${input.isNull} ? ${CodeGenerator.defaultValue(dataType)} : $serialize;
      """
     ev.copy(code = code, isNull = input.isNull)
   }
@@ -1223,13 +1229,14 @@ case class DecodeUsingSerializer[T](child: Expression, 
tag: ClassTag[T], kryo: B
 
     // Code to deserialize.
     val input = child.genCode(ctx)
-    val javaType = ctx.javaType(dataType)
+    val javaType = CodeGenerator.javaType(dataType)
     val deserialize =
       s"($javaType) 
$serializer.deserialize(java.nio.ByteBuffer.wrap(${input.value}), null)"
 
     val code = s"""
       ${input.code}
-      final $javaType ${ev.value} = ${input.isNull} ? 
${ctx.defaultValue(javaType)} : $deserialize;
+      final $javaType ${ev.value} =
+         ${input.isNull} ? ${CodeGenerator.defaultValue(dataType)} : 
$deserialize;
      """
     ev.copy(code = code, isNull = input.isNull)
   }
@@ -1254,7 +1261,7 @@ case class InitializeJavaBean(beanInstance: Expression, 
setters: Map[String, Exp
     val instanceGen = beanInstance.genCode(ctx)
 
     val javaBeanInstance = ctx.freshName("javaBean")
-    val beanInstanceJavaType = ctx.javaType(beanInstance.dataType)
+    val beanInstanceJavaType = CodeGenerator.javaType(beanInstance.dataType)
 
     val initialize = setters.map {
       case (setterMethod, fieldValue) =>
@@ -1405,15 +1412,15 @@ case class ValidateExternalType(child: Expression, 
expected: DataType)
       case _: ArrayType =>
         s"$obj instanceof ${classOf[Seq[_]].getName} || 
$obj.getClass().isArray()"
       case _ =>
-        s"$obj instanceof ${ctx.boxedType(dataType)}"
+        s"$obj instanceof ${CodeGenerator.boxedType(dataType)}"
     }
 
     val code = s"""
       ${input.code}
-      ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)};
+      ${CodeGenerator.javaType(dataType)} ${ev.value} = 
${CodeGenerator.defaultValue(dataType)};
       if (!${input.isNull}) {
         if ($typeCheck) {
-          ${ev.value} = (${ctx.boxedType(dataType)}) $obj;
+          ${ev.value} = (${CodeGenerator.boxedType(dataType)}) $obj;
         } else {
           throw new RuntimeException($obj.getClass().getName() + $errMsgField);
         }

http://git-wip-us.apache.org/repos/asf/spark/blob/2ce37b50/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
----------------------------------------------------------------------
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 a6d41ea..4b85d9a 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
@@ -21,7 +21,7 @@ import scala.collection.immutable.TreeSet
 
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
-import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
ExprCode, GenerateSafeProjection, GenerateUnsafeProjection, Predicate => 
BasePredicate}
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
CodeGenerator, ExprCode, GenerateSafeProjection, GenerateUnsafeProjection, 
Predicate => BasePredicate}
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.catalyst.util.TypeUtils
 import org.apache.spark.sql.types._
@@ -235,7 +235,7 @@ case class In(value: Expression, list: Seq[Expression]) 
extends Predicate {
   }
 
   override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
-    val javaDataType = ctx.javaType(value.dataType)
+    val javaDataType = CodeGenerator.javaType(value.dataType)
     val valueGen = value.genCode(ctx)
     val listGen = list.map(_.genCode(ctx))
     // inTmpResult has 3 possible values:
@@ -263,8 +263,8 @@ case class In(value: Expression, list: Seq[Expression]) 
extends Predicate {
     val codes = ctx.splitExpressionsWithCurrentInputs(
       expressions = listCode,
       funcName = "valueIn",
-      extraArguments = (javaDataType, valueArg) :: (ctx.JAVA_BYTE, tmpResult) 
:: Nil,
-      returnType = ctx.JAVA_BYTE,
+      extraArguments = (javaDataType, valueArg) :: (CodeGenerator.JAVA_BYTE, 
tmpResult) :: Nil,
+      returnType = CodeGenerator.JAVA_BYTE,
       makeSplitFunction = body =>
         s"""
            |do {
@@ -348,8 +348,8 @@ case class InSet(child: Expression, hset: Set[Any]) extends 
UnaryExpression with
     ev.copy(code =
       s"""
          |${childGen.code}
-         |${ctx.JAVA_BOOLEAN} ${ev.isNull} = ${childGen.isNull};
-         |${ctx.JAVA_BOOLEAN} ${ev.value} = false;
+         |${CodeGenerator.JAVA_BOOLEAN} ${ev.isNull} = ${childGen.isNull};
+         |${CodeGenerator.JAVA_BOOLEAN} ${ev.value} = false;
          |if (!${ev.isNull}) {
          |  ${ev.value} = $setTerm.contains(${childGen.value});
          |  $setIsNull
@@ -505,7 +505,7 @@ abstract class BinaryComparison extends BinaryOperator with 
Predicate {
   }
 
   override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
-    if (ctx.isPrimitiveType(left.dataType)
+    if (CodeGenerator.isPrimitiveType(left.dataType)
         && left.dataType != BooleanType // java boolean doesn't support > or < 
operator
         && left.dataType != FloatType
         && left.dataType != DoubleType) {

http://git-wip-us.apache.org/repos/asf/spark/blob/2ce37b50/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala
----------------------------------------------------------------------
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 8bc936f..6c9937d 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
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions
 
 import org.apache.spark.sql.AnalysisException
 import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
CodeGenerator, ExprCode}
 import org.apache.spark.sql.types._
 import org.apache.spark.util.Utils
 import org.apache.spark.util.random.XORShiftRandom
@@ -82,7 +82,8 @@ case class Rand(child: Expression) extends RDG {
     ctx.addPartitionInitializationStatement(
       s"$rngTerm = new $className(${seed}L + partitionIndex);")
     ev.copy(code = s"""
-      final ${ctx.javaType(dataType)} ${ev.value} = $rngTerm.nextDouble();""", 
isNull = "false")
+      final ${CodeGenerator.javaType(dataType)} ${ev.value} = 
$rngTerm.nextDouble();""",
+      isNull = "false")
   }
 }
 
@@ -116,7 +117,8 @@ case class Randn(child: Expression) extends RDG {
     ctx.addPartitionInitializationStatement(
       s"$rngTerm = new $className(${seed}L + partitionIndex);")
     ev.copy(code = s"""
-      final ${ctx.javaType(dataType)} ${ev.value} = 
$rngTerm.nextGaussian();""", isNull = "false")
+      final ${CodeGenerator.javaType(dataType)} ${ev.value} = 
$rngTerm.nextGaussian();""",
+      isNull = "false")
   }
 }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/2ce37b50/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
----------------------------------------------------------------------
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 f3e8f6d..ad0c079 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
@@ -126,7 +126,7 @@ case class Like(left: Expression, right: Expression) 
extends StringRegexExpressi
         ev.copy(code = s"""
           ${eval.code}
           boolean ${ev.isNull} = ${eval.isNull};
-          ${ctx.javaType(dataType)} ${ev.value} = 
${ctx.defaultValue(dataType)};
+          ${CodeGenerator.javaType(dataType)} ${ev.value} = 
${CodeGenerator.defaultValue(dataType)};
           if (!${ev.isNull}) {
             ${ev.value} = $pattern.matcher(${eval.value}.toString()).matches();
           }
@@ -134,7 +134,7 @@ case class Like(left: Expression, right: Expression) 
extends StringRegexExpressi
       } else {
         ev.copy(code = s"""
           boolean ${ev.isNull} = true;
-          ${ctx.javaType(dataType)} ${ev.value} = 
${ctx.defaultValue(dataType)};
+          ${CodeGenerator.javaType(dataType)} ${ev.value} = 
${CodeGenerator.defaultValue(dataType)};
         """)
       }
     } else {
@@ -201,7 +201,7 @@ case class RLike(left: Expression, right: Expression) 
extends StringRegexExpress
         ev.copy(code = s"""
           ${eval.code}
           boolean ${ev.isNull} = ${eval.isNull};
-          ${ctx.javaType(dataType)} ${ev.value} = 
${ctx.defaultValue(dataType)};
+          ${CodeGenerator.javaType(dataType)} ${ev.value} = 
${CodeGenerator.defaultValue(dataType)};
           if (!${ev.isNull}) {
             ${ev.value} = $pattern.matcher(${eval.value}.toString()).find(0);
           }
@@ -209,7 +209,7 @@ case class RLike(left: Expression, right: Expression) 
extends StringRegexExpress
       } else {
         ev.copy(code = s"""
           boolean ${ev.isNull} = true;
-          ${ctx.javaType(dataType)} ${ev.value} = 
${ctx.defaultValue(dataType)};
+          ${CodeGenerator.javaType(dataType)} ${ev.value} = 
${CodeGenerator.defaultValue(dataType)};
         """)
       }
     } else {

http://git-wip-us.apache.org/repos/asf/spark/blob/2ce37b50/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
----------------------------------------------------------------------
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 d7612e3..22fbb89 100755
--- 
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
@@ -102,11 +102,11 @@ case class Concat(children: Seq[Expression]) extends 
Expression {
     val codes = ctx.splitExpressionsWithCurrentInputs(
       expressions = inputs,
       funcName = "valueConcat",
-      extraArguments = (s"${ctx.javaType(dataType)}[]", args) :: Nil)
+      extraArguments = (s"${CodeGenerator.javaType(dataType)}[]", args) :: Nil)
     ev.copy(s"""
       $initCode
       $codes
-      ${ctx.javaType(dataType)} ${ev.value} = $concatenator.concat($args);
+      ${CodeGenerator.javaType(dataType)} ${ev.value} = 
$concatenator.concat($args);
       boolean ${ev.isNull} = ${ev.value} == null;
     """)
   }
@@ -196,7 +196,7 @@ case class ConcatWs(children: Seq[Expression])
     } else {
       val array = ctx.freshName("array")
       val varargNum = ctx.freshName("varargNum")
-      val idxInVararg = ctx.freshName("idxInVararg")
+      val idxVararg = ctx.freshName("idxInVararg")
 
       val evals = children.map(_.genCode(ctx))
       val (varargCount, varargBuild) = children.tail.zip(evals.tail).map { 
case (child, eval) =>
@@ -206,7 +206,7 @@ case class ConcatWs(children: Seq[Expression])
              if (eval.isNull == "true") {
                ""
              } else {
-               s"$array[$idxInVararg ++] = ${eval.isNull} ? (UTF8String) null 
: ${eval.value};"
+               s"$array[$idxVararg ++] = ${eval.isNull} ? (UTF8String) null : 
${eval.value};"
              })
           case _: ArrayType =>
             val size = ctx.freshName("n")
@@ -222,7 +222,7 @@ case class ConcatWs(children: Seq[Expression])
                 if (!${eval.isNull}) {
                   final int $size = ${eval.value}.numElements();
                   for (int j = 0; j < $size; j ++) {
-                    $array[$idxInVararg ++] = ${ctx.getValue(eval.value, 
StringType, "j")};
+                    $array[$idxVararg ++] = 
${CodeGenerator.getValue(eval.value, StringType, "j")};
                   }
                 }
                 """)
@@ -247,20 +247,20 @@ case class ConcatWs(children: Seq[Expression])
       val varargBuilds = ctx.splitExpressionsWithCurrentInputs(
         expressions = varargBuild,
         funcName = "varargBuildsConcatWs",
-        extraArguments = ("UTF8String []", array) :: ("int", idxInVararg) :: 
Nil,
+        extraArguments = ("UTF8String []", array) :: ("int", idxVararg) :: Nil,
         returnType = "int",
         makeSplitFunction = body =>
           s"""
              |$body
-             |return $idxInVararg;
+             |return $idxVararg;
            """.stripMargin,
-        foldFunctions = _.map(funcCall => s"$idxInVararg = 
$funcCall;").mkString("\n"))
+        foldFunctions = _.map(funcCall => s"$idxVararg = 
$funcCall;").mkString("\n"))
 
       ev.copy(
         s"""
         $codes
         int $varargNum = ${children.count(_.dataType == StringType) - 1};
-        int $idxInVararg = 0;
+        int $idxVararg = 0;
         $varargCounts
         UTF8String[] $array = new UTF8String[$varargNum];
         $varargBuilds
@@ -333,7 +333,7 @@ case class Elt(children: Seq[Expression]) extends 
Expression {
     val indexVal = ctx.freshName("index")
     val indexMatched = ctx.freshName("eltIndexMatched")
 
-    val inputVal = ctx.addMutableState(ctx.javaType(dataType), "inputVal")
+    val inputVal = ctx.addMutableState(CodeGenerator.javaType(dataType), 
"inputVal")
 
     val assignInputValue = inputs.zipWithIndex.map { case (eval, index) =>
       s"""
@@ -350,10 +350,10 @@ case class Elt(children: Seq[Expression]) extends 
Expression {
       expressions = assignInputValue,
       funcName = "eltFunc",
       extraArguments = ("int", indexVal) :: Nil,
-      returnType = ctx.JAVA_BOOLEAN,
+      returnType = CodeGenerator.JAVA_BOOLEAN,
       makeSplitFunction = body =>
         s"""
-           |${ctx.JAVA_BOOLEAN} $indexMatched = false;
+           |${CodeGenerator.JAVA_BOOLEAN} $indexMatched = false;
            |do {
            |  $body
            |} while (false);
@@ -372,12 +372,12 @@ case class Elt(children: Seq[Expression]) extends 
Expression {
       s"""
          |${index.code}
          |final int $indexVal = ${index.value};
-         |${ctx.JAVA_BOOLEAN} $indexMatched = false;
+         |${CodeGenerator.JAVA_BOOLEAN} $indexMatched = false;
          |$inputVal = null;
          |do {
          |  $codes
          |} while (false);
-         |final ${ctx.javaType(dataType)} ${ev.value} = $inputVal;
+         |final ${CodeGenerator.javaType(dataType)} ${ev.value} = $inputVal;
          |final boolean ${ev.isNull} = ${ev.value} == null;
        """.stripMargin)
   }
@@ -1410,10 +1410,10 @@ case class FormatString(children: Expression*) extends 
Expression with ImplicitC
     val numArgLists = argListGen.length
     val argListCode = argListGen.zipWithIndex.map { case(v, index) =>
       val value =
-        if (ctx.boxedType(v._1) != ctx.javaType(v._1)) {
+        if (CodeGenerator.boxedType(v._1) != CodeGenerator.javaType(v._1)) {
           // Java primitives get boxed in order to allow null values.
-          s"(${v._2.isNull}) ? (${ctx.boxedType(v._1)}) null : " +
-            s"new ${ctx.boxedType(v._1)}(${v._2.value})"
+          s"(${v._2.isNull}) ? (${CodeGenerator.boxedType(v._1)}) null : " +
+            s"new ${CodeGenerator.boxedType(v._1)}(${v._2.value})"
         } else {
           s"(${v._2.isNull}) ? null : ${v._2.value}"
         }
@@ -1434,7 +1434,7 @@ case class FormatString(children: Expression*) extends 
Expression with ImplicitC
     ev.copy(code = s"""
       ${pattern.code}
       boolean ${ev.isNull} = ${pattern.isNull};
-      ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)};
+      ${CodeGenerator.javaType(dataType)} ${ev.value} = 
${CodeGenerator.defaultValue(dataType)};
       if (!${ev.isNull}) {
         $stringBuffer $sb = new $stringBuffer();
         $formatter $form = new $formatter($sb, ${classOf[Locale].getName}.US);
@@ -2110,7 +2110,8 @@ case class FormatNumber(x: Expression, d: Expression)
       val usLocale = "US"
       val i = ctx.freshName("i")
       val dFormat = ctx.freshName("dFormat")
-      val lastDValue = ctx.addMutableState(ctx.JAVA_INT, "lastDValue", v => 
s"$v = -100;")
+      val lastDValue =
+        ctx.addMutableState(CodeGenerator.JAVA_INT, "lastDValue", v => s"$v = 
-100;")
       val pattern = ctx.addMutableState(sb, "pattern", v => s"$v = new $sb();")
       val numberFormat = ctx.addMutableState(df, "numberFormat",
         v => s"""$v = new $df("", new $dfs($l.$usLocale));""")

http://git-wip-us.apache.org/repos/asf/spark/blob/2ce37b50/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala
index 676ba39..1e48c7b 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala
@@ -405,12 +405,12 @@ class CodeGenerationSuite extends SparkFunSuite with 
ExpressionEvalHelper {
   test("SPARK-18016: define mutable states by using an array") {
     val ctx1 = new CodegenContext
     for (i <- 1 to CodeGenerator.OUTER_CLASS_VARIABLES_THRESHOLD + 10) {
-      ctx1.addMutableState(ctx1.JAVA_INT, "i", v => s"$v = $i;")
+      ctx1.addMutableState(CodeGenerator.JAVA_INT, "i", v => s"$v = $i;")
     }
     assert(ctx1.inlinedMutableStates.size == 
CodeGenerator.OUTER_CLASS_VARIABLES_THRESHOLD)
     // When the number of primitive type mutable states is over the threshold, 
others are
     // allocated into an array
-    
assert(ctx1.arrayCompactedMutableStates.get(ctx1.JAVA_INT).get.arrayNames.size 
== 1)
+    
assert(ctx1.arrayCompactedMutableStates.get(CodeGenerator.JAVA_INT).get.arrayNames.size
 == 1)
     assert(ctx1.mutableStateInitCode.size == 
CodeGenerator.OUTER_CLASS_VARIABLES_THRESHOLD + 10)
 
     val ctx2 = new CodegenContext

http://git-wip-us.apache.org/repos/asf/spark/blob/2ce37b50/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala
index 04f2619..392906a 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala
@@ -18,7 +18,7 @@
 package org.apache.spark.sql.execution
 
 import org.apache.spark.sql.catalyst.expressions.{BoundReference, UnsafeRow}
-import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
CodeGenerator, ExprCode}
 import org.apache.spark.sql.execution.metric.SQLMetrics
 import org.apache.spark.sql.types.DataType
 import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
@@ -49,15 +49,15 @@ private[sql] trait ColumnarBatchScan extends CodegenSupport 
{
       ordinal: String,
       dataType: DataType,
       nullable: Boolean): ExprCode = {
-    val javaType = ctx.javaType(dataType)
-    val value = ctx.getValueFromVector(columnVar, dataType, ordinal)
+    val javaType = CodeGenerator.javaType(dataType)
+    val value = CodeGenerator.getValueFromVector(columnVar, dataType, ordinal)
     val isNullVar = if (nullable) { ctx.freshName("isNull") } else { "false" }
     val valueVar = ctx.freshName("value")
     val str = s"columnVector[$columnVar, $ordinal, ${dataType.simpleString}]"
     val code = s"${ctx.registerComment(str)}\n" + (if (nullable) {
       s"""
         boolean $isNullVar = $columnVar.isNullAt($ordinal);
-        $javaType $valueVar = $isNullVar ? ${ctx.defaultValue(dataType)} : 
($value);
+        $javaType $valueVar = $isNullVar ? 
${CodeGenerator.defaultValue(dataType)} : ($value);
       """
     } else {
       s"$javaType $valueVar = $value;"
@@ -85,12 +85,13 @@ private[sql] trait ColumnarBatchScan extends CodegenSupport 
{
     // metrics
     val numOutputRows = metricTerm(ctx, "numOutputRows")
     val scanTimeMetric = metricTerm(ctx, "scanTime")
-    val scanTimeTotalNs = ctx.addMutableState(ctx.JAVA_LONG, "scanTime") // 
init as scanTime = 0
+    val scanTimeTotalNs =
+      ctx.addMutableState(CodeGenerator.JAVA_LONG, "scanTime") // init as 
scanTime = 0
 
     val columnarBatchClz = classOf[ColumnarBatch].getName
     val batch = ctx.addMutableState(columnarBatchClz, "batch")
 
-    val idx = ctx.addMutableState(ctx.JAVA_INT, "batchIdx") // init as 
batchIdx = 0
+    val idx = ctx.addMutableState(CodeGenerator.JAVA_INT, "batchIdx") // init 
as batchIdx = 0
     val columnVectorClzs = vectorTypes.getOrElse(
       Seq.fill(output.indices.size)(classOf[ColumnVector].getName))
     val (colVars, columnAssigns) = columnVectorClzs.zipWithIndex.map {

http://git-wip-us.apache.org/repos/asf/spark/blob/2ce37b50/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala
index a7bd5eb..12ae1ea 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala
@@ -21,7 +21,7 @@ import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.errors._
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
CodeGenerator, ExprCode}
 import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, 
UnknownPartitioning}
 import org.apache.spark.sql.execution.metric.SQLMetrics
 
@@ -154,7 +154,8 @@ case class ExpandExec(
         val value = ctx.freshName("value")
         val code = s"""
           |boolean $isNull = true;
-          |${ctx.javaType(firstExpr.dataType)} $value = 
${ctx.defaultValue(firstExpr.dataType)};
+          |${CodeGenerator.javaType(firstExpr.dataType)} $value =
+          |  ${CodeGenerator.defaultValue(firstExpr.dataType)};
          """.stripMargin
         ExprCode(code, isNull, value)
       }

http://git-wip-us.apache.org/repos/asf/spark/blob/2ce37b50/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala
index 0c2c4a1..384f039 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.execution
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
CodeGenerator, ExprCode}
 import org.apache.spark.sql.catalyst.plans.physical.Partitioning
 import org.apache.spark.sql.execution.metric.SQLMetrics
 import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructType}
@@ -305,15 +305,15 @@ case class GenerateExec(
       nullable: Boolean,
       initialChecks: Seq[String]): ExprCode = {
     val value = ctx.freshName(name)
-    val javaType = ctx.javaType(dt)
-    val getter = ctx.getValue(source, dt, index)
+    val javaType = CodeGenerator.javaType(dt)
+    val getter = CodeGenerator.getValue(source, dt, index)
     val checks = initialChecks ++ optionalCode(nullable, 
s"$source.isNullAt($index)")
     if (checks.nonEmpty) {
       val isNull = ctx.freshName("isNull")
       val code =
         s"""
            |boolean $isNull = ${checks.mkString(" || ")};
-           |$javaType $value = $isNull ? ${ctx.defaultValue(dt)} : $getter;
+           |$javaType $value = $isNull ? ${CodeGenerator.defaultValue(dt)} : 
$getter;
          """.stripMargin
       ExprCode(code, isNull, value)
     } else {

http://git-wip-us.apache.org/repos/asf/spark/blob/2ce37b50/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala
index ac1c34d..0dc16ba 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala
@@ -22,7 +22,7 @@ import org.apache.spark.executor.TaskMetrics
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
CodeGenerator, ExprCode}
 import org.apache.spark.sql.catalyst.plans.physical._
 import org.apache.spark.sql.execution.metric.SQLMetrics
 
@@ -133,7 +133,8 @@ case class SortExec(
   override def needStopCheck: Boolean = false
 
   override protected def doProduce(ctx: CodegenContext): String = {
-    val needToSort = ctx.addMutableState(ctx.JAVA_BOOLEAN, "needToSort", v => 
s"$v = true;")
+    val needToSort =
+      ctx.addMutableState(CodeGenerator.JAVA_BOOLEAN, "needToSort", v => s"$v 
= true;")
 
     // Initialize the class member variables. This includes the instance of 
the Sorter and
     // the iterator to return sorted rows.

http://git-wip-us.apache.org/repos/asf/spark/blob/2ce37b50/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala
index deb0a04..f89e3fb 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala
@@ -234,7 +234,7 @@ trait CodegenSupport extends SparkPlan {
 
     variables.zipWithIndex.foreach { case (ev, i) =>
       val paramName = ctx.freshName(s"expr_$i")
-      val paramType = ctx.javaType(attributes(i).dataType)
+      val paramType = CodeGenerator.javaType(attributes(i).dataType)
 
       arguments += ev.value
       parameters += s"$paramType $paramName"

http://git-wip-us.apache.org/repos/asf/spark/blob/2ce37b50/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala
index ce3c688..1926e93 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala
@@ -178,7 +178,7 @@ case class HashAggregateExec(
   private var bufVars: Seq[ExprCode] = _
 
   private def doProduceWithoutKeys(ctx: CodegenContext): String = {
-    val initAgg = ctx.addMutableState(ctx.JAVA_BOOLEAN, "initAgg")
+    val initAgg = ctx.addMutableState(CodeGenerator.JAVA_BOOLEAN, "initAgg")
     // The generated function doesn't have input row in the code context.
     ctx.INPUT_ROW = null
 
@@ -186,8 +186,8 @@ case class HashAggregateExec(
     val functions = 
aggregateExpressions.map(_.aggregateFunction.asInstanceOf[DeclarativeAggregate])
     val initExpr = functions.flatMap(f => f.initialValues)
     bufVars = initExpr.map { e =>
-      val isNull = ctx.addMutableState(ctx.JAVA_BOOLEAN, "bufIsNull")
-      val value = ctx.addMutableState(ctx.javaType(e.dataType), "bufValue")
+      val isNull = ctx.addMutableState(CodeGenerator.JAVA_BOOLEAN, "bufIsNull")
+      val value = ctx.addMutableState(CodeGenerator.javaType(e.dataType), 
"bufValue")
       // The initial expression should not access any column
       val ev = e.genCode(ctx)
       val initVars = s"""
@@ -532,7 +532,7 @@ case class HashAggregateExec(
    */
   private def checkIfFastHashMapSupported(ctx: CodegenContext): Boolean = {
     val isSupported =
-      (groupingKeySchema ++ bufferSchema).forall(f => 
ctx.isPrimitiveType(f.dataType) ||
+      (groupingKeySchema ++ bufferSchema).forall(f => 
CodeGenerator.isPrimitiveType(f.dataType) ||
         f.dataType.isInstanceOf[DecimalType] || 
f.dataType.isInstanceOf[StringType]) &&
         bufferSchema.nonEmpty && modes.forall(mode => mode == Partial || mode 
== PartialMerge)
 
@@ -565,7 +565,7 @@ case class HashAggregateExec(
   }
 
   private def doProduceWithKeys(ctx: CodegenContext): String = {
-    val initAgg = ctx.addMutableState(ctx.JAVA_BOOLEAN, "initAgg")
+    val initAgg = ctx.addMutableState(CodeGenerator.JAVA_BOOLEAN, "initAgg")
     if (sqlContext.conf.enableTwoLevelAggMap) {
       enableTwoLevelHashMap(ctx)
     } else {
@@ -757,7 +757,7 @@ case class HashAggregateExec(
 
     val (checkFallbackForGeneratedHashMap, checkFallbackForBytesToBytesMap, 
resetCounter,
     incCounter) = if (testFallbackStartsAt.isDefined) {
-      val countTerm = ctx.addMutableState(ctx.JAVA_INT, "fallbackCounter")
+      val countTerm = ctx.addMutableState(CodeGenerator.JAVA_INT, 
"fallbackCounter")
       (s"$countTerm < ${testFallbackStartsAt.get._1}",
         s"$countTerm < ${testFallbackStartsAt.get._2}", s"$countTerm = 0;", 
s"$countTerm += 1;")
     } else {
@@ -832,7 +832,7 @@ case class HashAggregateExec(
       }
       val updateUnsafeRowBuffer = unsafeRowBufferEvals.zipWithIndex.map { case 
(ev, i) =>
         val dt = updateExpr(i).dataType
-        ctx.updateColumn(unsafeRowBuffer, dt, i, ev, updateExpr(i).nullable)
+        CodeGenerator.updateColumn(unsafeRowBuffer, dt, i, ev, 
updateExpr(i).nullable)
       }
       s"""
          |// common sub-expressions
@@ -855,7 +855,7 @@ case class HashAggregateExec(
         }
         val updateFastRow = fastRowEvals.zipWithIndex.map { case (ev, i) =>
           val dt = updateExpr(i).dataType
-          ctx.updateColumn(
+          CodeGenerator.updateColumn(
             fastRowBuffer, dt, i, ev, updateExpr(i).nullable, 
isVectorizedHashMapEnabled)
         }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/2ce37b50/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashMapGenerator.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashMapGenerator.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashMapGenerator.scala
index 1c613b1..6b60b41 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashMapGenerator.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashMapGenerator.scala
@@ -18,7 +18,7 @@
 package org.apache.spark.sql.execution.aggregate
 
 import 
org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, 
DeclarativeAggregate}
-import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
CodeGenerator, ExprCode}
 import org.apache.spark.sql.types._
 
 /**
@@ -41,13 +41,13 @@ abstract class HashMapGenerator(
   val groupingKeys = groupingKeySchema.map(k => Buffer(k.dataType, 
ctx.freshName("key")))
   val bufferValues = bufferSchema.map(k => Buffer(k.dataType, 
ctx.freshName("value")))
   val groupingKeySignature =
-    groupingKeys.map(key => s"${ctx.javaType(key.dataType)} 
${key.name}").mkString(", ")
+    groupingKeys.map(key => s"${CodeGenerator.javaType(key.dataType)} 
${key.name}").mkString(", ")
   val buffVars: Seq[ExprCode] = {
     val functions = 
aggregateExpressions.map(_.aggregateFunction.asInstanceOf[DeclarativeAggregate])
     val initExpr = functions.flatMap(f => f.initialValues)
     initExpr.map { e =>
-      val isNull = ctx.addMutableState(ctx.JAVA_BOOLEAN, "bufIsNull")
-      val value = ctx.addMutableState(ctx.javaType(e.dataType), "bufValue")
+      val isNull = ctx.addMutableState(CodeGenerator.JAVA_BOOLEAN, "bufIsNull")
+      val value = ctx.addMutableState(CodeGenerator.javaType(e.dataType), 
"bufValue")
       val ev = e.genCode(ctx)
       val initVars =
         s"""

http://git-wip-us.apache.org/repos/asf/spark/blob/2ce37b50/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/RowBasedHashMapGenerator.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/RowBasedHashMapGenerator.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/RowBasedHashMapGenerator.scala
index fd25707..8617be8 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/RowBasedHashMapGenerator.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/RowBasedHashMapGenerator.scala
@@ -18,8 +18,8 @@
 package org.apache.spark.sql.execution.aggregate
 
 import org.apache.spark.sql.catalyst.expressions.UnsafeRow
-import 
org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression}
-import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
CodeGenerator}
 import org.apache.spark.sql.types._
 
 /**
@@ -114,7 +114,7 @@ class RowBasedHashMapGenerator(
 
     def genEqualsForKeys(groupingKeys: Seq[Buffer]): String = {
       groupingKeys.zipWithIndex.map { case (key: Buffer, ordinal: Int) =>
-        s"""(${ctx.genEqual(key.dataType, ctx.getValue("row",
+        s"""(${ctx.genEqual(key.dataType, CodeGenerator.getValue("row",
           key.dataType, ordinal.toString()), key.name)})"""
       }.mkString(" && ")
     }
@@ -147,7 +147,7 @@ class RowBasedHashMapGenerator(
         case t: DecimalType =>
           s"agg_rowWriter.write(${ordinal}, ${key.name}, ${t.precision}, 
${t.scale})"
         case t: DataType =>
-          if (!t.isInstanceOf[StringType] && !ctx.isPrimitiveType(t)) {
+          if (!t.isInstanceOf[StringType] && 
!CodeGenerator.isPrimitiveType(t)) {
             throw new IllegalArgumentException(s"cannot generate code for 
unsupported type: $t")
           }
           s"agg_rowWriter.write(${ordinal}, ${key.name})"

http://git-wip-us.apache.org/repos/asf/spark/blob/2ce37b50/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/VectorizedHashMapGenerator.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/VectorizedHashMapGenerator.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/VectorizedHashMapGenerator.scala
index 633eeac..7b3580c 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/VectorizedHashMapGenerator.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/VectorizedHashMapGenerator.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.aggregate
 
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
-import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
CodeGenerator}
 import org.apache.spark.sql.execution.vectorized.{MutableColumnarRow, 
OnHeapColumnVector}
 import org.apache.spark.sql.types._
 import org.apache.spark.sql.vectorized.ColumnarBatch
@@ -127,7 +127,8 @@ class VectorizedHashMapGenerator(
 
     def genEqualsForKeys(groupingKeys: Seq[Buffer]): String = {
       groupingKeys.zipWithIndex.map { case (key: Buffer, ordinal: Int) =>
-        val value = ctx.getValueFromVector(s"vectors[$ordinal]", key.dataType, 
"buckets[idx]")
+        val value = CodeGenerator.getValueFromVector(s"vectors[$ordinal]", 
key.dataType,
+          "buckets[idx]")
         s"(${ctx.genEqual(key.dataType, value, key.name)})"
       }.mkString(" && ")
     }
@@ -182,14 +183,14 @@ class VectorizedHashMapGenerator(
 
     def genCodeToSetKeys(groupingKeys: Seq[Buffer]): Seq[String] = {
       groupingKeys.zipWithIndex.map { case (key: Buffer, ordinal: Int) =>
-        ctx.setValue(s"vectors[$ordinal]", "numRows", key.dataType, key.name)
+        CodeGenerator.setValue(s"vectors[$ordinal]", "numRows", key.dataType, 
key.name)
       }
     }
 
     def genCodeToSetAggBuffers(bufferValues: Seq[Buffer]): Seq[String] = {
       bufferValues.zipWithIndex.map { case (key: Buffer, ordinal: Int) =>
-        ctx.updateColumn(s"vectors[${groupingKeys.length + ordinal}]", 
"numRows", key.dataType,
-          buffVars(ordinal), nullable = true)
+        CodeGenerator.updateColumn(s"vectors[${groupingKeys.length + 
ordinal}]", "numRows",
+          key.dataType, buffVars(ordinal), nullable = true)
       }
     }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/2ce37b50/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala
index a15a8d1..4707022 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala
@@ -24,7 +24,7 @@ import org.apache.spark.{InterruptibleIterator, Partition, 
SparkContext, TaskCon
 import org.apache.spark.rdd.{EmptyRDD, PartitionwiseSampledRDD, RDD}
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
ExprCode, ExpressionCanonicalizer}
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
CodeGenerator, ExprCode, ExpressionCanonicalizer}
 import org.apache.spark.sql.catalyst.plans.physical._
 import org.apache.spark.sql.execution.metric.SQLMetrics
 import org.apache.spark.sql.types.LongType
@@ -364,8 +364,8 @@ case class RangeExec(range: 
org.apache.spark.sql.catalyst.plans.logical.Range)
   protected override def doProduce(ctx: CodegenContext): String = {
     val numOutput = metricTerm(ctx, "numOutputRows")
 
-    val initTerm = ctx.addMutableState(ctx.JAVA_BOOLEAN, "initRange")
-    val number = ctx.addMutableState(ctx.JAVA_LONG, "number")
+    val initTerm = ctx.addMutableState(CodeGenerator.JAVA_BOOLEAN, "initRange")
+    val number = ctx.addMutableState(CodeGenerator.JAVA_LONG, "number")
 
     val value = ctx.freshName("value")
     val ev = ExprCode("", "false", value)
@@ -385,10 +385,10 @@ case class RangeExec(range: 
org.apache.spark.sql.catalyst.plans.logical.Range)
     // the metrics.
 
     // Once number == batchEnd, it's time to progress to the next batch.
-    val batchEnd = ctx.addMutableState(ctx.JAVA_LONG, "batchEnd")
+    val batchEnd = ctx.addMutableState(CodeGenerator.JAVA_LONG, "batchEnd")
 
     // How many values should still be generated by this range operator.
-    val numElementsTodo = ctx.addMutableState(ctx.JAVA_LONG, "numElementsTodo")
+    val numElementsTodo = ctx.addMutableState(CodeGenerator.JAVA_LONG, 
"numElementsTodo")
 
     // How many values should be generated in the next batch.
     val nextBatchTodo = ctx.freshName("nextBatchTodo")

http://git-wip-us.apache.org/repos/asf/spark/blob/2ce37b50/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala
index 4f28eeb..3b5655b 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala
@@ -91,7 +91,7 @@ object GenerateColumnAccessor extends 
CodeGenerator[Seq[DataType], ColumnarItera
       val accessorName = ctx.addMutableState(accessorCls, "accessor")
 
       val createCode = dt match {
-        case t if ctx.isPrimitiveType(dt) =>
+        case t if CodeGenerator.isPrimitiveType(dt) =>
           s"$accessorName = new 
$accessorCls(ByteBuffer.wrap(buffers[$index]).order(nativeOrder));"
         case NullType | StringType | BinaryType =>
           s"$accessorName = new 
$accessorCls(ByteBuffer.wrap(buffers[$index]).order(nativeOrder));"

http://git-wip-us.apache.org/repos/asf/spark/blob/2ce37b50/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala
index 1918fcc..487d6a2 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala
@@ -22,7 +22,7 @@ import org.apache.spark.broadcast.Broadcast
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
ExprCode, GenerateUnsafeProjection}
+import org.apache.spark.sql.catalyst.expressions.codegen._
 import org.apache.spark.sql.catalyst.plans._
 import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, 
Distribution, UnspecifiedDistribution}
 import org.apache.spark.sql.execution.{BinaryExecNode, CodegenSupport, 
SparkPlan}
@@ -182,9 +182,10 @@ case class BroadcastHashJoinExec(
         // the variables are needed even there is no matched rows
         val isNull = ctx.freshName("isNull")
         val value = ctx.freshName("value")
+        val javaType = CodeGenerator.javaType(a.dataType)
         val code = s"""
           |boolean $isNull = true;
-          |${ctx.javaType(a.dataType)} $value = 
${ctx.defaultValue(a.dataType)};
+          |$javaType $value = ${CodeGenerator.defaultValue(a.dataType)};
           |if ($matched != null) {
           |  ${ev.code}
           |  $isNull = ${ev.isNull};

http://git-wip-us.apache.org/repos/asf/spark/blob/2ce37b50/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala
index 2de2f30..5a511b3 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala
@@ -22,7 +22,7 @@ import scala.collection.mutable.ArrayBuffer
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
CodeGenerator, ExprCode}
 import org.apache.spark.sql.catalyst.plans._
 import org.apache.spark.sql.catalyst.plans.physical._
 import org.apache.spark.sql.execution.{BinaryExecNode, CodegenSupport,
@@ -516,9 +516,9 @@ case class SortMergeJoinExec(
     ctx.INPUT_ROW = leftRow
     left.output.zipWithIndex.map { case (a, i) =>
       val value = ctx.freshName("value")
-      val valueCode = ctx.getValue(leftRow, a.dataType, i.toString)
-      val javaType = ctx.javaType(a.dataType)
-      val defaultValue = ctx.defaultValue(a.dataType)
+      val valueCode = CodeGenerator.getValue(leftRow, a.dataType, i.toString)
+      val javaType = CodeGenerator.javaType(a.dataType)
+      val defaultValue = CodeGenerator.defaultValue(a.dataType)
       if (a.nullable) {
         val isNull = ctx.freshName("isNull")
         val code =

http://git-wip-us.apache.org/repos/asf/spark/blob/2ce37b50/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala
index cccee63..66bcda8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala
@@ -21,7 +21,7 @@ import org.apache.spark.rdd.RDD
 import org.apache.spark.serializer.Serializer
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
ExprCode, LazilyGeneratedOrdering}
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
CodeGenerator, ExprCode, LazilyGeneratedOrdering}
 import org.apache.spark.sql.catalyst.plans.physical._
 import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
 import org.apache.spark.util.Utils
@@ -71,7 +71,8 @@ trait BaseLimitExec extends UnaryExecNode with CodegenSupport 
{
   }
 
   override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: 
ExprCode): String = {
-    val stopEarly = ctx.addMutableState(ctx.JAVA_BOOLEAN, "stopEarly") // init 
as stopEarly = false
+    val stopEarly =
+      ctx.addMutableState(CodeGenerator.JAVA_BOOLEAN, "stopEarly") // init as 
stopEarly = false
 
     ctx.addNewFunction("stopEarly", s"""
       @Override
@@ -79,7 +80,7 @@ trait BaseLimitExec extends UnaryExecNode with CodegenSupport 
{
         return $stopEarly;
       }
     """, inlineToOuterClass = true)
-    val countTerm = ctx.addMutableState(ctx.JAVA_INT, "count") // init as 
count = 0
+    val countTerm = ctx.addMutableState(CodeGenerator.JAVA_INT, "count") // 
init as count = 0
     s"""
        | if ($countTerm < $limit) {
        |   $countTerm += 1;


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to