You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by DonnyZone <gi...@git.apache.org> on 2017/10/23 11:50:56 UTC

[GitHub] spark pull request #19559: [SPARK-22333][SQL]ColumnReference should get high...

GitHub user DonnyZone opened a pull request:

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

    [SPARK-22333][SQL]ColumnReference should get higher priority than timeFunctionCall(CURRENT_DATE, CURRENT_TIMESTAMP)

    ## What changes were proposed in this pull request?
    https://issues.apache.org/jira/browse/SPARK-22333
    
    In current version, users can use CURRENT_DATE() and CURRENT_TIMESTAMP without specifying braces.
    However, when a table has columns named as "current_date" or "current_timestamp", it will still be parsed as function call.
    
    There are many such cases in our production cluster. We get the wrong answer due to this inappropriate behevior. In general, ColumnReference should get higher priority than timeFunctionCall.
    
    ## How was this patch tested?
    unit test
    manul test


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

    $ git pull https://github.com/DonnyZone/spark master

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

    https://github.com/apache/spark/pull/19559.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 #19559
    
----
commit 60a5a56a77245f3467920c60cc39ae6cd4989572
Author: donnyzone <we...@gmail.com>
Date:   2017-10-23T11:40:31Z

    spark-22333

----


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    **[Test build #83080 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83080/testReport)** for PR 19559 at commit [`6c932b7`](https://github.com/apache/spark/commit/6c932b7e99e3478b54a8c0494761c7b2c8be100e).
     * 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 #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    **[Test build #83080 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83080/testReport)** for PR 19559 at commit [`6c932b7`](https://github.com/apache/spark/commit/6c932b7e99e3478b54a8c0494761c7b2c8be100e).


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    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 #19559: [SPARK-22333][SQL]ColumnReference should get higher prio...

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

    https://github.com/apache/spark/pull/19559
  
    @hvanhovell Yes! I made something wrong. The `timeFunctionCall` has conflicts with `columnReference`. This fix will break every use of CURRENT_DATE/CURRENT_TIMESTAMP.
    
    For [SPARK-16836](https://github.com/apache/spark/pull/14442), 
    I think this feature should be implemented in analysis phase rather than in parser phase. When there is no such columns, they can be transformed as functions. Another approach is to define a configuration.


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    @gatorsmile @gatorsmile 
    There are still two issues need to be figured out.
    (1)It will be complicated to determine whether a literal function should be resolved as Expression or NamedExpression.
    Current fix just resolves them as NamedExpressions (i.e., Alias). 
    However, this leads to different schema in some cases, for example, the end-to-end test sql.
    `select current_date = current_date()`
    The output schema will be
    `struct<(current_date() AS ’current_date()‘ = current_date()):boolean>`
    (2)Shall we also support the feature in ResolveMissingReference rule?
    e.g., `select id from table order by current_date`
    The same logic in different rules brings redundant code.


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]ColumnReference should get higher prio...

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

    https://github.com/apache/spark/pull/19559
  
    ping @hvanhovell @gatorsmile 


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    **[Test build #83077 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83077/testReport)** for PR 19559 at commit [`5323fbb`](https://github.com/apache/spark/commit/5323fbb97902cab83a1f35fa519429093324b5aa).


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    **[Test build #83138 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83138/testReport)** for PR 19559 at commit [`a57fb81`](https://github.com/apache/spark/commit/a57fb8102b6f9a66e74e7038df94c8d3f93c4f00).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `      throw new IllegalStateException(\"The main method in the given main class must be static\")`
      * `class UnrecognizedBlockId(name: String)`
      * `      assert(currentClass != null, \"The outer class can't be null.\")`
      * `    assert(currentClass != null, \"The outer class can't be null.\")`
      * `              assert(currentClass != null, \"The outer class can't be null.\")`
      * `class CrossValidator(Estimator, ValidatorParams, HasParallelism, MLReadable, MLWritable):`
      * `class TrainValidationSplit(Estimator, ValidatorParams, HasParallelism, MLReadable, MLWritable):`
      * `class ArrowWriter(val root: VectorSchemaRoot, fields: Array[ArrowFieldWriter]) `


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

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


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

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


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    **[Test build #83122 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83122/testReport)** for PR 19559 at commit [`2efd4f7`](https://github.com/apache/spark/commit/2efd4f710c66060536c27239371d714bda228cab).
     * 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 #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, ...

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

    https://github.com/apache/spark/pull/19559#discussion_r147318994
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala ---
    @@ -56,6 +56,24 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext {
         checkAnswer(sql("""SELECT CURRENT_TIMESTAMP() = NOW()"""), Row(true))
       }
     
    +  test("SPARK-22333: timeFunctionCall has conflicts with columnReference ") {
    +    val df = Seq((1, 2), (2, 3)).toDF("current_date", "current_timestamp")
    +    df.createOrReplaceTempView("ttf")
    +    withTempView("ttf") {
    +      checkAnswer(sql("SELECT current_date, current_timestamp FROM ttf"),
    +        Seq(Row(1, 2), Row(2, 3)))
    +    }
    +
    +    val df1 = Seq((1, 2), (2, 3)).toDF("a", "b")
    +    df1.createOrReplaceTempView("ttf1")
    +    withTempView("ttf1") {
    +      checkAnswer(
    +        sql("SELECT current_date = current_date(), current_timestamp = current_timestamp(), " +
    +          "a, b FROM ttf1"),
    +        Seq(Row(true, true, 1, 2), Row(true, true, 2, 3)))
    +    }
    +  }
    --- End diff --
    
    Move these to `datetime.sql`?


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

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


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    @DonnyZone Analyzer is the best place to fix the issue.


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    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 #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

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


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

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


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    **[Test build #83081 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83081/testReport)** for PR 19559 at commit [`b8075e1`](https://github.com/apache/spark/commit/b8075e1ad0351446326534ac2a5d94b54095ca0b).
     * 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 #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    **[Test build #83117 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83117/testReport)** for PR 19559 at commit [`5baf98e`](https://github.com/apache/spark/commit/5baf98e0422d10bb7ecc5d222e1b27224c444a58).


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    **[Test build #83110 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83110/testReport)** for PR 19559 at commit [`4b13343`](https://github.com/apache/spark/commit/4b13343c3faa2300af3c92afb6e68f3a717b596b).


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    **[Test build #83122 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83122/testReport)** for PR 19559 at commit [`2efd4f7`](https://github.com/apache/spark/commit/2efd4f710c66060536c27239371d714bda228cab).


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    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 #19559: [SPARK-22333][SQL]ColumnReference should get higher prio...

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

    https://github.com/apache/spark/pull/19559
  
    **[Test build #82988 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82988/testReport)** for PR 19559 at commit [`60a5a56`](https://github.com/apache/spark/commit/60a5a56a77245f3467920c60cc39ae6cd4989572).
     * 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 #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    **[Test build #83078 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83078/testReport)** for PR 19559 at commit [`5323fbb`](https://github.com/apache/spark/commit/5323fbb97902cab83a1f35fa519429093324b5aa).


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    @gatorsmile 
    It seems that we should also support this logic in `resolveExpressions` for Sort plan.
    `select a from t order by current_date`
    Therefore, I think current `resolveAsLiteralFunctions` can be moved out from `ResolveReference` rule to be  a common function `resolveLiteralFunctions`


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    Yes, ordering in `Sort(ordering, global, child)` is resolved in `resolveExpression`


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]ColumnReference should get higher prio...

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

    https://github.com/apache/spark/pull/19559
  
    ok to test


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    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 #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

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


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    **[Test build #83114 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83114/testReport)** for PR 19559 at commit [`87c2073`](https://github.com/apache/spark/commit/87c2073fc51cd1805174f0557d6a05a584a77c5d).
     * 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 #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, ...

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

    https://github.com/apache/spark/pull/19559#discussion_r147318866
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala ---
    @@ -783,6 +783,37 @@ class Analyzer(
           }
         }
     
    +    /**
    +     * Literal functions do not require the user to specify braces when calling them
    +     * When an attributes is not resolvable, we try to resolve it as a literal function.
    +     */
    +    private def resolveAsLiteralFunctions(
    +        nameParts: Seq[String],
    +        attribute: UnresolvedAttribute,
    +        plan: LogicalPlan): Option[Expression] = {
    +      if (nameParts.length != 1) {
    +        return None
    +      }
    +      val isNamedExpression = plan match {
    +        case a @ Aggregate(_, aggs, _) if (aggs.contains(attribute)) =>
    +          true
    +        case p @ Project(projList, _) if (projList.contains(attribute)) =>
    +          true
    +        case _ =>
    --- End diff --
    
    Miss `Windows`


---

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


[GitHub] spark pull request #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, ...

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

    https://github.com/apache/spark/pull/19559#discussion_r147041980
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala ---
    @@ -139,6 +139,7 @@ class Analyzer(
           ExtractGenerator ::
           ResolveGenerate ::
           ResolveFunctions ::
    +      ResolveLiteralFunctions ::
    --- End diff --
    
    Agree! I will refactor it.


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/83071/
    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 #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, ...

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

    https://github.com/apache/spark/pull/19559#discussion_r147035765
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala ---
    @@ -139,6 +139,7 @@ class Analyzer(
           ExtractGenerator ::
           ResolveGenerate ::
           ResolveFunctions ::
    +      ResolveLiteralFunctions ::
    --- End diff --
    
    The order matters. It assumes `ResolveReferences` should be run before this rule. However, `ResolveReferences` might need multiple passes to resolve all the references. Thus, how about moving the logics into `ResolveReferences `? If the attributes are not resolvable, we try to see whether it is a function literal?


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

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


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    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 #19559: [SPARK-22333][SQL]ColumnReference should get high...

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

    https://github.com/apache/spark/pull/19559#discussion_r146309332
  
    --- Diff: sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 ---
    @@ -585,6 +584,7 @@ primaryExpression
         | identifier                                                                               #columnReference
         | base=primaryExpression '.' fieldName=identifier                                          #dereference
         | '(' expression ')'                                                                       #parenthesizedExpression
    +    | name=(CURRENT_DATE | CURRENT_TIMESTAMP)                                                  #timeFunctionCall
    --- End diff --
    
    Won't this break every use of `CURRENT_DATE`/`CURRENT_TIMESTAMP`? They will now be interpreted as an identifier.


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

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


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]ColumnReference should get higher prio...

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

    https://github.com/apache/spark/pull/19559
  
    **[Test build #82988 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82988/testReport)** for PR 19559 at commit [`60a5a56`](https://github.com/apache/spark/commit/60a5a56a77245f3467920c60cc39ae6cd4989572).


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    **[Test build #83121 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83121/testReport)** for PR 19559 at commit [`846cee4`](https://github.com/apache/spark/commit/846cee48e4dd97ed2bbc1f0a01e5797fb4bd321b).
     * 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 #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    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 #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, ...

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

    https://github.com/apache/spark/pull/19559#discussion_r147068227
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala ---
    @@ -844,7 +863,12 @@ class Analyzer(
             q.transformExpressionsUp  {
               case u @ UnresolvedAttribute(nameParts) =>
                 // Leave unchanged if resolution fails. Hopefully will be resolved next round.
    -            val result = withPosition(u) { q.resolveChildren(nameParts, resolver).getOrElse(u) }
    +            val result =
    +              withPosition(u) {
    +                q.resolveChildren(nameParts, resolver).getOrElse {
    --- End diff --
    
    You can also use `orElse`:
    `q.resolveChildren(nameParts, resolver).orElse(resolveAsLiteralFunctions(nameParts)).getOrElse(u)`


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    **[Test build #83110 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83110/testReport)** for PR 19559 at commit [`4b13343`](https://github.com/apache/spark/commit/4b13343c3faa2300af3c92afb6e68f3a717b596b).
     * 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 issue #19559: [SPARK-22333][SQL]ColumnReference should get higher prio...

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

    https://github.com/apache/spark/pull/19559
  
    ping @gatorsmile @hvanhovell @cloud-fan 


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]ColumnReference should get higher prio...

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

    https://github.com/apache/spark/pull/19559
  
    Can one of the admins verify this patch?


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]ColumnReference should get higher prio...

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

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


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    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 #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    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 #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, ...

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

    https://github.com/apache/spark/pull/19559#discussion_r147330270
  
    --- Diff: sql/core/src/test/resources/sql-tests/inputs/datetime.sql ---
    @@ -8,3 +8,18 @@ select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd')
     select to_timestamp(null), to_timestamp('2016-12-31 00:12:00'), to_timestamp('2016-12-31', 'yyyy-MM-dd');
     
     select dayofweek('2007-02-03'), dayofweek('2009-07-30'), dayofweek('2017-05-27'), dayofweek(null), dayofweek('1582-10-15 13:10:15');
    +
    +-- [SPARK-22333]: timeFunctionCall has conflicts with columnReference
    +create temporary view ttf1 as select * from values
    +  (1, 2),
    +  (2, 3),
    +  as ttf1(current_date, current_timestamp);
    +  
    +select current_date, current_timestamp from ttf1
    +
    +create temporary view ttf2 as select * from values
    +  (1, 2),
    +  (2, 3),
    +  as ttf2(a, b);
    +  
    +select current_date = current_date(), current_timestamp = current_timestamp(), a, b from ttf2
    --- End diff --
    
    Add an empty line 


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    LGTM and cc @hvanhovell for a final sign-off. 
    
    BTW, I might merge https://github.com/apache/spark/pull/19585 before this PR. Sorry for the conflicts. 


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    **[Test build #83044 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83044/testReport)** for PR 19559 at commit [`c38ab56`](https://github.com/apache/spark/commit/c38ab563a04ea48bc191fc6912305f0b924927be).
     * This patch **fails Scala style 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 #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    **[Test build #83078 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83078/testReport)** for PR 19559 at commit [`5323fbb`](https://github.com/apache/spark/commit/5323fbb97902cab83a1f35fa519429093324b5aa).
     * 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 #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    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 #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

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


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    **[Test build #83121 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83121/testReport)** for PR 19559 at commit [`846cee4`](https://github.com/apache/spark/commit/846cee48e4dd97ed2bbc1f0a01e5797fb4bd321b).


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    Sure, I will submit it later.


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    `select a from t order by current_date` is not working?


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    **[Test build #83114 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83114/testReport)** for PR 19559 at commit [`87c2073`](https://github.com/apache/spark/commit/87c2073fc51cd1805174f0557d6a05a584a77c5d).


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    **[Test build #83077 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83077/testReport)** for PR 19559 at commit [`5323fbb`](https://github.com/apache/spark/commit/5323fbb97902cab83a1f35fa519429093324b5aa).
     * 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 issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/83078/
    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 #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, ...

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

    https://github.com/apache/spark/pull/19559#discussion_r147068164
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala ---
    @@ -783,6 +783,25 @@ class Analyzer(
           }
         }
     
    +    /**
    +     * Literal functions do not require the user to specify braces when calling them
    +     * When an attributes is not resolvable, we try to resolve it as a literal function.
    +     */
    +    private def resolveAsLiteralFunctions(nameParts: Seq[String]): Option[NamedExpression] = {
    +      if (nameParts.length != 1) {
    +        return None
    +      }
    +      // support CURRENT_DATE and CURRENT_TIMESTAMP
    +      val literalFunctions = Seq(CurrentDate(), CurrentTimestamp())
    +      val name = nameParts.head
    +      val func = literalFunctions.find(e => resolver(e.prettyName, name))
    +      if (func.isDefined) {
    --- End diff --
    
    Thanks, I will refactor it.


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

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


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    **[Test build #83076 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83076/testReport)** for PR 19559 at commit [`2d42abf`](https://github.com/apache/spark/commit/2d42abf32d53a1f753bcdff367d476ac0638ccd5).
     * 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 #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    @gatorsmile Thank for your advice, I will work on it.


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    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 #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    Could you submit a backport PR to 2.2?


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    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 #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    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 #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    **[Test build #83117 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83117/testReport)** for PR 19559 at commit [`5baf98e`](https://github.com/apache/spark/commit/5baf98e0422d10bb7ecc5d222e1b27224c444a58).
     * 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 #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    **[Test build #83111 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83111/testReport)** for PR 19559 at commit [`a96d945`](https://github.com/apache/spark/commit/a96d945773a86dd6d4d54ec35529a11197cb2f67).
     * 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 issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    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 #19559: [SPARK-22333][SQL]ColumnReference should get higher prio...

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

    https://github.com/apache/spark/pull/19559
  
    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 #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    You do not need to close it and just need to resolve the conflicts. If you are not familiar with it, I can wait and merge this PR first.


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    **[Test build #83071 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83071/testReport)** for PR 19559 at commit [`d485e25`](https://github.com/apache/spark/commit/d485e25649ca90558b7639629be920274876e27c).
     * 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 #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    Thanks! Merged to master.


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

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


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

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


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

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


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    **[Test build #83076 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83076/testReport)** for PR 19559 at commit [`2d42abf`](https://github.com/apache/spark/commit/2d42abf32d53a1f753bcdff367d476ac0638ccd5).


---

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


[GitHub] spark pull request #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, ...

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

    https://github.com/apache/spark/pull/19559#discussion_r147067888
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala ---
    @@ -783,6 +783,25 @@ class Analyzer(
           }
         }
     
    +    /**
    +     * Literal functions do not require the user to specify braces when calling them
    +     * When an attributes is not resolvable, we try to resolve it as a literal function.
    +     */
    +    private def resolveAsLiteralFunctions(nameParts: Seq[String]): Option[NamedExpression] = {
    +      if (nameParts.length != 1) {
    +        return None
    +      }
    +      // support CURRENT_DATE and CURRENT_TIMESTAMP
    +      val literalFunctions = Seq(CurrentDate(), CurrentTimestamp())
    +      val name = nameParts.head
    +      val func = literalFunctions.find(e => resolver(e.prettyName, name))
    +      if (func.isDefined) {
    --- End diff --
    
    Just map over the `func` option.


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    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 #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    **[Test build #83112 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83112/testReport)** for PR 19559 at commit [`36b4bbb`](https://github.com/apache/spark/commit/36b4bbbed03a6fb6375467e14cb94f2ccd2375ff).
     * This patch **fails Scala style 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 #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    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 #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    OK, I will close this PR after review and submit a new one, after merging https://github.com/apache/spark/pull/19585


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    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 #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

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


---

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


[GitHub] spark pull request #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, ...

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

    https://github.com/apache/spark/pull/19559#discussion_r147318831
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala ---
    @@ -783,6 +783,37 @@ class Analyzer(
           }
         }
     
    +    /**
    +     * Literal functions do not require the user to specify braces when calling them
    +     * When an attributes is not resolvable, we try to resolve it as a literal function.
    +     */
    +    private def resolveAsLiteralFunctions(
    +        nameParts: Seq[String],
    +        attribute: UnresolvedAttribute,
    +        plan: LogicalPlan): Option[Expression] = {
    +      if (nameParts.length != 1) {
    +        return None
    +      }
    +      val isNamedExpression = plan match {
    +        case a @ Aggregate(_, aggs, _) if (aggs.contains(attribute)) =>
    +          true
    +        case p @ Project(projList, _) if (projList.contains(attribute)) =>
    +          true
    +        case _ =>
    +          false
    +      }
    +      val wrapper: Expression => Expression = if (isNamedExpression) {
    +        f => Alias(f, toPrettySQL(f))()
    +      } else {
    +        f => f
    +      }
    --- End diff --
    
    ```Scala
          if (nameParts.length != 1) return None
          val isNamedExpression = plan match {
            case Aggregate(_, aggs, _) => aggs.contains(attribute)
            case Project(projList, _) => projList.contains(attribute)
            case Window(windowExpressions, _, _, _) => windowExpressions.contains(attribute)
            case _ => false
          }
          val wrapper: Expression => Expression =
            if (isNamedExpression) f => Alias(f, usePrettyExpression(f).sql)() else identity
    ```


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

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


---

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


[GitHub] spark pull request #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, ...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

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


---

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


[GitHub] spark pull request #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, ...

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

    https://github.com/apache/spark/pull/19559#discussion_r147330565
  
    --- Diff: sql/core/src/test/resources/sql-tests/inputs/datetime.sql ---
    @@ -8,3 +8,18 @@ select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd')
     select to_timestamp(null), to_timestamp('2016-12-31 00:12:00'), to_timestamp('2016-12-31', 'yyyy-MM-dd');
     
     select dayofweek('2007-02-03'), dayofweek('2009-07-30'), dayofweek('2017-05-27'), dayofweek(null), dayofweek('1582-10-15 13:10:15');
    +
    +-- [SPARK-22333]: timeFunctionCall has conflicts with columnReference
    +create temporary view ttf1 as select * from values
    +  (1, 2),
    +  (2, 3),
    +  as ttf1(current_date, current_timestamp);
    +  
    +select current_date, current_timestamp from ttf1
    +
    +create temporary view ttf2 as select * from values
    +  (1, 2),
    +  (2, 3),
    +  as ttf2(a, b);
    +  
    +select current_date = current_date(), current_timestamp = current_timestamp(), a, b from ttf2
    --- End diff --
    
    Fixed.


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

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


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

    https://github.com/apache/spark/pull/19559
  
    **[Test build #83112 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83112/testReport)** for PR 19559 at commit [`36b4bbb`](https://github.com/apache/spark/commit/36b4bbbed03a6fb6375467e14cb94f2ccd2375ff).


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

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


---

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


[GitHub] spark issue #19559: [SPARK-22333][SQL]timeFunctionCall(CURRENT_DATE, CURRENT...

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

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


---

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