Skip to content

Commit 2cbfc97

Browse files
janewangfbgatorsmile
authored andcommitted
[SPARK-12139][SQL] REGEX Column Specification
## 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 - Run spark-shell tested the original failed query: scala> hc.sql("SELECT `(a|b)?+.+` from test1").collect.foreach(println) Author: Jane Wang <[email protected]> Closes apache#18023 from janewangfb/support_select_regex.
1 parent d3e0716 commit 2cbfc97

File tree

17 files changed

+825
-255
lines changed

17 files changed

+825
-255
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala

Lines changed: 28 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@ import org.apache.spark.sql.catalyst.{FunctionIdentifier, InternalRow, TableIden
2222
import org.apache.spark.sql.catalyst.errors.TreeNodeException
2323
import org.apache.spark.sql.catalyst.expressions._
2424
import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodegenFallback, ExprCode}
25+
import org.apache.spark.sql.catalyst.parser.ParserUtils
2526
import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan}
2627
import org.apache.spark.sql.catalyst.trees.TreeNode
2728
import org.apache.spark.sql.catalyst.util.quoteIdentifier
@@ -123,7 +124,10 @@ case class UnresolvedAttribute(nameParts: Seq[String]) extends Attribute with Un
123124

124125
override def toString: String = s"'$name"
125126

126-
override def sql: String = quoteIdentifier(name)
127+
override def sql: String = name match {
128+
case ParserUtils.escapedIdentifier(_) | ParserUtils.qualifiedEscapedIdentifier(_, _) => name
129+
case _ => quoteIdentifier(name)
130+
}
127131
}
128132

129133
object UnresolvedAttribute {
@@ -306,6 +310,29 @@ case class UnresolvedStar(target: Option[Seq[String]]) extends Star with Unevalu
306310
override def toString: String = target.map(_ + ".").getOrElse("") + "*"
307311
}
308312

313+
/**
314+
* Represents all of the input attributes to a given relational operator, for example in
315+
* "SELECT `(id)?+.+` FROM ...".
316+
*
317+
* @param table an optional table that should be the target of the expansion. If omitted all
318+
* tables' columns are produced.
319+
*/
320+
case class UnresolvedRegex(regexPattern: String, table: Option[String], caseSensitive: Boolean)
321+
extends Star with Unevaluable {
322+
override def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] = {
323+
val pattern = if (caseSensitive) regexPattern else s"(?i)$regexPattern"
324+
table match {
325+
// If there is no table specified, use all input attributes that match expr
326+
case None => input.output.filter(_.name.matches(pattern))
327+
// If there is a table, pick out attributes that are part of this table that match expr
328+
case Some(t) => input.output.filter(_.qualifier.exists(resolver(_, t)))
329+
.filter(_.name.matches(pattern))
330+
}
331+
}
332+
333+
override def toString: String = table.map(_ + "." + regexPattern).getOrElse(regexPattern)
334+
}
335+
309336
/**
310337
* Used to assign new names to Generator's output, such as hive udtf.
311338
* For example the SQL expression "stack(2, key, value, key, value) as (a, b)" could be represented

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala

Lines changed: 36 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -1261,25 +1261,54 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
12611261
}
12621262

12631263
/**
1264-
* Create a dereference expression. The return type depends on the type of the parent, this can
1265-
* either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
1266-
* [[UnresolvedExtractValue]] if the parent is some expression.
1264+
* Currently only regex in expressions of SELECT statements are supported; in other
1265+
* places, e.g., where `(a)?+.+` = 2, regex are not meaningful.
1266+
*/
1267+
private def canApplyRegex(ctx: ParserRuleContext): Boolean = withOrigin(ctx) {
1268+
var parent = ctx.getParent
1269+
while (parent != null) {
1270+
if (parent.isInstanceOf[NamedExpressionContext]) return true
1271+
parent = parent.getParent
1272+
}
1273+
return false
1274+
}
1275+
1276+
/**
1277+
* Create a dereference expression. The return type depends on the type of the parent.
1278+
* If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
1279+
* a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression,
1280+
* it can be [[UnresolvedExtractValue]].
12671281
*/
12681282
override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
12691283
val attr = ctx.fieldName.getText
12701284
expression(ctx.base) match {
1271-
case UnresolvedAttribute(nameParts) =>
1272-
UnresolvedAttribute(nameParts :+ attr)
1285+
case unresolved_attr @ UnresolvedAttribute(nameParts) =>
1286+
ctx.fieldName.getStart.getText match {
1287+
case escapedIdentifier(columnNameRegex)
1288+
if conf.supportQuotedRegexColumnName && canApplyRegex(ctx) =>
1289+
UnresolvedRegex(columnNameRegex, Some(unresolved_attr.name),
1290+
conf.caseSensitiveAnalysis)
1291+
case _ =>
1292+
UnresolvedAttribute(nameParts :+ attr)
1293+
}
12731294
case e =>
12741295
UnresolvedExtractValue(e, Literal(attr))
12751296
}
12761297
}
12771298

