Skip to content

Commit 0e45c19

Browse files
committed
address review comments
1 parent 31914c0 commit 0e45c19

File tree

8 files changed

+21
-12
lines changed

8 files changed

+21
-12
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala

Lines changed: 6 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -208,10 +208,12 @@ class CodegenContext {
208208
* this field. The argument is the name of the mutable state variable.
209209
* If left blank, the field will be default-initialized.
210210
* @param forceInline whether the declaration and initialization code may be inlined rather than
211-
* compacted. Please set `true` into forceInline, if you want to access the
212-
* status fast (e.g. frequently accessed) or if you want to use the original
213-
* variable name
214-
* @param useFreshName If this is false and forceInline is true, the name is not changed
211+
* compacted. Please set `true` into forceInline for one of the followings:
212+
* 1. use the original name of the status
213+
* 2. expect to non-frequently generate the status
214+
* (e.g. not much sort operators in one stage)
215+
* @param useFreshName If this is false and the mutable state ends up inlining in the outer
216+
* class, the name is not changed
215217
* @return the name of the mutable state variable, which is the original name or fresh name if
216218
* the variable is inlined to the outer class, or an array access if the variable is to
217219
* be stored in an array of variables of the same type.
@@ -221,7 +223,6 @@ class CodegenContext {
221223
* 2. its type is primitive type and the total number of the inlined mutable variables
222224
* is less than `CodeGenerator.OUTER_CLASS_VARIABLES_THRESHOLD`
223225
* 3. its type is multi-dimensional array
224-
* A primitive type variable will be inlined into outer class when the total number of
225226
* When a variable is compacted into an array, the max size of the array for compaction
226227
* is given by `CodeGenerator.MUTABLESTATEARRAY_SIZE_LIMIT`.
227228
*/

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

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -118,6 +118,7 @@ case class Like(left: Expression, right: Expression) extends StringRegexExpressi
118118
if (rVal != null) {
119119
val regexStr =
120120
StringEscapeUtils.escapeJava(escape(rVal.asInstanceOf[UTF8String].toString()))
121+
// inline mutable state since not many Like operations in a task
121122
val pattern = ctx.addMutableState(patternClass, "patternLike",
122123
v => s"""$v = ${patternClass}.compile("$regexStr");""", forceInline = true)
123124

@@ -193,6 +194,7 @@ case class RLike(left: Expression, right: Expression) extends StringRegexExpress
193194
if (rVal != null) {
194195
val regexStr =
195196
StringEscapeUtils.escapeJava(rVal.asInstanceOf[UTF8String].toString())
197+
// inline mutable state since not many RLike operations in a task
196198
val pattern = ctx.addMutableState(patternClass, "patternRLike",
197199
v => s"""$v = ${patternClass}.compile("$regexStr");""", forceInline = true)
198200

sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -138,6 +138,7 @@ case class SortExec(
138138
// Initialize the class member variables. This includes the instance of the Sorter and
139139
// the iterator to return sorted rows.
140140
val thisPlan = ctx.addReferenceObj("plan", this)
141+
// inline mutable state since not many Sort operations in a task
141142
sorterVariable = ctx.addMutableState(classOf[UnsafeExternalRowSorter].getName, "sorter",
142143
v => s"$v = $thisPlan.createSorter();", forceInline = true)
143144
val metrics = ctx.addMutableState(classOf[TaskMetrics].getName, "metrics",

sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -283,6 +283,7 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp
283283

284284
override def doProduce(ctx: CodegenContext): String = {
285285
// Right now, InputAdapter is only used when there is one input RDD.
286+
// inline mutable state since an inputAdaptor in a task
286287
val input = ctx.addMutableState("scala.collection.Iterator", "input", v => s"$v = inputs[0];",
287288
forceInline = true)
288289
val row = ctx.freshName("row")

sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala

Lines changed: 5 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -605,12 +605,15 @@ case class HashAggregateExec(
605605
}
606606

607607
// Create a name for the iterator from the regular hash map.
608-
val iterTerm = ctx.addMutableState(classOf[KVIterator[UnsafeRow, UnsafeRow]].getName, "mapIter")
608+
// inline mutable state since not many aggregation operations in a task
609+
val iterTerm = ctx.addMutableState(classOf[KVIterator[UnsafeRow, UnsafeRow]].getName,
610+
"mapIter", forceInline = true)
609611
// create hashMap
610612
val hashMapClassName = classOf[UnsafeFixedWidthAggregationMap].getName
611613
hashMapTerm = ctx.addMutableState(hashMapClassName, "hashMap",
612614
v => s"$v = $thisPlan.createHashMap();")
613-
sorterTerm = ctx.addMutableState(classOf[UnsafeKVExternalSorter].getName, "sorter")
615+
sorterTerm = ctx.addMutableState(classOf[UnsafeKVExternalSorter].getName, "sorter",
616+
forceInline = true)
614617

615618
val doAgg = ctx.freshName("doAggregateWithKeys")
616619
val peakMemory = metricTerm(ctx, "peakMemory")

sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala

Lines changed: 3 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -284,6 +284,7 @@ case class SampleExec(
284284
val samplerClass = classOf[PoissonSampler[UnsafeRow]].getName
285285
val initSampler = ctx.freshName("initSampler")
286286

287+
// inline mutable state since not many Sample operations in a task
287288
val sampler = ctx.addMutableState(s"$samplerClass<UnsafeRow>", "sampleReplace",
288289
v => {
289290
val initSamplerFuncName = ctx.addNewFunction(initSampler,
@@ -317,7 +318,7 @@ case class SampleExec(
317318
v => s"""
318319
| $v = new $samplerClass<UnsafeRow>($lowerBound, $upperBound, false);
319320
| $v.setSeed(${seed}L + partitionIndex);
320-
""".stripMargin.trim, forceInline = true)
321+
""".stripMargin.trim)
321322

322323
s"""
323324
| if ($sampler.sample() != 0) {
@@ -370,6 +371,7 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range)
370371
val ev = ExprCode("", "false", value)
371372
val BigInt = classOf[java.math.BigInteger].getName
372373

374+
// inline mutable state since not many Range operations in a task
373375
val taskContext = ctx.addMutableState("TaskContext", "taskContext",
374376
v => s"$v = TaskContext.get();", forceInline = true)
375377
val inputMetrics = ctx.addMutableState("InputMetrics", "inputMetrics",
@@ -435,10 +437,6 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range)
435437
| }
436438
""".stripMargin)
437439

438-
// Right now, Range is only used when there is one upstream.
439-
val input = ctx.addMutableState("scala.collection.Iterator", "input",
440-
v => s"$v = inputs[0];", forceInline = true)
441-
442440
val localIdx = ctx.freshName("localIdx")
443441
val localEnd = ctx.freshName("localEnd")
444442
val range = ctx.freshName("range")

sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -139,6 +139,7 @@ case class BroadcastHashJoinExec(
139139
// At the end of the task, we update the avg hash probe.
140140
val avgHashProbe = metricTerm(ctx, "avgHashProbe")
141141

142+
// inline mutable state since not many join operations in a task
142143
val relationTerm = ctx.addMutableState(clsName, "relation",
143144
v => s"""
144145
| $v = (($clsName) $broadcast.value()).asReadOnlyCopy();

sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -422,6 +422,7 @@ case class SortMergeJoinExec(
422422
*/
423423
private def genScanner(ctx: CodegenContext): (String, String) = {
424424
// Create class member for next row from both sides.
425+
// inline mutable state since not many join operations in a task
425426
val leftRow = ctx.addMutableState("InternalRow", "leftRow", forceInline = true)
426427
val rightRow = ctx.addMutableState("InternalRow", "rightRow", forceInline = true)
427428

@@ -575,6 +576,7 @@ case class SortMergeJoinExec(
575576
override def needCopyResult: Boolean = true
576577

577578
override def doProduce(ctx: CodegenContext): String = {
579+
// inline mutable state since not many join operations in a task
578580
val leftInput = ctx.addMutableState("scala.collection.Iterator", "leftInput",
579581
v => s"$v = inputs[0];", forceInline = true)
580582
val rightInput = ctx.addMutableState("scala.collection.Iterator", "rightInput",

0 commit comments

Comments
 (0)