Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
36 commits
Select commit Hold shift + click to select a range
4cbcebe
IDENTIFIER everywhere
srielau Oct 28, 2025
7b0da84
fix testcases
srielau Oct 28, 2025
d75a9d5
Merge branch 'master' into identifier-lite
srielau Oct 28, 2025
6b3f044
Merge branch 'master' into identifier-lite
srielau Nov 3, 2025
c3a12e0
Lots of fixes
srielau Nov 4, 2025
2ee8073
More fixes
srielau Nov 5, 2025
ebe421a
Introduce SQLCOnf
srielau Nov 5, 2025
87cffc8
Fix SQL Scripting
srielau Nov 5, 2025
e10648c
Fix testcases, linting
srielau Nov 5, 2025
193f946
Code review
srielau Nov 6, 2025
9c8a5ff
Code review improvements: refactor ParserUtils for better clarity and…
srielau Nov 6, 2025
0567456
Fix more testcases, some rework
srielau Nov 6, 2025
70833cc
Merge branch 'master' into identifier-lite
srielau Nov 6, 2025
582ab51
Fix flaky test
srielau Nov 6, 2025
b9f46a3
Delete temporray files
srielau Nov 6, 2025
733f874
remove unecessary testcases from ParametersSuite
srielau Nov 6, 2025
c43af8f
Make identifier-clause.sql less flaky
srielau Nov 6, 2025
e24a0ed
Simplify code, style fixes in error messages
srielau Nov 6, 2025
9b2454a
Fix testcases
srielau Nov 6, 2025
608e1ea
Introduce simpleIdentifier
srielau Nov 6, 2025
9715ccb
Fix build
srielau Nov 7, 2025
774eb2b
Address comment by Wenchen
srielau Nov 7, 2025
cdafbc6
Add more tests
srielau Nov 7, 2025
af00d2d
Fix parameter coalescing and internal error on unbound parameters
srielau Nov 7, 2025
55a245b
more fixes
srielau Nov 7, 2025
f6a6214
Regen execute-immediate and suppress unnecessary error context
srielau Nov 8, 2025
6aad3dd
Undo mistaken testacses updates
srielau Nov 8, 2025
e963e97
fix explain testcases
srielau Nov 8, 2025
db238de
Refine explain fix
srielau Nov 8, 2025
d0836e0
Mocve instructions around
srielau Nov 10, 2025
7fab3d4
Comments by Daniel
srielau Nov 11, 2025
c20e2de
Fix compile error
srielau Nov 11, 2025
480b3ae
Fix compile error
srielau Nov 11, 2025
3593f5a
Still fixing compoile errors
srielau Nov 11, 2025
6f3e060
simplify
srielau Nov 11, 2025
ff191f0
Fix sql scripting regression
srielau Nov 11, 2025
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
7 changes: 1 addition & 6 deletions common/utils/src/main/resources/error/error-conditions.json
Original file line number Diff line number Diff line change
Expand Up @@ -2030,7 +2030,7 @@
},
"IDENTIFIER_TOO_MANY_NAME_PARTS" : {
"message" : [
"<identifier> is not a valid identifier as it has more than 2 name parts."
"<identifier> is not a valid identifier as it has more than <limit> name parts."
],
"sqlState" : "42601"
},
Expand Down Expand Up @@ -8539,11 +8539,6 @@
"Failed to merge incompatible schemas <left> and <right>."
]
},
"_LEGACY_ERROR_TEMP_2096" : {
"message" : [
"<ddl> is not supported temporarily."
]
},
"_LEGACY_ERROR_TEMP_2097" : {
"message" : [
"Could not execute broadcast in <timeout> secs. You can increase the timeout for broadcasts via <broadcastTimeout> or disable broadcast join by setting <autoBroadcastJoinThreshold> to -1 or remove the broadcast hint if it exists in your code."
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,13 @@ options { tokenVocab = SqlBaseLexer; }
* When true, parameter markers are allowed everywhere a literal is supported.
*/
public boolean parameter_substitution_enabled = true;

/**
* When false (default), IDENTIFIER('literal') is resolved to an identifier at parse time (identifier-lite).
* When true, only the legacy IDENTIFIER(expression) function syntax is allowed.
* Controlled by spark.sql.legacy.identifierClause configuration.
*/
public boolean legacy_identifier_clause_only = false;
}

compoundOrSingleStatement
Expand Down Expand Up @@ -321,7 +328,7 @@ statement
| SHOW VIEWS ((FROM | IN) identifierReference)?
(LIKE? pattern=stringLit)? #showViews
| SHOW PARTITIONS identifierReference partitionSpec? #showPartitions
| SHOW identifier? FUNCTIONS ((FROM | IN) ns=identifierReference)?
| SHOW functionScope=simpleIdentifier? FUNCTIONS ((FROM | IN) ns=identifierReference)?
(LIKE? (legacy=multipartIdentifier | pattern=stringLit))? #showFunctions
| SHOW PROCEDURES ((FROM | IN) identifierReference)? #showProcedures
| SHOW CREATE TABLE identifierReference (AS SERDE)? #showCreateTable
Expand Down Expand Up @@ -833,8 +840,8 @@ hint
;

hintStatement
: hintName=identifier
| hintName=identifier LEFT_PAREN parameters+=primaryExpression (COMMA parameters+=primaryExpression)* RIGHT_PAREN
: hintName=simpleIdentifier
| hintName=simpleIdentifier LEFT_PAREN parameters+=primaryExpression (COMMA parameters+=primaryExpression)* RIGHT_PAREN
;

fromClause
Expand Down Expand Up @@ -1241,7 +1248,7 @@ primaryExpression
| identifier #columnReference
| base=primaryExpression DOT fieldName=identifier #dereference
| LEFT_PAREN expression RIGHT_PAREN #parenthesizedExpression
| EXTRACT LEFT_PAREN field=identifier FROM source=valueExpression RIGHT_PAREN #extract
| EXTRACT LEFT_PAREN field=simpleIdentifier FROM source=valueExpression RIGHT_PAREN #extract
| (SUBSTR | SUBSTRING) LEFT_PAREN str=valueExpression (FROM | COMMA) pos=valueExpression
((FOR | COMMA) len=valueExpression)? RIGHT_PAREN #substring
| TRIM LEFT_PAREN trimOption=(BOTH | LEADING | TRAILING)? (trimStr=valueExpression)?
Expand Down Expand Up @@ -1297,7 +1304,7 @@ constant
;

namedParameterMarker
: COLON identifier
: COLON simpleIdentifier
;
comparisonOperator
: EQ | NEQ | NEQJ | LT | LTE | GT | GTE | NSEQ
Expand Down Expand Up @@ -1585,7 +1592,8 @@ qualifiedName
// replace identifier with errorCapturingIdentifier where the immediate follow symbol is not an expression, otherwise
// valid expressions such as "a-b" can be recognized as an identifier
errorCapturingIdentifier
: identifier errorCapturingIdentifierExtra
: identifier errorCapturingIdentifierExtra #errorCapturingIdentifierBase
| {!legacy_identifier_clause_only}? IDENTIFIER_KW LEFT_PAREN stringLit RIGHT_PAREN errorCapturingIdentifierExtra #identifierLiteralWithExtra
;

// extra left-factoring grammar
Expand All @@ -1599,13 +1607,32 @@ identifier
| {!SQL_standard_keyword_behavior}? strictNonReserved
;

// simpleIdentifier: like identifier but without IDENTIFIER('literal') support
// Use this for contexts where IDENTIFIER() syntax is not appropriate:
// - Named parameters (:param_name)
// - Extract field names (EXTRACT(field FROM ...))
// - Other keyword-like or string-like uses
simpleIdentifier
: simpleStrictIdentifier
| {!SQL_standard_keyword_behavior}? strictNonReserved
;

strictIdentifier
: IDENTIFIER #unquotedIdentifier
| quotedIdentifier #quotedIdentifierAlternative
| {!legacy_identifier_clause_only}? IDENTIFIER_KW LEFT_PAREN stringLit RIGHT_PAREN #identifierLiteral
| {SQL_standard_keyword_behavior}? ansiNonReserved #unquotedIdentifier
| {!SQL_standard_keyword_behavior}? nonReserved #unquotedIdentifier
;

// simpleStrictIdentifier: like strictIdentifier but without IDENTIFIER('literal') support
simpleStrictIdentifier
: IDENTIFIER #simpleUnquotedIdentifier
| quotedIdentifier #simpleQuotedIdentifierAlternative
| {SQL_standard_keyword_behavior}? ansiNonReserved #simpleUnquotedIdentifier
| {!SQL_standard_keyword_behavior}? nonReserved #simpleUnquotedIdentifier
;

quotedIdentifier
: BACKQUOTED_IDENTIFIER
| {double_quoted_identifiers}? DOUBLEQUOTED_STRING
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,15 +20,15 @@ import java.util.Locale

import scala.jdk.CollectionConverters._

import org.antlr.v4.runtime.Token
import org.antlr.v4.runtime.{ParserRuleContext, Token}
import org.antlr.v4.runtime.tree.ParseTree

import org.apache.spark.SparkException
import org.apache.spark.sql.catalyst.parser.SqlBaseParser._
import org.apache.spark.sql.catalyst.util.CollationFactory
import org.apache.spark.sql.catalyst.util.SparkParserUtils.{string, withOrigin}
import org.apache.spark.sql.connector.catalog.IdentityColumnSpec
import org.apache.spark.sql.errors.QueryParsingErrors
import org.apache.spark.sql.errors.{DataTypeErrorsBase, QueryParsingErrors}
import org.apache.spark.sql.internal.SqlApiConf
import org.apache.spark.sql.types.{ArrayType, BinaryType, BooleanType, ByteType, CalendarIntervalType, CharType, DataType, DateType, DayTimeIntervalType, DecimalType, DoubleType, FloatType, GeographyType, GeometryType, IntegerType, LongType, MapType, MetadataBuilder, NullType, ShortType, StringType, StructField, StructType, TimestampNTZType, TimestampType, TimeType, VarcharType, VariantType, YearMonthIntervalType}

Expand Down Expand Up @@ -60,12 +60,52 @@ import org.apache.spark.sql.types.{ArrayType, BinaryType, BooleanType, ByteType,
*
* @see
* [[org.apache.spark.sql.catalyst.parser.AstBuilder]] for the full SQL statement parser
*
* ==CRITICAL: Extracting Identifier Names==
*
* When extracting identifier names from parser contexts, you MUST use the helper methods provided
* by this class instead of calling ctx.getText() directly:
*
* - '''getIdentifierText(ctx)''': For single identifiers (column names, aliases, window names)
* - '''getIdentifierParts(ctx)''': For qualified identifiers (table names, schema.table)
*
* '''DO NOT use ctx.getText() or ctx.identifier.getText()''' directly! These methods do not
* handle the IDENTIFIER('literal') syntax and will cause incorrect behavior.
*
* The IDENTIFIER('literal') syntax allows string literals to be used as identifiers at parse time
* (e.g., IDENTIFIER('my_col') resolves to the identifier my_col). If you use getText(), you'll
* get the raw text "IDENTIFIER('my_col')" instead of "my_col", breaking the feature.
*
* Example:
* {{{
* // WRONG - does not handle IDENTIFIER('literal'):
* val name = ctx.identifier.getText
* SubqueryAlias(ctx.name.getText, plan)
*
* // CORRECT - handles both regular identifiers and IDENTIFIER('literal'):
* val name = getIdentifierText(ctx.identifier)
* SubqueryAlias(getIdentifierText(ctx.name), plan)
* }}}
*/
class DataTypeAstBuilder extends SqlBaseParserBaseVisitor[AnyRef] {
class DataTypeAstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with DataTypeErrorsBase {
protected def typedVisit[T](ctx: ParseTree): T = {
ctx.accept(this).asInstanceOf[T]
}

/**
* Public helper to extract identifier parts from a context. This is exposed as public to allow
* utility classes like ParserUtils to reuse the identifier resolution logic without duplicating
* code.
*
* @param ctx
* The parser context containing the identifier.
* @return
* Sequence of identifier parts.
*/
def extractIdentifierParts(ctx: ParserRuleContext): Seq[String] = {
getIdentifierParts(ctx)
}

override def visitSingleDataType(ctx: SingleDataTypeContext): DataType = withOrigin(ctx) {
typedVisit[DataType](ctx.dataType)
}
Expand Down Expand Up @@ -161,11 +201,112 @@ class DataTypeAstBuilder extends SqlBaseParserBaseVisitor[AnyRef] {
}

/**
* Create a multi-part identifier.
* Parse a string into a multi-part identifier. Subclasses should override this method to
* provide proper multi-part identifier parsing with access to a full SQL parser.
*
* For example, in AstBuilder, this would parse "`catalog`.`schema`.`table`" into Seq("catalog",
* "schema", "table").
*
* The base implementation fails with an assertion to catch cases where multi-part identifiers
* are used without a proper parser implementation.
*
* @param identifier
* The identifier string to parse, potentially containing dots and backticks.
* @return
* Sequence of identifier parts.
*/
protected def parseMultipartIdentifier(identifier: String): Seq[String] = {
throw SparkException.internalError(
"parseMultipartIdentifier must be overridden by subclasses. " +
s"Attempted to parse: $identifier")
}

/**
* Get the identifier parts from a context, handling both regular identifiers and
* IDENTIFIER('literal'). This method is used to support identifier-lite syntax where
* IDENTIFIER('string') is folded at parse time. For qualified identifiers like
* IDENTIFIER('`catalog`.`schema`'), this will parse the string and return multiple parts.
*
* Subclasses should override this method to provide actual parsing logic.
*/
protected def getIdentifierParts(ctx: ParserRuleContext): Seq[String] = {
ctx match {
case idCtx: IdentifierContext =>
// identifier can be either strictIdentifier or strictNonReserved.
// Recursively process the strictIdentifier.
Option(idCtx.strictIdentifier()).map(getIdentifierParts).getOrElse(Seq(ctx.getText))

case idLitCtx: IdentifierLiteralContext =>
// For IDENTIFIER('literal') in strictIdentifier.
val literalValue = string(visitStringLit(idLitCtx.stringLit()))
// Parse the string to handle qualified identifiers like "`cat`.`schema`".
parseMultipartIdentifier(literalValue)

case idLitCtx: IdentifierLiteralWithExtraContext =>
// For IDENTIFIER('literal') in errorCapturingIdentifier.
val literalValue = string(visitStringLit(idLitCtx.stringLit()))
// Parse the string to handle qualified identifiers like "`cat`.`schema`".
parseMultipartIdentifier(literalValue)

case base: ErrorCapturingIdentifierBaseContext =>
// Regular identifier with errorCapturingIdentifierExtra.
// Need to recursively handle identifier which might itself be IDENTIFIER('literal').
Option(base.identifier())
.flatMap(id => Option(id.strictIdentifier()).map(getIdentifierParts))
.getOrElse(Seq(ctx.getText))

case _ =>
// For regular identifiers, just return the text as a single part.
Seq(ctx.getText)
}
}

/**
* Get the text of a SINGLE identifier, handling both regular identifiers and
* IDENTIFIER('literal'). This method REQUIRES that the identifier be unqualified (single part
* only). If IDENTIFIER('qualified.name') is used where a single identifier is required, this
* will error.
*/
protected def getIdentifierText(ctx: ParserRuleContext): String = {
val parts = getIdentifierParts(ctx)
if (parts.size > 1) {
throw new ParseException(
errorClass = "IDENTIFIER_TOO_MANY_NAME_PARTS",
messageParameters = Map("identifier" -> toSQLId(parts), "limit" -> "1"),
ctx)
}
parts.head
}

/**
* Create a multi-part identifier. Handles identifier-lite with qualified identifiers like
* IDENTIFIER('`cat`.`schema`').table
*/
override def visitMultipartIdentifier(ctx: MultipartIdentifierContext): Seq[String] =
withOrigin(ctx) {
ctx.parts.asScala.map(_.getText).toSeq
ctx.parts.asScala.flatMap { part =>
// Each part is an errorCapturingIdentifier, which can be either:
// 1. identifier errorCapturingIdentifierExtra (regular path) - labeled as
// #errorCapturingIdentifierBase
// 2. IDENTIFIER_KW LEFT_PAREN stringLit RIGHT_PAREN errorCapturingIdentifierExtra
// (identifier-lite path) - labeled as #identifierLiteralWithExtra
part match {
case idLitWithExtra: IdentifierLiteralWithExtraContext =>
// This is identifier-lite: IDENTIFIER('string')
getIdentifierParts(idLitWithExtra)
case base: ErrorCapturingIdentifierBaseContext =>
// Regular identifier path
val identifierCtx = base.identifier()
if (identifierCtx != null && identifierCtx.strictIdentifier() != null) {
getIdentifierParts(identifierCtx.strictIdentifier())
} else {
Seq(part.getText)
}
case _ =>
// Fallback for other cases
Seq(part.getText)
}
}.toSeq
}

/**
Expand Down Expand Up @@ -351,7 +492,7 @@ class DataTypeAstBuilder extends SqlBaseParserBaseVisitor[AnyRef] {
}

StructField(
name = colName.getText,
name = getIdentifierText(colName),
dataType = typedVisit[DataType](ctx.dataType),
nullable = NULL == null,
metadata = builder.build())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,8 @@ class SubstituteParmsAstBuilder extends SqlBaseParserBaseVisitor[AnyRef] {
*/
override def visitNamedParameterLiteral(ctx: NamedParameterLiteralContext): AnyRef =
withOrigin(ctx) {
val paramName = ctx.namedParameterMarker().identifier().getText
// Named parameters use simpleIdentifier, so .getText() is correct.
val paramName = ctx.namedParameterMarker().simpleIdentifier().getText
namedParams += paramName

// Calculate the location of the entire parameter (including the colon)
Expand Down Expand Up @@ -117,7 +118,8 @@ class SubstituteParmsAstBuilder extends SqlBaseParserBaseVisitor[AnyRef] {
*/
override def visitNamedParameterMarkerRule(ctx: NamedParameterMarkerRuleContext): AnyRef =
withOrigin(ctx) {
val paramName = ctx.namedParameterMarker().identifier().getText
// Named parameters use simpleIdentifier, so .getText() is correct.
val paramName = ctx.namedParameterMarker().simpleIdentifier().getText
namedParams += paramName

// Calculate the location of the entire parameter (including the colon)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -430,7 +430,15 @@ case class UnclosedCommentProcessor(command: String, tokenStream: CommonTokenStr
}

object DataTypeParser extends AbstractParser {
override protected def astBuilder: DataTypeAstBuilder = new DataTypeAstBuilder
override protected def astBuilder: DataTypeAstBuilder = new DataTypeAstBuilder {
// DataTypeParser only parses data types, not full SQL.
// Multi-part identifiers should not appear in IDENTIFIER() within type definitions.
override protected def parseMultipartIdentifier(identifier: String): Seq[String] = {
throw SparkException.internalError(
"DataTypeParser does not support multi-part identifiers in IDENTIFIER(). " +
s"Attempted to parse: $identifier")
}
}
}

object AbstractParser extends Logging {
Expand Down Expand Up @@ -476,6 +484,7 @@ object AbstractParser extends Logging {
parser.SQL_standard_keyword_behavior = conf.enforceReservedKeywords
parser.double_quoted_identifiers = conf.doubleQuotedIdentifiers
parser.parameter_substitution_enabled = !conf.legacyParameterSubstitutionConstantsOnly
parser.legacy_identifier_clause_only = conf.legacyIdentifierClauseOnly
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -477,7 +477,7 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase {
ctx)
}

def showFunctionsUnsupportedError(identifier: String, ctx: IdentifierContext): Throwable = {
def showFunctionsUnsupportedError(identifier: String, ctx: ParserRuleContext): Throwable = {
new ParseException(
errorClass = "INVALID_SQL_SYNTAX.SHOW_FUNCTIONS_INVALID_SCOPE",
messageParameters = Map("scope" -> toSQLId(identifier)),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@ private[sql] trait SqlApiConf {
def parserDfaCacheFlushThreshold: Int
def parserDfaCacheFlushRatio: Double
def legacyParameterSubstitutionConstantsOnly: Boolean
def legacyIdentifierClauseOnly: Boolean
}

private[sql] object SqlApiConf {
Expand Down Expand Up @@ -104,4 +105,5 @@ private[sql] object DefaultSqlApiConf extends SqlApiConf {
override def parserDfaCacheFlushThreshold: Int = -1
override def parserDfaCacheFlushRatio: Double = -1.0
override def legacyParameterSubstitutionConstantsOnly: Boolean = false
override def legacyIdentifierClauseOnly: Boolean = false
}
Loading