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

[GitHub] spark pull request #18023: Fix SPARK-12139: REGEX Column Specification for H...

GitHub user janewangfb opened a pull request:

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

    Fix SPARK-12139: REGEX Column Specification for Hive Queries

    ## What changes were proposed in this pull request?
    Hive interprets regular expression, e.g., `(a)?+.+` in query specification. This PR enables spark to support this feature when hive.support.quoted.identifiers is set to true.
    
    ## How was this patch tested?
    
    - Add unittests in SQLQuerySuite.scala
    - Iin spark-shell tested the original failed query:
    scala> hc.sql("SELECT `(appid|ds|host|instance|offset|ts)?+.+`, IF(FB_IS_VALID_HIVE_PARTITION_VALUE(appid), appid, 'BAD_APPID'), IF(FB_IS_VALID_HIVE_PARTITION_VALUE(ts), ts, 'BAD_TS') FROM time_spent_bit_array_mobile_current WHERE ds='2017-05-14' AND instance='cc_deterministic_loader' AND ts='2017-05-14+15:00:99' limit 100").collect.foreach(println)
    
    result:
    [1.4947744605006E9,Map(delta -> 803, ip -> 84.16.234.63, ig_id -> 1928710114, hces_extra -> {"radio_type":"wifi-none","auth_flag":"unable_to_verify"}),0.0,1494774434,1.494774459676E9,WrappedArray(517867, 0),26,0,lncny1,e46e8616-9763-475a-b80f-a46094b263a6,9,188,10.20.0,4C0175EC-B421-4676-ACFF-8E1E353D53E5,,57944460,null,6f72336f74c9f85c6e1b7b16c64e9dec,,567067343352427,2017-05-14+15:00:99]
    ....


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

    $ git pull https://github.com/janewangfb/spark support_select_regex

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

    https://github.com/apache/spark/pull/18023.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 #18023
    
