Skip to content

Commit 6972389

Browse files
maropurdblue
authored andcommitted
[SPARK-21871][SQL] Check actual bytecode size when compiling generated code
This pr added code to check actual bytecode size when compiling generated code. In apache#18810, we added code to give up code compilation and use interpreter execution in `SparkPlan` if the line number of generated functions goes over `maxLinesPerFunction`. But, we already have code to collect metrics for compiled bytecode size in `CodeGenerator` object. So,we could easily reuse the code for this purpose. Added tests in `WholeStageCodegenSuite`. Author: Takeshi Yamamuro <[email protected]> Closes apache#19083 from maropu/SPARK-21871.
1 parent 430e0b1 commit 6972389

File tree

12 files changed

+108
-31
lines changed

12 files changed

+108
-31
lines changed

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

Lines changed: 28 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -898,17 +898,23 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin
898898
}
899899

900900
object CodeGenerator extends Logging {
901+
902+
// This is the value of HugeMethodLimit in the OpenJDK JVM settings
903+
val DEFAULT_JVM_HUGE_METHOD_LIMIT = 8000
904+
901905
/**
902906
* Compile the Java source code into a Java class, using Janino.
907+
*
908+
* @return a pair of a generated class and the max bytecode size of generated functions.
903909
*/
904-
def compile(code: CodeAndComment): GeneratedClass = {
910+
def compile(code: CodeAndComment): (GeneratedClass, Int) = {
905911
cache.get(code)
906912
}
907913

908914
/**
909915
* Compile the Java source code into a Java class, using Janino.
910916
*/
911-
private[this] def doCompile(code: CodeAndComment): GeneratedClass = {
917+
private[this] def doCompile(code: CodeAndComment): (GeneratedClass, Int) = {
912918
val evaluator = new ClassBodyEvaluator()
913919

914920
// A special classloader used to wrap the actual parent classloader of
@@ -946,22 +952,24 @@ object CodeGenerator extends Logging {
946952
s"\n$formatted"
947953
})
948954

949-
try {
955+
val maxCodeSize = try {
950956
evaluator.cook("generated.java", code.body)
951-
recordCompilationStats(evaluator)
957+
updateAndGetCompilationStats(evaluator)
952958
} catch {
953959
case e: Exception =>
954960
val msg = s"failed to compile: $e\n$formatted"
955961
logError(msg, e)
956962
throw new Exception(msg, e)
957963
}
958-
evaluator.getClazz().newInstance().asInstanceOf[GeneratedClass]
964+
965+
(evaluator.getClazz().newInstance().asInstanceOf[GeneratedClass], maxCodeSize)
959966
}
960967

961968
/**
962-
* Records the generated class and method bytecode sizes by inspecting janino private fields.
969+
* Returns the max bytecode size of the generated functions by inspecting janino private fields.
970+
* Also, this method updates the metrics information.
963971
*/
964-
private def recordCompilationStats(evaluator: ClassBodyEvaluator): Unit = {
972+
private def updateAndGetCompilationStats(evaluator: ClassBodyEvaluator): Int = {
965973
// First retrieve the generated classes.
966974
val classes = {
967975
val resultField = classOf[SimpleCompiler].getDeclaredField("result")
@@ -976,23 +984,26 @@ object CodeGenerator extends Logging {
976984
val codeAttr = Utils.classForName("org.codehaus.janino.util.ClassFile$CodeAttribute")
977985
val codeAttrField = codeAttr.getDeclaredField("code")
978986
codeAttrField.setAccessible(true)
979-
classes.foreach { case (_, classBytes) =>
987+
val codeSizes = classes.flatMap { case (_, classBytes) =>
980988
CodegenMetrics.METRIC_GENERATED_CLASS_BYTECODE_SIZE.update(classBytes.length)
981989
try {
982990
val cf = new ClassFile(new ByteArrayInputStream(classBytes))
983-
cf.methodInfos.asScala.foreach { method =>
984-
method.getAttributes().foreach { a =>
985-
if (a.getClass.getName == codeAttr.getName) {
986-
CodegenMetrics.METRIC_GENERATED_METHOD_BYTECODE_SIZE.update(
987-
codeAttrField.get(a).asInstanceOf[Array[Byte]].length)
988-
}
991+
val stats = cf.methodInfos.asScala.flatMap { method =>
992+
method.getAttributes().filter(_.getClass.getName == codeAttr.getName).map { a =>
993+
val byteCodeSize = codeAttrField.get(a).asInstanceOf[Array[Byte]].length
994+
CodegenMetrics.METRIC_GENERATED_METHOD_BYTECODE_SIZE.update(byteCodeSize)
995+
byteCodeSize
989996
}
990997
}
998+
Some(stats)
991999
} catch {
9921000
case NonFatal(e) =>
9931001
logWarning("Error calculating stats of compiled class.", e)
1002+
None
9941003
}
995-
}
1004+
}.flatten
1005+
1006+
codeSizes.max
9961007
}
9971008

9981009
/**
@@ -1007,8 +1018,8 @@ object CodeGenerator extends Logging {
10071018
private val cache = CacheBuilder.newBuilder()
10081019
.maximumSize(100)
10091020
.build(
1010-
new CacheLoader[CodeAndComment, GeneratedClass]() {
1011-
override def load(code: CodeAndComment): GeneratedClass = {
1021+
new CacheLoader[CodeAndComment, (GeneratedClass, Int)]() {
1022+
override def load(code: CodeAndComment): (GeneratedClass, Int) = {
10121023
val startTime = System.nanoTime()
10131024
val result = doCompile(code)
10141025
val endTime = System.nanoTime()

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -142,7 +142,7 @@ object GenerateMutableProjection extends CodeGenerator[Seq[Expression], MutableP
142142
new CodeAndComment(codeBody, ctx.getPlaceHolderToComments()))
143143
logDebug(s"code for ${expressions.mkString(",")}:\n${CodeFormatter.format(code)}")
144144

145-
val c = CodeGenerator.compile(code)
146-
c.generate(ctx.references.toArray).asInstanceOf[MutableProjection]
145+
val (clazz, _) = CodeGenerator.compile(code)
146+
clazz.generate(ctx.references.toArray).asInstanceOf[MutableProjection]
147147
}
148148
}

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

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -185,7 +185,8 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[InternalR
185185
new CodeAndComment(codeBody, ctx.getPlaceHolderToComments()))
186186
logDebug(s"Generated Ordering by ${ordering.mkString(",")}:\n${CodeFormatter.format(code)}")
187187

188-
CodeGenerator.compile(code).generate(ctx.references.toArray).asInstanceOf[BaseOrdering]
188+
val (clazz, _) = CodeGenerator.compile(code)
189+
clazz.generate(ctx.references.toArray).asInstanceOf[BaseOrdering]
189190
}
190191
}
191192

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

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -78,6 +78,7 @@ object GeneratePredicate extends CodeGenerator[Expression, Predicate] {
7878
new CodeAndComment(codeBody, ctx.getPlaceHolderToComments()))
7979
logDebug(s"Generated predicate '$predicate':\n${CodeFormatter.format(code)}")
8080

81-
CodeGenerator.compile(code).generate(ctx.references.toArray).asInstanceOf[Predicate]
81+
val (clazz, _) = CodeGenerator.compile(code)
82+
clazz.generate(ctx.references.toArray).asInstanceOf[Predicate]
8283
}
8384
}

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -191,8 +191,8 @@ object GenerateSafeProjection extends CodeGenerator[Seq[Expression], Projection]
191191
new CodeAndComment(codeBody, ctx.getPlaceHolderToComments()))
192192
logDebug(s"code for ${expressions.mkString(",")}:\n${CodeFormatter.format(code)}")
193193

194-
val c = CodeGenerator.compile(code)
194+
val (clazz, _) = CodeGenerator.compile(code)
195195
val resultRow = new SpecificInternalRow(expressions.map(_.dataType))
196-
c.generate(ctx.references.toArray :+ resultRow).asInstanceOf[Projection]
196+
clazz.generate(ctx.references.toArray :+ resultRow).asInstanceOf[Projection]
197197
}
198198
}

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -409,7 +409,7 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro
409409
new CodeAndComment(codeBody, ctx.getPlaceHolderToComments()))
410410
logDebug(s"code for ${expressions.mkString(",")}:\n${CodeFormatter.format(code)}")
411411

412-
val c = CodeGenerator.compile(code)
413-
c.generate(ctx.references.toArray).asInstanceOf[UnsafeProjection]
412+
val (clazz, _) = CodeGenerator.compile(code)
413+
clazz.generate(ctx.references.toArray).asInstanceOf[UnsafeProjection]
414414
}
415415
}

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -196,7 +196,7 @@ object GenerateUnsafeRowJoiner extends CodeGenerator[(StructType, StructType), U
196196
val code = CodeFormatter.stripOverlappingComments(new CodeAndComment(codeBody, Map.empty))
197197
logDebug(s"SpecificUnsafeRowJoiner($schema1, $schema2):\n${CodeFormatter.format(code)}")
198198

199-
val c = CodeGenerator.compile(code)
200-
c.generate(Array.empty).asInstanceOf[UnsafeRowJoiner]
199+
val (clazz, _) = CodeGenerator.compile(code)
200+
clazz.generate(Array.empty).asInstanceOf[UnsafeRowJoiner]
201201
}
202202
}

sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -30,6 +30,7 @@ import org.apache.spark.internal.Logging
3030
import org.apache.spark.internal.config._
3131
import org.apache.spark.network.util.ByteUnit
3232
import org.apache.spark.sql.catalyst.analysis.Resolver
33+
import org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator
3334

3435
////////////////////////////////////////////////////////////////////////////////////////////////////
3536
// This file defines the configuration options for Spark SQL.
@@ -585,6 +586,16 @@ object SQLConf {
585586
.intConf
586587
.createWithDefault(20)
587588

589+
val WHOLESTAGE_HUGE_METHOD_LIMIT = SQLConfigBuilder("spark.sql.codegen.hugeMethodLimit")
590+
.internal()
591+
.doc("The maximum bytecode size of a single compiled Java function generated by whole-stage " +
592+
"codegen. When the compiled function exceeds this threshold, " +
593+
"the whole-stage codegen is deactivated for this subtree of the current query plan. " +
594+
s"The default value is ${CodeGenerator.DEFAULT_JVM_HUGE_METHOD_LIMIT} and " +
595+
"this is a limit in the OpenJDK JVM implementation.")
596+
.intConf
597+
.createWithDefault(CodeGenerator.DEFAULT_JVM_HUGE_METHOD_LIMIT)
598+
588599
val FILES_MAX_PARTITION_BYTES = SQLConfigBuilder("spark.sql.files.maxPartitionBytes")
589600
.doc("The maximum number of bytes to pack into a single partition when reading files.")
590601
.longConf
@@ -895,6 +906,8 @@ class SQLConf extends Serializable with Logging {
895906

896907
def maxCaseBranchesForCodegen: Int = getConf(MAX_CASES_BRANCHES)
897908

909+
def hugeMethodLimit: Int = getConf(WHOLESTAGE_HUGE_METHOD_LIMIT)
910+
898911
def exchangeReuseEnabled: Boolean = getConf(EXCHANGE_REUSE_ENABLED)
899912

900913
def caseSensitiveAnalysis: Boolean = getConf(SQLConf.CASE_SENSITIVE)

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

Lines changed: 14 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -353,14 +353,25 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co
353353
override def doExecute(): RDD[InternalRow] = {
354354
val (ctx, cleanedSource) = doCodeGen()
355355
// try to compile and fallback if it failed
356-
try {
356+
val (_, maxCodeSize) = try {
357357
CodeGenerator.compile(cleanedSource)
358358
} catch {
359359
case e: Exception if !Utils.isTesting && sqlContext.conf.wholeStageFallback =>
360360
// We should already saw the error message
361361
logWarning(s"Whole-stage codegen disabled for this plan:\n $treeString")
362362
return child.execute()
363363
}
364+
365+
// Check if compiled code has a too large function
366+
if (maxCodeSize > sqlContext.conf.hugeMethodLimit) {
367+
logWarning(s"Found too long generated codes and JIT optimization might not work: " +
368+
s"the bytecode size was $maxCodeSize, this value went over the limit " +
369+
s"${sqlContext.conf.hugeMethodLimit}, and the whole-stage codegen was disabled " +
370+
s"for this plan. To avoid this, you can raise the limit " +
371+
s"${SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key}:\n$treeString")
372+
return child.execute()
373+
}
374+
364375
val references = ctx.references.toArray
365376

366377
val durationMs = longMetric("pipelineTime")
@@ -371,7 +382,7 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co
371382
if (rdds.length == 1) {
372383
rdds.head.mapPartitionsWithIndex { (index, iter) =>
373384
WholeStageCodegenExec.this.logInfo(logMsg)
374-
val clazz = CodeGenerator.compile(cleanedSource)
385+
val (clazz, _) = CodeGenerator.compile(cleanedSource)
375386
val buffer = clazz.generate(references).asInstanceOf[BufferedRowIterator]
376387
buffer.init(index, Array(iter))
377388
new Iterator[InternalRow] {
@@ -391,7 +402,7 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co
391402
}.mapPartitionsWithIndex { (index, zippedIter) =>
392403
WholeStageCodegenExec.this.logInfo(logMsg)
393404
val (leftIter, rightIter) = zippedIter.next()
394-
val clazz = CodeGenerator.compile(cleanedSource)
405+
val (clazz, _) = CodeGenerator.compile(cleanedSource)
395406
val buffer = clazz.generate(references).asInstanceOf[BufferedRowIterator]
396407
buffer.init(index, Array(leftIter, rightIter))
397408
new Iterator[InternalRow] {

sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -230,6 +230,7 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera
230230
new CodeAndComment(codeBody, ctx.getPlaceHolderToComments()))
231231
logDebug(s"Generated ColumnarIterator:\n${CodeFormatter.format(code)}")
232232

233-
CodeGenerator.compile(code).generate(Array.empty).asInstanceOf[ColumnarIterator]
233+
val (clazz, _) = CodeGenerator.compile(code)
234+
clazz.generate(Array.empty).asInstanceOf[ColumnarIterator]
234235
}
235236
}

0 commit comments

Comments
 (0)