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

[GitHub] spark pull request #18079: [SPARK-20841][SQL] Support column aliases for cat...

GitHub user maropu opened a pull request:

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

    [SPARK-20841][SQL] Support column aliases for catalog tables

    ## What changes were proposed in this pull request?
    This pr added parsing rules to support column aliases for catalog tables.
    
    ## How was this patch tested?
    Added tests in `PlanParserSuite`,  `SQLQueryTestSuite`, and `PlanParserSuite`.

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

    $ git pull https://github.com/maropu/spark SPARK-20841

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

    https://github.com/apache/spark/pull/18079.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 #18079
    
----
commit e2b50f37063098bad1df8ee65ad331b5694a39a6
Author: Takeshi Yamamuro <ya...@apache.org>
Date:   2017-05-23T23:12:03Z

    Support column aliases for catalog tables

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18079: [SPARK-20841][SQL] Support table column aliases i...

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

    https://github.com/apache/spark/pull/18079#discussion_r118836800
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableValuedFunctions.scala ---
    @@ -131,8 +131,9 @@ object ResolveTableValuedFunctions extends Rule[LogicalPlan] {
             val outputAttrs = resolvedFunc.output
             // Checks if the number of the aliases is equal to expected one
             if (u.outputNames.size != outputAttrs.size) {
    -          u.failAnalysis(s"expected ${outputAttrs.size} columns but " +
    -            s"found ${u.outputNames.size} columns")
    +          u.failAnalysis(s"Number of given aliases does not match number of output columns. " +
    +            s"Function name: ${u.functionName}; number of aliases: " +
    +            s"${u.outputNames.size}; number of output columns: ${outputAttrs.size}.")
             }
    --- End diff --
    
    This fix is not related to this pr though, I modified along with this comment: https://github.com/apache/spark/pull/18079/files#diff-57b3d87be744b7d79a9beacf8e5e5eb2R604


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18079: [SPARK-20841][SQL] Support column aliases for cat...

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

    https://github.com/apache/spark/pull/18079#discussion_r118833872
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala ---
    @@ -592,7 +592,23 @@ class Analyzer(
         def resolveRelation(plan: LogicalPlan): LogicalPlan = plan match {
           case u: UnresolvedRelation if !isRunningDirectlyOnFiles(u.tableIdentifier) =>
             val defaultDatabase = AnalysisContext.get.defaultDatabase
    -        val relation = lookupTableFromCatalog(u, defaultDatabase)
    +        val foundRelation = lookupTableFromCatalog(u, defaultDatabase)
    +
    +        // If alias names assigned, add `Project` with the aliases
    +        val relation = if (u.outputNames.nonEmpty) {
    +          val outputAttrs = foundRelation.output
    +          // Checks if the number of the aliases is equal to expected one
    +          if (u.outputNames.size != outputAttrs.size) {
    +            u.failAnalysis(s"expected ${outputAttrs.size} columns but found " +
    +              s"${u.outputNames.size} columns in alias names")
    --- End diff --
    
    >The number of column aliases does not match the number of column of the table `$tableName`. Number of column aliases: ${u.outputNames.size}; Number of columns: ${outputAttrs.size}.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18079: [SPARK-20841][SQL] Support table column aliases i...

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

    https://github.com/apache/spark/pull/18079#discussion_r118842292
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -676,9 +676,12 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
        * Create an aliased table reference. This is typically used in FROM clauses.
        */
       override def visitTableName(ctx: TableNameContext): LogicalPlan = withOrigin(ctx) {
    -    val table = UnresolvedRelation(visitTableIdentifier(ctx.tableIdentifier))
    -
    -    val tableWithAlias = Option(ctx.strictIdentifier).map(_.getText) match {
    +    val tableId = visitTableIdentifier(ctx.tableIdentifier)
    +    val table = Option(ctx.tableAlias.identifierList) match {
    --- End diff --
    
    Just if/else? Seems a bit heavy weight to wrap in an option...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18079: [SPARK-20841][SQL] Support column aliases for cat...

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

    https://github.com/apache/spark/pull/18079#discussion_r118833763
  
    --- Diff: sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 ---
    @@ -711,7 +715,7 @@ nonReserved
         | ADD
         | OVER | PARTITION | RANGE | ROWS | PRECEDING | FOLLOWING | CURRENT | ROW | LAST | FIRST | AFTER
         | MAP | ARRAY | STRUCT
    -    | LATERAL | WINDOW | REDUCE | TRANSFORM | USING | SERDE | SERDEPROPERTIES | RECORDREADER
    --- End diff --
    
    First, based on the link http://developer.mimer.se/validator/sql-reserved-words.tml, `USING` is a reserved word in SQL standard since SQL-92. 
    
    Second, since 1.2, Hive introduces a flag `hive.support.sql11.reserved.keywords` for backward compatbility, which defaults to `true`. 
    > Added In: Hive 1.2.0 with HIVE-6617: https://issues.apache.org/jira/browse/HIVE-6617
    > Whether to enable support for SQL2011 reserved keywords. When enabled, will support (part of) SQL2011 reserved keywords.
    
    In 2.2, Hive removes this flag and does not allow users to change it to false. That means, users are unable to use these reserved words as identifiers anymore, unless using them as quoted identifiers. See: https://issues.apache.org/jira/browse/HIVE-14872
    
    Thus, I think it is safe to remove `USING` from the non-reserved words. 
    
    cc @hvanhovell 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support table column aliases in FROM ...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support column aliases for catalog ta...

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

    https://github.com/apache/spark/pull/18079
  
    yea, I think so. I mean, in the name conflict case you described above, postgresql throws an error;
    ```
    postgres=# select * from testTable;
     col1 | col2 
    ------+------
    (0 rows)
    
    postgres=# select * from testTable t(a, b);
     a | b 
    ---+---
    (0 rows)
    
    postgres=# select col1 a, col2 b from testTable t(c, d);
    ERROR:  column "col1" does not exist at character 8
    STATEMENT:  select col1 a, col2 b from testTable t(c, d);
    ERROR:  column "col1" does not exist
    LINE 1: select col1 a, col2 b from testTable t(c, d);
    ```
    We donot support aliases for subquries. Should we include that support in this pr? Or, follow-up?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support column aliases for catalog ta...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support column aliases for catalog ta...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support column aliases for catalog ta...

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

    https://github.com/apache/spark/pull/18079
  
    **[Test build #77280 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77280/testReport)** for PR 18079 at commit [`902d2a3`](https://github.com/apache/spark/commit/902d2a35740f3c3bc0c97aae56c65d9d25df3a15).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class UnresolvedRelation(`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18079: [SPARK-20841][SQL] Support table column aliases i...

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

    https://github.com/apache/spark/pull/18079#discussion_r118844314
  
    --- Diff: sql/core/src/test/resources/sql-tests/inputs/aliases.sql ---
    @@ -0,0 +1,17 @@
    +-- Test data.
    --- End diff --
    
    ok


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support column aliases for catalog ta...

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

    https://github.com/apache/spark/pull/18079
  
    It sounds like PostgresSQL supports it. See the docs in https://www.postgresql.org/docs/9.2/static/sql-select.html 
    
    Actually, we also need to support the other alias in the from clauses:
    
    See the link: https://drill.apache.org/docs/from-clause/ and http://docs.aws.amazon.com/redshift/latest/dg/r_FROM_clause30.html
    ```SQL
       with_subquery_table_name [ [ AS ] alias [ ( column_alias [, ...] ) ] ]
       table_name [ [ AS ] alias [ ( column_alias [, ...] ) ] ]
       ( subquery ) [ AS ] alias [ ( column_alias [, ...] ) ]
    ```



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support column aliases for catalog ta...

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

    https://github.com/apache/spark/pull/18079
  
    Currently, we have the same behaviour;
    ```
    -- PostgreSQL
    postgres=# create table t1(col1 int, col2 int);
    CREATE TABLE
    
    postgres=# select col1 as a, col2 as b from t1 as t(c, d);
    ERROR:  column "col1" does not exist at character 8
    STATEMENT:  select col1 as a, col2 as b from t1 as t(c, d);
    ERROR:  column "col1" does not exist
    LINE 1: select col1 as a, col2 as b from t1 as t(c, d);
                   ^
    -- MySQL
    mysql> create table t1(col1 int, col2 int);
    Query OK, 0 rows affected (0.01 sec)
    
    mysql> select col1 as a, col2 as b from t1 as t(c, d);
    ERROR 1064 (42000): You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use near '(c, d)' at line 1
    mysql> 
    
    
    -- Spark with this pr
    scala> Seq((1, 2), (2, 3)).toDF("col1", "col2").createOrReplaceTempView("t")
    
    scala> sql("select col1 as a, col2 as b from t as t(c, d)").show
    org.apache.spark.sql.AnalysisException: cannot resolve '`col1`' given input columns: [c, d]; line 1 pos 7;
    'Project ['col1 AS a#18, 'col2 AS b#19]
    +- SubqueryAlias t
       +- Project [col1#5 AS c#20, col2#6 AS d#21]
          +- SubqueryAlias t
             +- Project [_1#2 AS col1#5, _2#3 AS col2#6]
                +- LocalRelation [_1#2, _2#3]
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support column aliases for catalog ta...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18079: [SPARK-20841][SQL] Support column aliases for cat...

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

    https://github.com/apache/spark/pull/18079#discussion_r118396605
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala ---
    @@ -49,7 +49,7 @@ class TableIdentifierParserSuite extends SparkFunSuite {
         "insert", "int", "into", "is", "lateral", "like", "local", "none", "null",
         "of", "order", "out", "outer", "partition", "percent", "procedure", "range", "reads", "revoke",
         "rollup", "row", "rows", "set", "smallint", "table", "timestamp", "to", "trigger",
    -    "true", "truncate", "update", "user", "using", "values", "with", "regexp", "rlike",
    --- End diff --
    
    ditto


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support table column aliases in FROM ...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support table column aliases in FROM ...

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

    https://github.com/apache/spark/pull/18079
  
    @gatorsmile ok, thanks for your suggestion. I'll check the doc. and make sub-tasks there.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support column aliases for catalog ta...

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

    https://github.com/apache/spark/pull/18079
  
    Yeah. That should be a negative case. The `FROM clause` should be resolved before `SELECT clause`.
    
    The PR title is not accurate. I think we should keep the original JIRA name. `Support table column aliases in FROM clause`.
    
    Yeah. These cases should be part of this JIRA. Please add the sub-tasks under this JIRA. Follow what Redshift documents and do them one by one?http://docs.aws.amazon.com/redshift/latest/dg/r_FROM_clause30.html 
    
    ```SQL
    with_subquery_table_name [ [ AS ] alias [ ( column_alias [, ...] ) ] ]
    table_name [ * ] [ [ AS ] alias [ ( column_alias [, ...] ) ] ]
    ( subquery ) [ AS ] alias [ ( column_alias [, ...] ) ]
    table_reference [ NATURAL ] join_type table_reference
    [ ON join_condition | USING ( join_column [, ...] ) ]
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support column aliases for catalog ta...

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

    https://github.com/apache/spark/pull/18079
  
    ok, I'll check.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support column aliases for catalog ta...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18079: [SPARK-20841][SQL] Support column aliases for cat...

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

    https://github.com/apache/spark/pull/18079#discussion_r118834583
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala ---
    @@ -36,8 +36,17 @@ class UnresolvedException[TreeType <: TreeNode[_]](tree: TreeType, function: Str
     
     /**
      * Holds the name of a relation that has yet to be looked up in a catalog.
    + * We could add alias names for columns in a relation:
    + * {{{
    + *   // Assign alias names
    + *   SELECT col1, col2 FROM testData AS t(col1, col2);
    + * }}}
      */
    -case class UnresolvedRelation(tableIdentifier: TableIdentifier) extends LeafNode {
    +case class UnresolvedRelation(
    +    tableIdentifier: TableIdentifier,
    +    outputNames: Seq[String] = Seq.empty)
    --- End diff --
    
    ok


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18079: [SPARK-20841][SQL] Support column aliases for cat...

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

    https://github.com/apache/spark/pull/18079#discussion_r118305063
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala ---
    @@ -49,7 +49,7 @@ class TableIdentifierParserSuite extends SparkFunSuite {
         "insert", "int", "into", "is", "lateral", "like", "local", "none", "null",
         "of", "order", "out", "outer", "partition", "percent", "procedure", "range", "reads", "revoke",
         "rollup", "row", "rows", "set", "smallint", "table", "timestamp", "to", "trigger",
    -    "true", "truncate", "update", "user", "using", "values", "with", "regexp", "rlike",
    --- End diff --
    
    Nit: keep it unchanged.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support table column aliases in FROM ...

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

    https://github.com/apache/spark/pull/18079
  
    **[Test build #77485 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77485/testReport)** for PR 18079 at commit [`25415ed`](https://github.com/apache/spark/commit/25415edb07a099cb963a3b4ecba7cae660c2d4fe).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support column aliases for catalog ta...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18079: [SPARK-20841][SQL] Support column aliases for cat...

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

    https://github.com/apache/spark/pull/18079#discussion_r118833888
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala ---
    @@ -36,8 +36,17 @@ class UnresolvedException[TreeType <: TreeNode[_]](tree: TreeType, function: Str
     
     /**
      * Holds the name of a relation that has yet to be looked up in a catalog.
    + * We could add alias names for columns in a relation:
    + * {{{
    + *   // Assign alias names
    + *   SELECT col1, col2 FROM testData AS t(col1, col2);
    + * }}}
      */
    -case class UnresolvedRelation(tableIdentifier: TableIdentifier) extends LeafNode {
    +case class UnresolvedRelation(
    +    tableIdentifier: TableIdentifier,
    +    outputNames: Seq[String] = Seq.empty)
    --- End diff --
    
    Please add `@param` for these two parms


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18079: [SPARK-20841][SQL] Support column aliases for cat...

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

    https://github.com/apache/spark/pull/18079#discussion_r118134084
  
    --- Diff: sql/core/src/test/resources/sql-tests/results/operators.sql.out ---
    @@ -1,5 +1,5 @@
     -- Automatically generated by SQLQueryTestSuite
    --- Number of queries: 38
    +-- Number of queries: 48
    --- End diff --
    
    This fix is not related to this pr though, this is not updated in https://github.com/apache/spark/commit/3c9eef35a85fd841efb4491b299345a2edf30e87


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support table column aliases in FROM ...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support column aliases for catalog ta...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support table column aliases in FROM ...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support table column aliases in FROM ...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support table column aliases in FROM ...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18079: [SPARK-20841][SQL] Support column aliases for cat...

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

    https://github.com/apache/spark/pull/18079#discussion_r118396728
  
    --- Diff: sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 ---
    @@ -472,13 +472,17 @@ identifierComment
         ;
     
     relationPrimary
    -    : tableIdentifier sample? (AS? strictIdentifier)?      #tableName
    +    : catalogTable                                         #tableName
    --- End diff --
    
    Ah, ok.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support column aliases for catalog ta...

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

    https://github.com/apache/spark/pull/18079
  
    ping


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support column aliases for catalog ta...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support table column aliases in FROM ...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18079: [SPARK-20841][SQL] Support table column aliases i...

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

    https://github.com/apache/spark/pull/18079#discussion_r118842688
  
    --- Diff: sql/core/src/test/resources/sql-tests/inputs/aliases.sql ---
    @@ -0,0 +1,17 @@
    +-- Test data.
    --- End diff --
    
    How about we name this file `table-aliases.sql`; that seems a little bit less confusing.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18079: [SPARK-20841][SQL] Support table column aliases i...

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

    https://github.com/apache/spark/pull/18079#discussion_r118842186
  
    --- Diff: sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 ---
    @@ -711,7 +711,7 @@ nonReserved
         | ADD
         | OVER | PARTITION | RANGE | ROWS | PRECEDING | FOLLOWING | CURRENT | ROW | LAST | FIRST | AFTER
         | MAP | ARRAY | STRUCT
    -    | LATERAL | WINDOW | REDUCE | TRANSFORM | USING | SERDE | SERDEPROPERTIES | RECORDREADER
    --- End diff --
    
    I have added as much to the non-reserved keyword list as possible (without creating ambiguities). The reason for this is that many datasources (for instance twitter4j) unfortunately use reserved keywords for column names, and working with these was quite cumbersome. I took the pragmatic approach.
    
    If we want to change this, then we need to do the same Hive did and create a config flag. We remove them for Spark 3.0...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18079: [SPARK-20841][SQL] Support column aliases for cat...

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

    https://github.com/apache/spark/pull/18079#discussion_r118833766
  
    --- Diff: sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 ---
    @@ -711,7 +711,7 @@ nonReserved
         | ADD
         | OVER | PARTITION | RANGE | ROWS | PRECEDING | FOLLOWING | CURRENT | ROW | LAST | FIRST | AFTER
         | MAP | ARRAY | STRUCT
    -    | LATERAL | WINDOW | REDUCE | TRANSFORM | USING | SERDE | SERDEPROPERTIES | RECORDREADER
    --- End diff --
    
    First, based on the link http://developer.mimer.se/validator/sql-reserved-words.tml, `USING` is a reserved word in SQL standard since SQL-92. 
    
    Second, since 1.2, Hive introduces a flag `hive.support.sql11.reserved.keywords` for backward compatbility, which defaults to `true`. 
    > Added In: Hive 1.2.0 with HIVE-6617: https://issues.apache.org/jira/browse/HIVE-6617
    > Whether to enable support for SQL2011 reserved keywords. When enabled, will support (part of) SQL2011 reserved keywords.
    
    In 2.2, Hive removes this flag and does not allow users to change it to false. That means, users are unable to use these reserved words as identifiers anymore, unless using them as quoted identifiers. See: https://issues.apache.org/jira/browse/HIVE-14872
    
    Thus, I think it is safe to remove `USING` from the non-reserved words. 
    
    cc @hvanhovell 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support column aliases for catalog ta...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18079: [SPARK-20841][SQL] Support column aliases for cat...

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

    https://github.com/apache/spark/pull/18079#discussion_r118833830
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala ---
    @@ -592,7 +592,23 @@ class Analyzer(
         def resolveRelation(plan: LogicalPlan): LogicalPlan = plan match {
           case u: UnresolvedRelation if !isRunningDirectlyOnFiles(u.tableIdentifier) =>
             val defaultDatabase = AnalysisContext.get.defaultDatabase
    -        val relation = lookupTableFromCatalog(u, defaultDatabase)
    +        val foundRelation = lookupTableFromCatalog(u, defaultDatabase)
    +
    +        // If alias names assigned, add `Project` with the aliases
    +        val relation = if (u.outputNames.nonEmpty) {
    +          val outputAttrs = foundRelation.output
    +          // Checks if the number of the aliases is equal to expected one
    --- End diff --
    
    > // Checks if the number of the aliases equals to the number of columns in the table.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support table column aliases in FROM ...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support column aliases for catalog ta...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support column aliases for catalog ta...

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

    https://github.com/apache/spark/pull/18079
  
    Could you check the alias precedence of the other database?
    ```SQL
    select col1 as a, col2 as b from t1 as (c, d);
    ```
    
    Which alias should be used as the output schema? `(a, b)` or `(c, d)`?
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18079: [SPARK-20841][SQL] Support column aliases for cat...

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

    https://github.com/apache/spark/pull/18079#discussion_r118310787
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala ---
    @@ -74,13 +74,13 @@ object TPCDSQueryBenchmark {
           // per-row processing time for those cases.
           val queryRelations = scala.collection.mutable.HashSet[String]()
           spark.sql(queryString).queryExecution.logical.map {
    -        case ur @ UnresolvedRelation(t: TableIdentifier) =>
    +        case ur @ UnresolvedRelation(t: TableIdentifier, _) =>
               queryRelations.add(t.table)
             case lp: LogicalPlan =>
               lp.expressions.foreach { _ foreach {
                 case subquery: SubqueryExpression =>
                   subquery.plan.foreach {
    -                case ur @ UnresolvedRelation(t: TableIdentifier) =>
    +                case ur @ UnresolvedRelation(t: TableIdentifier, _) =>
    --- End diff --
    
    Nit: `UnresolvedRelation(t: TableIdentifier, _)`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18079: [SPARK-20841][SQL] Support column aliases for cat...

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

    https://github.com/apache/spark/pull/18079#discussion_r118396491
  
    --- Diff: sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 ---
    @@ -711,7 +715,7 @@ nonReserved
         | ADD
         | OVER | PARTITION | RANGE | ROWS | PRECEDING | FOLLOWING | CURRENT | ROW | LAST | FIRST | AFTER
         | MAP | ARRAY | STRUCT
    -    | LATERAL | WINDOW | REDUCE | TRANSFORM | USING | SERDE | SERDEPROPERTIES | RECORDREADER
    --- End diff --
    
    I found this `USING` entry caused a failure in `PlanParserSuite`;
    https://github.com/apache/spark/blob/master/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala#L336
    ```
    - joins *** FAILED ***
      == FAIL: Plans do not match ===
       'Project [*]                               'Project [*]
      !+- 'Join Inner                             +- 'Join UsingJoin(Inner,List(a, b))
          :- 'UnresolvedRelation `t`                 :- 'UnresolvedRelation `t`
      !   +- 'SubqueryAlias using                    +- 'UnresolvedRelation `u`
      !      +- 'UnresolvedRelation `u`, [a, b] (PlanTest.scala:97)
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18079: [SPARK-20841][SQL] Support column aliases for cat...

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

    https://github.com/apache/spark/pull/18079#discussion_r118834758
  
    --- Diff: sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 ---
    @@ -711,7 +711,7 @@ nonReserved
         | ADD
         | OVER | PARTITION | RANGE | ROWS | PRECEDING | FOLLOWING | CURRENT | ROW | LAST | FIRST | AFTER
         | MAP | ARRAY | STRUCT
    -    | LATERAL | WINDOW | REDUCE | TRANSFORM | USING | SERDE | SERDEPROPERTIES | RECORDREADER
    --- End diff --
    
    BTW, how did we decide these non-reserved words for Spark? It seems a lot of non-reserved words (e.g., `CUBE` and `GROUPING`) in Spark is the reserved ones in the ANSI standard...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18079: [SPARK-20841][SQL] Support table column aliases i...

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

    https://github.com/apache/spark/pull/18079#discussion_r118842345
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -676,9 +676,12 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
        * Create an aliased table reference. This is typically used in FROM clauses.
        */
       override def visitTableName(ctx: TableNameContext): LogicalPlan = withOrigin(ctx) {
    -    val table = UnresolvedRelation(visitTableIdentifier(ctx.tableIdentifier))
    -
    -    val tableWithAlias = Option(ctx.strictIdentifier).map(_.getText) match {
    +    val tableId = visitTableIdentifier(ctx.tableIdentifier)
    +    val table = Option(ctx.tableAlias.identifierList) match {
    --- End diff --
    
    ...or something like this:
    ```scala
    val outputNames = Option(ctx.tableAlias.identifierList).map(visitIdentifierList).getOrElse(Nil)
    val table = UnresolvedRelation(visitTableIdentifier(ctx.tableIdentifier), outputNames)
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18079: [SPARK-20841][SQL] Support column aliases for cat...

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

    https://github.com/apache/spark/pull/18079#discussion_r118304894
  
    --- Diff: sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 ---
    @@ -711,7 +715,7 @@ nonReserved
         | ADD
         | OVER | PARTITION | RANGE | ROWS | PRECEDING | FOLLOWING | CURRENT | ROW | LAST | FIRST | AFTER
         | MAP | ARRAY | STRUCT
    -    | LATERAL | WINDOW | REDUCE | TRANSFORM | USING | SERDE | SERDEPROPERTIES | RECORDREADER
    --- End diff --
    
    Nit: keep it untouched. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support table column aliases in FROM ...

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

    https://github.com/apache/spark/pull/18079
  
    Thanks! Merging to master.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support table column aliases in FROM ...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18079: [SPARK-20841][SQL] Support column aliases for cat...

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

    https://github.com/apache/spark/pull/18079#discussion_r118310422
  
    --- Diff: sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 ---
    @@ -472,13 +472,17 @@ identifierComment
         ;
     
     relationPrimary
    -    : tableIdentifier sample? (AS? strictIdentifier)?      #tableName
    +    : catalogTable                                         #tableName
    --- End diff --
    
    Just change it to `tableIdentifier sample? tableAlias`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support column aliases for catalog ta...

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

    https://github.com/apache/spark/pull/18079
  
    @gatorsmile oh, you joined Databricks :))) congrats!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support table column aliases in FROM ...

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

    https://github.com/apache/spark/pull/18079
  
    LGTM too! : )


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18079: [SPARK-20841][SQL] Support column aliases for cat...

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

    https://github.com/apache/spark/pull/18079#discussion_r118833821
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala ---
    @@ -592,7 +592,23 @@ class Analyzer(
         def resolveRelation(plan: LogicalPlan): LogicalPlan = plan match {
           case u: UnresolvedRelation if !isRunningDirectlyOnFiles(u.tableIdentifier) =>
             val defaultDatabase = AnalysisContext.get.defaultDatabase
    -        val relation = lookupTableFromCatalog(u, defaultDatabase)
    +        val foundRelation = lookupTableFromCatalog(u, defaultDatabase)
    +
    +        // If alias names assigned, add `Project` with the aliases
    --- End diff --
    
    Also gives a simple (SQL) example here to explain why we did this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18079: [SPARK-20841][SQL] Support column aliases for cat...

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

    https://github.com/apache/spark/pull/18079#discussion_r118310768
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala ---
    @@ -74,13 +74,13 @@ object TPCDSQueryBenchmark {
           // per-row processing time for those cases.
           val queryRelations = scala.collection.mutable.HashSet[String]()
           spark.sql(queryString).queryExecution.logical.map {
    -        case ur @ UnresolvedRelation(t: TableIdentifier) =>
    +        case ur @ UnresolvedRelation(t: TableIdentifier, _) =>
    --- End diff --
    
    Nit: `UnresolvedRelation(t: TableIdentifier, _)`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support column aliases for catalog ta...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support column aliases for catalog ta...

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

    https://github.com/apache/spark/pull/18079
  
    **[Test build #77280 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77280/testReport)** for PR 18079 at commit [`902d2a3`](https://github.com/apache/spark/commit/902d2a35740f3c3bc0c97aae56c65d9d25df3a15).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18079: [SPARK-20841][SQL] Support table column aliases i...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support column aliases for catalog ta...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support column aliases for catalog ta...

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

    https://github.com/apache/spark/pull/18079
  
    **[Test build #77270 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77270/testReport)** for PR 18079 at commit [`e2b50f3`](https://github.com/apache/spark/commit/e2b50f37063098bad1df8ee65ad331b5694a39a6).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class UnresolvedRelation(`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support table column aliases in FROM ...

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

    https://github.com/apache/spark/pull/18079
  
    **[Test build #77485 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77485/testReport)** for PR 18079 at commit [`25415ed`](https://github.com/apache/spark/commit/25415edb07a099cb963a3b4ecba7cae660c2d4fe).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support column aliases for catalog ta...

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

    https://github.com/apache/spark/pull/18079
  
    **[Test build #77272 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77272/testReport)** for PR 18079 at commit [`bb68f65`](https://github.com/apache/spark/commit/bb68f6540aba5588518e536ce14c765b80853f85).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class UnresolvedRelation(`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support table column aliases in FROM ...

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

    https://github.com/apache/spark/pull/18079
  
    **[Test build #77486 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77486/testReport)** for PR 18079 at commit [`49186ac`](https://github.com/apache/spark/commit/49186ac60175d3c5ddde33f4120864b834860ba0).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18079: [SPARK-20841][SQL] Support table column aliases i...

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

    https://github.com/apache/spark/pull/18079#discussion_r118836821
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala ---
    @@ -71,6 +84,11 @@ case class UnresolvedInlineTable(
      *   // Assign alias names
      *   select t.a from range(10) t(a);
      * }}}
    + * 
    + * @param functionName name of this table-value function
    + * @param functionArgs list of function arguments
    + * @param outputNames alias names of function output columns. If these names given, an analyzer
    + *                    adds [[Project]] to rename the output columns.
    --- End diff --
    
    This fix is not related to this pr though, I modified along with this fix: https://github.com/apache/spark/pull/18079/files#diff-b4f9cbed8a042aeb12aeceb13b39d25aR50


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18079: [SPARK-20841][SQL] Support column aliases for catalog ta...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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