Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
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
Original file line number Diff line number Diff line change
Expand Up @@ -96,7 +96,7 @@ statement
| ctes? dmlStatementNoWith #dmlStatement
| USE NAMESPACE? multipartIdentifier #use
| CREATE namespace (IF NOT EXISTS)? multipartIdentifier
((COMMENT comment=STRING) |
(commentSpec |
locationSpec |
(WITH (DBPROPERTIES | PROPERTIES) tablePropertyList))* #createNamespace
| ALTER namespace multipartIdentifier
Expand All @@ -116,7 +116,7 @@ statement
createTableClauses
(AS? query)? #createTable
| createTableHeader ('(' columns=colTypeList ')')?
((COMMENT comment=STRING) |
(commentSpec |
(PARTITIONED BY '(' partitionColumns=colTypeList ')' |
PARTITIONED BY partitionColumnNames=identifierList) |
bucketSpec |
Expand Down Expand Up @@ -160,7 +160,7 @@ statement
UNSET TBLPROPERTIES (IF EXISTS)? tablePropertyList #unsetTableProperties
| ALTER TABLE table=multipartIdentifier
(ALTER | CHANGE) COLUMN? column=multipartIdentifier
(TYPE dataType)? (COMMENT comment=STRING)? colPosition? #alterTableColumn
(TYPE dataType)? commentSpec? colPosition? #alterTableColumn
| ALTER TABLE table=multipartIdentifier partitionSpec?
CHANGE COLUMN?
colName=multipartIdentifier colType colPosition? #hiveChangeColumn
Expand All @@ -182,7 +182,7 @@ statement
| CREATE (OR REPLACE)? (GLOBAL? TEMPORARY)?
VIEW (IF NOT EXISTS)? multipartIdentifier
identifierCommentList?
((COMMENT STRING) |
(commentSpec |
(PARTITIONED ON identifierList) |
(TBLPROPERTIES tablePropertyList))*
AS query #createView
Expand Down Expand Up @@ -216,8 +216,8 @@ statement
multipartIdentifier partitionSpec? describeColName? #describeTable
| (DESC | DESCRIBE) QUERY? query #describeQuery
| COMMENT ON namespace multipartIdentifier IS
commennt=(STRING | NULL) #commentNamespace
| COMMENT ON TABLE multipartIdentifier IS commennt=(STRING | NULL) #commentTable
comment=(STRING | NULL) #commentNamespace
| COMMENT ON TABLE multipartIdentifier IS comment=(STRING | NULL) #commentTable
| REFRESH TABLE multipartIdentifier #refreshTable
| REFRESH (STRING | .*?) #refreshResource
| CACHE LAZY? TABLE multipartIdentifier
Expand Down Expand Up @@ -306,6 +306,10 @@ locationSpec
: LOCATION STRING
;

commentSpec
: COMMENT STRING
;

query
: ctes? queryTerm queryOrganization
;
Expand Down Expand Up @@ -364,7 +368,7 @@ createTableClauses
(PARTITIONED BY partitioning=transformList) |
bucketSpec |
locationSpec |
(COMMENT comment=STRING) |
commentSpec |
(TBLPROPERTIES tableProps=tablePropertyList))*
;

Expand Down Expand Up @@ -652,7 +656,7 @@ identifierCommentList
;

identifierComment
: identifier (COMMENT STRING)?
: identifier commentSpec?
;

relationPrimary
Expand Down Expand Up @@ -865,23 +869,23 @@ qualifiedColTypeWithPositionList
;

qualifiedColTypeWithPosition
: name=multipartIdentifier dataType (COMMENT comment=STRING)? colPosition?
: name=multipartIdentifier dataType commentSpec? colPosition?
;

colTypeList
: colType (',' colType)*
;

colType
: colName=errorCapturingIdentifier dataType (COMMENT STRING)?
: colName=errorCapturingIdentifier dataType commentSpec?
;

complexColTypeList
: complexColType (',' complexColType)*
;

complexColType
: identifier ':' dataType (COMMENT STRING)?
: identifier ':' dataType commentSpec?
;

whenClause
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2227,9 +2227,10 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging

val builder = new MetadataBuilder
// Add comment to metadata
if (STRING != null) {
builder.putString("comment", string(STRING))
Option(commentSpec()).map(visitCommentSpec).foreach {
builder.putString("comment", _)
}

// Add Hive type string to metadata.
val rawDataType = typedVisit[DataType](ctx.dataType)
val cleanedDataType = HiveStringType.replaceCharType(rawDataType)
Expand Down Expand Up @@ -2265,16 +2266,37 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
override def visitComplexColType(ctx: ComplexColTypeContext): StructField = withOrigin(ctx) {
import ctx._
val structField = StructField(identifier.getText, typedVisit(dataType), nullable = true)
if (STRING == null) structField else structField.withComment(string(STRING))
Option(commentSpec).map(visitCommentSpec).map(structField.withComment).getOrElse(structField)
}

/**
* Create location string.
* Create a location string.
*/
override def visitLocationSpec(ctx: LocationSpecContext): String = withOrigin(ctx) {
string(ctx.STRING)
}

/**
* Create an optional location string.
*/
protected def visitLocationSpecList(ctx: java.util.List[LocationSpecContext]): Option[String] = {
ctx.asScala.headOption.map(visitLocationSpec)
}

/**
* Create a comment string.
*/
override def visitCommentSpec(ctx: CommentSpecContext): String = withOrigin(ctx) {
string(ctx.STRING)
}

/**
* Create an optional comment string.
*/
protected def visitCommentSpecList(ctx: java.util.List[CommentSpecContext]): Option[String] = {
ctx.asScala.headOption.map(visitCommentSpec)
}

/**
* Create a [[BucketSpec]].
*/
Expand Down Expand Up @@ -2513,7 +2535,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
* }}}
*/
override def visitCreateNamespace(ctx: CreateNamespaceContext): LogicalPlan = withOrigin(ctx) {
checkDuplicateClauses(ctx.COMMENT, "COMMENT", ctx)
checkDuplicateClauses(ctx.commentSpec(), "COMMENT", ctx)
checkDuplicateClauses(ctx.locationSpec, "LOCATION", ctx)
checkDuplicateClauses(ctx.PROPERTIES, "WITH PROPERTIES", ctx)
checkDuplicateClauses(ctx.DBPROPERTIES, "WITH DBPROPERTIES", ctx)
Expand All @@ -2525,10 +2547,12 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
var properties = ctx.tablePropertyList.asScala.headOption
.map(visitPropertyKeyValues)
.getOrElse(Map.empty)
Option(ctx.comment).map(string).map {

visitCommentSpecList(ctx.commentSpec()).foreach {
properties += SupportsNamespaces.PROP_COMMENT -> _
}
ctx.locationSpec.asScala.headOption.map(visitLocationSpec).map {

visitLocationSpecList(ctx.locationSpec()).foreach {
properties += SupportsNamespaces.PROP_LOCATION -> _
}

Expand Down Expand Up @@ -2618,7 +2642,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
checkDuplicateClauses(ctx.TBLPROPERTIES, "TBLPROPERTIES", ctx)
checkDuplicateClauses(ctx.OPTIONS, "OPTIONS", ctx)
checkDuplicateClauses(ctx.PARTITIONED, "PARTITIONED BY", ctx)
checkDuplicateClauses(ctx.COMMENT, "COMMENT", ctx)
checkDuplicateClauses(ctx.commentSpec(), "COMMENT", ctx)
checkDuplicateClauses(ctx.bucketSpec(), "CLUSTERED BY", ctx)
checkDuplicateClauses(ctx.locationSpec, "LOCATION", ctx)

Expand All @@ -2627,8 +2651,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
val bucketSpec = ctx.bucketSpec().asScala.headOption.map(visitBucketSpec)
val properties = Option(ctx.tableProps).map(visitPropertyKeyValues).getOrElse(Map.empty)
val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty)
val location = ctx.locationSpec.asScala.headOption.map(visitLocationSpec)
val comment = Option(ctx.comment).map(string)
val location = visitLocationSpecList(ctx.locationSpec())
val comment = visitCommentSpecList(ctx.commentSpec())
(partitioning, bucketSpec, properties, options, location, comment)
}

Expand Down Expand Up @@ -2810,7 +2834,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
QualifiedColType(
typedVisit[Seq[String]](ctx.name),
typedVisit[DataType](ctx.dataType),
Option(ctx.comment).map(string),
Option(ctx.commentSpec()).map(visitCommentSpec),
Option(ctx.colPosition).map(typedVisit[ColumnPosition]))
}

Expand Down Expand Up @@ -2859,7 +2883,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
override def visitAlterTableColumn(
ctx: AlterTableColumnContext): LogicalPlan = withOrigin(ctx) {
val verb = if (ctx.CHANGE != null) "CHANGE" else "ALTER"
if (ctx.dataType == null && ctx.comment == null && ctx.colPosition == null) {
if (ctx.dataType == null && ctx.commentSpec() == null && ctx.colPosition == null) {
operationNotAllowed(
s"ALTER TABLE table $verb COLUMN requires a TYPE or a COMMENT or a FIRST/AFTER", ctx)
}
Expand All @@ -2868,7 +2892,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
visitMultipartIdentifier(ctx.table),
typedVisit[Seq[String]](ctx.column),
Option(ctx.dataType).map(typedVisit[DataType]),
Option(ctx.comment).map(string),
Option(ctx.commentSpec()).map(visitCommentSpec),
Option(ctx.colPosition).map(typedVisit[ColumnPosition]))
}

Expand Down Expand Up @@ -2896,7 +2920,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
typedVisit[Seq[String]](ctx.table),
columnNameParts,
Option(ctx.colType().dataType()).map(typedVisit[DataType]),
Option(ctx.colType().STRING()).map(string),
Option(ctx.colType().commentSpec()).map(visitCommentSpec),
Option(ctx.colPosition).map(typedVisit[ColumnPosition]))
}

