Skip to content

Commit

Permalink
fix mistakes (#9)
Browse files Browse the repository at this point in the history
  • Loading branch information
cloud-fan authored Nov 24, 2020
1 parent a471f33 commit 25ec746
Show file tree
Hide file tree
Showing 4 changed files with 22 additions and 11 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -2952,8 +2952,9 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
protected def getSerdeInfo(
rowFormatCtx: Seq[RowFormatContext],
createFileFormatCtx: Seq[CreateFileFormatContext],
ctx: ParserRuleContext): Option[SerdeInfo] = {
validateRowFormatFileFormat(rowFormatCtx, createFileFormatCtx, ctx)
ctx: ParserRuleContext,
skipCheck: Boolean = false): Option[SerdeInfo] = {
if (!skipCheck) validateRowFormatFileFormat(rowFormatCtx, createFileFormatCtx, ctx)
val rowFormatSerdeInfo = rowFormatCtx.map(visitRowFormat)
val fileFormatSerdeInfo = createFileFormatCtx.map(visitCreateFileFormat)
(fileFormatSerdeInfo ++ rowFormatSerdeInfo).reduceLeftOption((l, r) => l.merge(r))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -297,7 +297,7 @@ class ResolveSessionCatalog(
assertNoNullTypeInSchema(c.asSelect.schema)
}
val (storageFormat, provider) = getStorageFormatAndProvider(
c.provider, c.options, c.location, c.serde, ctas = false)
c.provider, c.options, c.location, c.serde, ctas = true)
if (!isV2Provider(provider)) {
val tableDesc = buildCatalogTable(tbl.asTableIdentifier, new StructType,
c.partitioning, c.bucketSpec, c.properties, provider, c.location,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -438,13 +438,23 @@ class SparkSqlAstBuilder extends AstBuilder {
checkDuplicateClauses(ctx.TBLPROPERTIES, "TBLPROPERTIES", ctx)
val provider = ctx.tableProvider.asScala.headOption.map(_.multipartIdentifier.getText)
val location = visitLocationSpecList(ctx.locationSpec())
// rowStorage used to determine CatalogStorageFormat.serde and
// CatalogStorageFormat.properties in STORED AS clause.
val serdeInfo = getSerdeInfo(ctx.rowFormat.asScala, ctx.createFileFormat.asScala, ctx)
// TODO: Do not skip serde check for CREATE TABLE LIKE.
val serdeInfo = getSerdeInfo(
ctx.rowFormat.asScala, ctx.createFileFormat.asScala, ctx, skipCheck = true)
if (provider.isDefined && serdeInfo.isDefined) {
operationNotAllowed(s"CREATE TABLE LIKE ... USING ... ${serdeInfo.get.describe}", ctx)
}

// TODO: remove this restriction as it seems unnecessary.
serdeInfo match {
case Some(SerdeInfo(storedAs, formatClasses, serde, _)) =>
if (storedAs.isEmpty && formatClasses.isEmpty && serde.isDefined) {
throw new ParseException("'ROW FORMAT' must be used with 'STORED AS'", ctx)
}
case _ =>
}

// TODO: also look at `HiveSerDe.getDefaultStorage`.
val storage = toStorageFormat(location, serdeInfo, ctx)
val properties = Option(ctx.tableProps).map(visitPropertyKeyValues).getOrElse(Map.empty)
CreateTableLikeCommand(
Expand Down Expand Up @@ -603,7 +613,8 @@ class SparkSqlAstBuilder extends AstBuilder {
*/
override def visitInsertOverwriteHiveDir(
ctx: InsertOverwriteHiveDirContext): InsertDirParams = withOrigin(ctx) {
val serdeInfo = getSerdeInfo(Seq(ctx.rowFormat), Seq(ctx.createFileFormat), ctx)
val serdeInfo = getSerdeInfo(
Option(ctx.rowFormat).toSeq, Option(ctx.createFileFormat).toSeq, ctx)
val path = string(ctx.path)
// The path field is required
if (path.isEmpty) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2780,7 +2780,7 @@ class HiveDDLSuite
|ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
""".stripMargin)
}.getMessage
assert(e.contains("'ROW FORMAT' must be used with 'STORED AS'"))
assert(e.contains("Operation not allowed: CREATE TABLE LIKE ... USING ... ROW FORMAT SERDE"))

// row format doesn't work with provider hive
e = intercept[AnalysisException] {
Expand All @@ -2791,7 +2791,7 @@ class HiveDDLSuite
|WITH SERDEPROPERTIES ('test' = 'test')
""".stripMargin)
}.getMessage
assert(e.contains("'ROW FORMAT' must be used with 'STORED AS'"))
assert(e.contains("Operation not allowed: CREATE TABLE LIKE ... USING ... ROW FORMAT SERDE"))

// row format doesn't work without 'STORED AS'
e = intercept[AnalysisException] {
Expand All @@ -2813,8 +2813,7 @@ class HiveDDLSuite
|ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
""".stripMargin)
}.getMessage
assert(e.contains(
"'INPUTFORMAT hiveFormat' and 'USING provider' should not be specified both"))
assert(e.contains("Operation not allowed: CREATE TABLE LIKE ... USING ... STORED AS"))

// row format works with STORED AS hive format (from hive table)
spark.sql(
Expand Down

0 comments on commit 25ec746

Please sign in to comment.