maropu commented on a change in pull request #20965: [SPARK-21870][SQL] Split 
aggregation code into small functions
URL: https://github.com/apache/spark/pull/20965#discussion_r319311479
 
 

 ##########
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala
 ##########
 @@ -255,41 +260,138 @@ case class HashAggregateExec(
      """.stripMargin
   }
 
+  // Splits aggregate code into small functions because the most of JVM 
implementations
+  // can not compile too long functions.
+  //
+  // Note: The difference from `CodeGenerator.splitExpressions` is that we 
define an individual
+  // function for each aggregation function (e.g., SUM and AVG). For example, 
in a query
+  // `SELECT SUM(a), AVG(a) FROM VALUES(1) t(a)`, we define two functions
+  // for `SUM(a)` and `AVG(a)`.
+  private def splitAggregateExpressions(
+      ctx: CodegenContext,
+      aggNames: Seq[String],
+      aggBufferUpdatingExprs: Seq[Seq[Expression]],
+      aggCodeBlocks: Seq[Block],
+      subExprs: Map[Expression, SubExprEliminationState]): Option[String] = {
+    val inputVars = aggBufferUpdatingExprs.map { aggExprsForOneFunc =>
+      val inputVarsForOneFunc = aggExprsForOneFunc.map(
+        CodeGenerator.getLocalInputVariableValues(ctx, _, subExprs)).reduce(_ 
++ _).toSeq
+      val paramLength = 
CodeGenerator.calculateParamLengthFromExprValues(inputVarsForOneFunc)
+
+      // Checks if a parameter length for the `aggExprsForOneFunc` does not go 
over the JVM limit
+      if (CodeGenerator.isValidParamLength(paramLength)) {
+        Some(inputVarsForOneFunc)
+      } else {
+        None
+      }
+    }
+
+    // Checks if all the aggregate code can be split into pieces.
+    // If the parameter length of at lease one `aggExprsForOneFunc` goes over 
the limit,
+    // we totally give up splitting aggregate code.
+    if (inputVars.forall(_.isDefined)) {
+      val splitCodes = inputVars.flatten.zipWithIndex.map { case (args, i) =>
+        val doAggVal = ctx.freshName(s"doAggregateVal_${aggNames(i)}")
+        val argList = args.map(v => s"${v.javaType.getName} 
${v.variableName}").mkString(", ")
+        val doAggValFuncName = ctx.addNewFunction(doAggVal,
+          s"""
+             |private void $doAggVal($argList) throws java.io.IOException {
+             |  ${aggCodeBlocks(i)}
+             |}
+           """.stripMargin)
+
+        val inputVariables = args.map(_.variableName).mkString(", ")
+        s"$doAggValFuncName($inputVariables);"
+      }
+      Some(splitCodes.mkString("\n").trim)
+    } else {
+      val errMsg = "Failed to split aggregate code into small functions 
because the parameter " +
+        "length of at least one split function went over the JVM limit: " +
+        CodeGenerator.MAX_JVM_METHOD_PARAMS_LENGTH
+      if (Utils.isTesting) {
+        throw new IllegalStateException(errMsg)
+      } else {
+        logInfo(errMsg)
+        None
+      }
+    }
+  }
+
   private def doConsumeWithoutKeys(ctx: CodegenContext, input: Seq[ExprCode]): 
String = {
     // only have DeclarativeAggregate
     val functions = 
aggregateExpressions.map(_.aggregateFunction.asInstanceOf[DeclarativeAggregate])
     val inputAttrs = functions.flatMap(_.aggBufferAttributes) ++ child.output
-    val updateExpr = aggregateExpressions.flatMap { e =>
+    // To individually generate code for each aggregate function, an element 
in `updateExprs` holds
+    // all the expressions for the buffer of an aggregation function.
+    val updateExprs = aggregateExpressions.map { e =>
       e.mode match {
         case Partial | Complete =>
           
e.aggregateFunction.asInstanceOf[DeclarativeAggregate].updateExpressions
         case PartialMerge | Final =>
           
e.aggregateFunction.asInstanceOf[DeclarativeAggregate].mergeExpressions
       }
     }
-    ctx.currentVars = bufVars ++ input
-    val boundUpdateExpr = bindReferences(updateExpr, inputAttrs)
-    val subExprs = 
ctx.subexpressionEliminationForWholeStageCodegen(boundUpdateExpr)
+    ctx.currentVars = bufVars.flatten ++ input
+    val boundUpdateExprs = updateExprs.map { updateExprsForOneFunc =>
+      bindReferences(updateExprsForOneFunc, inputAttrs)
+    }
+    val subExprs = 
ctx.subexpressionEliminationForWholeStageCodegen(boundUpdateExprs.flatten)
     val effectiveCodes = subExprs.codes.mkString("\n")
-    val aggVals = ctx.withSubExprEliminationExprs(subExprs.states) {
-      boundUpdateExpr.map(_.genCode(ctx))
+    val aggVals = boundUpdateExprs.map { boundUpdateExprsForOneFunc =>
+      ctx.withSubExprEliminationExprs(subExprs.states) {
+        boundUpdateExprsForOneFunc.map(_.genCode(ctx))
+      }
     }
-    // aggregate buffer should be updated atomic
-    val updates = aggVals.zipWithIndex.map { case (ev, i) =>
-      s"""
-         | ${bufVars(i).isNull} = ${ev.isNull};
-         | ${bufVars(i).value} = ${ev.value};
+
+    val aggNames = functions.map(_.prettyName)
+    val aggCodeBlocks = aggVals.zipWithIndex.map { case (aggValsForOneFunc, i) 
=>
+      val bufVarsForOneFunc = bufVars(i)
+      // All the update code for aggregation buffers should be placed in the 
end
+      // of each aggregation function code.
+      val updates = aggValsForOneFunc.zip(bufVarsForOneFunc).map { case (ev, 
bufVar) =>
+        s"""
+           |${bufVar.isNull} = ${ev.isNull};
+           |${bufVar.value} = ${ev.value};
+         """.stripMargin
+      }
+      code"""
+         |// do aggregate for ${aggNames(i)}
+         |// evaluate aggregate function
+         |${evaluateVariables(aggValsForOneFunc)}
+         |// update aggregation buffers
+         |${updates.mkString("\n").trim}
        """.stripMargin
     }
-    s"""
-       | // do aggregate
-       | // common sub-expressions
-       | $effectiveCodes
-       | // evaluate aggregate function
-       | ${evaluateVariables(aggVals)}
-       | // update aggregation buffer
-       | ${updates.mkString("\n").trim}
-     """.stripMargin
+
+    lazy val nonSplitAggCode = {
+       s"""
+         |// do aggregate
+         |// common sub-expressions
+         |$effectiveCodes
+         |// evaluate aggregate functions and update aggregation buffers
+         |${aggCodeBlocks.fold(EmptyBlock)(_ + _)}
+       """.stripMargin
+    }
+
+    if (conf.codegenSplitAggregateFunc &&
+        aggCodeBlocks.map(_.length).sum > conf.methodSplitThreshold) {
+      val splitAggCode = splitAggregateExpressions(
+        ctx, aggNames, boundUpdateExprs, aggCodeBlocks, subExprs.states)
+
+      splitAggCode.map { updateAggValCode =>
 
 Review comment:
   renamed to `updateAggCode`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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

Reply via email to