From b63d99c94a14d35b95b32d8f5d52836ea5826ef6 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Wed, 23 Aug 2017 22:25:02 +0900 Subject: [PATCH 01/14] Check compiled byte code size --- .../expressions/codegen/CodeGenerator.scala | 33 +++- .../resources/sql-tests/inputs/group-by.sql | 7 + .../sql-tests/results/group-by.sql.out | 88 ++++++----- .../spark/sql/DataFrameAggregateSuite.scala | 42 ++--- .../sql/DataFrameTimeWindowingSuite.scala | 143 ++++++++++-------- .../org/apache/spark/sql/SQLQuerySuite.scala | 44 +++--- .../execution/WholeStageCodegenSuite.scala | 26 +++- 7 files changed, 236 insertions(+), 147 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index f3b45799c568..4072b9138373 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -32,7 +32,7 @@ import org.codehaus.commons.compiler.CompileException import org.codehaus.janino.{ByteArrayClassLoader, ClassBodyEvaluator, JaninoRuntimeException, SimpleCompiler} import org.codehaus.janino.util.ClassFile -import org.apache.spark.{SparkEnv, TaskContext, TaskKilledException} +import org.apache.spark.{SparkContext, SparkEnv, TaskContext, TaskKilledException} import org.apache.spark.executor.InputMetrics import org.apache.spark.internal.Logging import org.apache.spark.metrics.source.CodegenMetrics @@ -1020,6 +1020,16 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin } object CodeGenerator extends Logging { + + // This is the value of `HugeMethodLimit` in JVM settings. Since we can't get this value on + // runtime and this is fixed in released JVMs, we hard-code this values here: + // + // $ java -XX:+UnlockDiagnosticVMOptions -XX:HugeMethodLimit=99999 + // Error: VM option 'HugeMethodLimit' is develop and is available only in debug version of VM. + // Error: Could not create the Java Virtual Machine. + // Error: A fatal exception has occurred. Program will exit. + private val hugeMethodLimit = 8000 + /** * Compile the Java source code into a Java class, using Janino. */ @@ -1077,7 +1087,17 @@ object CodeGenerator extends Logging { try { evaluator.cook("generated.java", code.body) - recordCompilationStats(evaluator) + val methodsToByteCodeSize = updateAndGetCompilationStats(evaluator) + methodsToByteCodeSize.foreach { case (name, byteCodeSize) => + if (byteCodeSize > hugeMethodLimit) { + val clazzName = evaluator.getClazz.getSimpleName + val methodName = name.replace("$", "") + throw new IllegalArgumentException( + s"the size of $clazzName.$methodName is $byteCodeSize and this value goes over " + + s"the HugeMethodLimit $hugeMethodLimit (JVM doesn't compile methods " + + "larger than this limit)") + } + } } catch { case e: JaninoRuntimeException => val msg = s"failed to compile: $e" @@ -1098,7 +1118,7 @@ object CodeGenerator extends Logging { /** * Records the generated class and method bytecode sizes by inspecting janino private fields. */ - private def recordCompilationStats(evaluator: ClassBodyEvaluator): Unit = { + private def updateAndGetCompilationStats(evaluator: ClassBodyEvaluator): Seq[(String, Int)] = { // First retrieve the generated classes. val classes = { val resultField = classOf[SimpleCompiler].getDeclaredField("result") @@ -1110,6 +1130,7 @@ object CodeGenerator extends Logging { } // Then walk the classes to get at the method bytecode. + val methodsToByteCodeSize = mutable.ArrayBuffer[(String, Int)]() val codeAttr = Utils.classForName("org.codehaus.janino.util.ClassFile$CodeAttribute") val codeAttrField = codeAttr.getDeclaredField("code") codeAttrField.setAccessible(true) @@ -1120,8 +1141,9 @@ object CodeGenerator extends Logging { cf.methodInfos.asScala.foreach { method => method.getAttributes().foreach { a => if (a.getClass.getName == codeAttr.getName) { - CodegenMetrics.METRIC_GENERATED_METHOD_BYTECODE_SIZE.update( - codeAttrField.get(a).asInstanceOf[Array[Byte]].length) + val byteCodeSize = codeAttrField.get(a).asInstanceOf[Array[Byte]].length + CodegenMetrics.METRIC_GENERATED_METHOD_BYTECODE_SIZE.update(byteCodeSize) + methodsToByteCodeSize.append((method.getName, byteCodeSize)) } } } @@ -1130,6 +1152,7 @@ object CodeGenerator extends Logging { logWarning("Error calculating stats of compiled class.", e) } } + methodsToByteCodeSize.toSeq } /** diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql index 1e1384549a41..58c7830f77dc 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql @@ -30,8 +30,15 @@ SELECT a + 2, COUNT(b) FROM testData GROUP BY a + 1; SELECT a + 1 + 1, COUNT(b) FROM testData GROUP BY a + 1; -- Aggregate with nulls. +-- +-- In SPARK-21871, we added code to check the bytecode size of gen'd methods. If the size +-- goes over `HugeMethodLimit`, Spark fails to compile the methods and the execution also fails +-- in a test mode. So, we explicitly turn off whole-stage codegen here. +-- This guard can be removed if this issue fixed. +SET spark.sql.codegen.wholeStage=false; SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a), AVG(a), VARIANCE(a), STDDEV(a), SUM(a), COUNT(a) FROM testData; +SET spark.sql.codegen.wholeStage=true; -- Aggregate with foldable input and multiple distinct groups. SELECT COUNT(DISTINCT b), COUNT(DISTINCT b, c) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a; diff --git a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out index 986bb01c13fe..11e8f9fbf988 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 25 +-- Number of queries: 27 -- !query 0 @@ -125,105 +125,121 @@ NULL 1 -- !query 13 +SET spark.sql.codegen.wholeStage=false +-- !query 13 schema +struct +-- !query 13 output +spark.sql.codegen.wholeStage false + + +-- !query 14 SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a), AVG(a), VARIANCE(a), STDDEV(a), SUM(a), COUNT(a) FROM testData --- !query 13 schema +-- !query 14 schema struct --- !query 13 output +-- !query 14 output -0.2723801058145729 -1.5069204152249134 1 3 2.142857142857143 0.8095238095238094 0.8997354108424372 15 7 --- !query 14 +-- !query 15 +SET spark.sql.codegen.wholeStage=true +-- !query 15 schema +struct +-- !query 15 output +spark.sql.codegen.wholeStage true + + +-- !query 16 SELECT COUNT(DISTINCT b), COUNT(DISTINCT b, c) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a --- !query 14 schema +-- !query 16 schema struct --- !query 14 output +-- !query 16 output 1 1 --- !query 15 +-- !query 17 SELECT a AS k, COUNT(b) FROM testData GROUP BY k --- !query 15 schema +-- !query 17 schema struct --- !query 15 output +-- !query 17 output 1 2 2 2 3 2 NULL 1 --- !query 16 +-- !query 18 SELECT a AS k, COUNT(b) FROM testData GROUP BY k HAVING k > 1 --- !query 16 schema +-- !query 18 schema struct --- !query 16 output +-- !query 18 output 2 2 3 2 --- !query 17 +-- !query 19 SELECT COUNT(b) AS k FROM testData GROUP BY k --- !query 17 schema +-- !query 19 schema struct<> --- !query 17 output +-- !query 19 output org.apache.spark.sql.AnalysisException aggregate functions are not allowed in GROUP BY, but found count(testdata.`b`); --- !query 18 +-- !query 20 CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES (1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v) --- !query 18 schema +-- !query 20 schema struct<> --- !query 18 output +-- !query 20 output --- !query 19 +-- !query 21 SELECT k AS a, COUNT(v) FROM testDataHasSameNameWithAlias GROUP BY a --- !query 19 schema +-- !query 21 schema struct<> --- !query 19 output +-- !query 21 output org.apache.spark.sql.AnalysisException expression 'testdatahassamenamewithalias.`k`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.; --- !query 20 +-- !query 22 set spark.sql.groupByAliases=false --- !query 20 schema +-- !query 22 schema struct --- !query 20 output +-- !query 22 output spark.sql.groupByAliases false --- !query 21 +-- !query 23 SELECT a AS k, COUNT(b) FROM testData GROUP BY k --- !query 21 schema +-- !query 23 schema struct<> --- !query 21 output +-- !query 23 output org.apache.spark.sql.AnalysisException cannot resolve '`k`' given input columns: [testdata.a, testdata.b]; line 1 pos 47 --- !query 22 +-- !query 24 SELECT a, COUNT(1) FROM testData WHERE false GROUP BY a --- !query 22 schema +-- !query 24 schema struct --- !query 22 output +-- !query 24 output --- !query 23 +-- !query 25 SELECT COUNT(1) FROM testData WHERE false --- !query 23 schema +-- !query 25 schema struct --- !query 23 output +-- !query 25 output 0 --- !query 24 +-- !query 26 SELECT 1 FROM (SELECT COUNT(1) FROM testData WHERE false) t --- !query 24 schema +-- !query 26 schema struct<1:int> --- !query 24 output +-- !query 26 output 1 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index 8549eac58ee9..0643f6dfb952 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -432,25 +432,31 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { } test("zero moments") { - val input = Seq((1, 2)).toDF("a", "b") - checkAnswer( - input.agg(stddev('a), stddev_samp('a), stddev_pop('a), variance('a), - var_samp('a), var_pop('a), skewness('a), kurtosis('a)), - Row(Double.NaN, Double.NaN, 0.0, Double.NaN, Double.NaN, 0.0, - Double.NaN, Double.NaN)) + // In SPARK-21871, we added code to check the bytecode size of gen'd methods. If the size + // goes over `HugeMethodLimit`, Spark fails to compile the methods and the execution also fails + // in a test mode. So, we explicitly turn off whole-stage codegen here. + // This guard can be removed if this issue fixed. + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { + val input = Seq((1, 2)).toDF("a", "b") + checkAnswer( + input.agg(stddev('a), stddev_samp('a), stddev_pop('a), variance('a), + var_samp('a), var_pop('a), skewness('a), kurtosis('a)), + Row(Double.NaN, Double.NaN, 0.0, Double.NaN, Double.NaN, 0.0, + Double.NaN, Double.NaN)) - checkAnswer( - input.agg( - expr("stddev(a)"), - expr("stddev_samp(a)"), - expr("stddev_pop(a)"), - expr("variance(a)"), - expr("var_samp(a)"), - expr("var_pop(a)"), - expr("skewness(a)"), - expr("kurtosis(a)")), - Row(Double.NaN, Double.NaN, 0.0, Double.NaN, Double.NaN, 0.0, - Double.NaN, Double.NaN)) + checkAnswer( + input.agg( + expr("stddev(a)"), + expr("stddev_samp(a)"), + expr("stddev_pop(a)"), + expr("variance(a)"), + expr("var_samp(a)"), + expr("var_pop(a)"), + expr("skewness(a)"), + expr("kurtosis(a)")), + Row(Double.NaN, Double.NaN, 0.0, Double.NaN, Double.NaN, 0.0, + Double.NaN, Double.NaN)) + } } test("null moments") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala index 6fe356877c26..195e3f4af17b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala @@ -21,6 +21,7 @@ import org.scalatest.BeforeAndAfterEach import org.apache.spark.sql.catalyst.plans.logical.Expand import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.StringType @@ -95,50 +96,62 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext with B } test("sliding window grouping") { - val df = Seq( - ("2016-03-27 19:39:34", 1, "a"), - ("2016-03-27 19:39:56", 2, "a"), - ("2016-03-27 19:39:27", 4, "b")).toDF("time", "value", "id") - - checkAnswer( - df.groupBy(window($"time", "10 seconds", "3 seconds", "0 second")) - .agg(count("*").as("counts")) - .orderBy($"window.start".asc) - .select($"window.start".cast("string"), $"window.end".cast("string"), $"counts"), - // 2016-03-27 19:39:27 UTC -> 4 bins - // 2016-03-27 19:39:34 UTC -> 3 bins - // 2016-03-27 19:39:56 UTC -> 3 bins - Seq( - Row("2016-03-27 19:39:18", "2016-03-27 19:39:28", 1), - Row("2016-03-27 19:39:21", "2016-03-27 19:39:31", 1), - Row("2016-03-27 19:39:24", "2016-03-27 19:39:34", 1), - Row("2016-03-27 19:39:27", "2016-03-27 19:39:37", 2), - Row("2016-03-27 19:39:30", "2016-03-27 19:39:40", 1), - Row("2016-03-27 19:39:33", "2016-03-27 19:39:43", 1), - Row("2016-03-27 19:39:48", "2016-03-27 19:39:58", 1), - Row("2016-03-27 19:39:51", "2016-03-27 19:40:01", 1), - Row("2016-03-27 19:39:54", "2016-03-27 19:40:04", 1)) - ) - } - - test("sliding window projection") { - val df = Seq( + // In SPARK-21871, we added code to check the bytecode size of gen'd methods. If the size + // goes over `HugeMethodLimit`, Spark fails to compile the methods and the execution also fails + // in a test mode. So, we explicitly turn off whole-stage codegen here. + // This guard can be removed if this issue fixed. + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { + val df = Seq( ("2016-03-27 19:39:34", 1, "a"), ("2016-03-27 19:39:56", 2, "a"), ("2016-03-27 19:39:27", 4, "b")).toDF("time", "value", "id") - .select(window($"time", "10 seconds", "3 seconds", "0 second"), $"value") - .orderBy($"window.start".asc, $"value".desc).select("value") - val expands = df.queryExecution.optimizedPlan.find(_.isInstanceOf[Expand]) - assert(expands.nonEmpty, "Sliding windows require expand") + checkAnswer( + df.groupBy(window($"time", "10 seconds", "3 seconds", "0 second")) + .agg(count("*").as("counts")) + .orderBy($"window.start".asc) + .select($"window.start".cast("string"), $"window.end".cast("string"), $"counts"), + // 2016-03-27 19:39:27 UTC -> 4 bins + // 2016-03-27 19:39:34 UTC -> 3 bins + // 2016-03-27 19:39:56 UTC -> 3 bins + Seq( + Row("2016-03-27 19:39:18", "2016-03-27 19:39:28", 1), + Row("2016-03-27 19:39:21", "2016-03-27 19:39:31", 1), + Row("2016-03-27 19:39:24", "2016-03-27 19:39:34", 1), + Row("2016-03-27 19:39:27", "2016-03-27 19:39:37", 2), + Row("2016-03-27 19:39:30", "2016-03-27 19:39:40", 1), + Row("2016-03-27 19:39:33", "2016-03-27 19:39:43", 1), + Row("2016-03-27 19:39:48", "2016-03-27 19:39:58", 1), + Row("2016-03-27 19:39:51", "2016-03-27 19:40:01", 1), + Row("2016-03-27 19:39:54", "2016-03-27 19:40:04", 1)) + ) + } + } - checkAnswer( - df, - // 2016-03-27 19:39:27 UTC -> 4 bins - // 2016-03-27 19:39:34 UTC -> 3 bins - // 2016-03-27 19:39:56 UTC -> 3 bins - Seq(Row(4), Row(4), Row(4), Row(4), Row(1), Row(1), Row(1), Row(2), Row(2), Row(2)) - ) + test("sliding window projection") { + // In SPARK-21871, we added code to check the bytecode size of gen'd methods. If the size + // goes over `HugeMethodLimit`, Spark fails to compile the methods and the execution also fails + // in a test mode. So, we explicitly turn off whole-stage codegen here. + // This guard can be removed if this issue fixed. + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { + val df = Seq( + ("2016-03-27 19:39:34", 1, "a"), + ("2016-03-27 19:39:56", 2, "a"), + ("2016-03-27 19:39:27", 4, "b")).toDF("time", "value", "id") + .select(window($"time", "10 seconds", "3 seconds", "0 second"), $"value") + .orderBy($"window.start".asc, $"value".desc).select("value") + + val expands = df.queryExecution.optimizedPlan.find(_.isInstanceOf[Expand]) + assert(expands.nonEmpty, "Sliding windows require expand") + + checkAnswer( + df, + // 2016-03-27 19:39:27 UTC -> 4 bins + // 2016-03-27 19:39:34 UTC -> 3 bins + // 2016-03-27 19:39:56 UTC -> 3 bins + Seq(Row(4), Row(4), Row(4), Row(4), Row(1), Row(1), Row(1), Row(2), Row(2), Row(2)) + ) + } } test("windowing combined with explode expression") { @@ -228,29 +241,35 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext with B } test("millisecond precision sliding windows") { - val df = Seq( - ("2016-03-27 09:00:00.41", 3), - ("2016-03-27 09:00:00.62", 6), - ("2016-03-27 09:00:00.715", 8)).toDF("time", "value") - checkAnswer( - df.groupBy(window($"time", "200 milliseconds", "40 milliseconds", "0 milliseconds")) - .agg(count("*").as("counts")) - .orderBy($"window.start".asc) - .select($"window.start".cast(StringType), $"window.end".cast(StringType), $"counts"), - Seq( - Row("2016-03-27 09:00:00.24", "2016-03-27 09:00:00.44", 1), - Row("2016-03-27 09:00:00.28", "2016-03-27 09:00:00.48", 1), - Row("2016-03-27 09:00:00.32", "2016-03-27 09:00:00.52", 1), - Row("2016-03-27 09:00:00.36", "2016-03-27 09:00:00.56", 1), - Row("2016-03-27 09:00:00.4", "2016-03-27 09:00:00.6", 1), - Row("2016-03-27 09:00:00.44", "2016-03-27 09:00:00.64", 1), - Row("2016-03-27 09:00:00.48", "2016-03-27 09:00:00.68", 1), - Row("2016-03-27 09:00:00.52", "2016-03-27 09:00:00.72", 2), - Row("2016-03-27 09:00:00.56", "2016-03-27 09:00:00.76", 2), - Row("2016-03-27 09:00:00.6", "2016-03-27 09:00:00.8", 2), - Row("2016-03-27 09:00:00.64", "2016-03-27 09:00:00.84", 1), - Row("2016-03-27 09:00:00.68", "2016-03-27 09:00:00.88", 1)) - ) + // In SPARK-21871, we added code to check the bytecode size of gen'd methods. If the size + // goes over `HugeMethodLimit`, Spark fails to compile the methods and the execution also fails + // in a test mode. So, we explicitly turn off whole-stage codegen here. + // This guard can be removed if this issue fixed. + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { + val df = Seq( + ("2016-03-27 09:00:00.41", 3), + ("2016-03-27 09:00:00.62", 6), + ("2016-03-27 09:00:00.715", 8)).toDF("time", "value") + checkAnswer( + df.groupBy(window($"time", "200 milliseconds", "40 milliseconds", "0 milliseconds")) + .agg(count("*").as("counts")) + .orderBy($"window.start".asc) + .select($"window.start".cast(StringType), $"window.end".cast(StringType), $"counts"), + Seq( + Row("2016-03-27 09:00:00.24", "2016-03-27 09:00:00.44", 1), + Row("2016-03-27 09:00:00.28", "2016-03-27 09:00:00.48", 1), + Row("2016-03-27 09:00:00.32", "2016-03-27 09:00:00.52", 1), + Row("2016-03-27 09:00:00.36", "2016-03-27 09:00:00.56", 1), + Row("2016-03-27 09:00:00.4", "2016-03-27 09:00:00.6", 1), + Row("2016-03-27 09:00:00.44", "2016-03-27 09:00:00.64", 1), + Row("2016-03-27 09:00:00.48", "2016-03-27 09:00:00.68", 1), + Row("2016-03-27 09:00:00.52", "2016-03-27 09:00:00.72", 2), + Row("2016-03-27 09:00:00.56", "2016-03-27 09:00:00.76", 2), + Row("2016-03-27 09:00:00.6", "2016-03-27 09:00:00.8", 2), + Row("2016-03-27 09:00:00.64", "2016-03-27 09:00:00.84", 1), + Row("2016-03-27 09:00:00.68", "2016-03-27 09:00:00.88", 1)) + ) + } } private def withTempTable(f: String => Unit): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 93a7777b70b4..5a21f674a228 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -2102,25 +2102,31 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-15327: fail to compile generated code with complex data structure") { - withTempDir{ dir => - val json = - """ - |{"h": {"b": {"c": [{"e": "adfgd"}], "a": [{"e": "testing", "count": 3}], - |"b": [{"e": "test", "count": 1}]}}, "d": {"b": {"c": [{"e": "adfgd"}], - |"a": [{"e": "testing", "count": 3}], "b": [{"e": "test", "count": 1}]}}, - |"c": {"b": {"c": [{"e": "adfgd"}], "a": [{"count": 3}], - |"b": [{"e": "test", "count": 1}]}}, "a": {"b": {"c": [{"e": "adfgd"}], - |"a": [{"count": 3}], "b": [{"e": "test", "count": 1}]}}, - |"e": {"b": {"c": [{"e": "adfgd"}], "a": [{"e": "testing", "count": 3}], - |"b": [{"e": "test", "count": 1}]}}, "g": {"b": {"c": [{"e": "adfgd"}], - |"a": [{"e": "testing", "count": 3}], "b": [{"e": "test", "count": 1}]}}, - |"f": {"b": {"c": [{"e": "adfgd"}], "a": [{"e": "testing", "count": 3}], - |"b": [{"e": "test", "count": 1}]}}, "b": {"b": {"c": [{"e": "adfgd"}], - |"a": [{"count": 3}], "b": [{"e": "test", "count": 1}]}}}' - | - """.stripMargin - spark.read.json(Seq(json).toDS()).write.mode("overwrite").parquet(dir.toString) - spark.read.parquet(dir.toString).collect() + // In SPARK-21871, we added code to check the bytecode size of gen'd methods. If the size + // goes over `HugeMethodLimit`, Spark fails to compile the methods and the execution also fails + // in a test mode. So, we explicitly turn off whole-stage codegen here. + // This guard can be removed if this issue fixed. + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { + withTempDir { dir => + val json = + """ + |{"h": {"b": {"c": [{"e": "adfgd"}], "a": [{"e": "testing", "count": 3}], + |"b": [{"e": "test", "count": 1}]}}, "d": {"b": {"c": [{"e": "adfgd"}], + |"a": [{"e": "testing", "count": 3}], "b": [{"e": "test", "count": 1}]}}, + |"c": {"b": {"c": [{"e": "adfgd"}], "a": [{"count": 3}], + |"b": [{"e": "test", "count": 1}]}}, "a": {"b": {"c": [{"e": "adfgd"}], + |"a": [{"count": 3}], "b": [{"e": "test", "count": 1}]}}, + |"e": {"b": {"c": [{"e": "adfgd"}], "a": [{"e": "testing", "count": 3}], + |"b": [{"e": "test", "count": 1}]}}, "g": {"b": {"c": [{"e": "adfgd"}], + |"a": [{"e": "testing", "count": 3}], "b": [{"e": "test", "count": 1}]}}, + |"f": {"b": {"c": [{"e": "adfgd"}], "a": [{"e": "testing", "count": 3}], + |"b": [{"e": "test", "count": 1}]}}, "b": {"b": {"c": [{"e": "adfgd"}], + |"a": [{"count": 3}], "b": [{"e": "test", "count": 1}]}}}' + | + """.stripMargin + spark.read.json(Seq(json).toDS()).write.mode("overwrite").parquet(dir.toString) + spark.read.parquet(dir.toString).collect() + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index beeee6a97c8d..9a0d75fc7bbe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.{Column, Dataset, Row} import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions.{Add, Literal, Stack} -import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext +import org.apache.spark.sql.catalyst.expressions.codegen.{CodeAndComment, CodegenContext, CodeGenerator} import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec import org.apache.spark.sql.execution.joins.SortMergeJoinExec @@ -151,7 +151,7 @@ class WholeStageCodegenSuite extends SparkPlanTest with SharedSQLContext { } } - def genGroupByCodeGenContext(caseNum: Int): CodegenContext = { + def genGroupByCodeGenContext(caseNum: Int): (CodegenContext, CodeAndComment) = { val caseExp = (1 to caseNum).map { i => s"case when id > $i and id <= ${i + 1} then 1 else 0 end as v$i" }.toList @@ -176,34 +176,46 @@ class WholeStageCodegenSuite extends SparkPlanTest with SharedSQLContext { }) assert(wholeStageCodeGenExec.isDefined) - wholeStageCodeGenExec.get.asInstanceOf[WholeStageCodegenExec].doCodeGen()._1 + wholeStageCodeGenExec.get.asInstanceOf[WholeStageCodegenExec].doCodeGen() } test("SPARK-21603 check there is a too long generated function") { withSQLConf(SQLConf.WHOLESTAGE_MAX_LINES_PER_FUNCTION.key -> "1500") { - val ctx = genGroupByCodeGenContext(30) + val (ctx, _) = genGroupByCodeGenContext(30) assert(ctx.isTooLongGeneratedFunction === true) } } test("SPARK-21603 check there is not a too long generated function") { withSQLConf(SQLConf.WHOLESTAGE_MAX_LINES_PER_FUNCTION.key -> "1500") { - val ctx = genGroupByCodeGenContext(1) + val (ctx, _) = genGroupByCodeGenContext(1) assert(ctx.isTooLongGeneratedFunction === false) } } test("SPARK-21603 check there is not a too long generated function when threshold is Int.Max") { withSQLConf(SQLConf.WHOLESTAGE_MAX_LINES_PER_FUNCTION.key -> Int.MaxValue.toString) { - val ctx = genGroupByCodeGenContext(30) + val (ctx, _) = genGroupByCodeGenContext(30) assert(ctx.isTooLongGeneratedFunction === false) } } test("SPARK-21603 check there is a too long generated function when threshold is 0") { withSQLConf(SQLConf.WHOLESTAGE_MAX_LINES_PER_FUNCTION.key -> "0") { - val ctx = genGroupByCodeGenContext(1) + val (ctx, _) = genGroupByCodeGenContext(1) assert(ctx.isTooLongGeneratedFunction === true) } } + + test("SPARK-21871 turn off whole-stage codegen if bytecode size goes over HugeMethodLimit") { + withSQLConf(SQLConf.WHOLESTAGE_MAX_LINES_PER_FUNCTION.key -> Int.MaxValue.toString) { + val (_, code) = genGroupByCodeGenContext(20) + val errMsg = intercept[IllegalArgumentException] { + CodeGenerator.compile(code) + }.getMessage + assert(errMsg.contains("the size of GeneratedClass.agg_doAggregateWithKeys is 9182 and " + + "this value goes over the HugeMethodLimit 8000 (JVM doesn't compile methods " + + "larger than this limit)")) + } + } } From ad7881d5682320313be1d287ff00118ab8b56ac2 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Fri, 1 Sep 2017 14:37:51 +0900 Subject: [PATCH 02/14] Add a new option for hugeMethodLimit --- .../expressions/codegen/CodeGenerator.scala | 63 ++++++++++--------- .../apache/spark/sql/internal/SQLConf.scala | 22 ++++++- .../resources/sql-tests/inputs/group-by.sql | 2 +- .../spark/sql/DataFrameAggregateSuite.scala | 2 +- .../sql/DataFrameTimeWindowingSuite.scala | 6 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 2 +- .../execution/WholeStageCodegenSuite.scala | 6 +- 7 files changed, 59 insertions(+), 44 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 4072b9138373..ef765a0915e7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -1021,14 +1021,8 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin object CodeGenerator extends Logging { - // This is the value of `HugeMethodLimit` in JVM settings. Since we can't get this value on - // runtime and this is fixed in released JVMs, we hard-code this values here: - // - // $ java -XX:+UnlockDiagnosticVMOptions -XX:HugeMethodLimit=99999 - // Error: VM option 'HugeMethodLimit' is develop and is available only in debug version of VM. - // Error: Could not create the Java Virtual Machine. - // Error: A fatal exception has occurred. Program will exit. - private val hugeMethodLimit = 8000 + // This is the value of `HugeMethodLimit` in the OpenJDK JVM settings. + val DEFAULT_OPENJDK_JVM_HUGE_METHOD_LIMIT = 8000 /** * Compile the Java source code into a Java class, using Janino. @@ -1085,19 +1079,9 @@ object CodeGenerator extends Logging { s"\n${CodeFormatter.format(code)}" }) - try { + val methodsToByteCodeSize = try { evaluator.cook("generated.java", code.body) - val methodsToByteCodeSize = updateAndGetCompilationStats(evaluator) - methodsToByteCodeSize.foreach { case (name, byteCodeSize) => - if (byteCodeSize > hugeMethodLimit) { - val clazzName = evaluator.getClazz.getSimpleName - val methodName = name.replace("$", "") - throw new IllegalArgumentException( - s"the size of $clazzName.$methodName is $byteCodeSize and this value goes over " + - s"the HugeMethodLimit $hugeMethodLimit (JVM doesn't compile methods " + - "larger than this limit)") - } - } + updateAndGetCompilationStats(evaluator) } catch { case e: JaninoRuntimeException => val msg = s"failed to compile: $e" @@ -1112,11 +1096,28 @@ object CodeGenerator extends Logging { logInfo(s"\n${CodeFormatter.format(code, maxLines)}") throw new CompileException(msg, e.getLocation) } + + // Check if compiled code has a too large function. + methodsToByteCodeSize.foreach { case (name, byteCodeSize) => + if (byteCodeSize > SQLConf.get.hugeMethodLimit) { + val clazzName = evaluator.getClazz.getSimpleName + val methodName = name.replace("$", "") + val msg = s"failed to compile: the size of $clazzName.$methodName was $byteCodeSize and " + + "this value went over the limit `spark.sql.codegen.hugeMethodLimit`" + + s"(${SQLConf.get.hugeMethodLimit}). To avoid this error, you can make this limit higher." + logError(msg) + val maxLines = SQLConf.get.loggingMaxLinesForCodegen + logInfo(s"\n${CodeFormatter.format(code, maxLines)}") + throw new IllegalArgumentException(msg) + } + } + evaluator.getClazz().newInstance().asInstanceOf[GeneratedClass] } /** - * Records the generated class and method bytecode sizes by inspecting janino private fields. + * Returns the pairs of the generated class and method bytecode sizes by inspecting janino + * private fields. Also, this method updates the metrics information. */ private def updateAndGetCompilationStats(evaluator: ClassBodyEvaluator): Seq[(String, Int)] = { // First retrieve the generated classes. @@ -1130,28 +1131,28 @@ object CodeGenerator extends Logging { } // Then walk the classes to get at the method bytecode. - val methodsToByteCodeSize = mutable.ArrayBuffer[(String, Int)]() val codeAttr = Utils.classForName("org.codehaus.janino.util.ClassFile$CodeAttribute") val codeAttrField = codeAttr.getDeclaredField("code") codeAttrField.setAccessible(true) - classes.foreach { case (_, classBytes) => + val methodsToByteCodeSize = classes.flatMap { case (_, classBytes) => CodegenMetrics.METRIC_GENERATED_CLASS_BYTECODE_SIZE.update(classBytes.length) try { val cf = new ClassFile(new ByteArrayInputStream(classBytes)) - cf.methodInfos.asScala.foreach { method => - method.getAttributes().foreach { a => - if (a.getClass.getName == codeAttr.getName) { - val byteCodeSize = codeAttrField.get(a).asInstanceOf[Array[Byte]].length - CodegenMetrics.METRIC_GENERATED_METHOD_BYTECODE_SIZE.update(byteCodeSize) - methodsToByteCodeSize.append((method.getName, byteCodeSize)) - } + val stats = cf.methodInfos.asScala.flatMap { method => + method.getAttributes().filter(_.getClass.getName == codeAttr.getName).map { a => + val byteCodeSize = codeAttrField.get(a).asInstanceOf[Array[Byte]].length + CodegenMetrics.METRIC_GENERATED_METHOD_BYTECODE_SIZE.update(byteCodeSize) + (method.getName, byteCodeSize) } } + Some(stats) } catch { case NonFatal(e) => logWarning("Error calculating stats of compiled class.", e) + None } - } + }.flatten + methodsToByteCodeSize.toSeq } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 1a73d168b9b6..66ab7328a537 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -30,6 +30,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.network.util.ByteUnit import org.apache.spark.sql.catalyst.analysis.Resolver +import org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter //////////////////////////////////////////////////////////////////////////////////////////////////// @@ -580,10 +581,22 @@ object SQLConf { .doc("The maximum lines of a single Java function generated by whole-stage codegen. " + "When the generated function exceeds this threshold, " + "the whole-stage codegen is deactivated for this subtree of the current query plan. " + - "The default value 4000 is the max length of byte code JIT supported " + - "for a single function(8000) divided by 2.") + s"The default value ${CodeGenerator.DEFAULT_OPENJDK_JVM_HUGE_METHOD_LIMIT / 2} is " + + "the max length of byte code JIT supported for a single function" + + s"(${CodeGenerator.DEFAULT_OPENJDK_JVM_HUGE_METHOD_LIMIT}}) divided by 2.") .intConf - .createWithDefault(4000) + .createOptional + + val WHOLESTAGE_HUGE_METHOD_LIMIT = buildConf("spark.sql.codegen.hugeMethodLimit") + .internal() + .doc("The bytecode size of a single compiled Java function generated by whole-stage codegen." + + "When the compiled function exceeds this threshold, " + + "the whole-stage codegen is deactivated for this subtree of the current query plan. " + + s"The default value is ${CodeGenerator.DEFAULT_OPENJDK_JVM_HUGE_METHOD_LIMIT} and " + + "this is a limit in the OpenJDK JVM implementation." + ) + .intConf + .createWithDefault(CodeGenerator.DEFAULT_OPENJDK_JVM_HUGE_METHOD_LIMIT) val FILES_MAX_PARTITION_BYTES = buildConf("spark.sql.files.maxPartitionBytes") .doc("The maximum number of bytes to pack into a single partition when reading files.") @@ -1059,6 +1072,9 @@ class SQLConf extends Serializable with Logging { def loggingMaxLinesForCodegen: Int = getConf(CODEGEN_LOGGING_MAX_LINES) def maxLinesPerFunction: Int = getConf(WHOLESTAGE_MAX_LINES_PER_FUNCTION) + .getOrElse(getConf(WHOLESTAGE_HUGE_METHOD_LIMIT) / 2) + + def hugeMethodLimit: Int = getConf(WHOLESTAGE_HUGE_METHOD_LIMIT) def tableRelationCacheSize: Int = getConf(StaticSQLConf.FILESOURCE_TABLE_RELATION_CACHE_SIZE) diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql index 58c7830f77dc..41a0f8e28857 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql @@ -32,7 +32,7 @@ SELECT a + 1 + 1, COUNT(b) FROM testData GROUP BY a + 1; -- Aggregate with nulls. -- -- In SPARK-21871, we added code to check the bytecode size of gen'd methods. If the size --- goes over `HugeMethodLimit`, Spark fails to compile the methods and the execution also fails +-- goes over `hugeMethodLimit`, Spark fails to compile the methods and the execution also fails -- in a test mode. So, we explicitly turn off whole-stage codegen here. -- This guard can be removed if this issue fixed. SET spark.sql.codegen.wholeStage=false; diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index 0643f6dfb952..8f6b3eb675bb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -433,7 +433,7 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { test("zero moments") { // In SPARK-21871, we added code to check the bytecode size of gen'd methods. If the size - // goes over `HugeMethodLimit`, Spark fails to compile the methods and the execution also fails + // goes over `hugeMethodLimit`, Spark fails to compile the methods and the execution also fails // in a test mode. So, we explicitly turn off whole-stage codegen here. // This guard can be removed if this issue fixed. withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala index 195e3f4af17b..8d852b51d6c5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala @@ -97,7 +97,7 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext with B test("sliding window grouping") { // In SPARK-21871, we added code to check the bytecode size of gen'd methods. If the size - // goes over `HugeMethodLimit`, Spark fails to compile the methods and the execution also fails + // goes over `hugeMethodLimit`, Spark fails to compile the methods and the execution also fails // in a test mode. So, we explicitly turn off whole-stage codegen here. // This guard can be removed if this issue fixed. withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { @@ -130,7 +130,7 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext with B test("sliding window projection") { // In SPARK-21871, we added code to check the bytecode size of gen'd methods. If the size - // goes over `HugeMethodLimit`, Spark fails to compile the methods and the execution also fails + // goes over `hugeMethodLimit`, Spark fails to compile the methods and the execution also fails // in a test mode. So, we explicitly turn off whole-stage codegen here. // This guard can be removed if this issue fixed. withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { @@ -242,7 +242,7 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext with B test("millisecond precision sliding windows") { // In SPARK-21871, we added code to check the bytecode size of gen'd methods. If the size - // goes over `HugeMethodLimit`, Spark fails to compile the methods and the execution also fails + // goes over `hugeMethodLimit`, Spark fails to compile the methods and the execution also fails // in a test mode. So, we explicitly turn off whole-stage codegen here. // This guard can be removed if this issue fixed. withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 5a21f674a228..23b9de3c631f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -2103,7 +2103,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-15327: fail to compile generated code with complex data structure") { // In SPARK-21871, we added code to check the bytecode size of gen'd methods. If the size - // goes over `HugeMethodLimit`, Spark fails to compile the methods and the execution also fails + // goes over `hugeMethodLimit`, Spark fails to compile the methods and the execution also fails // in a test mode. So, we explicitly turn off whole-stage codegen here. // This guard can be removed if this issue fixed. withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index 9a0d75fc7bbe..64c75807434f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -207,15 +207,13 @@ class WholeStageCodegenSuite extends SparkPlanTest with SharedSQLContext { } } - test("SPARK-21871 turn off whole-stage codegen if bytecode size goes over HugeMethodLimit") { + test("SPARK-21871 turn off whole-stage codegen if bytecode size goes over hugeMethodLimit") { withSQLConf(SQLConf.WHOLESTAGE_MAX_LINES_PER_FUNCTION.key -> Int.MaxValue.toString) { val (_, code) = genGroupByCodeGenContext(20) val errMsg = intercept[IllegalArgumentException] { CodeGenerator.compile(code) }.getMessage - assert(errMsg.contains("the size of GeneratedClass.agg_doAggregateWithKeys is 9182 and " + - "this value goes over the HugeMethodLimit 8000 (JVM doesn't compile methods " + - "larger than this limit)")) + assert(errMsg.contains("this value went over the limit `spark.sql.codegen.hugeMethodLimit`")) } } } From 954bb947828cb612e0b5fed9989fefa3069bfda5 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Fri, 1 Sep 2017 17:37:56 +0900 Subject: [PATCH 03/14] Set WHOLESTAGE_HUGE_METHOD_LIMIT explicitly in some existing tests --- .../expressions/codegen/CodeGenerator.scala | 2 +- .../expressions/CodeGenerationSuite.scala | 59 +++++----- .../catalyst/expressions/OrderingSuite.scala | 20 ++-- .../codegen/GeneratedProjectionSuite.scala | 102 +++++++++--------- .../resources/sql-tests/inputs/group-by.sql | 2 +- .../spark/sql/DataFrameAggregateSuite.scala | 2 +- .../sql/DataFrameTimeWindowingSuite.scala | 6 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 2 +- 8 files changed, 107 insertions(+), 88 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index ef765a0915e7..e40f27057179 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -1097,7 +1097,7 @@ object CodeGenerator extends Logging { throw new CompileException(msg, e.getLocation) } - // Check if compiled code has a too large function. + // Check if compiled code has a too large function methodsToByteCodeSize.foreach { case (name, byteCodeSize) => if (byteCodeSize > SQLConf.get.hugeMethodLimit) { val clazzName = evaluator.getClazz.getSimpleName diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala index 7ea0bec14548..1b7c1694e35e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala @@ -19,14 +19,15 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.Timestamp -import org.apache.spark.SparkFunSuite import org.apache.spark.metrics.source.CodegenMetrics import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.objects.{AssertNotNull, CreateExternalRow, GetExternalRowField, ValidateExternalType} +import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.ThreadUtils @@ -34,7 +35,7 @@ import org.apache.spark.util.ThreadUtils /** * Additional tests for code generation. */ -class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { +class CodeGenerationSuite extends PlanTest with ExpressionEvalHelper { test("multithreaded eval") { import scala.concurrent._ @@ -98,19 +99,23 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { } test("SPARK-18091: split large if expressions into blocks due to JVM code size limit") { - var strExpr: Expression = Literal("abc") - for (_ <- 1 to 150) { - strExpr = Decode(Encode(strExpr, "utf-8"), "utf-8") - } + // Set the max value at `WHOLESTAGE_HUGE_METHOD_LIMIT` to compile gen'd code by janino + withSQLConf(SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key -> Int.MaxValue.toString) { + var strExpr: Expression = Literal("abc") + for (_ <- 1 to 150) { + strExpr = Decode(Encode(strExpr, "utf-8"), "utf-8") + } - val expressions = Seq(If(EqualTo(strExpr, strExpr), strExpr, strExpr)) - val plan = GenerateMutableProjection.generate(expressions) - val actual = plan(null).toSeq(expressions.map(_.dataType)) - assert(actual.length == 1) - val expected = UTF8String.fromString("abc") + val expressions = Seq(If(EqualTo(strExpr, strExpr), strExpr, strExpr)) + val plan = GenerateMutableProjection.generate(expressions) + val actual = plan(null).toSeq(expressions.map(_.dataType)) + assert(actual.length == 1) + val expected = UTF8String.fromString("abc") - if (!checkResult(actual.head, expected, expressions.head.dataType)) { - fail(s"Incorrect Evaluation: expressions: $expressions, actual: $actual, expected: $expected") + if (!checkResult(actual.head, expected, expressions.head.dataType)) { + fail(s"Incorrect Evaluation: expressions: $expressions, actual: $actual, " + + s"expected: $expected") + } } } @@ -128,18 +133,22 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { } test("SPARK-14793: split wide map creation into blocks due to JVM code size limit") { - val length = 5000 - val expressions = Seq(CreateMap( - List.fill(length)(EqualTo(Literal(1), Literal(1))).zipWithIndex.flatMap { - case (expr, i) => Seq(Literal(i), expr) - })) - val plan = GenerateMutableProjection.generate(expressions) - val actual = plan(new GenericInternalRow(length)).toSeq(expressions.map(_.dataType)) - assert(actual.length == 1) - val expected = ArrayBasedMapData((0 until length).toArray, Array.fill(length)(true)) - - if (!checkResult(actual.head, expected, expressions.head.dataType)) { - fail(s"Incorrect Evaluation: expressions: $expressions, actual: $actual, expected: $expected") + // Set the max value at `WHOLESTAGE_HUGE_METHOD_LIMIT` to compile gen'd code by janino + withSQLConf(SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key -> Int.MaxValue.toString) { + val length = 5000 + val expressions = Seq(CreateMap( + List.fill(length)(EqualTo(Literal(1), Literal(1))).zipWithIndex.flatMap { + case (expr, i) => Seq(Literal(i), expr) + })) + val plan = GenerateMutableProjection.generate(expressions) + val actual = plan(new GenericInternalRow(length)).toSeq(expressions.map(_.dataType)) + assert(actual.length == 1) + val expected = ArrayBasedMapData((0 until length).toArray, Array.fill(length)(true)) + + if (!checkResult(actual.head, expected, expressions.head.dataType)) { + fail(s"Incorrect Evaluation: expressions: $expressions, actual: $actual, " + + "expected: $expected") + } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/OrderingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/OrderingSuite.scala index aa61ba2bff2b..9b4004043872 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/OrderingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/OrderingSuite.scala @@ -19,15 +19,17 @@ package org.apache.spark.sql.catalyst.expressions import scala.math._ -import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.SparkConf import org.apache.spark.serializer.KryoSerializer import org.apache.spark.sql.{RandomDataGenerator, Row} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateOrdering, LazilyGeneratedOrdering} +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -class OrderingSuite extends SparkFunSuite with ExpressionEvalHelper { +class OrderingSuite extends PlanTest with ExpressionEvalHelper { def compareArrays(a: Seq[Any], b: Seq[Any], expected: Int): Unit = { test(s"compare two arrays: a = $a, b = $b") { @@ -129,13 +131,17 @@ class OrderingSuite extends SparkFunSuite with ExpressionEvalHelper { } test("SPARK-16845: GeneratedClass$SpecificOrdering grows beyond 64 KB") { - val sortOrder = Literal("abc").asc + // To just check if janino can compile gen'd code, we set the max value at + // `WHOLESTAGE_HUGE_METHOD_LIMIT` explicitly. + withSQLConf(SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key -> Int.MaxValue.toString) { + val sortOrder = Literal("abc").asc - // this is passing prior to SPARK-16845, and it should also be passing after SPARK-16845 - GenerateOrdering.generate(Array.fill(40)(sortOrder)) + // this is passing prior to SPARK-16845, and it should also be passing after SPARK-16845 + GenerateOrdering.generate(Array.fill(40)(sortOrder)) - // verify that we can support up to 5000 ordering comparisons, which should be sufficient - GenerateOrdering.generate(Array.fill(5000)(sortOrder)) + // verify that we can support up to 5000 ordering comparisons, which should be sufficient + GenerateOrdering.generate(Array.fill(5000)(sortOrder)) + } } test("SPARK-21344: BinaryType comparison does signed byte array comparison") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala index 0cd0d8859145..bfe302a2fc1d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala @@ -19,17 +19,18 @@ package org.apache.spark.sql.catalyst.expressions.codegen import java.nio.charset.StandardCharsets -import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.util.GenericArrayData +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String /** * A test suite for generated projections */ -class GeneratedProjectionSuite extends SparkFunSuite { +class GeneratedProjectionSuite extends PlanTest { test("generated projections on wider table") { val N = 1000 @@ -83,54 +84,57 @@ class GeneratedProjectionSuite extends SparkFunSuite { } test("SPARK-18016: generated projections on wider table requiring class-splitting") { - val N = 4000 - 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) - - // test generated UnsafeProjection - val unsafeProj = UnsafeProjection.create(nestedSchema) - val unsafe: UnsafeRow = unsafeProj(nested) - (0 until N).foreach { i => - val s = UTF8String.fromString(i.toString) - assert(i === unsafe.getInt(i + 2)) - assert(s === unsafe.getUTF8String(i + 2 + N)) - assert(i === unsafe.getStruct(0, N * 2).getInt(i)) - assert(s === unsafe.getStruct(0, N * 2).getUTF8String(i + N)) - assert(i === unsafe.getStruct(1, N * 2).getInt(i)) - assert(s === unsafe.getStruct(1, N * 2).getUTF8String(i + N)) - } - - // test generated SafeProjection - val safeProj = FromUnsafeProjection(nestedSchema) - val result = safeProj(unsafe) - // Can't compare GenericInternalRow with JoinedRow directly - (0 until N).foreach { i => - val s = UTF8String.fromString(i.toString) - assert(i === result.getInt(i + 2)) - assert(s === result.getUTF8String(i + 2 + N)) - assert(i === result.getStruct(0, N * 2).getInt(i)) - assert(s === result.getStruct(0, N * 2).getUTF8String(i + N)) - assert(i === result.getStruct(1, N * 2).getInt(i)) - assert(s === result.getStruct(1, N * 2).getUTF8String(i + N)) - } - - // test generated MutableProjection - val exprs = nestedSchema.fields.zipWithIndex.map { case (f, i) => - BoundReference(i, f.dataType, true) + // Set the max value at `WHOLESTAGE_HUGE_METHOD_LIMIT` to compile gen'd code by janino + withSQLConf(SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key -> Int.MaxValue.toString) { + val N = 4000 + 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) + + // test generated UnsafeProjection + val unsafeProj = UnsafeProjection.create(nestedSchema) + val unsafe: UnsafeRow = unsafeProj(nested) + (0 until N).foreach { i => + val s = UTF8String.fromString(i.toString) + assert(i === unsafe.getInt(i + 2)) + assert(s === unsafe.getUTF8String(i + 2 + N)) + assert(i === unsafe.getStruct(0, N * 2).getInt(i)) + assert(s === unsafe.getStruct(0, N * 2).getUTF8String(i + N)) + assert(i === unsafe.getStruct(1, N * 2).getInt(i)) + assert(s === unsafe.getStruct(1, N * 2).getUTF8String(i + N)) + } + + // test generated SafeProjection + val safeProj = FromUnsafeProjection(nestedSchema) + val result = safeProj(unsafe) + // Can't compare GenericInternalRow with JoinedRow directly + (0 until N).foreach { i => + val s = UTF8String.fromString(i.toString) + assert(i === result.getInt(i + 2)) + assert(s === result.getUTF8String(i + 2 + N)) + assert(i === result.getStruct(0, N * 2).getInt(i)) + assert(s === result.getStruct(0, N * 2).getUTF8String(i + N)) + assert(i === result.getStruct(1, N * 2).getInt(i)) + assert(s === result.getStruct(1, N * 2).getUTF8String(i + N)) + } + + // test generated MutableProjection + val exprs = nestedSchema.fields.zipWithIndex.map { case (f, i) => + BoundReference(i, f.dataType, true) + } + val mutableProj = GenerateMutableProjection.generate(exprs) + val row1 = mutableProj(result) + assert(result === row1) + val row2 = mutableProj(result) + assert(result === row2) } - val mutableProj = GenerateMutableProjection.generate(exprs) - val row1 = mutableProj(result) - assert(result === row1) - val row2 = mutableProj(result) - assert(result === row2) } test("generated unsafe projection with array of binary") { diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql index 41a0f8e28857..e79cc49c0fb7 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql @@ -31,7 +31,7 @@ SELECT a + 1 + 1, COUNT(b) FROM testData GROUP BY a + 1; -- Aggregate with nulls. -- --- In SPARK-21871, we added code to check the bytecode size of gen'd methods. If the size +-- In SPARK-21871, we added code to check the actual bytecode size of gen'd methods. If the size -- goes over `hugeMethodLimit`, Spark fails to compile the methods and the execution also fails -- in a test mode. So, we explicitly turn off whole-stage codegen here. -- This guard can be removed if this issue fixed. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index 8f6b3eb675bb..9ad6bc7d9ed9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -432,7 +432,7 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { } test("zero moments") { - // In SPARK-21871, we added code to check the bytecode size of gen'd methods. If the size + // In SPARK-21871, we added code to check the actual bytecode size of gen'd methods. If the size // goes over `hugeMethodLimit`, Spark fails to compile the methods and the execution also fails // in a test mode. So, we explicitly turn off whole-stage codegen here. // This guard can be removed if this issue fixed. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala index 8d852b51d6c5..640a29b9aa8b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala @@ -96,7 +96,7 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext with B } test("sliding window grouping") { - // In SPARK-21871, we added code to check the bytecode size of gen'd methods. If the size + // In SPARK-21871, we added code to check the actual bytecode size of gen'd methods. If the size // goes over `hugeMethodLimit`, Spark fails to compile the methods and the execution also fails // in a test mode. So, we explicitly turn off whole-stage codegen here. // This guard can be removed if this issue fixed. @@ -129,7 +129,7 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext with B } test("sliding window projection") { - // In SPARK-21871, we added code to check the bytecode size of gen'd methods. If the size + // In SPARK-21871, we added code to check the actual bytecode size of gen'd methods. If the size // goes over `hugeMethodLimit`, Spark fails to compile the methods and the execution also fails // in a test mode. So, we explicitly turn off whole-stage codegen here. // This guard can be removed if this issue fixed. @@ -241,7 +241,7 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext with B } test("millisecond precision sliding windows") { - // In SPARK-21871, we added code to check the bytecode size of gen'd methods. If the size + // In SPARK-21871, we added code to check the actual bytecode size of gen'd methods. If the size // goes over `hugeMethodLimit`, Spark fails to compile the methods and the execution also fails // in a test mode. So, we explicitly turn off whole-stage codegen here. // This guard can be removed if this issue fixed. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 23b9de3c631f..c054437b6021 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -2102,7 +2102,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-15327: fail to compile generated code with complex data structure") { - // In SPARK-21871, we added code to check the bytecode size of gen'd methods. If the size + // In SPARK-21871, we added code to check the actual bytecode size of gen'd methods. If the size // goes over `hugeMethodLimit`, Spark fails to compile the methods and the execution also fails // in a test mode. So, we explicitly turn off whole-stage codegen here. // This guard can be removed if this issue fixed. From 7c398e76bacb4534688cfd3a45ffd64a5e5b8d6f Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Sat, 2 Sep 2017 08:04:14 +0900 Subject: [PATCH 04/14] Add more fixes --- .../sql/catalyst/expressions/codegen/CodeGenerator.scala | 2 +- .../apache/spark/sql/execution/WholeStageCodegenSuite.scala | 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index e40f27057179..82605b37bf8f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -1021,7 +1021,7 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin object CodeGenerator extends Logging { - // This is the value of `HugeMethodLimit` in the OpenJDK JVM settings. + // This is the value of HugeMethodLimit in the OpenJDK JVM settings val DEFAULT_OPENJDK_JVM_HUGE_METHOD_LIMIT = 8000 /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index 64c75807434f..fd2aacc0350a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -208,7 +208,9 @@ class WholeStageCodegenSuite extends SparkPlanTest with SharedSQLContext { } test("SPARK-21871 turn off whole-stage codegen if bytecode size goes over hugeMethodLimit") { - withSQLConf(SQLConf.WHOLESTAGE_MAX_LINES_PER_FUNCTION.key -> Int.MaxValue.toString) { + withSQLConf( + SQLConf.WHOLESTAGE_MAX_LINES_PER_FUNCTION.key -> Int.MaxValue.toString, + SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key -> "8000") { val (_, code) = genGroupByCodeGenContext(20) val errMsg = intercept[IllegalArgumentException] { CodeGenerator.compile(code) From 897c789528a015f34a26c6ebae6deb61305ba38f Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Thu, 7 Sep 2017 15:18:26 +0900 Subject: [PATCH 05/14] Rename the parameter --- .../main/scala/org/apache/spark/sql/internal/SQLConf.scala | 6 +++--- .../sql/catalyst/expressions/CodeGenerationSuite.scala | 4 ++-- .../spark/sql/catalyst/expressions/OrderingSuite.scala | 2 +- .../expressions/codegen/GeneratedProjectionSuite.scala | 2 +- .../apache/spark/sql/execution/WholeStageCodegenSuite.scala | 2 +- 5 files changed, 8 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 66ab7328a537..6b8572ac406f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -587,7 +587,7 @@ object SQLConf { .intConf .createOptional - val WHOLESTAGE_HUGE_METHOD_LIMIT = buildConf("spark.sql.codegen.hugeMethodLimit") + val CODEGEN_HUGE_METHOD_LIMIT = buildConf("spark.sql.codegen.hugeMethodLimit") .internal() .doc("The bytecode size of a single compiled Java function generated by whole-stage codegen." + "When the compiled function exceeds this threshold, " + @@ -1072,9 +1072,9 @@ class SQLConf extends Serializable with Logging { def loggingMaxLinesForCodegen: Int = getConf(CODEGEN_LOGGING_MAX_LINES) def maxLinesPerFunction: Int = getConf(WHOLESTAGE_MAX_LINES_PER_FUNCTION) - .getOrElse(getConf(WHOLESTAGE_HUGE_METHOD_LIMIT) / 2) + .getOrElse(getConf(CODEGEN_HUGE_METHOD_LIMIT) / 2) - def hugeMethodLimit: Int = getConf(WHOLESTAGE_HUGE_METHOD_LIMIT) + def hugeMethodLimit: Int = getConf(CODEGEN_HUGE_METHOD_LIMIT) def tableRelationCacheSize: Int = getConf(StaticSQLConf.FILESOURCE_TABLE_RELATION_CACHE_SIZE) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala index 1b7c1694e35e..4a3341e169be 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala @@ -100,7 +100,7 @@ class CodeGenerationSuite extends PlanTest with ExpressionEvalHelper { test("SPARK-18091: split large if expressions into blocks due to JVM code size limit") { // Set the max value at `WHOLESTAGE_HUGE_METHOD_LIMIT` to compile gen'd code by janino - withSQLConf(SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key -> Int.MaxValue.toString) { + withSQLConf(SQLConf.CODEGEN_HUGE_METHOD_LIMIT.key -> Int.MaxValue.toString) { var strExpr: Expression = Literal("abc") for (_ <- 1 to 150) { strExpr = Decode(Encode(strExpr, "utf-8"), "utf-8") @@ -134,7 +134,7 @@ class CodeGenerationSuite extends PlanTest with ExpressionEvalHelper { test("SPARK-14793: split wide map creation into blocks due to JVM code size limit") { // Set the max value at `WHOLESTAGE_HUGE_METHOD_LIMIT` to compile gen'd code by janino - withSQLConf(SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key -> Int.MaxValue.toString) { + withSQLConf(SQLConf.CODEGEN_HUGE_METHOD_LIMIT.key -> Int.MaxValue.toString) { val length = 5000 val expressions = Seq(CreateMap( List.fill(length)(EqualTo(Literal(1), Literal(1))).zipWithIndex.flatMap { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/OrderingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/OrderingSuite.scala index 9b4004043872..2c9ac69b65e7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/OrderingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/OrderingSuite.scala @@ -133,7 +133,7 @@ class OrderingSuite extends PlanTest with ExpressionEvalHelper { test("SPARK-16845: GeneratedClass$SpecificOrdering grows beyond 64 KB") { // To just check if janino can compile gen'd code, we set the max value at // `WHOLESTAGE_HUGE_METHOD_LIMIT` explicitly. - withSQLConf(SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key -> Int.MaxValue.toString) { + withSQLConf(SQLConf.CODEGEN_HUGE_METHOD_LIMIT.key -> Int.MaxValue.toString) { val sortOrder = Literal("abc").asc // this is passing prior to SPARK-16845, and it should also be passing after SPARK-16845 diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala index bfe302a2fc1d..a19c92e4735a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala @@ -85,7 +85,7 @@ class GeneratedProjectionSuite extends PlanTest { test("SPARK-18016: generated projections on wider table requiring class-splitting") { // Set the max value at `WHOLESTAGE_HUGE_METHOD_LIMIT` to compile gen'd code by janino - withSQLConf(SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key -> Int.MaxValue.toString) { + withSQLConf(SQLConf.CODEGEN_HUGE_METHOD_LIMIT.key -> Int.MaxValue.toString) { val N = 4000 val wideRow1 = new GenericInternalRow((0 until N).toArray[Any]) val schema1 = StructType((1 to N).map(i => StructField("", IntegerType))) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index fd2aacc0350a..eca0d2ef40af 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -210,7 +210,7 @@ class WholeStageCodegenSuite extends SparkPlanTest with SharedSQLContext { test("SPARK-21871 turn off whole-stage codegen if bytecode size goes over hugeMethodLimit") { withSQLConf( SQLConf.WHOLESTAGE_MAX_LINES_PER_FUNCTION.key -> Int.MaxValue.toString, - SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key -> "8000") { + SQLConf.CODEGEN_HUGE_METHOD_LIMIT.key -> "8000") { val (_, code) = genGroupByCodeGenContext(20) val errMsg = intercept[IllegalArgumentException] { CodeGenerator.compile(code) From e5fde065c547fab3e30f5908b246243ab6577136 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Sat, 30 Sep 2017 16:43:45 +0900 Subject: [PATCH 06/14] Use CompileException --- .../sql/catalyst/expressions/codegen/CodeGenerator.scala | 2 +- .../spark/sql/execution/WholeStageCodegenSuite.scala | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 82605b37bf8f..6b84cb8f8950 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -1108,7 +1108,7 @@ object CodeGenerator extends Logging { logError(msg) val maxLines = SQLConf.get.loggingMaxLinesForCodegen logInfo(s"\n${CodeFormatter.format(code, maxLines)}") - throw new IllegalArgumentException(msg) + throw new CompileException(msg, null) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index eca0d2ef40af..009d2dec8824 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -17,9 +17,9 @@ package org.apache.spark.sql.execution -import org.apache.spark.sql.{Column, Dataset, Row} -import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute -import org.apache.spark.sql.catalyst.expressions.{Add, Literal, Stack} +import java.util.concurrent.ExecutionException + +import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.codegen.{CodeAndComment, CodegenContext, CodeGenerator} import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec @@ -212,7 +212,7 @@ class WholeStageCodegenSuite extends SparkPlanTest with SharedSQLContext { SQLConf.WHOLESTAGE_MAX_LINES_PER_FUNCTION.key -> Int.MaxValue.toString, SQLConf.CODEGEN_HUGE_METHOD_LIMIT.key -> "8000") { val (_, code) = genGroupByCodeGenContext(20) - val errMsg = intercept[IllegalArgumentException] { + val errMsg = intercept[ExecutionException] { CodeGenerator.compile(code) }.getMessage assert(errMsg.contains("this value went over the limit `spark.sql.codegen.hugeMethodLimit`")) From d3c103311ab7d3a6692a59311f66bf7ee2da03a0 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Sun, 1 Oct 2017 10:45:38 +0900 Subject: [PATCH 07/14] Update doc --- .../main/scala/org/apache/spark/sql/internal/SQLConf.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 6b8572ac406f..2e6000a5543c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -589,9 +589,8 @@ object SQLConf { val CODEGEN_HUGE_METHOD_LIMIT = buildConf("spark.sql.codegen.hugeMethodLimit") .internal() - .doc("The bytecode size of a single compiled Java function generated by whole-stage codegen." + - "When the compiled function exceeds this threshold, " + - "the whole-stage codegen is deactivated for this subtree of the current query plan. " + + .doc("The maximum bytecode size of a single compiled Java function generated by codegen. " + + "When the compiled function exceeds this threshold, the codegen is deactivated. " + s"The default value is ${CodeGenerator.DEFAULT_OPENJDK_JVM_HUGE_METHOD_LIMIT} and " + "this is a limit in the OpenJDK JVM implementation." ) From 53abab9b40a1e144a6404ce0db792c372191b3d6 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Mon, 2 Oct 2017 00:02:08 +0900 Subject: [PATCH 08/14] Fix workaround --- .../resources/sql-tests/inputs/group-by.sql | 8 ++++---- .../sql-tests/results/group-by.sql.out | 8 ++++---- .../spark/sql/DataFrameAggregateSuite.scala | 6 +++--- .../sql/DataFrameTimeWindowingSuite.scala | 18 +++++++++--------- .../org/apache/spark/sql/SQLQuerySuite.scala | 6 +++--- 5 files changed, 23 insertions(+), 23 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql index e79cc49c0fb7..bcb45ccbf4d6 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql @@ -33,12 +33,12 @@ SELECT a + 1 + 1, COUNT(b) FROM testData GROUP BY a + 1; -- -- In SPARK-21871, we added code to check the actual bytecode size of gen'd methods. If the size -- goes over `hugeMethodLimit`, Spark fails to compile the methods and the execution also fails --- in a test mode. So, we explicitly turn off whole-stage codegen here. --- This guard can be removed if this issue fixed. -SET spark.sql.codegen.wholeStage=false; +-- in a test mode. So, we explicitly made this threshold higher here. +-- This workaround can be removed if this issue fixed. +SET spark.sql.codegen.hugeMethodLimit=16000; SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a), AVG(a), VARIANCE(a), STDDEV(a), SUM(a), COUNT(a) FROM testData; -SET spark.sql.codegen.wholeStage=true; +SET spark.sql.codegen.hugeMethodLimit=8000; -- Aggregate with foldable input and multiple distinct groups. SELECT COUNT(DISTINCT b), COUNT(DISTINCT b, c) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a; diff --git a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out index 11e8f9fbf988..2a02e67a06af 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out @@ -125,11 +125,11 @@ NULL 1 -- !query 13 -SET spark.sql.codegen.wholeStage=false +SET spark.sql.codegen.hugeMethodLimit=16000 -- !query 13 schema struct -- !query 13 output -spark.sql.codegen.wholeStage false +spark.sql.codegen.hugeMethodLimit 16000 -- !query 14 @@ -142,11 +142,11 @@ struct -- !query 15 output -spark.sql.codegen.wholeStage true +spark.sql.codegen.hugeMethodLimit 8000 -- !query 16 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index 9ad6bc7d9ed9..72d79c16282a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -434,9 +434,9 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { test("zero moments") { // In SPARK-21871, we added code to check the actual bytecode size of gen'd methods. If the size // goes over `hugeMethodLimit`, Spark fails to compile the methods and the execution also fails - // in a test mode. So, we explicitly turn off whole-stage codegen here. - // This guard can be removed if this issue fixed. - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { + // in a test mode. So, we explicitly made this threshold higher here. + // This workaround can be removed if this issue fixed. + withSQLConf(SQLConf.CODEGEN_HUGE_METHOD_LIMIT.key -> "16000") { val input = Seq((1, 2)).toDF("a", "b") checkAnswer( input.agg(stddev('a), stddev_samp('a), stddev_pop('a), variance('a), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala index 640a29b9aa8b..48159c4451af 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala @@ -98,9 +98,9 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext with B test("sliding window grouping") { // In SPARK-21871, we added code to check the actual bytecode size of gen'd methods. If the size // goes over `hugeMethodLimit`, Spark fails to compile the methods and the execution also fails - // in a test mode. So, we explicitly turn off whole-stage codegen here. - // This guard can be removed if this issue fixed. - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { + // in a test mode. So, we explicitly made this threshold higher here. + // This workaround can be removed if this issue fixed. + withSQLConf(SQLConf.CODEGEN_HUGE_METHOD_LIMIT.key -> "16000") { val df = Seq( ("2016-03-27 19:39:34", 1, "a"), ("2016-03-27 19:39:56", 2, "a"), @@ -131,9 +131,9 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext with B test("sliding window projection") { // In SPARK-21871, we added code to check the actual bytecode size of gen'd methods. If the size // goes over `hugeMethodLimit`, Spark fails to compile the methods and the execution also fails - // in a test mode. So, we explicitly turn off whole-stage codegen here. - // This guard can be removed if this issue fixed. - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { + // in a test mode. So, we explicitly made this threshold higher here. + // This workaround can be removed if this issue fixed. + withSQLConf(SQLConf.CODEGEN_HUGE_METHOD_LIMIT.key -> "16000") { val df = Seq( ("2016-03-27 19:39:34", 1, "a"), ("2016-03-27 19:39:56", 2, "a"), @@ -243,9 +243,9 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext with B test("millisecond precision sliding windows") { // In SPARK-21871, we added code to check the actual bytecode size of gen'd methods. If the size // goes over `hugeMethodLimit`, Spark fails to compile the methods and the execution also fails - // in a test mode. So, we explicitly turn off whole-stage codegen here. - // This guard can be removed if this issue fixed. - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { + // in a test mode. So, we explicitly made this threshold higher here. + // This workaround can be removed if this issue fixed. + withSQLConf(SQLConf.CODEGEN_HUGE_METHOD_LIMIT.key -> "16000") { val df = Seq( ("2016-03-27 09:00:00.41", 3), ("2016-03-27 09:00:00.62", 6), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index c054437b6021..da6946cd705f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -2104,9 +2104,9 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-15327: fail to compile generated code with complex data structure") { // In SPARK-21871, we added code to check the actual bytecode size of gen'd methods. If the size // goes over `hugeMethodLimit`, Spark fails to compile the methods and the execution also fails - // in a test mode. So, we explicitly turn off whole-stage codegen here. - // This guard can be removed if this issue fixed. - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { + // in a test mode. So, we explicitly made this threshold higher here. + // This workaround can be removed if this issue fixed. + withSQLConf(SQLConf.CODEGEN_HUGE_METHOD_LIMIT.key -> "16000") { withTempDir { dir => val json = """ From 2d834eda8d6bba6030675d052f2aee9d75f25b56 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Mon, 2 Oct 2017 00:40:19 +0900 Subject: [PATCH 09/14] Drop spark.sql.codegen.maxLinesPerFunction --- .../expressions/codegen/CodeFormatter.scala | 8 ----- .../expressions/codegen/CodeGenerator.scala | 14 -------- .../apache/spark/sql/internal/SQLConf.scala | 14 -------- .../codegen/CodeFormatterSuite.scala | 32 ------------------- .../sql/execution/WholeStageCodegenExec.scala | 8 ----- .../execution/WholeStageCodegenSuite.scala | 32 +------------------ .../benchmark/AggregateBenchmark.scala | 30 +++++++---------- 7 files changed, 13 insertions(+), 125 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala index 7b398f424cea..60e600d8dbd8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala @@ -89,14 +89,6 @@ object CodeFormatter { } new CodeAndComment(code.result().trim(), map) } - - def stripExtraNewLinesAndComments(input: String): String = { - val commentReg = - ("""([ |\t]*?\/\*[\s|\S]*?\*\/[ |\t]*?)|""" + // strip /*comment*/ - """([ |\t]*?\/\/[\s\S]*?\n)""").r // strip //comment - val codeWithoutComment = commentReg.replaceAllIn(input, "") - codeWithoutComment.replaceAll("""\n\s*\n""", "\n") // strip ExtraNewLines - } } private class CodeFormatter { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 6b84cb8f8950..1dec443d43f6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -373,20 +373,6 @@ class CodegenContext { */ private val placeHolderToComments = new mutable.HashMap[String, String] - /** - * It will count the lines of every Java function generated by whole-stage codegen, - * if there is a function of length greater than spark.sql.codegen.maxLinesPerFunction, - * it will return true. - */ - def isTooLongGeneratedFunction: Boolean = { - classFunctions.values.exists { _.values.exists { - code => - val codeWithoutComments = CodeFormatter.stripExtraNewLinesAndComments(code) - codeWithoutComments.count(_ == '\n') > SQLConf.get.maxLinesPerFunction - } - } - } - /** * Returns a term name that is unique within this instance of a `CodegenContext`. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 2e6000a5543c..6e5cbd0fbd31 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -576,17 +576,6 @@ object SQLConf { "disable logging or -1 to apply no limit.") .createWithDefault(1000) - val WHOLESTAGE_MAX_LINES_PER_FUNCTION = buildConf("spark.sql.codegen.maxLinesPerFunction") - .internal() - .doc("The maximum lines of a single Java function generated by whole-stage codegen. " + - "When the generated function exceeds this threshold, " + - "the whole-stage codegen is deactivated for this subtree of the current query plan. " + - s"The default value ${CodeGenerator.DEFAULT_OPENJDK_JVM_HUGE_METHOD_LIMIT / 2} is " + - "the max length of byte code JIT supported for a single function" + - s"(${CodeGenerator.DEFAULT_OPENJDK_JVM_HUGE_METHOD_LIMIT}}) divided by 2.") - .intConf - .createOptional - val CODEGEN_HUGE_METHOD_LIMIT = buildConf("spark.sql.codegen.hugeMethodLimit") .internal() .doc("The maximum bytecode size of a single compiled Java function generated by codegen. " + @@ -1070,9 +1059,6 @@ class SQLConf extends Serializable with Logging { def loggingMaxLinesForCodegen: Int = getConf(CODEGEN_LOGGING_MAX_LINES) - def maxLinesPerFunction: Int = getConf(WHOLESTAGE_MAX_LINES_PER_FUNCTION) - .getOrElse(getConf(CODEGEN_HUGE_METHOD_LIMIT) / 2) - def hugeMethodLimit: Int = getConf(CODEGEN_HUGE_METHOD_LIMIT) def tableRelationCacheSize: Int = diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatterSuite.scala index a0f1a64b0ab0..9d0a41661bea 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatterSuite.scala @@ -53,38 +53,6 @@ class CodeFormatterSuite extends SparkFunSuite { assert(reducedCode.body === "/*project_c4*/") } - test("removing extra new lines and comments") { - val code = - """ - |/* - | * multi - | * line - | * comments - | */ - | - |public function() { - |/*comment*/ - | /*comment_with_space*/ - |code_body - |//comment - |code_body - | //comment_with_space - | - |code_body - |} - """.stripMargin - - val reducedCode = CodeFormatter.stripExtraNewLinesAndComments(code) - assert(reducedCode === - """ - |public function() { - |code_body - |code_body - |code_body - |} - """.stripMargin) - } - testCase("basic example") { """ |class A { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 268ccfa4edfa..661f5b3774df 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -380,14 +380,6 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co override def doExecute(): RDD[InternalRow] = { val (ctx, cleanedSource) = doCodeGen() - if (ctx.isTooLongGeneratedFunction) { - logWarning("Found too long generated codes and JIT optimization might not work, " + - "Whole-stage codegen disabled for this plan, " + - "You can change the config spark.sql.codegen.MaxFunctionLength " + - "to adjust the function length limit:\n " - + s"$treeString") - return child.execute() - } // try to compile and fallback if it failed try { CodeGenerator.compile(cleanedSource) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index 009d2dec8824..5226b76ba251 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -179,38 +179,8 @@ class WholeStageCodegenSuite extends SparkPlanTest with SharedSQLContext { wholeStageCodeGenExec.get.asInstanceOf[WholeStageCodegenExec].doCodeGen() } - test("SPARK-21603 check there is a too long generated function") { - withSQLConf(SQLConf.WHOLESTAGE_MAX_LINES_PER_FUNCTION.key -> "1500") { - val (ctx, _) = genGroupByCodeGenContext(30) - assert(ctx.isTooLongGeneratedFunction === true) - } - } - - test("SPARK-21603 check there is not a too long generated function") { - withSQLConf(SQLConf.WHOLESTAGE_MAX_LINES_PER_FUNCTION.key -> "1500") { - val (ctx, _) = genGroupByCodeGenContext(1) - assert(ctx.isTooLongGeneratedFunction === false) - } - } - - test("SPARK-21603 check there is not a too long generated function when threshold is Int.Max") { - withSQLConf(SQLConf.WHOLESTAGE_MAX_LINES_PER_FUNCTION.key -> Int.MaxValue.toString) { - val (ctx, _) = genGroupByCodeGenContext(30) - assert(ctx.isTooLongGeneratedFunction === false) - } - } - - test("SPARK-21603 check there is a too long generated function when threshold is 0") { - withSQLConf(SQLConf.WHOLESTAGE_MAX_LINES_PER_FUNCTION.key -> "0") { - val (ctx, _) = genGroupByCodeGenContext(1) - assert(ctx.isTooLongGeneratedFunction === true) - } - } - test("SPARK-21871 turn off whole-stage codegen if bytecode size goes over hugeMethodLimit") { - withSQLConf( - SQLConf.WHOLESTAGE_MAX_LINES_PER_FUNCTION.key -> Int.MaxValue.toString, - SQLConf.CODEGEN_HUGE_METHOD_LIMIT.key -> "8000") { + withSQLConf(SQLConf.CODEGEN_HUGE_METHOD_LIMIT.key -> "8000") { val (_, code) = genGroupByCodeGenContext(20) val errMsg = intercept[ExecutionException] { CodeGenerator.compile(code) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala index 691fa9ac5e1e..23b1db0f2257 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala @@ -301,10 +301,10 @@ class AggregateBenchmark extends BenchmarkBase { */ } - ignore("max function length of wholestagecodegen") { + ignore("max function length of codegen") { val N = 20 << 15 - val benchmark = new Benchmark("max function length of wholestagecodegen", N) + val benchmark = new Benchmark("max function length of codegen", N) def f(): Unit = sparkSession.range(N) .selectExpr( "id", @@ -333,33 +333,27 @@ class AggregateBenchmark extends BenchmarkBase { .sum() .collect() - benchmark.addCase(s"codegen = F") { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", "false") - f() - } - - benchmark.addCase(s"codegen = T maxLinesPerFunction = 10000") { iter => + benchmark.addCase(s"hugeMethodLimit = 8000") { iter => sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") - sparkSession.conf.set("spark.sql.codegen.maxLinesPerFunction", "10000") + sparkSession.conf.set("spark.sql.codegen.hugeMethodLimit", "8000") f() } - benchmark.addCase(s"codegen = T maxLinesPerFunction = 1500") { iter => + benchmark.addCase(s"hugeMethodLimit = 16000") { iter => sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") - sparkSession.conf.set("spark.sql.codegen.maxLinesPerFunction", "1500") + sparkSession.conf.set("spark.sql.codegen.hugeMethodLimit", "16000") f() } benchmark.run() /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_111-b14 on Windows 7 6.1 - Intel64 Family 6 Model 58 Stepping 9, GenuineIntel - max function length of wholestagecodegen: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ---------------------------------------------------------------------------------------------- - codegen = F 462 / 533 1.4 704.4 1.0X - codegen = T maxLinesPerFunction = 10000 3444 / 3447 0.2 5255.3 0.1X - codegen = T maxLinesPerFunction = 1500 447 / 478 1.5 682.1 1.0X + Java HotSpot(TM) 64-Bit Server VM 1.8.0_31-b13 on Mac OS X 10.10.2 + Intel(R) Core(TM) i7-4578U CPU @ 3.00GHz + max function length of codegen: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + hugeMethodLimit = 8000 704 / 868 0.9 1074.2 1.0X + hugeMethodLimit = 16000 2447 / 2457 0.3 3733.1 0.3X */ } From b185e4996a2ca67bd3928daa8f7b88f24faaeeff Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Wed, 4 Oct 2017 00:07:43 +0900 Subject: [PATCH 10/14] Apply reviews --- .../expressions/codegen/CodeGenerator.scala | 41 ++--- .../codegen/GenerateMutableProjection.scala | 4 +- .../codegen/GenerateOrdering.scala | 3 +- .../codegen/GeneratePredicate.scala | 3 +- .../codegen/GenerateSafeProjection.scala | 4 +- .../codegen/GenerateUnsafeProjection.scala | 4 +- .../codegen/GenerateUnsafeRowJoiner.scala | 4 +- .../apache/spark/sql/internal/SQLConf.scala | 16 +- .../expressions/CodeGenerationSuite.scala | 59 +++----- .../catalyst/expressions/OrderingSuite.scala | 20 +-- .../codegen/GeneratedProjectionSuite.scala | 102 ++++++------- .../sql/execution/WholeStageCodegenExec.scala | 15 +- .../columnar/GenerateColumnAccessor.scala | 3 +- .../resources/sql-tests/inputs/group-by.sql | 7 - .../sql-tests/results/group-by.sql.out | 88 +++++------ .../spark/sql/DataFrameAggregateSuite.scala | 42 +++-- .../sql/DataFrameTimeWindowingSuite.scala | 143 ++++++++---------- .../org/apache/spark/sql/SQLQuerySuite.scala | 44 +++--- .../execution/WholeStageCodegenSuite.scala | 15 +- .../benchmark/AggregateBenchmark.scala | 20 +-- 20 files changed, 281 insertions(+), 356 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 1dec443d43f6..3fff900af743 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -1008,12 +1008,12 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin object CodeGenerator extends Logging { // This is the value of HugeMethodLimit in the OpenJDK JVM settings - val DEFAULT_OPENJDK_JVM_HUGE_METHOD_LIMIT = 8000 + val DEFAULT_JVM_HUGE_METHOD_LIMIT = 8000 /** * Compile the Java source code into a Java class, using Janino. */ - def compile(code: CodeAndComment): GeneratedClass = try { + def compile(code: CodeAndComment): (GeneratedClass, Int) = try { cache.get(code) } catch { // Cache.get() may wrap the original exception. See the following URL @@ -1026,7 +1026,7 @@ object CodeGenerator extends Logging { /** * Compile the Java source code into a Java class, using Janino. */ - private[this] def doCompile(code: CodeAndComment): GeneratedClass = { + private[this] def doCompile(code: CodeAndComment): (GeneratedClass, Int) = { val evaluator = new ClassBodyEvaluator() // A special classloader used to wrap the actual parent classloader of @@ -1065,7 +1065,7 @@ object CodeGenerator extends Logging { s"\n${CodeFormatter.format(code)}" }) - val methodsToByteCodeSize = try { + val maxCodeSize = try { evaluator.cook("generated.java", code.body) updateAndGetCompilationStats(evaluator) } catch { @@ -1083,29 +1083,14 @@ object CodeGenerator extends Logging { throw new CompileException(msg, e.getLocation) } - // Check if compiled code has a too large function - methodsToByteCodeSize.foreach { case (name, byteCodeSize) => - if (byteCodeSize > SQLConf.get.hugeMethodLimit) { - val clazzName = evaluator.getClazz.getSimpleName - val methodName = name.replace("$", "") - val msg = s"failed to compile: the size of $clazzName.$methodName was $byteCodeSize and " + - "this value went over the limit `spark.sql.codegen.hugeMethodLimit`" + - s"(${SQLConf.get.hugeMethodLimit}). To avoid this error, you can make this limit higher." - logError(msg) - val maxLines = SQLConf.get.loggingMaxLinesForCodegen - logInfo(s"\n${CodeFormatter.format(code, maxLines)}") - throw new CompileException(msg, null) - } - } - - evaluator.getClazz().newInstance().asInstanceOf[GeneratedClass] + (evaluator.getClazz().newInstance().asInstanceOf[GeneratedClass], maxCodeSize) } /** - * Returns the pairs of the generated class and method bytecode sizes by inspecting janino - * private fields. Also, this method updates the metrics information. + * Returns the max bytecode size of the generated functions by inspecting janino private fields. + * Also, this method updates the metrics information. */ - private def updateAndGetCompilationStats(evaluator: ClassBodyEvaluator): Seq[(String, Int)] = { + private def updateAndGetCompilationStats(evaluator: ClassBodyEvaluator): Int = { // First retrieve the generated classes. val classes = { val resultField = classOf[SimpleCompiler].getDeclaredField("result") @@ -1120,7 +1105,7 @@ object CodeGenerator extends Logging { val codeAttr = Utils.classForName("org.codehaus.janino.util.ClassFile$CodeAttribute") val codeAttrField = codeAttr.getDeclaredField("code") codeAttrField.setAccessible(true) - val methodsToByteCodeSize = classes.flatMap { case (_, classBytes) => + val codeSizes = classes.flatMap { case (_, classBytes) => CodegenMetrics.METRIC_GENERATED_CLASS_BYTECODE_SIZE.update(classBytes.length) try { val cf = new ClassFile(new ByteArrayInputStream(classBytes)) @@ -1128,7 +1113,7 @@ object CodeGenerator extends Logging { method.getAttributes().filter(_.getClass.getName == codeAttr.getName).map { a => val byteCodeSize = codeAttrField.get(a).asInstanceOf[Array[Byte]].length CodegenMetrics.METRIC_GENERATED_METHOD_BYTECODE_SIZE.update(byteCodeSize) - (method.getName, byteCodeSize) + byteCodeSize } } Some(stats) @@ -1139,7 +1124,7 @@ object CodeGenerator extends Logging { } }.flatten - methodsToByteCodeSize.toSeq + codeSizes.max } /** @@ -1154,8 +1139,8 @@ object CodeGenerator extends Logging { private val cache = CacheBuilder.newBuilder() .maximumSize(100) .build( - new CacheLoader[CodeAndComment, GeneratedClass]() { - override def load(code: CodeAndComment): GeneratedClass = { + new CacheLoader[CodeAndComment, (GeneratedClass, Int)]() { + override def load(code: CodeAndComment): (GeneratedClass, Int) = { val startTime = System.nanoTime() val result = doCompile(code) val endTime = System.nanoTime() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala index 3768dcde00a4..b5429fade53c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala @@ -142,7 +142,7 @@ object GenerateMutableProjection extends CodeGenerator[Seq[Expression], MutableP new CodeAndComment(codeBody, ctx.getPlaceHolderToComments())) logDebug(s"code for ${expressions.mkString(",")}:\n${CodeFormatter.format(code)}") - val c = CodeGenerator.compile(code) - c.generate(ctx.references.toArray).asInstanceOf[MutableProjection] + val (clazz, _) = CodeGenerator.compile(code) + clazz.generate(ctx.references.toArray).asInstanceOf[MutableProjection] } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala index 4e4789598520..1639d1b9dda1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala @@ -185,7 +185,8 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[InternalR new CodeAndComment(codeBody, ctx.getPlaceHolderToComments())) logDebug(s"Generated Ordering by ${ordering.mkString(",")}:\n${CodeFormatter.format(code)}") - CodeGenerator.compile(code).generate(ctx.references.toArray).asInstanceOf[BaseOrdering] + val (clazz, _) = CodeGenerator.compile(code) + clazz.generate(ctx.references.toArray).asInstanceOf[BaseOrdering] } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala index e35b9dda6c01..e0fabad6d089 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala @@ -78,6 +78,7 @@ object GeneratePredicate extends CodeGenerator[Expression, Predicate] { new CodeAndComment(codeBody, ctx.getPlaceHolderToComments())) logDebug(s"Generated predicate '$predicate':\n${CodeFormatter.format(code)}") - CodeGenerator.compile(code).generate(ctx.references.toArray).asInstanceOf[Predicate] + val (clazz, _) = CodeGenerator.compile(code) + clazz.generate(ctx.references.toArray).asInstanceOf[Predicate] } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala index 192701a82968..1e4ac3f2afd5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala @@ -189,8 +189,8 @@ object GenerateSafeProjection extends CodeGenerator[Seq[Expression], Projection] new CodeAndComment(codeBody, ctx.getPlaceHolderToComments())) logDebug(s"code for ${expressions.mkString(",")}:\n${CodeFormatter.format(code)}") - val c = CodeGenerator.compile(code) + val (clazz, _) = CodeGenerator.compile(code) val resultRow = new SpecificInternalRow(expressions.map(_.dataType)) - c.generate(ctx.references.toArray :+ resultRow).asInstanceOf[Projection] + clazz.generate(ctx.references.toArray :+ resultRow).asInstanceOf[Projection] } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala index f2a66efc98e7..4bd50aee0551 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala @@ -409,7 +409,7 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro new CodeAndComment(codeBody, ctx.getPlaceHolderToComments())) logDebug(s"code for ${expressions.mkString(",")}:\n${CodeFormatter.format(code)}") - val c = CodeGenerator.compile(code) - c.generate(ctx.references.toArray).asInstanceOf[UnsafeProjection] + val (clazz, _) = CodeGenerator.compile(code) + clazz.generate(ctx.references.toArray).asInstanceOf[UnsafeProjection] } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala index 4aa5ec82471e..6bc72a0d75c6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala @@ -196,7 +196,7 @@ object GenerateUnsafeRowJoiner extends CodeGenerator[(StructType, StructType), U val code = CodeFormatter.stripOverlappingComments(new CodeAndComment(codeBody, Map.empty)) logDebug(s"SpecificUnsafeRowJoiner($schema1, $schema2):\n${CodeFormatter.format(code)}") - val c = CodeGenerator.compile(code) - c.generate(Array.empty).asInstanceOf[UnsafeRowJoiner] + val (clazz, _) = CodeGenerator.compile(code) + clazz.generate(Array.empty).asInstanceOf[UnsafeRowJoiner] } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 6e5cbd0fbd31..58323740b80c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -576,15 +576,15 @@ object SQLConf { "disable logging or -1 to apply no limit.") .createWithDefault(1000) - val CODEGEN_HUGE_METHOD_LIMIT = buildConf("spark.sql.codegen.hugeMethodLimit") + val WHOLESTAGE_HUGE_METHOD_LIMIT = buildConf("spark.sql.codegen.hugeMethodLimit") .internal() - .doc("The maximum bytecode size of a single compiled Java function generated by codegen. " + - "When the compiled function exceeds this threshold, the codegen is deactivated. " + - s"The default value is ${CodeGenerator.DEFAULT_OPENJDK_JVM_HUGE_METHOD_LIMIT} and " + - "this is a limit in the OpenJDK JVM implementation." - ) + .doc("The maximum bytecode size of a single compiled Java function generated by whole-stage " + + "codegen. When the compiled function exceeds this threshold, " + + "the whole-stage codegen is deactivated for this subtree of the current query plan. " + + s"The default value is ${CodeGenerator.DEFAULT_JVM_HUGE_METHOD_LIMIT} and " + + "this is a limit in the OpenJDK JVM implementation.") .intConf - .createWithDefault(CodeGenerator.DEFAULT_OPENJDK_JVM_HUGE_METHOD_LIMIT) + .createWithDefault(CodeGenerator.DEFAULT_JVM_HUGE_METHOD_LIMIT) val FILES_MAX_PARTITION_BYTES = buildConf("spark.sql.files.maxPartitionBytes") .doc("The maximum number of bytes to pack into a single partition when reading files.") @@ -1059,7 +1059,7 @@ class SQLConf extends Serializable with Logging { def loggingMaxLinesForCodegen: Int = getConf(CODEGEN_LOGGING_MAX_LINES) - def hugeMethodLimit: Int = getConf(CODEGEN_HUGE_METHOD_LIMIT) + def hugeMethodLimit: Int = getConf(WHOLESTAGE_HUGE_METHOD_LIMIT) def tableRelationCacheSize: Int = getConf(StaticSQLConf.FILESOURCE_TABLE_RELATION_CACHE_SIZE) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala index 4a3341e169be..7ea0bec14548 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala @@ -19,15 +19,14 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.Timestamp +import org.apache.spark.SparkFunSuite import org.apache.spark.metrics.source.CodegenMetrics import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.objects.{AssertNotNull, CreateExternalRow, GetExternalRowField, ValidateExternalType} -import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils} -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.ThreadUtils @@ -35,7 +34,7 @@ import org.apache.spark.util.ThreadUtils /** * Additional tests for code generation. */ -class CodeGenerationSuite extends PlanTest with ExpressionEvalHelper { +class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { test("multithreaded eval") { import scala.concurrent._ @@ -99,23 +98,19 @@ class CodeGenerationSuite extends PlanTest with ExpressionEvalHelper { } test("SPARK-18091: split large if expressions into blocks due to JVM code size limit") { - // Set the max value at `WHOLESTAGE_HUGE_METHOD_LIMIT` to compile gen'd code by janino - withSQLConf(SQLConf.CODEGEN_HUGE_METHOD_LIMIT.key -> Int.MaxValue.toString) { - var strExpr: Expression = Literal("abc") - for (_ <- 1 to 150) { - strExpr = Decode(Encode(strExpr, "utf-8"), "utf-8") - } + var strExpr: Expression = Literal("abc") + for (_ <- 1 to 150) { + strExpr = Decode(Encode(strExpr, "utf-8"), "utf-8") + } - val expressions = Seq(If(EqualTo(strExpr, strExpr), strExpr, strExpr)) - val plan = GenerateMutableProjection.generate(expressions) - val actual = plan(null).toSeq(expressions.map(_.dataType)) - assert(actual.length == 1) - val expected = UTF8String.fromString("abc") + val expressions = Seq(If(EqualTo(strExpr, strExpr), strExpr, strExpr)) + val plan = GenerateMutableProjection.generate(expressions) + val actual = plan(null).toSeq(expressions.map(_.dataType)) + assert(actual.length == 1) + val expected = UTF8String.fromString("abc") - if (!checkResult(actual.head, expected, expressions.head.dataType)) { - fail(s"Incorrect Evaluation: expressions: $expressions, actual: $actual, " + - s"expected: $expected") - } + if (!checkResult(actual.head, expected, expressions.head.dataType)) { + fail(s"Incorrect Evaluation: expressions: $expressions, actual: $actual, expected: $expected") } } @@ -133,22 +128,18 @@ class CodeGenerationSuite extends PlanTest with ExpressionEvalHelper { } test("SPARK-14793: split wide map creation into blocks due to JVM code size limit") { - // Set the max value at `WHOLESTAGE_HUGE_METHOD_LIMIT` to compile gen'd code by janino - withSQLConf(SQLConf.CODEGEN_HUGE_METHOD_LIMIT.key -> Int.MaxValue.toString) { - val length = 5000 - val expressions = Seq(CreateMap( - List.fill(length)(EqualTo(Literal(1), Literal(1))).zipWithIndex.flatMap { - case (expr, i) => Seq(Literal(i), expr) - })) - val plan = GenerateMutableProjection.generate(expressions) - val actual = plan(new GenericInternalRow(length)).toSeq(expressions.map(_.dataType)) - assert(actual.length == 1) - val expected = ArrayBasedMapData((0 until length).toArray, Array.fill(length)(true)) - - if (!checkResult(actual.head, expected, expressions.head.dataType)) { - fail(s"Incorrect Evaluation: expressions: $expressions, actual: $actual, " + - "expected: $expected") - } + val length = 5000 + val expressions = Seq(CreateMap( + List.fill(length)(EqualTo(Literal(1), Literal(1))).zipWithIndex.flatMap { + case (expr, i) => Seq(Literal(i), expr) + })) + val plan = GenerateMutableProjection.generate(expressions) + val actual = plan(new GenericInternalRow(length)).toSeq(expressions.map(_.dataType)) + assert(actual.length == 1) + val expected = ArrayBasedMapData((0 until length).toArray, Array.fill(length)(true)) + + if (!checkResult(actual.head, expected, expressions.head.dataType)) { + fail(s"Incorrect Evaluation: expressions: $expressions, actual: $actual, expected: $expected") } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/OrderingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/OrderingSuite.scala index 2c9ac69b65e7..aa61ba2bff2b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/OrderingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/OrderingSuite.scala @@ -19,17 +19,15 @@ package org.apache.spark.sql.catalyst.expressions import scala.math._ -import org.apache.spark.SparkConf +import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.serializer.KryoSerializer import org.apache.spark.sql.{RandomDataGenerator, Row} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateOrdering, LazilyGeneratedOrdering} -import org.apache.spark.sql.catalyst.plans.PlanTest -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -class OrderingSuite extends PlanTest with ExpressionEvalHelper { +class OrderingSuite extends SparkFunSuite with ExpressionEvalHelper { def compareArrays(a: Seq[Any], b: Seq[Any], expected: Int): Unit = { test(s"compare two arrays: a = $a, b = $b") { @@ -131,17 +129,13 @@ class OrderingSuite extends PlanTest with ExpressionEvalHelper { } test("SPARK-16845: GeneratedClass$SpecificOrdering grows beyond 64 KB") { - // To just check if janino can compile gen'd code, we set the max value at - // `WHOLESTAGE_HUGE_METHOD_LIMIT` explicitly. - withSQLConf(SQLConf.CODEGEN_HUGE_METHOD_LIMIT.key -> Int.MaxValue.toString) { - val sortOrder = Literal("abc").asc + val sortOrder = Literal("abc").asc - // this is passing prior to SPARK-16845, and it should also be passing after SPARK-16845 - GenerateOrdering.generate(Array.fill(40)(sortOrder)) + // this is passing prior to SPARK-16845, and it should also be passing after SPARK-16845 + GenerateOrdering.generate(Array.fill(40)(sortOrder)) - // verify that we can support up to 5000 ordering comparisons, which should be sufficient - GenerateOrdering.generate(Array.fill(5000)(sortOrder)) - } + // verify that we can support up to 5000 ordering comparisons, which should be sufficient + GenerateOrdering.generate(Array.fill(5000)(sortOrder)) } test("SPARK-21344: BinaryType comparison does signed byte array comparison") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala index a19c92e4735a..0cd0d8859145 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala @@ -19,18 +19,17 @@ package org.apache.spark.sql.catalyst.expressions.codegen import java.nio.charset.StandardCharsets +import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.util.GenericArrayData -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String /** * A test suite for generated projections */ -class GeneratedProjectionSuite extends PlanTest { +class GeneratedProjectionSuite extends SparkFunSuite { test("generated projections on wider table") { val N = 1000 @@ -84,57 +83,54 @@ class GeneratedProjectionSuite extends PlanTest { } test("SPARK-18016: generated projections on wider table requiring class-splitting") { - // Set the max value at `WHOLESTAGE_HUGE_METHOD_LIMIT` to compile gen'd code by janino - withSQLConf(SQLConf.CODEGEN_HUGE_METHOD_LIMIT.key -> Int.MaxValue.toString) { - val N = 4000 - 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) - - // test generated UnsafeProjection - val unsafeProj = UnsafeProjection.create(nestedSchema) - val unsafe: UnsafeRow = unsafeProj(nested) - (0 until N).foreach { i => - val s = UTF8String.fromString(i.toString) - assert(i === unsafe.getInt(i + 2)) - assert(s === unsafe.getUTF8String(i + 2 + N)) - assert(i === unsafe.getStruct(0, N * 2).getInt(i)) - assert(s === unsafe.getStruct(0, N * 2).getUTF8String(i + N)) - assert(i === unsafe.getStruct(1, N * 2).getInt(i)) - assert(s === unsafe.getStruct(1, N * 2).getUTF8String(i + N)) - } - - // test generated SafeProjection - val safeProj = FromUnsafeProjection(nestedSchema) - val result = safeProj(unsafe) - // Can't compare GenericInternalRow with JoinedRow directly - (0 until N).foreach { i => - val s = UTF8String.fromString(i.toString) - assert(i === result.getInt(i + 2)) - assert(s === result.getUTF8String(i + 2 + N)) - assert(i === result.getStruct(0, N * 2).getInt(i)) - assert(s === result.getStruct(0, N * 2).getUTF8String(i + N)) - assert(i === result.getStruct(1, N * 2).getInt(i)) - assert(s === result.getStruct(1, N * 2).getUTF8String(i + N)) - } - - // test generated MutableProjection - val exprs = nestedSchema.fields.zipWithIndex.map { case (f, i) => - BoundReference(i, f.dataType, true) - } - val mutableProj = GenerateMutableProjection.generate(exprs) - val row1 = mutableProj(result) - assert(result === row1) - val row2 = mutableProj(result) - assert(result === row2) + val N = 4000 + 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) + + // test generated UnsafeProjection + val unsafeProj = UnsafeProjection.create(nestedSchema) + val unsafe: UnsafeRow = unsafeProj(nested) + (0 until N).foreach { i => + val s = UTF8String.fromString(i.toString) + assert(i === unsafe.getInt(i + 2)) + assert(s === unsafe.getUTF8String(i + 2 + N)) + assert(i === unsafe.getStruct(0, N * 2).getInt(i)) + assert(s === unsafe.getStruct(0, N * 2).getUTF8String(i + N)) + assert(i === unsafe.getStruct(1, N * 2).getInt(i)) + assert(s === unsafe.getStruct(1, N * 2).getUTF8String(i + N)) + } + + // test generated SafeProjection + val safeProj = FromUnsafeProjection(nestedSchema) + val result = safeProj(unsafe) + // Can't compare GenericInternalRow with JoinedRow directly + (0 until N).foreach { i => + val s = UTF8String.fromString(i.toString) + assert(i === result.getInt(i + 2)) + assert(s === result.getUTF8String(i + 2 + N)) + assert(i === result.getStruct(0, N * 2).getInt(i)) + assert(s === result.getStruct(0, N * 2).getUTF8String(i + N)) + assert(i === result.getStruct(1, N * 2).getInt(i)) + assert(s === result.getStruct(1, N * 2).getUTF8String(i + N)) + } + + // test generated MutableProjection + val exprs = nestedSchema.fields.zipWithIndex.map { case (f, i) => + BoundReference(i, f.dataType, true) } + val mutableProj = GenerateMutableProjection.generate(exprs) + val row1 = mutableProj(result) + assert(result === row1) + val row2 = mutableProj(result) + assert(result === row2) } test("generated unsafe projection with array of binary") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 661f5b3774df..f129cffdf843 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -381,7 +381,7 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co override def doExecute(): RDD[InternalRow] = { val (ctx, cleanedSource) = doCodeGen() // try to compile and fallback if it failed - try { + val (_, maxCodeSize) = try { CodeGenerator.compile(cleanedSource) } catch { case _: Exception if !Utils.isTesting && sqlContext.conf.codegenFallback => @@ -389,6 +389,15 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co logWarning(s"Whole-stage codegen disabled for this plan:\n $treeString") return child.execute() } + + // Check if compiled code has a too large function + if (maxCodeSize > sqlContext.conf.hugeMethodLimit) { + logWarning(s"Found too long generated codes: the bytecode size was $maxCodeSize and " + + s"this value went over the limit ${sqlContext.conf.hugeMethodLimit}. To avoid this, " + + s"you can the limit ${SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key} higher:\n$treeString") + return child.execute() + } + val references = ctx.references.toArray val durationMs = longMetric("pipelineTime") @@ -397,7 +406,7 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co assert(rdds.size <= 2, "Up to two input RDDs can be supported") if (rdds.length == 1) { rdds.head.mapPartitionsWithIndex { (index, iter) => - val clazz = CodeGenerator.compile(cleanedSource) + val (clazz, _) = CodeGenerator.compile(cleanedSource) val buffer = clazz.generate(references).asInstanceOf[BufferedRowIterator] buffer.init(index, Array(iter)) new Iterator[InternalRow] { @@ -416,7 +425,7 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co // a small hack to obtain the correct partition index }.mapPartitionsWithIndex { (index, zippedIter) => val (leftIter, rightIter) = zippedIter.next() - val clazz = CodeGenerator.compile(cleanedSource) + val (clazz, _) = CodeGenerator.compile(cleanedSource) val buffer = clazz.generate(references).asInstanceOf[BufferedRowIterator] buffer.init(index, Array(leftIter, rightIter)) new Iterator[InternalRow] { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala index da3464328191..ae600c1ffae8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala @@ -227,6 +227,7 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera new CodeAndComment(codeBody, ctx.getPlaceHolderToComments())) logDebug(s"Generated ColumnarIterator:\n${CodeFormatter.format(code)}") - CodeGenerator.compile(code).generate(Array.empty).asInstanceOf[ColumnarIterator] + val (clazz, _) = CodeGenerator.compile(code) + clazz.generate(Array.empty).asInstanceOf[ColumnarIterator] } } diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql index bcb45ccbf4d6..1e1384549a41 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql @@ -30,15 +30,8 @@ SELECT a + 2, COUNT(b) FROM testData GROUP BY a + 1; SELECT a + 1 + 1, COUNT(b) FROM testData GROUP BY a + 1; -- Aggregate with nulls. --- --- In SPARK-21871, we added code to check the actual bytecode size of gen'd methods. If the size --- goes over `hugeMethodLimit`, Spark fails to compile the methods and the execution also fails --- in a test mode. So, we explicitly made this threshold higher here. --- This workaround can be removed if this issue fixed. -SET spark.sql.codegen.hugeMethodLimit=16000; SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a), AVG(a), VARIANCE(a), STDDEV(a), SUM(a), COUNT(a) FROM testData; -SET spark.sql.codegen.hugeMethodLimit=8000; -- Aggregate with foldable input and multiple distinct groups. SELECT COUNT(DISTINCT b), COUNT(DISTINCT b, c) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a; diff --git a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out index 2a02e67a06af..986bb01c13fe 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 27 +-- Number of queries: 25 -- !query 0 @@ -125,121 +125,105 @@ NULL 1 -- !query 13 -SET spark.sql.codegen.hugeMethodLimit=16000 --- !query 13 schema -struct --- !query 13 output -spark.sql.codegen.hugeMethodLimit 16000 - - --- !query 14 SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a), AVG(a), VARIANCE(a), STDDEV(a), SUM(a), COUNT(a) FROM testData --- !query 14 schema +-- !query 13 schema struct --- !query 14 output +-- !query 13 output -0.2723801058145729 -1.5069204152249134 1 3 2.142857142857143 0.8095238095238094 0.8997354108424372 15 7 --- !query 15 -SET spark.sql.codegen.hugeMethodLimit=8000 --- !query 15 schema -struct --- !query 15 output -spark.sql.codegen.hugeMethodLimit 8000 - - --- !query 16 +-- !query 14 SELECT COUNT(DISTINCT b), COUNT(DISTINCT b, c) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a --- !query 16 schema +-- !query 14 schema struct --- !query 16 output +-- !query 14 output 1 1 --- !query 17 +-- !query 15 SELECT a AS k, COUNT(b) FROM testData GROUP BY k --- !query 17 schema +-- !query 15 schema struct --- !query 17 output +-- !query 15 output 1 2 2 2 3 2 NULL 1 --- !query 18 +-- !query 16 SELECT a AS k, COUNT(b) FROM testData GROUP BY k HAVING k > 1 --- !query 18 schema +-- !query 16 schema struct --- !query 18 output +-- !query 16 output 2 2 3 2 --- !query 19 +-- !query 17 SELECT COUNT(b) AS k FROM testData GROUP BY k --- !query 19 schema +-- !query 17 schema struct<> --- !query 19 output +-- !query 17 output org.apache.spark.sql.AnalysisException aggregate functions are not allowed in GROUP BY, but found count(testdata.`b`); --- !query 20 +-- !query 18 CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES (1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v) --- !query 20 schema +-- !query 18 schema struct<> --- !query 20 output +-- !query 18 output --- !query 21 +-- !query 19 SELECT k AS a, COUNT(v) FROM testDataHasSameNameWithAlias GROUP BY a --- !query 21 schema +-- !query 19 schema struct<> --- !query 21 output +-- !query 19 output org.apache.spark.sql.AnalysisException expression 'testdatahassamenamewithalias.`k`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.; --- !query 22 +-- !query 20 set spark.sql.groupByAliases=false --- !query 22 schema +-- !query 20 schema struct --- !query 22 output +-- !query 20 output spark.sql.groupByAliases false --- !query 23 +-- !query 21 SELECT a AS k, COUNT(b) FROM testData GROUP BY k --- !query 23 schema +-- !query 21 schema struct<> --- !query 23 output +-- !query 21 output org.apache.spark.sql.AnalysisException cannot resolve '`k`' given input columns: [testdata.a, testdata.b]; line 1 pos 47 --- !query 24 +-- !query 22 SELECT a, COUNT(1) FROM testData WHERE false GROUP BY a --- !query 24 schema +-- !query 22 schema struct --- !query 24 output +-- !query 22 output --- !query 25 +-- !query 23 SELECT COUNT(1) FROM testData WHERE false --- !query 25 schema +-- !query 23 schema struct --- !query 25 output +-- !query 23 output 0 --- !query 26 +-- !query 24 SELECT 1 FROM (SELECT COUNT(1) FROM testData WHERE false) t --- !query 26 schema +-- !query 24 schema struct<1:int> --- !query 26 output +-- !query 24 output 1 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index 72d79c16282a..8549eac58ee9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -432,31 +432,25 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { } test("zero moments") { - // In SPARK-21871, we added code to check the actual bytecode size of gen'd methods. If the size - // goes over `hugeMethodLimit`, Spark fails to compile the methods and the execution also fails - // in a test mode. So, we explicitly made this threshold higher here. - // This workaround can be removed if this issue fixed. - withSQLConf(SQLConf.CODEGEN_HUGE_METHOD_LIMIT.key -> "16000") { - val input = Seq((1, 2)).toDF("a", "b") - checkAnswer( - input.agg(stddev('a), stddev_samp('a), stddev_pop('a), variance('a), - var_samp('a), var_pop('a), skewness('a), kurtosis('a)), - Row(Double.NaN, Double.NaN, 0.0, Double.NaN, Double.NaN, 0.0, - Double.NaN, Double.NaN)) + val input = Seq((1, 2)).toDF("a", "b") + checkAnswer( + input.agg(stddev('a), stddev_samp('a), stddev_pop('a), variance('a), + var_samp('a), var_pop('a), skewness('a), kurtosis('a)), + Row(Double.NaN, Double.NaN, 0.0, Double.NaN, Double.NaN, 0.0, + Double.NaN, Double.NaN)) - checkAnswer( - input.agg( - expr("stddev(a)"), - expr("stddev_samp(a)"), - expr("stddev_pop(a)"), - expr("variance(a)"), - expr("var_samp(a)"), - expr("var_pop(a)"), - expr("skewness(a)"), - expr("kurtosis(a)")), - Row(Double.NaN, Double.NaN, 0.0, Double.NaN, Double.NaN, 0.0, - Double.NaN, Double.NaN)) - } + checkAnswer( + input.agg( + expr("stddev(a)"), + expr("stddev_samp(a)"), + expr("stddev_pop(a)"), + expr("variance(a)"), + expr("var_samp(a)"), + expr("var_pop(a)"), + expr("skewness(a)"), + expr("kurtosis(a)")), + Row(Double.NaN, Double.NaN, 0.0, Double.NaN, Double.NaN, 0.0, + Double.NaN, Double.NaN)) } test("null moments") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala index 48159c4451af..6fe356877c26 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala @@ -21,7 +21,6 @@ import org.scalatest.BeforeAndAfterEach import org.apache.spark.sql.catalyst.plans.logical.Expand import org.apache.spark.sql.functions._ -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.StringType @@ -96,62 +95,50 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext with B } test("sliding window grouping") { - // In SPARK-21871, we added code to check the actual bytecode size of gen'd methods. If the size - // goes over `hugeMethodLimit`, Spark fails to compile the methods and the execution also fails - // in a test mode. So, we explicitly made this threshold higher here. - // This workaround can be removed if this issue fixed. - withSQLConf(SQLConf.CODEGEN_HUGE_METHOD_LIMIT.key -> "16000") { - val df = Seq( - ("2016-03-27 19:39:34", 1, "a"), - ("2016-03-27 19:39:56", 2, "a"), - ("2016-03-27 19:39:27", 4, "b")).toDF("time", "value", "id") + val df = Seq( + ("2016-03-27 19:39:34", 1, "a"), + ("2016-03-27 19:39:56", 2, "a"), + ("2016-03-27 19:39:27", 4, "b")).toDF("time", "value", "id") - checkAnswer( - df.groupBy(window($"time", "10 seconds", "3 seconds", "0 second")) - .agg(count("*").as("counts")) - .orderBy($"window.start".asc) - .select($"window.start".cast("string"), $"window.end".cast("string"), $"counts"), - // 2016-03-27 19:39:27 UTC -> 4 bins - // 2016-03-27 19:39:34 UTC -> 3 bins - // 2016-03-27 19:39:56 UTC -> 3 bins - Seq( - Row("2016-03-27 19:39:18", "2016-03-27 19:39:28", 1), - Row("2016-03-27 19:39:21", "2016-03-27 19:39:31", 1), - Row("2016-03-27 19:39:24", "2016-03-27 19:39:34", 1), - Row("2016-03-27 19:39:27", "2016-03-27 19:39:37", 2), - Row("2016-03-27 19:39:30", "2016-03-27 19:39:40", 1), - Row("2016-03-27 19:39:33", "2016-03-27 19:39:43", 1), - Row("2016-03-27 19:39:48", "2016-03-27 19:39:58", 1), - Row("2016-03-27 19:39:51", "2016-03-27 19:40:01", 1), - Row("2016-03-27 19:39:54", "2016-03-27 19:40:04", 1)) - ) - } + checkAnswer( + df.groupBy(window($"time", "10 seconds", "3 seconds", "0 second")) + .agg(count("*").as("counts")) + .orderBy($"window.start".asc) + .select($"window.start".cast("string"), $"window.end".cast("string"), $"counts"), + // 2016-03-27 19:39:27 UTC -> 4 bins + // 2016-03-27 19:39:34 UTC -> 3 bins + // 2016-03-27 19:39:56 UTC -> 3 bins + Seq( + Row("2016-03-27 19:39:18", "2016-03-27 19:39:28", 1), + Row("2016-03-27 19:39:21", "2016-03-27 19:39:31", 1), + Row("2016-03-27 19:39:24", "2016-03-27 19:39:34", 1), + Row("2016-03-27 19:39:27", "2016-03-27 19:39:37", 2), + Row("2016-03-27 19:39:30", "2016-03-27 19:39:40", 1), + Row("2016-03-27 19:39:33", "2016-03-27 19:39:43", 1), + Row("2016-03-27 19:39:48", "2016-03-27 19:39:58", 1), + Row("2016-03-27 19:39:51", "2016-03-27 19:40:01", 1), + Row("2016-03-27 19:39:54", "2016-03-27 19:40:04", 1)) + ) } test("sliding window projection") { - // In SPARK-21871, we added code to check the actual bytecode size of gen'd methods. If the size - // goes over `hugeMethodLimit`, Spark fails to compile the methods and the execution also fails - // in a test mode. So, we explicitly made this threshold higher here. - // This workaround can be removed if this issue fixed. - withSQLConf(SQLConf.CODEGEN_HUGE_METHOD_LIMIT.key -> "16000") { - val df = Seq( - ("2016-03-27 19:39:34", 1, "a"), - ("2016-03-27 19:39:56", 2, "a"), - ("2016-03-27 19:39:27", 4, "b")).toDF("time", "value", "id") - .select(window($"time", "10 seconds", "3 seconds", "0 second"), $"value") - .orderBy($"window.start".asc, $"value".desc).select("value") - - val expands = df.queryExecution.optimizedPlan.find(_.isInstanceOf[Expand]) - assert(expands.nonEmpty, "Sliding windows require expand") + val df = Seq( + ("2016-03-27 19:39:34", 1, "a"), + ("2016-03-27 19:39:56", 2, "a"), + ("2016-03-27 19:39:27", 4, "b")).toDF("time", "value", "id") + .select(window($"time", "10 seconds", "3 seconds", "0 second"), $"value") + .orderBy($"window.start".asc, $"value".desc).select("value") - checkAnswer( - df, - // 2016-03-27 19:39:27 UTC -> 4 bins - // 2016-03-27 19:39:34 UTC -> 3 bins - // 2016-03-27 19:39:56 UTC -> 3 bins - Seq(Row(4), Row(4), Row(4), Row(4), Row(1), Row(1), Row(1), Row(2), Row(2), Row(2)) - ) - } + val expands = df.queryExecution.optimizedPlan.find(_.isInstanceOf[Expand]) + assert(expands.nonEmpty, "Sliding windows require expand") + + checkAnswer( + df, + // 2016-03-27 19:39:27 UTC -> 4 bins + // 2016-03-27 19:39:34 UTC -> 3 bins + // 2016-03-27 19:39:56 UTC -> 3 bins + Seq(Row(4), Row(4), Row(4), Row(4), Row(1), Row(1), Row(1), Row(2), Row(2), Row(2)) + ) } test("windowing combined with explode expression") { @@ -241,35 +228,29 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext with B } test("millisecond precision sliding windows") { - // In SPARK-21871, we added code to check the actual bytecode size of gen'd methods. If the size - // goes over `hugeMethodLimit`, Spark fails to compile the methods and the execution also fails - // in a test mode. So, we explicitly made this threshold higher here. - // This workaround can be removed if this issue fixed. - withSQLConf(SQLConf.CODEGEN_HUGE_METHOD_LIMIT.key -> "16000") { - val df = Seq( - ("2016-03-27 09:00:00.41", 3), - ("2016-03-27 09:00:00.62", 6), - ("2016-03-27 09:00:00.715", 8)).toDF("time", "value") - checkAnswer( - df.groupBy(window($"time", "200 milliseconds", "40 milliseconds", "0 milliseconds")) - .agg(count("*").as("counts")) - .orderBy($"window.start".asc) - .select($"window.start".cast(StringType), $"window.end".cast(StringType), $"counts"), - Seq( - Row("2016-03-27 09:00:00.24", "2016-03-27 09:00:00.44", 1), - Row("2016-03-27 09:00:00.28", "2016-03-27 09:00:00.48", 1), - Row("2016-03-27 09:00:00.32", "2016-03-27 09:00:00.52", 1), - Row("2016-03-27 09:00:00.36", "2016-03-27 09:00:00.56", 1), - Row("2016-03-27 09:00:00.4", "2016-03-27 09:00:00.6", 1), - Row("2016-03-27 09:00:00.44", "2016-03-27 09:00:00.64", 1), - Row("2016-03-27 09:00:00.48", "2016-03-27 09:00:00.68", 1), - Row("2016-03-27 09:00:00.52", "2016-03-27 09:00:00.72", 2), - Row("2016-03-27 09:00:00.56", "2016-03-27 09:00:00.76", 2), - Row("2016-03-27 09:00:00.6", "2016-03-27 09:00:00.8", 2), - Row("2016-03-27 09:00:00.64", "2016-03-27 09:00:00.84", 1), - Row("2016-03-27 09:00:00.68", "2016-03-27 09:00:00.88", 1)) - ) - } + val df = Seq( + ("2016-03-27 09:00:00.41", 3), + ("2016-03-27 09:00:00.62", 6), + ("2016-03-27 09:00:00.715", 8)).toDF("time", "value") + checkAnswer( + df.groupBy(window($"time", "200 milliseconds", "40 milliseconds", "0 milliseconds")) + .agg(count("*").as("counts")) + .orderBy($"window.start".asc) + .select($"window.start".cast(StringType), $"window.end".cast(StringType), $"counts"), + Seq( + Row("2016-03-27 09:00:00.24", "2016-03-27 09:00:00.44", 1), + Row("2016-03-27 09:00:00.28", "2016-03-27 09:00:00.48", 1), + Row("2016-03-27 09:00:00.32", "2016-03-27 09:00:00.52", 1), + Row("2016-03-27 09:00:00.36", "2016-03-27 09:00:00.56", 1), + Row("2016-03-27 09:00:00.4", "2016-03-27 09:00:00.6", 1), + Row("2016-03-27 09:00:00.44", "2016-03-27 09:00:00.64", 1), + Row("2016-03-27 09:00:00.48", "2016-03-27 09:00:00.68", 1), + Row("2016-03-27 09:00:00.52", "2016-03-27 09:00:00.72", 2), + Row("2016-03-27 09:00:00.56", "2016-03-27 09:00:00.76", 2), + Row("2016-03-27 09:00:00.6", "2016-03-27 09:00:00.8", 2), + Row("2016-03-27 09:00:00.64", "2016-03-27 09:00:00.84", 1), + Row("2016-03-27 09:00:00.68", "2016-03-27 09:00:00.88", 1)) + ) } private def withTempTable(f: String => Unit): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index da6946cd705f..93a7777b70b4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -2102,31 +2102,25 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-15327: fail to compile generated code with complex data structure") { - // In SPARK-21871, we added code to check the actual bytecode size of gen'd methods. If the size - // goes over `hugeMethodLimit`, Spark fails to compile the methods and the execution also fails - // in a test mode. So, we explicitly made this threshold higher here. - // This workaround can be removed if this issue fixed. - withSQLConf(SQLConf.CODEGEN_HUGE_METHOD_LIMIT.key -> "16000") { - withTempDir { dir => - val json = - """ - |{"h": {"b": {"c": [{"e": "adfgd"}], "a": [{"e": "testing", "count": 3}], - |"b": [{"e": "test", "count": 1}]}}, "d": {"b": {"c": [{"e": "adfgd"}], - |"a": [{"e": "testing", "count": 3}], "b": [{"e": "test", "count": 1}]}}, - |"c": {"b": {"c": [{"e": "adfgd"}], "a": [{"count": 3}], - |"b": [{"e": "test", "count": 1}]}}, "a": {"b": {"c": [{"e": "adfgd"}], - |"a": [{"count": 3}], "b": [{"e": "test", "count": 1}]}}, - |"e": {"b": {"c": [{"e": "adfgd"}], "a": [{"e": "testing", "count": 3}], - |"b": [{"e": "test", "count": 1}]}}, "g": {"b": {"c": [{"e": "adfgd"}], - |"a": [{"e": "testing", "count": 3}], "b": [{"e": "test", "count": 1}]}}, - |"f": {"b": {"c": [{"e": "adfgd"}], "a": [{"e": "testing", "count": 3}], - |"b": [{"e": "test", "count": 1}]}}, "b": {"b": {"c": [{"e": "adfgd"}], - |"a": [{"count": 3}], "b": [{"e": "test", "count": 1}]}}}' - | - """.stripMargin - spark.read.json(Seq(json).toDS()).write.mode("overwrite").parquet(dir.toString) - spark.read.parquet(dir.toString).collect() - } + withTempDir{ dir => + val json = + """ + |{"h": {"b": {"c": [{"e": "adfgd"}], "a": [{"e": "testing", "count": 3}], + |"b": [{"e": "test", "count": 1}]}}, "d": {"b": {"c": [{"e": "adfgd"}], + |"a": [{"e": "testing", "count": 3}], "b": [{"e": "test", "count": 1}]}}, + |"c": {"b": {"c": [{"e": "adfgd"}], "a": [{"count": 3}], + |"b": [{"e": "test", "count": 1}]}}, "a": {"b": {"c": [{"e": "adfgd"}], + |"a": [{"count": 3}], "b": [{"e": "test", "count": 1}]}}, + |"e": {"b": {"c": [{"e": "adfgd"}], "a": [{"e": "testing", "count": 3}], + |"b": [{"e": "test", "count": 1}]}}, "g": {"b": {"c": [{"e": "adfgd"}], + |"a": [{"e": "testing", "count": 3}], "b": [{"e": "test", "count": 1}]}}, + |"f": {"b": {"c": [{"e": "adfgd"}], "a": [{"e": "testing", "count": 3}], + |"b": [{"e": "test", "count": 1}]}}, "b": {"b": {"c": [{"e": "adfgd"}], + |"a": [{"count": 3}], "b": [{"e": "test", "count": 1}]}}}' + | + """.stripMargin + spark.read.json(Seq(json).toDS()).write.mode("overwrite").parquet(dir.toString) + spark.read.parquet(dir.toString).collect() } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index 5226b76ba251..a926fef8a4e6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -179,13 +179,12 @@ class WholeStageCodegenSuite extends SparkPlanTest with SharedSQLContext { wholeStageCodeGenExec.get.asInstanceOf[WholeStageCodegenExec].doCodeGen() } - test("SPARK-21871 turn off whole-stage codegen if bytecode size goes over hugeMethodLimit") { - withSQLConf(SQLConf.CODEGEN_HUGE_METHOD_LIMIT.key -> "8000") { - val (_, code) = genGroupByCodeGenContext(20) - val errMsg = intercept[ExecutionException] { - CodeGenerator.compile(code) - }.getMessage - assert(errMsg.contains("this value went over the limit `spark.sql.codegen.hugeMethodLimit`")) - } + test("SPARK-21871 check if we can get large code size when compiling too long functions") { + val (_, codeWithShortFunctions) = genGroupByCodeGenContext(3) + val (_, maxCodeSize1) = CodeGenerator.compile(codeWithShortFunctions) + assert(maxCodeSize1 < SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.defaultValue.get) + val (_, codeWithLongFunctions) = genGroupByCodeGenContext(20) + val (_, maxCodeSize2) = CodeGenerator.compile(codeWithLongFunctions) + assert(maxCodeSize2 > SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.defaultValue.get) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala index 23b1db0f2257..46fe489be657 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala @@ -24,6 +24,7 @@ import org.apache.spark.memory.{StaticMemoryManager, TaskMemoryManager} import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.execution.joins.LongToUnsafeRowMap import org.apache.spark.sql.execution.vectorized.AggregateHashMap +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{LongType, StructType} import org.apache.spark.unsafe.Platform import org.apache.spark.unsafe.hash.Murmur3_x86_32 @@ -301,10 +302,10 @@ class AggregateBenchmark extends BenchmarkBase { */ } - ignore("max function length of codegen") { + ignore("max function bytecode size of wholestagecodegen") { val N = 20 << 15 - val benchmark = new Benchmark("max function length of codegen", N) + val benchmark = new Benchmark("max function bytecode size", N) def f(): Unit = sparkSession.range(N) .selectExpr( "id", @@ -334,14 +335,14 @@ class AggregateBenchmark extends BenchmarkBase { .collect() benchmark.addCase(s"hugeMethodLimit = 8000") { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") - sparkSession.conf.set("spark.sql.codegen.hugeMethodLimit", "8000") + sparkSession.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") + sparkSession.conf.set(SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key, "8000") f() } benchmark.addCase(s"hugeMethodLimit = 16000") { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") - sparkSession.conf.set("spark.sql.codegen.hugeMethodLimit", "16000") + sparkSession.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") + sparkSession.conf.set(SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key, "16000") f() } @@ -350,10 +351,11 @@ class AggregateBenchmark extends BenchmarkBase { /* Java HotSpot(TM) 64-Bit Server VM 1.8.0_31-b13 on Mac OS X 10.10.2 Intel(R) Core(TM) i7-4578U CPU @ 3.00GHz - max function length of codegen: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + + max function bytecode size: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - hugeMethodLimit = 8000 704 / 868 0.9 1074.2 1.0X - hugeMethodLimit = 16000 2447 / 2457 0.3 3733.1 0.3X + hugeMethodLimit = 8000 1043 / 1159 0.6 1591.5 1.0X + hugeMethodLimit = 16000 3908 / 3996 0.2 5962.9 0.3X */ } From dfde49bcc487ecbc0135cd301e8d9c3ad17921be Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Wed, 4 Oct 2017 08:03:49 +0900 Subject: [PATCH 11/14] Fix minor issues --- .../sql/catalyst/expressions/codegen/CodeGenerator.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 3fff900af743..f9c5ef843908 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -32,7 +32,7 @@ import org.codehaus.commons.compiler.CompileException import org.codehaus.janino.{ByteArrayClassLoader, ClassBodyEvaluator, JaninoRuntimeException, SimpleCompiler} import org.codehaus.janino.util.ClassFile -import org.apache.spark.{SparkContext, SparkEnv, TaskContext, TaskKilledException} +import org.apache.spark.{SparkEnv, TaskContext, TaskKilledException} import org.apache.spark.executor.InputMetrics import org.apache.spark.internal.Logging import org.apache.spark.metrics.source.CodegenMetrics @@ -1012,6 +1012,8 @@ object CodeGenerator extends Logging { /** * Compile the Java source code into a Java class, using Janino. + * + * @return a pair of a generated class and the max bytecode size of generated functions. */ def compile(code: CodeAndComment): (GeneratedClass, Int) = try { cache.get(code) From fca22b767fddb061303cddd4e06c87130b1b32dc Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Wed, 4 Oct 2017 10:48:35 +0900 Subject: [PATCH 12/14] Fix --- .../spark/sql/execution/WholeStageCodegenExec.scala | 8 +++++--- .../spark/sql/execution/WholeStageCodegenSuite.scala | 8 ++++---- 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index f129cffdf843..d06deb190849 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -392,9 +392,11 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co // Check if compiled code has a too large function if (maxCodeSize > sqlContext.conf.hugeMethodLimit) { - logWarning(s"Found too long generated codes: the bytecode size was $maxCodeSize and " + - s"this value went over the limit ${sqlContext.conf.hugeMethodLimit}. To avoid this, " + - s"you can the limit ${SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key} higher:\n$treeString") + logWarning(s"Found too long generated codes and JIT optimization might not work: " + + s"the bytecode size was $maxCodeSize, this value went over the limit " + + s"${sqlContext.conf.hugeMethodLimit}, and the whole-stage codegen was disable " + + s"for this plan. To avoid this, you can set the limit " + + s"${SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key} higher:\n$treeString") return child.execute() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index a926fef8a4e6..595f18bbb650 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -151,7 +151,7 @@ class WholeStageCodegenSuite extends SparkPlanTest with SharedSQLContext { } } - def genGroupByCodeGenContext(caseNum: Int): (CodegenContext, CodeAndComment) = { + def genGroupByCodeGenContext(caseNum: Int): CodeAndComment = { val caseExp = (1 to caseNum).map { i => s"case when id > $i and id <= ${i + 1} then 1 else 0 end as v$i" }.toList @@ -176,14 +176,14 @@ class WholeStageCodegenSuite extends SparkPlanTest with SharedSQLContext { }) assert(wholeStageCodeGenExec.isDefined) - wholeStageCodeGenExec.get.asInstanceOf[WholeStageCodegenExec].doCodeGen() + wholeStageCodeGenExec.get.asInstanceOf[WholeStageCodegenExec].doCodeGen()._2 } test("SPARK-21871 check if we can get large code size when compiling too long functions") { - val (_, codeWithShortFunctions) = genGroupByCodeGenContext(3) + val codeWithShortFunctions = genGroupByCodeGenContext(3) val (_, maxCodeSize1) = CodeGenerator.compile(codeWithShortFunctions) assert(maxCodeSize1 < SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.defaultValue.get) - val (_, codeWithLongFunctions) = genGroupByCodeGenContext(20) + val codeWithLongFunctions = genGroupByCodeGenContext(20) val (_, maxCodeSize2) = CodeGenerator.compile(codeWithLongFunctions) assert(maxCodeSize2 > SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.defaultValue.get) } From 433f13b03e995bbb47641b44ed1f7961cc4ea2ec Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Wed, 4 Oct 2017 13:22:15 +0900 Subject: [PATCH 13/14] Fix --- .../sql/execution/WholeStageCodegenSuite.scala | 10 ++++------ .../benchmark/AggregateBenchmark.scala | 18 ++++++++++++------ 2 files changed, 16 insertions(+), 12 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index 595f18bbb650..aaa77b3ee620 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -17,10 +17,8 @@ package org.apache.spark.sql.execution -import java.util.concurrent.ExecutionException - import org.apache.spark.sql.Row -import org.apache.spark.sql.catalyst.expressions.codegen.{CodeAndComment, CodegenContext, CodeGenerator} +import org.apache.spark.sql.catalyst.expressions.codegen.{CodeAndComment, CodeGenerator} import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec import org.apache.spark.sql.execution.joins.SortMergeJoinExec @@ -151,7 +149,7 @@ class WholeStageCodegenSuite extends SparkPlanTest with SharedSQLContext { } } - def genGroupByCodeGenContext(caseNum: Int): CodeAndComment = { + def genGroupByCode(caseNum: Int): CodeAndComment = { val caseExp = (1 to caseNum).map { i => s"case when id > $i and id <= ${i + 1} then 1 else 0 end as v$i" }.toList @@ -180,10 +178,10 @@ class WholeStageCodegenSuite extends SparkPlanTest with SharedSQLContext { } test("SPARK-21871 check if we can get large code size when compiling too long functions") { - val codeWithShortFunctions = genGroupByCodeGenContext(3) + val codeWithShortFunctions = genGroupByCode(3) val (_, maxCodeSize1) = CodeGenerator.compile(codeWithShortFunctions) assert(maxCodeSize1 < SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.defaultValue.get) - val codeWithLongFunctions = genGroupByCodeGenContext(20) + val codeWithLongFunctions = genGroupByCode(20) val (_, maxCodeSize2) = CodeGenerator.compile(codeWithLongFunctions) assert(maxCodeSize2 > SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.defaultValue.get) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala index 46fe489be657..aca1be01fa3d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala @@ -334,15 +334,20 @@ class AggregateBenchmark extends BenchmarkBase { .sum() .collect() - benchmark.addCase(s"hugeMethodLimit = 8000") { iter => + benchmark.addCase("codegen = F") { iter => + sparkSession.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "false") + f() + } + + benchmark.addCase("codegen = T hugeMethodLimit = 10000") { iter => sparkSession.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") - sparkSession.conf.set(SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key, "8000") + sparkSession.conf.set(SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key, "10000") f() } - benchmark.addCase(s"hugeMethodLimit = 16000") { iter => + benchmark.addCase("codegen = T hugeMethodLimit = 1500") { iter => sparkSession.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") - sparkSession.conf.set(SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key, "16000") + sparkSession.conf.set(SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key, "1500") f() } @@ -354,8 +359,9 @@ class AggregateBenchmark extends BenchmarkBase { max function bytecode size: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - hugeMethodLimit = 8000 1043 / 1159 0.6 1591.5 1.0X - hugeMethodLimit = 16000 3908 / 3996 0.2 5962.9 0.3X + codegen = F 709 / 803 0.9 1082.1 1.0X + codegen = T hugeMethodLimit = 10000 3485 / 3548 0.2 5317.7 0.2X + codegen = T hugeMethodLimit = 1500 636 / 701 1.0 969.9 1.1X */ } From 09ae105c101a1b31d2a8873976c01590c50411d2 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Wed, 4 Oct 2017 14:25:23 +0900 Subject: [PATCH 14/14] Fix --- .../apache/spark/sql/execution/WholeStageCodegenExec.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index d06deb190849..9073d599ac43 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -394,9 +394,9 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co if (maxCodeSize > sqlContext.conf.hugeMethodLimit) { logWarning(s"Found too long generated codes and JIT optimization might not work: " + s"the bytecode size was $maxCodeSize, this value went over the limit " + - s"${sqlContext.conf.hugeMethodLimit}, and the whole-stage codegen was disable " + - s"for this plan. To avoid this, you can set the limit " + - s"${SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key} higher:\n$treeString") + s"${sqlContext.conf.hugeMethodLimit}, and the whole-stage codegen was disabled " + + s"for this plan. To avoid this, you can raise the limit " + + s"${SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key}:\n$treeString") return child.execute() }