Expand Down Expand Up @@ -3305,13 +3329,13 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
operationNotAllowed("CREATE VIEW ... PARTITIONED ON", ctx)
}

checkDuplicateClauses(ctx.COMMENT, "COMMENT", ctx)
checkDuplicateClauses(ctx.commentSpec(), "COMMENT", ctx)
checkDuplicateClauses(ctx.PARTITIONED, "PARTITIONED ON", ctx)
checkDuplicateClauses(ctx.TBLPROPERTIES, "TBLPROPERTIES", ctx)

val userSpecifiedColumns = Option(ctx.identifierCommentList).toSeq.flatMap { icl =>
icl.identifierComment.asScala.map { ic =>
ic.identifier.getText -> Option(ic.STRING).map(string)
ic.identifier.getText -> Option(ic.commentSpec()).map(visitCommentSpec)
}
}

Expand All @@ -3325,7 +3349,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
CreateViewStatement(
visitMultipartIdentifier(ctx.multipartIdentifier),
userSpecifiedColumns,
ctx.STRING.asScala.headOption.map(string),
visitCommentSpecList(ctx.commentSpec()),
ctx.tablePropertyList.asScala.headOption.map(visitPropertyKeyValues)
.getOrElse(Map.empty),
Option(source(ctx.query)),
Expand Down Expand Up @@ -3431,7 +3455,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
}

