Github user cloud-fan commented on a diff in the pull request:
https://github.com/apache/spark/pull/19813#discussion_r156086133
--- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala
---
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.codegen
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.types.DataType
+
+/**
+ * Defines util methods used in expression code generation.
+ */
+object ExpressionCodegen {
+
+ // Type alias for a tuple representing the data type and nullable for an expression.
+ type ExprProperty = (DataType, Boolean)
+
+ /**
+ * Given an expression, returns the all necessary parameters to evaluate it, so the
generated
+ * code of this expression can be split in a function.
+ * The 1st string in returned tuple is the parameter strings used to call the function.
+ * The 2nd string in returned tuple is the parameter strings used to declare the function.
+ *
+ * Returns `None` if it can't produce valid parameters.
+ *
+ * Params to include:
+ * 1. Evaluated columns referred by this, children or deferred expressions.
+ * 2. Rows referred by this, children or deferred expressions.
+ * 3. Eliminated subexpressions referred by children expressions.
+ */
+ def getExpressionInputParams(
+ ctx: CodegenContext,
+ expr: Expression): Option[(Seq[String], Seq[String])] = {
+ val (inputAttrs, inputVars) = getInputVarsForChildren(ctx, expr)
+ val paramsFromColumns = prepareFunctionParams(ctx, inputAttrs, inputVars)
+
+ val subExprs = getSubExprInChildren(ctx, expr)
+ val subExprCodes = getSubExprCodes(ctx, subExprs)
+ val subAttrs = subExprs.map(e => (e.dataType, e.nullable))
+ val paramsFromSubExprs = prepareFunctionParams(ctx, subAttrs, subExprCodes)
+
+ val inputRows = ctx.INPUT_ROW +: getInputRowsForChildren(ctx, expr)
+ val paramsFromRows = inputRows.distinct.filter(_ != null).map { row =>
+ (row, s"InternalRow $row")
+ }
+
+ val paramsLength = getParamLength(ctx, inputAttrs ++ subAttrs) + paramsFromRows.length
+ // Maximum allowed parameter number for Java's method descriptor.
+ if (paramsLength > 255) {
+ None
+ } else {
+ val allParams = (paramsFromRows ++ paramsFromColumns ++ paramsFromSubExprs).unzip
+ val callParams = allParams._1.distinct
+ val declParams = allParams._2.distinct
+ Some((callParams, declParams))
+ }
+ }
+
+ /**
+ * Returns the eliminated subexpressions in the children expressions.
+ */
+ def getSubExprInChildren(ctx: CodegenContext, expr: Expression): Seq[Expression] =
{
+ expr.children.flatMap { child =>
+ child.collect {
+ case e if ctx.subExprEliminationExprs.contains(e) => e
+ }
+ }.distinct
+ }
+
+ /**
+ * A small helper function to return `ExprCode`s that represent subexpressions.
+ */
+ def getSubExprCodes(ctx: CodegenContext, subExprs: Seq[Expression]): Seq[ExprCode]
= {
--- End diff --
in normal codegen the sub-expr `value` and `isNull` are all global variables.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org
|