Skip to content

Commit 53661eb

Browse files
committed
initial commit
1 parent ef10f45 commit 53661eb

File tree

5 files changed

+16
-13
lines changed

5 files changed

+16
-13
lines changed

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -178,7 +178,7 @@ class CodegenContext {
178178

179179
/**
180180
* Returns the reference of next available slot in current compacted array. The size of each
181-
* compacted array is controlled by the config `CodeGenerator.MUTABLESTATEARRAY_SIZE_LIMIT`.
181+
* compacted array is controlled by the constant `CodeGenerator.MUTABLESTATEARRAY_SIZE_LIMIT`.
182182
* Once reaching the threshold, new compacted array is created.
183183
*/
184184
def getNextSlot(): String = {
@@ -299,7 +299,7 @@ class CodegenContext {
299299
def initMutableStates(): String = {
300300
// It's possible that we add same mutable state twice, e.g. the `mergeExpressions` in
301301
// `TypedAggregateExpression`, we should call `distinct` here to remove the duplicated ones.
302-
val initCodes = mutableStateInitCode.distinct
302+
val initCodes = mutableStateInitCode.distinct.map(_ + "\n")
303303

304304
// The generated initialization code may exceed 64kb function size limit in JVM if there are too
305305
// many mutable states, so split it into multiple functions.

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

Lines changed: 2 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -118,9 +118,8 @@ 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
122121
val pattern = ctx.addMutableState(patternClass, "patternLike",
123-
v => s"""$v = ${patternClass}.compile("$regexStr");""", forceInline = true)
122+
v => s"""$v = ${patternClass}.compile("$regexStr");""")
124123

125124
// We don't use nullSafeCodeGen here because we don't want to re-evaluate right again.
126125
val eval = left.genCode(ctx)
@@ -194,9 +193,8 @@ case class RLike(left: Expression, right: Expression) extends StringRegexExpress
194193
if (rVal != null) {
195194
val regexStr =
196195
StringEscapeUtils.escapeJava(rVal.asInstanceOf[UTF8String].toString())
197-
// inline mutable state since not many RLike operations in a task
198196
val pattern = ctx.addMutableState(patternClass, "patternRLike",
199-
v => s"""$v = ${patternClass}.compile("$regexStr");""", forceInline = true)
197+
v => s"""$v = ${patternClass}.compile("$regexStr");""")
200198

201199
// We don't use nullSafeCodeGen here because we don't want to re-evaluate right again.
202200
val eval = left.genCode(ctx)

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -283,7 +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
286+
// inline mutable state since an InputAdapter in a task
287287
val input = ctx.addMutableState("scala.collection.Iterator", "input", v => s"$v = inputs[0];",
288288
forceInline = true)
289289
val row = ctx.freshName("row")

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

Lines changed: 9 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -587,20 +587,24 @@ case class HashAggregateExec(
587587
fastHashMapClassName, groupingKeySchema, bufferSchema).generate()
588588
ctx.addInnerClass(generatedMap)
589589

590+
// inline mutable state since not many aggregation operations in a task
590591
fastHashMapTerm = ctx.addMutableState(fastHashMapClassName, "vectorizedHastHashMap",
591-
v => s"$v = new $fastHashMapClassName();")
592-
ctx.addMutableState(s"java.util.Iterator<InternalRow>", "vectorizedFastHashMapIter")
592+
v => s"$v = new $fastHashMapClassName();", forceInline = true)
593+
ctx.addMutableState(s"java.util.Iterator<InternalRow>", "vectorizedFastHashMapIter",
594+
forceInline = true)
593595
} else {
594596
val generatedMap = new RowBasedHashMapGenerator(ctx, aggregateExpressions,
595597
fastHashMapClassName, groupingKeySchema, bufferSchema).generate()
596598
ctx.addInnerClass(generatedMap)
597599

600+
// inline mutable state since not many aggregation operations in a task
598601
fastHashMapTerm = ctx.addMutableState(fastHashMapClassName, "fastHashMap",
599602
v => s"$v = new $fastHashMapClassName(" +
600-
s"$thisPlan.getTaskMemoryManager(), $thisPlan.getEmptyAggregationBuffer());")
603+
s"$thisPlan.getTaskMemoryManager(), $thisPlan.getEmptyAggregationBuffer());",
604+
forceInline = true)
601605
ctx.addMutableState(
602606
"org.apache.spark.unsafe.KVIterator<UnsafeRow, UnsafeRow>",
603-
"fastHashMapIter")
607+
"fastHashMapIter", forceInline = true)
604608
}
605609
}
606610

@@ -611,7 +615,7 @@ case class HashAggregateExec(
611615
// create hashMap
612616
val hashMapClassName = classOf[UnsafeFixedWidthAggregationMap].getName
613617
hashMapTerm = ctx.addMutableState(hashMapClassName, "hashMap",
614-
v => s"$v = $thisPlan.createHashMap();")
618+
v => s"$v = $thisPlan.createHashMap();", forceInline = true)
615619
sorterTerm = ctx.addMutableState(classOf[UnsafeKVExternalSorter].getName, "sorter",
616620
forceInline = true)
617621

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

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -440,8 +440,9 @@ case class SortMergeJoinExec(
440440
val spillThreshold = getSpillThreshold
441441
val inMemoryThreshold = getInMemoryThreshold
442442

443+
// inline mutable state since not many join operations in a task
443444
val matches = ctx.addMutableState(clsName, "matches",
444-
v => s"$v = new $clsName($inMemoryThreshold, $spillThreshold);")
445+
v => s"$v = new $clsName($inMemoryThreshold, $spillThreshold);", forceInline = true)
445446
// Copy the left keys as class members so they could be used in next function call.
446447
val matchedKeyVars = copyKeys(ctx, leftKeyVars)
447448

0 commit comments

Comments
 (0)