Skip to content

Commit 4953646

Browse files
committed
comment address
1 parent a250886 commit 4953646

File tree

6 files changed

+40
-29
lines changed

6 files changed

+40
-29
lines changed

sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4

Lines changed: 13 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -46,9 +46,9 @@ grammar SqlBase;
4646
}
4747

4848
/**
49-
* When true, ANSI SQL parsing mode is enabled.
49+
* When true, use ANSI SQL standard keywords.
5050
*/
51-
public boolean ansi = false;
51+
public boolean use_SQL_standard_keywords = false;
5252
}
5353

5454
singleStatement
@@ -748,7 +748,7 @@ primaryExpression
748748
| qualifiedName '.' ASTERISK #star
749749
| '(' namedExpression (',' namedExpression)+ ')' #rowConstructor
750750
| '(' query ')' #subqueryExpression
751-
| functionCallName '(' (setQuantifier? argument+=expression (',' argument+=expression)*)? ')'
751+
| functionName '(' (setQuantifier? argument+=expression (',' argument+=expression)*)? ')'
752752
(OVER windowSpec)? #functionCall
753753
| identifier '->' expression #lambda
754754
| '(' identifier (',' identifier)+ ')' '->' expression #lambda
@@ -792,7 +792,7 @@ booleanValue
792792

793793
interval
794794
: negativeSign=MINUS? INTERVAL (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval)?
795-
| {ansi}? (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval)
795+
| {use_SQL_standard_keywords}? (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval)
796796
;
797797

798798
errorCapturingMultiUnitsInterval
@@ -912,7 +912,7 @@ qualifiedNameList
912912
: qualifiedName (',' qualifiedName)*
913913
;
914914

915-
functionCallName
915+
functionName
916916
: qualifiedName
917917
| LEFT
918918
| RIGHT
@@ -937,14 +937,14 @@ errorCapturingIdentifierExtra
937937

938938
identifier
939939
: strictIdentifier
940-
| {!ansi}? strictNonReserved
940+
| {!use_SQL_standard_keywords}? strictNonReserved
941941
;
942942

943943
strictIdentifier
944944
: IDENTIFIER #unquotedIdentifier
945945
| quotedIdentifier #quotedIdentifierAlternative
946-
| {ansi}? ansiNonReserved #unquotedIdentifier
947-
| {!ansi}? nonReserved #unquotedIdentifier
946+
| {use_SQL_standard_keywords}? ansiNonReserved #unquotedIdentifier
947+
| {!use_SQL_standard_keywords}? nonReserved #unquotedIdentifier
948948
;
949949

950950
quotedIdentifier
@@ -961,7 +961,8 @@ number
961961
| MINUS? BIGDECIMAL_LITERAL #bigDecimalLiteral
962962
;
963963

964-
// When `spark.sql.dialect.spark.ansi.enabled=true`, there are 2 kinds of keywords in Spark SQL.
964+
// When we use PostgreSQL dialect or use Spark dialect with
965+
// `spark.sql.dialect.spark.ansi.enabled=true`, there are 2 kinds of keywords in Spark SQL.
965966
// - Reserved keywords:
966967
// Keywords that are reserved and can't be used as identifiers for table, view, column,
967968
// function, alias, etc.
@@ -1161,9 +1162,10 @@ ansiNonReserved
11611162
| YEARS
11621163
;
11631164

1164-
// When `spark.sql.dialect.spark.ansi.enabled=false`, there are 2 kinds of keywords in Spark SQL.
1165+
// When we use Spark dialect with `spark.sql.dialect.spark.ansi.enabled=false`,
1166+
// there are 2 kinds of keywords in Spark SQL.
11651167
// - Non-reserved keywords:
1166-
// Same definition as the one when `spark.sql.dialect.spark.ansi.enabled=true`.
1168+
// Same definition as the one when the ANSI mode enabled.
11671169
// - Strict-non-reserved keywords:
11681170
// A strict version of non-reserved keywords, which can not be used as table alias.
11691171
// You can find the full keywords list by searching "Start of the keywords list" in this file.

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

Lines changed: 13 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -1585,7 +1585,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
15851585
*/
15861586
override def visitFunctionCall(ctx: FunctionCallContext): Expression = withOrigin(ctx) {
15871587
// Create the function call.
1588-
val name = ctx.functionCallName.getText
1588+
val name = ctx.functionName.getText
15891589
val isDistinct = Option(ctx.setQuantifier()).exists(_.DISTINCT != null)
15901590
val arguments = ctx.argument.asScala.map(expression) match {
15911591
case Seq(UnresolvedStar(None))
@@ -1596,7 +1596,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
15961596
expressions
15971597
}
15981598
val function = UnresolvedFunction(
1599-
visitFunctionName(ctx.functionCallName), arguments, isDistinct)
1599+
getFunctionIdentifier(ctx.functionName), arguments, isDistinct)
16001600