override def visitCommentNamespace(ctx: CommentNamespaceContext): LogicalPlan = withOrigin(ctx) {
val comment = ctx.commennt.getType match {
val comment = ctx.comment.getType match {
case SqlBaseParser.NULL => ""
case _ => string(ctx.STRING)
}
Expand All @@ -3440,7 +3464,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
}

override def visitCommentTable(ctx: CommentTableContext): LogicalPlan = withOrigin(ctx) {
val comment = ctx.commennt.getType match {
val comment = ctx.comment.getType match {
case SqlBaseParser.NULL => ""
case _ => string(ctx.STRING)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -151,7 +151,7 @@ class ParserUtilsSuite extends SparkFunSuite {

test("string") {
assert(string(showDbsContext.pattern) == "identifier_with_wildcards")
assert(string(createDbContext.comment) == "database_comment")
assert(string(createDbContext.commentSpec().get(0).STRING()) == "database_comment")

assert(string(createDbContext.locationSpec.asScala.head.STRING) == "/home/user/db")
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -368,7 +368,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {

checkDuplicateClauses(ctx.TBLPROPERTIES, "TBLPROPERTIES", ctx)
checkDuplicateClauses(ctx.PARTITIONED, "PARTITIONED BY", ctx)
checkDuplicateClauses(ctx.COMMENT, "COMMENT", ctx)
checkDuplicateClauses(ctx.commentSpec(), "COMMENT", ctx)
checkDuplicateClauses(ctx.bucketSpec(), "CLUSTERED BY", ctx)
checkDuplicateClauses(ctx.createFileFormat, "STORED AS/BY", ctx)
checkDuplicateClauses(ctx.rowFormat, "ROW FORMAT", ctx)
Expand All @@ -391,7 +391,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
.getOrElse(CatalogStorageFormat.empty)
val rowStorage = ctx.rowFormat.asScala.headOption.map(visitRowFormat)
.getOrElse(CatalogStorageFormat.empty)
val location = ctx.locationSpec.asScala.headOption.map(visitLocationSpec)
val location = visitLocationSpecList(ctx.locationSpec())
// If we are creating an EXTERNAL table, then the LOCATION field is required
if (external && location.isEmpty) {
operationNotAllowed("CREATE EXTERNAL TABLE must be accompanied by LOCATION", ctx)
Expand Down Expand Up @@ -425,7 +425,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
provider = Some(DDLUtils.HIVE_PROVIDER),
partitionColumnNames = partitionCols.map(_.name),
properties = properties,
comment = Option(ctx.comment).map(string))
comment = visitCommentSpecList(ctx.commentSpec()))

val mode = if (ifNotExists) SaveMode.Ignore else SaveMode.ErrorIfExists

Expand Down Expand Up @@ -504,7 +504,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
checkDuplicateClauses(ctx.locationSpec, "LOCATION", ctx)
checkDuplicateClauses(ctx.TBLPROPERTIES, "TBLPROPERTIES", ctx)
val provider = ctx.tableProvider.asScala.headOption.map(_.multipartIdentifier.getText)
val location = ctx.locationSpec.asScala.headOption.map(visitLocationSpec)
val location = visitLocationSpecList(ctx.locationSpec())
// rowStorage used to determine CatalogStorageFormat.serde and
// CatalogStorageFormat.properties in STORED AS clause.
val rowStorage = ctx.rowFormat.asScala.headOption.map(visitRowFormat)
Expand Down