Github user viirya commented on a diff in the pull request: https://github.com/apache/spark/pull/18931#discussion_r136742282 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala --- @@ -149,14 +149,146 @@ trait CodegenSupport extends SparkPlan { ctx.freshNamePrefix = parent.variablePrefix val evaluated = evaluateRequiredVariables(output, inputVars, parent.usedInputs) + + // Under certain conditions, we can put the logic to consume the rows of this operator into + // another function. So we can prevent a generated function too long to be optimized by JIT. + // The conditions: + // 1. The parent uses all variables in output. we can't defer variable evaluation when consume + // in another function. + // 2. The output variables are not empty. If it's empty, we don't bother to do that. + // 3. We don't use row variable. The construction of row uses deferred variable evaluation. We + // can't do it. + // 4. The number of output variables must less than maximum number of parameters in Java method + // declaration. + val requireAllOutput = output.forall(parent.usedInputs.contains(_)) + val consumeFunc = + if (row == null && outputVars.nonEmpty && requireAllOutput && outputVars.length < 255) { + constructDoConsumeFunction(ctx, inputVars) --- End diff -- I was thinking to check it. But the whole-stage codegen is a non-breaking processing which produce/consume calls are embeded together. You don't have a break to check the function length here. Actually I think it should have no negative effect to split consume functions always. From the benchmarking numbers, looks it shows no harm to normal queries.
--- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org