----
commit af55afd8d6839e38337f67e19a614ea3eae9a2cf
Author: Jane Wang <ja...@fb.com>
Date:   2017-05-18T00:21:14Z

    Fix SPARK-12139: REGEX Column Specification for Hive Queries

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #79535 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79535/testReport)** for PR 18023 at commit [`a5f9c44`](https://github.com/apache/spark/commit/a5f9c44875b67d9b6590a8cef6778667388fc162).
     * 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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77063/
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117368022
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1230,24 +1230,49 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    --- End diff --
    
    Please use a guard, e.g.: `case unresolved_attr @ UnresolvedAttribute(nameParts) if conf.supportQuotedIdentifiers => `. That makes the logic down the line much simpler.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117398110
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala ---
    @@ -84,6 +84,33 @@ case class UnresolvedTableValuedFunction(
     }
     
     /**
    + * Represents all of the input attributes to a given relational operator, for example in
    + * "SELECT `(id)?+.+` FROM ...".
    + *
    + * @param table an optional table that should be the target of the expansion.  If omitted all
    + *              tables' columns are produced.
    + */
    +case class UnresolvedRegex(expr: String, table: Option[String]) extends Star with Unevaluable {
    +  override def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] = {
    +    val expandedAttributes: Seq[Attribute] = table match {
    +      // If there is no table specified, use all input attributes that match expr
    +      case None => input.output.filter(_.name.matches(expr))
    +      // If there is a table, pick out attributes that are part of this table that match expr
    +      case Some(t) => input.output.filter(_.qualifier.filter(resolver(_, t)).nonEmpty)
    --- End diff --
    
    updated.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117591611
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala ---
    @@ -244,6 +244,71 @@ class DatasetSuite extends QueryTest with SharedSQLContext {
           ("a", ClassData("a", 1)), ("b", ClassData("b", 2)), ("c", ClassData("c", 3)))
       }
     
    +  test("select 3, regex") {
    --- End diff --
    
    -> `test("REGEX column specification")`


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

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


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

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


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #78815 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/78815/testReport)** for PR 18023 at commit [`321211d`](https://github.com/apache/spark/commit/321211d39826748e0c81aa73f3873636a1018152).
     * 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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r125728877
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -847,6 +847,12 @@ object SQLConf {
           .intConf
           .createWithDefault(UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD.toInt)
     
    +  val SUPPORT_QUOTED_REGEX_COLUMN_NAME = buildConf("spark.sql.parser.quotedRegexColumnNames")
    +    .doc("When true, quoted Identifiers (using backticks) in SELECT statement are interpreted" +
    --- End diff --
    
    we should only support select. It does not make sense to do select a from test where `(a)?+.+`=3.
    
    Also, for hive (https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Select), it only supports the select statements:
    "REGEX Column Specification
    A SELECT statement can take regex-based column specification in Hive releases prior to 0.13.0, or in 0.13.0 and later releases if the configuration property hive.support.quoted.identifiers is set to none." 


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #79003 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79003/testReport)** for PR 18023 at commit [`4e36ed9`](https://github.com/apache/spark/commit/4e36ed903973dcf637348825b5726892f2c13f77).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class UnresolvedRegex(regexPattern: String, table: Option[String], caseSensitive: Boolean)`


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r124973097
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala ---
    @@ -307,6 +311,28 @@ case class UnresolvedStar(target: Option[Seq[String]]) extends Star with Unevalu
     }
     
     /**
    + * Represents all of the input attributes to a given relational operator, for example in
    + * "SELECT `(id)?+.+` FROM ...".
    + *
    + * @param table an optional table that should be the target of the expansion.  If omitted all
    + *              tables' columns are produced.
    + */
    +case class UnresolvedRegex(regexPattern: String, table: Option[String])
    +  extends Star with Unevaluable {
    +  override def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] = {
    +    table match {
    +      // If there is no table specified, use all input attributes that match expr
    +      case None => input.output.filter(_.name.matches(s"(?i)$regexPattern"))
    --- End diff --
    
    Updated the code with conf caseSensitiveAnalysis


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117402527
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala ---
    @@ -177,6 +177,18 @@ object ParserUtils {
         sb.toString()
       }
     
    +  val escapedIdentifier = "`(.+)`".r
    --- End diff --
    
    Please add a comment for 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 issue #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77443/
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #78955 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/78955/testReport)** for PR 18023 at commit [`4e36ed9`](https://github.com/apache/spark/commit/4e36ed903973dcf637348825b5726892f2c13f77).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class UnresolvedRegex(regexPattern: String, table: Option[String], caseSensitive: Boolean)`


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #77099 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77099/testReport)** for PR 18023 at commit [`979bfb6`](https://github.com/apache/spark/commit/979bfb61b10f808fb751a6867ca297e0a9e69bad).
     * 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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117399885
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1230,24 +1230,49 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    +        if (conf.supportQuotedIdentifiers) {
    +          val escapedIdentifier = "`(.+)`".r
    +          val ret = Option(ctx.fieldName.getStart).map(_.getText match {
    +            case r@escapedIdentifier(i) =>
    +              UnresolvedRegex(i, Some(unresolved_attr.name))
    +            case _ =>
    +              UnresolvedAttribute(nameParts :+ attr)
    +          })
    +          return ret.get
    +        }
    +
             UnresolvedAttribute(nameParts :+ attr)
           case e =>
             UnresolvedExtractValue(e, Literal(attr))
         }
       }
     
       /**
    -   * Create an [[UnresolvedAttribute]] expression.
    +   * Create an [[UnresolvedAttribute]] expression or a [[UnresolvedRegex]] if it is a regex
    +   * quoted in ``
        */
       override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) {
    +    if (conf.supportQuotedIdentifiers) {
    +      val escapedIdentifier = "`(.+)`".r
    +      val ret = Option(ctx.getStart).map(_.getText match {
    --- End diff --
    
    removed the 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 issue #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #79240 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79240/testReport)** for PR 18023 at commit [`65886cd`](https://github.com/apache/spark/commit/65886cd18c82cc2e5401a5abcc91d06a9765b457).


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117807843
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala ---
    @@ -84,6 +84,28 @@ case class UnresolvedTableValuedFunction(
     }
     
     /**
    + * Represents all of the input attributes to a given relational operator, for example in
    + * "SELECT `(id)?+.+` FROM ...".
    + *
    + * @param table an optional table that should be the target of the expansion.  If omitted all
    + *              tables' columns are produced.
    + */
    +case class UnresolvedRegex(regexPattern: String, table: Option[String])
    +  extends Star with Unevaluable {
    +  override def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] = {
    +    table match {
    +      // If there is no table specified, use all input attributes that match expr
    +      case None => input.output.filter(_.name.matches(regexPattern))
    +      // If there is a table, pick out attributes that are part of this table that match expr
    --- End diff --
    
    ```
    * @param target an optional name that should be the target of the expansion.  If omitted all
      *              targets' columns are produced. This can either be a table name or struct name. This
      *              is a list of identifiers that is the path of the expansion.
    ```
    
    shall we support ```record.`(id)?+.+` ```?


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r124455888
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -1189,8 +1189,24 @@ class Dataset[T] private[sql](
         case "*" =>
           Column(ResolvedStar(queryExecution.analyzed.output))
         case _ =>
    -      val expr = resolve(colName)
    -      Column(expr)
    +      if (sqlContext.conf.supportQuotedRegexColumnName) {
    +        colRegex(colName)
    +      } else {
    +        val expr = resolve(colName)
    +        Column(expr)
    +      }
    +  }
    +
    +  /**
    +   * Selects column based on the column name specified as a regex and return it as [[Column]].
    --- End diff --
    
    added


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117539904
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -795,6 +795,12 @@ object SQLConf {
           .intConf
           .createWithDefault(UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD.toInt)
     
    +  val SUPPORT_QUOTED_REGEX_COLUMN_NAME = buildConf("spark.sql.parser.quotedRegexColumnNames")
    +    .internal()
    --- End diff --
    
    should be public


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117402094
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -795,6 +795,12 @@ object SQLConf {
           .intConf
           .createWithDefault(UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD.toInt)
     
    +  val SUPPORT_QUOTED_IDENTIFIERS = buildConf("spark.sql.support.quoted.identifiers")
    +    .internal()
    +    .doc("When true, identifiers specified by regex patterns will be expanded.")
    --- End diff --
    
    We only do it for the column names, right?


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #77442 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77442/testReport)** for PR 18023 at commit [`779724d`](https://github.com/apache/spark/commit/779724daae609976d6893a4aec7198bbb4f90095).


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    @janewangfb If we turn on the flag `spark.sql.parser.quotedRegexColumnNames` by default, the following test cases failed. Could you do some investigations? Thanks! 
    ```
    org.apache.spark.sql.SQLQuerySuite
    org.apache.spark.sql.SingleLevelAggregateHashMapSuite
    org.apache.spark.sql.SQLQueryTestSuite
    org.apache.spark.sql.sources.TableScanSuite
    org.apache.spark.sql.DataFrameAggregateSuite
    org.apache.spark.sql.DataFrameSuite
    org.apache.spark.sql.DataFrameNaFunctionsSuite
    org.apache.spark.sql.TwoLevelAggregateHashMapSuite
    org.apache.spark.sql.DatasetSuite
    org.apache.spark.sql.execution.datasources.json.JsonSuite
    org.apache.spark.sql.DataFrameStatSuite
    org.apache.spark.sql.TwoLevelAggregateHashMapWithVectorizedMapSuite
    org.apache.spark.sql.execution.datasources.parquet.ParquetFilterSuite
    ```


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117403590
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala ---
    @@ -177,6 +177,18 @@ object ParserUtils {
         sb.toString()
       }
     
    +  val escapedIdentifier = "`(.+)`".r
    --- End diff --
    
    added.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    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 pull request #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r126749439
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1260,26 +1260,51 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
         CaseWhen(branches, Option(ctx.elseExpression).map(expression))
       }
     
    +  private def canApplyRegex(ctx: ParserRuleContext): Boolean = withOrigin(ctx) {
    +    var parent = ctx.getParent
    +    while (parent != null) {
    +      if (parent.isInstanceOf[NamedExpressionContext]) return true
    +      parent = parent.getParent
    +    }
    +    return false
    +  }
    +
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    -        UnresolvedAttribute(nameParts :+ attr)
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    +        ctx.fieldName.getStart.getText match {
    +          case escapedIdentifier(columnNameRegex)
    +            if SQLConf.get.supportQuotedRegexColumnName && canApplyRegex(ctx) =>
    --- End diff --
    
    updated


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117690933
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1230,25 +1230,37 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    -        UnresolvedAttribute(nameParts :+ attr)
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    +        ctx.fieldName.getStart.getText match {
    +          case escapedIdentifier(columnNameRegex) if conf.supportQuotedRegexColumnName =>
    +            UnresolvedRegex(columnNameRegex, Some(unresolved_attr.name))
    +          case _ =>
    +            UnresolvedAttribute(nameParts :+ attr)
    +        }
           case e =>
             UnresolvedExtractValue(e, Literal(attr))
         }
       }
     
       /**
    -   * Create an [[UnresolvedAttribute]] expression.
    +   * Create an [[UnresolvedAttribute]] expression or a [[UnresolvedRegex]] if it is a regex
    --- End diff --
    
    what if we always create `UnresolvedRegex`?


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

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


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #78773 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/78773/testReport)** for PR 18023 at commit [`f98207b`](https://github.com/apache/spark/commit/f98207bbbca3641a20832da4983ce7e443560d9a).
     * 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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r125147853
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -847,6 +847,11 @@ object SQLConf {
           .intConf
           .createWithDefault(UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD.toInt)
     
    +  val SUPPORT_QUOTED_REGEX_COLUMN_NAME = buildConf("spark.sql.parser.quotedRegexColumnNames")
    +    .doc("When true, a SELECT statement can take regex-based column specification.")
    --- End diff --
    
    updated


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r126749797
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1260,26 +1260,51 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
         CaseWhen(branches, Option(ctx.elseExpression).map(expression))
       }
     
    +  private def canApplyRegex(ctx: ParserRuleContext): Boolean = withOrigin(ctx) {
    +    var parent = ctx.getParent
    +    while (parent != null) {
    +      if (parent.isInstanceOf[NamedExpressionContext]) return true
    +      parent = parent.getParent
    +    }
    +    return false
    +  }
    +
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    -        UnresolvedAttribute(nameParts :+ attr)
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    +        ctx.fieldName.getStart.getText match {
    +          case escapedIdentifier(columnNameRegex)
    +            if SQLConf.get.supportQuotedRegexColumnName && canApplyRegex(ctx) =>
    +            UnresolvedRegex(columnNameRegex, Some(unresolved_attr.name),
    +              SQLConf.get.caseSensitiveAnalysis)
    +          case _ =>
    +            UnresolvedAttribute(nameParts :+ attr)
    +        }
           case e =>
             UnresolvedExtractValue(e, Literal(attr))
         }
       }
     
       /**
    -   * Create an [[UnresolvedAttribute]] expression.
    +   * Create an [[UnresolvedAttribute]] expression or a [[UnresolvedRegex]] if it is a regex
    +   * quoted in ``
        */
       override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) {
    -    UnresolvedAttribute.quoted(ctx.getText)
    +    ctx.getStart.getText match {
    +      case escapedIdentifier(columnNameRegex)
    +        if SQLConf.get.supportQuotedRegexColumnName && canApplyRegex(ctx) =>
    --- End diff --
    
    rolled back to conf


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    @gatorsmile regarding
    > Could you add some test cases in the other parts of the query? For example, group by clauses.
    
    I added some tests for aggreation. But for group by, we cannot have regex there. group by requires the fields to be orderable. As hive states and our previous comments, we should only support regex in SELECT.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r126753014
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala ---
    @@ -20,14 +20,17 @@ package org.apache.spark.sql.sources
     import org.apache.spark.rdd.RDD
     import org.apache.spark.sql._
     import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.internal.SQLConf
     import org.apache.spark.sql.types._
     import org.apache.spark.unsafe.types.UTF8String
     
     private[sql] abstract class DataSourceTest extends QueryTest {
     
    -  protected def sqlTest(sqlString: String, expectedAnswer: Seq[Row]) {
    +  protected def sqlTest(sqlString: String, expectedAnswer: Seq[Row], enableRegex: String = "true") {
    --- End diff --
    
    updated


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r126271044
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -847,6 +847,12 @@ object SQLConf {
           .intConf
           .createWithDefault(UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD.toInt)
     
    +  val SUPPORT_QUOTED_REGEX_COLUMN_NAME = buildConf("spark.sql.parser.quotedRegexColumnNames")
    +    .doc("When true, quoted Identifiers (using backticks) in SELECT statement are interpreted" +
    --- End diff --
    
    @janewangfb You can do something like:
    
        scala> val df = Seq((1, 2), (2, 3), (3, 4)).toDF("a", "b")
        df: org.apache.spark.sql.DataFrame = [a: int, b: int]
        scala> df.groupBy("a", "b").agg(df.col("*")).show
        +---+---+---+---+
        |  a|  b|  a|  b|
        +---+---+---+---+
        |  2|  3|  2|  3|
        |  1|  2|  1|  2|
        |  3|  4|  3|  4|
        +---+---+---+---+
    
    So I guess you can also do something like:
    
        scala> df.groupBy("a", "b").agg(df.colRegex("`...`"))
    



---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r118159701
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala ---
    @@ -177,6 +177,12 @@ object ParserUtils {
         sb.toString()
       }
     
    +  /** the column name pattern in quoted regex without qualifier */
    +  val escapedIdentifier = "`(.+)`".r
    +
    +  /** the column name pattern in quoted regex with qualifier */
    +  val qualifiedEscapedIdentifier = ("(.+)" + """.""" + "`(.+)`").r
    --- End diff --
    
    when the config is on, we need to extract XYZ from `XYZ` pattern, thats why we need these patterns.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r118794998
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1230,25 +1230,37 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    -        UnresolvedAttribute(nameParts :+ attr)
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    --- End diff --
    
    yes, ctx.fieldName.getText will trim the backquote


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r124455687
  
    --- Diff: sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql ---
    @@ -0,0 +1,24 @@
    +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES
    +(1, "1", "11"), (2, "2", "22"), (3, "3", "33"), (4, "4", "44"), (5, "5", "55"), (6, "6", "66")
    +AS testData(key, value1, value2);
    +
    +CREATE OR REPLACE TEMPORARY VIEW testData2 AS SELECT * FROM VALUES
    +(1, 1, 1, 2), (1, 2, 1, 2), (2, 1, 2, 3), (2, 2, 2, 3), (3, 1, 3, 4), (3, 2, 3, 4)
    +AS testData2(a, b, c, d);
    +
    +-- AnalysisException
    +SELECT `(a)?+.+` FROM testData2 WHERE a = 1;
    +SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1;
    +SELECT `(a|b)` FROM testData2 WHERE a = 2;
    +SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2;
    +
    +set spark.sql.parser.quotedRegexColumnNames=true;
    +
    +-- Regex columns
    +SELECT `(a)?+.+` FROM testData2 WHERE a = 1;
    +SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1;
    +SELECT `(a|b)` FROM testData2 WHERE a = 2;
    +SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2;
    +SELECT `(e|f)` FROM testData2;
    +SELECT t.`(e|f)` FROM testData2 t;
    +SELECT p.`(key)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3;
    --- End diff --
    
    added


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79504/
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #79476 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79476/testReport)** for PR 18023 at commit [`8adad7c`](https://github.com/apache/spark/commit/8adad7cedbedaa68418851f728cc022fecb5c324).
     * This patch **fails SparkR unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging `
      * `class CatalystSqlParser(conf: SQLConf) extends AbstractSqlParser `
      * `class SparkSqlParser(conf: SQLConf) extends AbstractSqlParser `
      * `class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) `
      * `class VariableSubstitution(conf: SQLConf) `


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    @gatorsmile regarding:
    > Could you add some test cases in the other parts of the query? For example, group by clauses.
    Yes, added.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r126749650
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1260,26 +1260,51 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
         CaseWhen(branches, Option(ctx.elseExpression).map(expression))
       }
     
    +  private def canApplyRegex(ctx: ParserRuleContext): Boolean = withOrigin(ctx) {
    +    var parent = ctx.getParent
    +    while (parent != null) {
    +      if (parent.isInstanceOf[NamedExpressionContext]) return true
    +      parent = parent.getParent
    +    }
    +    return false
    +  }
    +
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    -        UnresolvedAttribute(nameParts :+ attr)
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    +        ctx.fieldName.getStart.getText match {
    +          case escapedIdentifier(columnNameRegex)
    +            if SQLConf.get.supportQuotedRegexColumnName && canApplyRegex(ctx) =>
    +            UnresolvedRegex(columnNameRegex, Some(unresolved_attr.name),
    +              SQLConf.get.caseSensitiveAnalysis)
    +          case _ =>
    +            UnresolvedAttribute(nameParts :+ attr)
    +        }
           case e =>
             UnresolvedExtractValue(e, Literal(attr))
         }
       }
     
       /**
    -   * Create an [[UnresolvedAttribute]] expression.
    +   * Create an [[UnresolvedAttribute]] expression or a [[UnresolvedRegex]] if it is a regex
    +   * quoted in ``
        */
       override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) {
    -    UnresolvedAttribute.quoted(ctx.getText)
    +    ctx.getStart.getText match {
    +      case escapedIdentifier(columnNameRegex)
    +        if SQLConf.get.supportQuotedRegexColumnName && canApplyRegex(ctx) =>
    +        UnresolvedRegex(columnNameRegex, None, SQLConf.get.caseSensitiveAnalysis)
    --- End diff --
    
    rolled back to conf.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/78955/
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #78815 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/78815/testReport)** for PR 18023 at commit [`321211d`](https://github.com/apache/spark/commit/321211d39826748e0c81aa73f3873636a1018152).


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77081/
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Could you please add another case when no column is matched into `query_regex_column.sql`? Thanks!


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r123901447
  
    --- Diff: sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out ---
    @@ -0,0 +1,113 @@
    +-- Automatically generated by SQLQueryTestSuite
    +-- Number of queries: 12
    +
    +
    +-- !query 0
    +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES
    +(1, "1", "11"), (2, "2", "22"), (3, "3", "33"), (4, "4", "44"), (5, "5", "55"), (6, "6", "66")
    +AS testData(key, value1, value2)
    +-- !query 0 schema
    +struct<>
    +-- !query 0 output
    +
    +
    +
    +-- !query 1
    +CREATE OR REPLACE TEMPORARY VIEW testData2 AS SELECT * FROM VALUES
    +(1, 1, 1, 2), (1, 2, 1, 2), (2, 1, 2, 3), (2, 2, 2, 3), (3, 1, 3, 4), (3, 2, 3, 4)
    +AS testData2(a, b, c, d)
    +-- !query 1 schema
    +struct<>
    +-- !query 1 output
    +
    +
    +
    +-- !query 2
    +SELECT `(a)?+.+` FROM testData2 WHERE a = 1
    +-- !query 2 schema
    +struct<>
    +-- !query 2 output
    +org.apache.spark.sql.AnalysisException
    +cannot resolve '```(a)?+.+```' given input columns: [a, b, c, d]; line 1 pos 7
    --- End diff --
    
    The error message is confusing. Three backquote marks are being used. Could you please improve it?


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

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


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r126262758
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -847,6 +847,12 @@ object SQLConf {
           .intConf
           .createWithDefault(UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD.toInt)
     
    +  val SUPPORT_QUOTED_REGEX_COLUMN_NAME = buildConf("spark.sql.parser.quotedRegexColumnNames")
    +    .doc("When true, quoted Identifiers (using backticks) in SELECT statement are interpreted" +
    --- End diff --
    
    @viirya I tried it out, e.g.,  
    
    val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS()
    ds.groupByKey(_._1).agg(sum("*").as[Long]) is not supported. 


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    @cloud-fan, there are difference between `xyz` and xyz. we always need to extract xyz part from `xyz`, so we will need to pattern match. if we do not get pattern matched, we know it is not regex, then why bother to use UnsolvedRegex, which will need to project list expansion later.
    
    Hive supper regex column specification, see https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Select.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r126084902
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1256,26 +1256,51 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
         CaseWhen(branches, Option(ctx.elseExpression).map(expression))
       }
     
    +  private def isContextNamedExpression(ctx: ParserRuleContext): Boolean = withOrigin(ctx) {
    +    var parent = ctx.getParent
    +    while (parent != null) {
    +      if (parent.isInstanceOf[NamedExpressionContext]) return true
    +      parent = parent.getParent
    +    }
    +    return false
    +  }
    +
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    -        UnresolvedAttribute(nameParts :+ attr)
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    +        ctx.fieldName.getStart.getText match {
    +          case escapedIdentifier(columnNameRegex)
    +            if SQLConf.get.supportQuotedRegexColumnName && isContextNamedExpression(ctx) =>
    --- End diff --
    
    Is it required to have named expression context?


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #79239 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79239/testReport)** for PR 18023 at commit [`65e5eec`](https://github.com/apache/spark/commit/65e5eec5c2d281f0b213c89c74d5333851cd3867).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging `
      * `        throw new AnalysisException(s\"UDF class $`
      * `        throw new AnalysisException(s\"It is invalid to implement multiple UDF interfaces, UDF class $`
      * `              throw new AnalysisException(s\"UDF class with $`
      * `            throw new AnalysisException(s\"Can not instantiate class $`
      * `      case e: ClassNotFoundException => throw new AnalysisException(s\"Can not load class $`
      * `        throw new AnalysisException(s\"class $className doesn't implement interface UserDefinedAggregateFunction\")`
      * `      case e: ClassNotFoundException => throw new AnalysisException(s\"Can not load class $`
      * `        throw new AnalysisException(s\"Can not instantiate class $`
      * `class SparkSqlParser extends AbstractSqlParser `
      * `class SparkSqlAstBuilder extends AstBuilder `
      * `class VariableSubstitution `


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

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


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117399718
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1230,24 +1230,49 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    +        if (conf.supportQuotedIdentifiers) {
    +          val escapedIdentifier = "`(.+)`".r
    +          val ret = Option(ctx.fieldName.getStart).map(_.getText match {
    --- End diff --
    
    removed the 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 issue #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77192/
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #77548 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77548/testReport)** for PR 18023 at commit [`a0e3773`](https://github.com/apache/spark/commit/a0e3773b67dfd20b0b1c340741ab174969823a6e).
     * 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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77548/
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #77086 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77086/testReport)** for PR 18023 at commit [`bee07cd`](https://github.com/apache/spark/commit/bee07cd569f60d4b6bfe33bf64dd99e00255cd19).
     * 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 pull request #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117402461
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -795,6 +795,12 @@ object SQLConf {
           .intConf
           .createWithDefault(UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD.toInt)
     
    +  val SUPPORT_QUOTED_IDENTIFIERS = buildConf("spark.sql.support.quoted.identifiers")
    +    .internal()
    +    .doc("When true, identifiers specified by regex patterns will be expanded.")
    --- End diff --
    
    It must be quoted. Thus, we also need to mention it in the description. 


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117366828
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala ---
    @@ -84,6 +84,33 @@ case class UnresolvedTableValuedFunction(
     }
     
     /**
    + * Represents all of the input attributes to a given relational operator, for example in
    + * "SELECT `(id)?+.+` FROM ...".
    + *
    + * @param table an optional table that should be the target of the expansion.  If omitted all
    + *              tables' columns are produced.
    + */
    +case class UnresolvedRegex(expr: String, table: Option[String]) extends Star with Unevaluable {
    +  override def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] = {
    +    val expandedAttributes: Seq[Attribute] = table match {
    +      // If there is no table specified, use all input attributes that match expr
    +      case None => input.output.filter(_.name.matches(expr))
    +      // If there is a table, pick out attributes that are part of this table that match expr
    +      case Some(t) => input.output.filter(_.qualifier.filter(resolver(_, t)).nonEmpty)
    +        .filter(_.name.matches(expr))
    +    }
    +
    +    expandedAttributes.zip(input.output).map {
    --- End diff --
    
    An `Attribute` is always a `NamedExpression`, why do we need 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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r118145333
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1230,25 +1230,37 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    -        UnresolvedAttribute(nameParts :+ attr)
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    --- End diff --
    
    how about
    ```
    case u @ UnresolvedAttribute(nameParts) if nameParts.length == 1 && conf.supportQuotedRegexColumnName =>
      UnresolvedRegex(ctx.fieldName.getStart.getText, Some(nameParts.head))
    
    // If there are more dereferences, turn `UnresolvedRegex` back to `UnresolvedAttribute`
    case UnresolvedRegex(regex, table) =>
      UnresolvedAttribute(table.toSeq + regex)
    ```


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    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 pull request #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r126749474
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1260,26 +1260,51 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
         CaseWhen(branches, Option(ctx.elseExpression).map(expression))
       }
     
    +  private def canApplyRegex(ctx: ParserRuleContext): Boolean = withOrigin(ctx) {
    +    var parent = ctx.getParent
    +    while (parent != null) {
    +      if (parent.isInstanceOf[NamedExpressionContext]) return true
    +      parent = parent.getParent
    +    }
    +    return false
    +  }
    +
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    -        UnresolvedAttribute(nameParts :+ attr)
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    +        ctx.fieldName.getStart.getText match {
    +          case escapedIdentifier(columnNameRegex)
    +            if SQLConf.get.supportQuotedRegexColumnName && canApplyRegex(ctx) =>
    +            UnresolvedRegex(columnNameRegex, Some(unresolved_attr.name),
    +              SQLConf.get.caseSensitiveAnalysis)
    --- End diff --
    
    rolled back to conf.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r124456024
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala ---
    @@ -123,7 +124,14 @@ case class UnresolvedAttribute(nameParts: Seq[String]) extends Attribute with Un
     
       override def toString: String = s"'$name"
     
    -  override def sql: String = quoteIdentifier(name)
    +  override def sql: String = {
    +    name match {
    +      case ParserUtils.escapedIdentifier(_) |
    +           ParserUtils.qualifiedEscapedIdentifier(_, _) => name
    --- End diff --
    
    shortened


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

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


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #77192 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77192/testReport)** for PR 18023 at commit [`0284d01`](https://github.com/apache/spark/commit/0284d01f24cbe2e5362530aa744e2d4e06e9d1e5).


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r126606904
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1260,26 +1260,51 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
         CaseWhen(branches, Option(ctx.elseExpression).map(expression))
       }
     
    +  private def canApplyRegex(ctx: ParserRuleContext): Boolean = withOrigin(ctx) {
    +    var parent = ctx.getParent
    +    while (parent != null) {
    +      if (parent.isInstanceOf[NamedExpressionContext]) return true
    +      parent = parent.getParent
    +    }
    +    return false
    +  }
    +
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    -        UnresolvedAttribute(nameParts :+ attr)
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    +        ctx.fieldName.getStart.getText match {
    +          case escapedIdentifier(columnNameRegex)
    +            if SQLConf.get.supportQuotedRegexColumnName && canApplyRegex(ctx) =>
    +            UnresolvedRegex(columnNameRegex, Some(unresolved_attr.name),
    +              SQLConf.get.caseSensitiveAnalysis)
    +          case _ =>
    +            UnresolvedAttribute(nameParts :+ attr)
    +        }
           case e =>
             UnresolvedExtractValue(e, Literal(attr))
         }
       }
     
       /**
    -   * Create an [[UnresolvedAttribute]] expression.
    +   * Create an [[UnresolvedAttribute]] expression or a [[UnresolvedRegex]] if it is a regex
    +   * quoted in ``
        */
       override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) {
    -    UnresolvedAttribute.quoted(ctx.getText)
    +    ctx.getStart.getText match {
    +      case escapedIdentifier(columnNameRegex)
    +        if SQLConf.get.supportQuotedRegexColumnName && canApplyRegex(ctx) =>
    --- End diff --
    
    The same here


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Left a comment https://github.com/apache/spark/pull/18023#discussion_r126079978


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Could you add some test cases in the other parts of the query? For example, group by clauses.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117591864
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala ---
    @@ -244,6 +244,71 @@ class DatasetSuite extends QueryTest with SharedSQLContext {
           ("a", ClassData("a", 1)), ("b", ClassData("b", 2)), ("c", ClassData("c", 3)))
       }
     
    +  test("select 3, regex") {
    --- End diff --
    
    updated.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r126262446
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -1188,8 +1188,29 @@ class Dataset[T] private[sql](
         case "*" =>
           Column(ResolvedStar(queryExecution.analyzed.output))
         case _ =>
    -      val expr = resolve(colName)
    -      Column(expr)
    +      if (sqlContext.conf.supportQuotedRegexColumnName) {
    +        colRegex(colName)
    +      } else {
    +        val expr = resolve(colName)
    +        Column(expr)
    +      }
    +  }
    +
    +  /**
    +   * Selects column based on the column name specified as a regex and return it as [[Column]].
    +   * @group untypedrel
    +   * @since 2.3.0
    +   */
    +  def colRegex(colName: String): Column = {
    --- End diff --
    
     I have tested out. it works for both cases. and I have added testcase DatasetSuite.scala.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r119194989
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1230,25 +1230,37 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    -        UnresolvedAttribute(nameParts :+ attr)
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    +        ctx.fieldName.getStart.getText match {
    +          case escapedIdentifier(columnNameRegex) if conf.supportQuotedRegexColumnName =>
    +            UnresolvedRegex(columnNameRegex, Some(unresolved_attr.name))
    +          case _ =>
    +            UnresolvedAttribute(nameParts :+ attr)
    +        }
           case e =>
             UnresolvedExtractValue(e, Literal(attr))
         }
       }
     
       /**
    -   * Create an [[UnresolvedAttribute]] expression.
    +   * Create an [[UnresolvedAttribute]] expression or a [[UnresolvedRegex]] if it is a regex
    --- End diff --
    
    @cloud-fan, the code path is shared by by both select a, select a.b and where cause. If it is select a.b, the table part also go here and it will not be right. I rolled back to the code before last friday (June 23rd, 2017).
    
    Do you have any suggestion? Currently Hive only supports select column regex expansion. and this PR matches the hive behavior.s
    



---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117398452
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala ---
    @@ -84,6 +84,33 @@ case class UnresolvedTableValuedFunction(
     }
     
     /**
    + * Represents all of the input attributes to a given relational operator, for example in
    + * "SELECT `(id)?+.+` FROM ...".
    + *
    + * @param table an optional table that should be the target of the expansion.  If omitted all
    + *              tables' columns are produced.
    + */
    +case class UnresolvedRegex(expr: String, table: Option[String]) extends Star with Unevaluable {
    +  override def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] = {
    +    val expandedAttributes: Seq[Attribute] = table match {
    +      // If there is no table specified, use all input attributes that match expr
    +      case None => input.output.filter(_.name.matches(expr))
    +      // If there is a table, pick out attributes that are part of this table that match expr
    +      case Some(t) => input.output.filter(_.qualifier.filter(resolver(_, t)).nonEmpty)
    +        .filter(_.name.matches(expr))
    +    }
    +
    +    expandedAttributes.zip(input.output).map {
    --- End diff --
    
    you are right. we dont need it any more. removed.


---
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 #18023: Fix SPARK-12139: REGEX Column Specification for Hive Que...

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

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


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r123901251
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -795,6 +795,11 @@ object SQLConf {
           .intConf
           .createWithDefault(UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD.toInt)
     
    +  val SUPPORT_QUOTED_REGEX_COLUMN_NAME = buildConf("spark.sql.parser.quotedRegexColumnNames")
    +    .doc("When true, column names specified by quoted regex pattern will be expanded.")
    --- End diff --
    
    Please also update the description here.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117588977
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -1188,6 +1188,11 @@ class Dataset[T] private[sql](
       def col(colName: String): Column = colName match {
         case "*" =>
           Column(ResolvedStar(queryExecution.analyzed.output))
    +    case ParserUtils.escapedIdentifier(i) if sqlContext.conf.supportQuotedRegexColumnName =>
    +      Column(UnresolvedRegex(i, None))
    +    case ParserUtils.qualifiedEscapedIdentifier(i, j)
    +      if sqlContext.conf.supportQuotedRegexColumnName =>
    --- End diff --
    
    updated.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r126517322
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -847,6 +847,12 @@ object SQLConf {
           .intConf
           .createWithDefault(UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD.toInt)
     
    +  val SUPPORT_QUOTED_REGEX_COLUMN_NAME = buildConf("spark.sql.parser.quotedRegexColumnNames")
    +    .doc("When true, quoted Identifiers (using backticks) in SELECT statement are interpreted" +
    --- End diff --
    
    yes, df.groupBy("a", "b").agg(df.col("`(a)?+.+`")).show works 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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117808657
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1230,25 +1230,37 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    -        UnresolvedAttribute(nameParts :+ attr)
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    +        ctx.fieldName.getStart.getText match {
    +          case escapedIdentifier(columnNameRegex) if conf.supportQuotedRegexColumnName =>
    +            UnresolvedRegex(columnNameRegex, Some(unresolved_attr.name))
    +          case _ =>
    +            UnresolvedAttribute(nameParts :+ attr)
    +        }
           case e =>
             UnresolvedExtractValue(e, Literal(attr))
         }
       }
     
       /**
    -   * Create an [[UnresolvedAttribute]] expression.
    +   * Create an [[UnresolvedAttribute]] expression or a [[UnresolvedRegex]] if it is a regex
    --- End diff --
    
    there seems no problem if we always go with the `UnresolvedRegex`, then we can simplify the code and remove the logic to detect regex string.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117397712
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala ---
    @@ -84,6 +84,33 @@ case class UnresolvedTableValuedFunction(
     }
     
     /**
    + * Represents all of the input attributes to a given relational operator, for example in
    + * "SELECT `(id)?+.+` FROM ...".
    + *
    + * @param table an optional table that should be the target of the expansion.  If omitted all
    + *              tables' columns are produced.
    + */
    +case class UnresolvedRegex(expr: String, table: Option[String]) extends Star with Unevaluable {
    --- End diff --
    
    renamed to pattern


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117522248
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala ---
    @@ -2624,4 +2624,92 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
         val e = intercept[AnalysisException](sql("SELECT nvl(1, 2, 3)"))
         assert(e.message.contains("Invalid number of arguments"))
       }
    +
    +  test("SPARK-12139: REGEX Column Specification for Hive Queries") {
    --- End diff --
    
    Could you create a file in `https://github.com/apache/spark/tree/master/sql/core/src/test/resources/sql-tests/inputs`? Now, all the new SQL test cases need to be moved there.
    
    You can run `SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/test-only *SQLQueryTestSuite"` to generate the result files. Thanks!


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r118133059
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala ---
    @@ -84,6 +84,28 @@ case class UnresolvedTableValuedFunction(
     }
     
     /**
    + * Represents all of the input attributes to a given relational operator, for example in
    + * "SELECT `(id)?+.+` FROM ...".
    + *
    + * @param table an optional table that should be the target of the expansion.  If omitted all
    + *              tables' columns are produced.
    + */
    +case class UnresolvedRegex(regexPattern: String, table: Option[String])
    +  extends Star with Unevaluable {
    +  override def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] = {
    +    table match {
    +      // If there is no table specified, use all input attributes that match expr
    +      case None => input.output.filter(_.name.matches(regexPattern))
    +      // If there is a table, pick out attributes that are part of this table that match expr
    --- End diff --
    
    for this diff, we support column only.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117367232
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala ---
    @@ -84,6 +84,33 @@ case class UnresolvedTableValuedFunction(
     }
     
     /**
    + * Represents all of the input attributes to a given relational operator, for example in
    + * "SELECT `(id)?+.+` FROM ...".
    + *
    + * @param table an optional table that should be the target of the expansion.  If omitted all
    + *              tables' columns are produced.
    + */
    +case class UnresolvedRegex(expr: String, table: Option[String]) extends Star with Unevaluable {
    +  override def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] = {
    +    val expandedAttributes: Seq[Attribute] = table match {
    +      // If there is no table specified, use all input attributes that match expr
    +      case None => input.output.filter(_.name.matches(expr))
    +      // If there is a table, pick out attributes that are part of this table that match expr
    +      case Some(t) => input.output.filter(_.qualifier.filter(resolver(_, t)).nonEmpty)
    --- End diff --
    
    `input.output.filter(_.qualifier.exists(resolver(_, t)))` is a bit more concise.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Ping. I went through the individual reviewers' comment, it seems I addressed every one. Not sure if I am missing anyone, please let me know. Thanks! 


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117586747
  
    --- Diff: sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql ---
    @@ -0,0 +1,24 @@
    +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES
    +(1, "1"), (2, "2"), (3, "3"), (4, "4"), (5, "5"), (6, "6")
    +AS testData(key, value);
    +
    +CREATE OR REPLACE TEMPORARY VIEW testData2 AS SELECT * FROM VALUES
    +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2)
    +AS testData2(a, b);
    --- End diff --
    
    Since the test cases are testing the regex pattern matching in column names, could you add more names and let the regex pattern match more columns?


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117797005
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala ---
    @@ -84,6 +84,28 @@ case class UnresolvedTableValuedFunction(
     }
     
     /**
    + * Represents all of the input attributes to a given relational operator, for example in
    + * "SELECT `(id)?+.+` FROM ...".
    + *
    + * @param table an optional table that should be the target of the expansion.  If omitted all
    + *              tables' columns are produced.
    + */
    +case class UnresolvedRegex(regexPattern: String, table: Option[String])
    +  extends Star with Unevaluable {
    +  override def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] = {
    +    table match {
    +      // If there is no table specified, use all input attributes that match expr
    +      case None => input.output.filter(_.name.matches(regexPattern))
    +      // If there is a table, pick out attributes that are part of this table that match expr
    --- End diff --
    
    this regex is only for column names. 


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117587285
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1230,25 +1230,37 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    -        UnresolvedAttribute(nameParts :+ attr)
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    +        ctx.fieldName.getStart.getText match {
    +          case escapedIdentifier(i) if conf.supportQuotedRegexColumnName =>
    --- End diff --
    
    The same here


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117690219
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala ---
    @@ -84,6 +84,28 @@ case class UnresolvedTableValuedFunction(
     }
     
     /**
    + * Represents all of the input attributes to a given relational operator, for example in
    + * "SELECT `(id)?+.+` FROM ...".
    + *
    + * @param table an optional table that should be the target of the expansion.  If omitted all
    + *              tables' columns are produced.
    + */
    +case class UnresolvedRegex(regexPattern: String, table: Option[String])
    +  extends Star with Unevaluable {
    +  override def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] = {
    +    table match {
    +      // If there is no table specified, use all input attributes that match expr
    +      case None => input.output.filter(_.name.matches(regexPattern))
    +      // If there is a table, pick out attributes that are part of this table that match expr
    --- End diff --
    
    shall we consider "struct expansion" like what we did in `UnresolvedStar`?


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #77443 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77443/testReport)** for PR 18023 at commit [`a27023c`](https://github.com/apache/spark/commit/a27023c69ace6026a345dc745206c20e71ac4299).
     * This patch **fails Spark unit 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 #18023: Fix SPARK-12139: REGEX Column Specification for Hive Que...

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

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


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r126606843
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1260,26 +1260,51 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
         CaseWhen(branches, Option(ctx.elseExpression).map(expression))
       }
     
    +  private def canApplyRegex(ctx: ParserRuleContext): Boolean = withOrigin(ctx) {
    +    var parent = ctx.getParent
    +    while (parent != null) {
    +      if (parent.isInstanceOf[NamedExpressionContext]) return true
    +      parent = parent.getParent
    +    }
    +    return false
    +  }
    +
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    -        UnresolvedAttribute(nameParts :+ attr)
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    +        ctx.fieldName.getStart.getText match {
    +          case escapedIdentifier(columnNameRegex)
    +            if SQLConf.get.supportQuotedRegexColumnName && canApplyRegex(ctx) =>
    --- End diff --
    
    Sorry, recently, we reverted a PR back. In the parser, we are unable to use `SQLConf.get`.
    
    Could you please change `SQLConf.get` back to `conf`?


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r118197805
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1230,25 +1230,37 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    -        UnresolvedAttribute(nameParts :+ attr)
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    --- End diff --
    
    oh sorry I made a mistake, `ctx.fieldName.getText` will trim the backquote?


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77086/
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r124099738
  
    --- Diff: sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out ---
    @@ -0,0 +1,113 @@
    +-- Automatically generated by SQLQueryTestSuite
    +-- Number of queries: 12
    +
    +
    +-- !query 0
    +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES
    +(1, "1", "11"), (2, "2", "22"), (3, "3", "33"), (4, "4", "44"), (5, "5", "55"), (6, "6", "66")
    +AS testData(key, value1, value2)
    +-- !query 0 schema
    +struct<>
    +-- !query 0 output
    +
    +
    +
    +-- !query 1
    +CREATE OR REPLACE TEMPORARY VIEW testData2 AS SELECT * FROM VALUES
    +(1, 1, 1, 2), (1, 2, 1, 2), (2, 1, 2, 3), (2, 2, 2, 3), (3, 1, 3, 4), (3, 2, 3, 4)
    +AS testData2(a, b, c, d)
    +-- !query 1 schema
    +struct<>
    +-- !query 1 output
    +
    +
    +
    +-- !query 2
    +SELECT `(a)?+.+` FROM testData2 WHERE a = 1
    +-- !query 2 schema
    +struct<>
    +-- !query 2 output
    +org.apache.spark.sql.AnalysisException
    +cannot resolve '```(a)?+.+```' given input columns: [a, b, c, d]; line 1 pos 7
    --- End diff --
    
    fixed the three backquote to single quote


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r126247287
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -847,6 +847,12 @@ object SQLConf {
           .intConf
           .createWithDefault(UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD.toInt)
     
    +  val SUPPORT_QUOTED_REGEX_COLUMN_NAME = buildConf("spark.sql.parser.quotedRegexColumnNames")
    +    .doc("When true, quoted Identifiers (using backticks) in SELECT statement are interpreted" +
    --- End diff --
    
    @gatorsmile for quoted Identifiers; if it not in select, it would not be affected.
    however, if it is in project/aggregation, it will be affected, e.g., before `(a)?+.+` is invalid, now, it will be expanded.
    



---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Do we still care `hive.support.quoted.identifiers`? If no, please update the PR description accordingly.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r126079978
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -847,6 +847,12 @@ object SQLConf {
           .intConf
           .createWithDefault(UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD.toInt)
     
    +  val SUPPORT_QUOTED_REGEX_COLUMN_NAME = buildConf("spark.sql.parser.quotedRegexColumnNames")
    +    .doc("When true, quoted Identifiers (using backticks) in SELECT statement are interpreted" +
    --- End diff --
    
    I agree. It only makes sense when we use it in SELECT statement. However, our parser allows `quoted Identifiers (using backticks)` in any part of the SQL statement. Below is the just the example. If we turn on this conf flag, will it cause the problem for the other users when they have quoted identifiers in the query except Project/SELECT list?


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r126606893
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1260,26 +1260,51 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
         CaseWhen(branches, Option(ctx.elseExpression).map(expression))
       }
     
    +  private def canApplyRegex(ctx: ParserRuleContext): Boolean = withOrigin(ctx) {
    +    var parent = ctx.getParent
    +    while (parent != null) {
    +      if (parent.isInstanceOf[NamedExpressionContext]) return true
    +      parent = parent.getParent
    +    }
    +    return false
    +  }
    +
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    -        UnresolvedAttribute(nameParts :+ attr)
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    +        ctx.fieldName.getStart.getText match {
    +          case escapedIdentifier(columnNameRegex)
    +            if SQLConf.get.supportQuotedRegexColumnName && canApplyRegex(ctx) =>
    +            UnresolvedRegex(columnNameRegex, Some(unresolved_attr.name),
    +              SQLConf.get.caseSensitiveAnalysis)
    +          case _ =>
    +            UnresolvedAttribute(nameParts :+ attr)
    +        }
           case e =>
             UnresolvedExtractValue(e, Literal(attr))
         }
       }
     
       /**
    -   * Create an [[UnresolvedAttribute]] expression.
    +   * Create an [[UnresolvedAttribute]] expression or a [[UnresolvedRegex]] if it is a regex
    +   * quoted in ``
        */
       override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) {
    -    UnresolvedAttribute.quoted(ctx.getText)
    +    ctx.getStart.getText match {
    +      case escapedIdentifier(columnNameRegex)
    +        if SQLConf.get.supportQuotedRegexColumnName && canApplyRegex(ctx) =>
    +        UnresolvedRegex(columnNameRegex, None, SQLConf.get.caseSensitiveAnalysis)
    --- End diff --
    
    The same here


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117367155
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala ---
    @@ -84,6 +84,33 @@ case class UnresolvedTableValuedFunction(
     }
     
     /**
    + * Represents all of the input attributes to a given relational operator, for example in
    + * "SELECT `(id)?+.+` FROM ...".
    + *
    + * @param table an optional table that should be the target of the expansion.  If omitted all
    + *              tables' columns are produced.
    + */
    +case class UnresolvedRegex(expr: String, table: Option[String]) extends Star with Unevaluable {
    --- End diff --
    
    `expr` is the pattern right? Maybe we should give it a better name.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117587207
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -1188,6 +1188,11 @@ class Dataset[T] private[sql](
       def col(colName: String): Column = colName match {
         case "*" =>
           Column(ResolvedStar(queryExecution.analyzed.output))
    +    case ParserUtils.escapedIdentifier(i) if sqlContext.conf.supportQuotedRegexColumnName =>
    +      Column(UnresolvedRegex(i, None))
    +    case ParserUtils.qualifiedEscapedIdentifier(i, j)
    +      if sqlContext.conf.supportQuotedRegexColumnName =>
    --- End diff --
    
    Nit: style issue. We prefer to add the extra two space before `if` in this case.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

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


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    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 pull request #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r118749165
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1230,25 +1230,37 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    -        UnresolvedAttribute(nameParts :+ attr)
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    +        ctx.fieldName.getStart.getText match {
    +          case escapedIdentifier(columnNameRegex) if conf.supportQuotedRegexColumnName =>
    +            UnresolvedRegex(columnNameRegex, Some(unresolved_attr.name))
    +          case _ =>
    +            UnresolvedAttribute(nameParts :+ attr)
    +        }
           case e =>
             UnresolvedExtractValue(e, Literal(attr))
         }
       }
     
       /**
    -   * Create an [[UnresolvedAttribute]] expression.
    +   * Create an [[UnresolvedAttribute]] expression or a [[UnresolvedRegex]] if it is a regex
    --- End diff --
    
    we cannot avoid detecting regex string. but the string passed in is `xyz`, we need to match with the regex pattern to extract xyz part.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #77099 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77099/testReport)** for PR 18023 at commit [`979bfb6`](https://github.com/apache/spark/commit/979bfb61b10f808fb751a6867ca297e0a9e69bad).


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    @gatorsmile, regarding:
    > @janewangfb If we turn on the flag spark.sql.parser.quotedRegexColumnNames by default, the > following test cases failed. Could you do some investigations? Thanks!
    > 
    > org.apache.spark.sql.SQLQuerySuite
    the struct type was not supported now in the regex
    some special characters has different meaning in regex.
    
    > org.apache.spark.sql.DataFrameSuite
    some special characters has different meaning in regex.
    
    > org.apache.spark.sql.SingleLevelAggregateHashMapSuite
    > org.apache.spark.sql.DataFrameAggregateSuite
    > org.apache.spark.sql.TwoLevelAggregateHashMapSuite
    > org.apache.spark.sql.TwoLevelAggregateHashMapWithVectorizedMapSuite
    > org.apache.spark.sql.DataFrameNaFunctionsSuite
    > org.apache.spark.sql.DataFrameStatSuite
    These four failed for the same testcase. in AS alias, regex is not allowed. 
    
    > org.apache.spark.sql.SQLQueryTestSuite
    This suite has the same behavior wether spark.sql.parser.quotedRegexColumnNames default value is true/false. 
    
    > org.apache.spark.sql.execution.datasources.json.JsonSuite
    for map struct, regex should not be allowed in A[B] part.
    
    > org.apache.spark.sql.DatasetSuite
    Expected. Explicitly set the spark.sql.parser.quotedRegexColumnNames = false to false for those tests.
    
    > org.apache.spark.sql.sources.TableScanSuite
    some special characters has different meaning in regex
    
    > org.apache.spark.sql.execution.datasources.parquet.ParquetFilterSuite
     regex is not allowed in where.
    
    



---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79476/
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    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 pull request #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r124401692
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -1189,8 +1189,24 @@ class Dataset[T] private[sql](
         case "*" =>
           Column(ResolvedStar(queryExecution.analyzed.output))
         case _ =>
    -      val expr = resolve(colName)
    -      Column(expr)
    +      if (sqlContext.conf.supportQuotedRegexColumnName) {
    +        colRegex(colName)
    +      } else {
    +        val expr = resolve(colName)
    +        Column(expr)
    +      }
    +  }
    +
    +  /**
    +   * Selects column based on the column name specified as a regex and return it as [[Column]].
    --- End diff --
    
    Please add `@group untypedrel` and `@since 2.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 issue #18023: Fix SPARK-12139: REGEX Column Specification for Hive Que...

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

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


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Yes. We need to consider it.
    
    > (?i) makes the regex case insensitive.



---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r124099510
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -795,6 +795,11 @@ object SQLConf {
           .intConf
           .createWithDefault(UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD.toInt)
     
    +  val SUPPORT_QUOTED_REGEX_COLUMN_NAME = buildConf("spark.sql.parser.quotedRegexColumnNames")
    +    .doc("When true, column names specified by quoted regex pattern will be expanded.")
    --- End diff --
    
    Updated the description.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #79503 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79503/testReport)** for PR 18023 at commit [`56e2b83`](https://github.com/apache/spark/commit/56e2b83670b209c68c2a6ced0934d60e3f6973af).


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79003/
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117795667
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala ---
    @@ -84,6 +84,28 @@ case class UnresolvedTableValuedFunction(
     }
     
     /**
    + * Represents all of the input attributes to a given relational operator, for example in
    + * "SELECT `(id)?+.+` FROM ...".
    + *
    + * @param table an optional table that should be the target of the expansion.  If omitted all
    + *              tables' columns are produced.
    + */
    +case class UnresolvedRegex(regexPattern: String, table: Option[String])
    +  extends Star with Unevaluable {
    +  override def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] = {
    +    table match {
    +      // If there is no table specified, use all input attributes that match expr
    +      case None => input.output.filter(_.name.matches(regexPattern))
    +      // If there is a table, pick out attributes that are part of this table that match expr
    +      case Some(t) => input.output.filter(_.qualifier.exists(resolver(_, t)))
    +        .filter(_.name.matches(regexPattern))
    +    }
    +  }
    +
    +  override def toString: String = table.map(_ + ".").getOrElse("") + regexPattern
    --- End diff --
    
    updated.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r126751775
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1260,26 +1260,51 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
         CaseWhen(branches, Option(ctx.elseExpression).map(expression))
       }
     
    +  private def canApplyRegex(ctx: ParserRuleContext): Boolean = withOrigin(ctx) {
    --- End diff --
    
    added


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117540055
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala ---
    @@ -2624,4 +2624,92 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
         val e = intercept[AnalysisException](sql("SELECT nvl(1, 2, 3)"))
         assert(e.message.contains("Invalid number of arguments"))
       }
    +
    +  test("SPARK-12139: REGEX Column Specification for Hive Queries") {
    --- End diff --
    
    Yes let's use those rather than adding more files to SQLQuerySUite. I'd love to get rid of SQLQuerySuite ....



---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #79238 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79238/testReport)** for PR 18023 at commit [`d65c462`](https://github.com/apache/spark/commit/d65c462040c07b2eea537b717f6ff28f7f02b031).
     * This patch **fails to build**.
     * 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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/78773/
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117380037
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1230,24 +1230,49 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    +        if (conf.supportQuotedIdentifiers) {
    +          val escapedIdentifier = "`(.+)`".r
    +          val ret = Option(ctx.fieldName.getStart).map(_.getText match {
    --- End diff --
    
    Using an option here does not add a thing.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Jenkins test this please


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #77081 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77081/testReport)** for PR 18023 at commit [`6e37517`](https://github.com/apache/spark/commit/6e375177e68a216cdd53de1e5d600d898b2b59d5).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class UnresolvedRegex(regexPattern: String, table: Option[String])`


---
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 #18023: Fix SPARK-12139: REGEX Column Specification for H...

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

    https://github.com/apache/spark/pull/18023#discussion_r117326089
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1229,25 +1228,56 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
         CaseWhen(branches, Option(ctx.elseExpression).map(expression))
       }
     
    +  def enableHiveSupportQuotedIdentifiers() : Boolean = {
    +    SparkEnv.get != null &&
    +      SparkEnv.get.conf != null &&
    +      SparkEnv.get.conf.getBoolean("hive.support.quoted.identifiers", false)
    --- End diff --
    
    Spark SQL always supports quoted identifiers. However, the missing part is the `REGEX Column Specification`. How about adding such a conf to `SQLConf`?


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r118806698
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -1188,6 +1188,12 @@ class Dataset[T] private[sql](
       def col(colName: String): Column = colName match {
         case "*" =>
           Column(ResolvedStar(queryExecution.analyzed.output))
    +    case ParserUtils.escapedIdentifier(columnNameRegex)
    --- End diff --
    
    @cloud-fan I might have misunderstood your last comment "I don't think sql a-like syntax is really useful here. How about we create a special cased col function that takes a regex?". Can you clarify with some examples? Thanks.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117379878
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1230,24 +1230,49 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    +        if (conf.supportQuotedIdentifiers) {
    +          val escapedIdentifier = "`(.+)`".r
    +          val ret = Option(ctx.fieldName.getStart).map(_.getText match {
    +            case r@escapedIdentifier(i) =>
    +              UnresolvedRegex(i, Some(unresolved_attr.name))
    +            case _ =>
    +              UnresolvedAttribute(nameParts :+ attr)
    +          })
    +          return ret.get
    +        }
    +
             UnresolvedAttribute(nameParts :+ attr)
           case e =>
             UnresolvedExtractValue(e, Literal(attr))
         }
       }
     
       /**
    -   * Create an [[UnresolvedAttribute]] expression.
    +   * Create an [[UnresolvedAttribute]] expression or a [[UnresolvedRegex]] if it is a regex
    +   * quoted in ``
        */
       override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) {
    +    if (conf.supportQuotedIdentifiers) {
    +      val escapedIdentifier = "`(.+)`".r
    --- End diff --
    
    We don't need to compile the same regex over and over. Can you move this to the ParserUtils...


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

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


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #77442 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77442/testReport)** for PR 18023 at commit [`779724d`](https://github.com/apache/spark/commit/779724daae609976d6893a4aec7198bbb4f90095).
     * This patch **fails to build**.
     * 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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77068/
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    @janewangfb That is fine we do not support, but we still need to add test cases for these negative cases. Thanks!


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #77109 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77109/testReport)** for PR 18023 at commit [`48c54aa`](https://github.com/apache/spark/commit/48c54aad51d38c99c5fe39de9d382d2f56f42314).
     * 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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117380055
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1230,24 +1230,49 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    +        if (conf.supportQuotedIdentifiers) {
    +          val escapedIdentifier = "`(.+)`".r
    +          val ret = Option(ctx.fieldName.getStart).map(_.getText match {
    +            case r@escapedIdentifier(i) =>
    +              UnresolvedRegex(i, Some(unresolved_attr.name))
    +            case _ =>
    +              UnresolvedAttribute(nameParts :+ attr)
    +          })
    +          return ret.get
    +        }
    +
             UnresolvedAttribute(nameParts :+ attr)
           case e =>
             UnresolvedExtractValue(e, Literal(attr))
         }
       }
     
       /**
    -   * Create an [[UnresolvedAttribute]] expression.
    +   * Create an [[UnresolvedAttribute]] expression or a [[UnresolvedRegex]] if it is a regex
    +   * quoted in ``
        */
       override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) {
    +    if (conf.supportQuotedIdentifiers) {
    +      val escapedIdentifier = "`(.+)`".r
    +      val ret = Option(ctx.getStart).map(_.getText match {
    --- End diff --
    
    Using an option here does not add a thing.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    @viirya regarding
    > Do we still care hive.support.quoted.identifiers? If no, please update the PR description accordingly.
    
    Yes, we still have hive.support.quoted.identifiers. only when spark.sql.parser.quotedRegexColumnNames = true, `...` in SELECT statements will be treated in regex as in Hive


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #79504 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79504/testReport)** for PR 18023 at commit [`d613ff9`](https://github.com/apache/spark/commit/d613ff90596652d7e22859c0684dbfe3602344e0).
     * This patch **fails due to an unknown error code, -9**.
     * 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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r124099358
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -1191,6 +1191,12 @@ class Dataset[T] private[sql](
       def col(colName: String): Column = colName match {
         case "*" =>
           Column(ResolvedStar(queryExecution.analyzed.output))
    +    case ParserUtils.escapedIdentifier(columnNameRegex)
    --- End diff --
    
    refactored the code with a new function colRegex().


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117813681
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -1188,6 +1188,12 @@ class Dataset[T] private[sql](
       def col(colName: String): Column = colName match {
         case "*" =>
           Column(ResolvedStar(queryExecution.analyzed.output))
    +    case ParserUtils.escapedIdentifier(columnNameRegex)
    --- End diff --
    
    I don't think sql a-like syntax is really useful here. How about we create a special cased `col` function that takes a regex?


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117521173
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -795,6 +795,12 @@ object SQLConf {
           .intConf
           .createWithDefault(UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD.toInt)
     
    +  val SUPPORT_QUOTED_REGEX_COLUMN_NAME = buildConf("spark.sql.parser.quotedRegexColumnNames")
    +    .internal()
    --- End diff --
    
    @rxin @hvanhovell @cloud-fan Should we keep it internal?


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r126256818
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1256,26 +1256,51 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
         CaseWhen(branches, Option(ctx.elseExpression).map(expression))
       }
     
    +  private def isContextNamedExpression(ctx: ParserRuleContext): Boolean = withOrigin(ctx) {
    +    var parent = ctx.getParent
    +    while (parent != null) {
    +      if (parent.isInstanceOf[NamedExpressionContext]) return true
    +      parent = parent.getParent
    +    }
    +    return false
    +  }
    +
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    -        UnresolvedAttribute(nameParts :+ attr)
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    +        ctx.fieldName.getStart.getText match {
    +          case escapedIdentifier(columnNameRegex)
    +            if SQLConf.get.supportQuotedRegexColumnName && isContextNamedExpression(ctx) =>
    --- End diff --
    
    I think we should limit it to named expression; otherwise, other place like where will expand the regex, which does not make sense (see my comment to @gatorsmile), e.g., where `(a)?+.+` = 1 


---
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 #18023: Fix SPARK-12139: REGEX Column Specification for Hive Que...

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

    https://github.com/apache/spark/pull/18023
  
    Please update the PR title to `[SPARK-12139] [SQL] REGEX Column Specification`


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Will review it this weekend. Thanks for your work!


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    LGTM pending Jenkins
    
    cc @hvanhovell @cloud-fan 


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79240/
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    @gatorsmile regarding:
    > Could you revert back all the unneeded changes? (in JsonSuite.scala). 
    
    (I saw this comment in email but didn't find in the PR) I have reverted the unneeded changes.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r118129729
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1230,25 +1230,37 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    -        UnresolvedAttribute(nameParts :+ attr)
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    +        ctx.fieldName.getStart.getText match {
    +          case escapedIdentifier(columnNameRegex) if conf.supportQuotedRegexColumnName =>
    +            UnresolvedRegex(columnNameRegex, Some(unresolved_attr.name))
    +          case _ =>
    +            UnresolvedAttribute(nameParts :+ attr)
    +        }
           case e =>
             UnresolvedExtractValue(e, Literal(attr))
         }
       }
     
       /**
    -   * Create an [[UnresolvedAttribute]] expression.
    +   * Create an [[UnresolvedAttribute]] expression or a [[UnresolvedRegex]] if it is a regex
    --- End diff --
    
    the code complexity will be similar, because if the column is ``, we need to extract the pattern; 


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r126606864
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1260,26 +1260,51 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
         CaseWhen(branches, Option(ctx.elseExpression).map(expression))
       }
     
    +  private def canApplyRegex(ctx: ParserRuleContext): Boolean = withOrigin(ctx) {
    +    var parent = ctx.getParent
    +    while (parent != null) {
    +      if (parent.isInstanceOf[NamedExpressionContext]) return true
    +      parent = parent.getParent
    +    }
    +    return false
    +  }
    +
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    -        UnresolvedAttribute(nameParts :+ attr)
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    +        ctx.fieldName.getStart.getText match {
    +          case escapedIdentifier(columnNameRegex)
    +            if SQLConf.get.supportQuotedRegexColumnName && canApplyRegex(ctx) =>
    +            UnresolvedRegex(columnNameRegex, Some(unresolved_attr.name),
    +              SQLConf.get.caseSensitiveAnalysis)
    --- End diff --
    
    The same here.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77442/
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

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


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79535/
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r124964462
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala ---
    @@ -307,6 +311,28 @@ case class UnresolvedStar(target: Option[Seq[String]]) extends Star with Unevalu
     }
     
     /**
    + * Represents all of the input attributes to a given relational operator, for example in
    + * "SELECT `(id)?+.+` FROM ...".
    + *
    + * @param table an optional table that should be the target of the expansion.  If omitted all
    + *              tables' columns are produced.
    + */
    +case class UnresolvedRegex(regexPattern: String, table: Option[String])
    +  extends Star with Unevaluable {
    +  override def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] = {
    +    table match {
    +      // If there is no table specified, use all input attributes that match expr
    +      case None => input.output.filter(_.name.matches(s"(?i)$regexPattern"))
    --- End diff --
    
    You need to check the conf `sparkSession.sessionState.conf.caseSensitiveAnalysis`


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79004/
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79238/
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r126609171
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1260,26 +1260,51 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
         CaseWhen(branches, Option(ctx.elseExpression).map(expression))
       }
     
    +  private def canApplyRegex(ctx: ParserRuleContext): Boolean = withOrigin(ctx) {
    --- End diff --
    
    Please add a comment above this function to explain why `regex` can be applied under `NamedExpression` only. Thanks!


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Like what we did for `*` in `Column.scala`, we also need to handle the Dataset APIs. You can follow the way we handle star there.
    ```Scala
    df.select(df("(a|b)?+.+"))
    ```


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #77109 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77109/testReport)** for PR 18023 at commit [`48c54aa`](https://github.com/apache/spark/commit/48c54aad51d38c99c5fe39de9d382d2f56f42314).


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117367722
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1230,24 +1230,49 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    +        if (conf.supportQuotedIdentifiers) {
    +          val escapedIdentifier = "`(.+)`".r
    --- End diff --
    
    We don't need to compile the same regex over and over. Can you move this to the ParserUtils...
    
    I am also wondering if we shouldn't do the match in the parser it self.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117797457
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1230,25 +1230,37 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    -        UnresolvedAttribute(nameParts :+ attr)
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    +        ctx.fieldName.getStart.getText match {
    +          case escapedIdentifier(columnNameRegex) if conf.supportQuotedRegexColumnName =>
    +            UnresolvedRegex(columnNameRegex, Some(unresolved_attr.name))
    +          case _ =>
    +            UnresolvedAttribute(nameParts :+ attr)
    +        }
           case e =>
             UnresolvedExtractValue(e, Literal(attr))
         }
       }
     
       /**
    -   * Create an [[UnresolvedAttribute]] expression.
    +   * Create an [[UnresolvedAttribute]] expression or a [[UnresolvedRegex]] if it is a regex
    --- End diff --
    
    we should only create UnresolvedRegex when necessary.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r124456075
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala ---
    @@ -244,6 +244,71 @@ class DatasetSuite extends QueryTest with SharedSQLContext {
           ("a", ClassData("a", 1)), ("b", ClassData("b", 2)), ("c", ClassData("c", 3)))
       }
     
    +  test("REGEX column specification") {
    +    val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS()
    +
    +    intercept[AnalysisException] {
    +      ds.select(expr("`(_1)?+.+`").as[Int])
    +    }
    +
    +    intercept[AnalysisException] {
    +      ds.select(expr("`(_1|_2)`").as[Int])
    +    }
    +
    +    intercept[AnalysisException] {
    +      ds.select(ds("`(_1)?+.+`"))
    +    }
    +
    +    intercept[AnalysisException] {
    +      ds.select(ds("`(_1|_2)`"))
    +    }
    --- End diff --
    
    updated with message


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117587910
  
    --- Diff: sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql ---
    @@ -0,0 +1,24 @@
    +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES
    +(1, "1"), (2, "2"), (3, "3"), (4, "4"), (5, "5"), (6, "6")
    +AS testData(key, value);
    +
    +CREATE OR REPLACE TEMPORARY VIEW testData2 AS SELECT * FROM VALUES
    +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2)
    +AS testData2(a, b);
    +
    +-- AnalysisException
    +SELECT `(a)?+.+` FROM testData2 WHERE a = 1;
    +
    +-- AnalysisException
    +SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1;
    +
    +set spark.sql.parser.quotedRegexColumnNames=true;
    +
    +-- Regex columns
    +SELECT `(a)?+.+` FROM testData2 WHERE a = 1;
    +SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1;
    +SELECT p.`(key)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3;
    +
    +-- Clean-up
    +DROP VIEW IF EXISTS testData;
    +DROP VIEW IF EXISTS testData2;
    --- End diff --
    
    removed


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r118160405
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1230,25 +1230,37 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    -        UnresolvedAttribute(nameParts :+ attr)
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    --- End diff --
    
    this wont work.  In your first "case", ctx.fieldName.getStart.getText is `XYZ`, nameparts is XYZ. and the table part should come from ctx.base.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r125155245
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -847,6 +847,12 @@ object SQLConf {
           .intConf
           .createWithDefault(UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD.toInt)
     
    +  val SUPPORT_QUOTED_REGEX_COLUMN_NAME = buildConf("spark.sql.parser.quotedRegexColumnNames")
    +    .doc("When true, quoted Identifiers (using backticks) in SELECT statement are interpreted" +
    --- End diff --
    
    Not only select statement. It can be almost any query.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    The last comment is about `DataFrameNaFunctions.fill`. It does not work when `spark.sql.parser.quotedRegexColumnNames` is on. Could you resolve that in the follow-up PR?


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r118145607
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala ---
    @@ -84,6 +84,28 @@ case class UnresolvedTableValuedFunction(
     }
     
     /**
    + * Represents all of the input attributes to a given relational operator, for example in
    + * "SELECT `(id)?+.+` FROM ...".
    + *
    + * @param table an optional table that should be the target of the expansion.  If omitted all
    + *              tables' columns are produced.
    + */
    +case class UnresolvedRegex(regexPattern: String, table: Option[String])
    +  extends Star with Unevaluable {
    +  override def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] = {
    +    table match {
    +      // If there is no table specified, use all input attributes that match expr
    +      case None => input.output.filter(_.name.matches(regexPattern))
    +      // If there is a table, pick out attributes that are part of this table that match expr
    --- End diff --
    
    does hive support it?


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r123901234
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -1191,6 +1191,12 @@ class Dataset[T] private[sql](
       def col(colName: String): Column = colName match {
         case "*" =>
           Column(ResolvedStar(queryExecution.analyzed.output))
    +    case ParserUtils.escapedIdentifier(columnNameRegex)
    --- End diff --
    
    @janewangfb Based on the above comment from @cloud-fan and @hvanhovell , how about creating a new function `colRegex`? 
    
    `spark.sql.parser.quotedRegexColumnNames` can be defined as a SQL Parser specific configuration. That configuration will not affect the new function `colRegex`.


---
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 #18023: Fix SPARK-12139: REGEX Column Specification for Hive Que...

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

    https://github.com/apache/spark/pull/18023
  
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    @gatorsmile 
    
    Sure, I could have a follow-up PR to resolve DataFrameNaFunctions.fill. 
    
    thanks for reviewing this PR.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    LGTM 


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117586530
  
    --- Diff: sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql ---
    @@ -0,0 +1,24 @@
    +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES
    +(1, "1"), (2, "2"), (3, "3"), (4, "4"), (5, "5"), (6, "6")
    +AS testData(key, value);
    +
    +CREATE OR REPLACE TEMPORARY VIEW testData2 AS SELECT * FROM VALUES
    +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2)
    +AS testData2(a, b);
    +
    +-- AnalysisException
    +SELECT `(a)?+.+` FROM testData2 WHERE a = 1;
    +
    +-- AnalysisException
    +SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1;
    +
    +set spark.sql.parser.quotedRegexColumnNames=true;
    +
    +-- Regex columns
    +SELECT `(a)?+.+` FROM testData2 WHERE a = 1;
    +SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1;
    +SELECT p.`(key)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3;
    +
    +-- Clean-up
    +DROP VIEW IF EXISTS testData;
    +DROP VIEW IF EXISTS testData2;
    --- End diff --
    
    No need to drop the temp views.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    @gatorsmile, regarding
    > That is fine we do not support it, but we still need to add test cases for these negative cases. Thanks!
    
    Yes, I have added testcases.



---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117587254
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1230,25 +1230,37 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    -        UnresolvedAttribute(nameParts :+ attr)
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    +        ctx.fieldName.getStart.getText match {
    +          case escapedIdentifier(i) if conf.supportQuotedRegexColumnName =>
    +            UnresolvedRegex(i, Some(unresolved_attr.name))
    +          case _ =>
    +            UnresolvedAttribute(nameParts :+ attr)
    +        }
           case e =>
             UnresolvedExtractValue(e, Literal(attr))
         }
       }
     
       /**
    -   * Create an [[UnresolvedAttribute]] expression.
    +   * Create an [[UnresolvedAttribute]] expression or a [[UnresolvedRegex]] if it is a regex
    +   * quoted in ``
        */
       override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) {
    -    UnresolvedAttribute.quoted(ctx.getText)
    +    ctx.getStart.getText match {
    +      case escapedIdentifier(i) if conf.supportQuotedRegexColumnName =>
    --- End diff --
    
    The same here


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117588000
  
    --- Diff: sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql ---
    @@ -0,0 +1,24 @@
    +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES
    +(1, "1"), (2, "2"), (3, "3"), (4, "4"), (5, "5"), (6, "6")
    +AS testData(key, value);
    +
    +CREATE OR REPLACE TEMPORARY VIEW testData2 AS SELECT * FROM VALUES
    +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2)
    +AS testData2(a, b);
    --- End diff --
    
    sure. added two more columns


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

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


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

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


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r126607036
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala ---
    @@ -922,59 +922,61 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData {
       }
     
       test("Applying schemas with MapType") {
    -    val schemaWithSimpleMap = StructType(
    -      StructField("map", MapType(StringType, IntegerType, true), false) :: Nil)
    -    val jsonWithSimpleMap = spark.read.schema(schemaWithSimpleMap).json(mapType1)
    +    withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") {
    +      val schemaWithSimpleMap = StructType(
    +        StructField("map", MapType(StringType, IntegerType, true), false) :: Nil)
    +      val jsonWithSimpleMap = spark.read.schema(schemaWithSimpleMap).json(mapType1)
     
    -    jsonWithSimpleMap.createOrReplaceTempView("jsonWithSimpleMap")
    +      jsonWithSimpleMap.createOrReplaceTempView("jsonWithSimpleMap")
     
    -    checkAnswer(
    -      sql("select `map` from jsonWithSimpleMap"),
    -      Row(Map("a" -> 1)) ::
    -      Row(Map("b" -> 2)) ::
    -      Row(Map("c" -> 3)) ::
    -      Row(Map("c" -> 1, "d" -> 4)) ::
    -      Row(Map("e" -> null)) :: Nil
    -    )
    +      checkAnswer(
    +        sql("select `map` from jsonWithSimpleMap"),
    +        Row(Map("a" -> 1)) ::
    +          Row(Map("b" -> 2)) ::
    +          Row(Map("c" -> 3)) ::
    +          Row(Map("c" -> 1, "d" -> 4)) ::
    +          Row(Map("e" -> null)) :: Nil
    +      )
     
    -    checkAnswer(
    -      sql("select `map`['c'] from jsonWithSimpleMap"),
    -      Row(null) ::
    -      Row(null) ::
    -      Row(3) ::
    -      Row(1) ::
    -      Row(null) :: Nil
    -    )
    +      checkAnswer(
    +        sql("select `map`['c'] from jsonWithSimpleMap"),
    +        Row(null) ::
    +          Row(null) ::
    +          Row(3) ::
    +          Row(1) ::
    +          Row(null) :: Nil
    +      )
     
    -    val innerStruct = StructType(
    -      StructField("field1", ArrayType(IntegerType, true), true) ::
    -      StructField("field2", IntegerType, true) :: Nil)
    -    val schemaWithComplexMap = StructType(
    -      StructField("map", MapType(StringType, innerStruct, true), false) :: Nil)
    +      val innerStruct = StructType(
    +        StructField("field1", ArrayType(IntegerType, true), true) ::
    +          StructField("field2", IntegerType, true) :: Nil)
    +      val schemaWithComplexMap = StructType(
    +        StructField("map", MapType(StringType, innerStruct, true), false) :: Nil)
     
    -    val jsonWithComplexMap = spark.read.schema(schemaWithComplexMap).json(mapType2)
    +      val jsonWithComplexMap = spark.read.schema(schemaWithComplexMap).json(mapType2)
     
    -    jsonWithComplexMap.createOrReplaceTempView("jsonWithComplexMap")
    +      jsonWithComplexMap.createOrReplaceTempView("jsonWithComplexMap")
     
    -    checkAnswer(
    -      sql("select `map` from jsonWithComplexMap"),
    -      Row(Map("a" -> Row(Seq(1, 2, 3, null), null))) ::
    -      Row(Map("b" -> Row(null, 2))) ::
    -      Row(Map("c" -> Row(Seq(), 4))) ::
    -      Row(Map("c" -> Row(null, 3), "d" -> Row(Seq(null), null))) ::
    -      Row(Map("e" -> null)) ::
    -      Row(Map("f" -> Row(null, null))) :: Nil
    -    )
    +      checkAnswer(
    +        sql("select `map` from jsonWithComplexMap"),
    +        Row(Map("a" -> Row(Seq(1, 2, 3, null), null))) ::
    +          Row(Map("b" -> Row(null, 2))) ::
    +          Row(Map("c" -> Row(Seq(), 4))) ::
    +          Row(Map("c" -> Row(null, 3), "d" -> Row(Seq(null), null))) ::
    +          Row(Map("e" -> null)) ::
    +          Row(Map("f" -> Row(null, null))) :: Nil
    +      )
    --- End diff --
    
    Could you revert back all the unneeded changes?


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117533359
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala ---
    @@ -2624,4 +2624,92 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
         val e = intercept[AnalysisException](sql("SELECT nvl(1, 2, 3)"))
         assert(e.message.contains("Invalid number of arguments"))
       }
    +
    +  test("SPARK-12139: REGEX Column Specification for Hive Queries") {
    +    // hive.support.quoted.identifiers is turned off by default
    +    checkAnswer(
    +      sql(
    +        """
    +          |SELECT b
    +          |FROM testData2
    +          |WHERE a = 1
    +        """.stripMargin),
    +      Row(1) :: Row(2) :: Nil)
    +
    +    checkAnswer(
    +      sql(
    +        """
    +          |SELECT t.b
    +          |FROM testData2 t
    +          |WHERE a = 1
    +        """.stripMargin),
    +      Row(1) :: Row(2) :: Nil)
    --- End diff --
    
    removed. I was trying to make sure that the existing behaviors are not broken.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #79503 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79503/testReport)** for PR 18023 at commit [`56e2b83`](https://github.com/apache/spark/commit/56e2b83670b209c68c2a6ced0934d60e3f6973af).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class ArrowSerializer(FramedSerializer):`


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r125144859
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -847,6 +847,11 @@ object SQLConf {
           .intConf
           .createWithDefault(UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD.toInt)
     
    +  val SUPPORT_QUOTED_REGEX_COLUMN_NAME = buildConf("spark.sql.parser.quotedRegexColumnNames")
    +    .doc("When true, a SELECT statement can take regex-based column specification.")
    --- End diff --
    
    We also need to explain the impact in the description. For example,
    > When true, quoted Identifiers (using backticks) are interpreted as regular expressions. 
    
    Feel free to input the above text


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #77192 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77192/testReport)** for PR 18023 at commit [`0284d01`](https://github.com/apache/spark/commit/0284d01f24cbe2e5362530aa744e2d4e06e9d1e5).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class UnresolvedRegex(regexPattern: String, table: Option[String])`


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117545031
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala ---
    @@ -2624,4 +2624,92 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
         val e = intercept[AnalysisException](sql("SELECT nvl(1, 2, 3)"))
         assert(e.message.contains("Invalid number of arguments"))
       }
    +
    +  test("SPARK-12139: REGEX Column Specification for Hive Queries") {
    --- End diff --
    
    ok. moved the test to sql/core/src/test/resources/sql-tests/inputs


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117797804
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -1188,6 +1188,12 @@ class Dataset[T] private[sql](
       def col(colName: String): Column = colName match {
         case "*" =>
           Column(ResolvedStar(queryExecution.analyzed.output))
    +    case ParserUtils.escapedIdentifier(columnNameRegex)
    --- End diff --
    
    no. this improves the current behavior. when it is `a`, if a is the column name, it will expand to just column a. if it is `(a)?+.+`, it will be treated it as regular expression and expand. (the current behavior is to throw AnalysisException.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #78645 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/78645/testReport)** for PR 18023 at commit [`da60368`](https://github.com/apache/spark/commit/da60368d0c90d12f4ddd33aa0894615618aa01dc).
     * 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 #18023: Fix SPARK-12139: REGEX Column Specification for Hive Que...

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

    https://github.com/apache/spark/pull/18023
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77062/
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117403303
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -795,6 +795,12 @@ object SQLConf {
           .intConf
           .createWithDefault(UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD.toInt)
     
    +  val SUPPORT_QUOTED_IDENTIFIERS = buildConf("spark.sql.support.quoted.identifiers")
    --- End diff --
    
    renamed.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117795801
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala ---
    @@ -84,6 +84,28 @@ case class UnresolvedTableValuedFunction(
     }
     
     /**
    + * Represents all of the input attributes to a given relational operator, for example in
    + * "SELECT `(id)?+.+` FROM ...".
    + *
    + * @param table an optional table that should be the target of the expansion.  If omitted all
    + *              tables' columns are produced.
    + */
    +case class UnresolvedRegex(regexPattern: String, table: Option[String])
    --- End diff --
    
    moved


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117589166
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1230,25 +1230,37 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    -        UnresolvedAttribute(nameParts :+ attr)
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    +        ctx.fieldName.getStart.getText match {
    +          case escapedIdentifier(i) if conf.supportQuotedRegexColumnName =>
    +            UnresolvedRegex(i, Some(unresolved_attr.name))
    +          case _ =>
    +            UnresolvedAttribute(nameParts :+ attr)
    +        }
           case e =>
             UnresolvedExtractValue(e, Literal(attr))
         }
       }
     
       /**
    -   * Create an [[UnresolvedAttribute]] expression.
    +   * Create an [[UnresolvedAttribute]] expression or a [[UnresolvedRegex]] if it is a regex
    +   * quoted in ``
        */
       override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) {
    -    UnresolvedAttribute.quoted(ctx.getText)
    +    ctx.getStart.getText match {
    +      case escapedIdentifier(i) if conf.supportQuotedRegexColumnName =>
    --- End diff --
    
    updated.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77109/
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r118145019
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala ---
    @@ -177,6 +177,12 @@ object ParserUtils {
         sb.toString()
       }
     
    +  /** the column name pattern in quoted regex without qualifier */
    +  val escapedIdentifier = "`(.+)`".r
    +
    +  /** the column name pattern in quoted regex with qualifier */
    +  val qualifiedEscapedIdentifier = ("(.+)" + """.""" + "`(.+)`").r
    --- End diff --
    
    these 2 seems hacky to me, we can always create `UnresolvedRegex` if the config is on, and `UnresolvedAttribute` otherwise.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117689813
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala ---
    @@ -84,6 +84,28 @@ case class UnresolvedTableValuedFunction(
     }
     
     /**
    + * Represents all of the input attributes to a given relational operator, for example in
    + * "SELECT `(id)?+.+` FROM ...".
    + *
    + * @param table an optional table that should be the target of the expansion.  If omitted all
    + *              tables' columns are produced.
    + */
    +case class UnresolvedRegex(regexPattern: String, table: Option[String])
    +  extends Star with Unevaluable {
    +  override def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] = {
    +    table match {
    +      // If there is no table specified, use all input attributes that match expr
    +      case None => input.output.filter(_.name.matches(regexPattern))
    +      // If there is a table, pick out attributes that are part of this table that match expr
    +      case Some(t) => input.output.filter(_.qualifier.exists(resolver(_, t)))
    +        .filter(_.name.matches(regexPattern))
    +    }
    +  }
    +
    +  override def toString: String = table.map(_ + ".").getOrElse("") + regexPattern
    --- End diff --
    
    nit: `table.map(_ + "." + regexPattern).getOrElse(regexPattern)`


---
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 #18023: Fix SPARK-12139: REGEX Column Specification for Hive Que...

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #77062 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77062/testReport)** for PR 18023 at commit [`af55afd`](https://github.com/apache/spark/commit/af55afd8d6839e38337f67e19a614ea3eae9a2cf).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class UnresolvedRegex(expr: String, table: Option[String]) extends Star with Unevaluable `


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79503/
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

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


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r126607452
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala ---
    @@ -20,14 +20,17 @@ package org.apache.spark.sql.sources
     import org.apache.spark.rdd.RDD
     import org.apache.spark.sql._
     import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.internal.SQLConf
     import org.apache.spark.sql.types._
     import org.apache.spark.unsafe.types.UTF8String
     
     private[sql] abstract class DataSourceTest extends QueryTest {
     
    -  protected def sqlTest(sqlString: String, expectedAnswer: Seq[Row]) {
    +  protected def sqlTest(sqlString: String, expectedAnswer: Seq[Row], enableRegex: String = "true") {
    --- End diff --
    
    `enableRegex: String = "true"` -> `enableRegex: Boolean = false`
    
    Could you change the type to Boolean and call .toString below and set the default to `false`?


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    @gatorsmile, regarding:
    > Left a comment #18023 (comment)
    I have already fixed the issue.
    



---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117588878
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -1188,6 +1188,11 @@ class Dataset[T] private[sql](
       def col(colName: String): Column = colName match {
         case "*" =>
           Column(ResolvedStar(queryExecution.analyzed.output))
    +    case ParserUtils.escapedIdentifier(i) if sqlContext.conf.supportQuotedRegexColumnName =>
    --- End diff --
    
    updated


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/78645/
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117399811
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1230,24 +1230,49 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    +        if (conf.supportQuotedIdentifiers) {
    +          val escapedIdentifier = "`(.+)`".r
    --- End diff --
    
    Add API in ParserUtils. 
    
    I think in the parser, it can still get ``; after that, the `` are stripped off.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    do we need to consider case sensitivity when matching the regex?


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #79240 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79240/testReport)** for PR 18023 at commit [`65886cd`](https://github.com/apache/spark/commit/65886cd18c82cc2e5401a5abcc91d06a9765b457).
     * 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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117403331
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -795,6 +795,12 @@ object SQLConf {
           .intConf
           .createWithDefault(UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD.toInt)
     
    +  val SUPPORT_QUOTED_IDENTIFIERS = buildConf("spark.sql.support.quoted.identifiers")
    +    .internal()
    +    .doc("When true, identifiers specified by regex patterns will be expanded.")
    --- End diff --
    
    yes. this only applies to column names. updated the doc.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

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


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117399399
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1230,24 +1230,49 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    --- End diff --
    
    updated.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117402025
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -795,6 +795,12 @@ object SQLConf {
           .intConf
           .createWithDefault(UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD.toInt)
     
    +  val SUPPORT_QUOTED_IDENTIFIERS = buildConf("spark.sql.support.quoted.identifiers")
    --- End diff --
    
    How about renaming it to `spark.sql.parser.regexColumnNames`?


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117589317
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1230,25 +1230,37 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    -        UnresolvedAttribute(nameParts :+ attr)
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    +        ctx.fieldName.getStart.getText match {
    +          case escapedIdentifier(i) if conf.supportQuotedRegexColumnName =>
    --- End diff --
    
    updated


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117522441
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala ---
    @@ -2624,4 +2624,92 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
         val e = intercept[AnalysisException](sql("SELECT nvl(1, 2, 3)"))
         assert(e.message.contains("Invalid number of arguments"))
       }
    +
    +  test("SPARK-12139: REGEX Column Specification for Hive Queries") {
    +    // hive.support.quoted.identifiers is turned off by default
    +    checkAnswer(
    +      sql(
    +        """
    +          |SELECT b
    +          |FROM testData2
    +          |WHERE a = 1
    +        """.stripMargin),
    +      Row(1) :: Row(2) :: Nil)
    +
    +    checkAnswer(
    +      sql(
    +        """
    +          |SELECT t.b
    +          |FROM testData2 t
    +          |WHERE a = 1
    +        """.stripMargin),
    +      Row(1) :: Row(2) :: Nil)
    --- End diff --
    
    The above two test queries are not needed in the new suite. 


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117588953
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -1188,6 +1188,11 @@ class Dataset[T] private[sql](
       def col(colName: String): Column = colName match {
         case "*" =>
           Column(ResolvedStar(queryExecution.analyzed.output))
    +    case ParserUtils.escapedIdentifier(i) if sqlContext.conf.supportQuotedRegexColumnName =>
    +      Column(UnresolvedRegex(i, None))
    +    case ParserUtils.qualifiedEscapedIdentifier(i, j)
    --- End diff --
    
    updated.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r126087546
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -847,6 +847,12 @@ object SQLConf {
           .intConf
           .createWithDefault(UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD.toInt)
     
    +  val SUPPORT_QUOTED_REGEX_COLUMN_NAME = buildConf("spark.sql.parser.quotedRegexColumnNames")
    +    .doc("When true, quoted Identifiers (using backticks) in SELECT statement are interpreted" +
    --- End diff --
    
    Is it possible users to use regex column in agg such as `testData2.groupBy($"a", $"b").agg($"`...`")`? In analyzer, seems we process `Star` in both `Project` and `Aggregate`.
    



---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79239/
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #77108 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77108/testReport)** for PR 18023 at commit [`612bedf`](https://github.com/apache/spark/commit/612bedf9bb1181687fa536d2e927923901c19582).


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117399877
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1230,24 +1230,49 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    +        if (conf.supportQuotedIdentifiers) {
    +          val escapedIdentifier = "`(.+)`".r
    +          val ret = Option(ctx.fieldName.getStart).map(_.getText match {
    +            case r@escapedIdentifier(i) =>
    +              UnresolvedRegex(i, Some(unresolved_attr.name))
    +            case _ =>
    +              UnresolvedAttribute(nameParts :+ attr)
    +          })
    +          return ret.get
    +        }
    +
             UnresolvedAttribute(nameParts :+ attr)
           case e =>
             UnresolvedExtractValue(e, Literal(attr))
         }
       }
     
       /**
    -   * Create an [[UnresolvedAttribute]] expression.
    +   * Create an [[UnresolvedAttribute]] expression or a [[UnresolvedRegex]] if it is a regex
    +   * quoted in ``
        */
       override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) {
    +    if (conf.supportQuotedIdentifiers) {
    +      val escapedIdentifier = "`(.+)`".r
    --- End diff --
    
    Add API in ParserUtils.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77099/
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r118805375
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1230,25 +1230,37 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    -        UnresolvedAttribute(nameParts :+ attr)
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    +        ctx.fieldName.getStart.getText match {
    +          case escapedIdentifier(columnNameRegex) if conf.supportQuotedRegexColumnName =>
    +            UnresolvedRegex(columnNameRegex, Some(unresolved_attr.name))
    +          case _ =>
    +            UnresolvedAttribute(nameParts :+ attr)
    +        }
           case e =>
             UnresolvedExtractValue(e, Literal(attr))
         }
       }
     
       /**
    -   * Create an [[UnresolvedAttribute]] expression.
    +   * Create an [[UnresolvedAttribute]] expression or a [[UnresolvedRegex]] if it is a regex
    --- End diff --
    
    @cloud-fan, I updated the code such that for the column field part, always use regex if supportQuotedRegexColumnName


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #77108 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77108/testReport)** for PR 18023 at commit [`612bedf`](https://github.com/apache/spark/commit/612bedf9bb1181687fa536d2e927923901c19582).
     * 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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117691835
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -1188,6 +1188,12 @@ class Dataset[T] private[sql](
       def col(colName: String): Column = colName match {
         case "*" =>
           Column(ResolvedStar(queryExecution.analyzed.output))
    +    case ParserUtils.escapedIdentifier(columnNameRegex)
    --- End diff --
    
    this is a breaking change, previously we always treat the input string as a column name(except for star), even it's quoted. 


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #77063 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77063/testReport)** for PR 18023 at commit [`43beb07`](https://github.com/apache/spark/commit/43beb07b4f92b6cd2549129b5b5f68f55dd8e37c).
     * 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 pull request #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r118145159
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1230,25 +1230,37 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    -        UnresolvedAttribute(nameParts :+ attr)
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    +        ctx.fieldName.getStart.getText match {
    +          case escapedIdentifier(columnNameRegex) if conf.supportQuotedRegexColumnName =>
    +            UnresolvedRegex(columnNameRegex, Some(unresolved_attr.name))
    +          case _ =>
    +            UnresolvedAttribute(nameParts :+ attr)
    +        }
           case e =>
             UnresolvedExtractValue(e, Literal(attr))
         }
       }
     
       /**
    -   * Create an [[UnresolvedAttribute]] expression.
    +   * Create an [[UnresolvedAttribute]] expression or a [[UnresolvedRegex]] if it is a regex
    --- End diff --
    
    I'm not talking about algorithm complexity, but saying that we can simplify the logic by avoiding detecting the regex string.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r124415882
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala ---
    @@ -244,6 +244,71 @@ class DatasetSuite extends QueryTest with SharedSQLContext {
           ("a", ClassData("a", 1)), ("b", ClassData("b", 2)), ("c", ClassData("c", 3)))
       }
     
    +  test("REGEX column specification") {
    +    val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS()
    +
    +    intercept[AnalysisException] {
    +      ds.select(expr("`(_1)?+.+`").as[Int])
    +    }
    +
    +    intercept[AnalysisException] {
    +      ds.select(expr("`(_1|_2)`").as[Int])
    +    }
    +
    +    intercept[AnalysisException] {
    +      ds.select(ds("`(_1)?+.+`"))
    +    }
    +
    +    intercept[AnalysisException] {
    +      ds.select(ds("`(_1|_2)`"))
    +    }
    --- End diff --
    
    Could you capture the exception error messages? It can help reviewers ensure the error messages are correct. 
    
    For example
    ```Scala
        val e = intercept[AnalysisException] { ds.select(expr("`(_1)?+.+`").as[Int]) }.getMessage
        assert(e.contains("xyz"))
    ```


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117690015
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala ---
    @@ -84,6 +84,28 @@ case class UnresolvedTableValuedFunction(
     }
     
     /**
    + * Represents all of the input attributes to a given relational operator, for example in
    + * "SELECT `(id)?+.+` FROM ...".
    + *
    + * @param table an optional table that should be the target of the expansion.  If omitted all
    + *              tables' columns are produced.
    + */
    +case class UnresolvedRegex(regexPattern: String, table: Option[String])
    --- End diff --
    
    can we move it below `UnresolvedStar`?


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #79004 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79004/testReport)** for PR 18023 at commit [`448c3e2`](https://github.com/apache/spark/commit/448c3e2d200ad9530cfd43e8200afc7b7b7f1469).
     * 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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r124401574
  
    --- Diff: sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql ---
    @@ -0,0 +1,24 @@
    +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES
    +(1, "1", "11"), (2, "2", "22"), (3, "3", "33"), (4, "4", "44"), (5, "5", "55"), (6, "6", "66")
    +AS testData(key, value1, value2);
    +
    +CREATE OR REPLACE TEMPORARY VIEW testData2 AS SELECT * FROM VALUES
    +(1, 1, 1, 2), (1, 2, 1, 2), (2, 1, 2, 3), (2, 2, 2, 3), (3, 1, 3, 4), (3, 2, 3, 4)
    +AS testData2(a, b, c, d);
    +
    +-- AnalysisException
    +SELECT `(a)?+.+` FROM testData2 WHERE a = 1;
    +SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1;
    +SELECT `(a|b)` FROM testData2 WHERE a = 2;
    +SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2;
    +
    +set spark.sql.parser.quotedRegexColumnNames=true;
    +
    +-- Regex columns
    +SELECT `(a)?+.+` FROM testData2 WHERE a = 1;
    +SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1;
    +SELECT `(a|b)` FROM testData2 WHERE a = 2;
    +SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2;
    +SELECT `(e|f)` FROM testData2;
    +SELECT t.`(e|f)` FROM testData2 t;
    +SELECT p.`(key)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3;
    --- End diff --
    
    Nit: Please add a new line.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r126088785
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -1188,8 +1188,29 @@ class Dataset[T] private[sql](
         case "*" =>
           Column(ResolvedStar(queryExecution.analyzed.output))
         case _ =>
    -      val expr = resolve(colName)
    -      Column(expr)
    +      if (sqlContext.conf.supportQuotedRegexColumnName) {
    +        colRegex(colName)
    +      } else {
    +        val expr = resolve(colName)
    +        Column(expr)
    +      }
    +  }
    +
    +  /**
    +   * Selects column based on the column name specified as a regex and return it as [[Column]].
    +   * @group untypedrel
    +   * @since 2.3.0
    +   */
    +  def colRegex(colName: String): Column = {
    --- End diff --
    
    `col` returns a column resolved on the current `Dataset`. `colRegex` now can return an unresolved one. Seems ok, but any possibility to go wrong with it?
    
    For example, we can do:
    
        val colRegex1 = df1.colRegex("`...`")  // colRegex1 is unresolved.
        df2.select(colRegex1)
    
    But you can't do the same thing with `col`.



---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77108/
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117334957
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1229,25 +1228,56 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
         CaseWhen(branches, Option(ctx.elseExpression).map(expression))
       }
     
    +  def enableHiveSupportQuotedIdentifiers() : Boolean = {
    +    SparkEnv.get != null &&
    +      SparkEnv.get.conf != null &&
    +      SparkEnv.get.conf.getBoolean("hive.support.quoted.identifiers", false)
    --- End diff --
    
    Added to SQLConf


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

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


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117587075
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -1188,6 +1188,11 @@ class Dataset[T] private[sql](
       def col(colName: String): Column = colName match {
         case "*" =>
           Column(ResolvedStar(queryExecution.analyzed.output))
    +    case ParserUtils.escapedIdentifier(i) if sqlContext.conf.supportQuotedRegexColumnName =>
    --- End diff --
    
    Please avoid using `i` or `j`. Instead, using some meaningful variable names.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r118130193
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -1188,6 +1188,12 @@ class Dataset[T] private[sql](
       def col(colName: String): Column = colName match {
         case "*" =>
           Column(ResolvedStar(queryExecution.analyzed.output))
    +    case ParserUtils.escapedIdentifier(columnNameRegex)
    --- End diff --
    
    I am not sure if I understand what you said. no matter what, the colName is string, you need to figure out it is a regex or not. 


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

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


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #77068 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77068/testReport)** for PR 18023 at commit [`7699e87`](https://github.com/apache/spark/commit/7699e871a31e37755b35c88b893faf9df8f7664f).


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r117533040
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -795,6 +795,12 @@ object SQLConf {
           .intConf
           .createWithDefault(UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD.toInt)
     
    +  val SUPPORT_QUOTED_REGEX_COLUMN_NAME = buildConf("spark.sql.parser.quotedRegexColumnNames")
    +    .internal()
    --- End diff --
    
    I think it should be public. I didn't realize that that I put it under internal.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r118748843
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala ---
    @@ -84,6 +84,28 @@ case class UnresolvedTableValuedFunction(
     }
     
     /**
    + * Represents all of the input attributes to a given relational operator, for example in
    + * "SELECT `(id)?+.+` FROM ...".
    + *
    + * @param table an optional table that should be the target of the expansion.  If omitted all
    + *              tables' columns are produced.
    + */
    +case class UnresolvedRegex(regexPattern: String, table: Option[String])
    +  extends Star with Unevaluable {
    +  override def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] = {
    +    table match {
    +      // If there is no table specified, use all input attributes that match expr
    +      case None => input.output.filter(_.name.matches(regexPattern))
    +      // If there is a table, pick out attributes that are part of this table that match expr
    --- End diff --
    
    Hive supper regex column specification, see https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Select.


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/78815/
    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 #18023: Fix SPARK-12139: REGEX Column Specification for H...

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

    https://github.com/apache/spark/pull/18023#discussion_r117326828
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -1229,25 +1228,56 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
         CaseWhen(branches, Option(ctx.elseExpression).map(expression))
       }
     
    +  def enableHiveSupportQuotedIdentifiers() : Boolean = {
    +    SparkEnv.get != null &&
    +      SparkEnv.get.conf != null &&
    +      SparkEnv.get.conf.getBoolean("hive.support.quoted.identifiers", false)
    +  }
    +
       /**
    -   * Create a dereference expression. The return type depends on the type of the parent, this can
    -   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
    -   * [[UnresolvedExtractValue]] if the parent is some expression.
    +   * Create a dereference expression. The return type depends on the type of the parent.
    +   * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
    +   * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
    +   * it can be [[UnresolvedExtractValue]].
        */
       override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
         val attr = ctx.fieldName.getText
         expression(ctx.base) match {
    -      case UnresolvedAttribute(nameParts) =>
    +      case unresolved_attr @ UnresolvedAttribute(nameParts) =>
    +        if (enableHiveSupportQuotedIdentifiers) {
    +          val escapedIdentifier = "`(.+)`".r
    +          val ret = Option(ctx.fieldName.getStart).map(_.getText match {
    +            case r@escapedIdentifier(i) =>
    +              UnresolvedRegex(i, Some(unresolved_attr.name))
    --- End diff --
    
    How about no change in the parser?
    
    Is that possible we can simply resolve it in `ResolveReferences`? 
    
    BTW, we also need to handle the same issue in the Dataset APIs. 


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #77068 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77068/testReport)** for PR 18023 at commit [`7699e87`](https://github.com/apache/spark/commit/7699e871a31e37755b35c88b893faf9df8f7664f).
     * 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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023
  
    **[Test build #79004 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79004/testReport)** for PR 18023 at commit [`448c3e2`](https://github.com/apache/spark/commit/448c3e2d200ad9530cfd43e8200afc7b7b7f1469).


---
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 #18023: [SPARK-12139] [SQL] REGEX Column Specification

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

    https://github.com/apache/spark/pull/18023#discussion_r124414653
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala ---
    @@ -123,7 +124,14 @@ case class UnresolvedAttribute(nameParts: Seq[String]) extends Attribute with Un
     
       override def toString: String = s"'$name"
     
    -  override def sql: String = quoteIdentifier(name)
    +  override def sql: String = {
    +    name match {
    +      case ParserUtils.escapedIdentifier(_) |
    +           ParserUtils.qualifiedEscapedIdentifier(_, _) => name
    --- End diff --
    
    Nit: shorten it to one line


---
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