12781299
/**
1279-
* Create an [[UnresolvedAttribute]] expression.
1300+
* Create an [[UnresolvedAttribute]] expression or a [[UnresolvedRegex]] if it is a regex
1301+
* quoted in ``
12801302
*/
12811303
override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) {
1282-
UnresolvedAttribute.quoted(ctx.getText)
1304+
ctx.getStart.getText match {
1305+
case escapedIdentifier(columnNameRegex)
1306+
if conf.supportQuotedRegexColumnName && canApplyRegex(ctx) =>
1307+
UnresolvedRegex(columnNameRegex, None, conf.caseSensitiveAnalysis)
1308+
case _ =>
1309+
UnresolvedAttribute.quoted(ctx.getText)
1310+
}
1311+
12831312
}
12841313

12851314
/**

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -177,6 +177,12 @@ object ParserUtils {
177177
sb.toString()
178178
}
179179

180+
/** the column name pattern in quoted regex without qualifier */
181+
val escapedIdentifier = "`(.+)`".r
182+
183+
/** the column name pattern in quoted regex with qualifier */
184+
val qualifiedEscapedIdentifier = ("(.+)" + """.""" + "`(.+)`").r
185+
180186
/** Some syntactic sugar which makes it easier to work with optional clauses for LogicalPlans. */
181187
implicit class EnhancedLogicalPlan(val plan: LogicalPlan) extends AnyVal {
182188
/**

sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -855,6 +855,12 @@ object SQLConf {
855855
.intConf
856856
.createWithDefault(UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD.toInt)
857857

858+
val SUPPORT_QUOTED_REGEX_COLUMN_NAME = buildConf("spark.sql.parser.quotedRegexColumnNames")
859+
.doc("When true, quoted Identifiers (using backticks) in SELECT statement are interpreted" +
860+
" as regular expressions.")
861+
.booleanConf
862+
.createWithDefault(false)
863+
858864
val ARROW_EXECUTION_ENABLE =
859865
buildConf("spark.sql.execution.arrow.enable")
860866
.internal()
@@ -1133,6 +1139,8 @@ class SQLConf extends Serializable with Logging {
11331139

11341140
def starSchemaFTRatio: Double = getConf(STARSCHEMA_FACT_TABLE_RATIO)
11351141

1142+
def supportQuotedRegexColumnName: Boolean = getConf(SUPPORT_QUOTED_REGEX_COLUMN_NAME)
1143+
11361144
def arrowEnable: Boolean = getConf(ARROW_EXECUTION_ENABLE)
11371145

11381146
def arrowMaxRecordsPerBatch: Int = getConf(ARROW_EXECUTION_MAX_RECORDS_PER_BATCH)

sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala

Lines changed: 24 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -40,7 +40,7 @@ import org.apache.spark.sql.catalyst.encoders._
4040
import org.apache.spark.sql.catalyst.expressions._
4141
import org.apache.spark.sql.catalyst.json.{JacksonGenerator, JSONOptions}
4242
import org.apache.spark.sql.catalyst.optimizer.CombineUnions
43-
import org.apache.spark.sql.catalyst.parser.ParseException
43+
import org.apache.spark.sql.catalyst.parser.{ParseException, ParserUtils}
4444
import org.apache.spark.sql.catalyst.plans._
4545
import org.apache.spark.sql.catalyst.plans.logical._
4646
import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection}
@@ -1178,8 +1178,29 @@ class Dataset[T] private[sql](
11781178
case "*" =>
11791179
Column(ResolvedStar(queryExecution.analyzed.output))
11801180
case _ =>
1181-
val expr = resolve(colName)
1182-
Column(expr)
1181+
if (sqlContext.conf.supportQuotedRegexColumnName) {
1182+
colRegex(colName)
1183+
} else {
1184+
val expr = resolve(colName)
1185+
Column(expr)
1186+
}
1187+
}
1188+
1189+
/**
1190+
* Selects column based on the column name specified as a regex and return it as [[Column]].
1191+
* @group untypedrel
1192+
* @since 2.3.0
1193+
*/
1194+
def colRegex(colName: String): Column = {
1195+
val caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis
1196+
colName match {
1197+
case ParserUtils.escapedIdentifier(columnNameRegex) =>
1198+
Column(UnresolvedRegex(columnNameRegex, None, caseSensitive))
1199+
case ParserUtils.qualifiedEscapedIdentifier(nameParts, columnNameRegex) =>
1200+
Column(UnresolvedRegex(columnNameRegex, Some(nameParts), caseSensitive))
1201+
case _ =>
1202+
Column(resolve(colName))
1203+
}
11831204
}
11841205

11851206
/**
Lines changed: 52 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,52 @@
1+
set spark.sql.parser.quotedRegexColumnNames=false;
2+
3+
CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES
4+
(1, "1", "11"), (2, "2", "22"), (3, "3", "33"), (4, "4", "44"), (5, "5", "55"), (6, "6", "66")
5+
AS testData(key, value1, value2);
6+
7+
CREATE OR REPLACE TEMPORARY VIEW testData2 AS SELECT * FROM VALUES
8+
(1, 1, 1, 2), (1, 2, 1, 2), (2, 1, 2, 3), (2, 2, 2, 3), (3, 1, 3, 4), (3, 2, 3, 4)
9+
AS testData2(A, B, c, d);
10+
11+
-- AnalysisException
12+
SELECT `(a)?+.+` FROM testData2 WHERE a = 1;
13+
SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1;
14+
SELECT `(a|b)` FROM testData2 WHERE a = 2;
15+
SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2;
16+
SELECT SUM(`(a|b)?+.+`) FROM testData2;
17+
SELECT SUM(`(a)`) FROM testData2;
18+
19+
set spark.sql.parser.quotedRegexColumnNames=true;
20+
21+
-- Regex columns
22+
SELECT `(a)?+.+` FROM testData2 WHERE a = 1;
23+
SELECT `(A)?+.+` FROM testData2 WHERE a = 1;
24+
SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1;
25+
SELECT t.`(A)?+.+` FROM testData2 t WHERE a = 1;
26+
SELECT `(a|B)` FROM testData2 WHERE a = 2;
27+
SELECT `(A|b)` FROM testData2 WHERE a = 2;
28+
SELECT `(a|B)?+.+` FROM testData2 WHERE a = 2;
29+
SELECT `(A|b)?+.+` FROM testData2 WHERE a = 2;
30+
SELECT `(e|f)` FROM testData2;
31+
SELECT t.`(e|f)` FROM testData2 t;
32+
SELECT p.`(KEY)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3;
33+
SELECT p.`(key)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3;
34+
35+
set spark.sql.caseSensitive=true;
36+
37+
CREATE OR REPLACE TEMPORARY VIEW testdata3 AS SELECT * FROM VALUES
38+
(0, 1), (1, 2), (2, 3), (3, 4)
39+
AS testdata3(a, b);
40+
41+
-- Regex columns
42+
SELECT `(A)?+.+` FROM testdata3;
43+
SELECT `(a)?+.+` FROM testdata3;
44+
SELECT `(A)?+.+` FROM testdata3 WHERE a > 1;
45+
SELECT `(a)?+.+` FROM testdata3 where `a` > 1;
46+
SELECT SUM(`a`) FROM testdata3;
47+
SELECT SUM(`(a)`) FROM testdata3;
48+
SELECT SUM(`(a)?+.+`) FROM testdata3;
49+
SELECT SUM(a) FROM testdata3 GROUP BY `a`;
50+
-- AnalysisException
51+
SELECT SUM(a) FROM testdata3 GROUP BY `(a)`;
52+
SELECT SUM(a) FROM testdata3 GROUP BY `(a)?+.+`;

0 commit comments

Comments
 (0)