Skip to content

Commit 1421811

Browse files
huaxingaodongjoon-hyun
authored andcommitted
[SPARK-38173][SQL][3.2] Quoted column cannot be recognized correctly when quotedRegexColumnNames is true
### What changes were proposed in this pull request? backporting apache#35476 to 3.2 ### Why are the changes needed? bug fixing in 3.2 ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? new UT Closes apache#39473 from huaxingao/3.2. Authored-by: huaxingao <[email protected]> Signed-off-by: Dongjoon Hyun <[email protected]>
1 parent 7eca60d commit 1421811

File tree

2 files changed

+25
-2
lines changed

2 files changed

+25
-2
lines changed

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

Lines changed: 12 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -2030,6 +2030,14 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
20302030
return false
20312031
}
20322032

2033+
/**
2034+
* Returns whether the pattern is a regex expression (instead of a normal
2035+
* string). Normal string is a string with all alphabets/digits and "_".
2036+
*/
2037+
private def isRegex(pattern: String): Boolean = {
2038+
pattern.exists(p => !Character.isLetterOrDigit(p) && p != '_')
2039+
}
2040+
20332041
/**
20342042
* Create a dereference expression. The return type depends on the type of the parent.
20352043
* If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or
@@ -2042,7 +2050,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
20422050
case unresolved_attr @ UnresolvedAttribute(nameParts) =>
20432051
ctx.fieldName.getStart.getText match {
20442052
case escapedIdentifier(columnNameRegex)
2045-
if conf.supportQuotedRegexColumnName && canApplyRegex(ctx) =>
2053+
if conf.supportQuotedRegexColumnName &&
2054+
isRegex(columnNameRegex) && canApplyRegex(ctx) =>
20462055
UnresolvedRegex(columnNameRegex, Some(unresolved_attr.name),
20472056
conf.caseSensitiveAnalysis)
20482057
case _ =>
@@ -2060,7 +2069,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
20602069
override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) {
20612070
ctx.getStart.getText match {
20622071
case escapedIdentifier(columnNameRegex)
2063-
if conf.supportQuotedRegexColumnName && canApplyRegex(ctx) =>
2072+
if conf.supportQuotedRegexColumnName &&
2073+
isRegex(columnNameRegex) && canApplyRegex(ctx) =>
20642074
UnresolvedRegex(columnNameRegex, None, conf.caseSensitiveAnalysis)
20652075
case _ =>
20662076
UnresolvedAttribute.quoted(ctx.getText)

sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -4259,6 +4259,19 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark
42594259
sql("select * from test_temp_view"),
42604260
Row(1, 2, 3, 1, 2, 3, 1, 1))
42614261
}
4262+
4263+
test("SPARK-38173: Quoted column cannot be recognized correctly " +
4264+
"when quotedRegexColumnNames is true") {
4265+
withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "true") {
4266+
checkAnswer(
4267+
sql(
4268+
"""
4269+
|SELECT `(C3)?+.+`,T.`C1` * `C2` AS CC
4270+
|FROM (SELECT 3 AS C1,2 AS C2,1 AS C3) T
4271+
|""".stripMargin),
4272+
Row(3, 2, 6) :: Nil)
4273+
}
4274+
}
42624275
}
42634276

42644277
case class Foo(bar: Option[String])

0 commit comments

Comments
 (0)