[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-20 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r158060614
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
 ---
@@ -112,15 +112,15 @@ case class Like(left: Expression, right: Expression) 
extends StringRegexExpressi
   override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): 
ExprCode = {
 val patternClass = classOf[Pattern].getName
 val escapeFunc = StringUtils.getClass.getName.stripSuffix("$") + 
".escapeLikeRegex"
-val pattern = ctx.freshName("pattern")
 
 if (right.foldable) {
   val rVal = right.eval()
   if (rVal != null) {
 val regexStr =
   
StringEscapeUtils.escapeJava(escape(rVal.asInstanceOf[UTF8String].toString()))
-ctx.addMutableState(patternClass, pattern,
-  s"""$pattern = ${patternClass}.compile("$regexStr");""")
+// inline mutable state since not many Like operations in a task
--- End diff --

Sure


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-19 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/spark/pull/19811


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-19 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157798132
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala
 ---
@@ -583,42 +579,41 @@ case class HashAggregateExec(
 val thisPlan = ctx.addReferenceObj("plan", this)
 
 // Create a name for the iterator from the fast hash map.
-val iterTermForFastHashMap = ctx.freshName("fastHashMapIter")
-if (isFastHashMapEnabled) {
+val iterTermForFastHashMap = if (isFastHashMapEnabled) {
   // Generates the fast hash map class and creates the fash hash map 
term.
-  fastHashMapTerm = ctx.freshName("fastHashMap")
   val fastHashMapClassName = ctx.freshName("FastHashMap")
   if (isVectorizedHashMapEnabled) {
 val generatedMap = new VectorizedHashMapGenerator(ctx, 
aggregateExpressions,
   fastHashMapClassName, groupingKeySchema, bufferSchema).generate()
 ctx.addInnerClass(generatedMap)
 
-ctx.addMutableState(fastHashMapClassName, fastHashMapTerm,
-  s"$fastHashMapTerm = new $fastHashMapClassName();")
-ctx.addMutableState(s"java.util.Iterator", 
iterTermForFastHashMap)
+fastHashMapTerm = ctx.addMutableState(fastHashMapClassName, 
"vectorizedHastHashMap",
--- End diff --

shall we force inline it too?


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-19 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157798448
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala
 ---
@@ -583,42 +579,41 @@ case class HashAggregateExec(
 val thisPlan = ctx.addReferenceObj("plan", this)
 
 // Create a name for the iterator from the fast hash map.
-val iterTermForFastHashMap = ctx.freshName("fastHashMapIter")
-if (isFastHashMapEnabled) {
+val iterTermForFastHashMap = if (isFastHashMapEnabled) {
   // Generates the fast hash map class and creates the fash hash map 
term.
-  fastHashMapTerm = ctx.freshName("fastHashMap")
   val fastHashMapClassName = ctx.freshName("FastHashMap")
   if (isVectorizedHashMapEnabled) {
 val generatedMap = new VectorizedHashMapGenerator(ctx, 
aggregateExpressions,
   fastHashMapClassName, groupingKeySchema, bufferSchema).generate()
 ctx.addInnerClass(generatedMap)
 
-ctx.addMutableState(fastHashMapClassName, fastHashMapTerm,
-  s"$fastHashMapTerm = new $fastHashMapClassName();")
-ctx.addMutableState(s"java.util.Iterator", 
iterTermForFastHashMap)
+fastHashMapTerm = ctx.addMutableState(fastHashMapClassName, 
"vectorizedHastHashMap",
+  v => s"$v = new $fastHashMapClassName();")
+ctx.addMutableState(s"java.util.Iterator", 
"vectorizedFastHashMapIter")
   } else {
 val generatedMap = new RowBasedHashMapGenerator(ctx, 
aggregateExpressions,
   fastHashMapClassName, groupingKeySchema, bufferSchema).generate()
 ctx.addInnerClass(generatedMap)
 
-ctx.addMutableState(fastHashMapClassName, fastHashMapTerm,
-  s"$fastHashMapTerm = new $fastHashMapClassName(" +
+fastHashMapTerm = ctx.addMutableState(fastHashMapClassName, 
"fastHashMap",
+  v => s"$v = new $fastHashMapClassName(" +
 s"$thisPlan.getTaskMemoryManager(), 
$thisPlan.getEmptyAggregationBuffer());")
 ctx.addMutableState(
   "org.apache.spark.unsafe.KVIterator",
-  iterTermForFastHashMap)
+  "fastHashMapIter")
   }
 }
 
 // Create a name for the iterator from the regular hash map.
-val iterTerm = ctx.freshName("mapIter")
-ctx.addMutableState(classOf[KVIterator[UnsafeRow, UnsafeRow]].getName, 
iterTerm)
+// inline mutable state since not many aggregation operations in a task
+val iterTerm = ctx.addMutableState(classOf[KVIterator[UnsafeRow, 
UnsafeRow]].getName,
+  "mapIter", forceInline = true)
 // create hashMap
-hashMapTerm = ctx.freshName("hashMap")
 val hashMapClassName = classOf[UnsafeFixedWidthAggregationMap].getName
-ctx.addMutableState(hashMapClassName, hashMapTerm, s"$hashMapTerm = 
$thisPlan.createHashMap();")
-sorterTerm = ctx.freshName("sorter")
-ctx.addMutableState(classOf[UnsafeKVExternalSorter].getName, 
sorterTerm)
+hashMapTerm = ctx.addMutableState(hashMapClassName, "hashMap",
+  v => s"$v = $thisPlan.createHashMap();")
--- End diff --

ditto


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-19 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157797890
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala
 ---
@@ -282,9 +282,10 @@ case class InputAdapter(child: SparkPlan) extends 
UnaryExecNode with CodegenSupp
   }
 
   override def doProduce(ctx: CodegenContext): String = {
-val input = ctx.freshName("input")
 // Right now, InputAdapter is only used when there is one input RDD.
-ctx.addMutableState("scala.collection.Iterator", input, s"$input = 
inputs[0];")
+// inline mutable state since an inputAdaptor in a task
--- End diff --

typo


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-19 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157794484
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -137,22 +137,63 @@ class CodegenContext {
   var currentVars: Seq[ExprCode] = null
 
   /**
-   * Holding expressions' mutable states like 
`MonotonicallyIncreasingID.count` as a
-   * 3-tuple: java type, variable name, code to init it.
-   * As an example, ("int", "count", "count = 0;") will produce code:
+   * Holding expressions' inlined mutable states like 
`MonotonicallyIncreasingID.count` as a
+   * 2-tuple: java type, variable name.
+   * As an example, ("int", "count") will produce code:
* {{{
*   private int count;
* }}}
-   * as a member variable, and add
-   * {{{
-   *   count = 0;
-   * }}}
-   * to the constructor.
+   * as a member variable
*
* They will be kept as member variables in generated classes like 
`SpecificProjection`.
*/
-  val mutableStates: mutable.ArrayBuffer[(String, String, String)] =
-mutable.ArrayBuffer.empty[(String, String, String)]
+  val inlinedMutableStates: mutable.ArrayBuffer[(String, String)] =
+mutable.ArrayBuffer.empty[(String, String)]
+
+  /**
+   * The mapping between mutable state types and corrseponding compacted 
arrays.
+   * The keys are java type string. The values are [[MutableStateArrays]] 
which encapsulates
+   * the compacted arrays for the mutable states with the same java type.
+   */
+  val arrayCompactedMutableStates: mutable.Map[String, MutableStateArrays] 
=
+mutable.Map.empty[String, MutableStateArrays]
+
+  // An array holds the code that will initialize each state
+  val mutableStateInitCode: mutable.ArrayBuffer[String] =
+mutable.ArrayBuffer.empty[String]
+
+  /**
+   * This class holds a set of names of mutableStateArrays that is used 
for compacting mutable
+   * states for a certain type, and holds the next available slot of the 
current compacted array.
+   */
+  class MutableStateArrays {
+val arrayNames = mutable.ListBuffer.empty[String]
+createNewArray()
+
+private[this] var currentIndex = 0
+
+private def createNewArray() = 
arrayNames.append(freshName("mutableStateArray"))
+
+def getCurrentIndex: Int = currentIndex
+
+/**
+ * Returns the reference of next available slot in current compacted 
array. The size of each
+ * compacted array is controlled by the config 
`CodeGenerator.MUTABLESTATEARRAY_SIZE_LIMIT`.
--- End diff --

nit: `CodeGenerator.MUTABLESTATEARRAY_SIZE_LIMIT` is not a config


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-19 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157797565
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
 ---
@@ -187,15 +188,15 @@ case class RLike(left: Expression, right: Expression) 
extends StringRegexExpress
 
   override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): 
ExprCode = {
 val patternClass = classOf[Pattern].getName
-val pattern = ctx.freshName("pattern")
 
 if (right.foldable) {
   val rVal = right.eval()
   if (rVal != null) {
 val regexStr =
   
StringEscapeUtils.escapeJava(rVal.asInstanceOf[UTF8String].toString())
-ctx.addMutableState(patternClass, pattern,
-  s"""$pattern = ${patternClass}.compile("$regexStr");""")
+// inline mutable state since not many RLike operations in a task
--- End diff --

ditto


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-19 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157797472
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
 ---
@@ -112,15 +112,15 @@ case class Like(left: Expression, right: Expression) 
extends StringRegexExpressi
   override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): 
ExprCode = {
 val patternClass = classOf[Pattern].getName
 val escapeFunc = StringUtils.getClass.getName.stripSuffix("$") + 
".escapeLikeRegex"
-val pattern = ctx.freshName("pattern")
 
 if (right.foldable) {
   val rVal = right.eval()
   if (rVal != null) {
 val regexStr =
   
StringEscapeUtils.escapeJava(escape(rVal.asInstanceOf[UTF8String].toString()))
-ctx.addMutableState(patternClass, pattern,
-  s"""$pattern = ${patternClass}.compile("$regexStr");""")
+// inline mutable state since not many Like operations in a task
--- End diff --

I'm not very sure about this, since `Like` is an expression and can appear 
many times, like other expressions.


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-19 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157798917
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala
 ---
@@ -436,14 +435,13 @@ case class SortMergeJoinExec(
 val rightKeyVars = copyKeys(ctx, rightKeyTmpVars)
 
 // A list to hold all matched rows from right side.
-val matches = ctx.freshName("matches")
 val clsName = classOf[ExternalAppendOnlyUnsafeRowArray].getName
 
 val spillThreshold = getSpillThreshold
 val inMemoryThreshold = getInMemoryThreshold
 
-ctx.addMutableState(clsName, matches,
-  s"$matches = new $clsName($inMemoryThreshold, $spillThreshold);")
+val matches = ctx.addMutableState(clsName, "matches",
+  v => s"$v = new $clsName($inMemoryThreshold, $spillThreshold);")
--- End diff --

ditto


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-19 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157796723
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
 ---
@@ -112,15 +112,14 @@ case class Like(left: Expression, right: Expression) 
extends StringRegexExpressi
   override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): 
ExprCode = {
 val patternClass = classOf[Pattern].getName
 val escapeFunc = StringUtils.getClass.getName.stripSuffix("$") + 
".escapeLikeRegex"
-val pattern = ctx.freshName("pattern")
 
 if (right.foldable) {
   val rVal = right.eval()
   if (rVal != null) {
 val regexStr =
   
StringEscapeUtils.escapeJava(escape(rVal.asInstanceOf[UTF8String].toString()))
-ctx.addMutableState(patternClass, pattern,
-  s"""$pattern = ${patternClass}.compile("$regexStr");""")
+val pattern = ctx.addMutableState(patternClass, "patternLike",
+  v => s"""$v = ${patternClass}.compile("$regexStr");""", 
forceInline = true)
--- End diff --

Based on comments, I simplified rules.
1. Use the original name
1. Expect to be not-frequently used.

In the latter, I put comment regarding the reason at each site .


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-18 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157518488
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
 ---
@@ -112,15 +112,14 @@ case class Like(left: Expression, right: Expression) 
extends StringRegexExpressi
   override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): 
ExprCode = {
 val patternClass = classOf[Pattern].getName
 val escapeFunc = StringUtils.getClass.getName.stripSuffix("$") + 
".escapeLikeRegex"
-val pattern = ctx.freshName("pattern")
 
 if (right.foldable) {
   val rVal = right.eval()
   if (rVal != null) {
 val regexStr =
   
StringEscapeUtils.escapeJava(escape(rVal.asInstanceOf[UTF8String].toString()))
-ctx.addMutableState(patternClass, pattern,
-  s"""$pattern = ${patternClass}.compile("$regexStr");""")
+val pattern = ctx.addMutableState(patternClass, "patternLike",
+  v => s"""$v = ${patternClass}.compile("$regexStr");""", 
forceInline = true)
--- End diff --

If 3 is a precondition for 2, then it is ok. Thanks for the explanation.


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-18 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157517422
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala
 ---
@@ -385,20 +385,43 @@ class CodeGenerationSuite extends SparkFunSuite with 
ExpressionEvalHelper {
 val ctx = new CodegenContext
 val schema = new StructType().add("a", IntegerType).add("b", 
StringType)
 CreateExternalRow(Seq(Literal(1), Literal("x")), schema).genCode(ctx)
-assert(ctx.mutableStates.isEmpty)
+assert(ctx.inlinedMutableStates.isEmpty)
   }
 
   test("SPARK-22696: InitializeJavaBean should not use global variables") {
 val ctx = new CodegenContext
 InitializeJavaBean(Literal.fromObject(new java.util.LinkedList[Int]),
   Map("add" -> Literal(1))).genCode(ctx)
-assert(ctx.mutableStates.isEmpty)
+assert(ctx.inlinedMutableStates.isEmpty)
   }
 
   test("SPARK-22716: addReferenceObj should not add mutable states") {
 val ctx = new CodegenContext
 val foo = new Object()
 ctx.addReferenceObj("foo", foo)
-assert(ctx.mutableStates.isEmpty)
+assert(ctx.inlinedMutableStates.isEmpty)
+  }
+
+  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;")
+}
+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
--- End diff --

Yeah, I agree. In the future, we hope we have an AST based codegen 
framework.


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-18 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157517090
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
 ---
@@ -112,15 +112,14 @@ case class Like(left: Expression, right: Expression) 
extends StringRegexExpressi
   override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): 
ExprCode = {
 val patternClass = classOf[Pattern].getName
 val escapeFunc = StringUtils.getClass.getName.stripSuffix("$") + 
".escapeLikeRegex"
-val pattern = ctx.freshName("pattern")
 
 if (right.foldable) {
   val rVal = right.eval()
   if (rVal != null) {
 val regexStr =
   
StringEscapeUtils.escapeJava(escape(rVal.asInstanceOf[UTF8String].toString()))
-ctx.addMutableState(patternClass, pattern,
-  s"""$pattern = ${patternClass}.compile("$regexStr");""")
+val pattern = ctx.addMutableState(patternClass, "patternLike",
+  v => s"""$v = ${patternClass}.compile("$regexStr");""", 
forceInline = true)
--- End diff --

Sorry for confusing you about 2. I updated the statement. It is frequently 
referenced, but generated once. Therefore, if we have advantage for 
performance, we think it is safer since 3. will also apply inline.


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-17 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157417517
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -163,11 +204,51 @@ class CodegenContext {
* the list of default imports available.
* Also, generic type arguments are accepted but ignored.
* @param variableName Name of the field.
-   * @param initCode The statement(s) to put into the init() method to 
initialize this field.
+   * @param initFunc Function includes statement(s) to put into the init() 
method to initialize
+   * this field. The argument is the name of the mutable 
state variable.
* If left blank, the field will be default-initialized.
+   * @param forceInline whether the declaration and initialization code 
may be inlined rather than
+   *compacted. Please set `true` into forceInline, if 
you want to access the
+   *status fast (e.g. frequently accessed) or if you 
want to use the original
+   *variable name
+   * @param useFreshName If this is false and forceInline is true, the 
name is not changed
+   * @return the name of the mutable state variable, which is the original 
name or fresh name if
+   * the variable is inlined to the outer class, or an array 
access if the variable is to
+   * be stored in an array of variables of the same type.
+   * A variable will be inlined into the outer class when one of 
the following conditions
+   * are satisfied:
+   * 1. forceInline is true
+   * 2. its type is primitive type and the total number of the 
inlined mutable variables
+   *is less than 
`CodeGenerator.OUTER_CLASS_VARIABLES_THRESHOLD`
+   * 3. its type is multi-dimensional array
+   * A primitive type variable will be inlined into outer class 
when the total number of
+   * When a variable is compacted into an array, the max size of 
the array for compaction
--- End diff --

Actually this line `A primitive type variable will be inlined into outer 
class when the total number of` looks redundant.


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-17 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157417320
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -163,11 +204,51 @@ class CodegenContext {
* the list of default imports available.
* Also, generic type arguments are accepted but ignored.
* @param variableName Name of the field.
-   * @param initCode The statement(s) to put into the init() method to 
initialize this field.
+   * @param initFunc Function includes statement(s) to put into the init() 
method to initialize
+   * this field. The argument is the name of the mutable 
state variable.
* If left blank, the field will be default-initialized.
+   * @param forceInline whether the declaration and initialization code 
may be inlined rather than
+   *compacted. Please set `true` into forceInline, if 
you want to access the
+   *status fast (e.g. frequently accessed) or if you 
want to use the original
+   *variable name
+   * @param useFreshName If this is false and forceInline is true, the 
name is not changed
+   * @return the name of the mutable state variable, which is the original 
name or fresh name if
+   * the variable is inlined to the outer class, or an array 
access if the variable is to
+   * be stored in an array of variables of the same type.
+   * A variable will be inlined into the outer class when one of 
the following conditions
+   * are satisfied:
+   * 1. forceInline is true
+   * 2. its type is primitive type and the total number of the 
inlined mutable variables
+   *is less than 
`CodeGenerator.OUTER_CLASS_VARIABLES_THRESHOLD`
+   * 3. its type is multi-dimensional array
+   * A primitive type variable will be inlined into outer class 
when the total number of
+   * When a variable is compacted into an array, the max size of 
the array for compaction
--- End diff --

The sentences looks broken? I.e., `...total number of`


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-17 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157415970
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
 ---
@@ -112,15 +112,14 @@ case class Like(left: Expression, right: Expression) 
extends StringRegexExpressi
   override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): 
ExprCode = {
 val patternClass = classOf[Pattern].getName
 val escapeFunc = StringUtils.getClass.getName.stripSuffix("$") + 
".escapeLikeRegex"
-val pattern = ctx.freshName("pattern")
 
 if (right.foldable) {
   val rVal = right.eval()
   if (rVal != null) {
 val regexStr =
   
StringEscapeUtils.escapeJava(escape(rVal.asInstanceOf[UTF8String].toString()))
-ctx.addMutableState(patternClass, pattern,
-  s"""$pattern = ${patternClass}.compile("$regexStr");""")
+val pattern = ctx.addMutableState(patternClass, "patternLike",
+  v => s"""$v = ${patternClass}.compile("$regexStr");""", 
forceInline = true)
--- End diff --

my only concern is about point 2. I think it is a dangerous thing to do. 
What if we generate a lot of frequently used variable? I think it is safer at 
the moment to consider only 1 and 3 in the decision whether to inline or not. 
In the future, with a different codegen method, we might then define a 
threshold over which we generate an array for the given class, otherwise we use 
plain variables, which IMHO would be the best option but at the moment it is 
not feasible...


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-17 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157410773
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
 ---
@@ -112,15 +112,14 @@ case class Like(left: Expression, right: Expression) 
extends StringRegexExpressi
   override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): 
ExprCode = {
 val patternClass = classOf[Pattern].getName
 val escapeFunc = StringUtils.getClass.getName.stripSuffix("$") + 
".escapeLikeRegex"
-val pattern = ctx.freshName("pattern")
 
 if (right.foldable) {
   val rVal = right.eval()
   if (rVal != null) {
 val regexStr =
   
StringEscapeUtils.escapeJava(escape(rVal.asInstanceOf[UTF8String].toString()))
-ctx.addMutableState(patternClass, pattern,
-  s"""$pattern = ${patternClass}.compile("$regexStr");""")
+val pattern = ctx.addMutableState(patternClass, "patternLike",
+  v => s"""$v = ${patternClass}.compile("$regexStr");""", 
forceInline = true)
--- End diff --

yes, if it's not a lot of them...


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-17 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157410521
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala
 ---
@@ -440,9 +435,9 @@ case class RangeExec(range: 
org.apache.spark.sql.catalyst.plans.logical.Range)
 | }
""".stripMargin)
 
-val input = ctx.freshName("input")
 // Right now, Range is only used when there is one upstream.
-ctx.addMutableState("scala.collection.Iterator", input, s"$input = 
inputs[0];")
+val input = ctx.addMutableState("scala.collection.Iterator", "input",
--- End diff --

seems it's never used


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-17 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157410349
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala ---
@@ -133,20 +133,17 @@ case class SortExec(
   override def needStopCheck: Boolean = false
 
   override protected def doProduce(ctx: CodegenContext): String = {
-val needToSort = ctx.freshName("needToSort")
-ctx.addMutableState(ctx.JAVA_BOOLEAN, needToSort, s"$needToSort = 
true;")
+val needToSort = ctx.addMutableState(ctx.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.
 val thisPlan = ctx.addReferenceObj("plan", this)
-sorterVariable = ctx.freshName("sorter")
-ctx.addMutableState(classOf[UnsafeExternalRowSorter].getName, 
sorterVariable,
-  s"$sorterVariable = $thisPlan.createSorter();")
-val metrics = ctx.freshName("metrics")
-ctx.addMutableState(classOf[TaskMetrics].getName, metrics,
-  s"$metrics = org.apache.spark.TaskContext.get().taskMetrics();")
-val sortedIterator = ctx.freshName("sortedIter")
-ctx.addMutableState("scala.collection.Iterator", 
sortedIterator, "")
+sorterVariable = 
ctx.addMutableState(classOf[UnsafeExternalRowSorter].getName, "sorter",
+  v => s"$v = $thisPlan.createSorter();", forceInline = true)
+val metrics = ctx.addMutableState(classOf[TaskMetrics].getName, 
"metrics",
+  v => s"$v = org.apache.spark.TaskContext.get().taskMetrics();", 
forceInline = true)
+val sortedIterator = 
ctx.addMutableState("scala.collection.Iterator", "sortedIter",
+  forceInline = true)
--- End diff --

e.g. 
https://github.com/apache/spark/pull/19811/files#diff-2eb948516b5beaeb746aadac27fbd5b4R613
 ?


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-17 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157410191
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala ---
@@ -133,20 +133,17 @@ case class SortExec(
   override def needStopCheck: Boolean = false
 
   override protected def doProduce(ctx: CodegenContext): String = {
-val needToSort = ctx.freshName("needToSort")
-ctx.addMutableState(ctx.JAVA_BOOLEAN, needToSort, s"$needToSort = 
true;")
+val needToSort = ctx.addMutableState(ctx.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.
 val thisPlan = ctx.addReferenceObj("plan", this)
-sorterVariable = ctx.freshName("sorter")
-ctx.addMutableState(classOf[UnsafeExternalRowSorter].getName, 
sorterVariable,
-  s"$sorterVariable = $thisPlan.createSorter();")
-val metrics = ctx.freshName("metrics")
-ctx.addMutableState(classOf[TaskMetrics].getName, metrics,
-  s"$metrics = org.apache.spark.TaskContext.get().taskMetrics();")
-val sortedIterator = ctx.freshName("sortedIter")
-ctx.addMutableState("scala.collection.Iterator", 
sortedIterator, "")
+sorterVariable = 
ctx.addMutableState(classOf[UnsafeExternalRowSorter].getName, "sorter",
+  v => s"$v = $thisPlan.createSorter();", forceInline = true)
+val metrics = ctx.addMutableState(classOf[TaskMetrics].getName, 
"metrics",
+  v => s"$v = org.apache.spark.TaskContext.get().taskMetrics();", 
forceInline = true)
+val sortedIterator = 
ctx.addMutableState("scala.collection.Iterator", "sortedIter",
+  forceInline = true)
--- End diff --

one question: is there any other places like this? do you have a list?


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-17 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157410164
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
 ---
@@ -112,15 +112,14 @@ case class Like(left: Expression, right: Expression) 
extends StringRegexExpressi
   override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): 
ExprCode = {
 val patternClass = classOf[Pattern].getName
 val escapeFunc = StringUtils.getClass.getName.stripSuffix("$") + 
".escapeLikeRegex"
-val pattern = ctx.freshName("pattern")
 
 if (right.foldable) {
   val rVal = right.eval()
   if (rVal != null) {
 val regexStr =
   
StringEscapeUtils.escapeJava(escape(rVal.asInstanceOf[UTF8String].toString()))
-ctx.addMutableState(patternClass, pattern,
-  s"""$pattern = ${patternClass}.compile("$regexStr");""")
+val pattern = ctx.addMutableState(patternClass, "patternLike",
+  v => s"""$v = ${patternClass}.compile("$regexStr");""", 
forceInline = true)
--- End diff --

Now, we have three rules to apply inlining
1. Have to use the original name
2. Frequently used in the hot spot
3. [Not expected to be frequently 
generated](https://github.com/apache/spark/pull/19811#discussion_r157110933) 
proposed by @viirya 

Now, we have no rule for 2. I will try to run microbenchmark for 2. Is it 
better to add these benchmarks into the benchmark directory?



---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-17 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157410133
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala ---
@@ -133,20 +133,17 @@ case class SortExec(
   override def needStopCheck: Boolean = false
 
   override protected def doProduce(ctx: CodegenContext): String = {
-val needToSort = ctx.freshName("needToSort")
-ctx.addMutableState(ctx.JAVA_BOOLEAN, needToSort, s"$needToSort = 
true;")
+val needToSort = ctx.addMutableState(ctx.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.
 val thisPlan = ctx.addReferenceObj("plan", this)
-sorterVariable = ctx.freshName("sorter")
-ctx.addMutableState(classOf[UnsafeExternalRowSorter].getName, 
sorterVariable,
-  s"$sorterVariable = $thisPlan.createSorter();")
-val metrics = ctx.freshName("metrics")
-ctx.addMutableState(classOf[TaskMetrics].getName, metrics,
-  s"$metrics = org.apache.spark.TaskContext.get().taskMetrics();")
-val sortedIterator = ctx.freshName("sortedIter")
-ctx.addMutableState("scala.collection.Iterator", 
sortedIterator, "")
+sorterVariable = 
ctx.addMutableState(classOf[UnsafeExternalRowSorter].getName, "sorter",
+  v => s"$v = $thisPlan.createSorter();", forceInline = true)
+val metrics = ctx.addMutableState(classOf[TaskMetrics].getName, 
"metrics",
+  v => s"$v = org.apache.spark.TaskContext.get().taskMetrics();", 
forceInline = true)
+val sortedIterator = 
ctx.addMutableState("scala.collection.Iterator", "sortedIter",
+  forceInline = true)
--- End diff --

this looks reasonable as it's very unlikely we have a lot of sort operators 
in one stage. We have to inline it manually as we don't have the ability to 
find this out automatically yet. Same as 
https://github.com/apache/spark/pull/19811/files#r157408804


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-17 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157409537
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
 ---
@@ -217,7 +217,7 @@ case class Stack(children: Seq[Expression]) extends 
Generator {
 ctx.addMutableState(
   s"$wrapperClass",
   ev.value,
-  s"${ev.value} = $wrapperClass$$.MODULE$$.make($rowData);")
--- End diff --

I will do it in another PR.


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-17 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157408804
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala
 ---
@@ -385,20 +385,43 @@ class CodeGenerationSuite extends SparkFunSuite with 
ExpressionEvalHelper {
 val ctx = new CodegenContext
 val schema = new StructType().add("a", IntegerType).add("b", 
StringType)
 CreateExternalRow(Seq(Literal(1), Literal("x")), schema).genCode(ctx)
-assert(ctx.mutableStates.isEmpty)
+assert(ctx.inlinedMutableStates.isEmpty)
   }
 
   test("SPARK-22696: InitializeJavaBean should not use global variables") {
 val ctx = new CodegenContext
 InitializeJavaBean(Literal.fromObject(new java.util.LinkedList[Int]),
   Map("add" -> Literal(1))).genCode(ctx)
-assert(ctx.mutableStates.isEmpty)
+assert(ctx.inlinedMutableStates.isEmpty)
   }
 
   test("SPARK-22716: addReferenceObj should not add mutable states") {
 val ctx = new CodegenContext
 val foo = new Object()
 ctx.addReferenceObj("foo", foo)
-assert(ctx.mutableStates.isEmpty)
+assert(ctx.inlinedMutableStates.isEmpty)
+  }
+
+  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;")
+}
+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
--- End diff --

Some notes: It's better if we can collect all mutable states before 
deciding which one should be inlined. However it's impossible to do with the 
current string based codegen framework, we need to decide inline or not 
immediately. We can revisit this in the future when we have an AST based 
codegen framework.


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-17 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157408425
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
 ---
@@ -112,15 +112,14 @@ case class Like(left: Expression, right: Expression) 
extends StringRegexExpressi
   override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): 
ExprCode = {
 val patternClass = classOf[Pattern].getName
 val escapeFunc = StringUtils.getClass.getName.stripSuffix("$") + 
".escapeLikeRegex"
-val pattern = ctx.freshName("pattern")
 
 if (right.foldable) {
   val rVal = right.eval()
   if (rVal != null) {
 val regexStr =
   
StringEscapeUtils.escapeJava(escape(rVal.asInstanceOf[UTF8String].toString()))
-ctx.addMutableState(patternClass, pattern,
-  s"""$pattern = ${patternClass}.compile("$regexStr");""")
+val pattern = ctx.addMutableState(patternClass, "patternLike",
+  v => s"""$v = ${patternClass}.compile("$regexStr");""", 
forceInline = true)
--- End diff --

Do we have a clear rule when a global variable should be inlined for better 
performance? e.g. a microbenchmark  showing noteworthy difference definitely 
proves we should inline.


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-17 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157408061
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
 ---
@@ -217,7 +217,7 @@ case class Stack(children: Seq[Expression]) extends 
Generator {
 ctx.addMutableState(
   s"$wrapperClass",
   ev.value,
-  s"${ev.value} = $wrapperClass$$.MODULE$$.make($rowData);")
--- End diff --

We can localize the global variable `ev.value` here to save one global 
variable slot.


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-17 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157407681
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -163,11 +204,51 @@ class CodegenContext {
* the list of default imports available.
* Also, generic type arguments are accepted but ignored.
* @param variableName Name of the field.
-   * @param initCode The statement(s) to put into the init() method to 
initialize this field.
+   * @param initFunc Function includes statement(s) to put into the init() 
method to initialize
+   * this field. The argument is the name of the mutable 
state variable.
* If left blank, the field will be default-initialized.
+   * @param forceInline whether the declaration and initialization code 
may be inlined rather than
+   *compacted. Please set `true` into forceInline, if 
you want to access the
+   *status fast (e.g. frequently accessed) or if you 
want to use the original
+   *variable name
+   * @param useFreshName If this is false and forceInline is true, the 
name is not changed
--- End diff --

more accurate: `If this is false and mutable state ends up inlining in the 
outer class, the name is not changed`


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-15 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157153803
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -1039,9 +1127,8 @@ 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.
-  addMutableState(JAVA_BOOLEAN, isNull, s"$isNull = false;")
-  addMutableState(javaType(expr.dataType), value,
-s"$value = ${defaultValue(expr.dataType)};")
+  addMutableState(JAVA_BOOLEAN, isNull, forceInline = true, 
useFreshName = false)
+  addMutableState(javaType(expr.dataType), value, forceInline = true, 
useFreshName = false)
--- End diff --

can we do
```
val isNull = addMutableState(JAVA_BOOLEAN, "subExprIsNull")
val value = addMutableState(javaType(expr.dataType), "subExprValue")
val fn = ...
```
at the beginning?


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-15 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157153467
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -1039,9 +1127,8 @@ 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.
-  addMutableState(JAVA_BOOLEAN, isNull, s"$isNull = false;")
-  addMutableState(javaType(expr.dataType), value,
-s"$value = ${defaultValue(expr.dataType)};")
+  addMutableState(JAVA_BOOLEAN, isNull, forceInline = true, 
useFreshName = false)
+  addMutableState(javaType(expr.dataType), value, forceInline = true, 
useFreshName = false)
--- End diff --

actually we can use fresh name here. just do
```
val isNull = freshName("subExprIsNull")
val value = freshName("subExprIsValue")
val fn = ...
```
at the beginning


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-15 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157152448
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -189,15 +255,37 @@ class CodegenContext {
   def declareMutableStates(): String = {
 // It's possible that we add same mutable state twice, e.g. the 
`mergeExpressions` in
 // `TypedAggregateExpression`, we should call `distinct` here to 
remove the duplicated ones.
-mutableStates.distinct.map { case (javaType, variableName, _) =>
+val inlinedStates = inlinedMutableStates.distinct.map { case 
(javaType, variableName) =>
   s"private $javaType $variableName;"
-}.mkString("\n")
+}
+
+val arrayStates = arrayCompactedMutableStates.flatMap { case 
(javaType, mutableStateArrays) =>
--- End diff --

how would `arrayCompactedMutableStates` has duplications?


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-15 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157151571
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala
 ---
@@ -385,20 +385,44 @@ class CodeGenerationSuite extends SparkFunSuite with 
ExpressionEvalHelper {
 val ctx = new CodegenContext
 val schema = new StructType().add("a", IntegerType).add("b", 
StringType)
 CreateExternalRow(Seq(Literal(1), Literal("x")), schema).genCode(ctx)
-assert(ctx.mutableStates.isEmpty)
+assert(ctx.inlinedMutableStates.isEmpty)
   }
 
   test("SPARK-22696: InitializeJavaBean should not use global variables") {
 val ctx = new CodegenContext
 InitializeJavaBean(Literal.fromObject(new java.util.LinkedList[Int]),
   Map("add" -> Literal(1))).genCode(ctx)
-assert(ctx.mutableStates.isEmpty)
+assert(ctx.inlinedMutableStates.isEmpty)
   }
 
   test("SPARK-22716: addReferenceObj should not add mutable states") {
 val ctx = new CodegenContext
 val foo = new Object()
 ctx.addReferenceObj("foo", foo)
-assert(ctx.mutableStates.isEmpty)
+assert(ctx.inlinedMutableStates.isEmpty)
+  }
+
+  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;")
+}
+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.mutableStateInitCode.size == 
CodeGenerator.OUTER_CLASS_VARIABLES_THRESHOLD + 10)
+
+val ctx2 = new CodegenContext
+for (i <- 1 to CodeGenerator.MUTABLESTATEARRAY_SIZE_LIMIT + 10) {
+  ctx2.addMutableState("InternalRow[]", "r", v => s"$v = new 
InternalRow[$i];")
+}
+// When the number of non-primitive type mutable states is over the 
threshold, others are
+// allocated into a new array
+assert(ctx2.inlinedMutableStates.isEmpty)
+
assert(ctx2.arrayCompactedMutableStates.get("InternalRow[]").get.arrayNames.size
 == 2)
+
assert(ctx2.arrayCompactedMutableStates("InternalRow[]").getCurrentIndex == 10)
+assert(ctx2.mutableStateInitCode.size == 
CodeGenerator.MUTABLESTATEARRAY_SIZE_LIMIT + 10)
   }
 }
+
--- End diff --

nit: useless extra line at the end of file


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-15 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157148074
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -138,21 +138,50 @@ class CodegenContext {
 
   /**
* Holding expressions' mutable states like 
`MonotonicallyIncreasingID.count` as a
-   * 3-tuple: java type, variable name, code to init it.
-   * As an example, ("int", "count", "count = 0;") will produce code:
+   * 2-tuple: java type, variable name.
+   * As an example, ("int", "count") will produce code:
* {{{
*   private int count;
* }}}
-   * as a member variable, and add
-   * {{{
-   *   count = 0;
-   * }}}
-   * to the constructor.
+   * as a member variable
*
* They will be kept as member variables in generated classes like 
`SpecificProjection`.
*/
-  val mutableStates: mutable.ArrayBuffer[(String, String, String)] =
-mutable.ArrayBuffer.empty[(String, String, String)]
+  val inlinedMutableStates: mutable.ArrayBuffer[(String, String)] =
+mutable.ArrayBuffer.empty[(String, String)]
+
+  // An map keyed by mutable states' types holds the status of 
mutableStateArray
+  val arrayCompactedMutableStates: mutable.Map[String, MutableStateArrays] 
=
+mutable.Map.empty[String, MutableStateArrays]
+
+  // An array holds the code that will initialize each state
+  val mutableStateInitCode: mutable.ArrayBuffer[String] =
+mutable.ArrayBuffer.empty[String]
+
+  // Holding names and current index of mutableStateArrays for a certain 
type
+  class MutableStateArrays {
--- End diff --

oh, right.


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-15 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157147636
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -138,21 +138,50 @@ class CodegenContext {
 
   /**
* Holding expressions' mutable states like 
`MonotonicallyIncreasingID.count` as a
-   * 3-tuple: java type, variable name, code to init it.
-   * As an example, ("int", "count", "count = 0;") will produce code:
+   * 2-tuple: java type, variable name.
+   * As an example, ("int", "count") will produce code:
* {{{
*   private int count;
* }}}
-   * as a member variable, and add
-   * {{{
-   *   count = 0;
-   * }}}
-   * to the constructor.
+   * as a member variable
*
* They will be kept as member variables in generated classes like 
`SpecificProjection`.
*/
-  val mutableStates: mutable.ArrayBuffer[(String, String, String)] =
-mutable.ArrayBuffer.empty[(String, String, String)]
+  val inlinedMutableStates: mutable.ArrayBuffer[(String, String)] =
+mutable.ArrayBuffer.empty[(String, String)]
+
+  // An map keyed by mutable states' types holds the status of 
mutableStateArray
+  val arrayCompactedMutableStates: mutable.Map[String, MutableStateArrays] 
=
+mutable.Map.empty[String, MutableStateArrays]
+
+  // An array holds the code that will initialize each state
+  val mutableStateInitCode: mutable.ArrayBuffer[String] =
+mutable.ArrayBuffer.empty[String]
+
+  // Holding names and current index of mutableStateArrays for a certain 
type
+  class MutableStateArrays {
--- End diff --

do we need to? the generated code looks like 
```
String[] mutableStateArray = ...
```
So the type info is already there.


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157127908
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala
 ---
@@ -219,4 +219,31 @@ class GeneratedProjectionSuite extends SparkFunSuite {
 // - one is the mutableRow
 assert(globalVariables.length == 3)
   }
+
+  test("SPARK-18016: generated projections on wider table requiring state 
compaction") {
+val N = 6000
+val wideRow1 = new GenericInternalRow((0 until N).toArray[Any])
+val schema1 = StructType((1 to N).map(i => StructField("", 
IntegerType)))
+val wideRow2 = new GenericInternalRow(
+  (0 until N).map(i => UTF8String.fromString(i.toString)).toArray[Any])
+val schema2 = StructType((1 to N).map(i => StructField("", 
StringType)))
+val joined = new JoinedRow(wideRow1, wideRow2)
+val joinedSchema = StructType(schema1 ++ schema2)
+val nested = new JoinedRow(InternalRow(joined, joined), joined)
+val nestedSchema = StructType(
+  Seq(StructField("", joinedSchema), StructField("", joinedSchema)) ++ 
joinedSchema)
+
+val safeProj = FromUnsafeProjection(nestedSchema)
+val result = safeProj(nested)
+
+// test generated MutableProjection
+val exprs = nestedSchema.fields.zipWithIndex.map { case (f, i) =>
+  BoundReference(i, f.dataType, true)
+}
+val mutableProj = GenerateMutableProjection.generate(exprs)
--- End diff --

yea, the original pr has such a test though, IIUC state compaction in 
`GenerateMutableProjection`  never happens?


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157111250
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala ---
@@ -71,17 +71,15 @@ trait BaseLimitExec extends UnaryExecNode with 
CodegenSupport {
   }
 
   override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: 
ExprCode): String = {
-val stopEarly = ctx.freshName("stopEarly")
-ctx.addMutableState(ctx.JAVA_BOOLEAN, stopEarly, s"$stopEarly = 
false;")
+val stopEarly = ctx.addMutableState(ctx.JAVA_BOOLEAN, "stopEarly") // 
init as stopEarly = 0
--- End diff --

0 =>`false`?


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157110993
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala ---
@@ -133,20 +133,17 @@ case class SortExec(
   override def needStopCheck: Boolean = false
 
   override protected def doProduce(ctx: CodegenContext): String = {
-val needToSort = ctx.freshName("needToSort")
-ctx.addMutableState(ctx.JAVA_BOOLEAN, needToSort, s"$needToSort = 
true;")
+val needToSort = ctx.addMutableState(ctx.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.
 val thisPlan = ctx.addReferenceObj("plan", this)
-sorterVariable = ctx.freshName("sorter")
-ctx.addMutableState(classOf[UnsafeExternalRowSorter].getName, 
sorterVariable,
-  s"$sorterVariable = $thisPlan.createSorter();")
-val metrics = ctx.freshName("metrics")
-ctx.addMutableState(classOf[TaskMetrics].getName, metrics,
-  s"$metrics = org.apache.spark.TaskContext.get().taskMetrics();")
-val sortedIterator = ctx.freshName("sortedIter")
-ctx.addMutableState("scala.collection.Iterator", 
sortedIterator, "")
+sorterVariable = 
ctx.addMutableState(classOf[UnsafeExternalRowSorter].getName, "sorter",
+  v => s"$v = $thisPlan.createSorter();")
+val metrics = ctx.addMutableState(classOf[TaskMetrics].getName, 
"metrics",
+  v => s"$v = org.apache.spark.TaskContext.get().taskMetrics();")
--- End diff --

ditto.


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157110933
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala ---
@@ -133,20 +133,17 @@ case class SortExec(
   override def needStopCheck: Boolean = false
 
   override protected def doProduce(ctx: CodegenContext): String = {
-val needToSort = ctx.freshName("needToSort")
-ctx.addMutableState(ctx.JAVA_BOOLEAN, needToSort, s"$needToSort = 
true;")
+val needToSort = ctx.addMutableState(ctx.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.
 val thisPlan = ctx.addReferenceObj("plan", this)
-sorterVariable = ctx.freshName("sorter")
-ctx.addMutableState(classOf[UnsafeExternalRowSorter].getName, 
sorterVariable,
-  s"$sorterVariable = $thisPlan.createSorter();")
-val metrics = ctx.freshName("metrics")
-ctx.addMutableState(classOf[TaskMetrics].getName, metrics,
-  s"$metrics = org.apache.spark.TaskContext.get().taskMetrics();")
-val sortedIterator = ctx.freshName("sortedIter")
-ctx.addMutableState("scala.collection.Iterator", 
sortedIterator, "")
+sorterVariable = 
ctx.addMutableState(classOf[UnsafeExternalRowSorter].getName, "sorter",
+  v => s"$v = $thisPlan.createSorter();")
--- End diff --

I think we don't have too many `UnsafeExternalRowSorter`, it should be 
inlined?


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157110352
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala
 ---
@@ -57,41 +57,37 @@ object GenerateMutableProjection extends 
CodeGenerator[Seq[Expression], MutableP
   case _ => true
 }.unzip
 val exprVals = ctx.generateExpressions(validExpr, 
useSubexprElimination)
-val projectionCodes = exprVals.zip(index).map {
+
+// 4-tuples: (code for projection, isNull variable name, value 
variable name, column index)
+val projectionCodes: Seq[(String, String, String, Int)] = 
exprVals.zip(index).map {
--- End diff --

I see.


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157110169
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala
 ---
@@ -57,41 +57,37 @@ object GenerateMutableProjection extends 
CodeGenerator[Seq[Expression], MutableP
   case _ => true
 }.unzip
 val exprVals = ctx.generateExpressions(validExpr, 
useSubexprElimination)
-val projectionCodes = exprVals.zip(index).map {
+
+// 4-tuples: (code for projection, isNull variable name, value 
variable name, column index)
+val projectionCodes: Seq[(String, String, String, Int)] = 
exprVals.zip(index).map {
--- End diff --

Why needs to return column index?


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157109584
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -189,15 +255,37 @@ class CodegenContext {
   def declareMutableStates(): String = {
 // It's possible that we add same mutable state twice, e.g. the 
`mergeExpressions` in
 // `TypedAggregateExpression`, we should call `distinct` here to 
remove the duplicated ones.
-mutableStates.distinct.map { case (javaType, variableName, _) =>
+val inlinedStates = inlinedMutableStates.distinct.map { case 
(javaType, variableName) =>
   s"private $javaType $variableName;"
-}.mkString("\n")
+}
+
+val arrayStates = arrayCompactedMutableStates.flatMap { case 
(javaType, mutableStateArrays) =>
--- End diff --

Is it possible that we add same mutable state to 
`arrayCompactedMutableStates` like `inlinedMutableStates`? We can't just do 
distinct like `inlinedMutableStates`.


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157108625
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -138,21 +138,50 @@ class CodegenContext {
 
   /**
* Holding expressions' mutable states like 
`MonotonicallyIncreasingID.count` as a
-   * 3-tuple: java type, variable name, code to init it.
-   * As an example, ("int", "count", "count = 0;") will produce code:
+   * 2-tuple: java type, variable name.
+   * As an example, ("int", "count") will produce code:
* {{{
*   private int count;
* }}}
-   * as a member variable, and add
-   * {{{
-   *   count = 0;
-   * }}}
-   * to the constructor.
+   * as a member variable
*
* They will be kept as member variables in generated classes like 
`SpecificProjection`.
*/
-  val mutableStates: mutable.ArrayBuffer[(String, String, String)] =
-mutable.ArrayBuffer.empty[(String, String, String)]
+  val inlinedMutableStates: mutable.ArrayBuffer[(String, String)] =
+mutable.ArrayBuffer.empty[(String, String)]
+
+  // An map keyed by mutable states' types holds the status of 
mutableStateArray
+  val arrayCompactedMutableStates: mutable.Map[String, MutableStateArrays] 
=
+mutable.Map.empty[String, MutableStateArrays]
+
+  // An array holds the code that will initialize each state
+  val mutableStateInitCode: mutable.ArrayBuffer[String] =
+mutable.ArrayBuffer.empty[String]
+
+  // Holding names and current index of mutableStateArrays for a certain 
type
+  class MutableStateArrays {
--- End diff --

Can we add a java type string into this as parameter. So we can let the 
array name with type info?


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157108429
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -138,21 +138,50 @@ class CodegenContext {
 
   /**
* Holding expressions' mutable states like 
`MonotonicallyIncreasingID.count` as a
-   * 3-tuple: java type, variable name, code to init it.
-   * As an example, ("int", "count", "count = 0;") will produce code:
+   * 2-tuple: java type, variable name.
+   * As an example, ("int", "count") will produce code:
* {{{
*   private int count;
* }}}
-   * as a member variable, and add
-   * {{{
-   *   count = 0;
-   * }}}
-   * to the constructor.
+   * as a member variable
*
* They will be kept as member variables in generated classes like 
`SpecificProjection`.
*/
-  val mutableStates: mutable.ArrayBuffer[(String, String, String)] =
-mutable.ArrayBuffer.empty[(String, String, String)]
+  val inlinedMutableStates: mutable.ArrayBuffer[(String, String)] =
+mutable.ArrayBuffer.empty[(String, String)]
+
+  // An map keyed by mutable states' types holds the status of 
mutableStateArray
+  val arrayCompactedMutableStates: mutable.Map[String, MutableStateArrays] 
=
+mutable.Map.empty[String, MutableStateArrays]
+
+  // An array holds the code that will initialize each state
+  val mutableStateInitCode: mutable.ArrayBuffer[String] =
+mutable.ArrayBuffer.empty[String]
+
+  // Holding names and current index of mutableStateArrays for a certain 
type
+  class MutableStateArrays {
+val arrayNames = mutable.ListBuffer.empty[String]
+createNewArray()
+
+private[this] var currentIndex = 0
+
+private def createNewArray() = 
arrayNames.append(freshName("mutableStateArray"))
+
+def getCurrentIndex: Int = currentIndex
+
+def getNextSlot(): String = {
--- End diff --

```scala
/**
 * Returns the reference of next available slot in current compacted array. 
The size of each compacted array
 * is controlled by the config 
`CodeGenerator.MUTABLESTATEARRAY_SIZE_LIMIT`. Once reaching the
 * threshold, new compacted array is created.
 */
```


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157108041
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -138,21 +138,50 @@ class CodegenContext {
 
   /**
* Holding expressions' mutable states like 
`MonotonicallyIncreasingID.count` as a
-   * 3-tuple: java type, variable name, code to init it.
-   * As an example, ("int", "count", "count = 0;") will produce code:
+   * 2-tuple: java type, variable name.
+   * As an example, ("int", "count") will produce code:
* {{{
*   private int count;
* }}}
-   * as a member variable, and add
-   * {{{
-   *   count = 0;
-   * }}}
-   * to the constructor.
+   * as a member variable
*
* They will be kept as member variables in generated classes like 
`SpecificProjection`.
*/
-  val mutableStates: mutable.ArrayBuffer[(String, String, String)] =
-mutable.ArrayBuffer.empty[(String, String, String)]
+  val inlinedMutableStates: mutable.ArrayBuffer[(String, String)] =
+mutable.ArrayBuffer.empty[(String, String)]
+
+  // An map keyed by mutable states' types holds the status of 
mutableStateArray
+  val arrayCompactedMutableStates: mutable.Map[String, MutableStateArrays] 
=
--- End diff --

```scala
/**
 * The mapping between mutable state types and corrseponding compacted 
arrays.
 * The keys are java type string. The values are [[MutableStateArrays]] 
which encapsulates
 * the compacted arrays for the mutable states with the same java type.  
 */
```


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157107687
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -138,21 +138,50 @@ class CodegenContext {
 
   /**
* Holding expressions' mutable states like 
`MonotonicallyIncreasingID.count` as a
-   * 3-tuple: java type, variable name, code to init it.
-   * As an example, ("int", "count", "count = 0;") will produce code:
+   * 2-tuple: java type, variable name.
+   * As an example, ("int", "count") will produce code:
* {{{
*   private int count;
* }}}
-   * as a member variable, and add
-   * {{{
-   *   count = 0;
-   * }}}
-   * to the constructor.
+   * as a member variable
*
* They will be kept as member variables in generated classes like 
`SpecificProjection`.
*/
-  val mutableStates: mutable.ArrayBuffer[(String, String, String)] =
-mutable.ArrayBuffer.empty[(String, String, String)]
+  val inlinedMutableStates: mutable.ArrayBuffer[(String, String)] =
+mutable.ArrayBuffer.empty[(String, String)]
+
+  // An map keyed by mutable states' types holds the status of 
mutableStateArray
+  val arrayCompactedMutableStates: mutable.Map[String, MutableStateArrays] 
=
+mutable.Map.empty[String, MutableStateArrays]
+
+  // An array holds the code that will initialize each state
+  val mutableStateInitCode: mutable.ArrayBuffer[String] =
+mutable.ArrayBuffer.empty[String]
+
+  // Holding names and current index of mutableStateArrays for a certain 
type
--- End diff --

The comment is too simple. Let's explain it more.


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157107378
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -138,21 +138,50 @@ class CodegenContext {
 
   /**
* Holding expressions' mutable states like 
`MonotonicallyIncreasingID.count` as a
-   * 3-tuple: java type, variable name, code to init it.
-   * As an example, ("int", "count", "count = 0;") will produce code:
+   * 2-tuple: java type, variable name.
+   * As an example, ("int", "count") will produce code:
* {{{
*   private int count;
* }}}
-   * as a member variable, and add
-   * {{{
-   *   count = 0;
-   * }}}
-   * to the constructor.
+   * as a member variable
*
* They will be kept as member variables in generated classes like 
`SpecificProjection`.
*/
-  val mutableStates: mutable.ArrayBuffer[(String, String, String)] =
-mutable.ArrayBuffer.empty[(String, String, String)]
+  val inlinedMutableStates: mutable.ArrayBuffer[(String, String)] =
--- End diff --

Add comment explaining this stores only mutable states inlined in outer 
class.


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157107145
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -163,11 +192,49 @@ class CodegenContext {
* the list of default imports available.
* Also, generic type arguments are accepted but ignored.
* @param variableName Name of the field.
-   * @param initCode The statement(s) to put into the init() method to 
initialize this field.
+   * @param initFunc Function includes statement(s) to put into the init() 
method to initialize
+   * this field. The argument is the name of the mutable 
state variable.
* If left blank, the field will be default-initialized.
+   * @param forceInline whether the declaration and initialization code 
may be inlined rather than
+   *compacted. Please set `true` into forceInline, if 
you want to access the
+   *status fast (e.g. frequently accessed) or if you 
want to use the original
+   *variable name
+   * @param useFreshName If false and inline is true, the name is not 
changed
+   * @return the name of the mutable state variable, which is either the 
original name if the
+   * variable is inlined to the outer class, or an array access if 
the variable is to be
+   * stored in an array of variables of the same type.
+   * There are two use cases. One is to use the original name for 
global variable instead
+   * of fresh name. Second is to use the original initialization 
statement since it is
+   * complex (e.g. allocate multi-dimensional array or object 
constructor has varibles).
+   * Primitive type variables will be inlined into outer class 
when the total number of
+   * mutable variables is less than 
`CodeGenerator.OUTER_CLASS_VARIABLES_THRESHOLD`
+   * the max size of an array for compaction is given by
+   * `CodeGenerator.MUTABLESTATEARRAY_SIZE_LIMIT`.
*/
-  def addMutableState(javaType: String, variableName: String, initCode: 
String = ""): Unit = {
-mutableStates += ((javaType, variableName, initCode))
+  def addMutableState(
+  javaType: String,
+  variableName: String,
+  initFunc: String => String = _ => "",
+  forceInline: Boolean = false,
+  useFreshName: Boolean = true): String = {
+
+// want to put a primitive type variable at outerClass for performance
+val canInlinePrimitive = isPrimitiveType(javaType) &&
+  (inlinedMutableStates.length < 
CodeGenerator.OUTER_CLASS_VARIABLES_THRESHOLD)
+if (forceInline || canInlinePrimitive || javaType.contains("[][]")) {
--- End diff --

Explain what conditions we do inline the state in comment.


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157106984
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -163,11 +192,49 @@ class CodegenContext {
* the list of default imports available.
* Also, generic type arguments are accepted but ignored.
* @param variableName Name of the field.
-   * @param initCode The statement(s) to put into the init() method to 
initialize this field.
+   * @param initFunc Function includes statement(s) to put into the init() 
method to initialize
+   * this field. The argument is the name of the mutable 
state variable.
* If left blank, the field will be default-initialized.
+   * @param forceInline whether the declaration and initialization code 
may be inlined rather than
+   *compacted. Please set `true` into forceInline, if 
you want to access the
+   *status fast (e.g. frequently accessed) or if you 
want to use the original
+   *variable name
+   * @param useFreshName If false and inline is true, the name is not 
changed
+   * @return the name of the mutable state variable, which is either the 
original name if the
+   * variable is inlined to the outer class, or an array access if 
the variable is to be
+   * stored in an array of variables of the same type.
+   * There are two use cases. One is to use the original name for 
global variable instead
+   * of fresh name. Second is to use the original initialization 
statement since it is
+   * complex (e.g. allocate multi-dimensional array or object 
constructor has varibles).
+   * Primitive type variables will be inlined into outer class 
when the total number of
--- End diff --

Some sentences here are not related to `return` doc. We should put it to 
proper place, maybe move them above the param doc.


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r157097867
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala
 ---
@@ -279,29 +279,29 @@ case class SampleExec(
 
   override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: 
ExprCode): String = {
 val numOutput = metricTerm(ctx, "numOutputRows")
-val sampler = ctx.freshName("sampler")
 
 if (withReplacement) {
   val samplerClass = classOf[PoissonSampler[UnsafeRow]].getName
   val initSampler = ctx.freshName("initSampler")
 
-  val initSamplerFuncName = ctx.addNewFunction(initSampler,
-s"""
-  | private void $initSampler() {
-  |   $sampler = new $samplerClass($upperBound - 
$lowerBound, false);
-  |   java.util.Random random = new java.util.Random(${seed}L);
-  |   long randomSeed = random.nextLong();
-  |   int loopCount = 0;
-  |   while (loopCount < partitionIndex) {
-  | randomSeed = random.nextLong();
-  | loopCount += 1;
-  |   }
-  |   $sampler.setSeed(randomSeed);
-  | }
- """.stripMargin.trim)
-
-  ctx.addMutableState(s"$samplerClass", sampler,
-s"$initSamplerFuncName();")
+  val sampler = ctx.addMutableState(s"$samplerClass", 
"sampleReplace",
+v => {
+  val initSamplerFuncName = ctx.addNewFunction(initSampler,
+s"""
+  | private void $initSampler() {
+  |   $v = new $samplerClass($upperBound - 
$lowerBound, false);
+  |   java.util.Random random = new java.util.Random(${seed}L);
+  |   long randomSeed = random.nextLong();
+  |   int loopCount = 0;
+  |   while (loopCount < partitionIndex) {
+  | randomSeed = random.nextLong();
+  | loopCount += 1;
+  |   }
+  |   $v.setSeed(randomSeed);
+  | }
+   """.stripMargin.trim)
--- End diff --

I think trim is not needed


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156996431
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -163,11 +192,49 @@ class CodegenContext {
* the list of default imports available.
* Also, generic type arguments are accepted but ignored.
* @param variableName Name of the field.
-   * @param initCode The statement(s) to put into the init() method to 
initialize this field.
+   * @param initFunc Function includes statement(s) to put into the init() 
method to initialize
+   * this field. The argument is the name of the mutable 
state variable.
* If left blank, the field will be default-initialized.
+   * @param forceInline whether the declaration and initialization code 
may be inlined rather than
+   *compacted. Please set `true` into forceInline, if 
you want to access the
+   *status fast (e.g. frequently accessed) or if you 
want to use the original
+   *variable name
+   * @param useFreshName If false and inline is true, the name is not 
changed
+   * @return the name of the mutable state variable, which is either the 
original name if the
+   * variable is inlined to the outer class, or an array access if 
the variable is to be
+   * stored in an array of variables of the same type and 
initialization.
+   * There are two use cases. One is to use the original name for 
global variable instead
+   * of fresh name. Second is to use the original initialization 
statement since it is
+   * complex (e.g. allocate multi-dimensional array or object 
constructor has varibles).
+   * Primitive type variables will be inlined into outer class 
when the total number of
+   * mutable variables is less than 
`CodeGenerator.OUTER_CLASS_VARIABLES_THRESHOLD`
+   * the max size of an array for compaction is given by
+   * `CodeGenerator.MUTABLESTATEARRAY_SIZE_LIMIT`.
*/
-  def addMutableState(javaType: String, variableName: String, initCode: 
String = ""): Unit = {
-mutableStates += ((javaType, variableName, initCode))
+  def addMutableState(
+  javaType: String,
+  variableName: String,
+  initFunc: String => String = _ => "",
+  forceInline: Boolean = false,
+  useFreshName: Boolean = true): String = {
+val varName = if (useFreshName) freshName(variableName) else 
variableName
--- End diff --

Sure, let us discuss in another PR.


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156968572
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -138,21 +138,50 @@ class CodegenContext {
 
   /**
* Holding expressions' mutable states like 
`MonotonicallyIncreasingID.count` as a
-   * 3-tuple: java type, variable name, code to init it.
-   * As an example, ("int", "count", "count = 0;") will produce code:
+   * 2-tuple: java type, variable name.
+   * As an example, ("int", "count") will produce code:
* {{{
*   private int count;
* }}}
-   * as a member variable, and add
-   * {{{
-   *   count = 0;
-   * }}}
-   * to the constructor.
+   * as a member variable
*
* They will be kept as member variables in generated classes like 
`SpecificProjection`.
*/
-  val mutableStates: mutable.ArrayBuffer[(String, String, String)] =
-mutable.ArrayBuffer.empty[(String, String, String)]
+  val mutableStates: mutable.ArrayBuffer[(String, String)] =
+mutable.ArrayBuffer.empty[(String, String)]
+
+  // An map keyed by mutable states' types holds the status of 
mutableStateArray
+  val mutableStateArrayMap: mutable.Map[String, MutableStateArrays] =
+mutable.Map.empty[String, MutableStateArrays]
+
+  // An array holds the code that will initialize each state
+  val mutableStateInitCodes: mutable.ArrayBuffer[String] =
--- End diff --

Sure, I would appreciate it if you put the link to a note


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156967829
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala 
---
@@ -353,8 +353,8 @@ case class FileSourceScanExec(
 }
 val numOutputRows = metricTerm(ctx, "numOutputRows")
 // PhysicalRDD always just has one input
-val input = ctx.freshName("input")
-ctx.addMutableState("scala.collection.Iterator", input, s"$input = 
inputs[0];")
+val input = ctx.addMutableState("scala.collection.Iterator", "input", 
v => s"$v = inputs[0];",
+  forceInline = true)
--- End diff --

ditto


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156967588
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala 
---
@@ -110,8 +110,8 @@ case class RowDataSourceScanExec(
   override protected def doProduce(ctx: CodegenContext): String = {
 val numOutputRows = metricTerm(ctx, "numOutputRows")
 // PhysicalRDD always just has one input
-val input = ctx.freshName("input")
-ctx.addMutableState("scala.collection.Iterator", input, s"$input = 
inputs[0];")
+val input = ctx.addMutableState("scala.collection.Iterator", "input", 
v => s"$v = inputs[0];",
+  forceInline = true)
--- End diff --

ditto


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156967475
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala 
---
@@ -68,30 +68,26 @@ private[sql] trait ColumnarBatchScan extends 
CodegenSupport {
*/
   // TODO: return ColumnarBatch.Rows instead
   override protected def doProduce(ctx: CodegenContext): String = {
-val input = ctx.freshName("input")
 // PhysicalRDD always just has one input
-ctx.addMutableState("scala.collection.Iterator", input, s"$input = 
inputs[0];")
+val input = ctx.addMutableState("scala.collection.Iterator", "input",
+  v => s"$v = inputs[0];", forceInline = true)
--- End diff --

In the original implementation, an assignment that refers to a variable was 
inlined. Let me stop inline.


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156966971
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
 ---
@@ -1205,14 +1203,14 @@ case class DecodeUsingSerializer[T](child: 
Expression, tag: ClassTag[T], kryo: B
 // try conf from env, otherwise create a new one
 val env = s"${classOf[SparkEnv].getName}.get()"
 val sparkConf = s"new ${classOf[SparkConf].getName}()"
-val serializerInit = s"""
-  if ($env == null) {
-$serializer = ($serializerInstanceClass) new 
$serializerClass($sparkConf).newInstance();
+val serializer = ctx.addMutableState(serializerInstanceClass, 
"serializerForDecode",
+  v => s"""
+   if ($env == null) {
+ $v = ($serializerInstanceClass) new 
$serializerClass($sparkConf).newInstance();
} else {
- $serializer = ($serializerInstanceClass) new 
$serializerClass($env.conf()).newInstance();
+ $v = ($serializerInstanceClass) new 
$serializerClass($env.conf()).newInstance();
}
- """
-ctx.addMutableState(serializerInstanceClass, serializer, 
serializerInit)
+ """, forceInline = true)
--- End diff --

ditto


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156967021
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -138,21 +138,50 @@ class CodegenContext {
 
   /**
* Holding expressions' mutable states like 
`MonotonicallyIncreasingID.count` as a
-   * 3-tuple: java type, variable name, code to init it.
-   * As an example, ("int", "count", "count = 0;") will produce code:
+   * 2-tuple: java type, variable name.
+   * As an example, ("int", "count") will produce code:
* {{{
*   private int count;
* }}}
-   * as a member variable, and add
-   * {{{
-   *   count = 0;
-   * }}}
-   * to the constructor.
+   * as a member variable
*
* They will be kept as member variables in generated classes like 
`SpecificProjection`.
*/
-  val mutableStates: mutable.ArrayBuffer[(String, String, String)] =
-mutable.ArrayBuffer.empty[(String, String, String)]
+  val mutableStates: mutable.ArrayBuffer[(String, String)] =
+mutable.ArrayBuffer.empty[(String, String)]
+
+  // An map keyed by mutable states' types holds the status of 
mutableStateArray
+  val mutableStateArrayMap: mutable.Map[String, MutableStateArrays] =
+mutable.Map.empty[String, MutableStateArrays]
+
+  // An array holds the code that will initialize each state
+  val mutableStateInitCodes: mutable.ArrayBuffer[String] =
--- End diff --

yea let's use `code`, I picked `codes` previously by mistake, you can also 
fix all of them.


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156966787
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
 ---
@@ -1159,14 +1158,14 @@ case class EncodeUsingSerializer(child: Expression, 
kryo: Boolean)
 // try conf from env, otherwise create a new one
 val env = s"${classOf[SparkEnv].getName}.get()"
 val sparkConf = s"new ${classOf[SparkConf].getName}()"
-val serializerInit = s"""
-  if ($env == null) {
-$serializer = ($serializerInstanceClass) new 
$serializerClass($sparkConf).newInstance();
-   } else {
- $serializer = ($serializerInstanceClass) new 
$serializerClass($env.conf()).newInstance();
-   }
- """
-ctx.addMutableState(serializerInstanceClass, serializer, 
serializerInit)
+val serializer = ctx.addMutableState(serializerInstanceClass, 
"serializerForEncode", v =>
+  s"""
+ |if ($env == null) {
+ |  $v = ($serializerInstanceClass) new 
$serializerClass($sparkConf).newInstance();
+ |} else {
+ |  $v = ($serializerInstanceClass) new 
$serializerClass($env.conf()).newInstance();
+ |}
+   """, forceInline = true)
--- End diff --

In the original implementation, constructor with non-constant value is 
inlined. Let me stop inline.


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156966742
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -163,11 +192,49 @@ class CodegenContext {
* the list of default imports available.
* Also, generic type arguments are accepted but ignored.
* @param variableName Name of the field.
-   * @param initCode The statement(s) to put into the init() method to 
initialize this field.
+   * @param initFunc Function includes statement(s) to put into the init() 
method to initialize
+   * this field. The argument is the name of the mutable 
state variable.
* If left blank, the field will be default-initialized.
+   * @param forceInline whether the declaration and initialization code 
may be inlined rather than
+   *compacted. Please set `true` into forceInline, if 
you want to access the
+   *status fast (e.g. frequently accessed) or if you 
want to use the original
+   *variable name
+   * @param useFreshName If false and inline is true, the name is not 
changed
+   * @return the name of the mutable state variable, which is either the 
original name if the
+   * variable is inlined to the outer class, or an array access if 
the variable is to be
+   * stored in an array of variables of the same type and 
initialization.
+   * There are two use cases. One is to use the original name for 
global variable instead
+   * of fresh name. Second is to use the original initialization 
statement since it is
+   * complex (e.g. allocate multi-dimensional array or object 
constructor has varibles).
+   * Primitive type variables will be inlined into outer class 
when the total number of
+   * mutable variables is less than 
`CodeGenerator.OUTER_CLASS_VARIABLES_THRESHOLD`
+   * the max size of an array for compaction is given by
+   * `CodeGenerator.MUTABLESTATEARRAY_SIZE_LIMIT`.
*/
-  def addMutableState(javaType: String, variableName: String, initCode: 
String = ""): Unit = {
-mutableStates += ((javaType, variableName, initCode))
+  def addMutableState(
+  javaType: String,
+  variableName: String,
+  initFunc: String => String = _ => "",
+  forceInline: Boolean = false,
+  useFreshName: Boolean = true): String = {
+val varName = if (useFreshName) freshName(variableName) else 
variableName
--- End diff --

isn't it an existing problem? Let's fix it in another PR to make this PR 
more consistent with the previous code.


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156964736
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
 ---
@@ -484,18 +484,17 @@ case class WeekOfYear(child: Expression) extends 
UnaryExpression with ImplicitCa
   override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
 nullSafeCodeGen(ctx, ev, time => {
   val cal = classOf[Calendar].getName
-  val c = ctx.freshName("cal")
   val dtu = DateTimeUtils.getClass.getName.stripSuffix("$")
-  ctx.addMutableState(cal, c,
+  val c = ctx.addMutableState(cal, "cal", v =>
 s"""
-  $c = $cal.getInstance($dtu.getTimeZone("UTC"));
-  $c.setFirstDayOfWeek($cal.MONDAY);
-  $c.setMinimalDaysInFirstWeek(4);
+   |$v = $cal.getInstance($dtu.getTimeZone("UTC"));
+   |$v.setFirstDayOfWeek($cal.MONDAY);
+   |$v.setMinimalDaysInFirstWeek(4);
  """)
--- End diff --

good catch, thanks


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156959928
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -138,21 +138,50 @@ class CodegenContext {
 
   /**
* Holding expressions' mutable states like 
`MonotonicallyIncreasingID.count` as a
-   * 3-tuple: java type, variable name, code to init it.
-   * As an example, ("int", "count", "count = 0;") will produce code:
+   * 2-tuple: java type, variable name.
+   * As an example, ("int", "count") will produce code:
* {{{
*   private int count;
* }}}
-   * as a member variable, and add
-   * {{{
-   *   count = 0;
-   * }}}
-   * to the constructor.
+   * as a member variable
*
* They will be kept as member variables in generated classes like 
`SpecificProjection`.
*/
-  val mutableStates: mutable.ArrayBuffer[(String, String, String)] =
-mutable.ArrayBuffer.empty[(String, String, String)]
+  val mutableStates: mutable.ArrayBuffer[(String, String)] =
+mutable.ArrayBuffer.empty[(String, String)]
+
+  // An map keyed by mutable states' types holds the status of 
mutableStateArray
+  val mutableStateArrayMap: mutable.Map[String, MutableStateArrays] =
+mutable.Map.empty[String, MutableStateArrays]
+
+  // An array holds the code that will initialize each state
+  val mutableStateInitCodes: mutable.ArrayBuffer[String] =
--- End diff --

I don't think it is a big deal, but I remember a note by @gatorsmile who 
advised not to use it anymore.


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156954383
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -138,21 +138,50 @@ class CodegenContext {
 
   /**
* Holding expressions' mutable states like 
`MonotonicallyIncreasingID.count` as a
-   * 3-tuple: java type, variable name, code to init it.
-   * As an example, ("int", "count", "count = 0;") will produce code:
+   * 2-tuple: java type, variable name.
+   * As an example, ("int", "count") will produce code:
* {{{
*   private int count;
* }}}
-   * as a member variable, and add
-   * {{{
-   *   count = 0;
-   * }}}
-   * to the constructor.
+   * as a member variable
*
* They will be kept as member variables in generated classes like 
`SpecificProjection`.
*/
-  val mutableStates: mutable.ArrayBuffer[(String, String, String)] =
-mutable.ArrayBuffer.empty[(String, String, String)]
+  val mutableStates: mutable.ArrayBuffer[(String, String)] =
+mutable.ArrayBuffer.empty[(String, String)]
+
+  // An map keyed by mutable states' types holds the status of 
mutableStateArray
+  val mutableStateArrayMap: mutable.Map[String, MutableStateArrays] =
+mutable.Map.empty[String, MutableStateArrays]
+
+  // An array holds the code that will initialize each state
+  val mutableStateInitCodes: mutable.ArrayBuffer[String] =
--- End diff --

You are right in English. We are seeing some `codes` variable in source 
files. Is this renaming a big deal?
WDYT?


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156953610
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala
 ---
@@ -219,4 +219,31 @@ class GeneratedProjectionSuite extends SparkFunSuite {
 // - one is the mutableRow
 assert(globalVariables.length == 3)
   }
+
+  test("SPARK-18016: generated projections on wider table requiring state 
compaction") {
+val N = 6000
+val wideRow1 = new GenericInternalRow((0 until N).toArray[Any])
+val schema1 = StructType((1 to N).map(i => StructField("", 
IntegerType)))
+val wideRow2 = new GenericInternalRow(
+  (0 until N).map(i => UTF8String.fromString(i.toString)).toArray[Any])
+val schema2 = StructType((1 to N).map(i => StructField("", 
StringType)))
+val joined = new JoinedRow(wideRow1, wideRow2)
+val joinedSchema = StructType(schema1 ++ schema2)
+val nested = new JoinedRow(InternalRow(joined, joined), joined)
+val nestedSchema = StructType(
+  Seq(StructField("", joinedSchema), StructField("", joinedSchema)) ++ 
joinedSchema)
+
+val safeProj = FromUnsafeProjection(nestedSchema)
+val result = safeProj(nested)
+
+// test generated MutableProjection
+val exprs = nestedSchema.fields.zipWithIndex.map { case (f, i) =>
+  BoundReference(i, f.dataType, true)
+}
+val mutableProj = GenerateMutableProjection.generate(exprs)
--- End diff --

cc @maropu


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156953675
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -163,11 +192,49 @@ class CodegenContext {
* the list of default imports available.
* Also, generic type arguments are accepted but ignored.
* @param variableName Name of the field.
-   * @param initCode The statement(s) to put into the init() method to 
initialize this field.
+   * @param initFunc Function includes statement(s) to put into the init() 
method to initialize
+   * this field. The argument is the name of the mutable 
state variable.
* If left blank, the field will be default-initialized.
+   * @param forceInline whether the declaration and initialization code 
may be inlined rather than
+   *compacted. Please set `true` into forceInline, if 
you want to access the
+   *status fast (e.g. frequently accessed) or if you 
want to use the original
+   *variable name
+   * @param useFreshName If false and inline is true, the name is not 
changed
+   * @return the name of the mutable state variable, which is either the 
original name if the
+   * variable is inlined to the outer class, or an array access if 
the variable is to be
+   * stored in an array of variables of the same type and 
initialization.
+   * There are two use cases. One is to use the original name for 
global variable instead
+   * of fresh name. Second is to use the original initialization 
statement since it is
+   * complex (e.g. allocate multi-dimensional array or object 
constructor has varibles).
+   * Primitive type variables will be inlined into outer class 
when the total number of
+   * mutable variables is less than 
`CodeGenerator.OUTER_CLASS_VARIABLES_THRESHOLD`
+   * the max size of an array for compaction is given by
+   * `CodeGenerator.MUTABLESTATEARRAY_SIZE_LIMIT`.
*/
-  def addMutableState(javaType: String, variableName: String, initCode: 
String = ""): Unit = {
-mutableStates += ((javaType, variableName, initCode))
+  def addMutableState(
+  javaType: String,
+  variableName: String,
+  initFunc: String => String = _ => "",
+  forceInline: Boolean = false,
+  useFreshName: Boolean = true): String = {
+val varName = if (useFreshName) freshName(variableName) else 
variableName
--- End diff --

Good catch


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156943716
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala 
---
@@ -68,30 +68,26 @@ private[sql] trait ColumnarBatchScan extends 
CodegenSupport {
*/
   // TODO: return ColumnarBatch.Rows instead
   override protected def doProduce(ctx: CodegenContext): String = {
-val input = ctx.freshName("input")
 // PhysicalRDD always just has one input
-ctx.addMutableState("scala.collection.Iterator", input, s"$input = 
inputs[0];")
+val input = ctx.addMutableState("scala.collection.Iterator", "input",
+  v => s"$v = inputs[0];", forceInline = true)
 
 // metrics
 val numOutputRows = metricTerm(ctx, "numOutputRows")
 val scanTimeMetric = metricTerm(ctx, "scanTime")
-val scanTimeTotalNs = ctx.freshName("scanTime")
-ctx.addMutableState(ctx.JAVA_LONG, scanTimeTotalNs, s"$scanTimeTotalNs 
= 0;")
+val scanTimeTotalNs = ctx.addMutableState(ctx.JAVA_LONG, "scanTime")
--- End diff --

I see. Let us leave default value as a comment for clarity.


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156941474
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -163,11 +192,49 @@ class CodegenContext {
* the list of default imports available.
* Also, generic type arguments are accepted but ignored.
* @param variableName Name of the field.
-   * @param initCode The statement(s) to put into the init() method to 
initialize this field.
+   * @param initFunc Function includes statement(s) to put into the init() 
method to initialize
+   * this field. The argument is the name of the mutable 
state variable.
* If left blank, the field will be default-initialized.
+   * @param forceInline whether the declaration and initialization code 
may be inlined rather than
+   *compacted. Please set `true` into forceInline, if 
you want to access the
+   *status fast (e.g. frequently accessed) or if you 
want to use the original
+   *variable name
+   * @param useFreshName If false and inline is true, the name is not 
changed
+   * @return the name of the mutable state variable, which is either the 
original name if the
+   * variable is inlined to the outer class, or an array access if 
the variable is to be
+   * stored in an array of variables of the same type and 
initialization.
+   * There are two use cases. One is to use the original name for 
global variable instead
+   * of fresh name. Second is to use the original initialization 
statement since it is
+   * complex (e.g. allocate multi-dimensional array or object 
constructor has varibles).
+   * Primitive type variables will be inlined into outer class 
when the total number of
+   * mutable variables is less than 
`CodeGenerator.OUTER_CLASS_VARIABLES_THRESHOLD`
+   * the max size of an array for compaction is given by
+   * `CodeGenerator.MUTABLESTATEARRAY_SIZE_LIMIT`.
*/
-  def addMutableState(javaType: String, variableName: String, initCode: 
String = ""): Unit = {
-mutableStates += ((javaType, variableName, initCode))
+  def addMutableState(
+  javaType: String,
+  variableName: String,
+  initFunc: String => String = _ => "",
+  forceInline: Boolean = false,
+  useFreshName: Boolean = true): String = {
+val varName = if (useFreshName) freshName(variableName) else 
variableName
--- End diff --

Since I noticed that most of caller sides executes `freshName`, I decided 
to use the new style that can simply caller code.  If a developer want to 
guarantee the given name is unique at caller site (currently, they are only 
several cases), it is OK by using `useFreshName = true`.  


Do we need redundant code at caller side? WDYT? @cloud-fan 


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156940683
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala
 ---
@@ -219,4 +219,31 @@ class GeneratedProjectionSuite extends SparkFunSuite {
 // - one is the mutableRow
 assert(globalVariables.length == 3)
   }
+
+  test("SPARK-18016: generated projections on wider table requiring state 
compaction") {
+val N = 6000
+val wideRow1 = new GenericInternalRow((0 until N).toArray[Any])
+val schema1 = StructType((1 to N).map(i => StructField("", 
IntegerType)))
+val wideRow2 = new GenericInternalRow(
+  (0 until N).map(i => UTF8String.fromString(i.toString)).toArray[Any])
+val schema2 = StructType((1 to N).map(i => StructField("", 
StringType)))
+val joined = new JoinedRow(wideRow1, wideRow2)
+val joinedSchema = StructType(schema1 ++ schema2)
+val nested = new JoinedRow(InternalRow(joined, joined), joined)
+val nestedSchema = StructType(
+  Seq(StructField("", joinedSchema), StructField("", joinedSchema)) ++ 
joinedSchema)
+
+val safeProj = FromUnsafeProjection(nestedSchema)
+val result = safeProj(nested)
+
+// test generated MutableProjection
+val exprs = nestedSchema.fields.zipWithIndex.map { case (f, i) =>
+  BoundReference(i, f.dataType, true)
+}
+val mutableProj = GenerateMutableProjection.generate(exprs)
--- End diff --

Current `GenerateUnsafeProjection` uses only the fixed number of mutable 
states. IIUC, we cannot do such a test.


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156935514
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -138,21 +138,50 @@ class CodegenContext {
 
   /**
* Holding expressions' mutable states like 
`MonotonicallyIncreasingID.count` as a
-   * 3-tuple: java type, variable name, code to init it.
-   * As an example, ("int", "count", "count = 0;") will produce code:
+   * 2-tuple: java type, variable name.
+   * As an example, ("int", "count") will produce code:
* {{{
*   private int count;
* }}}
-   * as a member variable, and add
-   * {{{
-   *   count = 0;
-   * }}}
-   * to the constructor.
+   * as a member variable
*
* They will be kept as member variables in generated classes like 
`SpecificProjection`.
*/
-  val mutableStates: mutable.ArrayBuffer[(String, String, String)] =
-mutable.ArrayBuffer.empty[(String, String, String)]
+  val mutableStates: mutable.ArrayBuffer[(String, String)] =
+mutable.ArrayBuffer.empty[(String, String)]
+
+  // An map keyed by mutable states' types holds the status of 
mutableStateArray
+  val mutableStateArrayMap: mutable.Map[String, MutableStateArrays] =
+mutable.Map.empty[String, MutableStateArrays]
+
+  // An array holds the code that will initialize each state
+  val mutableStateInitCodes: mutable.ArrayBuffer[String] =
--- End diff --

nit: since code is uncountable, maybe we can rename to 
`mutableStatesInitCode`


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156932945
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala ---
@@ -71,17 +71,15 @@ trait BaseLimitExec extends UnaryExecNode with 
CodegenSupport {
   }
 
   override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: 
ExprCode): String = {
-val stopEarly = ctx.freshName("stopEarly")
-ctx.addMutableState(ctx.JAVA_BOOLEAN, stopEarly, s"$stopEarly = 
false;")
+val stopEarly = ctx.addMutableState(ctx.JAVA_BOOLEAN, "stopEarly")
 
 ctx.addNewFunction("stopEarly", s"""
   @Override
   protected boolean stopEarly() {
 return $stopEarly;
   }
 """, inlineToOuterClass = true)
-val countTerm = ctx.freshName("count")
-ctx.addMutableState(ctx.JAVA_INT, countTerm, s"$countTerm = 0;")
+val countTerm = ctx.addMutableState(ctx.JAVA_INT, "count")
--- End diff --

ditto


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156932923
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala ---
@@ -71,17 +71,15 @@ trait BaseLimitExec extends UnaryExecNode with 
CodegenSupport {
   }
 
   override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: 
ExprCode): String = {
-val stopEarly = ctx.freshName("stopEarly")
-ctx.addMutableState(ctx.JAVA_BOOLEAN, stopEarly, s"$stopEarly = 
false;")
+val stopEarly = ctx.addMutableState(ctx.JAVA_BOOLEAN, "stopEarly")
--- End diff --

ditto


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156932789
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -163,11 +192,49 @@ class CodegenContext {
* the list of default imports available.
* Also, generic type arguments are accepted but ignored.
* @param variableName Name of the field.
-   * @param initCode The statement(s) to put into the init() method to 
initialize this field.
+   * @param initFunc Function includes statement(s) to put into the init() 
method to initialize
+   * this field. The argument is the name of the mutable 
state variable.
* If left blank, the field will be default-initialized.
+   * @param forceInline whether the declaration and initialization code 
may be inlined rather than
+   *compacted. Please set `true` into forceInline, if 
you want to access the
+   *status fast (e.g. frequently accessed) or if you 
want to use the original
+   *variable name
+   * @param useFreshName If false and inline is true, the name is not 
changed
+   * @return the name of the mutable state variable, which is either the 
original name if the
+   * variable is inlined to the outer class, or an array access if 
the variable is to be
+   * stored in an array of variables of the same type and 
initialization.
+   * There are two use cases. One is to use the original name for 
global variable instead
+   * of fresh name. Second is to use the original initialization 
statement since it is
+   * complex (e.g. allocate multi-dimensional array or object 
constructor has varibles).
+   * Primitive type variables will be inlined into outer class 
when the total number of
+   * mutable variables is less than 
`CodeGenerator.OUTER_CLASS_VARIABLES_THRESHOLD`
+   * the max size of an array for compaction is given by
+   * `CodeGenerator.MUTABLESTATEARRAY_SIZE_LIMIT`.
*/
-  def addMutableState(javaType: String, variableName: String, initCode: 
String = ""): Unit = {
-mutableStates += ((javaType, variableName, initCode))
+  def addMutableState(
+  javaType: String,
+  variableName: String,
+  initFunc: String => String = _ => "",
+  forceInline: Boolean = false,
+  useFreshName: Boolean = true): String = {
+val varName = if (useFreshName) freshName(variableName) else 
variableName
--- End diff --

I think this can be moved in the if for clarity


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156932214
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala 
---
@@ -68,30 +68,26 @@ private[sql] trait ColumnarBatchScan extends 
CodegenSupport {
*/
   // TODO: return ColumnarBatch.Rows instead
   override protected def doProduce(ctx: CodegenContext): String = {
-val input = ctx.freshName("input")
 // PhysicalRDD always just has one input
-ctx.addMutableState("scala.collection.Iterator", input, s"$input = 
inputs[0];")
+val input = ctx.addMutableState("scala.collection.Iterator", "input",
+  v => s"$v = inputs[0];", forceInline = true)
 
 // metrics
 val numOutputRows = metricTerm(ctx, "numOutputRows")
 val scanTimeMetric = metricTerm(ctx, "scanTime")
-val scanTimeTotalNs = ctx.freshName("scanTime")
-ctx.addMutableState(ctx.JAVA_LONG, scanTimeTotalNs, s"$scanTimeTotalNs 
= 0;")
+val scanTimeTotalNs = ctx.addMutableState(ctx.JAVA_LONG, "scanTime")
 
 val columnarBatchClz = classOf[ColumnarBatch].getName
-val batch = ctx.freshName("batch")
-ctx.addMutableState(columnarBatchClz, batch, s"$batch = null;")
+val batch = ctx.addMutableState(columnarBatchClz, "batch")
 
-val idx = ctx.freshName("batchIdx")
-ctx.addMutableState(ctx.JAVA_INT, idx, s"$idx = 0;")
-val colVars = output.indices.map(i => ctx.freshName("colInstance" + i))
+val idx = ctx.addMutableState(ctx.JAVA_INT, "batchIdx")
--- End diff --

ditto


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156932172
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala 
---
@@ -68,30 +68,26 @@ private[sql] trait ColumnarBatchScan extends 
CodegenSupport {
*/
   // TODO: return ColumnarBatch.Rows instead
   override protected def doProduce(ctx: CodegenContext): String = {
-val input = ctx.freshName("input")
 // PhysicalRDD always just has one input
-ctx.addMutableState("scala.collection.Iterator", input, s"$input = 
inputs[0];")
+val input = ctx.addMutableState("scala.collection.Iterator", "input",
+  v => s"$v = inputs[0];", forceInline = true)
 
 // metrics
 val numOutputRows = metricTerm(ctx, "numOutputRows")
 val scanTimeMetric = metricTerm(ctx, "scanTime")
-val scanTimeTotalNs = ctx.freshName("scanTime")
-ctx.addMutableState(ctx.JAVA_LONG, scanTimeTotalNs, s"$scanTimeTotalNs 
= 0;")
+val scanTimeTotalNs = ctx.addMutableState(ctx.JAVA_LONG, "scanTime")
--- End diff --

I see that the initialization is not needed since 0 is the default value, 
but maybe we can leave it for clarity?


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156930498
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -189,15 +255,37 @@ class CodegenContext {
   def declareMutableStates(): String = {
 // It's possible that we add same mutable state twice, e.g. the 
`mergeExpressions` in
 // `TypedAggregateExpression`, we should call `distinct` here to 
remove the duplicated ones.
-mutableStates.distinct.map { case (javaType, variableName, _) =>
+val inlinedStates = mutableStates.distinct.map { case (javaType, 
variableName) =>
   s"private $javaType $variableName;"
-}.mkString("\n")
+}
+
+val arrayStates = mutableStateArrayMap.flatMap { case (javaType, 
mutableStateArrays) =>
+  val numArrays = mutableStateArrays.arrayNames.size
+  mutableStateArrays.arrayNames.zipWithIndex.map { case (arrayName, 
index) =>
+val length = if (index + 1 == numArrays) {
+  mutableStateArrays.getCurrentIndex
+} else {
+  CodeGenerator.MUTABLESTATEARRAY_SIZE_LIMIT
--- End diff --

sorry, stupid question.


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156930403
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -163,11 +192,49 @@ class CodegenContext {
* the list of default imports available.
* Also, generic type arguments are accepted but ignored.
* @param variableName Name of the field.
-   * @param initCode The statement(s) to put into the init() method to 
initialize this field.
+   * @param initFunc Function includes statement(s) to put into the init() 
method to initialize
+   * this field. The argument is the name of the mutable 
state variable.
* If left blank, the field will be default-initialized.
+   * @param forceInline whether the declaration and initialization code 
may be inlined rather than
+   *compacted. Please set `true` into forceInline, if 
you want to access the
+   *status fast (e.g. frequently accessed) or if you 
want to use the original
+   *variable name
+   * @param useFreshName If false and inline is true, the name is not 
changed
+   * @return the name of the mutable state variable, which is either the 
original name if the
+   * variable is inlined to the outer class, or an array access if 
the variable is to be
+   * stored in an array of variables of the same type and 
initialization.
+   * There are two use cases. One is to use the original name for 
global variable instead
+   * of fresh name. Second is to use the original initialization 
statement since it is
+   * complex (e.g. allocate multi-dimensional array or object 
constructor has varibles).
+   * Primitive type variables will be inlined into outer class 
when the total number of
+   * mutable variables is less than 
`CodeGenerator.OUTER_CLASS_VARIABLES_THRESHOLD`
+   * the max size of an array for compaction is given by
+   * `CodeGenerator.MUTABLESTATEARRAY_SIZE_LIMIT`.
*/
-  def addMutableState(javaType: String, variableName: String, initCode: 
String = ""): Unit = {
-mutableStates += ((javaType, variableName, initCode))
+  def addMutableState(
+  javaType: String,
+  variableName: String,
+  initFunc: String => String = _ => "",
+  forceInline: Boolean = false,
+  useFreshName: Boolean = true): String = {
+val varName = if (useFreshName) freshName(variableName) else 
variableName
--- End diff --

instead of calling `freshName` here and adding a `useFreshName` parameter, 
can we follow the previous style and ask the caller side to guarantee the given 
name is unique? i.e. call `freshName` at caller side


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156928231
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala 
---
@@ -110,8 +110,8 @@ case class RowDataSourceScanExec(
   override protected def doProduce(ctx: CodegenContext): String = {
 val numOutputRows = metricTerm(ctx, "numOutputRows")
 // PhysicalRDD always just has one input
-val input = ctx.freshName("input")
-ctx.addMutableState("scala.collection.Iterator", input, s"$input = 
inputs[0];")
+val input = ctx.addMutableState("scala.collection.Iterator", "input", 
v => s"$v = inputs[0];",
+  forceInline = true)
--- End diff --

why inline


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156928270
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala 
---
@@ -353,8 +353,8 @@ case class FileSourceScanExec(
 }
 val numOutputRows = metricTerm(ctx, "numOutputRows")
 // PhysicalRDD always just has one input
-val input = ctx.freshName("input")
-ctx.addMutableState("scala.collection.Iterator", input, s"$input = 
inputs[0];")
+val input = ctx.addMutableState("scala.collection.Iterator", "input", 
v => s"$v = inputs[0];",
+  forceInline = true)
--- End diff --

ditto


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156928205
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala 
---
@@ -68,30 +68,26 @@ private[sql] trait ColumnarBatchScan extends 
CodegenSupport {
*/
   // TODO: return ColumnarBatch.Rows instead
   override protected def doProduce(ctx: CodegenContext): String = {
-val input = ctx.freshName("input")
 // PhysicalRDD always just has one input
-ctx.addMutableState("scala.collection.Iterator", input, s"$input = 
inputs[0];")
+val input = ctx.addMutableState("scala.collection.Iterator", "input",
+  v => s"$v = inputs[0];", forceInline = true)
--- End diff --

why inline?


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156926848
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala
 ---
@@ -219,4 +219,31 @@ class GeneratedProjectionSuite extends SparkFunSuite {
 // - one is the mutableRow
 assert(globalVariables.length == 3)
   }
+
+  test("SPARK-18016: generated projections on wider table requiring state 
compaction") {
+val N = 6000
+val wideRow1 = new GenericInternalRow((0 until N).toArray[Any])
+val schema1 = StructType((1 to N).map(i => StructField("", 
IntegerType)))
+val wideRow2 = new GenericInternalRow(
+  (0 until N).map(i => UTF8String.fromString(i.toString)).toArray[Any])
+val schema2 = StructType((1 to N).map(i => StructField("", 
StringType)))
+val joined = new JoinedRow(wideRow1, wideRow2)
+val joinedSchema = StructType(schema1 ++ schema2)
+val nested = new JoinedRow(InternalRow(joined, joined), joined)
+val nestedSchema = StructType(
+  Seq(StructField("", joinedSchema), StructField("", joinedSchema)) ++ 
joinedSchema)
+
+val safeProj = FromUnsafeProjection(nestedSchema)
+val result = safeProj(nested)
+
+// test generated MutableProjection
+val exprs = nestedSchema.fields.zipWithIndex.map { case (f, i) =>
+  BoundReference(i, f.dataType, true)
+}
+val mutableProj = GenerateMutableProjection.generate(exprs)
--- End diff --

shall we also test `GenerateUnsafeProjection`?


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156926555
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala
 ---
@@ -219,4 +219,31 @@ class GeneratedProjectionSuite extends SparkFunSuite {
 // - one is the mutableRow
 assert(globalVariables.length == 3)
   }
+
+  test("SPARK-18016: generated projections on wider table requiring state 
compaction") {
+val N = 6000
+val wideRow1 = new GenericInternalRow((0 until N).toArray[Any])
+val schema1 = StructType((1 to N).map(i => StructField("", 
IntegerType)))
+val wideRow2 = new GenericInternalRow(
+  (0 until N).map(i => UTF8String.fromString(i.toString)).toArray[Any])
--- End diff --

nit: `Array[Any].fill(N)(i => UTF8String.fromString(i.toString))`


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156926377
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala
 ---
@@ -219,4 +219,31 @@ class GeneratedProjectionSuite extends SparkFunSuite {
 // - one is the mutableRow
 assert(globalVariables.length == 3)
   }
+
+  test("SPARK-18016: generated projections on wider table requiring state 
compaction") {
+val N = 6000
+val wideRow1 = new GenericInternalRow((0 until N).toArray[Any])
--- End diff --

(0 until N).toArray[Any] -> new Array[Any](N)


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156926146
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala
 ---
@@ -401,4 +401,26 @@ class CodeGenerationSuite extends SparkFunSuite with 
ExpressionEvalHelper {
 ctx.addReferenceObj("foo", foo)
 assert(ctx.mutableStates.isEmpty)
   }
+
+  test("SPARK-18016: def  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;")
+}
+assert(ctx1.mutableStates.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.mutableStateArrayMap.get(ctx1.JAVA_INT).get.arrayNames.size == 1)
+assert(ctx1.mutableStateInitCodes.size == 
CodeGenerator.OUTER_CLASS_VARIABLES_THRESHOLD + 10)
+
+val ctx2 = new CodegenContext
+for (i <- 1 to CodeGenerator.MUTABLESTATEARRAY_SIZE_LIMIT + 10) {
+  ctx2.addMutableState("InternalRow[]", "r", v => s"$v = new 
InternalRow[$i];")
+}
+// When the number of non-primitive type mutable states is over the 
threshold, others are
+// allocated into a new array
+
assert(ctx2.mutableStateArrayMap.get("InternalRow[]").get.arrayNames.size == 2)
--- End diff --

and `assert(ctx2.mutableStateArrayMap("InternalRow[]").getCurrentIndex == 
10)`


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156925915
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala
 ---
@@ -401,4 +401,26 @@ class CodeGenerationSuite extends SparkFunSuite with 
ExpressionEvalHelper {
 ctx.addReferenceObj("foo", foo)
 assert(ctx.mutableStates.isEmpty)
   }
+
+  test("SPARK-18016: def  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;")
+}
+assert(ctx1.mutableStates.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.mutableStateArrayMap.get(ctx1.JAVA_INT).get.arrayNames.size == 1)
+assert(ctx1.mutableStateInitCodes.size == 
CodeGenerator.OUTER_CLASS_VARIABLES_THRESHOLD + 10)
+
+val ctx2 = new CodegenContext
+for (i <- 1 to CodeGenerator.MUTABLESTATEARRAY_SIZE_LIMIT + 10) {
+  ctx2.addMutableState("InternalRow[]", "r", v => s"$v = new 
InternalRow[$i];")
+}
+// When the number of non-primitive type mutable states is over the 
threshold, others are
+// allocated into a new array
+
assert(ctx2.mutableStateArrayMap.get("InternalRow[]").get.arrayNames.size == 2)
--- End diff --

`assert(ctx2.mutableStates.isEmpty)`?


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156925749
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala
 ---
@@ -401,4 +401,26 @@ class CodeGenerationSuite extends SparkFunSuite with 
ExpressionEvalHelper {
 ctx.addReferenceObj("foo", foo)
 assert(ctx.mutableStates.isEmpty)
   }
+
+  test("SPARK-18016: def  mutable states by using an array") {
--- End diff --

def -> `define`


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156925450
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
 ---
@@ -1205,14 +1203,14 @@ case class DecodeUsingSerializer[T](child: 
Expression, tag: ClassTag[T], kryo: B
 // try conf from env, otherwise create a new one
 val env = s"${classOf[SparkEnv].getName}.get()"
 val sparkConf = s"new ${classOf[SparkConf].getName}()"
-val serializerInit = s"""
-  if ($env == null) {
-$serializer = ($serializerInstanceClass) new 
$serializerClass($sparkConf).newInstance();
+val serializer = ctx.addMutableState(serializerInstanceClass, 
"serializerForDecode",
+  v => s"""
+   if ($env == null) {
--- End diff --

nit: use the new multiline string style


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156925357
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
 ---
@@ -1159,14 +1158,14 @@ case class EncodeUsingSerializer(child: Expression, 
kryo: Boolean)
 // try conf from env, otherwise create a new one
 val env = s"${classOf[SparkEnv].getName}.get()"
 val sparkConf = s"new ${classOf[SparkConf].getName}()"
-val serializerInit = s"""
-  if ($env == null) {
-$serializer = ($serializerInstanceClass) new 
$serializerClass($sparkConf).newInstance();
-   } else {
- $serializer = ($serializerInstanceClass) new 
$serializerClass($env.conf()).newInstance();
-   }
- """
-ctx.addMutableState(serializerInstanceClass, serializer, 
serializerInit)
+val serializer = ctx.addMutableState(serializerInstanceClass, 
"serializerForEncode", v =>
+  s"""
+ |if ($env == null) {
+ |  $v = ($serializerInstanceClass) new 
$serializerClass($sparkConf).newInstance();
+ |} else {
+ |  $v = ($serializerInstanceClass) new 
$serializerClass($env.conf()).newInstance();
+ |}
+   """, forceInline = true)
--- End diff --

why inline?


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156925475
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
 ---
@@ -1205,14 +1203,14 @@ case class DecodeUsingSerializer[T](child: 
Expression, tag: ClassTag[T], kryo: B
 // try conf from env, otherwise create a new one
 val env = s"${classOf[SparkEnv].getName}.get()"
 val sparkConf = s"new ${classOf[SparkConf].getName}()"
-val serializerInit = s"""
-  if ($env == null) {
-$serializer = ($serializerInstanceClass) new 
$serializerClass($sparkConf).newInstance();
+val serializer = ctx.addMutableState(serializerInstanceClass, 
"serializerForDecode",
+  v => s"""
+   if ($env == null) {
+ $v = ($serializerInstanceClass) new 
$serializerClass($sparkConf).newInstance();
} else {
- $serializer = ($serializerInstanceClass) new 
$serializerClass($env.conf()).newInstance();
+ $v = ($serializerInstanceClass) new 
$serializerClass($env.conf()).newInstance();
}
- """
-ctx.addMutableState(serializerInstanceClass, serializer, 
serializerInit)
+ """, forceInline = true)
--- End diff --

why inline?


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156924217
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
 ---
@@ -484,18 +484,17 @@ case class WeekOfYear(child: Expression) extends 
UnaryExpression with ImplicitCa
   override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
 nullSafeCodeGen(ctx, ev, time => {
   val cal = classOf[Calendar].getName
-  val c = ctx.freshName("cal")
   val dtu = DateTimeUtils.getClass.getName.stripSuffix("$")
-  ctx.addMutableState(cal, c,
+  val c = ctx.addMutableState(cal, "cal", v =>
 s"""
-  $c = $cal.getInstance($dtu.getTimeZone("UTC"));
-  $c.setFirstDayOfWeek($cal.MONDAY);
-  $c.setMinimalDaysInFirstWeek(4);
+   |$v = $cal.getInstance($dtu.getTimeZone("UTC"));
+   |$v.setFirstDayOfWeek($cal.MONDAY);
+   |$v.setMinimalDaysInFirstWeek(4);
  """)
--- End diff --

you forget the `stripMargin`, now the `|` will be in the code and fail to 
compile...


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156923234
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala
 ---
@@ -57,41 +57,35 @@ object GenerateMutableProjection extends 
CodeGenerator[Seq[Expression], MutableP
   case _ => true
 }.unzip
 val exprVals = ctx.generateExpressions(validExpr, 
useSubexprElimination)
-val projectionCodes = exprVals.zip(index).map {
+val projectionCodes: Seq[(String, String, String, Int)] = 
exprVals.zip(index).map {
--- End diff --

add a comment to explain this tuple-4


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156922972
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -189,15 +255,37 @@ class CodegenContext {
   def declareMutableStates(): String = {
 // It's possible that we add same mutable state twice, e.g. the 
`mergeExpressions` in
 // `TypedAggregateExpression`, we should call `distinct` here to 
remove the duplicated ones.
-mutableStates.distinct.map { case (javaType, variableName, _) =>
+val inlinedStates = mutableStates.distinct.map { case (javaType, 
variableName) =>
   s"private $javaType $variableName;"
-}.mkString("\n")
+}
+
+val arrayStates = mutableStateArrayMap.flatMap { case (javaType, 
mutableStateArrays) =>
+  val numArrays = mutableStateArrays.arrayNames.size
+  mutableStateArrays.arrayNames.zipWithIndex.map { case (arrayName, 
index) =>
+val length = if (index + 1 == numArrays) {
+  mutableStateArrays.getCurrentIndex
+} else {
+  CodeGenerator.MUTABLESTATEARRAY_SIZE_LIMIT
--- End diff --

one type can have multiple arrays if there are many global variables of 
this type. The `getCurrentIndex` is the index of the array to be filled, and 
other arrays are already full and their size is `MUTABLESTATEARRAY_SIZE_LIMIT`.


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156922110
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -163,11 +192,49 @@ class CodegenContext {
* the list of default imports available.
* Also, generic type arguments are accepted but ignored.
* @param variableName Name of the field.
-   * @param initCode The statement(s) to put into the init() method to 
initialize this field.
+   * @param initFunc Function includes statement(s) to put into the init() 
method to initialize
+   * this field. The argument is the name of the mutable 
state variable.
* If left blank, the field will be default-initialized.
+   * @param forceInline whether the declaration and initialization code 
may be inlined rather than
+   *compacted. Please set `true` into forceInline, if 
you want to access the
+   *status fast (e.g. frequently accessed) or if you 
want to use the original
+   *variable name
+   * @param useFreshName If false and inline is true, the name is not 
changed
+   * @return the name of the mutable state variable, which is either the 
original name if the
+   * variable is inlined to the outer class, or an array access if 
the variable is to be
+   * stored in an array of variables of the same type and 
initialization.
--- End diff --

`of same type` ~and initialization~


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156921749
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -138,21 +138,50 @@ class CodegenContext {
 
   /**
* Holding expressions' mutable states like 
`MonotonicallyIncreasingID.count` as a
-   * 3-tuple: java type, variable name, code to init it.
-   * As an example, ("int", "count", "count = 0;") will produce code:
+   * 2-tuple: java type, variable name.
+   * As an example, ("int", "count") will produce code:
* {{{
*   private int count;
* }}}
-   * as a member variable, and add
-   * {{{
-   *   count = 0;
-   * }}}
-   * to the constructor.
+   * as a member variable
*
* They will be kept as member variables in generated classes like 
`SpecificProjection`.
*/
-  val mutableStates: mutable.ArrayBuffer[(String, String, String)] =
-mutable.ArrayBuffer.empty[(String, String, String)]
+  val mutableStates: mutable.ArrayBuffer[(String, String)] =
+mutable.ArrayBuffer.empty[(String, String)]
+
+  // An map keyed by mutable states' types holds the status of 
mutableStateArray
+  val mutableStateArrayMap: mutable.Map[String, MutableStateArrays] =
--- End diff --

`arrayCompactedMutaleStates`?


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156921623
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -138,21 +138,50 @@ class CodegenContext {
 
   /**
* Holding expressions' mutable states like 
`MonotonicallyIncreasingID.count` as a
-   * 3-tuple: java type, variable name, code to init it.
-   * As an example, ("int", "count", "count = 0;") will produce code:
+   * 2-tuple: java type, variable name.
+   * As an example, ("int", "count") will produce code:
* {{{
*   private int count;
* }}}
-   * as a member variable, and add
-   * {{{
-   *   count = 0;
-   * }}}
-   * to the constructor.
+   * as a member variable
*
* They will be kept as member variables in generated classes like 
`SpecificProjection`.
*/
-  val mutableStates: mutable.ArrayBuffer[(String, String, String)] =
-mutable.ArrayBuffer.empty[(String, String, String)]
+  val mutableStates: mutable.ArrayBuffer[(String, String)] =
--- End diff --

rename it to `inlinedMutableStates`?


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156919894
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -189,15 +255,37 @@ class CodegenContext {
   def declareMutableStates(): String = {
 // It's possible that we add same mutable state twice, e.g. the 
`mergeExpressions` in
 // `TypedAggregateExpression`, we should call `distinct` here to 
remove the duplicated ones.
-mutableStates.distinct.map { case (javaType, variableName, _) =>
+val inlinedStates = mutableStates.distinct.map { case (javaType, 
variableName) =>
   s"private $javaType $variableName;"
-}.mkString("\n")
+}
+
+val arrayStates = mutableStateArrayMap.flatMap { case (javaType, 
mutableStateArrays) =>
+  val numArrays = mutableStateArrays.arrayNames.size
+  mutableStateArrays.arrayNames.zipWithIndex.map { case (arrayName, 
index) =>
+val length = if (index + 1 == numArrays) {
+  mutableStateArrays.getCurrentIndex
+} else {
+  CodeGenerator.MUTABLESTATEARRAY_SIZE_LIMIT
--- End diff --

why can't we always use `mutableStateArrays.getCurrentIndex`?


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156892758
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala
 ---
@@ -318,13 +316,12 @@ object GenerateUnsafeProjection extends 
CodeGenerator[Seq[Expression], UnsafePro
   case _ => true
 }
 
-val result = ctx.freshName("result")
-ctx.addMutableState("UnsafeRow", result, s"$result = new 
UnsafeRow(${expressions.length});")
+val result = ctx.addMutableState("UnsafeRow", "result",
+  v => s"$v = new UnsafeRow(${expressions.length});", inline = true)
--- End diff --

sure


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156885639
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala
 ---
@@ -318,13 +316,12 @@ object GenerateUnsafeProjection extends 
CodeGenerator[Seq[Expression], UnsafePro
   case _ => true
 }
 
-val result = ctx.freshName("result")
-ctx.addMutableState("UnsafeRow", result, s"$result = new 
UnsafeRow(${expressions.length});")
+val result = ctx.addMutableState("UnsafeRow", "result",
+  v => s"$v = new UnsafeRow(${expressions.length});", inline = true)
--- End diff --

what if we have a single buffer for both inline and array compacted global 
variable init code?


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156880726
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala
 ---
@@ -318,13 +316,12 @@ object GenerateUnsafeProjection extends 
CodeGenerator[Seq[Expression], UnsafePro
   case _ => true
 }
 
-val result = ctx.freshName("result")
-ctx.addMutableState("UnsafeRow", result, s"$result = new 
UnsafeRow(${expressions.length});")
+val result = ctx.addMutableState("UnsafeRow", "result",
+  v => s"$v = new UnsafeRow(${expressions.length});", inline = true)
--- End diff --

Or, is it better to introduce another flag to keep order of initialization 
even when compacted?  e.g. `keepInitializationOrder`


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156880398
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
 ---
@@ -212,12 +212,13 @@ case class Stack(children: Seq[Expression]) extends 
Generator {
   s"${eval.code}\n$rowData[$row] = ${eval.value};"
 })
 
-// Create the collection.
+// Create the collection. Inline to outer class.
--- End diff --

Good catch, since we have dependency with `new InternalRow[$numRows]`. We 
have to inline `new InternalRow[$numRows]`, too.


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156879619
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
 ---
@@ -484,14 +484,13 @@ case class WeekOfYear(child: Expression) extends 
UnaryExpression with ImplicitCa
   override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
 nullSafeCodeGen(ctx, ev, time => {
   val cal = classOf[Calendar].getName
-  val c = ctx.freshName("cal")
   val dtu = DateTimeUtils.getClass.getName.stripSuffix("$")
-  ctx.addMutableState(cal, c,
-s"""
-  $c = $cal.getInstance($dtu.getTimeZone("UTC"));
-  $c.setFirstDayOfWeek($cal.MONDAY);
-  $c.setMinimalDaysInFirstWeek(4);
- """)
+  val c = ctx.addMutableState(cal, "cal",
+v => s"""
+  $v = $cal.getInstance($dtu.getTimeZone("UTC"));
+  $v.setFirstDayOfWeek($cal.MONDAY);
+  $v.setMinimalDaysInFirstWeek(4);
+ """, inline = true)
--- End diff --

Good catch, could not inline


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156879324
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala
 ---
@@ -73,9 +73,8 @@ object GenerateUnsafeProjection extends 
CodeGenerator[Seq[Expression], UnsafePro
   bufferHolder: String,
   isTopLevel: Boolean = false): String = {
 val rowWriterClass = classOf[UnsafeRowWriter].getName
-val rowWriter = ctx.freshName("rowWriter")
-ctx.addMutableState(rowWriterClass, rowWriter,
-  s"$rowWriter = new $rowWriterClass($bufferHolder, 
${inputs.length});")
+val rowWriter = ctx.addMutableState(rowWriterClass, "rowWriter",
+  v => s"$v = new $rowWriterClass($bufferHolder, ${inputs.length});", 
inline = true)
--- End diff --

We want to keep order of initialization since they have dependency (`b = 
new BufferHolder(a, ...);` and `c = new RowWriterClass(b, ...);`).


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156878854
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -163,11 +201,52 @@ class CodegenContext {
* the list of default imports available.
* Also, generic type arguments are accepted but ignored.
* @param variableName Name of the field.
-   * @param initCode The statement(s) to put into the init() method to 
initialize this field.
+   * @param initFunc Function includes statement(s) to put into the init() 
method to initialize
+   * this field. The argument is the name of the mutable 
state variable.
* If left blank, the field will be default-initialized.
+   * @param inline whether the declaration and initialization code may be 
inlined rather than
--- End diff --

do we have a clear rule when to force inline it?


---

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



[GitHub] spark pull request #19811: [SPARK-18016][SQL] Code Generation: Constant Pool...

2017-12-14 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19811#discussion_r156878552
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
 ---
@@ -1159,14 +1156,14 @@ case class EncodeUsingSerializer(child: Expression, 
kryo: Boolean)
 // try conf from env, otherwise create a new one
 val env = s"${classOf[SparkEnv].getName}.get()"
 val sparkConf = s"new ${classOf[SparkConf].getName}()"
-val serializerInit = s"""
-  if ($env == null) {
-$serializer = ($serializerInstanceClass) new 
$serializerClass($sparkConf).newInstance();
+val serializer = ctx.addMutableState(serializerInstanceClass, 
"serializerForEncode",
+  v => s"""
+   if ($env == null) {
--- End diff --

nit: use the new multiline string style


---

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



  1   2   >