[GitHub] spark pull request #18023: Fix SPARK-12139: REGEX Column Specification for H...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #18023: Fix SPARK-12139: REGEX Column Specification for H...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[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 WangDate: 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org