16011601
// Check if the function is evaluated in a windowed context.
16021602
ctx.windowSpec match {
@@ -1624,17 +1624,6 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
16241624
visitFunctionName(ctx, ctx.identifier().asScala.map(_.getText))
16251625
}
16261626

1627-
/**
1628-
* Create a function database (optional) and name pair.
1629-
*/
1630-
protected def visitFunctionName(ctx: FunctionCallNameContext): FunctionIdentifier = {
1631-
if (ctx.qualifiedName != null) {
1632-
visitFunctionName(ctx.qualifiedName)
1633-
} else {
1634-
FunctionIdentifier(ctx.getText, None)
1635-
}
1636-
}
1637-
16381627
/**
16391628
* Create a function database (optional) and name pair.
16401629
*/
@@ -1647,6 +1636,17 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
16471636
}
16481637
}
16491638

1639+
/**
1640+
* Get a function identifier consist by database (optional) and name.
1641+
*/
1642+
protected def getFunctionIdentifier(ctx: FunctionNameContext): FunctionIdentifier = {
1643+
if (ctx.qualifiedName != null) {
1644+
visitFunctionName(ctx.qualifiedName)
1645+
} else {
1646+
FunctionIdentifier(ctx.getText, None)
1647+
}
1648+
}
1649+
16501650
/**
16511651
* Create an [[LambdaFunction]].
16521652
*/

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

Lines changed: 8 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.Expression
2828
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
2929
import org.apache.spark.sql.catalyst.trees.Origin
3030
import org.apache.spark.sql.internal.SQLConf
31+
import org.apache.spark.sql.internal.SQLConf.Dialect
3132
import org.apache.spark.sql.types.{DataType, StructType}
3233
import org.apache.spark.unsafe.types.CalendarInterval
3334

@@ -97,19 +98,24 @@ abstract class AbstractSqlParser(conf: SQLConf) extends ParserInterface with Log
9798
protected def parse[T](command: String)(toResult: SqlBaseParser => T): T = {
9899
logDebug(s"Parsing command: $command")
99100

101+
val useSQLStandardKeywords = Dialect.withName(conf.dialect) match {
102+
case Dialect.POSTGRESQL => true
103+
case Dialect.SPARK => conf.dialectSparkAnsiEnabled
104+
}
105+
100106
val lexer = new SqlBaseLexer(new UpperCaseCharStream(CharStreams.fromString(command)))
101107
lexer.removeErrorListeners()
102108
lexer.addErrorListener(ParseErrorListener)
103109
lexer.legacy_setops_precedence_enbled = conf.setOpsPrecedenceEnforced
104-
lexer.ansi = conf.ansiEnabled
110+
lexer.use_SQL_standard_keywords = useSQLStandardKeywords
105111

106112
val tokenStream = new CommonTokenStream(lexer)
107113
val parser = new SqlBaseParser(tokenStream)
108114
parser.addParseListener(PostProcessor)
109115
parser.removeErrorListeners()
110116
parser.addErrorListener(ParseErrorListener)
111117
parser.legacy_setops_precedence_enbled = conf.setOpsPrecedenceEnforced
112-
parser.ansi = conf.ansiEnabled
118+
parser.use_SQL_standard_keywords = useSQLStandardKeywords
113119

114120
try {
115121
try {

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

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -2527,7 +2527,9 @@ class SQLConf extends Serializable with Logging {
25272527
def storeAssignmentPolicy: StoreAssignmentPolicy.Value =
25282528
StoreAssignmentPolicy.withName(getConf(STORE_ASSIGNMENT_POLICY))
25292529

2530-
def usePostgreSQLDialect: Boolean = getConf(DIALECT) == Dialect.POSTGRESQL.toString
2530+
def dialect: String = getConf(DIALECT)
2531+
2532+
def usePostgreSQLDialect: Boolean = dialect == Dialect.POSTGRESQL.toString
25312533

25322534
def dialectSparkAnsiEnabled: Boolean = getConf(DIALECT_SPARK_ANSI_ENABLED)
25332535

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

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -519,7 +519,8 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession {
519519
}
520520
} else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}postgreSQL")) {
521521
PgSQLTestCase(testCaseName, absPath, resultFile) :: Nil
522-
} else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}ansi")) {
522+
} else if (file.getAbsolutePath.startsWith(
523+
s"$inputFilePath${File.separator}use_SQL_standard_keywords")) {
523524
AnsiTestCase(testCaseName, absPath, resultFile) :: Nil
524525
} else {
525526
RegularTestCase(testCaseName, absPath, resultFile) :: Nil

sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -240,7 +240,7 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite {
240240
Seq.empty
241241
} else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}postgreSQL")) {
242242
PgSQLTestCase(testCaseName, absPath, resultFile) :: Nil
243-
} else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}ansi")) {
243+
} else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}use_SQL_standard_keywords")) {
244244
AnsiTestCase(testCaseName, absPath, resultFile) :: Nil
245245
} else {
246246
RegularTestCase(testCaseName, absPath, resultFile) :: Nil

0 commit comments

Comments
 (0)