You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by viirya <gi...@git.apache.org> on 2017/11/24 09:59:44 UTC

[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

GitHub user viirya opened a pull request:

    https://github.com/apache/spark/pull/19813

    [SPARK-22600][SQL] Fix 64kb limit for deeply nested expressions under wholestage codegen

    ## What changes were proposed in this pull request?
    
    SPARK-22543 fixes the 64kb compile error for deeply nested expression for non-wholestage codegen. This PR extends it to support wholestage codegen.
    
    ## How was this patch tested?
    
    Added test.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/viirya/spark-1 reduce-expr-code-for-wholestage

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/19813.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #19813
    
----
commit 1cf6a482ec378964af26dc576a2b819c230558d2
Author: Liang-Chi Hsieh <vi...@gmail.com>
Date:   2017-11-24T02:16:47Z

    Support wholestage codegen for reducing expression codes to prevent 64k limit.

----


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156057427
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -1001,16 +1039,25 @@ class CodegenContext {
         commonExprs.foreach { e =>
           val expr = e.head
           val fnName = freshName("evalExpr")
    -      val isNull = s"${fnName}IsNull"
    +      val isNull = if (expr.nullable) {
    +        s"${fnName}IsNull"
    +      } else {
    +        ""
    +      }
           val value = s"${fnName}Value"
     
           // Generate the code for this expression tree and wrap it in a function.
           val eval = expr.genCode(this)
    +      val nullValue = if (expr.nullable) {
    --- End diff --
    
    nit: `assignIsNull`


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84275 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84275/testReport)** for PR 19813 at commit [`8c7f749`](https://github.com/apache/spark/commit/8c7f7496e610fdf4b512c57efd108ccf0238b126).


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154862787
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu 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 paramsFromSubExprs = prepareFunctionParams(ctx, subExprs, subExprCodes)
    --- End diff --
    
    Oh, then it is no. Firstly we don't try to eliminate subexpressions before splitting codes now. Secondly, `splitExpressions` doesn't support subexpression elimination too.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154573755
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -1001,16 +1017,25 @@ class CodegenContext {
         commonExprs.foreach { e =>
           val expr = e.head
           val fnName = freshName("evalExpr")
    -      val isNull = s"${fnName}IsNull"
    +      val isNull = if (expr.nullable) {
    +        s"${fnName}IsNull"
    +      } else {
    +        ""
    +      }
           val value = s"${fnName}Value"
     
           // Generate the code for this expression tree and wrap it in a function.
           val eval = expr.genCode(this)
    +      val nullValue = if (expr.nullable) {
    +        s"$isNull = ${eval.isNull};"
    --- End diff --
    
    nit:
    ```
    val setIsNull = if (expr.nullable) {
      s"${fnName}IsNull = ${eval.isNull};"
    } else ""
    ```


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    > it also means the chained evaluation of expressions needs to be run at every occurrence.
    
    We can introduce some mechanism to save statement to local variables if it's going to be re-computed. A possible benefit is reducing code size. Anyway I think this is a valid possibility to improve the codegen framework and we should not totally give it up.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    can we have a google doc so that we can leave comments inline? thanks!


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    I wrote a design doc at https://paper.dropbox.com/doc/Split-deeply-nested-expressions-under-wholestage-codegen-WXkQ9iIlN3zkGdn8MHgiW
    
    The design is based on what did in this PR.
    
    Please give me feedbacks if you have time to go through it. Thank you.
    
    cc @cloud-fan @kiszk @mgaido91 @maropu @gatorsmile 


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154267761
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -1028,12 +1053,18 @@ class CodegenContext {
           //   2. Less code.
           // Currently, we will do this for all non-leaf only expression trees (i.e. expr trees with
           // at least two nodes) as the cost of doing it is expected to be low.
    -      addMutableState(JAVA_BOOLEAN, isNull, s"$isNull = false;")
    +      if (expr.nullable) {
    +        addMutableState(JAVA_BOOLEAN, isNull)
    +      }
           addMutableState(javaType(expr.dataType), value,
             s"$value = ${defaultValue(expr.dataType)};")
    --- End diff --
    
    Yeah, I think so.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r155523874
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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.
    --- End diff --
    
    ah i see, thanks for the explanation!


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84641 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84641/testReport)** for PR 19813 at commit [`7fc3e04`](https://github.com/apache/spark/commit/7fc3e04b9894214e67227beeb14165610e4887d2).


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156065731
  
    --- 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)
    --- End diff --
    
    nit: I feel it's more readable to first handle sub-expr, then input var, finally input row.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84370 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84370/testReport)** for PR 19813 at commit [`98850a6`](https://github.com/apache/spark/commit/98850a67c99b02b9ba98ec544f7d0d5671142716).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84260 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84260/testReport)** for PR 19813 at commit [`8c7f749`](https://github.com/apache/spark/commit/8c7f7496e610fdf4b512c57efd108ccf0238b126).


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156115050
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala ---
    @@ -105,6 +105,12 @@ abstract class Expression extends TreeNode[Expression] {
           val isNull = ctx.freshName("isNull")
           val value = ctx.freshName("value")
           val eval = doGenCode(ctx, ExprCode("", isNull, value))
    +      eval.isNull = if (this.nullable) eval.isNull else "false"
    --- End diff --
    
    any findings for the more aggressive approach? `val isNull = if (this.nullable) ...`


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156238903
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala ---
    @@ -105,6 +105,12 @@ abstract class Expression extends TreeNode[Expression] {
           val isNull = ctx.freshName("isNull")
           val value = ctx.freshName("value")
           val eval = doGenCode(ctx, ExprCode("", isNull, value))
    +      eval.isNull = if (this.nullable) eval.isNull else "false"
    --- End diff --
    
    Compilation error like:
    ```
    [info]   Cause: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 59, Column 14: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 59, Column 14: Identifier expected instead of 'false'
    ```
    
    Seems some expressions uses `eval.isNull` as lvalue. Do you think we should find them out and change them?
    
    



---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154880558
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu 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 paramsFromSubExprs = prepareFunctionParams(ctx, subExprs, subExprCodes)
    --- End diff --
    
    Maybe we should move the supporting of subexpression elimination in `splitExpressions` to a new PR and merge that first. Reviewing 2 non-trivial optimizations at once is hard.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156552256
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala ---
    @@ -105,6 +105,12 @@ abstract class Expression extends TreeNode[Expression] {
           val isNull = ctx.freshName("isNull")
           val value = ctx.freshName("value")
           val eval = doGenCode(ctx, ExprCode("", isNull, value))
    +      eval.isNull = if (this.nullable) eval.isNull else "false"
    --- End diff --
    
    BTW after some glance, this may not worth. It's annoying to check `nullable` and deal with how to assign `isNull` in `Expressin.doGenCode`


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156055243
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -55,8 +55,42 @@ import org.apache.spark.util.{ParentClassLoader, Utils}
      *                 to null.
      * @param value A term for a (possibly primitive) value of the result of the evaluation. Not
      *              valid if `isNull` is set to `true`.
    + * @param inputRow A term that holds the input row name when generating this code.
    + * @param inputVars A list of [[ExprInputVar]] that holds input variables when generating this code.
      */
    -case class ExprCode(var code: String, var isNull: String, var value: String)
    +case class ExprCode(
    +    var code: String,
    +    var isNull: String,
    +    var value: String,
    +    var inputRow: String = null,
    +    var inputVars: Seq[ExprInputVar] = Seq.empty) {
    --- End diff --
    
    do we have a requirement that only one of them can be not null/Nil?


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    This patch needs #19800 to fix another issue in codegen.


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    `splitExpressions` is the most common way we use in the codegen framework to deal with large code. If we can't make it work with whole stage codegen, we are not making many values.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84769 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84769/testReport)** for PR 19813 at commit [`f35974e`](https://github.com/apache/spark/commit/f35974e1dfb47387dc952d30a55eee0354bdea63).


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154588045
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala ---
    @@ -108,7 +108,10 @@ private[sql] trait ColumnarBatchScan extends CodegenSupport {
              |}""".stripMargin)
     
         ctx.currentVars = null
    +    // `rowIdx` isn't in `ctx.currentVars`. If the expressions are split later, we can't track it.
    +    // So making it as global variable.
    --- End diff --
    
    I think it works, although it feels a bit hacky. Like:
    
    ```scala
      val rowidx = ctx.freshName("rowIdx")
      val rowidxExpr = AttributeReference("rowIdx", IntegerType, nullable = false)()
      val columnsBatchInput = (output zip colVars).map { case (attr, colVar) =>
        val exprCode = genCodeColumnVector(ctx, colVar, rowidx, attr.dataType, attr.nullable)
        exprCode.inputVars = Seq(ExprInputVar(rowidxExpr,
          ExprCode("", isNull = "false", value = rowidx)))
          exprCode
        }
      }
    ```
    
    This just adds one global variable. I think it is not a big problem?


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84642 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84642/testReport)** for PR 19813 at commit [`9443011`](https://github.com/apache/spark/commit/9443011978c32c611e950a6193f05aa666437f50).


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84239 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84239/testReport)** for PR 19813 at commit [`d051f9e`](https://github.com/apache/spark/commit/d051f9eef4d03f9027571419857f690c866dbd98).


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154951455
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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.
    --- End diff --
    
    I think the whole stage codegen framework guarantees that, if one of the `ctx.currentVars` is referenced, it must be evaluated. Did you hit cases that break it?


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154261964
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,259 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu children expressions.
    +   */
    +  def getExpressionInputParams(
    +      ctx: CodegenContext,
    +      expr: Expression): Option[(Seq[String], Seq[String])] = {
    +    val (inputAttrs, inputVars) = getInputVarsForChildren(ctx, expr)
    --- End diff --
    
    Moved.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154965647
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -55,8 +55,20 @@ import org.apache.spark.util.{ParentClassLoader, Utils}
      *                 to null.
      * @param value A term for a (possibly primitive) value of the result of the evaluation. Not
      *              valid if `isNull` is set to `true`.
    + * @param inputRow A term that holds the input row name when generating this code.
    + * @param inputVars A list of [[ExprInputVar]] that holds input variables when generating this code.
      */
    -case class ExprCode(var code: String, var isNull: String, var value: String)
    +case class ExprCode(
    +    var code: String,
    +    var isNull: String,
    +    var value: String,
    +    var inputRow: String = null,
    +    var inputVars: Seq[ExprInputVar] = Seq.empty)
    +
    +/**
    + * Represents an input variable [[ExprCode]] to an evaluation of an [[Expression]].
    + */
    +case class ExprInputVar(val expr: Expression, val exprCode: ExprCode)
    --- End diff --
    
    Ok. Removed `val`.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by mgaido91 <gi...@git.apache.org>.
Github user mgaido91 commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    I am sorry, but I cannot give a precise answer. I can't think of any case where this can happen. There are many places where we assume that what we get is a literal  or a variable name. But there may be some cases I am not aware of where this is not true. I think that making the proposed contract is the easiest thing to do. Otherwise we might have to revisit the whole code generation method.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154576646
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu 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 paramsFromSubExprs = prepareFunctionParams(ctx, subExprs, subExprCodes)
    --- End diff --
    
    do we support subexpression elimination when splitting codes to methods in the non-whole-stage-codegen path?


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by maropu <gi...@git.apache.org>.
Github user maropu commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    LGTM, great work!


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156284378
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala ---
    @@ -105,6 +105,12 @@ abstract class Expression extends TreeNode[Expression] {
           val isNull = ctx.freshName("isNull")
           val value = ctx.freshName("value")
           val eval = doGenCode(ctx, ExprCode("", isNull, value))
    +      eval.isNull = if (this.nullable) eval.isNull else "false"
    --- End diff --
    
    at least we should do it in a new PR. do you wanna do it after this one get merge?


---

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


[GitHub] spark pull request #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply...

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r153852136
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala ---
    @@ -109,6 +109,7 @@ private[sql] trait ColumnarBatchScan extends CodegenSupport {
     
         ctx.currentVars = null
         val rowidx = ctx.freshName("rowIdx")
    +    ctx.addMutableState(ctx.JAVA_INT, rowidx)
    --- End diff --
    
    Do we have to make this global? If so, it would be good to add a comment since it is not easy to understand where `$rowidx` is used.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r155543898
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines util methods used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu 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 paramsFromSubExprs = prepareFunctionParams(ctx, subExprs, 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, subExprs) + 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] = {
    +    subExprs.map { subExpr =>
    +      val state = ctx.subExprEliminationExprs(subExpr)
    +      ExprCode(code = "", value = state.value, isNull = state.isNull)
    +    }
    +  }
    +
    +  /**
    +   * Retrieves previous input rows referred by children and deferred expressions.
    +   */
    +  def getInputRowsForChildren(ctx: CodegenContext, expr: Expression): Seq[String] = {
    +    expr.children.flatMap(getInputRows(ctx, _)).distinct
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previous input rows referred by it or deferred expressions
    +   * which are needed to evaluate it.
    +   */
    +  def getInputRows(ctx: CodegenContext, child: Expression): Seq[String] = {
    +    child.flatMap {
    +      // An expression directly evaluates on current input row.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars == null ||
    +          ctx.currentVars(ordinal) == null =>
    +        Seq(ctx.INPUT_ROW)
    +
    +      // An expression which is not evaluated yet. Tracks down to find input rows.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal).code != "" =>
    --- End diff --
    
    Yea, it means the bound reference is based on an evaluated variable.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r155574067
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines util methods used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu 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 paramsFromSubExprs = prepareFunctionParams(ctx, subExprs, 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, subExprs) + 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] = {
    +    subExprs.map { subExpr =>
    +      val state = ctx.subExprEliminationExprs(subExpr)
    +      ExprCode(code = "", value = state.value, isNull = state.isNull)
    +    }
    +  }
    +
    +  /**
    +   * Retrieves previous input rows referred by children and deferred expressions.
    +   */
    +  def getInputRowsForChildren(ctx: CodegenContext, expr: Expression): Seq[String] = {
    +    expr.children.flatMap(getInputRows(ctx, _)).distinct
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previous input rows referred by it or deferred expressions
    +   * which are needed to evaluate it.
    +   */
    +  def getInputRows(ctx: CodegenContext, child: Expression): Seq[String] = {
    +    child.flatMap {
    +      // An expression directly evaluates on current input row.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars == null ||
    +          ctx.currentVars(ordinal) == null =>
    +        Seq(ctx.INPUT_ROW)
    +
    +      // An expression which is not evaluated yet. Tracks down to find input rows.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal).code != "" =>
    +        trackDownRow(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down input rows referred by the generated code snippet.
    +   */
    +  def trackDownRow(ctx: CodegenContext, exprCode: ExprCode): Seq[String] = {
    +    val exprCodes = mutable.Queue[ExprCode](exprCode)
    +    val inputRows = mutable.ArrayBuffer.empty[String]
    +
    +    while (exprCodes.nonEmpty) {
    +      val curExprCode = exprCodes.dequeue()
    +      if (curExprCode.inputRow != null) {
    +        inputRows += curExprCode.inputRow
    +      }
    +      curExprCode.inputVars.foreach { inputVar =>
    +        if (inputVar.exprCode.code != "") {
    +          exprCodes.enqueue(inputVar.exprCode)
    +        }
    +      }
    +    }
    +    inputRows
    +  }
    +
    +  /**
    +   * Retrieves previously evaluated columns referred by children and deferred expressions.
    +   * Returned tuple contains the list of expressions and the list of generated codes.
    +   */
    +  def getInputVarsForChildren(
    +      ctx: CodegenContext,
    +      expr: Expression): (Seq[Expression], Seq[ExprCode]) = {
    +    expr.children.flatMap(getInputVars(ctx, _)).distinct.unzip
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previously evaluated columns referred by it or
    +   * deferred expressions which are needed to evaluate it.
    +   */
    +  def getInputVars(ctx: CodegenContext, child: Expression): Seq[(Expression, ExprCode)] = {
    +    if (ctx.currentVars == null) {
    +      return Seq.empty
    +    }
    +
    +    child.flatMap {
    +      // An evaluated variable.
    +      case b @ BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null &&
    +          ctx.currentVars(ordinal).code == "" =>
    +        Seq((b, ctx.currentVars(ordinal)))
    +
    +      // An input variable which is not evaluated yet. Tracks down to find any evaluated variables
    +      // in the expression path.
    +      // E.g., if this expression is "d = c + 1" and "c" is not evaluated. We need to track to
    +      // "c = a + b" and see if "a" and "b" are evaluated. If they are, we need to return them so
    +      // to include them into parameters, if not, we track down further.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null =>
    +        trackDownVar(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down previously evaluated columns referred by the generated code snippet.
    +   */
    +  def trackDownVar(ctx: CodegenContext, exprCode: ExprCode): Seq[(Expression, ExprCode)] = {
    +    val exprCodes = mutable.Queue[ExprCode](exprCode)
    +    val inputVars = mutable.ArrayBuffer.empty[(Expression, ExprCode)]
    +
    +    while (exprCodes.nonEmpty) {
    +      exprCodes.dequeue().inputVars.foreach { inputVar =>
    +        if (inputVar.exprCode.code == "") {
    +          inputVars += ((inputVar.expr, inputVar.exprCode))
    +        } else {
    +          exprCodes.enqueue(inputVar.exprCode)
    +        }
    +      }
    +    }
    +    inputVars
    +  }
    +
    +  /**
    +   * Helper function to calculate the size of an expression as function parameter.
    +   */
    +  def calculateParamLength(ctx: CodegenContext, input: Expression): Int = {
    +    ctx.javaType(input.dataType) match {
    +      case (ctx.JAVA_LONG | ctx.JAVA_DOUBLE) if !input.nullable => 2
    +      case ctx.JAVA_LONG | ctx.JAVA_DOUBLE => 3
    +      case _ if !input.nullable => 1
    +      case _ => 2
    +    }
    +  }
    +
    +  /**
    +   * In Java, a method descriptor is valid only if it represents method parameters with a total
    +   * length of 255 or less. `this` contributes one unit and a parameter of type long or double
    +   * contributes two units.
    +   */
    +  def getParamLength(
    +      ctx: CodegenContext,
    +      inputs: Seq[Expression],
    +      subExprs: Seq[Expression]): Int = {
    +    // Initial value is 1 for `this`.
    +    1 + (inputs ++ subExprs).distinct.map(calculateParamLength(ctx, _)).sum
    +  }
    +
    +  /**
    +   * Given the lists of input attributes and variables to this expression, returns the strings of
    +   * funtion parameters. The first is the variable names used to call the function, the second is
    +   * the parameters used to declare the function in generated code.
    +   */
    +  def prepareFunctionParams(
    +      ctx: CodegenContext,
    +      inputAttrs: Seq[Expression],
    +      inputVars: Seq[ExprCode]): Seq[(String, String)] = {
    +    inputAttrs.zip(inputVars).flatMap { case (input, ev) =>
    +      val argType = ctx.javaType(input.dataType)
    +
    +      if (!input.nullable || ev.isNull == "true" || ev.isNull == "false") {
    +        Seq((ev.value, s"$argType ${ev.value}"))
    +      } else {
    +        Seq((ev.value, s"$argType ${ev.value}"), (ev.isNull, s"boolean ${ev.isNull}"))
    --- End diff --
    
    a common strategy we use: assign `ev.value` to a temp variable, and use that temp variable to build method parameters.
    
    cc @kiszk @mgaido91 who I think have done something like this.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84756 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84756/testReport)** for PR 19813 at commit [`1251dfa`](https://github.com/apache/spark/commit/1251dfa305f4f1f8e34d7deb235bfa500d057fb4).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156552741
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala ---
    @@ -105,6 +105,12 @@ abstract class Expression extends TreeNode[Expression] {
           val isNull = ctx.freshName("isNull")
           val value = ctx.freshName("value")
           val eval = doGenCode(ctx, ExprCode("", isNull, value))
    +      eval.isNull = if (this.nullable) eval.isNull else "false"
    --- End diff --
    
    So, shall we keep it as it is or restore it back?


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156087857
  
    --- 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 --
    
    Verified, yes. We can get rid of sub-expr parameters in normal codegen.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    I'm wondering if an output statement like `a + 1` is a problem.
    
    If we only include codegen output with valid java variable names, an output like `a + 1` won't be included as parameter. We only need to make sure the creation of `ExprCode` in `genCode` has correctly linked with its input row and variables (this is done in this PR). So an output like `a + 1` depends on other input `a`. This `a` will be included as parameter if necessary.
    
    cc @cloud-fan @kiszk @mgaido91 WDYT? Or do I miss something?


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Which one is more common? A chain of arithmetic expressions? Or a deeply nested expression? I don't see strong evidence that supports statement output from the discussion. The only one possibility for now is to reducing code size. This is also for performance, not stability. On the contrary, isn't using local variable more stable? Don't forget we need to introduce other mechanism to fix the problem of statement output like re-evaluation I pointed out above.
    
    I'm not saying it is not good to support statement output. But for now, the reason to support it is very vague.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    replacing them via string is too dangerous, logically we wanna replace some nodes in AST, which needs an AST based codegen framework, or we need to refactor the current framework a little bit to do it safely.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by mgaido91 <gi...@git.apache.org>.
Github user mgaido91 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154199646
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,259 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu children expressions.
    +   */
    +  def getExpressionInputParams(
    +      ctx: CodegenContext,
    +      expr: Expression): Option[(Seq[String], Seq[String])] = {
    +    val (inputAttrs, inputVars) = getInputVarsForChildren(ctx, expr)
    +    val inputRows = ctx.INPUT_ROW +: getInputRowsForChildren(ctx, expr)
    +    val subExprs = getSubExprInChildren(ctx, expr)
    +
    +    val paramsFromRows = inputRows.distinct.filter(_ != null).map { row =>
    +      (row, s"InternalRow $row")
    +    }
    +    val paramsFromColumns = prepareFunctionParams(ctx, inputAttrs, inputVars)
    +    val paramsFromSubExprs = getParamsForSubExprs(ctx, subExprs)
    +    val paramsLength = getParamLength(ctx, inputAttrs, subExprs) + 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
    +  }
    +
    +  /**
    +   * Given the list of eliminated subexpressions used in the children expressions, returns the
    +   * strings of funtion parameters. The first is the variable names used to call the function,
    +   * the second is the parameters used to declare the function in generated code.
    +   */
    +  def getParamsForSubExprs(
    +      ctx: CodegenContext,
    +      subExprs: Seq[Expression]): Seq[(String, String)] = {
    +    subExprs.flatMap { subExpr =>
    +      val argType = ctx.javaType(subExpr.dataType)
    +
    +      val subExprState = ctx.subExprEliminationExprs(subExpr)
    +      (subExprState.value, subExprState.isNull)
    --- End diff --
    
    shouldn't this line be removed? I think it is useless..


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by mgaido91 <gi...@git.apache.org>.
Github user mgaido91 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154318313
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,259 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu children expressions.
    +   */
    +  def getExpressionInputParams(
    +      ctx: CodegenContext,
    +      expr: Expression): Option[(Seq[String], Seq[String])] = {
    +    val (inputAttrs, inputVars) = getInputVarsForChildren(ctx, expr)
    +    val inputRows = ctx.INPUT_ROW +: getInputRowsForChildren(ctx, expr)
    +    val subExprs = getSubExprInChildren(ctx, expr)
    +
    +    val paramsFromRows = inputRows.distinct.filter(_ != null).map { row =>
    +      (row, s"InternalRow $row")
    +    }
    +    val paramsFromColumns = prepareFunctionParams(ctx, inputAttrs, inputVars)
    +    val paramsFromSubExprs = getParamsForSubExprs(ctx, subExprs)
    +    val paramsLength = getParamLength(ctx, inputAttrs, subExprs) + 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
    +  }
    +
    +  /**
    +   * Given the list of eliminated subexpressions used in the children expressions, returns the
    +   * strings of funtion parameters. The first is the variable names used to call the function,
    +   * the second is the parameters used to declare the function in generated code.
    +   */
    +  def getParamsForSubExprs(
    +      ctx: CodegenContext,
    +      subExprs: Seq[Expression]): Seq[(String, String)] = {
    +    subExprs.flatMap { subExpr =>
    +      val argType = ctx.javaType(subExpr.dataType)
    +
    +      val subExprState = ctx.subExprEliminationExprs(subExpr)
    +      (subExprState.value, subExprState.isNull)
    +
    +      if (!subExpr.nullable || subExprState.isNull == "true" || subExprState.isNull == "false") {
    +        Seq((subExprState.value, s"$argType ${subExprState.value}"))
    +      } else {
    +        Seq((subExprState.value, s"$argType ${subExprState.value}"),
    +          (subExprState.isNull, s"boolean ${subExprState.isNull}"))
    +      }
    +    }.distinct
    +  }
    +
    +  /**
    +   * Retrieves previous input rows referred by children and deferred expressions.
    +   */
    +  def getInputRowsForChildren(ctx: CodegenContext, expr: Expression): Seq[String] = {
    +    expr.children.flatMap(getInputRows(ctx, _)).distinct
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previous input rows referred by it or deferred expressions
    +   * which are needed to evaluate it.
    +   */
    +  def getInputRows(ctx: CodegenContext, child: Expression): Seq[String] = {
    +    child.flatMap {
    +      // An expression directly evaluates on current input row.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars == null ||
    +          ctx.currentVars(ordinal) == null =>
    +        Seq(ctx.INPUT_ROW)
    +
    +      // An expression which is not evaluated yet. Tracks down to find input rows.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal).code != "" =>
    +        trackDownRow(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down input rows referred by the generated code snippet.
    +   */
    +  def trackDownRow(ctx: CodegenContext, exprCode: ExprCode): Seq[String] = {
    +    var exprCodes: List[ExprCode] = List(exprCode)
    +    val inputRows = mutable.ArrayBuffer.empty[String]
    +
    +    while (exprCodes.nonEmpty) {
    +      exprCodes match {
    +        case first :: others =>
    +          exprCodes = others
    +          if (first.inputRow != null) {
    +            inputRows += first.inputRow
    +          }
    +          first.inputVars.foreach { inputVar =>
    +            if (inputVar.exprCode.code != "") {
    +              exprCodes = inputVar.exprCode :: exprCodes
    +            }
    +          }
    +        case _ =>
    +      }
    +    }
    +    inputRows.toSeq
    +  }
    +
    +  /**
    +   * Retrieves previously evaluated columns referred by children and deferred expressions.
    +   * Returned tuple contains the list of expressions and the list of generated codes.
    +   */
    +  def getInputVarsForChildren(
    +      ctx: CodegenContext,
    +      expr: Expression): (Seq[Expression], Seq[ExprCode]) = {
    +    expr.children.flatMap(getInputVars(ctx, _)).distinct.unzip
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previously evaluated columns referred by it or
    +   * deferred expressions which are needed to evaluate it.
    +   */
    +  def getInputVars(ctx: CodegenContext, child: Expression): Seq[(Expression, ExprCode)] = {
    +    if (ctx.currentVars == null) {
    +      return Seq.empty
    +    }
    +
    +    child.flatMap {
    +      // An evaluated variable.
    +      case b @ BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null &&
    +          ctx.currentVars(ordinal).code == "" =>
    +        Seq((b, ctx.currentVars(ordinal)))
    +
    +      // An input variable which is not evaluated yet. Tracks down to find any evaluated variables
    +      // in the expression path.
    +      // E.g., if this expression is "d = c + 1" and "c" is not evaluated. We need to track to
    +      // "c = a + b" and see if "a" and "b" are evaluated. If they are, we need to return them so
    +      // to include them into parameters, if not, we track down further.
    +      case b @ BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null =>
    +        trackDownVar(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down previously evaluated columns referred by the generated code snippet.
    +   */
    +  def trackDownVar(ctx: CodegenContext, exprCode: ExprCode): Seq[(Expression, ExprCode)] = {
    +    var exprCodes: List[ExprCode] = List(exprCode)
    +    val inputVars = mutable.ArrayBuffer.empty[(Expression, ExprCode)]
    +
    +    while (exprCodes.nonEmpty) {
    +      exprCodes match {
    +        case first :: others =>
    +          exprCodes = others
    +          first.inputVars.foreach { inputVar =>
    +            if (inputVar.exprCode.code == "") {
    +              inputVars += ((inputVar.expr, inputVar.exprCode))
    +            } else {
    +              exprCodes = inputVar.exprCode :: exprCodes
    +            }
    +          }
    +        case _ =>
    +      }
    +    }
    +    inputVars.toSeq
    +  }
    +
    +  /**
    +   * Helper function to calculate the size of an expression as function parameter.
    +   */
    +  def calculateParamLength(ctx: CodegenContext, input: Expression): Int = {
    +    ctx.javaType(input.dataType) match {
    +      case (ctx.JAVA_LONG | ctx.JAVA_DOUBLE) if !input.nullable => 2
    --- End diff --
    
    thank you very much! :)


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84367 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84367/testReport)** for PR 19813 at commit [`0d358d6`](https://github.com/apache/spark/commit/0d358d635494199582aa6e38fdbeec0f6446c029).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84710 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84710/testReport)** for PR 19813 at commit [`2f4014f`](https://github.com/apache/spark/commit/2f4014fe7de0ae634231a5aae36e7272defa3d9e).
     * This patch **fails SparkR unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by mgaido91 <gi...@git.apache.org>.
Github user mgaido91 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154199208
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,259 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu children expressions.
    +   */
    +  def getExpressionInputParams(
    +      ctx: CodegenContext,
    +      expr: Expression): Option[(Seq[String], Seq[String])] = {
    +    val (inputAttrs, inputVars) = getInputVarsForChildren(ctx, expr)
    +    val inputRows = ctx.INPUT_ROW +: getInputRowsForChildren(ctx, expr)
    +    val subExprs = getSubExprInChildren(ctx, expr)
    +
    +    val paramsFromRows = inputRows.distinct.filter(_ != null).map { row =>
    +      (row, s"InternalRow $row")
    +    }
    +    val paramsFromColumns = prepareFunctionParams(ctx, inputAttrs, inputVars)
    +    val paramsFromSubExprs = getParamsForSubExprs(ctx, subExprs)
    +    val paramsLength = getParamLength(ctx, inputAttrs, subExprs) + 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
    +  }
    +
    +  /**
    +   * Given the list of eliminated subexpressions used in the children expressions, returns the
    +   * strings of funtion parameters. The first is the variable names used to call the function,
    +   * the second is the parameters used to declare the function in generated code.
    +   */
    +  def getParamsForSubExprs(
    +      ctx: CodegenContext,
    +      subExprs: Seq[Expression]): Seq[(String, String)] = {
    +    subExprs.flatMap { subExpr =>
    +      val argType = ctx.javaType(subExpr.dataType)
    +
    +      val subExprState = ctx.subExprEliminationExprs(subExpr)
    +      (subExprState.value, subExprState.isNull)
    +
    +      if (!subExpr.nullable || subExprState.isNull == "true" || subExprState.isNull == "false") {
    +        Seq((subExprState.value, s"$argType ${subExprState.value}"))
    +      } else {
    +        Seq((subExprState.value, s"$argType ${subExprState.value}"),
    +          (subExprState.isNull, s"boolean ${subExprState.isNull}"))
    +      }
    +    }.distinct
    +  }
    +
    +  /**
    +   * Retrieves previous input rows referred by children and deferred expressions.
    +   */
    +  def getInputRowsForChildren(ctx: CodegenContext, expr: Expression): Seq[String] = {
    +    expr.children.flatMap(getInputRows(ctx, _)).distinct
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previous input rows referred by it or deferred expressions
    +   * which are needed to evaluate it.
    +   */
    +  def getInputRows(ctx: CodegenContext, child: Expression): Seq[String] = {
    +    child.flatMap {
    +      // An expression directly evaluates on current input row.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars == null ||
    +          ctx.currentVars(ordinal) == null =>
    +        Seq(ctx.INPUT_ROW)
    +
    +      // An expression which is not evaluated yet. Tracks down to find input rows.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal).code != "" =>
    +        trackDownRow(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down input rows referred by the generated code snippet.
    +   */
    +  def trackDownRow(ctx: CodegenContext, exprCode: ExprCode): Seq[String] = {
    +    var exprCodes: List[ExprCode] = List(exprCode)
    +    val inputRows = mutable.ArrayBuffer.empty[String]
    +
    +    while (exprCodes.nonEmpty) {
    +      exprCodes match {
    +        case first :: others =>
    +          exprCodes = others
    +          if (first.inputRow != null) {
    +            inputRows += first.inputRow
    +          }
    +          first.inputVars.foreach { inputVar =>
    +            if (inputVar.exprCode.code != "") {
    +              exprCodes = inputVar.exprCode :: exprCodes
    +            }
    +          }
    +        case _ =>
    +      }
    +    }
    +    inputRows.toSeq
    +  }
    +
    +  /**
    +   * Retrieves previously evaluated columns referred by children and deferred expressions.
    +   * Returned tuple contains the list of expressions and the list of generated codes.
    +   */
    +  def getInputVarsForChildren(
    +      ctx: CodegenContext,
    +      expr: Expression): (Seq[Expression], Seq[ExprCode]) = {
    +    expr.children.flatMap(getInputVars(ctx, _)).distinct.unzip
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previously evaluated columns referred by it or
    +   * deferred expressions which are needed to evaluate it.
    +   */
    +  def getInputVars(ctx: CodegenContext, child: Expression): Seq[(Expression, ExprCode)] = {
    +    if (ctx.currentVars == null) {
    +      return Seq.empty
    +    }
    +
    +    child.flatMap {
    +      // An evaluated variable.
    +      case b @ BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null &&
    +          ctx.currentVars(ordinal).code == "" =>
    +        Seq((b, ctx.currentVars(ordinal)))
    +
    +      // An input variable which is not evaluated yet. Tracks down to find any evaluated variables
    +      // in the expression path.
    +      // E.g., if this expression is "d = c + 1" and "c" is not evaluated. We need to track to
    +      // "c = a + b" and see if "a" and "b" are evaluated. If they are, we need to return them so
    +      // to include them into parameters, if not, we track down further.
    +      case b @ BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null =>
    +        trackDownVar(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down previously evaluated columns referred by the generated code snippet.
    +   */
    +  def trackDownVar(ctx: CodegenContext, exprCode: ExprCode): Seq[(Expression, ExprCode)] = {
    +    var exprCodes: List[ExprCode] = List(exprCode)
    +    val inputVars = mutable.ArrayBuffer.empty[(Expression, ExprCode)]
    +
    +    while (exprCodes.nonEmpty) {
    +      exprCodes match {
    +        case first :: others =>
    +          exprCodes = others
    +          first.inputVars.foreach { inputVar =>
    +            if (inputVar.exprCode.code == "") {
    +              inputVars += ((inputVar.expr, inputVar.exprCode))
    +            } else {
    +              exprCodes = inputVar.exprCode :: exprCodes
    +            }
    +          }
    +        case _ =>
    +      }
    +    }
    +    inputVars.toSeq
    --- End diff --
    
    why `toSeq`?


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84208 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84208/testReport)** for PR 19813 at commit [`9f848be`](https://github.com/apache/spark/commit/9f848be45dcc294d6f27f2c6eaeed1907d36f004).


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84241 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84241/testReport)** for PR 19813 at commit [`6368702`](https://github.com/apache/spark/commit/6368702e66948e26c41300da7136dffc5b963cb6).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84371 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84371/testReport)** for PR 19813 at commit [`aa3db2e`](https://github.com/apache/spark/commit/aa3db2edca66ab04ecb8fbd54750cbd46544eb1d).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156066138
  
    --- 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] = {
    +    subExprs.map { subExpr =>
    +      val state = ctx.subExprEliminationExprs(subExpr)
    +      ExprCode(code = "", value = state.value, isNull = state.isNull)
    +    }
    +  }
    +
    +  /**
    +   * Retrieves previous input rows referred by children and deferred expressions.
    +   */
    +  def getInputRowsForChildren(ctx: CodegenContext, expr: Expression): Seq[String] = {
    +    expr.children.flatMap(getInputRows(ctx, _)).distinct
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previous input rows referred by it or deferred expressions
    +   * which are needed to evaluate it.
    +   */
    +  def getInputRows(ctx: CodegenContext, child: Expression): Seq[String] = {
    +    child.flatMap {
    +      // An expression directly evaluates on current input row.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars == null ||
    +          ctx.currentVars(ordinal) == null =>
    +        Seq(ctx.INPUT_ROW)
    +
    +      // An expression which is not evaluated yet. Tracks down to find input rows.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal).code != "" =>
    +        trackDownRow(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down input rows referred by the generated code snippet.
    +   */
    +  def trackDownRow(ctx: CodegenContext, exprCode: ExprCode): Seq[String] = {
    +    val exprCodes = mutable.Queue[ExprCode](exprCode)
    +    val inputRows = mutable.ArrayBuffer.empty[String]
    +
    +    while (exprCodes.nonEmpty) {
    +      val curExprCode = exprCodes.dequeue()
    +      if (curExprCode.inputRow != null) {
    +        inputRows += curExprCode.inputRow
    +      }
    +      curExprCode.inputVars.foreach { inputVar =>
    +        if (inputVar.exprCode.code != "") {
    +          exprCodes.enqueue(inputVar.exprCode)
    +        }
    +      }
    +    }
    +    inputRows
    +  }
    +
    +  /**
    +   * Retrieves previously evaluated columns referred by children and deferred expressions.
    +   * Returned tuple contains the list of expressions and the list of generated codes.
    +   */
    +  def getInputVarsForChildren(
    +      ctx: CodegenContext,
    +      expr: Expression): (Seq[ExprProperty], Seq[ExprCode]) = {
    +    expr.children.flatMap(getInputVars(ctx, _)).distinct.unzip
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previously evaluated columns referred by it or
    +   * deferred expressions which are needed to evaluate it.
    +   */
    +  def getInputVars(ctx: CodegenContext, child: Expression): Seq[(ExprProperty, ExprCode)] = {
    +    if (ctx.currentVars == null) {
    +      return Seq.empty
    +    }
    +
    +    child.flatMap {
    +      // An evaluated variable.
    +      case b @ BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null &&
    +          ctx.currentVars(ordinal).code == "" =>
    +        Seq(((b.dataType, b.nullable), ctx.currentVars(ordinal)))
    +
    +      // An input variable which is not evaluated yet. Tracks down to find any evaluated variables
    +      // in the expression path.
    +      // E.g., if this expression is "d = c + 1" and "c" is not evaluated. We need to track to
    +      // "c = a + b" and see if "a" and "b" are evaluated. If they are, we need to return them so
    +      // to include them into parameters, if not, we track down further.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null =>
    +        trackDownVar(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down previously evaluated columns referred by the generated code snippet.
    +   */
    +  def trackDownVar(ctx: CodegenContext, exprCode: ExprCode): Seq[(ExprProperty, ExprCode)] = {
    +    val exprCodes = mutable.Queue[ExprCode](exprCode)
    +    val inputVars = mutable.ArrayBuffer.empty[(ExprProperty, ExprCode)]
    +
    +    while (exprCodes.nonEmpty) {
    +      exprCodes.dequeue().inputVars.foreach { inputVar =>
    +        if (inputVar.exprCode.code == "") {
    +          inputVars += (((inputVar.dataType, inputVar.nullable), inputVar.exprCode))
    +        } else {
    +          exprCodes.enqueue(inputVar.exprCode)
    +        }
    +      }
    +    }
    +    inputVars
    +  }
    +
    +  /**
    +   * Helper function to calculate the size of an expression as function parameter.
    +   */
    +  def calculateParamLength(ctx: CodegenContext, input: ExprProperty): Int = {
    +    ctx.javaType(input._1) match {
    +      case (ctx.JAVA_LONG | ctx.JAVA_DOUBLE) if !input._2 => 2
    +      case ctx.JAVA_LONG | ctx.JAVA_DOUBLE => 3
    +      case _ if !input._2 => 1
    +      case _ => 2
    +    }
    +  }
    +
    +  /**
    +   * In Java, a method descriptor is valid only if it represents method parameters with a total
    +   * length of 255 or less. `this` contributes one unit and a parameter of type long or double
    +   * contributes two units.
    +   */
    +  def getParamLength(ctx: CodegenContext, inputs: Seq[ExprProperty]): Int = {
    +    // Initial value is 1 for `this`.
    +    1 + inputs.map(calculateParamLength(ctx, _)).sum
    +  }
    +
    +  /**
    +   * Given the lists of input attributes and variables to this expression, returns the strings of
    +   * funtion parameters. The first is the variable names used to call the function, the second is
    +   * the parameters used to declare the function in generated code.
    +   */
    +  def prepareFunctionParams(
    +      ctx: CodegenContext,
    +      inputAttrs: Seq[ExprProperty],
    +      inputVars: Seq[ExprCode]): Seq[(String, String)] = {
    --- End diff --
    
    how about just accepts `inputs: Seq[ExprInputVar]`?


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    @cloud-fan Ok. Thanks.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84741/
    Test FAILed.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    @mgaido91 Thanks for the comment. I agreed that to make the contract is the easiest way. If we don't make this contract, seems to me a significant change is needed.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154950034
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -55,8 +55,20 @@ import org.apache.spark.util.{ParentClassLoader, Utils}
      *                 to null.
      * @param value A term for a (possibly primitive) value of the result of the evaluation. Not
      *              valid if `isNull` is set to `true`.
    + * @param inputRow A term that holds the input row name when generating this code.
    + * @param inputVars A list of [[ExprInputVar]] that holds input variables when generating this code.
      */
    -case class ExprCode(var code: String, var isNull: String, var value: String)
    +case class ExprCode(
    +    var code: String,
    +    var isNull: String,
    +    var value: String,
    +    var inputRow: String = null,
    +    var inputVars: Seq[ExprInputVar] = Seq.empty)
    +
    +/**
    + * Represents an input variable [[ExprCode]] to an evaluation of an [[Expression]].
    + */
    +case class ExprInputVar(val expr: Expression, val exprCode: ExprCode)
    --- End diff --
    
    we don't need `val` for case class fields.


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84323 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84323/testReport)** for PR 19813 at commit [`777eb7a`](https://github.com/apache/spark/commit/777eb7a0c4db6695ee993be7b5d3b2d40c161591).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84182/
    Test FAILed.


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84241 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84241/testReport)** for PR 19813 at commit [`6368702`](https://github.com/apache/spark/commit/6368702e66948e26c41300da7136dffc5b963cb6).


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154581731
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala ---
    @@ -115,9 +120,35 @@ abstract class Expression extends TreeNode[Expression] {
         }
       }
     
    +  /**
    +   * Returns the input variables to this expression.
    +   */
    +  private def findInputVars(ctx: CodegenContext, eval: ExprCode): Seq[ExprInputVar] = {
    +    if (ctx.currentVars != null) {
    +      val boundRefs = this.collect {
    +        case b @ BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null => (ordinal, b)
    --- End diff --
    
    It is not necessarily empty. We will also record it if its `code` is not empty.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156554154
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala ---
    @@ -105,6 +105,12 @@ abstract class Expression extends TreeNode[Expression] {
           val isNull = ctx.freshName("isNull")
           val value = ctx.freshName("value")
           val eval = doGenCode(ctx, ExprCode("", isNull, value))
    +      eval.isNull = if (this.nullable) eval.isNull else "false"
    --- End diff --
    
    oh I mean the more aggressive approach may not worth, we can even simplify some `Expression.doGenCode` that try to set `isNull` to `false`.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156115286
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala ---
    @@ -61,7 +61,11 @@ case class BoundReference(ordinal: Int, dataType: DataType, nullable: Boolean)
       override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
         if (ctx.currentVars != null && ctx.currentVars(ordinal) != null) {
           val oev = ctx.currentVars(ordinal)
    -      ev.isNull = oev.isNull
    +      if (nullable) {
    --- End diff --
    
    I think this kind of optimizations are already done in https://github.com/apache/spark/pull/19813/files#diff-b3ebf3b40b9d4b6e98bb29ac8bb5aadaR108


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154984724
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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.
    --- End diff --
    
    yea, because the expressions in `ProjectExec` may only need a subset of the input attributes. For those input attributes that are not pre-evaluated, they should not be referred, or do I miss something?


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154969965
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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.
    --- End diff --
    
    In whole stage codegen framework, each operator defines the input attributes it needs, and the framework will pre-evaluate them. This means, if one of the `ctx.currentVars` is referred, it must have already been evaluated.
    
    Do you still remember which tests were failed? Maybe it's a bug.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    @kiszk Yea, I have now checked through the codegen. I didn't find the places that can cause that issue (`a + 1` as the codegen output value) yet. I may submit another PR to let us easily identify such codegen output so we can easily do adding assertions for it.
    
    As we don't use such statement as codegen output, I think the easiest approach is adding assertions. @cloud-fan WDYT?


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156288523
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -55,8 +55,45 @@ import org.apache.spark.util.{ParentClassLoader, Utils}
      *                 to null.
      * @param value A term for a (possibly primitive) value of the result of the evaluation. Not
      *              valid if `isNull` is set to `true`.
    + * @param inputRow A term that holds the input row name when generating this code.
    + * @param inputVars A list of [[ExprInputVar]] that holds input variables when generating this code.
      */
    -case class ExprCode(var code: String, var isNull: String, var value: String)
    +case class ExprCode(
    +    var code: String,
    +    var isNull: String,
    +    var value: String,
    +    var inputRow: String = null,
    +    var inputVars: Seq[ExprInputVar] = Seq.empty) {
    +
    +  // Returns true if this value is a literal.
    +  def isLiteral(): Boolean = {
    +    assert(value.nonEmpty, "ExprCode.value can't be empty string.")
    +
    +    if (value == "true" || value == "false" || value == "null") {
    +      true
    +    } else {
    +      // The valid characters for the first character of a Java variable is [a-zA-Z_$].
    +      value.head match {
    +        case v if v >= 'a' && v <= 'z' => false
    --- End diff --
    
    I only consider literals and normal java variables now. If we say array accessing `arr[1]` are all global variables (I suppose they are, as usually we don't/can't use `arr[1]` as variable name), I'm thinking maybe we should disallow passing any global variables.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156291613
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -55,8 +55,45 @@ import org.apache.spark.util.{ParentClassLoader, Utils}
      *                 to null.
      * @param value A term for a (possibly primitive) value of the result of the evaluation. Not
      *              valid if `isNull` is set to `true`.
    + * @param inputRow A term that holds the input row name when generating this code.
    + * @param inputVars A list of [[ExprInputVar]] that holds input variables when generating this code.
      */
    -case class ExprCode(var code: String, var isNull: String, var value: String)
    +case class ExprCode(
    +    var code: String,
    +    var isNull: String,
    +    var value: String,
    +    var inputRow: String = null,
    +    var inputVars: Seq[ExprInputVar] = Seq.empty) {
    +
    +  // Returns true if this value is a literal.
    +  def isLiteral(): Boolean = {
    +    assert(value.nonEmpty, "ExprCode.value can't be empty string.")
    +
    +    if (value == "true" || value == "false" || value == "null") {
    +      true
    +    } else {
    +      // The valid characters for the first character of a Java variable is [a-zA-Z_$].
    +      value.head match {
    +        case v if v >= 'a' && v <= 'z' => false
    --- End diff --
    
    > another idea, let's move isLiteral and isEvaluated to ExpressionCodegen. Seems these 2 are not general definition of ExprCode, and only makes sense for ctx.currentVars.
    
    Ok.



---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    @cloud-fan Do you think we should still allow something like `a + 1` as the output of codegen?


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156285555
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -979,7 +1016,11 @@ class CodegenContext {
           val expr = e.head
           // Generate the code for this expression tree.
           val eval = expr.genCode(this)
    -      val state = SubExprEliminationState(eval.isNull, eval.value)
    +      val state = if (expr.nullable) {
    --- End diff --
    
    do we still need it?


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84742 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84742/testReport)** for PR 19813 at commit [`c083a79`](https://github.com/apache/spark/commit/c083a7955cd6fb54e0448176d9684496fae48e6f).
     * This patch **fails SparkR unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    I did a search but can't find one in the current codebase, but I do think this is a valid idea, e.g. a simple example would be `a + b + .... z`, if expressions can output statement, then we just generate code like
    ```
    int result = a + b ... + z;
    boolean isNull = false;
    ```
    instead of
    ```
    int result 1 = a + b;
    boolean isNull1 = false;
    int result2 = result1 + c;
    boolean isNull2 = false;
    ...
    ```
    
    This can apply to both whole stage codegen and normal codegen, and reduce the code size dramatically, and make whole stage codegen less likely to hit 64kb compile error.
    
    Another thing I'm working on is: do not create global variables if `ctx.spiltExpression` doesn't spit. This optimization should be much more useful if combined with this optimization.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154952068
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala ---
    @@ -236,4 +237,22 @@ class WholeStageCodegenSuite extends QueryTest with SharedSQLContext {
           }
         }
       }
    +
    +  test("SPARK-22551: Fix 64kb limit for deeply nested expressions under wholestage codegen") {
    +    import testImplicits._
    +    withTempPath { dir =>
    +      val path = dir.getCanonicalPath
    +      val df = Seq(("abc", 1)).toDF("key", "int")
    +      df.write.parquet(path)
    +
    +      var strExpr: Expression = col("key").expr
    +      for (_ <- 1 to 150) {
    +        strExpr = Decode(Encode(strExpr, Literal("utf-8")), Literal("utf-8"))
    +      }
    +      val expressions = Seq(If(EqualTo(strExpr, strExpr), strExpr, strExpr))
    +
    +      val df2 = spark.read.parquet(path).select(expressions.map(Column(_)): _*)
    --- End diff --
    
    just to double check, this test fails on the master branch, right?


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by mgaido91 <gi...@git.apache.org>.
Github user mgaido91 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154192687
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala ---
    @@ -115,9 +116,35 @@ abstract class Expression extends TreeNode[Expression] {
         }
       }
     
    +  /**
    +   * Records current input row and variables for this expression into created `ExprCode`.
    +   */
    +  private def populateInputs(ctx: CodegenContext, eval: ExprCode): Unit = {
    +    if (ctx.INPUT_ROW != null) {
    +      eval.inputRow = ctx.INPUT_ROW
    +    }
    +    if (ctx.currentVars != null) {
    +      val boundRefs = this.collect {
    +        case b @ BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null => (ordinal, b)
    +      }.toMap
    +
    +      ctx.currentVars.zipWithIndex.filter(_._1 != null).foreach { case (currentVar, idx) =>
    --- End diff --
    
    why is this filter needed?


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156285866
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -979,7 +1016,11 @@ class CodegenContext {
           val expr = e.head
           // Generate the code for this expression tree.
           val eval = expr.genCode(this)
    -      val state = SubExprEliminationState(eval.isNull, eval.value)
    +      val state = if (expr.nullable) {
    --- End diff --
    
    Yea, can remove it.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154854328
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu 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 paramsFromSubExprs = prepareFunctionParams(ctx, subExprs, subExprCodes)
    --- End diff --
    
    I mean before your PR.


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    retest this please.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156287780
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -55,8 +55,45 @@ import org.apache.spark.util.{ParentClassLoader, Utils}
      *                 to null.
      * @param value A term for a (possibly primitive) value of the result of the evaluation. Not
      *              valid if `isNull` is set to `true`.
    + * @param inputRow A term that holds the input row name when generating this code.
    + * @param inputVars A list of [[ExprInputVar]] that holds input variables when generating this code.
      */
    -case class ExprCode(var code: String, var isNull: String, var value: String)
    +case class ExprCode(
    +    var code: String,
    +    var isNull: String,
    +    var value: String,
    +    var inputRow: String = null,
    +    var inputVars: Seq[ExprInputVar] = Seq.empty) {
    +
    +  // Returns true if this value is a literal.
    +  def isLiteral(): Boolean = {
    +    assert(value.nonEmpty, "ExprCode.value can't be empty string.")
    +
    +    if (value == "true" || value == "false" || value == "null") {
    +      true
    +    } else {
    +      // The valid characters for the first character of a Java variable is [a-zA-Z_$].
    +      value.head match {
    +        case v if v >= 'a' && v <= 'z' => false
    --- End diff --
    
    also cc @kiszk @mgaido91 @maropu 


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84367 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84367/testReport)** for PR 19813 at commit [`0d358d6`](https://github.com/apache/spark/commit/0d358d635494199582aa6e38fdbeec0f6446c029).


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84275/
    Test FAILed.


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84303 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84303/testReport)** for PR 19813 at commit [`7f00515`](https://github.com/apache/spark/commit/7f005158b7b10fb2dc4db3ed15181e68ae33348f).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84303 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84303/testReport)** for PR 19813 at commit [`7f00515`](https://github.com/apache/spark/commit/7f005158b7b10fb2dc4db3ed15181e68ae33348f).


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84331/
    Test PASSed.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by mgaido91 <gi...@git.apache.org>.
Github user mgaido91 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154191102
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala ---
    @@ -105,6 +105,7 @@ abstract class Expression extends TreeNode[Expression] {
           val isNull = ctx.freshName("isNull")
           val value = ctx.freshName("value")
           val eval = doGenCode(ctx, ExprCode("", isNull, value))
    +      populateInputs(ctx, eval)
    --- End diff --
    
    nit: I would prefer not to change `eval` inside a function, in order to be more compliant to functional programming principle that methods should be without side effects. Since setting the `inputRow` is trivial I'd inline it here, while for the current variables, I'd prefer to assign them here by returning them from `populateInputs` (which would need to be renamed in this case)


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154267815
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,248 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu 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 paramsFromSubExprs = getParamsForSubExprs(ctx, subExprs)
    +
    +    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, subExprs) + 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
    +  }
    +
    +  /**
    +   * Given the list of eliminated subexpressions used in the children expressions, returns the
    +   * strings of funtion parameters. The first is the variable names used to call the function,
    +   * the second is the parameters used to declare the function in generated code.
    +   */
    +  def getParamsForSubExprs(
    +      ctx: CodegenContext,
    +      subExprs: Seq[Expression]): Seq[(String, String)] = {
    +    subExprs.flatMap { subExpr =>
    +      val argType = ctx.javaType(subExpr.dataType)
    +
    +      val subExprState = ctx.subExprEliminationExprs(subExpr)
    +
    +      if (!subExpr.nullable || subExprState.isNull == "true" || subExprState.isNull == "false") {
    +        Seq((subExprState.value, s"$argType ${subExprState.value}"))
    +      } else {
    +        Seq((subExprState.value, s"$argType ${subExprState.value}"),
    +          (subExprState.isNull, s"boolean ${subExprState.isNull}"))
    +      }
    +    }.distinct
    +  }
    +
    +  /**
    +   * Retrieves previous input rows referred by children and deferred expressions.
    +   */
    +  def getInputRowsForChildren(ctx: CodegenContext, expr: Expression): Seq[String] = {
    +    expr.children.flatMap(getInputRows(ctx, _)).distinct
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previous input rows referred by it or deferred expressions
    +   * which are needed to evaluate it.
    +   */
    +  def getInputRows(ctx: CodegenContext, child: Expression): Seq[String] = {
    +    child.flatMap {
    +      // An expression directly evaluates on current input row.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars == null ||
    +          ctx.currentVars(ordinal) == null =>
    +        Seq(ctx.INPUT_ROW)
    +
    +      // An expression which is not evaluated yet. Tracks down to find input rows.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal).code != "" =>
    +        trackDownRow(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down input rows referred by the generated code snippet.
    +   */
    +  def trackDownRow(ctx: CodegenContext, exprCode: ExprCode): Seq[String] = {
    +    val exprCodes = mutable.Queue[ExprCode](exprCode)
    +    val inputRows = mutable.ArrayBuffer.empty[String]
    +
    +    while (exprCodes.nonEmpty) {
    +      val curExprCode = exprCodes.dequeue()
    +      if (curExprCode.inputRow != null) {
    +        inputRows += curExprCode.inputRow
    +      }
    +      curExprCode.inputVars.foreach { inputVar =>
    +        if (inputVar.exprCode.code != "") {
    +          exprCodes.enqueue(inputVar.exprCode)
    +        }
    +      }
    +    }
    +    inputRows
    +  }
    +
    +  /**
    +   * Retrieves previously evaluated columns referred by children and deferred expressions.
    +   * Returned tuple contains the list of expressions and the list of generated codes.
    +   */
    +  def getInputVarsForChildren(
    +      ctx: CodegenContext,
    +      expr: Expression): (Seq[Expression], Seq[ExprCode]) = {
    +    expr.children.flatMap(getInputVars(ctx, _)).distinct.unzip
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previously evaluated columns referred by it or
    +   * deferred expressions which are needed to evaluate it.
    +   */
    +  def getInputVars(ctx: CodegenContext, child: Expression): Seq[(Expression, ExprCode)] = {
    +    if (ctx.currentVars == null) {
    +      return Seq.empty
    +    }
    +
    +    child.flatMap {
    +      // An evaluated variable.
    +      case b @ BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null &&
    +          ctx.currentVars(ordinal).code == "" =>
    +        Seq((b, ctx.currentVars(ordinal)))
    +
    +      // An input variable which is not evaluated yet. Tracks down to find any evaluated variables
    +      // in the expression path.
    +      // E.g., if this expression is "d = c + 1" and "c" is not evaluated. We need to track to
    +      // "c = a + b" and see if "a" and "b" are evaluated. If they are, we need to return them so
    +      // to include them into parameters, if not, we track down further.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null =>
    +        trackDownVar(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down previously evaluated columns referred by the generated code snippet.
    +   */
    +  def trackDownVar(ctx: CodegenContext, exprCode: ExprCode): Seq[(Expression, ExprCode)] = {
    +    val exprCodes = mutable.Queue[ExprCode](exprCode)
    +    val inputVars = mutable.ArrayBuffer.empty[(Expression, ExprCode)]
    +
    +    while (exprCodes.nonEmpty) {
    +      exprCodes.dequeue().inputVars.foreach { inputVar =>
    +        if (inputVar.exprCode.code == "") {
    +          inputVars += ((inputVar.expr, inputVar.exprCode))
    +        } else {
    +          exprCodes.enqueue(inputVar.exprCode)
    +        }
    +      }
    +    }
    +    inputVars
    +  }
    +
    +  /**
    +   * Helper function to calculate the size of an expression as function parameter.
    +   */
    +  def calculateParamLength(ctx: CodegenContext, input: Expression): Int = {
    +    ctx.javaType(input.dataType) match {
    +      case (ctx.JAVA_LONG | ctx.JAVA_DOUBLE) if !input.nullable => 2
    +      case ctx.JAVA_LONG | ctx.JAVA_DOUBLE => 3
    +      case _ if !input.nullable => 1
    +      case _ => 2
    +    }
    +  }
    +
    +  /**
    +   * In Java, a method descriptor is valid only if it represents method parameters with a total
    +   * length of 255 or less. `this` contributes one unit and a parameter of type long or double
    +   * contributes two units.
    +   */
    +  def getParamLength(
    +      ctx: CodegenContext,
    +      inputs: Seq[Expression],
    +      subExprs: Seq[Expression]): Int = {
    +    // Initial value is 1 for `this`.
    +    1 + (inputs ++ subExprs).distinct.map(calculateParamLength(ctx, _)).sum
    +  }
    +
    +  /**
    +   * Given the lists of input attributes and variables to this expression, returns the strings of
    +   * funtion parameters. The first is the variable names used to call the function, the second is
    +   * the parameters used to declare the function in generated code.
    +   */
    +  def prepareFunctionParams(
    --- End diff --
    
    Yes, sure.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156057964
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala ---
    @@ -105,6 +105,11 @@ abstract class Expression extends TreeNode[Expression] {
           val isNull = ctx.freshName("isNull")
           val value = ctx.freshName("value")
           val eval = doGenCode(ctx, ExprCode("", isNull, value))
    +
    +      // Records current input row and variables of this expression.
    +      eval.inputRow = ctx.INPUT_ROW
    +      eval.inputVars = findInputVars(ctx, eval)
    --- End diff --
    
    actually we can be more aggresive and do `val isNull = if (this.nullable) eval.isNull else "false"`, and see if there is any compile errors.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    @cloud-fan Since #20043 was merged now, I will go to polish this and implement the above idea. Will submit a PR for this when it is ready.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156077428
  
    --- 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] = {
    +    subExprs.map { subExpr =>
    +      val state = ctx.subExprEliminationExprs(subExpr)
    +      ExprCode(code = "", value = state.value, isNull = state.isNull)
    +    }
    +  }
    +
    +  /**
    +   * Retrieves previous input rows referred by children and deferred expressions.
    +   */
    +  def getInputRowsForChildren(ctx: CodegenContext, expr: Expression): Seq[String] = {
    +    expr.children.flatMap(getInputRows(ctx, _)).distinct
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previous input rows referred by it or deferred expressions
    +   * which are needed to evaluate it.
    +   */
    +  def getInputRows(ctx: CodegenContext, child: Expression): Seq[String] = {
    +    child.flatMap {
    +      // An expression directly evaluates on current input row.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars == null ||
    +          ctx.currentVars(ordinal) == null =>
    +        Seq(ctx.INPUT_ROW)
    +
    +      // An expression which is not evaluated yet. Tracks down to find input rows.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal).code != "" =>
    +        trackDownRow(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down input rows referred by the generated code snippet.
    +   */
    +  def trackDownRow(ctx: CodegenContext, exprCode: ExprCode): Seq[String] = {
    +    val exprCodes = mutable.Queue[ExprCode](exprCode)
    +    val inputRows = mutable.ArrayBuffer.empty[String]
    +
    +    while (exprCodes.nonEmpty) {
    +      val curExprCode = exprCodes.dequeue()
    +      if (curExprCode.inputRow != null) {
    +        inputRows += curExprCode.inputRow
    +      }
    +      curExprCode.inputVars.foreach { inputVar =>
    +        if (inputVar.exprCode.code != "") {
    +          exprCodes.enqueue(inputVar.exprCode)
    +        }
    +      }
    +    }
    +    inputRows
    +  }
    +
    +  /**
    +   * Retrieves previously evaluated columns referred by children and deferred expressions.
    +   * Returned tuple contains the list of expressions and the list of generated codes.
    +   */
    +  def getInputVarsForChildren(
    +      ctx: CodegenContext,
    +      expr: Expression): (Seq[ExprProperty], Seq[ExprCode]) = {
    --- End diff --
    
    Yea, just `Seq[ExprInputVar]`.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156287106
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -1028,12 +1078,17 @@ class CodegenContext {
           //   2. Less code.
           // Currently, we will do this for all non-leaf only expression trees (i.e. expr trees with
           // at least two nodes) as the cost of doing it is expected to be low.
    -      addMutableState(JAVA_BOOLEAN, isNull, s"$isNull = false;")
    -      addMutableState(javaType(expr.dataType), value,
    -        s"$value = ${defaultValue(expr.dataType)};")
    +      if (expr.nullable) {
    +        addMutableState(JAVA_BOOLEAN, isNull)
    +      }
    +      addMutableState(javaType(expr.dataType), value)
     
           subexprFunctions += s"${addNewFunction(fnName, fn)}($INPUT_ROW);"
    -      val state = SubExprEliminationState(isNull, value)
    +      val state = if (expr.nullable) {
    +        SubExprEliminationState(isNull, value)
    --- End diff --
    
    Because here it is not `SubExprEliminationState(ev.isNull, value)`.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    > However for whole stage codegen path, it's way more complex:
    > 1. some of ctx.currentVars are just variables, their codes have already been generated before. But some are not. For those whose codes are not generated, they are not valid inputs.
    > 2. ctx.currentVars is not null but has null slots, and ctx.INPUT_ROW is not null. Then both ctx.currentVars and ctx.INPUT_ROW are valid inputs.
    
    
    Yes, this is correct.
    
    So, for 1, only the variables not evaluate yet, we don't include them as parameters.
    For 2, null slots in ctx.currentVars won't be included as parameters too. ctx.INPUT_ROW will be included only if it is not null.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84769/
    Test PASSed.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    LGTM, good job!


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by mgaido91 <gi...@git.apache.org>.
Github user mgaido91 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154199027
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,259 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu children expressions.
    +   */
    +  def getExpressionInputParams(
    +      ctx: CodegenContext,
    +      expr: Expression): Option[(Seq[String], Seq[String])] = {
    +    val (inputAttrs, inputVars) = getInputVarsForChildren(ctx, expr)
    +    val inputRows = ctx.INPUT_ROW +: getInputRowsForChildren(ctx, expr)
    +    val subExprs = getSubExprInChildren(ctx, expr)
    +
    +    val paramsFromRows = inputRows.distinct.filter(_ != null).map { row =>
    +      (row, s"InternalRow $row")
    +    }
    +    val paramsFromColumns = prepareFunctionParams(ctx, inputAttrs, inputVars)
    +    val paramsFromSubExprs = getParamsForSubExprs(ctx, subExprs)
    +    val paramsLength = getParamLength(ctx, inputAttrs, subExprs) + 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
    +  }
    +
    +  /**
    +   * Given the list of eliminated subexpressions used in the children expressions, returns the
    +   * strings of funtion parameters. The first is the variable names used to call the function,
    +   * the second is the parameters used to declare the function in generated code.
    +   */
    +  def getParamsForSubExprs(
    +      ctx: CodegenContext,
    +      subExprs: Seq[Expression]): Seq[(String, String)] = {
    +    subExprs.flatMap { subExpr =>
    +      val argType = ctx.javaType(subExpr.dataType)
    +
    +      val subExprState = ctx.subExprEliminationExprs(subExpr)
    +      (subExprState.value, subExprState.isNull)
    +
    +      if (!subExpr.nullable || subExprState.isNull == "true" || subExprState.isNull == "false") {
    +        Seq((subExprState.value, s"$argType ${subExprState.value}"))
    +      } else {
    +        Seq((subExprState.value, s"$argType ${subExprState.value}"),
    +          (subExprState.isNull, s"boolean ${subExprState.isNull}"))
    +      }
    +    }.distinct
    +  }
    +
    +  /**
    +   * Retrieves previous input rows referred by children and deferred expressions.
    +   */
    +  def getInputRowsForChildren(ctx: CodegenContext, expr: Expression): Seq[String] = {
    +    expr.children.flatMap(getInputRows(ctx, _)).distinct
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previous input rows referred by it or deferred expressions
    +   * which are needed to evaluate it.
    +   */
    +  def getInputRows(ctx: CodegenContext, child: Expression): Seq[String] = {
    +    child.flatMap {
    +      // An expression directly evaluates on current input row.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars == null ||
    +          ctx.currentVars(ordinal) == null =>
    +        Seq(ctx.INPUT_ROW)
    +
    +      // An expression which is not evaluated yet. Tracks down to find input rows.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal).code != "" =>
    +        trackDownRow(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down input rows referred by the generated code snippet.
    +   */
    +  def trackDownRow(ctx: CodegenContext, exprCode: ExprCode): Seq[String] = {
    +    var exprCodes: List[ExprCode] = List(exprCode)
    +    val inputRows = mutable.ArrayBuffer.empty[String]
    +
    +    while (exprCodes.nonEmpty) {
    +      exprCodes match {
    +        case first :: others =>
    +          exprCodes = others
    +          if (first.inputRow != null) {
    +            inputRows += first.inputRow
    +          }
    +          first.inputVars.foreach { inputVar =>
    +            if (inputVar.exprCode.code != "") {
    +              exprCodes = inputVar.exprCode :: exprCodes
    +            }
    +          }
    +        case _ =>
    +      }
    +    }
    +    inputRows.toSeq
    +  }
    +
    +  /**
    +   * Retrieves previously evaluated columns referred by children and deferred expressions.
    +   * Returned tuple contains the list of expressions and the list of generated codes.
    +   */
    +  def getInputVarsForChildren(
    +      ctx: CodegenContext,
    +      expr: Expression): (Seq[Expression], Seq[ExprCode]) = {
    +    expr.children.flatMap(getInputVars(ctx, _)).distinct.unzip
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previously evaluated columns referred by it or
    +   * deferred expressions which are needed to evaluate it.
    +   */
    +  def getInputVars(ctx: CodegenContext, child: Expression): Seq[(Expression, ExprCode)] = {
    +    if (ctx.currentVars == null) {
    +      return Seq.empty
    +    }
    +
    +    child.flatMap {
    +      // An evaluated variable.
    +      case b @ BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null &&
    +          ctx.currentVars(ordinal).code == "" =>
    +        Seq((b, ctx.currentVars(ordinal)))
    +
    +      // An input variable which is not evaluated yet. Tracks down to find any evaluated variables
    +      // in the expression path.
    +      // E.g., if this expression is "d = c + 1" and "c" is not evaluated. We need to track to
    +      // "c = a + b" and see if "a" and "b" are evaluated. If they are, we need to return them so
    +      // to include them into parameters, if not, we track down further.
    +      case b @ BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null =>
    +        trackDownVar(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down previously evaluated columns referred by the generated code snippet.
    +   */
    +  def trackDownVar(ctx: CodegenContext, exprCode: ExprCode): Seq[(Expression, ExprCode)] = {
    +    var exprCodes: List[ExprCode] = List(exprCode)
    --- End diff --
    
    what about making this a mutable collection to avoid its complete copy here https://github.com/apache/spark/pull/19813/files#diff-0fcf78140c2c4edef3840822ac41563eR204?


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156293395
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -55,8 +55,45 @@ import org.apache.spark.util.{ParentClassLoader, Utils}
      *                 to null.
      * @param value A term for a (possibly primitive) value of the result of the evaluation. Not
      *              valid if `isNull` is set to `true`.
    + * @param inputRow A term that holds the input row name when generating this code.
    + * @param inputVars A list of [[ExprInputVar]] that holds input variables when generating this code.
      */
    -case class ExprCode(var code: String, var isNull: String, var value: String)
    +case class ExprCode(
    +    var code: String,
    +    var isNull: String,
    +    var value: String,
    +    var inputRow: String = null,
    +    var inputVars: Seq[ExprInputVar] = Seq.empty) {
    +
    +  // Returns true if this value is a literal.
    +  def isLiteral(): Boolean = {
    +    assert(value.nonEmpty, "ExprCode.value can't be empty string.")
    +
    +    if (value == "true" || value == "false" || value == "null") {
    +      true
    +    } else {
    +      // The valid characters for the first character of a Java variable is [a-zA-Z_$].
    +      value.head match {
    +        case v if v >= 'a' && v <= 'z' => false
    --- End diff --
    
    thanks! sorry for the trouble...


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84330/
    Test FAILed.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    ping @cloud-fan @kiszk Do you have more review on this? Thanks.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84422/
    Test PASSed.


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84260/
    Test FAILed.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156360136
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -55,8 +55,45 @@ import org.apache.spark.util.{ParentClassLoader, Utils}
      *                 to null.
      * @param value A term for a (possibly primitive) value of the result of the evaluation. Not
      *              valid if `isNull` is set to `true`.
    + * @param inputRow A term that holds the input row name when generating this code.
    + * @param inputVars A list of [[ExprInputVar]] that holds input variables when generating this code.
      */
    -case class ExprCode(var code: String, var isNull: String, var value: String)
    +case class ExprCode(
    +    var code: String,
    +    var isNull: String,
    +    var value: String,
    +    var inputRow: String = null,
    +    var inputVars: Seq[ExprInputVar] = Seq.empty) {
    +
    +  // Returns true if this value is a literal.
    +  def isLiteral(): Boolean = {
    --- End diff --
    
    shall we remove them now?


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    > What about future? Will we need to output statement for some reason? like reducing the usage of local variables?
    
    I think that we won't have strong motivation to use output statement. The reason is, although it helps to reduce the usage of local variables (is it any beneficial like reducing global variables?), it also means the chained evaluation of expressions needs to be run at every occurrence.
    



---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84275 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84275/testReport)** for PR 19813 at commit [`8c7f749`](https://github.com/apache/spark/commit/8c7f7496e610fdf4b512c57efd108ccf0238b126).
     * This patch **fails SparkR unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    This is a pretty cool idea that can work with the current string based codegen framework, LGTM!


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    This needs a design doc. 


---

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


[GitHub] spark pull request #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r153982636
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,259 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    --- End diff --
    
    I think I should add some tests for the APIs here.


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84207 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84207/testReport)** for PR 19813 at commit [`9f848be`](https://github.com/apache/spark/commit/9f848be45dcc294d6f27f2c6eaeed1907d36f004).


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156363603
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -55,8 +55,45 @@ import org.apache.spark.util.{ParentClassLoader, Utils}
      *                 to null.
      * @param value A term for a (possibly primitive) value of the result of the evaluation. Not
      *              valid if `isNull` is set to `true`.
    + * @param inputRow A term that holds the input row name when generating this code.
    + * @param inputVars A list of [[ExprInputVar]] that holds input variables when generating this code.
      */
    -case class ExprCode(var code: String, var isNull: String, var value: String)
    +case class ExprCode(
    +    var code: String,
    +    var isNull: String,
    +    var value: String,
    +    var inputRow: String = null,
    +    var inputVars: Seq[ExprInputVar] = Seq.empty) {
    +
    +  // Returns true if this value is a literal.
    +  def isLiteral(): Boolean = {
    --- End diff --
    
    Sorry, forgot it...


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r155688125
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines util methods used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu 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 paramsFromSubExprs = prepareFunctionParams(ctx, subExprs, 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, subExprs) + 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] = {
    +    subExprs.map { subExpr =>
    +      val state = ctx.subExprEliminationExprs(subExpr)
    +      ExprCode(code = "", value = state.value, isNull = state.isNull)
    +    }
    +  }
    +
    +  /**
    +   * Retrieves previous input rows referred by children and deferred expressions.
    +   */
    +  def getInputRowsForChildren(ctx: CodegenContext, expr: Expression): Seq[String] = {
    +    expr.children.flatMap(getInputRows(ctx, _)).distinct
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previous input rows referred by it or deferred expressions
    +   * which are needed to evaluate it.
    +   */
    +  def getInputRows(ctx: CodegenContext, child: Expression): Seq[String] = {
    +    child.flatMap {
    +      // An expression directly evaluates on current input row.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars == null ||
    +          ctx.currentVars(ordinal) == null =>
    +        Seq(ctx.INPUT_ROW)
    +
    +      // An expression which is not evaluated yet. Tracks down to find input rows.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal).code != "" =>
    +        trackDownRow(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down input rows referred by the generated code snippet.
    +   */
    +  def trackDownRow(ctx: CodegenContext, exprCode: ExprCode): Seq[String] = {
    +    val exprCodes = mutable.Queue[ExprCode](exprCode)
    +    val inputRows = mutable.ArrayBuffer.empty[String]
    +
    +    while (exprCodes.nonEmpty) {
    +      val curExprCode = exprCodes.dequeue()
    +      if (curExprCode.inputRow != null) {
    +        inputRows += curExprCode.inputRow
    +      }
    +      curExprCode.inputVars.foreach { inputVar =>
    +        if (inputVar.exprCode.code != "") {
    +          exprCodes.enqueue(inputVar.exprCode)
    +        }
    +      }
    +    }
    +    inputRows
    +  }
    +
    +  /**
    +   * Retrieves previously evaluated columns referred by children and deferred expressions.
    +   * Returned tuple contains the list of expressions and the list of generated codes.
    +   */
    +  def getInputVarsForChildren(
    +      ctx: CodegenContext,
    +      expr: Expression): (Seq[Expression], Seq[ExprCode]) = {
    +    expr.children.flatMap(getInputVars(ctx, _)).distinct.unzip
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previously evaluated columns referred by it or
    +   * deferred expressions which are needed to evaluate it.
    +   */
    +  def getInputVars(ctx: CodegenContext, child: Expression): Seq[(Expression, ExprCode)] = {
    +    if (ctx.currentVars == null) {
    +      return Seq.empty
    +    }
    +
    +    child.flatMap {
    +      // An evaluated variable.
    +      case b @ BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null &&
    +          ctx.currentVars(ordinal).code == "" =>
    +        Seq((b, ctx.currentVars(ordinal)))
    +
    +      // An input variable which is not evaluated yet. Tracks down to find any evaluated variables
    +      // in the expression path.
    +      // E.g., if this expression is "d = c + 1" and "c" is not evaluated. We need to track to
    +      // "c = a + b" and see if "a" and "b" are evaluated. If they are, we need to return them so
    +      // to include them into parameters, if not, we track down further.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null =>
    +        trackDownVar(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down previously evaluated columns referred by the generated code snippet.
    +   */
    +  def trackDownVar(ctx: CodegenContext, exprCode: ExprCode): Seq[(Expression, ExprCode)] = {
    +    val exprCodes = mutable.Queue[ExprCode](exprCode)
    +    val inputVars = mutable.ArrayBuffer.empty[(Expression, ExprCode)]
    +
    +    while (exprCodes.nonEmpty) {
    +      exprCodes.dequeue().inputVars.foreach { inputVar =>
    +        if (inputVar.exprCode.code == "") {
    +          inputVars += ((inputVar.expr, inputVar.exprCode))
    +        } else {
    +          exprCodes.enqueue(inputVar.exprCode)
    +        }
    +      }
    +    }
    +    inputVars
    +  }
    +
    +  /**
    +   * Helper function to calculate the size of an expression as function parameter.
    +   */
    +  def calculateParamLength(ctx: CodegenContext, input: Expression): Int = {
    +    ctx.javaType(input.dataType) match {
    +      case (ctx.JAVA_LONG | ctx.JAVA_DOUBLE) if !input.nullable => 2
    +      case ctx.JAVA_LONG | ctx.JAVA_DOUBLE => 3
    +      case _ if !input.nullable => 1
    +      case _ => 2
    +    }
    +  }
    +
    +  /**
    +   * In Java, a method descriptor is valid only if it represents method parameters with a total
    +   * length of 255 or less. `this` contributes one unit and a parameter of type long or double
    +   * contributes two units.
    +   */
    +  def getParamLength(
    +      ctx: CodegenContext,
    +      inputs: Seq[Expression],
    +      subExprs: Seq[Expression]): Int = {
    +    // Initial value is 1 for `this`.
    +    1 + (inputs ++ subExprs).distinct.map(calculateParamLength(ctx, _)).sum
    +  }
    +
    +  /**
    +   * Given the lists of input attributes and variables to this expression, returns the strings of
    +   * funtion parameters. The first is the variable names used to call the function, the second is
    +   * the parameters used to declare the function in generated code.
    +   */
    +  def prepareFunctionParams(
    +      ctx: CodegenContext,
    +      inputAttrs: Seq[Expression],
    +      inputVars: Seq[ExprCode]): Seq[(String, String)] = {
    +    inputAttrs.zip(inputVars).flatMap { case (input, ev) =>
    +      val argType = ctx.javaType(input.dataType)
    +
    +      if (!input.nullable || ev.isNull == "true" || ev.isNull == "false") {
    +        Seq((ev.value, s"$argType ${ev.value}"))
    +      } else {
    +        Seq((ev.value, s"$argType ${ev.value}"), (ev.isNull, s"boolean ${ev.isNull}"))
    --- End diff --
    
    sounds good, we can create a `isLiteral` method in `ExprCode`


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    ping @kiszk @cloud-fan any more thoughts or review for this?


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84641/
    Test PASSed.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r155673475
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines util methods used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu 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 paramsFromSubExprs = prepareFunctionParams(ctx, subExprs, 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, subExprs) + 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] = {
    +    subExprs.map { subExpr =>
    +      val state = ctx.subExprEliminationExprs(subExpr)
    +      ExprCode(code = "", value = state.value, isNull = state.isNull)
    +    }
    +  }
    +
    +  /**
    +   * Retrieves previous input rows referred by children and deferred expressions.
    +   */
    +  def getInputRowsForChildren(ctx: CodegenContext, expr: Expression): Seq[String] = {
    +    expr.children.flatMap(getInputRows(ctx, _)).distinct
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previous input rows referred by it or deferred expressions
    +   * which are needed to evaluate it.
    +   */
    +  def getInputRows(ctx: CodegenContext, child: Expression): Seq[String] = {
    +    child.flatMap {
    +      // An expression directly evaluates on current input row.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars == null ||
    +          ctx.currentVars(ordinal) == null =>
    +        Seq(ctx.INPUT_ROW)
    +
    +      // An expression which is not evaluated yet. Tracks down to find input rows.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal).code != "" =>
    +        trackDownRow(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down input rows referred by the generated code snippet.
    +   */
    +  def trackDownRow(ctx: CodegenContext, exprCode: ExprCode): Seq[String] = {
    +    val exprCodes = mutable.Queue[ExprCode](exprCode)
    +    val inputRows = mutable.ArrayBuffer.empty[String]
    +
    +    while (exprCodes.nonEmpty) {
    +      val curExprCode = exprCodes.dequeue()
    +      if (curExprCode.inputRow != null) {
    +        inputRows += curExprCode.inputRow
    +      }
    +      curExprCode.inputVars.foreach { inputVar =>
    +        if (inputVar.exprCode.code != "") {
    +          exprCodes.enqueue(inputVar.exprCode)
    +        }
    +      }
    +    }
    +    inputRows
    +  }
    +
    +  /**
    +   * Retrieves previously evaluated columns referred by children and deferred expressions.
    +   * Returned tuple contains the list of expressions and the list of generated codes.
    +   */
    +  def getInputVarsForChildren(
    +      ctx: CodegenContext,
    +      expr: Expression): (Seq[Expression], Seq[ExprCode]) = {
    +    expr.children.flatMap(getInputVars(ctx, _)).distinct.unzip
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previously evaluated columns referred by it or
    +   * deferred expressions which are needed to evaluate it.
    +   */
    +  def getInputVars(ctx: CodegenContext, child: Expression): Seq[(Expression, ExprCode)] = {
    +    if (ctx.currentVars == null) {
    +      return Seq.empty
    +    }
    +
    +    child.flatMap {
    +      // An evaluated variable.
    +      case b @ BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null &&
    +          ctx.currentVars(ordinal).code == "" =>
    +        Seq((b, ctx.currentVars(ordinal)))
    +
    +      // An input variable which is not evaluated yet. Tracks down to find any evaluated variables
    +      // in the expression path.
    +      // E.g., if this expression is "d = c + 1" and "c" is not evaluated. We need to track to
    +      // "c = a + b" and see if "a" and "b" are evaluated. If they are, we need to return them so
    +      // to include them into parameters, if not, we track down further.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null =>
    +        trackDownVar(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down previously evaluated columns referred by the generated code snippet.
    +   */
    +  def trackDownVar(ctx: CodegenContext, exprCode: ExprCode): Seq[(Expression, ExprCode)] = {
    +    val exprCodes = mutable.Queue[ExprCode](exprCode)
    +    val inputVars = mutable.ArrayBuffer.empty[(Expression, ExprCode)]
    +
    +    while (exprCodes.nonEmpty) {
    +      exprCodes.dequeue().inputVars.foreach { inputVar =>
    +        if (inputVar.exprCode.code == "") {
    +          inputVars += ((inputVar.expr, inputVar.exprCode))
    +        } else {
    +          exprCodes.enqueue(inputVar.exprCode)
    +        }
    +      }
    +    }
    +    inputVars
    +  }
    +
    +  /**
    +   * Helper function to calculate the size of an expression as function parameter.
    +   */
    +  def calculateParamLength(ctx: CodegenContext, input: Expression): Int = {
    +    ctx.javaType(input.dataType) match {
    +      case (ctx.JAVA_LONG | ctx.JAVA_DOUBLE) if !input.nullable => 2
    +      case ctx.JAVA_LONG | ctx.JAVA_DOUBLE => 3
    +      case _ if !input.nullable => 1
    +      case _ => 2
    +    }
    +  }
    +
    +  /**
    +   * In Java, a method descriptor is valid only if it represents method parameters with a total
    +   * length of 255 or less. `this` contributes one unit and a parameter of type long or double
    +   * contributes two units.
    +   */
    +  def getParamLength(
    +      ctx: CodegenContext,
    +      inputs: Seq[Expression],
    +      subExprs: Seq[Expression]): Int = {
    +    // Initial value is 1 for `this`.
    +    1 + (inputs ++ subExprs).distinct.map(calculateParamLength(ctx, _)).sum
    +  }
    +
    +  /**
    +   * Given the lists of input attributes and variables to this expression, returns the strings of
    +   * funtion parameters. The first is the variable names used to call the function, the second is
    +   * the parameters used to declare the function in generated code.
    +   */
    +  def prepareFunctionParams(
    +      ctx: CodegenContext,
    +      inputAttrs: Seq[Expression],
    +      inputVars: Seq[ExprCode]): Seq[(String, String)] = {
    +    inputAttrs.zip(inputVars).flatMap { case (input, ev) =>
    +      val argType = ctx.javaType(input.dataType)
    +
    +      if (!input.nullable || ev.isNull == "true" || ev.isNull == "false") {
    +        Seq((ev.value, s"$argType ${ev.value}"))
    +      } else {
    +        Seq((ev.value, s"$argType ${ev.value}"), (ev.isNull, s"boolean ${ev.isNull}"))
    --- End diff --
    
    Ok. Sounds good.


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84303/
    Test PASSed.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84182 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84182/testReport)** for PR 19813 at commit [`65d07d5`](https://github.com/apache/spark/commit/65d07d525344e1d00457d2f538b2ef0b1c38a8e8).


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r155553835
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines util methods used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu 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 paramsFromSubExprs = prepareFunctionParams(ctx, subExprs, subExprCodes)
    +
    +    val inputRows = ctx.INPUT_ROW +: getInputRowsForChildren(ctx, expr)
    --- End diff --
    
    I just quickly ran some tests. For example, `DataFrameSuite` has few tests failed if we don't consider this.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Cool. LGTM, too.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156288913
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,245 @@
    +/*
    + * 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 {
    +
    +  /**
    +   * 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 subExprs = getSubExprInChildren(ctx, expr)
    +    val subExprCodes = getSubExprCodes(ctx, subExprs)
    +    val subVars = subExprs.zip(subExprCodes).map { case (subExpr, subExprCode) =>
    +      ExprInputVar(subExprCode, subExpr.dataType, subExpr.nullable)
    +    }
    +    val paramsFromSubExprs = prepareFunctionParams(ctx, subVars)
    +
    +    val inputVars = getInputVarsForChildren(ctx, expr)
    +    val paramsFromColumns = prepareFunctionParams(ctx, inputVars)
    +
    +    val inputRows = ctx.INPUT_ROW +: getInputRowsForChildren(ctx, expr)
    +    val paramsFromRows = inputRows.distinct.filter(_ != null).map { row =>
    +      (row, s"InternalRow $row")
    +    }
    +
    +    val paramsLength = getParamLength(ctx, inputVars ++ subVars) + 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] = {
    +    subExprs.map { subExpr =>
    +      val state = ctx.subExprEliminationExprs(subExpr)
    +      ExprCode(code = "", value = state.value, isNull = state.isNull)
    +    }
    +  }
    +
    +  /**
    +   * Retrieves previous input rows referred by children and deferred expressions.
    +   */
    +  def getInputRowsForChildren(ctx: CodegenContext, expr: Expression): Seq[String] = {
    +    expr.children.flatMap(getInputRows(ctx, _)).distinct
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previous input rows referred by it or deferred expressions
    +   * which are needed to evaluate it.
    +   */
    +  def getInputRows(ctx: CodegenContext, child: Expression): Seq[String] = {
    +    child.flatMap {
    +      // An expression directly evaluates on current input row.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars == null ||
    +          ctx.currentVars(ordinal) == null =>
    +        Seq(ctx.INPUT_ROW)
    +
    +      // An expression which is not evaluated yet. Tracks down to find input rows.
    +      case BoundReference(ordinal, _, _) if !ctx.currentVars(ordinal).isEvaluated() =>
    +        trackDownRow(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down input rows referred by the generated code snippet.
    +   */
    +  def trackDownRow(ctx: CodegenContext, exprCode: ExprCode): Seq[String] = {
    +    val exprCodes = mutable.Queue[ExprCode](exprCode)
    +    val inputRows = mutable.ArrayBuffer.empty[String]
    +
    +    while (exprCodes.nonEmpty) {
    +      val curExprCode = exprCodes.dequeue()
    +      if (curExprCode.inputRow != null) {
    +        inputRows += curExprCode.inputRow
    +      }
    +      curExprCode.inputVars.foreach { inputVar =>
    +        if (!inputVar.exprCode.isEvaluated()) {
    +          exprCodes.enqueue(inputVar.exprCode)
    +        }
    +      }
    +    }
    +    inputRows
    +  }
    +
    +  /**
    +   * Retrieves previously evaluated columns referred by children and deferred expressions.
    +   * Returned tuple contains the list of expressions and the list of generated codes.
    +   */
    +  def getInputVarsForChildren(
    +      ctx: CodegenContext,
    +      expr: Expression): Seq[ExprInputVar] = {
    +    expr.children.flatMap(getInputVars(ctx, _)).distinct
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previously evaluated columns referred by it or
    +   * deferred expressions which are needed to evaluate it.
    +   */
    +  def getInputVars(ctx: CodegenContext, child: Expression): Seq[ExprInputVar] = {
    +    if (ctx.currentVars == null) {
    +      return Seq.empty
    +    }
    +
    +    child.flatMap {
    +      // An evaluated variable.
    +      case b @ BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null &&
    +          ctx.currentVars(ordinal).isEvaluated() =>
    +        Seq(ExprInputVar(ctx.currentVars(ordinal), b.dataType, b.nullable))
    +
    +      // An input variable which is not evaluated yet. Tracks down to find any evaluated variables
    +      // in the expression path.
    +      // E.g., if this expression is "d = c + 1" and "c" is not evaluated. We need to track to
    +      // "c = a + b" and see if "a" and "b" are evaluated. If they are, we need to return them so
    +      // to include them into parameters, if not, we track down further.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null =>
    +        trackDownVar(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down previously evaluated columns referred by the generated code snippet.
    +   */
    +  def trackDownVar(ctx: CodegenContext, exprCode: ExprCode): Seq[ExprInputVar] = {
    +    val exprCodes = mutable.Queue[ExprCode](exprCode)
    +    val inputVars = mutable.ArrayBuffer.empty[ExprInputVar]
    +
    +    while (exprCodes.nonEmpty) {
    +      exprCodes.dequeue().inputVars.foreach { inputVar =>
    +        if (inputVar.exprCode.isEvaluated()) {
    +          inputVars += inputVar
    +        } else {
    +          exprCodes.enqueue(inputVar.exprCode)
    +        }
    +      }
    +    }
    +    inputVars
    +  }
    +
    +  /**
    +   * Helper function to calculate the size of an expression as function parameter.
    +   */
    +  def calculateParamLength(ctx: CodegenContext, input: ExprInputVar): Int = {
    +    ctx.javaType(input.dataType) match {
    --- End diff --
    
    Ok. 


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    AST based codegen framework sounds a too far step from current point. I think we either follow the new contract or refactor the current framework a little bit.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154581864
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu 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 paramsFromSubExprs = prepareFunctionParams(ctx, subExprs, 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, subExprs) + 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] = {
    +    subExprs.map { subExpr =>
    +      val stat = ctx.subExprEliminationExprs(subExpr)
    --- End diff --
    
    It is for state. Yes, `state` is better and not to confuse.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156088540
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala ---
    @@ -105,6 +105,11 @@ abstract class Expression extends TreeNode[Expression] {
           val isNull = ctx.freshName("isNull")
           val value = ctx.freshName("value")
           val eval = doGenCode(ctx, ExprCode("", isNull, value))
    +
    +      // Records current input row and variables of this expression.
    +      eval.inputRow = ctx.INPUT_ROW
    +      eval.inputVars = findInputVars(ctx, eval)
    --- End diff --
    
    The aggressive version (`val isNull = if (this.nullable) ctx.freshName("isNull") else "false"`) fails compilation when testing locally.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r155526048
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -55,8 +55,23 @@ import org.apache.spark.util.{ParentClassLoader, Utils}
      *                 to null.
      * @param value A term for a (possibly primitive) value of the result of the evaluation. Not
      *              valid if `isNull` is set to `true`.
    + * @param inputRow A term that holds the input row name when generating this code.
    + * @param inputVars A list of [[ExprInputVar]] that holds input variables when generating this code.
      */
    -case class ExprCode(var code: String, var isNull: String, var value: String)
    +case class ExprCode(
    +    var code: String,
    +    var isNull: String,
    +    var value: String,
    +    var inputRow: String = null,
    +    var inputVars: Seq[ExprInputVar] = Seq.empty)
    +
    +/**
    + * Represents an input variable [[ExprCode]] to an evaluation of an [[Expression]].
    + *
    + * @param expr The expression that is evaluated to the input variable.
    + * @param exprCode The [[ExprCode]] that represents the evaluation result for the input variable.
    + */
    +case class ExprInputVar(expr: Expression, exprCode: ExprCode)
    --- End diff --
    
    Yes.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156285726
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -1028,12 +1078,17 @@ class CodegenContext {
           //   2. Less code.
           // Currently, we will do this for all non-leaf only expression trees (i.e. expr trees with
           // at least two nodes) as the cost of doing it is expected to be low.
    -      addMutableState(JAVA_BOOLEAN, isNull, s"$isNull = false;")
    -      addMutableState(javaType(expr.dataType), value,
    -        s"$value = ${defaultValue(expr.dataType)};")
    +      if (expr.nullable) {
    +        addMutableState(JAVA_BOOLEAN, isNull)
    +      }
    +      addMutableState(javaType(expr.dataType), value)
     
           subexprFunctions += s"${addNewFunction(fnName, fn)}($INPUT_ROW);"
    -      val state = SubExprEliminationState(isNull, value)
    +      val state = if (expr.nullable) {
    +        SubExprEliminationState(isNull, value)
    --- End diff --
    
    do we still need it?


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by mgaido91 <gi...@git.apache.org>.
Github user mgaido91 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154195688
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,259 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu children expressions.
    +   */
    +  def getExpressionInputParams(
    +      ctx: CodegenContext,
    +      expr: Expression): Option[(Seq[String], Seq[String])] = {
    +    val (inputAttrs, inputVars) = getInputVarsForChildren(ctx, expr)
    --- End diff --
    
    nit: can we move this closer to when it is used, ie. before `val paramsFromColumns ...`?


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84486 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84486/testReport)** for PR 19813 at commit [`48add65`](https://github.com/apache/spark/commit/48add652f2df45ce6506f9464c10a6425bd92214).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class ExprInputVar(expr: Expression, exprCode: ExprCode)`


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r155535295
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -55,8 +55,23 @@ import org.apache.spark.util.{ParentClassLoader, Utils}
      *                 to null.
      * @param value A term for a (possibly primitive) value of the result of the evaluation. Not
      *              valid if `isNull` is set to `true`.
    + * @param inputRow A term that holds the input row name when generating this code.
    + * @param inputVars A list of [[ExprInputVar]] that holds input variables when generating this code.
      */
    -case class ExprCode(var code: String, var isNull: String, var value: String)
    +case class ExprCode(
    +    var code: String,
    +    var isNull: String,
    +    var value: String,
    +    var inputRow: String = null,
    +    var inputVars: Seq[ExprInputVar] = Seq.empty)
    +
    +/**
    + * Represents an input variable [[ExprCode]] to an evaluation of an [[Expression]].
    + *
    + * @param expr The expression that is evaluated to the input variable.
    + * @param exprCode The [[ExprCode]] that represents the evaluation result for the input variable.
    + */
    +case class ExprInputVar(expr: Expression, exprCode: ExprCode)
    --- End diff --
    
    actually all we need is dataType and nullable, can we just define
    ```
    case class ExprInputVar(exprCode: ExprCode, dataType: DataType, nullable: Boolean)
    ```



---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84208 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84208/testReport)** for PR 19813 at commit [`9f848be`](https://github.com/apache/spark/commit/9f848be45dcc294d6f27f2c6eaeed1907d36f004).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class ExprCode(`
      * `case class ExprInputVar(val expr: Expression, val exprCode: ExprCode)`


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r155536998
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines util methods used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu 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 paramsFromSubExprs = prepareFunctionParams(ctx, subExprs, 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, subExprs) + 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] = {
    +    subExprs.map { subExpr =>
    +      val state = ctx.subExprEliminationExprs(subExpr)
    +      ExprCode(code = "", value = state.value, isNull = state.isNull)
    +    }
    +  }
    +
    +  /**
    +   * Retrieves previous input rows referred by children and deferred expressions.
    +   */
    +  def getInputRowsForChildren(ctx: CodegenContext, expr: Expression): Seq[String] = {
    +    expr.children.flatMap(getInputRows(ctx, _)).distinct
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previous input rows referred by it or deferred expressions
    +   * which are needed to evaluate it.
    +   */
    +  def getInputRows(ctx: CodegenContext, child: Expression): Seq[String] = {
    +    child.flatMap {
    +      // An expression directly evaluates on current input row.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars == null ||
    +          ctx.currentVars(ordinal) == null =>
    +        Seq(ctx.INPUT_ROW)
    +
    +      // An expression which is not evaluated yet. Tracks down to find input rows.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal).code != "" =>
    --- End diff --
    
    if code == "", we don't need to track down the row?


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84422 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84422/testReport)** for PR 19813 at commit [`429afba`](https://github.com/apache/spark/commit/429afbabef6f718870ca3c6caf0712a1e459681f).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154965556
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala ---
    @@ -115,9 +120,35 @@ abstract class Expression extends TreeNode[Expression] {
         }
       }
     
    +  /**
    +   * Returns the input variables to this expression.
    +   */
    +  private def findInputVars(ctx: CodegenContext, eval: ExprCode): Seq[ExprInputVar] = {
    +    if (ctx.currentVars != null) {
    +      val boundRefs = this.collect {
    +        case b @ BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null => (ordinal, b)
    +      }.toMap
    +
    +      ctx.currentVars.zipWithIndex.filter(_._1 != null).flatMap { case (currentVar, idx) =>
    --- End diff --
    
    We don't need to keep the order. Yes, this can be simplified.


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84239/
    Test FAILed.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by mgaido91 <gi...@git.apache.org>.
Github user mgaido91 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154197311
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,259 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu children expressions.
    +   */
    +  def getExpressionInputParams(
    +      ctx: CodegenContext,
    +      expr: Expression): Option[(Seq[String], Seq[String])] = {
    +    val (inputAttrs, inputVars) = getInputVarsForChildren(ctx, expr)
    +    val inputRows = ctx.INPUT_ROW +: getInputRowsForChildren(ctx, expr)
    +    val subExprs = getSubExprInChildren(ctx, expr)
    +
    +    val paramsFromRows = inputRows.distinct.filter(_ != null).map { row =>
    +      (row, s"InternalRow $row")
    +    }
    +    val paramsFromColumns = prepareFunctionParams(ctx, inputAttrs, inputVars)
    +    val paramsFromSubExprs = getParamsForSubExprs(ctx, subExprs)
    +    val paramsLength = getParamLength(ctx, inputAttrs, subExprs) + 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
    +  }
    +
    +  /**
    +   * Given the list of eliminated subexpressions used in the children expressions, returns the
    +   * strings of funtion parameters. The first is the variable names used to call the function,
    +   * the second is the parameters used to declare the function in generated code.
    +   */
    +  def getParamsForSubExprs(
    +      ctx: CodegenContext,
    +      subExprs: Seq[Expression]): Seq[(String, String)] = {
    +    subExprs.flatMap { subExpr =>
    +      val argType = ctx.javaType(subExpr.dataType)
    +
    +      val subExprState = ctx.subExprEliminationExprs(subExpr)
    +      (subExprState.value, subExprState.isNull)
    +
    +      if (!subExpr.nullable || subExprState.isNull == "true" || subExprState.isNull == "false") {
    +        Seq((subExprState.value, s"$argType ${subExprState.value}"))
    +      } else {
    +        Seq((subExprState.value, s"$argType ${subExprState.value}"),
    +          (subExprState.isNull, s"boolean ${subExprState.isNull}"))
    +      }
    +    }.distinct
    +  }
    +
    +  /**
    +   * Retrieves previous input rows referred by children and deferred expressions.
    +   */
    +  def getInputRowsForChildren(ctx: CodegenContext, expr: Expression): Seq[String] = {
    +    expr.children.flatMap(getInputRows(ctx, _)).distinct
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previous input rows referred by it or deferred expressions
    +   * which are needed to evaluate it.
    +   */
    +  def getInputRows(ctx: CodegenContext, child: Expression): Seq[String] = {
    +    child.flatMap {
    +      // An expression directly evaluates on current input row.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars == null ||
    +          ctx.currentVars(ordinal) == null =>
    +        Seq(ctx.INPUT_ROW)
    +
    +      // An expression which is not evaluated yet. Tracks down to find input rows.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal).code != "" =>
    +        trackDownRow(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down input rows referred by the generated code snippet.
    +   */
    +  def trackDownRow(ctx: CodegenContext, exprCode: ExprCode): Seq[String] = {
    +    var exprCodes: List[ExprCode] = List(exprCode)
    +    val inputRows = mutable.ArrayBuffer.empty[String]
    +
    +    while (exprCodes.nonEmpty) {
    +      exprCodes match {
    +        case first :: others =>
    +          exprCodes = others
    +          if (first.inputRow != null) {
    +            inputRows += first.inputRow
    +          }
    +          first.inputVars.foreach { inputVar =>
    +            if (inputVar.exprCode.code != "") {
    +              exprCodes = inputVar.exprCode :: exprCodes
    +            }
    +          }
    +        case _ =>
    +      }
    +    }
    +    inputRows.toSeq
    +  }
    +
    +  /**
    +   * Retrieves previously evaluated columns referred by children and deferred expressions.
    +   * Returned tuple contains the list of expressions and the list of generated codes.
    +   */
    +  def getInputVarsForChildren(
    +      ctx: CodegenContext,
    +      expr: Expression): (Seq[Expression], Seq[ExprCode]) = {
    +    expr.children.flatMap(getInputVars(ctx, _)).distinct.unzip
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previously evaluated columns referred by it or
    +   * deferred expressions which are needed to evaluate it.
    +   */
    +  def getInputVars(ctx: CodegenContext, child: Expression): Seq[(Expression, ExprCode)] = {
    +    if (ctx.currentVars == null) {
    +      return Seq.empty
    +    }
    +
    +    child.flatMap {
    +      // An evaluated variable.
    +      case b @ BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null &&
    +          ctx.currentVars(ordinal).code == "" =>
    +        Seq((b, ctx.currentVars(ordinal)))
    +
    +      // An input variable which is not evaluated yet. Tracks down to find any evaluated variables
    +      // in the expression path.
    +      // E.g., if this expression is "d = c + 1" and "c" is not evaluated. We need to track to
    +      // "c = a + b" and see if "a" and "b" are evaluated. If they are, we need to return them so
    +      // to include them into parameters, if not, we track down further.
    +      case b @ BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null =>
    --- End diff --
    
    nit: I would prefer an if after pattern matching instead of two pattern matching, but it is fine also as it is now. If you prefer to let it like this, please just `b` -> `_` since `b` is never used


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    We need to clearly define what is the current input according to the codegen context. For normal code path, it's always `ctx.INPUT_ROW`, which means when we split codes to methods, we just need to pass `InternalRow ctx.INPUT_ROW` to those methods.
    
    However for whole stage codegen path, it's way more complex:
    1. some of `ctx.currentVars` are just variables, their codes have already been generated before. But some are not. For those whose codes are not generated, they are not valid inputs.
    2. `ctx.currentVars` is not null but has null slots, and `ctx.INPUT_ROW` is not null. Then both `ctx.currentVars` and `ctx.INPUT_ROW` are valid inputs.


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    If we have a clear rule, I think it makes more sense to do this in `CodegenContext`, i.e. having a `def splitExpressions(expressions: Seq[String]): String`, which automatically extract the current inputs and put them into the parameter list.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by mgaido91 <gi...@git.apache.org>.
Github user mgaido91 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154193346
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -55,8 +55,20 @@ import org.apache.spark.util.{ParentClassLoader, Utils}
      *                 to null.
      * @param value A term for a (possibly primitive) value of the result of the evaluation. Not
      *              valid if `isNull` is set to `true`.
    + * @param inputRow A term that holds the input row name when generating this code.
    + * @param inputVars A list of [[ExprInputVar]] that holds input variables when generating this code.
      */
    -case class ExprCode(var code: String, var isNull: String, var value: String)
    +case class ExprCode(
    +    var code: String,
    +    var isNull: String,
    +    var value: String,
    +    var inputRow: String = null,
    +    val inputVars: mutable.ArrayBuffer[ExprInputVar] = mutable.ArrayBuffer.empty)
    --- End diff --
    
    if feasible, I'd prefer an immutable collection here (see my comment above)


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    A more advanced version of `splitExpressions` may work. We can provide necessary function parameters to it.


---

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


[GitHub] spark pull request #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply...

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r153847727
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -1028,12 +1053,18 @@ class CodegenContext {
           //   2. Less code.
           // Currently, we will do this for all non-leaf only expression trees (i.e. expr trees with
           // at least two nodes) as the cost of doing it is expected to be low.
    -      addMutableState(JAVA_BOOLEAN, isNull, s"$isNull = false;")
    +      if (expr.nullable) {
    +        addMutableState(JAVA_BOOLEAN, isNull, s"$isNull = false;")
    --- End diff --
    
    nit: can we eliminate `s"$isNull = false;"` to assign Java default value.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    ping @cloud-fan Previous comments are all addressed. Please review this again. Thanks.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84741 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84741/testReport)** for PR 19813 at commit [`655917c`](https://github.com/apache/spark/commit/655917cadf86ab17b8a730f282db544cb348d63f).


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154948628
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala ---
    @@ -115,9 +120,35 @@ abstract class Expression extends TreeNode[Expression] {
         }
       }
     
    +  /**
    +   * Returns the input variables to this expression.
    +   */
    +  private def findInputVars(ctx: CodegenContext, eval: ExprCode): Seq[ExprInputVar] = {
    +    if (ctx.currentVars != null) {
    +      val boundRefs = this.collect {
    +        case b @ BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null => (ordinal, b)
    +      }.toMap
    +
    +      ctx.currentVars.zipWithIndex.filter(_._1 != null).flatMap { case (currentVar, idx) =>
    --- End diff --
    
    Why these codes? to make sure the returned input vars follow the same order in `ctx.currentVars`?


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156088806
  
    --- 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 --
    
    let's not worried about it now, I feel we can get rid of global variables for normal sub-expr elimination.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156077509
  
    --- 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] = {
    +    subExprs.map { subExpr =>
    +      val state = ctx.subExprEliminationExprs(subExpr)
    +      ExprCode(code = "", value = state.value, isNull = state.isNull)
    +    }
    +  }
    +
    +  /**
    +   * Retrieves previous input rows referred by children and deferred expressions.
    +   */
    +  def getInputRowsForChildren(ctx: CodegenContext, expr: Expression): Seq[String] = {
    +    expr.children.flatMap(getInputRows(ctx, _)).distinct
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previous input rows referred by it or deferred expressions
    +   * which are needed to evaluate it.
    +   */
    +  def getInputRows(ctx: CodegenContext, child: Expression): Seq[String] = {
    +    child.flatMap {
    +      // An expression directly evaluates on current input row.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars == null ||
    +          ctx.currentVars(ordinal) == null =>
    +        Seq(ctx.INPUT_ROW)
    +
    +      // An expression which is not evaluated yet. Tracks down to find input rows.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal).code != "" =>
    +        trackDownRow(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down input rows referred by the generated code snippet.
    +   */
    +  def trackDownRow(ctx: CodegenContext, exprCode: ExprCode): Seq[String] = {
    +    val exprCodes = mutable.Queue[ExprCode](exprCode)
    +    val inputRows = mutable.ArrayBuffer.empty[String]
    +
    +    while (exprCodes.nonEmpty) {
    +      val curExprCode = exprCodes.dequeue()
    +      if (curExprCode.inputRow != null) {
    +        inputRows += curExprCode.inputRow
    +      }
    +      curExprCode.inputVars.foreach { inputVar =>
    +        if (inputVar.exprCode.code != "") {
    +          exprCodes.enqueue(inputVar.exprCode)
    +        }
    +      }
    +    }
    +    inputRows
    +  }
    +
    +  /**
    +   * Retrieves previously evaluated columns referred by children and deferred expressions.
    +   * Returned tuple contains the list of expressions and the list of generated codes.
    +   */
    +  def getInputVarsForChildren(
    +      ctx: CodegenContext,
    +      expr: Expression): (Seq[ExprProperty], Seq[ExprCode]) = {
    +    expr.children.flatMap(getInputVars(ctx, _)).distinct.unzip
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previously evaluated columns referred by it or
    +   * deferred expressions which are needed to evaluate it.
    +   */
    +  def getInputVars(ctx: CodegenContext, child: Expression): Seq[(ExprProperty, ExprCode)] = {
    +    if (ctx.currentVars == null) {
    +      return Seq.empty
    +    }
    +
    +    child.flatMap {
    +      // An evaluated variable.
    +      case b @ BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null &&
    +          ctx.currentVars(ordinal).code == "" =>
    +        Seq(((b.dataType, b.nullable), ctx.currentVars(ordinal)))
    +
    +      // An input variable which is not evaluated yet. Tracks down to find any evaluated variables
    +      // in the expression path.
    +      // E.g., if this expression is "d = c + 1" and "c" is not evaluated. We need to track to
    +      // "c = a + b" and see if "a" and "b" are evaluated. If they are, we need to return them so
    +      // to include them into parameters, if not, we track down further.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null =>
    +        trackDownVar(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down previously evaluated columns referred by the generated code snippet.
    +   */
    +  def trackDownVar(ctx: CodegenContext, exprCode: ExprCode): Seq[(ExprProperty, ExprCode)] = {
    +    val exprCodes = mutable.Queue[ExprCode](exprCode)
    +    val inputVars = mutable.ArrayBuffer.empty[(ExprProperty, ExprCode)]
    +
    +    while (exprCodes.nonEmpty) {
    +      exprCodes.dequeue().inputVars.foreach { inputVar =>
    +        if (inputVar.exprCode.code == "") {
    +          inputVars += (((inputVar.dataType, inputVar.nullable), inputVar.exprCode))
    +        } else {
    +          exprCodes.enqueue(inputVar.exprCode)
    +        }
    +      }
    +    }
    +    inputVars
    +  }
    +
    +  /**
    +   * Helper function to calculate the size of an expression as function parameter.
    +   */
    +  def calculateParamLength(ctx: CodegenContext, input: ExprProperty): Int = {
    +    ctx.javaType(input._1) match {
    +      case (ctx.JAVA_LONG | ctx.JAVA_DOUBLE) if !input._2 => 2
    +      case ctx.JAVA_LONG | ctx.JAVA_DOUBLE => 3
    +      case _ if !input._2 => 1
    +      case _ => 2
    +    }
    +  }
    +
    +  /**
    +   * In Java, a method descriptor is valid only if it represents method parameters with a total
    +   * length of 255 or less. `this` contributes one unit and a parameter of type long or double
    +   * contributes two units.
    +   */
    +  def getParamLength(ctx: CodegenContext, inputs: Seq[ExprProperty]): Int = {
    +    // Initial value is 1 for `this`.
    +    1 + inputs.map(calculateParamLength(ctx, _)).sum
    +  }
    +
    +  /**
    +   * Given the lists of input attributes and variables to this expression, returns the strings of
    +   * funtion parameters. The first is the variable names used to call the function, the second is
    +   * the parameters used to declare the function in generated code.
    +   */
    +  def prepareFunctionParams(
    +      ctx: CodegenContext,
    +      inputAttrs: Seq[ExprProperty],
    +      inputVars: Seq[ExprCode]): Seq[(String, String)] = {
    --- End diff --
    
    Sure.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    thanks, merging to master!


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156297517
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala ---
    @@ -236,4 +237,24 @@ class WholeStageCodegenSuite extends QueryTest with SharedSQLContext {
           }
         }
       }
    +
    +  test("SPARK-22551: Fix 64kb limit for deeply nested expressions under wholestage codegen") {
    +    import testImplicits._
    +    withTempPath { dir =>
    +      val path = dir.getCanonicalPath
    +      val df = Seq(("abc", 1)).toDF("key", "int")
    +      df.write.parquet(path)
    +
    +      var strExpr: Expression = col("key").expr
    +      for (_ <- 1 to 150) {
    +        strExpr = Decode(Encode(strExpr, Literal("utf-8")), Literal("utf-8"))
    +      }
    +      val expressions = Seq(If(EqualTo(strExpr, strExpr), strExpr, strExpr))
    +
    +      val df2 = spark.read.parquet(path).select(expressions.map(Column(_)): _*)
    +      val plan = df2.queryExecution.executedPlan
    +      assert(plan.find(_.isInstanceOf[WholeStageCodegenExec]).isDefined)
    --- End diff --
    
    Since it is in testing (`Utils.isTesting`), the compilation error won't be ignored.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84207 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84207/testReport)** for PR 19813 at commit [`9f848be`](https://github.com/apache/spark/commit/9f848be45dcc294d6f27f2c6eaeed1907d36f004).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class ExprCode(`
      * `case class ExprInputVar(val expr: Expression, val exprCode: ExprCode)`


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156238152
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala ---
    @@ -61,7 +61,11 @@ case class BoundReference(ordinal: Int, dataType: DataType, nullable: Boolean)
       override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
         if (ctx.currentVars != null && ctx.currentVars(ordinal) != null) {
           val oev = ctx.currentVars(ordinal)
    -      ev.isNull = oev.isNull
    +      if (nullable) {
    --- End diff --
    
    oh, right.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84371 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84371/testReport)** for PR 19813 at commit [`aa3db2e`](https://github.com/apache/spark/commit/aa3db2edca66ab04ecb8fbd54750cbd46544eb1d).


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r155578442
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines util methods used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu 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 paramsFromSubExprs = prepareFunctionParams(ctx, subExprs, 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, subExprs) + paramsFromRows.length
    +    // Maximum allowed parameter number for Java's method descriptor.
    +    if (paramsLength > 255) {
    --- End diff --
    
    It is true for all JVMs. See [here](https://docs.oracle.com/javase/specs/jvms/se7/html/jvms-4.html#jvms-4.11)


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r155524869
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -55,8 +55,23 @@ import org.apache.spark.util.{ParentClassLoader, Utils}
      *                 to null.
      * @param value A term for a (possibly primitive) value of the result of the evaluation. Not
      *              valid if `isNull` is set to `true`.
    + * @param inputRow A term that holds the input row name when generating this code.
    + * @param inputVars A list of [[ExprInputVar]] that holds input variables when generating this code.
      */
    -case class ExprCode(var code: String, var isNull: String, var value: String)
    +case class ExprCode(
    +    var code: String,
    +    var isNull: String,
    +    var value: String,
    +    var inputRow: String = null,
    +    var inputVars: Seq[ExprInputVar] = Seq.empty)
    +
    +/**
    + * Represents an input variable [[ExprCode]] to an evaluation of an [[Expression]].
    + *
    + * @param expr The expression that is evaluated to the input variable.
    + * @param exprCode The [[ExprCode]] that represents the evaluation result for the input variable.
    + */
    +case class ExprInputVar(expr: Expression, exprCode: ExprCode)
    --- End diff --
    
    so the `expr` is always `BoundReference`?


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    > Probably, we'd better to move this discussion to the jira?
    
    I summary it and also post the design doc link to the jira.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    This is only valid when by coincidence the all expressions involved can use statement as output. As I looked at the codebase, I think only few expressions can output statement. This may not apply generally to reduce code size.


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84330 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84330/testReport)** for PR 19813 at commit [`fd87e9b`](https://github.com/apache/spark/commit/fd87e9ba324e0b45685e7873884a4fa7a6feaf17).


---

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


[GitHub] spark pull request #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply...

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r153852592
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala ---
    @@ -115,9 +118,240 @@ abstract class Expression extends TreeNode[Expression] {
         }
       }
     
    +  /**
    +   * Records current input row and variables for this expression into created `ExprCode`.
    +   */
    +  private def populateInputs(ctx: CodegenContext, eval: ExprCode): Unit = {
    +    if (ctx.INPUT_ROW != null) {
    +      eval.inputRow = ctx.INPUT_ROW
    +    }
    +    if (ctx.currentVars != null) {
    +      val boundRefs = this.collect {
    +        case b @ BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null => (ordinal, b)
    +      }.toMap
    +
    +      ctx.currentVars.zipWithIndex.filter(_._1 != null).foreach { case (currentVar, idx) =>
    +        if (boundRefs.contains(idx)) {
    +          val inputVar = ExprInputVar(boundRefs(idx), exprCode = currentVar)
    +          eval.inputVars += inputVar
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Returns the eliminated subexpressions in the children expressions.
    +   */
    +  private def getSubExprInChildren(ctx: CodegenContext): Seq[Expression] = {
    +    children.flatMap { child =>
    +      child.collect {
    +        case e if ctx.subExprEliminationExprs.contains(e) => e
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Given the list of eliminated subexpressions used in the children expressions, returns the
    +   * strings of funtion parameters. The first is the variable names used to call the function,
    +   * the second is the parameters used to declare the function in generated code.
    +   */
    +  private def getParamsForSubExprs(
    +      ctx: CodegenContext,
    +      subExprs: Seq[Expression]): (Seq[String], Seq[String]) = {
    +    subExprs.flatMap { subExpr =>
    +      val arguType = ctx.javaType(subExpr.dataType)
    +
    +      val subExprState = ctx.subExprEliminationExprs(subExpr)
    +      (subExprState.value, subExprState.isNull)
    +
    +      if (!subExpr.nullable || subExprState.isNull == "true" || subExprState.isNull == "false") {
    +        Seq((subExprState.value, s"$arguType ${subExprState.value}"))
    +      } else {
    +        Seq((subExprState.value, s"$arguType ${subExprState.value}"),
    +          (subExprState.isNull, s"boolean ${subExprState.isNull}"))
    +      }
    +    }.unzip
    +  }
    +
    +  /**
    +   * Retrieves previous input rows referred by children and deferred expressions.
    +   */
    +  private def getInputRowsForChildren(ctx: CodegenContext): Seq[String] = {
    +    children.flatMap(getInputRows(ctx, _)).distinct
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previous input rows referred by it or deferred expressions
    +   * which are needed to evaluate it.
    +   */
    +  private def getInputRows(ctx: CodegenContext, child: Expression): Seq[String] = {
    +    child.flatMap {
    +      // An expression directly evaluates on current input row.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars == null ||
    +          ctx.currentVars(ordinal) == null =>
    +        Seq(ctx.INPUT_ROW)
    +
    +      // An expression which is not evaluated yet. Tracks down to find input rows.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal).code != "" =>
    +        trackDownRow(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down input rows referred by the generated code snippet.
    +   */
    +  private def trackDownRow(ctx: CodegenContext, exprCode: ExprCode): Seq[String] = {
    +    var exprCodes: List[ExprCode] = List(exprCode)
    +    val inputRows = mutable.ArrayBuffer.empty[String]
    +
    +    while (exprCodes.nonEmpty) {
    +      exprCodes match {
    +        case first :: others =>
    +          exprCodes = others
    +          if (first.inputRow != null) {
    +            inputRows += first.inputRow
    +          }
    +          first.inputVars.foreach { inputVar =>
    +            if (inputVar.exprCode.code != "") {
    +              exprCodes = inputVar.exprCode :: exprCodes
    +            }
    +          }
    +        case _ =>
    +      }
    +    }
    +    inputRows.toSeq
    +  }
    +
    +  /**
    +   * Retrieves previously evaluated columns referred by children and deferred expressions.
    +   * Returned tuple contains the list of expressions and the list of generated codes.
    +   */
    +  private def getInputVarsForChildren(ctx: CodegenContext): (Seq[Expression], Seq[ExprCode]) = {
    +    children.flatMap(getInputVars(ctx, _)).distinct.unzip
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previously evaluated columns referred by it or
    +   * deferred expressions which are needed to evaluate it.
    +   */
    +  private def getInputVars(ctx: CodegenContext, child: Expression): Seq[(Expression, ExprCode)] = {
    +    if (ctx.currentVars == null) {
    +      return Seq.empty
    +    }
    +
    +    child.flatMap {
    +      // An evaluated variable.
    +      case b @ BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null &&
    +          ctx.currentVars(ordinal).code == "" =>
    +        Seq((b, ctx.currentVars(ordinal)))
    +
    +      // An input variable which is not evaluated yet. Tracks down to find any evaluated variables
    +      // in the expression path.
    +      // E.g., if this expression is "d = c + 1" and "c" is not evaluated. We need to track to
    +      // "c = a + b" and see if "a" and "b" are evaluated. If they are, we need to return them so
    +      // to include them into parameters, if not, we tract down further.
    --- End diff --
    
    nit: `tract` -> `track`?


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    retest this please.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84370 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84370/testReport)** for PR 19813 at commit [`98850a6`](https://github.com/apache/spark/commit/98850a67c99b02b9ba98ec544f7d0d5671142716).


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156291204
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -55,8 +55,45 @@ import org.apache.spark.util.{ParentClassLoader, Utils}
      *                 to null.
      * @param value A term for a (possibly primitive) value of the result of the evaluation. Not
      *              valid if `isNull` is set to `true`.
    + * @param inputRow A term that holds the input row name when generating this code.
    + * @param inputVars A list of [[ExprInputVar]] that holds input variables when generating this code.
      */
    -case class ExprCode(var code: String, var isNull: String, var value: String)
    +case class ExprCode(
    +    var code: String,
    +    var isNull: String,
    +    var value: String,
    +    var inputRow: String = null,
    +    var inputVars: Seq[ExprInputVar] = Seq.empty) {
    +
    +  // Returns true if this value is a literal.
    +  def isLiteral(): Boolean = {
    +    assert(value.nonEmpty, "ExprCode.value can't be empty string.")
    +
    +    if (value == "true" || value == "false" || value == "null") {
    +      true
    +    } else {
    +      // The valid characters for the first character of a Java variable is [a-zA-Z_$].
    +      value.head match {
    +        case v if v >= 'a' && v <= 'z' => false
    --- End diff --
    
    It is doable. Instead of `isLiteral`, we may call it `isValidJavaVariable`.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r155684442
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines util methods used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu 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 paramsFromSubExprs = prepareFunctionParams(ctx, subExprs, 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, subExprs) + 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] = {
    +    subExprs.map { subExpr =>
    +      val state = ctx.subExprEliminationExprs(subExpr)
    +      ExprCode(code = "", value = state.value, isNull = state.isNull)
    +    }
    +  }
    +
    +  /**
    +   * Retrieves previous input rows referred by children and deferred expressions.
    +   */
    +  def getInputRowsForChildren(ctx: CodegenContext, expr: Expression): Seq[String] = {
    +    expr.children.flatMap(getInputRows(ctx, _)).distinct
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previous input rows referred by it or deferred expressions
    +   * which are needed to evaluate it.
    +   */
    +  def getInputRows(ctx: CodegenContext, child: Expression): Seq[String] = {
    +    child.flatMap {
    +      // An expression directly evaluates on current input row.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars == null ||
    +          ctx.currentVars(ordinal) == null =>
    +        Seq(ctx.INPUT_ROW)
    +
    +      // An expression which is not evaluated yet. Tracks down to find input rows.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal).code != "" =>
    +        trackDownRow(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down input rows referred by the generated code snippet.
    +   */
    +  def trackDownRow(ctx: CodegenContext, exprCode: ExprCode): Seq[String] = {
    +    val exprCodes = mutable.Queue[ExprCode](exprCode)
    +    val inputRows = mutable.ArrayBuffer.empty[String]
    +
    +    while (exprCodes.nonEmpty) {
    +      val curExprCode = exprCodes.dequeue()
    +      if (curExprCode.inputRow != null) {
    +        inputRows += curExprCode.inputRow
    +      }
    +      curExprCode.inputVars.foreach { inputVar =>
    +        if (inputVar.exprCode.code != "") {
    +          exprCodes.enqueue(inputVar.exprCode)
    +        }
    +      }
    +    }
    +    inputRows
    +  }
    +
    +  /**
    +   * Retrieves previously evaluated columns referred by children and deferred expressions.
    +   * Returned tuple contains the list of expressions and the list of generated codes.
    +   */
    +  def getInputVarsForChildren(
    +      ctx: CodegenContext,
    +      expr: Expression): (Seq[Expression], Seq[ExprCode]) = {
    +    expr.children.flatMap(getInputVars(ctx, _)).distinct.unzip
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previously evaluated columns referred by it or
    +   * deferred expressions which are needed to evaluate it.
    +   */
    +  def getInputVars(ctx: CodegenContext, child: Expression): Seq[(Expression, ExprCode)] = {
    +    if (ctx.currentVars == null) {
    +      return Seq.empty
    +    }
    +
    +    child.flatMap {
    +      // An evaluated variable.
    +      case b @ BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null &&
    +          ctx.currentVars(ordinal).code == "" =>
    +        Seq((b, ctx.currentVars(ordinal)))
    +
    +      // An input variable which is not evaluated yet. Tracks down to find any evaluated variables
    +      // in the expression path.
    +      // E.g., if this expression is "d = c + 1" and "c" is not evaluated. We need to track to
    +      // "c = a + b" and see if "a" and "b" are evaluated. If they are, we need to return them so
    +      // to include them into parameters, if not, we track down further.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null =>
    +        trackDownVar(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down previously evaluated columns referred by the generated code snippet.
    +   */
    +  def trackDownVar(ctx: CodegenContext, exprCode: ExprCode): Seq[(Expression, ExprCode)] = {
    +    val exprCodes = mutable.Queue[ExprCode](exprCode)
    +    val inputVars = mutable.ArrayBuffer.empty[(Expression, ExprCode)]
    +
    +    while (exprCodes.nonEmpty) {
    +      exprCodes.dequeue().inputVars.foreach { inputVar =>
    +        if (inputVar.exprCode.code == "") {
    +          inputVars += ((inputVar.expr, inputVar.exprCode))
    +        } else {
    +          exprCodes.enqueue(inputVar.exprCode)
    +        }
    +      }
    +    }
    +    inputVars
    +  }
    +
    +  /**
    +   * Helper function to calculate the size of an expression as function parameter.
    +   */
    +  def calculateParamLength(ctx: CodegenContext, input: Expression): Int = {
    +    ctx.javaType(input.dataType) match {
    +      case (ctx.JAVA_LONG | ctx.JAVA_DOUBLE) if !input.nullable => 2
    +      case ctx.JAVA_LONG | ctx.JAVA_DOUBLE => 3
    +      case _ if !input.nullable => 1
    +      case _ => 2
    +    }
    +  }
    +
    +  /**
    +   * In Java, a method descriptor is valid only if it represents method parameters with a total
    +   * length of 255 or less. `this` contributes one unit and a parameter of type long or double
    +   * contributes two units.
    +   */
    +  def getParamLength(
    +      ctx: CodegenContext,
    +      inputs: Seq[Expression],
    +      subExprs: Seq[Expression]): Int = {
    +    // Initial value is 1 for `this`.
    +    1 + (inputs ++ subExprs).distinct.map(calculateParamLength(ctx, _)).sum
    +  }
    +
    +  /**
    +   * Given the lists of input attributes and variables to this expression, returns the strings of
    +   * funtion parameters. The first is the variable names used to call the function, the second is
    +   * the parameters used to declare the function in generated code.
    +   */
    +  def prepareFunctionParams(
    +      ctx: CodegenContext,
    +      inputAttrs: Seq[Expression],
    +      inputVars: Seq[ExprCode]): Seq[(String, String)] = {
    +    inputAttrs.zip(inputVars).flatMap { case (input, ev) =>
    +      val argType = ctx.javaType(input.dataType)
    +
    +      if (!input.nullable || ev.isNull == "true" || ev.isNull == "false") {
    +        Seq((ev.value, s"$argType ${ev.value}"))
    +      } else {
    +        Seq((ev.value, s"$argType ${ev.value}"), (ev.isNull, s"boolean ${ev.isNull}"))
    --- End diff --
    
    The strategy is useful when only one parameter. We may have many parameters now, seems to me it is a bit inefficient to create temp variables for all parameters.
    
    We don't need regex to detect literals. Since we know the first character for Java variable is [a-zA-Z_$], can we just determine whether `ev.value` is a variable by looking at its first character?


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    I think that this PR is necessary to fix SPARK-22868 and SPARK-22869


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154866923
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu 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 paramsFromSubExprs = prepareFunctionParams(ctx, subExprs, subExprCodes)
    --- End diff --
    
    If I get rid of the part of extracting subexpressions as parameters, some tests will be failed.
    
    Because hash aggregation uses subexpression elimination under wholestage codegen, this PR enables splitting expression codes under wholestage codegen, so it inevitably needs to include subexpression parameters.
    
    This PR doesn't yet to support `splitExpressions` in wholestage codegen. This is only applied to split codes of deeply nested expression now. But the added API can be applied to `splitExpressions` later. That is a TODO.
    



---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84182 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84182/testReport)** for PR 19813 at commit [`65d07d5`](https://github.com/apache/spark/commit/65d07d525344e1d00457d2f538b2ef0b1c38a8e8).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156074485
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala ---
    @@ -105,6 +105,11 @@ abstract class Expression extends TreeNode[Expression] {
           val isNull = ctx.freshName("isNull")
           val value = ctx.freshName("value")
           val eval = doGenCode(ctx, ExprCode("", isNull, value))
    +
    +      // Records current input row and variables of this expression.
    +      eval.inputRow = ctx.INPUT_ROW
    +      eval.inputVars = findInputVars(ctx, eval)
    --- End diff --
    
    Ok. Let me try it in next commit.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84422 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84422/testReport)** for PR 19813 at commit [`429afba`](https://github.com/apache/spark/commit/429afbabef6f718870ca3c6caf0712a1e459681f).


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84742/
    Test FAILed.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156373480
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegenSuite.scala ---
    @@ -0,0 +1,168 @@
    +/*
    + * 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 org.apache.spark.SparkFunSuite
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.types.IntegerType
    +
    +class ExpressionCodegenSuite extends SparkFunSuite {
    +
    +  test("Returns eliminated subexpressions for expression") {
    +    val ctx = new CodegenContext()
    +    val subExpr = Add(Literal(1), Literal(2))
    +    val exprs = Seq(Add(subExpr, Literal(3)), Add(subExpr, Literal(4)))
    +
    +    ctx.generateExpressions(exprs, doSubexpressionElimination = true)
    +    val subexpressions = ExpressionCodegen.getSubExprInChildren(ctx, exprs(0))
    +    assert(subexpressions.length == 1 && subexpressions(0) == subExpr)
    +  }
    +
    +  test("Gets parameters for subexpressions") {
    +    val ctx = new CodegenContext()
    +    val subExprs = Seq(
    +      Add(Literal(1), AttributeReference("a", IntegerType, nullable = false)()), // non-nullable
    +      Add(Literal(2), AttributeReference("b", IntegerType, nullable = true)()))  // nullable
    +
    +    ctx.subExprEliminationExprs.put(subExprs(0), SubExprEliminationState("false", "value1"))
    +    ctx.subExprEliminationExprs.put(subExprs(1), SubExprEliminationState("isNull2", "value2"))
    +
    +    val subExprCodes = ExpressionCodegen.getSubExprCodes(ctx, subExprs)
    +    val subVars = subExprs.zip(subExprCodes).map { case (expr, exprCode) =>
    +      ExprInputVar(exprCode, expr.dataType, expr.nullable)
    +    }
    +    val params = ExpressionCodegen.prepareFunctionParams(ctx, subVars)
    +    assert(params.length == 3)
    +    assert(params(0) == Tuple2("value1", "int value1"))
    +    assert(params(1) == Tuple2("value2", "int value2"))
    +    assert(params(2) == Tuple2("isNull2", "boolean isNull2"))
    +  }
    +
    +  test("Returns input variables for expression: current variables") {
    +    val ctx = new CodegenContext()
    +    val currentVars = Seq(
    +      ExprCode("", isNull = "false", value = "value1"),             // evaluated
    +      ExprCode("", isNull = "isNull2", value = "value2"),           // evaluated
    +      ExprCode("fake code;", isNull = "isNull3", value = "value3")) // not evaluated
    +    ctx.currentVars = currentVars
    +    ctx.INPUT_ROW = null
    +
    +    val expr = If(Literal(false),
    +      Add(BoundReference(0, IntegerType, nullable = false),
    +          BoundReference(1, IntegerType, nullable = true)),
    +        BoundReference(2, IntegerType, nullable = true))
    +
    +    val inputVars = ExpressionCodegen.getInputVarsForChildren(ctx, expr)
    +    // Only two evaluated variables included.
    +    assert(inputVars.length == 2)
    +    assert(inputVars(0).dataType == IntegerType && inputVars(0).nullable == false)
    +    assert(inputVars(1).dataType == IntegerType && inputVars(1).nullable == true)
    +    assert(inputVars(0).exprCode == currentVars(0))
    +    assert(inputVars(1).exprCode == currentVars(1))
    +
    +    val params = ExpressionCodegen.prepareFunctionParams(ctx, inputVars)
    +    assert(params.length == 3)
    +    assert(params(0) == Tuple2("value1", "int value1"))
    +    assert(params(1) == Tuple2("value2", "int value2"))
    +    assert(params(2) == Tuple2("isNull2", "boolean isNull2"))
    +  }
    +
    +  test("Returns input variables for expression: deferred variables") {
    +    val ctx = new CodegenContext()
    +
    +    // The referred column is not evaluated yet. But it depends on an evaluated column from
    +    // other operator.
    +    val currentVars = Seq(ExprCode("fake code;", isNull = "isNull1", value = "value1"))
    +
    +    // currentVars(0) depends on this evaluated column.
    +    currentVars(0).inputVars = Seq(ExprInputVar(ExprCode("", isNull = "isNull2", value = "value2"),
    +      dataType = IntegerType, nullable = true))
    +    ctx.currentVars = currentVars
    +    ctx.INPUT_ROW = null
    +
    +    val expr = Add(Literal(1), BoundReference(0, IntegerType, nullable = false))
    +    val inputVars = ExpressionCodegen.getInputVarsForChildren(ctx, expr)
    +    assert(inputVars.length == 1)
    +    assert(inputVars(0).dataType == IntegerType && inputVars(0).nullable == true)
    +
    +    val params = ExpressionCodegen.prepareFunctionParams(ctx, inputVars)
    +    assert(params.length == 2)
    +    assert(params(0) == Tuple2("value2", "int value2"))
    +    assert(params(1) == Tuple2("isNull2", "boolean isNull2"))
    +  }
    +
    +  test("Returns input rows for expression") {
    +    val ctx = new CodegenContext()
    +    ctx.currentVars = null
    +    ctx.INPUT_ROW = "i"
    +
    +    val expr = Add(BoundReference(0, IntegerType, nullable = false),
    +      BoundReference(1, IntegerType, nullable = true))
    +    val inputRows = ExpressionCodegen.getInputRowsForChildren(ctx, expr)
    +    assert(inputRows.length == 1)
    +    assert(inputRows(0) == "i")
    +  }
    +
    +  test("Returns input rows for expression: deferred expression") {
    +    val ctx = new CodegenContext()
    +
    +    // The referred column is not evaluated yet. But it depends on an input row from
    +    // other operator.
    +    val currentVars = Seq(ExprCode("fake code;", isNull = "isNull1", value = "value1"))
    +    currentVars(0).inputRow = "inputadaptor_row1"
    +    ctx.currentVars = currentVars
    +    ctx.INPUT_ROW = null
    +
    +    val expr = Add(Literal(1), BoundReference(0, IntegerType, nullable = false))
    +    val inputRows = ExpressionCodegen.getInputRowsForChildren(ctx, expr)
    +    assert(inputRows.length == 1)
    +    assert(inputRows(0) == "inputadaptor_row1")
    +  }
    +
    --- End diff --
    
    Yea, added one more test.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156287015
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -1028,12 +1078,17 @@ class CodegenContext {
           //   2. Less code.
           // Currently, we will do this for all non-leaf only expression trees (i.e. expr trees with
           // at least two nodes) as the cost of doing it is expected to be low.
    -      addMutableState(JAVA_BOOLEAN, isNull, s"$isNull = false;")
    -      addMutableState(javaType(expr.dataType), value,
    -        s"$value = ${defaultValue(expr.dataType)};")
    +      if (expr.nullable) {
    +        addMutableState(JAVA_BOOLEAN, isNull)
    +      }
    +      addMutableState(javaType(expr.dataType), value)
     
           subexprFunctions += s"${addNewFunction(fnName, fn)}($INPUT_ROW);"
    -      val state = SubExprEliminationState(isNull, value)
    +      val state = if (expr.nullable) {
    +        SubExprEliminationState(isNull, value)
    --- End diff --
    
    I think this is still needed. 


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84742 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84742/testReport)** for PR 19813 at commit [`c083a79`](https://github.com/apache/spark/commit/c083a7955cd6fb54e0448176d9684496fae48e6f).


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    I'm not sure, it depends on how many places we already doing it and what's the drawback if we forbid it. Let's have a design doc and gather more feedbacks.
    
    Thanks for your understanding! both PRs are reverted.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154953139
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala ---
    @@ -236,4 +237,22 @@ class WholeStageCodegenSuite extends QueryTest with SharedSQLContext {
           }
         }
       }
    +
    +  test("SPARK-22551: Fix 64kb limit for deeply nested expressions under wholestage codegen") {
    +    import testImplicits._
    +    withTempPath { dir =>
    +      val path = dir.getCanonicalPath
    +      val df = Seq(("abc", 1)).toDF("key", "int")
    +      df.write.parquet(path)
    +
    +      var strExpr: Expression = col("key").expr
    +      for (_ <- 1 to 150) {
    +        strExpr = Decode(Encode(strExpr, Literal("utf-8")), Literal("utf-8"))
    +      }
    +      val expressions = Seq(If(EqualTo(strExpr, strExpr), strExpr, strExpr))
    +
    +      val df2 = spark.read.parquet(path).select(expressions.map(Column(_)): _*)
    --- End diff --
    
    Yes.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156062977
  
    --- 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] = {
    +    subExprs.map { subExpr =>
    +      val state = ctx.subExprEliminationExprs(subExpr)
    +      ExprCode(code = "", value = state.value, isNull = state.isNull)
    +    }
    +  }
    +
    +  /**
    +   * Retrieves previous input rows referred by children and deferred expressions.
    +   */
    +  def getInputRowsForChildren(ctx: CodegenContext, expr: Expression): Seq[String] = {
    +    expr.children.flatMap(getInputRows(ctx, _)).distinct
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previous input rows referred by it or deferred expressions
    +   * which are needed to evaluate it.
    +   */
    +  def getInputRows(ctx: CodegenContext, child: Expression): Seq[String] = {
    +    child.flatMap {
    +      // An expression directly evaluates on current input row.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars == null ||
    +          ctx.currentVars(ordinal) == null =>
    +        Seq(ctx.INPUT_ROW)
    +
    +      // An expression which is not evaluated yet. Tracks down to find input rows.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal).code != "" =>
    +        trackDownRow(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down input rows referred by the generated code snippet.
    +   */
    +  def trackDownRow(ctx: CodegenContext, exprCode: ExprCode): Seq[String] = {
    +    val exprCodes = mutable.Queue[ExprCode](exprCode)
    +    val inputRows = mutable.ArrayBuffer.empty[String]
    +
    +    while (exprCodes.nonEmpty) {
    +      val curExprCode = exprCodes.dequeue()
    +      if (curExprCode.inputRow != null) {
    +        inputRows += curExprCode.inputRow
    +      }
    +      curExprCode.inputVars.foreach { inputVar =>
    +        if (inputVar.exprCode.code != "") {
    +          exprCodes.enqueue(inputVar.exprCode)
    +        }
    +      }
    +    }
    +    inputRows
    +  }
    +
    +  /**
    +   * Retrieves previously evaluated columns referred by children and deferred expressions.
    +   * Returned tuple contains the list of expressions and the list of generated codes.
    +   */
    +  def getInputVarsForChildren(
    +      ctx: CodegenContext,
    +      expr: Expression): (Seq[ExprProperty], Seq[ExprCode]) = {
    --- End diff --
    
    is `(Seq[ExprProperty], Seq[ExprCode])` just `Seq[ExprInputVar]`?


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84260 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84260/testReport)** for PR 19813 at commit [`8c7f749`](https://github.com/apache/spark/commit/8c7f7496e610fdf4b512c57efd108ccf0238b126).
     * This patch **fails SparkR unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by mgaido91 <gi...@git.apache.org>.
Github user mgaido91 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154194781
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,259 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu children expressions.
    +   */
    +  def getExpressionInputParams(
    +      ctx: CodegenContext,
    +      expr: Expression): Option[(Seq[String], Seq[String])] = {
    +    val (inputAttrs, inputVars) = getInputVarsForChildren(ctx, expr)
    +    val inputRows = ctx.INPUT_ROW +: getInputRowsForChildren(ctx, expr)
    --- End diff --
    
    I see that as of now `ctx.INPUT_ROW` can't be `null` here, but can we add an assertion and maybe a comment for the reason of this? I think this would help people who might want to reuse this function in the future.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by mgaido91 <gi...@git.apache.org>.
Github user mgaido91 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154200835
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,259 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu children expressions.
    +   */
    +  def getExpressionInputParams(
    +      ctx: CodegenContext,
    +      expr: Expression): Option[(Seq[String], Seq[String])] = {
    +    val (inputAttrs, inputVars) = getInputVarsForChildren(ctx, expr)
    +    val inputRows = ctx.INPUT_ROW +: getInputRowsForChildren(ctx, expr)
    +    val subExprs = getSubExprInChildren(ctx, expr)
    +
    +    val paramsFromRows = inputRows.distinct.filter(_ != null).map { row =>
    +      (row, s"InternalRow $row")
    +    }
    +    val paramsFromColumns = prepareFunctionParams(ctx, inputAttrs, inputVars)
    +    val paramsFromSubExprs = getParamsForSubExprs(ctx, subExprs)
    +    val paramsLength = getParamLength(ctx, inputAttrs, subExprs) + 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
    +  }
    +
    +  /**
    +   * Given the list of eliminated subexpressions used in the children expressions, returns the
    +   * strings of funtion parameters. The first is the variable names used to call the function,
    +   * the second is the parameters used to declare the function in generated code.
    +   */
    +  def getParamsForSubExprs(
    +      ctx: CodegenContext,
    +      subExprs: Seq[Expression]): Seq[(String, String)] = {
    +    subExprs.flatMap { subExpr =>
    +      val argType = ctx.javaType(subExpr.dataType)
    +
    +      val subExprState = ctx.subExprEliminationExprs(subExpr)
    +      (subExprState.value, subExprState.isNull)
    +
    +      if (!subExpr.nullable || subExprState.isNull == "true" || subExprState.isNull == "false") {
    +        Seq((subExprState.value, s"$argType ${subExprState.value}"))
    +      } else {
    +        Seq((subExprState.value, s"$argType ${subExprState.value}"),
    +          (subExprState.isNull, s"boolean ${subExprState.isNull}"))
    +      }
    +    }.distinct
    +  }
    +
    +  /**
    +   * Retrieves previous input rows referred by children and deferred expressions.
    +   */
    +  def getInputRowsForChildren(ctx: CodegenContext, expr: Expression): Seq[String] = {
    +    expr.children.flatMap(getInputRows(ctx, _)).distinct
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previous input rows referred by it or deferred expressions
    +   * which are needed to evaluate it.
    +   */
    +  def getInputRows(ctx: CodegenContext, child: Expression): Seq[String] = {
    +    child.flatMap {
    +      // An expression directly evaluates on current input row.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars == null ||
    +          ctx.currentVars(ordinal) == null =>
    +        Seq(ctx.INPUT_ROW)
    +
    +      // An expression which is not evaluated yet. Tracks down to find input rows.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal).code != "" =>
    +        trackDownRow(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down input rows referred by the generated code snippet.
    +   */
    +  def trackDownRow(ctx: CodegenContext, exprCode: ExprCode): Seq[String] = {
    +    var exprCodes: List[ExprCode] = List(exprCode)
    +    val inputRows = mutable.ArrayBuffer.empty[String]
    +
    +    while (exprCodes.nonEmpty) {
    +      exprCodes match {
    +        case first :: others =>
    +          exprCodes = others
    +          if (first.inputRow != null) {
    +            inputRows += first.inputRow
    +          }
    +          first.inputVars.foreach { inputVar =>
    +            if (inputVar.exprCode.code != "") {
    +              exprCodes = inputVar.exprCode :: exprCodes
    +            }
    +          }
    +        case _ =>
    +      }
    +    }
    +    inputRows.toSeq
    +  }
    +
    +  /**
    +   * Retrieves previously evaluated columns referred by children and deferred expressions.
    +   * Returned tuple contains the list of expressions and the list of generated codes.
    +   */
    +  def getInputVarsForChildren(
    +      ctx: CodegenContext,
    +      expr: Expression): (Seq[Expression], Seq[ExprCode]) = {
    +    expr.children.flatMap(getInputVars(ctx, _)).distinct.unzip
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previously evaluated columns referred by it or
    +   * deferred expressions which are needed to evaluate it.
    +   */
    +  def getInputVars(ctx: CodegenContext, child: Expression): Seq[(Expression, ExprCode)] = {
    +    if (ctx.currentVars == null) {
    +      return Seq.empty
    +    }
    +
    +    child.flatMap {
    +      // An evaluated variable.
    +      case b @ BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null &&
    +          ctx.currentVars(ordinal).code == "" =>
    +        Seq((b, ctx.currentVars(ordinal)))
    +
    +      // An input variable which is not evaluated yet. Tracks down to find any evaluated variables
    +      // in the expression path.
    +      // E.g., if this expression is "d = c + 1" and "c" is not evaluated. We need to track to
    +      // "c = a + b" and see if "a" and "b" are evaluated. If they are, we need to return them so
    +      // to include them into parameters, if not, we track down further.
    +      case b @ BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null =>
    +        trackDownVar(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down previously evaluated columns referred by the generated code snippet.
    +   */
    +  def trackDownVar(ctx: CodegenContext, exprCode: ExprCode): Seq[(Expression, ExprCode)] = {
    +    var exprCodes: List[ExprCode] = List(exprCode)
    +    val inputVars = mutable.ArrayBuffer.empty[(Expression, ExprCode)]
    +
    +    while (exprCodes.nonEmpty) {
    +      exprCodes match {
    +        case first :: others =>
    +          exprCodes = others
    +          first.inputVars.foreach { inputVar =>
    +            if (inputVar.exprCode.code == "") {
    +              inputVars += ((inputVar.expr, inputVar.exprCode))
    +            } else {
    +              exprCodes = inputVar.exprCode :: exprCodes
    +            }
    +          }
    +        case _ =>
    +      }
    +    }
    +    inputVars.toSeq
    +  }
    +
    +  /**
    +   * Helper function to calculate the size of an expression as function parameter.
    +   */
    +  def calculateParamLength(ctx: CodegenContext, input: Expression): Int = {
    +    ctx.javaType(input.dataType) match {
    +      case (ctx.JAVA_LONG | ctx.JAVA_DOUBLE) if !input.nullable => 2
    +      case ctx.JAVA_LONG | ctx.JAVA_DOUBLE => 3
    +      case _ if !input.nullable => 1
    +      case _ => 2
    +    }
    +  }
    +
    +  /**
    +   * In Java, a method descriptor is valid only if it represents method parameters with a total
    +   * length of 255 or less. `this` contributes one unit and a parameter of type long or double
    +   * contributes two units.
    +   */
    +  def getParamLength(
    +      ctx: CodegenContext,
    +      inputs: Seq[Expression],
    +      subExprs: Seq[Expression]): Int = {
    +    // Start value is 1 for `this`.
    +    (inputs ++ subExprs).distinct.foldLeft(1) { case (curLength, input) =>
    --- End diff --
    
    what about `map(calculateParamLength(ctx, _)).sum`?


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r155538158
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -55,8 +55,23 @@ import org.apache.spark.util.{ParentClassLoader, Utils}
      *                 to null.
      * @param value A term for a (possibly primitive) value of the result of the evaluation. Not
      *              valid if `isNull` is set to `true`.
    + * @param inputRow A term that holds the input row name when generating this code.
    + * @param inputVars A list of [[ExprInputVar]] that holds input variables when generating this code.
      */
    -case class ExprCode(var code: String, var isNull: String, var value: String)
    +case class ExprCode(
    +    var code: String,
    +    var isNull: String,
    +    var value: String,
    +    var inputRow: String = null,
    +    var inputVars: Seq[ExprInputVar] = Seq.empty)
    +
    +/**
    + * Represents an input variable [[ExprCode]] to an evaluation of an [[Expression]].
    + *
    + * @param expr The expression that is evaluated to the input variable.
    + * @param exprCode The [[ExprCode]] that represents the evaluation result for the input variable.
    + */
    +case class ExprInputVar(expr: Expression, exprCode: ExprCode)
    --- End diff --
    
    Yea, I also think about this before. Just thinking taking an expression is simpler. I will change it.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84367/
    Test PASSed.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84642/
    Test PASSed.


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84323/
    Test PASSed.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r155557147
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines util methods used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu 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 paramsFromSubExprs = prepareFunctionParams(ctx, subExprs, 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, subExprs) + 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] = {
    +    subExprs.map { subExpr =>
    +      val state = ctx.subExprEliminationExprs(subExpr)
    +      ExprCode(code = "", value = state.value, isNull = state.isNull)
    +    }
    +  }
    +
    +  /**
    +   * Retrieves previous input rows referred by children and deferred expressions.
    +   */
    +  def getInputRowsForChildren(ctx: CodegenContext, expr: Expression): Seq[String] = {
    +    expr.children.flatMap(getInputRows(ctx, _)).distinct
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previous input rows referred by it or deferred expressions
    +   * which are needed to evaluate it.
    +   */
    +  def getInputRows(ctx: CodegenContext, child: Expression): Seq[String] = {
    +    child.flatMap {
    +      // An expression directly evaluates on current input row.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars == null ||
    +          ctx.currentVars(ordinal) == null =>
    +        Seq(ctx.INPUT_ROW)
    +
    +      // An expression which is not evaluated yet. Tracks down to find input rows.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal).code != "" =>
    +        trackDownRow(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down input rows referred by the generated code snippet.
    +   */
    +  def trackDownRow(ctx: CodegenContext, exprCode: ExprCode): Seq[String] = {
    +    val exprCodes = mutable.Queue[ExprCode](exprCode)
    +    val inputRows = mutable.ArrayBuffer.empty[String]
    +
    +    while (exprCodes.nonEmpty) {
    +      val curExprCode = exprCodes.dequeue()
    +      if (curExprCode.inputRow != null) {
    +        inputRows += curExprCode.inputRow
    +      }
    +      curExprCode.inputVars.foreach { inputVar =>
    +        if (inputVar.exprCode.code != "") {
    +          exprCodes.enqueue(inputVar.exprCode)
    +        }
    +      }
    +    }
    +    inputRows
    +  }
    +
    +  /**
    +   * Retrieves previously evaluated columns referred by children and deferred expressions.
    +   * Returned tuple contains the list of expressions and the list of generated codes.
    +   */
    +  def getInputVarsForChildren(
    +      ctx: CodegenContext,
    +      expr: Expression): (Seq[Expression], Seq[ExprCode]) = {
    +    expr.children.flatMap(getInputVars(ctx, _)).distinct.unzip
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previously evaluated columns referred by it or
    +   * deferred expressions which are needed to evaluate it.
    +   */
    +  def getInputVars(ctx: CodegenContext, child: Expression): Seq[(Expression, ExprCode)] = {
    +    if (ctx.currentVars == null) {
    +      return Seq.empty
    +    }
    +
    +    child.flatMap {
    +      // An evaluated variable.
    +      case b @ BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null &&
    +          ctx.currentVars(ordinal).code == "" =>
    +        Seq((b, ctx.currentVars(ordinal)))
    +
    +      // An input variable which is not evaluated yet. Tracks down to find any evaluated variables
    +      // in the expression path.
    +      // E.g., if this expression is "d = c + 1" and "c" is not evaluated. We need to track to
    +      // "c = a + b" and see if "a" and "b" are evaluated. If they are, we need to return them so
    +      // to include them into parameters, if not, we track down further.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null =>
    +        trackDownVar(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down previously evaluated columns referred by the generated code snippet.
    +   */
    +  def trackDownVar(ctx: CodegenContext, exprCode: ExprCode): Seq[(Expression, ExprCode)] = {
    +    val exprCodes = mutable.Queue[ExprCode](exprCode)
    +    val inputVars = mutable.ArrayBuffer.empty[(Expression, ExprCode)]
    +
    +    while (exprCodes.nonEmpty) {
    +      exprCodes.dequeue().inputVars.foreach { inputVar =>
    +        if (inputVar.exprCode.code == "") {
    +          inputVars += ((inputVar.expr, inputVar.exprCode))
    +        } else {
    +          exprCodes.enqueue(inputVar.exprCode)
    +        }
    +      }
    +    }
    +    inputVars
    +  }
    +
    +  /**
    +   * Helper function to calculate the size of an expression as function parameter.
    +   */
    +  def calculateParamLength(ctx: CodegenContext, input: Expression): Int = {
    +    ctx.javaType(input.dataType) match {
    +      case (ctx.JAVA_LONG | ctx.JAVA_DOUBLE) if !input.nullable => 2
    +      case ctx.JAVA_LONG | ctx.JAVA_DOUBLE => 3
    +      case _ if !input.nullable => 1
    +      case _ => 2
    +    }
    +  }
    +
    +  /**
    +   * In Java, a method descriptor is valid only if it represents method parameters with a total
    +   * length of 255 or less. `this` contributes one unit and a parameter of type long or double
    +   * contributes two units.
    +   */
    +  def getParamLength(
    +      ctx: CodegenContext,
    +      inputs: Seq[Expression],
    +      subExprs: Seq[Expression]): Int = {
    +    // Initial value is 1 for `this`.
    +    1 + (inputs ++ subExprs).distinct.map(calculateParamLength(ctx, _)).sum
    +  }
    +
    +  /**
    +   * Given the lists of input attributes and variables to this expression, returns the strings of
    +   * funtion parameters. The first is the variable names used to call the function, the second is
    +   * the parameters used to declare the function in generated code.
    +   */
    +  def prepareFunctionParams(
    +      ctx: CodegenContext,
    +      inputAttrs: Seq[Expression],
    +      inputVars: Seq[ExprCode]): Seq[(String, String)] = {
    +    inputAttrs.zip(inputVars).flatMap { case (input, ev) =>
    +      val argType = ctx.javaType(input.dataType)
    +
    +      if (!input.nullable || ev.isNull == "true" || ev.isNull == "false") {
    +        Seq((ev.value, s"$argType ${ev.value}"))
    +      } else {
    +        Seq((ev.value, s"$argType ${ev.value}"), (ev.isNull, s"boolean ${ev.isNull}"))
    --- End diff --
    
    Hmm, I think if an output is a literal, will we refer it as an attribute? Won't we replace all attribute reference for it with actual literal?


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by maropu <gi...@git.apache.org>.
Github user maropu commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Probably, we'd better to move this discussion to the jira?


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156285447
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -55,8 +55,45 @@ import org.apache.spark.util.{ParentClassLoader, Utils}
      *                 to null.
      * @param value A term for a (possibly primitive) value of the result of the evaluation. Not
      *              valid if `isNull` is set to `true`.
    + * @param inputRow A term that holds the input row name when generating this code.
    + * @param inputVars A list of [[ExprInputVar]] that holds input variables when generating this code.
      */
    -case class ExprCode(var code: String, var isNull: String, var value: String)
    +case class ExprCode(
    +    var code: String,
    +    var isNull: String,
    +    var value: String,
    +    var inputRow: String = null,
    +    var inputVars: Seq[ExprInputVar] = Seq.empty) {
    +
    +  // Returns true if this value is a literal.
    +  def isLiteral(): Boolean = {
    +    assert(value.nonEmpty, "ExprCode.value can't be empty string.")
    +
    +    if (value == "true" || value == "false" || value == "null") {
    +      true
    +    } else {
    +      // The valid characters for the first character of a Java variable is [a-zA-Z_$].
    +      value.head match {
    +        case v if v >= 'a' && v <= 'z' => false
    --- End diff --
    
    This won't work for array accessing like `arr[1]`, I think we should pick a more general solution like https://github.com/apache/spark/pull/19938#issuecomment-350935431 , i.e. generate parameter names instead of reusing the input name.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156067549
  
    --- 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 --
    
    seems we only need it for sub-expr of whole stage codegen?


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84239 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84239/testReport)** for PR 19813 at commit [`d051f9e`](https://github.com/apache/spark/commit/d051f9eef4d03f9027571419857f690c866dbd98).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156077696
  
    --- 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 --
    
    Is it? Don't we also do sub-expr elimination in normal codegen?


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156286336
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,245 @@
    +/*
    + * 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 {
    +
    +  /**
    +   * 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 subExprs = getSubExprInChildren(ctx, expr)
    +    val subExprCodes = getSubExprCodes(ctx, subExprs)
    +    val subVars = subExprs.zip(subExprCodes).map { case (subExpr, subExprCode) =>
    +      ExprInputVar(subExprCode, subExpr.dataType, subExpr.nullable)
    +    }
    +    val paramsFromSubExprs = prepareFunctionParams(ctx, subVars)
    +
    +    val inputVars = getInputVarsForChildren(ctx, expr)
    +    val paramsFromColumns = prepareFunctionParams(ctx, inputVars)
    +
    +    val inputRows = ctx.INPUT_ROW +: getInputRowsForChildren(ctx, expr)
    +    val paramsFromRows = inputRows.distinct.filter(_ != null).map { row =>
    +      (row, s"InternalRow $row")
    +    }
    +
    +    val paramsLength = getParamLength(ctx, inputVars ++ subVars) + 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] = {
    +    subExprs.map { subExpr =>
    +      val state = ctx.subExprEliminationExprs(subExpr)
    +      ExprCode(code = "", value = state.value, isNull = state.isNull)
    +    }
    +  }
    +
    +  /**
    +   * Retrieves previous input rows referred by children and deferred expressions.
    +   */
    +  def getInputRowsForChildren(ctx: CodegenContext, expr: Expression): Seq[String] = {
    +    expr.children.flatMap(getInputRows(ctx, _)).distinct
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previous input rows referred by it or deferred expressions
    +   * which are needed to evaluate it.
    +   */
    +  def getInputRows(ctx: CodegenContext, child: Expression): Seq[String] = {
    +    child.flatMap {
    +      // An expression directly evaluates on current input row.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars == null ||
    +          ctx.currentVars(ordinal) == null =>
    +        Seq(ctx.INPUT_ROW)
    +
    +      // An expression which is not evaluated yet. Tracks down to find input rows.
    +      case BoundReference(ordinal, _, _) if !ctx.currentVars(ordinal).isEvaluated() =>
    +        trackDownRow(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down input rows referred by the generated code snippet.
    +   */
    +  def trackDownRow(ctx: CodegenContext, exprCode: ExprCode): Seq[String] = {
    +    val exprCodes = mutable.Queue[ExprCode](exprCode)
    +    val inputRows = mutable.ArrayBuffer.empty[String]
    +
    +    while (exprCodes.nonEmpty) {
    +      val curExprCode = exprCodes.dequeue()
    +      if (curExprCode.inputRow != null) {
    +        inputRows += curExprCode.inputRow
    +      }
    +      curExprCode.inputVars.foreach { inputVar =>
    +        if (!inputVar.exprCode.isEvaluated()) {
    +          exprCodes.enqueue(inputVar.exprCode)
    +        }
    +      }
    +    }
    +    inputRows
    +  }
    +
    +  /**
    +   * Retrieves previously evaluated columns referred by children and deferred expressions.
    +   * Returned tuple contains the list of expressions and the list of generated codes.
    +   */
    +  def getInputVarsForChildren(
    +      ctx: CodegenContext,
    +      expr: Expression): Seq[ExprInputVar] = {
    +    expr.children.flatMap(getInputVars(ctx, _)).distinct
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previously evaluated columns referred by it or
    +   * deferred expressions which are needed to evaluate it.
    +   */
    +  def getInputVars(ctx: CodegenContext, child: Expression): Seq[ExprInputVar] = {
    +    if (ctx.currentVars == null) {
    +      return Seq.empty
    +    }
    +
    +    child.flatMap {
    +      // An evaluated variable.
    +      case b @ BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null &&
    +          ctx.currentVars(ordinal).isEvaluated() =>
    +        Seq(ExprInputVar(ctx.currentVars(ordinal), b.dataType, b.nullable))
    +
    +      // An input variable which is not evaluated yet. Tracks down to find any evaluated variables
    +      // in the expression path.
    +      // E.g., if this expression is "d = c + 1" and "c" is not evaluated. We need to track to
    +      // "c = a + b" and see if "a" and "b" are evaluated. If they are, we need to return them so
    +      // to include them into parameters, if not, we track down further.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null =>
    +        trackDownVar(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down previously evaluated columns referred by the generated code snippet.
    +   */
    +  def trackDownVar(ctx: CodegenContext, exprCode: ExprCode): Seq[ExprInputVar] = {
    +    val exprCodes = mutable.Queue[ExprCode](exprCode)
    +    val inputVars = mutable.ArrayBuffer.empty[ExprInputVar]
    +
    +    while (exprCodes.nonEmpty) {
    +      exprCodes.dequeue().inputVars.foreach { inputVar =>
    +        if (inputVar.exprCode.isEvaluated()) {
    +          inputVars += inputVar
    +        } else {
    +          exprCodes.enqueue(inputVar.exprCode)
    +        }
    +      }
    +    }
    +    inputVars
    +  }
    +
    +  /**
    +   * Helper function to calculate the size of an expression as function parameter.
    +   */
    +  def calculateParamLength(ctx: CodegenContext, input: ExprInputVar): Int = {
    +    ctx.javaType(input.dataType) match {
    --- End diff --
    
    nit
    ```
    ctx.javaType(input.dataType) match {
      case ctx.JAVA_LONG | ctx.JAVA_DOUBLE => 2
      case _ => 1
    } + (if (input.isNullable) 1 else 0)
    ```


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84327/
    Test PASSed.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    I agree that string replacement is too dangerous (e.g. `a + 1 = a + 10` with `a + 1`).  
    How about a contract with adding assertions?


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156287640
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -55,8 +55,45 @@ import org.apache.spark.util.{ParentClassLoader, Utils}
      *                 to null.
      * @param value A term for a (possibly primitive) value of the result of the evaluation. Not
      *              valid if `isNull` is set to `true`.
    + * @param inputRow A term that holds the input row name when generating this code.
    + * @param inputVars A list of [[ExprInputVar]] that holds input variables when generating this code.
      */
    -case class ExprCode(var code: String, var isNull: String, var value: String)
    +case class ExprCode(
    +    var code: String,
    +    var isNull: String,
    +    var value: String,
    +    var inputRow: String = null,
    +    var inputVars: Seq[ExprInputVar] = Seq.empty) {
    +
    +  // Returns true if this value is a literal.
    +  def isLiteral(): Boolean = {
    +    assert(value.nonEmpty, "ExprCode.value can't be empty string.")
    +
    +    if (value == "true" || value == "false" || value == "null") {
    +      true
    +    } else {
    +      // The valid characters for the first character of a Java variable is [a-zA-Z_$].
    +      value.head match {
    +        case v if v >= 'a' && v <= 'z' => false
    --- End diff --
    
    hmmm this seems very hard to do, the code is already generated and use the input names as whatever it is, e.g. java variable `a` or literal `123` or array accessing `arr[1]`. Ideally we need to analyze what the input really refers, e.g. `a` refers to a java variable `a`, `123` refers nothing, `arr[1]` refers to a java variable `arr`. This is kind of impossible for the current string based framework. We need to think more about how to deal with it.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156286913
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala ---
    @@ -236,4 +237,24 @@ class WholeStageCodegenSuite extends QueryTest with SharedSQLContext {
           }
         }
       }
    +
    +  test("SPARK-22551: Fix 64kb limit for deeply nested expressions under wholestage codegen") {
    +    import testImplicits._
    +    withTempPath { dir =>
    +      val path = dir.getCanonicalPath
    +      val df = Seq(("abc", 1)).toDF("key", "int")
    +      df.write.parquet(path)
    +
    +      var strExpr: Expression = col("key").expr
    +      for (_ <- 1 to 150) {
    +        strExpr = Decode(Encode(strExpr, Literal("utf-8")), Literal("utf-8"))
    +      }
    +      val expressions = Seq(If(EqualTo(strExpr, strExpr), strExpr, strExpr))
    +
    +      val df2 = spark.read.parquet(path).select(expressions.map(Column(_)): _*)
    +      val plan = df2.queryExecution.executedPlan
    +      assert(plan.find(_.isInstanceOf[WholeStageCodegenExec]).isDefined)
    --- End diff --
    
    can you give some insights about how this test fail without your PR? In `WholeStageCodegen.doExec`, we have
    ```
      val (ctx, cleanedSource) = doCodeGen()
        // try to compile and fallback if it failed
        val (_, maxCodeSize) = try {
          CodeGenerator.compile(cleanedSource)
        } catch {
          case _: Exception if !Utils.isTesting && sqlContext.conf.codegenFallback =>
            // We should already saw the error message
            logWarning(s"Whole-stage codegen disabled for this plan:\n $treeString")
            return child.execute()
        }
    ```


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    I found a few problems that this PR can ideally solve. If this is not available soon, I will use workaround in upcoming PRs.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Note: One possible problem is in `a + 1`, `a` is not an input variable from an operator.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r155126007
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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.
    --- End diff --
    
    Currently `ProjectExec` only pre-evaluates the input attributes which are referred more than twice. Otherwise, we defer the evaluation until input attribute is actually used.



---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154949925
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -55,8 +55,20 @@ import org.apache.spark.util.{ParentClassLoader, Utils}
      *                 to null.
      * @param value A term for a (possibly primitive) value of the result of the evaluation. Not
      *              valid if `isNull` is set to `true`.
    + * @param inputRow A term that holds the input row name when generating this code.
    + * @param inputVars A list of [[ExprInputVar]] that holds input variables when generating this code.
      */
    -case class ExprCode(var code: String, var isNull: String, var value: String)
    +case class ExprCode(
    +    var code: String,
    +    var isNull: String,
    +    var value: String,
    +    var inputRow: String = null,
    +    var inputVars: Seq[ExprInputVar] = Seq.empty)
    +
    +/**
    + * Represents an input variable [[ExprCode]] to an evaluation of an [[Expression]].
    --- End diff --
    
    please add parameter doc


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Do we have to sacrifice one of them? If we do then I agree deeply nested expression is more common than a long chain of arithmetic expressions and we should get this patch. I think we should explore more about how to split methods in whole stage codegen before making this decision, at least now I'm not convinced that we have to forbid expressions to output statement.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156088645
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala ---
    @@ -105,6 +105,11 @@ abstract class Expression extends TreeNode[Expression] {
           val isNull = ctx.freshName("isNull")
           val value = ctx.freshName("value")
           val eval = doGenCode(ctx, ExprCode("", isNull, value))
    +
    +      // Records current input row and variables of this expression.
    +      eval.inputRow = ctx.INPUT_ROW
    +      eval.inputVars = findInputVars(ctx, eval)
    --- End diff --
    
    `eval.isNull = if (this.nullable) eval.isNull else "false"` seems ok.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r155698248
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines util methods used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu 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 paramsFromSubExprs = prepareFunctionParams(ctx, subExprs, 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, subExprs) + paramsFromRows.length
    +    // Maximum allowed parameter number for Java's method descriptor.
    +    if (paramsLength > 255) {
    --- End diff --
    
    Since we split functions out if the code length is large enough, I think we won't possibly hit this limit. This is more like a safety guard.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84158/
    Test FAILed.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84331 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84331/testReport)** for PR 19813 at commit [`57a9fb7`](https://github.com/apache/spark/commit/57a9fb77d7628e8a5815b8571ca9c99490419252).


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by mgaido91 <gi...@git.apache.org>.
Github user mgaido91 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154186642
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -55,8 +55,20 @@ import org.apache.spark.util.{ParentClassLoader, Utils}
      *                 to null.
      * @param value A term for a (possibly primitive) value of the result of the evaluation. Not
      *              valid if `isNull` is set to `true`.
    + * @param inputRow A term that holds the input row name when generating this code.
    + * @param inputVars A list of [[ExprInputVar]] that holds input variables when generating this code.
      */
    -case class ExprCode(var code: String, var isNull: String, var value: String)
    +case class ExprCode(
    +    var code: String,
    +    var isNull: String,
    +    var value: String,
    +    var inputRow: String = null,
    +    val inputVars: mutable.ArrayBuffer[ExprInputVar] = mutable.ArrayBuffer.empty)
    +
    +/**
    + * Represents an input variable that holds the java type and the [[ExprCode]].
    --- End diff --
    
    I think this is outdated


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84760/
    Test PASSed.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154950384
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    --- End diff --
    
    Defines util methods


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    IMHO, in general, the output `ev.value` would be declared as local variable by parent as
    ```
    s"""${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)};
    ```
    
    Such as cases cannot have an expression in `ev.value`.  
    As @viirya pointed out, I imagine there are a few scenarios. Would it be possible to show an example and place in source code where an expression is used as output in order to correctly understand the issue?


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84486 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84486/testReport)** for PR 19813 at commit [`48add65`](https://github.com/apache/spark/commit/48add652f2df45ce6506f9464c10a6425bd92214).


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154983972
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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.
    --- End diff --
    
    I just did a quick test. Some tests like `org.apache.spark.ml.tuning.TrainValidationSplitSuite` and `org.apache.spark.ml.classification.GBTClassifierSuite` will fail.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154955437
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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.
    --- End diff --
    
    You mean deferred expressions? Yes, we can't pass tests if we don't consider it. Some variables can be deferred to evaluate in the operators. 


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84211/
    Test FAILed.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    To me whole stage codegen compilation fix is less important as we can fallback to non whole stage codegen, so we don't need to rush.
    
    > As we don't use such statement as codegen output, I think the easiest approach is adding assertions
    
    What about future? Will we need to output statement for some reason? like reducing the usage of local variables?


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154575588
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala ---
    @@ -108,7 +108,10 @@ private[sql] trait ColumnarBatchScan extends CodegenSupport {
              |}""".stripMargin)
     
         ctx.currentVars = null
    +    // `rowIdx` isn't in `ctx.currentVars`. If the expressions are split later, we can't track it.
    +    // So making it as global variable.
    --- End diff --
    
    can we add `rowId` to `ExprCode.inputVars` in `genCodeColumnVector`?


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84158 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84158/testReport)** for PR 19813 at commit [`1cf6a48`](https://github.com/apache/spark/commit/1cf6a482ec378964af26dc576a2b819c230558d2).


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84486/
    Test PASSed.


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by mgaido91 <gi...@git.apache.org>.
Github user mgaido91 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154200309
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,259 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu children expressions.
    +   */
    +  def getExpressionInputParams(
    +      ctx: CodegenContext,
    +      expr: Expression): Option[(Seq[String], Seq[String])] = {
    +    val (inputAttrs, inputVars) = getInputVarsForChildren(ctx, expr)
    +    val inputRows = ctx.INPUT_ROW +: getInputRowsForChildren(ctx, expr)
    +    val subExprs = getSubExprInChildren(ctx, expr)
    +
    +    val paramsFromRows = inputRows.distinct.filter(_ != null).map { row =>
    +      (row, s"InternalRow $row")
    +    }
    +    val paramsFromColumns = prepareFunctionParams(ctx, inputAttrs, inputVars)
    +    val paramsFromSubExprs = getParamsForSubExprs(ctx, subExprs)
    +    val paramsLength = getParamLength(ctx, inputAttrs, subExprs) + 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
    +  }
    +
    +  /**
    +   * Given the list of eliminated subexpressions used in the children expressions, returns the
    +   * strings of funtion parameters. The first is the variable names used to call the function,
    +   * the second is the parameters used to declare the function in generated code.
    +   */
    +  def getParamsForSubExprs(
    +      ctx: CodegenContext,
    +      subExprs: Seq[Expression]): Seq[(String, String)] = {
    +    subExprs.flatMap { subExpr =>
    +      val argType = ctx.javaType(subExpr.dataType)
    +
    +      val subExprState = ctx.subExprEliminationExprs(subExpr)
    +      (subExprState.value, subExprState.isNull)
    +
    +      if (!subExpr.nullable || subExprState.isNull == "true" || subExprState.isNull == "false") {
    +        Seq((subExprState.value, s"$argType ${subExprState.value}"))
    +      } else {
    +        Seq((subExprState.value, s"$argType ${subExprState.value}"),
    +          (subExprState.isNull, s"boolean ${subExprState.isNull}"))
    +      }
    +    }.distinct
    +  }
    +
    +  /**
    +   * Retrieves previous input rows referred by children and deferred expressions.
    +   */
    +  def getInputRowsForChildren(ctx: CodegenContext, expr: Expression): Seq[String] = {
    +    expr.children.flatMap(getInputRows(ctx, _)).distinct
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previous input rows referred by it or deferred expressions
    +   * which are needed to evaluate it.
    +   */
    +  def getInputRows(ctx: CodegenContext, child: Expression): Seq[String] = {
    +    child.flatMap {
    +      // An expression directly evaluates on current input row.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars == null ||
    +          ctx.currentVars(ordinal) == null =>
    +        Seq(ctx.INPUT_ROW)
    +
    +      // An expression which is not evaluated yet. Tracks down to find input rows.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal).code != "" =>
    +        trackDownRow(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down input rows referred by the generated code snippet.
    +   */
    +  def trackDownRow(ctx: CodegenContext, exprCode: ExprCode): Seq[String] = {
    +    var exprCodes: List[ExprCode] = List(exprCode)
    +    val inputRows = mutable.ArrayBuffer.empty[String]
    +
    +    while (exprCodes.nonEmpty) {
    +      exprCodes match {
    +        case first :: others =>
    +          exprCodes = others
    +          if (first.inputRow != null) {
    +            inputRows += first.inputRow
    +          }
    +          first.inputVars.foreach { inputVar =>
    +            if (inputVar.exprCode.code != "") {
    +              exprCodes = inputVar.exprCode :: exprCodes
    +            }
    +          }
    +        case _ =>
    +      }
    +    }
    +    inputRows.toSeq
    +  }
    +
    +  /**
    +   * Retrieves previously evaluated columns referred by children and deferred expressions.
    +   * Returned tuple contains the list of expressions and the list of generated codes.
    +   */
    +  def getInputVarsForChildren(
    +      ctx: CodegenContext,
    +      expr: Expression): (Seq[Expression], Seq[ExprCode]) = {
    +    expr.children.flatMap(getInputVars(ctx, _)).distinct.unzip
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previously evaluated columns referred by it or
    +   * deferred expressions which are needed to evaluate it.
    +   */
    +  def getInputVars(ctx: CodegenContext, child: Expression): Seq[(Expression, ExprCode)] = {
    +    if (ctx.currentVars == null) {
    +      return Seq.empty
    +    }
    +
    +    child.flatMap {
    +      // An evaluated variable.
    +      case b @ BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null &&
    +          ctx.currentVars(ordinal).code == "" =>
    +        Seq((b, ctx.currentVars(ordinal)))
    +
    +      // An input variable which is not evaluated yet. Tracks down to find any evaluated variables
    +      // in the expression path.
    +      // E.g., if this expression is "d = c + 1" and "c" is not evaluated. We need to track to
    +      // "c = a + b" and see if "a" and "b" are evaluated. If they are, we need to return them so
    +      // to include them into parameters, if not, we track down further.
    +      case b @ BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null =>
    +        trackDownVar(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down previously evaluated columns referred by the generated code snippet.
    +   */
    +  def trackDownVar(ctx: CodegenContext, exprCode: ExprCode): Seq[(Expression, ExprCode)] = {
    +    var exprCodes: List[ExprCode] = List(exprCode)
    +    val inputVars = mutable.ArrayBuffer.empty[(Expression, ExprCode)]
    +
    +    while (exprCodes.nonEmpty) {
    +      exprCodes match {
    +        case first :: others =>
    +          exprCodes = others
    +          first.inputVars.foreach { inputVar =>
    +            if (inputVar.exprCode.code == "") {
    +              inputVars += ((inputVar.expr, inputVar.exprCode))
    +            } else {
    +              exprCodes = inputVar.exprCode :: exprCodes
    +            }
    +          }
    +        case _ =>
    +      }
    +    }
    +    inputVars.toSeq
    +  }
    +
    +  /**
    +   * Helper function to calculate the size of an expression as function parameter.
    +   */
    +  def calculateParamLength(ctx: CodegenContext, input: Expression): Int = {
    +    ctx.javaType(input.dataType) match {
    +      case (ctx.JAVA_LONG | ctx.JAVA_DOUBLE) if !input.nullable => 2
    --- End diff --
    
    sorry for the question, I am just curious, why do longs and doubles require an additional param?


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84327 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84327/testReport)** for PR 19813 at commit [`7230997`](https://github.com/apache/spark/commit/7230997a54babaf62846ab538bb6756b3938d832).


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156056362
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala ---
    @@ -105,6 +105,11 @@ abstract class Expression extends TreeNode[Expression] {
           val isNull = ctx.freshName("isNull")
           val value = ctx.freshName("value")
           val eval = doGenCode(ctx, ExprCode("", isNull, value))
    +
    +      // Records current input row and variables of this expression.
    +      eval.inputRow = ctx.INPUT_ROW
    +      eval.inputVars = findInputVars(ctx, eval)
    --- End diff --
    
    shall we do one more thing here?
    ```
    eval.isNull = if (this.nullable) eval.isNull else "false"
    ```


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156297739
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -55,8 +55,45 @@ import org.apache.spark.util.{ParentClassLoader, Utils}
      *                 to null.
      * @param value A term for a (possibly primitive) value of the result of the evaluation. Not
      *              valid if `isNull` is set to `true`.
    + * @param inputRow A term that holds the input row name when generating this code.
    + * @param inputVars A list of [[ExprInputVar]] that holds input variables when generating this code.
      */
    -case class ExprCode(var code: String, var isNull: String, var value: String)
    +case class ExprCode(
    +    var code: String,
    +    var isNull: String,
    +    var value: String,
    +    var inputRow: String = null,
    +    var inputVars: Seq[ExprInputVar] = Seq.empty) {
    +
    +  // Returns true if this value is a literal.
    +  def isLiteral(): Boolean = {
    +    assert(value.nonEmpty, "ExprCode.value can't be empty string.")
    +
    +    if (value == "true" || value == "false" || value == "null") {
    +      true
    +    } else {
    +      // The valid characters for the first character of a Java variable is [a-zA-Z_$].
    +      value.head match {
    +        case v if v >= 'a' && v <= 'z' => false
    --- End diff --
    
    no problem at all.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154266575
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,248 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu 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 paramsFromSubExprs = getParamsForSubExprs(ctx, subExprs)
    +
    +    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, subExprs) + 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
    +  }
    +
    +  /**
    +   * Given the list of eliminated subexpressions used in the children expressions, returns the
    +   * strings of funtion parameters. The first is the variable names used to call the function,
    +   * the second is the parameters used to declare the function in generated code.
    +   */
    +  def getParamsForSubExprs(
    +      ctx: CodegenContext,
    +      subExprs: Seq[Expression]): Seq[(String, String)] = {
    +    subExprs.flatMap { subExpr =>
    +      val argType = ctx.javaType(subExpr.dataType)
    +
    +      val subExprState = ctx.subExprEliminationExprs(subExpr)
    +
    +      if (!subExpr.nullable || subExprState.isNull == "true" || subExprState.isNull == "false") {
    +        Seq((subExprState.value, s"$argType ${subExprState.value}"))
    +      } else {
    +        Seq((subExprState.value, s"$argType ${subExprState.value}"),
    +          (subExprState.isNull, s"boolean ${subExprState.isNull}"))
    +      }
    +    }.distinct
    +  }
    +
    +  /**
    +   * Retrieves previous input rows referred by children and deferred expressions.
    +   */
    +  def getInputRowsForChildren(ctx: CodegenContext, expr: Expression): Seq[String] = {
    +    expr.children.flatMap(getInputRows(ctx, _)).distinct
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previous input rows referred by it or deferred expressions
    +   * which are needed to evaluate it.
    +   */
    +  def getInputRows(ctx: CodegenContext, child: Expression): Seq[String] = {
    +    child.flatMap {
    +      // An expression directly evaluates on current input row.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars == null ||
    +          ctx.currentVars(ordinal) == null =>
    +        Seq(ctx.INPUT_ROW)
    +
    +      // An expression which is not evaluated yet. Tracks down to find input rows.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal).code != "" =>
    +        trackDownRow(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down input rows referred by the generated code snippet.
    +   */
    +  def trackDownRow(ctx: CodegenContext, exprCode: ExprCode): Seq[String] = {
    +    val exprCodes = mutable.Queue[ExprCode](exprCode)
    +    val inputRows = mutable.ArrayBuffer.empty[String]
    +
    +    while (exprCodes.nonEmpty) {
    +      val curExprCode = exprCodes.dequeue()
    +      if (curExprCode.inputRow != null) {
    +        inputRows += curExprCode.inputRow
    +      }
    +      curExprCode.inputVars.foreach { inputVar =>
    +        if (inputVar.exprCode.code != "") {
    +          exprCodes.enqueue(inputVar.exprCode)
    +        }
    +      }
    +    }
    +    inputRows
    +  }
    +
    +  /**
    +   * Retrieves previously evaluated columns referred by children and deferred expressions.
    +   * Returned tuple contains the list of expressions and the list of generated codes.
    +   */
    +  def getInputVarsForChildren(
    +      ctx: CodegenContext,
    +      expr: Expression): (Seq[Expression], Seq[ExprCode]) = {
    +    expr.children.flatMap(getInputVars(ctx, _)).distinct.unzip
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previously evaluated columns referred by it or
    +   * deferred expressions which are needed to evaluate it.
    +   */
    +  def getInputVars(ctx: CodegenContext, child: Expression): Seq[(Expression, ExprCode)] = {
    +    if (ctx.currentVars == null) {
    +      return Seq.empty
    +    }
    +
    +    child.flatMap {
    +      // An evaluated variable.
    +      case b @ BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null &&
    +          ctx.currentVars(ordinal).code == "" =>
    +        Seq((b, ctx.currentVars(ordinal)))
    +
    +      // An input variable which is not evaluated yet. Tracks down to find any evaluated variables
    +      // in the expression path.
    +      // E.g., if this expression is "d = c + 1" and "c" is not evaluated. We need to track to
    +      // "c = a + b" and see if "a" and "b" are evaluated. If they are, we need to return them so
    +      // to include them into parameters, if not, we track down further.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null =>
    +        trackDownVar(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down previously evaluated columns referred by the generated code snippet.
    +   */
    +  def trackDownVar(ctx: CodegenContext, exprCode: ExprCode): Seq[(Expression, ExprCode)] = {
    +    val exprCodes = mutable.Queue[ExprCode](exprCode)
    +    val inputVars = mutable.ArrayBuffer.empty[(Expression, ExprCode)]
    +
    +    while (exprCodes.nonEmpty) {
    +      exprCodes.dequeue().inputVars.foreach { inputVar =>
    +        if (inputVar.exprCode.code == "") {
    +          inputVars += ((inputVar.expr, inputVar.exprCode))
    +        } else {
    +          exprCodes.enqueue(inputVar.exprCode)
    +        }
    +      }
    +    }
    +    inputVars
    +  }
    +
    +  /**
    +   * Helper function to calculate the size of an expression as function parameter.
    +   */
    +  def calculateParamLength(ctx: CodegenContext, input: Expression): Int = {
    +    ctx.javaType(input.dataType) match {
    +      case (ctx.JAVA_LONG | ctx.JAVA_DOUBLE) if !input.nullable => 2
    +      case ctx.JAVA_LONG | ctx.JAVA_DOUBLE => 3
    +      case _ if !input.nullable => 1
    +      case _ => 2
    +    }
    +  }
    +
    +  /**
    +   * In Java, a method descriptor is valid only if it represents method parameters with a total
    +   * length of 255 or less. `this` contributes one unit and a parameter of type long or double
    +   * contributes two units.
    +   */
    +  def getParamLength(
    +      ctx: CodegenContext,
    +      inputs: Seq[Expression],
    +      subExprs: Seq[Expression]): Int = {
    +    // Initial value is 1 for `this`.
    +    1 + (inputs ++ subExprs).distinct.map(calculateParamLength(ctx, _)).sum
    +  }
    +
    +  /**
    +   * Given the lists of input attributes and variables to this expression, returns the strings of
    +   * funtion parameters. The first is the variable names used to call the function, the second is
    +   * the parameters used to declare the function in generated code.
    +   */
    +  def prepareFunctionParams(
    --- End diff --
    
    How about avoid duplication of similar logic in `getParamsForSubExprs`?  
    In `getParamsForSubExprs`, I think that `subExprs` corresponds to `inputAttrs` and `inputVars` corresponds to `subExprs.map(ctx.subExprEliminationExprs(_))`.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154575849
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu 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 paramsFromSubExprs = prepareFunctionParams(ctx, subExprs, 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, subExprs) + 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] = {
    +    subExprs.map { subExpr =>
    +      val stat = ctx.subExprEliminationExprs(subExpr)
    +      ExprCode(code = "", value = stat.value, isNull = stat.isNull)
    +    }
    +  }
    +
    +  /**
    +   * Retrieves previous input rows referred by children and deferred expressions.
    +   */
    +  def getInputRowsForChildren(ctx: CodegenContext, expr: Expression): Seq[String] = {
    +    expr.children.flatMap(getInputRows(ctx, _)).distinct
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previous input rows referred by it or deferred expressions
    +   * which are needed to evaluate it.
    +   */
    +  def getInputRows(ctx: CodegenContext, child: Expression): Seq[String] = {
    +    child.flatMap {
    +      // An expression directly evaluates on current input row.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars == null ||
    +          ctx.currentVars(ordinal) == null =>
    +        Seq(ctx.INPUT_ROW)
    +
    +      // An expression which is not evaluated yet. Tracks down to find input rows.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal).code != "" =>
    +        trackDownRow(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down input rows referred by the generated code snippet.
    +   */
    +  def trackDownRow(ctx: CodegenContext, exprCode: ExprCode): Seq[String] = {
    +    val exprCodes = mutable.Queue[ExprCode](exprCode)
    +    val inputRows = mutable.ArrayBuffer.empty[String]
    +
    +    while (exprCodes.nonEmpty) {
    +      val curExprCode = exprCodes.dequeue()
    +      if (curExprCode.inputRow != null) {
    +        inputRows += curExprCode.inputRow
    +      }
    +      curExprCode.inputVars.foreach { inputVar =>
    +        if (inputVar.exprCode.code != "") {
    +          exprCodes.enqueue(inputVar.exprCode)
    +        }
    +      }
    +    }
    +    inputRows
    +  }
    +
    +  /**
    +   * Retrieves previously evaluated columns referred by children and deferred expressions.
    --- End diff --
    
    how to define `previously evaluated columns`?


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r155536112
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines util methods used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu 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 paramsFromSubExprs = prepareFunctionParams(ctx, subExprs, 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, subExprs) + 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] = {
    +    subExprs.map { subExpr =>
    +      val state = ctx.subExprEliminationExprs(subExpr)
    +      ExprCode(code = "", value = state.value, isNull = state.isNull)
    +    }
    +  }
    +
    +  /**
    +   * Retrieves previous input rows referred by children and deferred expressions.
    +   */
    +  def getInputRowsForChildren(ctx: CodegenContext, expr: Expression): Seq[String] = {
    +    expr.children.flatMap(getInputRows(ctx, _)).distinct
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previous input rows referred by it or deferred expressions
    +   * which are needed to evaluate it.
    +   */
    +  def getInputRows(ctx: CodegenContext, child: Expression): Seq[String] = {
    +    child.flatMap {
    +      // An expression directly evaluates on current input row.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars == null ||
    +          ctx.currentVars(ordinal) == null =>
    +        Seq(ctx.INPUT_ROW)
    +
    +      // An expression which is not evaluated yet. Tracks down to find input rows.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal).code != "" =>
    +        trackDownRow(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down input rows referred by the generated code snippet.
    +   */
    +  def trackDownRow(ctx: CodegenContext, exprCode: ExprCode): Seq[String] = {
    +    val exprCodes = mutable.Queue[ExprCode](exprCode)
    +    val inputRows = mutable.ArrayBuffer.empty[String]
    +
    +    while (exprCodes.nonEmpty) {
    +      val curExprCode = exprCodes.dequeue()
    +      if (curExprCode.inputRow != null) {
    +        inputRows += curExprCode.inputRow
    +      }
    +      curExprCode.inputVars.foreach { inputVar =>
    +        if (inputVar.exprCode.code != "") {
    +          exprCodes.enqueue(inputVar.exprCode)
    +        }
    +      }
    +    }
    +    inputRows
    +  }
    +
    +  /**
    +   * Retrieves previously evaluated columns referred by children and deferred expressions.
    +   * Returned tuple contains the list of expressions and the list of generated codes.
    +   */
    +  def getInputVarsForChildren(
    +      ctx: CodegenContext,
    +      expr: Expression): (Seq[Expression], Seq[ExprCode]) = {
    +    expr.children.flatMap(getInputVars(ctx, _)).distinct.unzip
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previously evaluated columns referred by it or
    +   * deferred expressions which are needed to evaluate it.
    +   */
    +  def getInputVars(ctx: CodegenContext, child: Expression): Seq[(Expression, ExprCode)] = {
    +    if (ctx.currentVars == null) {
    +      return Seq.empty
    +    }
    +
    +    child.flatMap {
    +      // An evaluated variable.
    +      case b @ BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null &&
    +          ctx.currentVars(ordinal).code == "" =>
    +        Seq((b, ctx.currentVars(ordinal)))
    +
    +      // An input variable which is not evaluated yet. Tracks down to find any evaluated variables
    +      // in the expression path.
    +      // E.g., if this expression is "d = c + 1" and "c" is not evaluated. We need to track to
    +      // "c = a + b" and see if "a" and "b" are evaluated. If they are, we need to return them so
    +      // to include them into parameters, if not, we track down further.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal) != null =>
    +        trackDownVar(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down previously evaluated columns referred by the generated code snippet.
    +   */
    +  def trackDownVar(ctx: CodegenContext, exprCode: ExprCode): Seq[(Expression, ExprCode)] = {
    +    val exprCodes = mutable.Queue[ExprCode](exprCode)
    +    val inputVars = mutable.ArrayBuffer.empty[(Expression, ExprCode)]
    +
    +    while (exprCodes.nonEmpty) {
    +      exprCodes.dequeue().inputVars.foreach { inputVar =>
    +        if (inputVar.exprCode.code == "") {
    +          inputVars += ((inputVar.expr, inputVar.exprCode))
    +        } else {
    +          exprCodes.enqueue(inputVar.exprCode)
    +        }
    +      }
    +    }
    +    inputVars
    +  }
    +
    +  /**
    +   * Helper function to calculate the size of an expression as function parameter.
    +   */
    +  def calculateParamLength(ctx: CodegenContext, input: Expression): Int = {
    +    ctx.javaType(input.dataType) match {
    +      case (ctx.JAVA_LONG | ctx.JAVA_DOUBLE) if !input.nullable => 2
    +      case ctx.JAVA_LONG | ctx.JAVA_DOUBLE => 3
    +      case _ if !input.nullable => 1
    +      case _ => 2
    +    }
    +  }
    +
    +  /**
    +   * In Java, a method descriptor is valid only if it represents method parameters with a total
    +   * length of 255 or less. `this` contributes one unit and a parameter of type long or double
    +   * contributes two units.
    +   */
    +  def getParamLength(
    +      ctx: CodegenContext,
    +      inputs: Seq[Expression],
    +      subExprs: Seq[Expression]): Int = {
    +    // Initial value is 1 for `this`.
    +    1 + (inputs ++ subExprs).distinct.map(calculateParamLength(ctx, _)).sum
    +  }
    +
    +  /**
    +   * Given the lists of input attributes and variables to this expression, returns the strings of
    +   * funtion parameters. The first is the variable names used to call the function, the second is
    +   * the parameters used to declare the function in generated code.
    +   */
    +  def prepareFunctionParams(
    +      ctx: CodegenContext,
    +      inputAttrs: Seq[Expression],
    +      inputVars: Seq[ExprCode]): Seq[(String, String)] = {
    +    inputAttrs.zip(inputVars).flatMap { case (input, ev) =>
    +      val argType = ctx.javaType(input.dataType)
    +
    +      if (!input.nullable || ev.isNull == "true" || ev.isNull == "false") {
    +        Seq((ev.value, s"$argType ${ev.value}"))
    +      } else {
    +        Seq((ev.value, s"$argType ${ev.value}"), (ev.isNull, s"boolean ${ev.isNull}"))
    --- End diff --
    
    is it possible that `ev.value` is a literal?


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156297038
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala ---
    @@ -236,4 +237,24 @@ class WholeStageCodegenSuite extends QueryTest with SharedSQLContext {
           }
         }
       }
    +
    +  test("SPARK-22551: Fix 64kb limit for deeply nested expressions under wholestage codegen") {
    +    import testImplicits._
    +    withTempPath { dir =>
    +      val path = dir.getCanonicalPath
    +      val df = Seq(("abc", 1)).toDF("key", "int")
    +      df.write.parquet(path)
    +
    +      var strExpr: Expression = col("key").expr
    +      for (_ <- 1 to 150) {
    +        strExpr = Decode(Encode(strExpr, Literal("utf-8")), Literal("utf-8"))
    +      }
    +      val expressions = Seq(If(EqualTo(strExpr, strExpr), strExpr, strExpr))
    +
    +      val df2 = spark.read.parquet(path).select(expressions.map(Column(_)): _*)
    +      val plan = df2.queryExecution.executedPlan
    +      assert(plan.find(_.isInstanceOf[WholeStageCodegenExec]).isDefined)
    --- End diff --
    
    The error message is:
    ```
    [info]   Cause: org.codehaus.janino.InternalCompilerException: Code of method "processNext()V" of class "org.apache
    .spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator" grows beyond 64 KB
    [info]   at org.codehaus.janino.CodeContext.makeSpace(CodeContext.java:990)
    [info]   at org.codehaus.janino.CodeContext.write(CodeContext.java:867)
    [info]   at org.codehaus.janino.UnitCompiler.writeOpcode(UnitCompiler.java:11901)
    [info]   at org.codehaus.janino.UnitCompiler.pushConstant(UnitCompiler.java:10362)
    [info]   at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:5231)
    [info]   at org.codehaus.janino.UnitCompiler.access$8500(UnitCompiler.java:212)
    [info]   at org.codehaus.janino.UnitCompiler$12.visitIntegerLiteral(UnitCompiler.java:4099)
    [info]   at org.codehaus.janino.UnitCompiler$12.visitIntegerLiteral(UnitCompiler.java:4070)
    [info]   at org.codehaus.janino.Java$IntegerLiteral.accept(Java.java:5282)
    [info]   at org.codehaus.janino.UnitCompiler.compileGet(UnitCompiler.java:4070)
    [info]   at org.codehaus.janino.UnitCompiler.fakeCompile(UnitCompiler.java:3406)
    [info]   at org.codehaus.janino.UnitCompiler.compileGetValue(UnitCompiler.java:5247)
    [info]   at org.codehaus.janino.UnitCompiler.compileContext2(UnitCompiler.java:4033)
    [info]   at org.codehaus.janino.UnitCompiler.access$6000(UnitCompiler.java:212)
    ```


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156291349
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -55,8 +55,45 @@ import org.apache.spark.util.{ParentClassLoader, Utils}
      *                 to null.
      * @param value A term for a (possibly primitive) value of the result of the evaluation. Not
      *              valid if `isNull` is set to `true`.
    + * @param inputRow A term that holds the input row name when generating this code.
    + * @param inputVars A list of [[ExprInputVar]] that holds input variables when generating this code.
      */
    -case class ExprCode(var code: String, var isNull: String, var value: String)
    +case class ExprCode(
    +    var code: String,
    +    var isNull: String,
    +    var value: String,
    +    var inputRow: String = null,
    +    var inputVars: Seq[ExprInputVar] = Seq.empty) {
    +
    +  // Returns true if this value is a literal.
    +  def isLiteral(): Boolean = {
    +    assert(value.nonEmpty, "ExprCode.value can't be empty string.")
    +
    +    if (value == "true" || value == "false" || value == "null") {
    +      true
    +    } else {
    +      // The valid characters for the first character of a Java variable is [a-zA-Z_$].
    +      value.head match {
    +        case v if v >= 'a' && v <= 'z' => false
    --- End diff --
    
    or `isGloballyAccess`?


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by mgaido91 <gi...@git.apache.org>.
Github user mgaido91 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154201470
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,259 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu children expressions.
    +   */
    +  def getExpressionInputParams(
    +      ctx: CodegenContext,
    +      expr: Expression): Option[(Seq[String], Seq[String])] = {
    +    val (inputAttrs, inputVars) = getInputVarsForChildren(ctx, expr)
    +    val inputRows = ctx.INPUT_ROW +: getInputRowsForChildren(ctx, expr)
    +    val subExprs = getSubExprInChildren(ctx, expr)
    +
    +    val paramsFromRows = inputRows.distinct.filter(_ != null).map { row =>
    +      (row, s"InternalRow $row")
    +    }
    +    val paramsFromColumns = prepareFunctionParams(ctx, inputAttrs, inputVars)
    +    val paramsFromSubExprs = getParamsForSubExprs(ctx, subExprs)
    +    val paramsLength = getParamLength(ctx, inputAttrs, subExprs) + 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
    +  }
    +
    +  /**
    +   * Given the list of eliminated subexpressions used in the children expressions, returns the
    +   * strings of funtion parameters. The first is the variable names used to call the function,
    +   * the second is the parameters used to declare the function in generated code.
    +   */
    +  def getParamsForSubExprs(
    +      ctx: CodegenContext,
    +      subExprs: Seq[Expression]): Seq[(String, String)] = {
    +    subExprs.flatMap { subExpr =>
    +      val argType = ctx.javaType(subExpr.dataType)
    +
    +      val subExprState = ctx.subExprEliminationExprs(subExpr)
    +      (subExprState.value, subExprState.isNull)
    +
    +      if (!subExpr.nullable || subExprState.isNull == "true" || subExprState.isNull == "false") {
    +        Seq((subExprState.value, s"$argType ${subExprState.value}"))
    +      } else {
    +        Seq((subExprState.value, s"$argType ${subExprState.value}"),
    +          (subExprState.isNull, s"boolean ${subExprState.isNull}"))
    +      }
    +    }.distinct
    +  }
    +
    +  /**
    +   * Retrieves previous input rows referred by children and deferred expressions.
    +   */
    +  def getInputRowsForChildren(ctx: CodegenContext, expr: Expression): Seq[String] = {
    +    expr.children.flatMap(getInputRows(ctx, _)).distinct
    +  }
    +
    +  /**
    +   * Given a child expression, retrieves previous input rows referred by it or deferred expressions
    +   * which are needed to evaluate it.
    +   */
    +  def getInputRows(ctx: CodegenContext, child: Expression): Seq[String] = {
    +    child.flatMap {
    +      // An expression directly evaluates on current input row.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars == null ||
    +          ctx.currentVars(ordinal) == null =>
    +        Seq(ctx.INPUT_ROW)
    +
    +      // An expression which is not evaluated yet. Tracks down to find input rows.
    +      case BoundReference(ordinal, _, _) if ctx.currentVars(ordinal).code != "" =>
    +        trackDownRow(ctx, ctx.currentVars(ordinal))
    +
    +      case _ => Seq.empty
    +    }.distinct
    +  }
    +
    +  /**
    +   * Tracks down input rows referred by the generated code snippet.
    +   */
    +  def trackDownRow(ctx: CodegenContext, exprCode: ExprCode): Seq[String] = {
    +    var exprCodes: List[ExprCode] = List(exprCode)
    --- End diff --
    
    same comment as for `trackDownVar`


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    > can we have a google doc so that we can leave comments inline? thanks!
    
    Sure. The google doc is at https://docs.google.com/document/d/1By_V-A2sxCWbP7dZ5EzHIuMSe8K0fQL9lqovGWXnsfs/edit?usp=sharing



---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    @mgaido91 Thanks for the review.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156291182
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -55,8 +55,45 @@ import org.apache.spark.util.{ParentClassLoader, Utils}
      *                 to null.
      * @param value A term for a (possibly primitive) value of the result of the evaluation. Not
      *              valid if `isNull` is set to `true`.
    + * @param inputRow A term that holds the input row name when generating this code.
    + * @param inputVars A list of [[ExprInputVar]] that holds input variables when generating this code.
      */
    -case class ExprCode(var code: String, var isNull: String, var value: String)
    +case class ExprCode(
    +    var code: String,
    +    var isNull: String,
    +    var value: String,
    +    var inputRow: String = null,
    +    var inputVars: Seq[ExprInputVar] = Seq.empty) {
    +
    +  // Returns true if this value is a literal.
    +  def isLiteral(): Boolean = {
    +    assert(value.nonEmpty, "ExprCode.value can't be empty string.")
    +
    +    if (value == "true" || value == "false" || value == "null") {
    +      true
    +    } else {
    +      // The valid characters for the first character of a Java variable is [a-zA-Z_$].
    +      value.head match {
    +        case v if v >= 'a' && v <= 'z' => false
    --- End diff --
    
    another idea, let's move `isLiteral` and `isEvaluated` to `ExpressionCodegen`. Seems these 2 are not general definition of `ExprCode`, and only makes sense for `ctx.currentVars`.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    After some more thoughts, this PR makes a new contract that Spark doesn't promise before: `Expression.genCode` must output something that can be used as parameter name or literal.
    
    I do remember in some places we just output statement like `a + 1` for codegen, but I could be wrong. At least we need to check all the places and document this new contract before merging this PR.
    
    Another solution is to not make this contract. By a quick look this seems hard to do, because at the time of doing this, the code(method body) is already generated and we don't know how to replace statement like `a + 1` with the generated parameter name, inside the method body. We may need to do this fix earlier in the codegen procedure.
    
    I'm going to revert it, let's have a proper design doc and resubmit this. Sorry for the inconvenient!


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by mgaido91 <gi...@git.apache.org>.
Github user mgaido91 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154195724
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,259 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu children expressions.
    +   */
    +  def getExpressionInputParams(
    +      ctx: CodegenContext,
    +      expr: Expression): Option[(Seq[String], Seq[String])] = {
    +    val (inputAttrs, inputVars) = getInputVarsForChildren(ctx, expr)
    +    val inputRows = ctx.INPUT_ROW +: getInputRowsForChildren(ctx, expr)
    +    val subExprs = getSubExprInChildren(ctx, expr)
    --- End diff --
    
    ditto


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154974442
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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.
    --- End diff --
    
    For example, `ProjectExec` doesn't pre-evaluate all input attributes from its child operator.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84769 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84769/testReport)** for PR 19813 at commit [`f35974e`](https://github.com/apache/spark/commit/f35974e1dfb47387dc952d30a55eee0354bdea63).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84741 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84741/testReport)** for PR 19813 at commit [`655917c`](https://github.com/apache/spark/commit/655917cadf86ab17b8a730f282db544cb348d63f).
     * This patch **fails SparkR unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    Now the way to extract necessary parameters is moved out from `Expression`. We can use it to prepare parameters for an API such as `splitExpressions`.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154574087
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu 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 paramsFromSubExprs = prepareFunctionParams(ctx, subExprs, 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, subExprs) + 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] = {
    +    subExprs.map { subExpr =>
    +      val stat = ctx.subExprEliminationExprs(subExpr)
    --- End diff --
    
    nit: what does `stat` mean? In general, it is related to `statistics`. Is it better to use another variable name?


---

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


[GitHub] spark issue #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nested expr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    **[Test build #84756 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84756/testReport)** for PR 19813 at commit [`1251dfa`](https://github.com/apache/spark/commit/1251dfa305f4f1f8e34d7deb235bfa500d057fb4).


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r156363168
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegenSuite.scala ---
    @@ -0,0 +1,168 @@
    +/*
    + * 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 org.apache.spark.SparkFunSuite
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.types.IntegerType
    +
    +class ExpressionCodegenSuite extends SparkFunSuite {
    +
    +  test("Returns eliminated subexpressions for expression") {
    +    val ctx = new CodegenContext()
    +    val subExpr = Add(Literal(1), Literal(2))
    +    val exprs = Seq(Add(subExpr, Literal(3)), Add(subExpr, Literal(4)))
    +
    +    ctx.generateExpressions(exprs, doSubexpressionElimination = true)
    +    val subexpressions = ExpressionCodegen.getSubExprInChildren(ctx, exprs(0))
    +    assert(subexpressions.length == 1 && subexpressions(0) == subExpr)
    +  }
    +
    +  test("Gets parameters for subexpressions") {
    +    val ctx = new CodegenContext()
    +    val subExprs = Seq(
    +      Add(Literal(1), AttributeReference("a", IntegerType, nullable = false)()), // non-nullable
    +      Add(Literal(2), AttributeReference("b", IntegerType, nullable = true)()))  // nullable
    +
    +    ctx.subExprEliminationExprs.put(subExprs(0), SubExprEliminationState("false", "value1"))
    +    ctx.subExprEliminationExprs.put(subExprs(1), SubExprEliminationState("isNull2", "value2"))
    +
    +    val subExprCodes = ExpressionCodegen.getSubExprCodes(ctx, subExprs)
    +    val subVars = subExprs.zip(subExprCodes).map { case (expr, exprCode) =>
    +      ExprInputVar(exprCode, expr.dataType, expr.nullable)
    +    }
    +    val params = ExpressionCodegen.prepareFunctionParams(ctx, subVars)
    +    assert(params.length == 3)
    +    assert(params(0) == Tuple2("value1", "int value1"))
    +    assert(params(1) == Tuple2("value2", "int value2"))
    +    assert(params(2) == Tuple2("isNull2", "boolean isNull2"))
    +  }
    +
    +  test("Returns input variables for expression: current variables") {
    +    val ctx = new CodegenContext()
    +    val currentVars = Seq(
    +      ExprCode("", isNull = "false", value = "value1"),             // evaluated
    +      ExprCode("", isNull = "isNull2", value = "value2"),           // evaluated
    +      ExprCode("fake code;", isNull = "isNull3", value = "value3")) // not evaluated
    +    ctx.currentVars = currentVars
    +    ctx.INPUT_ROW = null
    +
    +    val expr = If(Literal(false),
    +      Add(BoundReference(0, IntegerType, nullable = false),
    +          BoundReference(1, IntegerType, nullable = true)),
    +        BoundReference(2, IntegerType, nullable = true))
    +
    +    val inputVars = ExpressionCodegen.getInputVarsForChildren(ctx, expr)
    +    // Only two evaluated variables included.
    +    assert(inputVars.length == 2)
    +    assert(inputVars(0).dataType == IntegerType && inputVars(0).nullable == false)
    +    assert(inputVars(1).dataType == IntegerType && inputVars(1).nullable == true)
    +    assert(inputVars(0).exprCode == currentVars(0))
    +    assert(inputVars(1).exprCode == currentVars(1))
    +
    +    val params = ExpressionCodegen.prepareFunctionParams(ctx, inputVars)
    +    assert(params.length == 3)
    +    assert(params(0) == Tuple2("value1", "int value1"))
    +    assert(params(1) == Tuple2("value2", "int value2"))
    +    assert(params(2) == Tuple2("isNull2", "boolean isNull2"))
    +  }
    +
    +  test("Returns input variables for expression: deferred variables") {
    +    val ctx = new CodegenContext()
    +
    +    // The referred column is not evaluated yet. But it depends on an evaluated column from
    +    // other operator.
    +    val currentVars = Seq(ExprCode("fake code;", isNull = "isNull1", value = "value1"))
    +
    +    // currentVars(0) depends on this evaluated column.
    +    currentVars(0).inputVars = Seq(ExprInputVar(ExprCode("", isNull = "isNull2", value = "value2"),
    +      dataType = IntegerType, nullable = true))
    +    ctx.currentVars = currentVars
    +    ctx.INPUT_ROW = null
    +
    +    val expr = Add(Literal(1), BoundReference(0, IntegerType, nullable = false))
    +    val inputVars = ExpressionCodegen.getInputVarsForChildren(ctx, expr)
    +    assert(inputVars.length == 1)
    +    assert(inputVars(0).dataType == IntegerType && inputVars(0).nullable == true)
    +
    +    val params = ExpressionCodegen.prepareFunctionParams(ctx, inputVars)
    +    assert(params.length == 2)
    +    assert(params(0) == Tuple2("value2", "int value2"))
    +    assert(params(1) == Tuple2("isNull2", "boolean isNull2"))
    +  }
    +
    +  test("Returns input rows for expression") {
    +    val ctx = new CodegenContext()
    +    ctx.currentVars = null
    +    ctx.INPUT_ROW = "i"
    +
    +    val expr = Add(BoundReference(0, IntegerType, nullable = false),
    +      BoundReference(1, IntegerType, nullable = true))
    +    val inputRows = ExpressionCodegen.getInputRowsForChildren(ctx, expr)
    +    assert(inputRows.length == 1)
    +    assert(inputRows(0) == "i")
    +  }
    +
    +  test("Returns input rows for expression: deferred expression") {
    +    val ctx = new CodegenContext()
    +
    +    // The referred column is not evaluated yet. But it depends on an input row from
    +    // other operator.
    +    val currentVars = Seq(ExprCode("fake code;", isNull = "isNull1", value = "value1"))
    +    currentVars(0).inputRow = "inputadaptor_row1"
    +    ctx.currentVars = currentVars
    +    ctx.INPUT_ROW = null
    +
    +    val expr = Add(Literal(1), BoundReference(0, IntegerType, nullable = false))
    +    val inputRows = ExpressionCodegen.getInputRowsForChildren(ctx, expr)
    +    assert(inputRows.length == 1)
    +    assert(inputRows(0) == "inputadaptor_row1")
    +  }
    +
    --- End diff --
    
    better to have one more test case that has both input rows and input variables.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154965681
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -55,8 +55,20 @@ import org.apache.spark.util.{ParentClassLoader, Utils}
      *                 to null.
      * @param value A term for a (possibly primitive) value of the result of the evaluation. Not
      *              valid if `isNull` is set to `true`.
    + * @param inputRow A term that holds the input row name when generating this code.
    + * @param inputVars A list of [[ExprInputVar]] that holds input variables when generating this code.
      */
    -case class ExprCode(var code: String, var isNull: String, var value: String)
    +case class ExprCode(
    +    var code: String,
    +    var isNull: String,
    +    var value: String,
    +    var inputRow: String = null,
    +    var inputVars: Seq[ExprInputVar] = Seq.empty)
    +
    +/**
    + * Represents an input variable [[ExprCode]] to an evaluation of an [[Expression]].
    --- End diff --
    
    Added.


---

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


[GitHub] spark pull request #19813: [SPARK-22600][SQL] Fix 64kb limit for deeply nest...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19813#discussion_r154582497
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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._
    +
    +/**
    + * Defines APIs used in expression code generation.
    + */
    +object ExpressionCodegen {
    +
    +  /**
    +   * 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 bu 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 paramsFromSubExprs = prepareFunctionParams(ctx, subExprs, subExprCodes)
    --- End diff --
    
    I think so. The eliminated subexpressions will be extracted as parameters too.


---

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


[GitHub] spark issue #19813: [WIP][SPARK-22600][SQL] Fix 64kb limit for deeply nested...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/19813
  
    BTW `splitExpressions` doesn't work with subexpressions since the beinning, it's another topic to integrate them.


---

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