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 @@ -149,10 +149,10 @@ statement
partitionSpec+ #addTablePartition
| ALTER TABLE tableIdentifier
from=partitionSpec RENAME TO to=partitionSpec #renameTablePartition
| ALTER TABLE tableIdentifier
DROP (IF EXISTS)? partitionSpec (',' partitionSpec)* PURGE? #dropTablePartitions
| ALTER VIEW tableIdentifier
DROP (IF EXISTS)? partitionSpec (',' partitionSpec)* #dropTablePartitions
| ALTER TABLE tableIdentifier DROP (IF EXISTS)?
dropPartitionSpec (',' dropPartitionSpec)* PURGE? #dropTablePartitions
| ALTER VIEW tableIdentifier DROP (IF EXISTS)?
dropPartitionSpec (',' dropPartitionSpec)* #dropTablePartitions
| ALTER TABLE multipartIdentifier SET locationSpec #setTableLocation
| ALTER TABLE tableIdentifier partitionSpec SET locationSpec #setPartitionLocation
| ALTER TABLE tableIdentifier RECOVER PARTITIONS #recoverPartitions
Expand Down Expand Up @@ -300,6 +300,14 @@ partitionVal
: identifier (EQ constant)?
;

dropPartitionSpec
: PARTITION '(' dropPartitionVal (',' dropPartitionVal)* ')'
;

dropPartitionVal
: identifier (comparisonOperator constant)?
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It has to be in this format? partCol1 > 2 How about 2 > partCol1?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes, in Hive it has to be like this. 2 > partCol1 is not supported by Hive.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hive also throws antler errors for the case 2 > partCol1?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hive does throw an error in that case, you mean asking that error is a parsing or another kind of exception?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yea, yes. I like user-understandable error messages.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hive throws this parser exception:

hive> alter table test1 drop partition(1 > c);
NoViableAltException(368@[])
	at org.apache.hadoop.hive.ql.parse.HiveParser_IdentifiersParser.identifier(HiveParser_IdentifiersParser.java:12014)
	at org.apache.hadoop.hive.ql.parse.HiveParser_IdentifiersParser.dropPartitionVal(HiveParser_IdentifiersParser.java:11684)
	at org.apache.hadoop.hive.ql.parse.HiveParser_IdentifiersParser.dropPartitionSpec(HiveParser_IdentifiersParser.java:11563)
	at org.apache.hadoop.hive.ql.parse.HiveParser.dropPartitionSpec(HiveParser.java:44851)
	at org.apache.hadoop.hive.ql.parse.HiveParser.alterStatementSuffixDropPartitions(HiveParser.java:11564)
	at org.apache.hadoop.hive.ql.parse.HiveParser.alterTableStatementSuffix(HiveParser.java:8000)
	at org.apache.hadoop.hive.ql.parse.HiveParser.alterStatement(HiveParser.java:7450)
	at org.apache.hadoop.hive.ql.parse.HiveParser.ddlStatement(HiveParser.java:4340)
	at org.apache.hadoop.hive.ql.parse.HiveParser.execStatement(HiveParser.java:2497)
	at org.apache.hadoop.hive.ql.parse.HiveParser.statement(HiveParser.java:1423)
	at org.apache.hadoop.hive.ql.parse.ParseDriver.parse(ParseDriver.java:209)
	at org.apache.hadoop.hive.ql.parse.ParseUtils.parse(ParseUtils.java:74)
	at org.apache.hadoop.hive.ql.parse.ParseUtils.parse(ParseUtils.java:67)
	at org.apache.hadoop.hive.ql.Driver.compile(Driver.java:615)
	at org.apache.hadoop.hive.ql.Driver.compileInternal(Driver.java:1829)
	at org.apache.hadoop.hive.ql.Driver.compileAndRespond(Driver.java:1776)
	at org.apache.hadoop.hive.ql.Driver.compileAndRespond(Driver.java:1771)
	at org.apache.hadoop.hive.ql.reexec.ReExecDriver.compileAndRespond(ReExecDriver.java:126)
	at org.apache.hadoop.hive.ql.reexec.ReExecDriver.run(ReExecDriver.java:214)
	at org.apache.hadoop.hive.cli.CliDriver.processLocalCmd(CliDriver.java:239)
	at org.apache.hadoop.hive.cli.CliDriver.processCmd(CliDriver.java:188)
	at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:402)
	at org.apache.hadoop.hive.cli.CliDriver.executeDriver(CliDriver.java:832)
	at org.apache.hadoop.hive.cli.CliDriver.run(CliDriver.java:770)
	at org.apache.hadoop.hive.cli.CliDriver.main(CliDriver.java:694)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at org.apache.hadoop.util.RunJar.run(RunJar.java:221)
	at org.apache.hadoop.util.RunJar.main(RunJar.java:136)
FAILED: ParseException line 1:33 cannot recognize input near '1' '>' 'c' in drop partition statement

so yes, it is analogous to this.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

thanks for the check. I still like meaningful messages though, we shold wait for other reviewer's comments.

;

database
: DATABASE
| SCHEMA
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -391,6 +391,35 @@ case class OuterReference(e: NamedExpression)
override def newInstance(): NamedExpression = OuterReference(e.newInstance())
}

/**
* A place holder used to hold the name of the partition attributes specified when running commands
* involving partitions, eg. ALTER TABLE ... DROP PARTITIONS.
*/
case class PartitioningAttribute(
name: String,
override val exprId: ExprId = NamedExpression.newExprId)
extends Attribute with Unevaluable {
// We need a dataType to be used during analysis for resolving the expressions (see
// checkInputDataTypes). The String type is used because all the literals in PARTITION operations
// are parsed as strings and eventually casted later.
override def dataType: DataType = StringType
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If it's not needed, can we throw exception here? We may need to override toString though.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Probably I should improve the comment then. it's misleading: this is actually needed because otherwise we may hit exceptions since the dataType is checked when running checkInputDataTypes of the comparison operator containing it. I'll improve the comment.

override def nullable: Boolean = false

override def qualifier: Seq[String] = throw new UnsupportedOperationException
override def withNullability(newNullability: Boolean): Attribute =
throw new UnsupportedOperationException
override def newInstance(): Attribute = throw new UnsupportedOperationException
override def withQualifier(newQualifier: Seq[String]): Attribute =
throw new UnsupportedOperationException
override def withName(newName: String): Attribute = throw new UnsupportedOperationException
override def withMetadata(newMetadata: Metadata): Attribute =
throw new UnsupportedOperationException

override lazy val canonicalized: Expression = this.copy(exprId = ExprId(0))

override def withExprId(newExprId: ExprId): Attribute = throw new UnsupportedOperationException
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

override def sql: String = name

Can make the error message more clear.

}

object VirtualColumn {
// The attribute name used by Hive, which has different result than Spark, deprecated.
val hiveGroupingIdName: String = "grouping__id"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -352,6 +352,29 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
}
}

/**
* Create a partition specification map with filters.
*/
override def visitDropPartitionSpec(
ctx: DropPartitionSpecContext): Seq[Expression] = withOrigin(ctx) {
ctx.dropPartitionVal().asScala.map { pFilter =>
if (pFilter.constant() == null || pFilter.comparisonOperator() == null) {
throw new ParseException(s"Invalid partition spec: ${pFilter.getText}", ctx)
}
// We cannot use UnresolvedAttribute because resolution is performed after Analysis, when
// running the command.
val partition = PartitioningAttribute(pFilter.identifier().getText)
val value = Literal(visitStringConstant(pFilter.constant()))
val operator = pFilter.comparisonOperator().getChild(0).asInstanceOf[TerminalNode]
val comparison = buildComparison(partition, value, operator)
if (comparison.isInstanceOf[EqualNullSafe]) {
throw new ParseException(
"'<=>' operator is not supported in ALTER TABLE ... DROP PARTITION.", ctx)
}
comparison
}
}

/**
* Convert a constant of any type into a string. This is typically used in DDL commands, and its
* main purpose is to prevent slight differences due to back to back conversions i.e.:
Expand Down Expand Up @@ -1174,6 +1197,23 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
val left = expression(ctx.left)
val right = expression(ctx.right)
val operator = ctx.comparisonOperator().getChild(0).asInstanceOf[TerminalNode]
buildComparison(left, right, operator)
}

/**
* Creates a comparison expression. The following comparison operators are supported:
* - Equal: '=' or '=='
* - Null-safe Equal: '<=>'
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Seems we can't support null-safe equality because it is not supported by Hive metastore partition predicate pushdown. See HiveShim.scala.

* - Not Equal: '<>' or '!='
* - Less than: '<'
* - Less then or Equal: '<='
* - Greater than: '>'
* - Greater then or Equal: '>='
*/
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hive also supports all the comparators above?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes, it does

private def buildComparison(
left: Expression,
right: Expression,
operator: TerminalNode): Expression = {
operator.getSymbol.getType match {
case SqlBaseParser.EQ =>
EqualTo(left, right)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -749,7 +749,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
}
AlterTableDropPartitionCommand(
visitTableIdentifier(ctx.tableIdentifier),
ctx.partitionSpec.asScala.map(visitNonOptionalPartitionSpec),
ctx.dropPartitionSpec().asScala.map(visitDropPartitionSpec),
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you update the comment?:

* ALTER TABLE table DROP [IF EXISTS] PARTITION spec1[, PARTITION spec2, ...] [PURGE];

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

mmmh, I am not sure how to update it. The only difference is that specN now can be any kind of filter instead of just partitionColumn = value. So it is actually the definition of specN which changed.

ifExists = ctx.EXISTS != null,
purge = ctx.PURGE != null,
retainData = false)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,10 +31,10 @@ import org.apache.hadoop.mapred.{FileInputFormat, JobConf}
import org.apache.spark.internal.config.RDD_PARALLEL_LISTING_THRESHOLD
import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.analysis.Resolver
import org.apache.spark.sql.catalyst.analysis.{Resolver, UnresolvedAttribute}
import org.apache.spark.sql.catalyst.catalog._
import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
import org.apache.spark.sql.catalyst.expressions._
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

too many imports?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

not sure what you mean here. The list of imports would be very long, as I use, EqualTo, And, Literal, Cast, BinaryComparison, etc. I can list all them, but I am not sure it is worth. What do you think?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I just wanted to check if your IDE wrongly folded this import, or not. It's ok.

import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PartitioningUtils}
import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat
Expand Down Expand Up @@ -524,35 +524,114 @@ case class AlterTableRenamePartitionCommand(
*/
case class AlterTableDropPartitionCommand(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Shall we make table relation as a child? then we can resolve the partitionsFilters automatically.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I thought about that. The point is that we have anyway to check that the attributes specified are the partitioning ones. So I am not sure it is worth to run the whole analyzer rules for something we have anyway to handle somehow.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

But it's also weird to use AttributeReference this way. Can we create a new Attribute implementation for this purpose? Basically we only need a resolved expression to hold the partition column name. The type doesn't matter.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sure, I will. Thanks.

tableName: TableIdentifier,
specs: Seq[TablePartitionSpec],
partitionsFilters: Seq[Seq[Expression]],
ifExists: Boolean,
purge: Boolean,
retainData: Boolean)
extends RunnableCommand {

override def run(sparkSession: SparkSession): Seq[Row] = {
val catalog = sparkSession.sessionState.catalog
val timeZone = Option(sparkSession.sessionState.conf.sessionLocalTimeZone)
val table = catalog.getTableMetadata(tableName)
val partitionColumns = table.partitionColumnNames
val partitionAttributes = table.partitionSchema.toAttributes.map(a => a.name -> a).toMap
DDLUtils.verifyAlterTableType(catalog, table, isView = false)
DDLUtils.verifyPartitionProviderIsHive(sparkSession, table, "ALTER TABLE DROP PARTITION")

val normalizedSpecs = specs.map { spec =>
PartitioningUtils.normalizePartitionSpec(
spec,
table.partitionColumnNames,
table.identifier.quotedString,
sparkSession.sessionState.conf.resolver)
val resolvedSpecs = partitionsFilters.flatMap { filtersSpec =>
if (hasComplexFilters(filtersSpec)) {
generatePartitionSpec(filtersSpec,
partitionColumns,
partitionAttributes,
table.identifier,
catalog,
sparkSession.sessionState.conf.resolver,
timeZone,
ifExists)
} else {
val partitionSpec = filtersSpec.map {
case EqualTo(key: Attribute, Literal(value, StringType)) =>
key.name -> value.toString
}.toMap
PartitioningUtils.normalizePartitionSpec(
partitionSpec,
partitionColumns,
table.identifier.quotedString,
sparkSession.sessionState.conf.resolver) :: Nil
}
}

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should check resolvedSpecs here to throw error message if total resolved spec is empty.

catalog.dropPartitions(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

does hive have an API to drop partitions with a predicate? I think the current approach is very inefficient with non-equal partition predicates.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

So the implementation here is similar to how hive implements it?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, this is my understanding. You can check DDLTaks.dropPartitions.

table.identifier, normalizedSpecs, ignoreIfNotExists = ifExists, purge = purge,
table.identifier, resolvedSpecs, ignoreIfNotExists = ifExists, purge = purge,
retainData = retainData)

CommandUtils.updateTableStats(sparkSession, table)

Seq.empty[Row]
}

def hasComplexFilters(partitionFilterSpec: Seq[Expression]): Boolean = {
partitionFilterSpec.exists(!_.isInstanceOf[EqualTo])
}

def generatePartitionSpec(
partitionFilterSpec: Seq[Expression],
partitionColumns: Seq[String],
partitionAttributes: Map[String, Attribute],
tableIdentifier: TableIdentifier,
catalog: SessionCatalog,
resolver: Resolver,
timeZone: Option[String],
ifExists: Boolean): Seq[TablePartitionSpec] = {
val filters = partitionFilterSpec.map { pFilter =>
pFilter.transform {
// Resolve the partition attributes
case partitionCol: PartitioningAttribute =>
val normalizedPartition = PartitioningUtils.normalizePartitionColumn(
partitionCol.name,
partitionColumns,
tableIdentifier.quotedString,
resolver)
partitionAttributes(normalizedPartition)
}.transform {
// Cast the partition value to the data type of the corresponding partition attribute
case cmp @ BinaryComparison(partitionAttr, value)
if !partitionAttr.dataType.sameType(value.dataType) =>
val dt = partitionAttr.dataType
val lit = Literal(Cast(value, dt, timeZone).eval(), dt)
cmp.withNewChildren(Seq(partitionAttr, lit))
}
}
val partitions = catalog.listPartitionsByFilter(tableIdentifier, filters)
if (partitions.isEmpty && !ifExists) {
throw new AnalysisException(s"There is no partition for ${filters.reduceLeft(And).sql}")
}
partitions.map(_.spec)
}
}


object AlterTableDropPartitionCommand {

def fromSpecs(
tableName: TableIdentifier,
specs: Seq[TablePartitionSpec],
ifExists: Boolean,
purge: Boolean,
retainData: Boolean): AlterTableDropPartitionCommand = {
AlterTableDropPartitionCommand(tableName,
specs.map(tablePartitionToPartitionFilters),
ifExists,
purge,
retainData)
}

def tablePartitionToPartitionFilters(spec: TablePartitionSpec): Seq[Expression] = {
spec.map {
case (key, value) => EqualTo(PartitioningAttribute(key), Literal(value))
}.toSeq
}
}


Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -148,7 +148,7 @@ case class InsertIntoHadoopFsRelationCommand(
if (mode == SaveMode.Overwrite && !dynamicPartitionOverwrite) {
val deletedPartitions = initialMatchingPartitions.toSet -- updatedPartitions
if (deletedPartitions.nonEmpty) {
AlterTableDropPartitionCommand(
AlterTableDropPartitionCommand.fromSpecs(
catalogTable.get.identifier, deletedPartitions.toSeq,
ifExists = true, purge = false,
retainData = true /* already deleted */).run(sparkSession)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -366,9 +366,7 @@ object PartitioningUtils {
tblName: String,
resolver: Resolver): Map[String, T] = {
val normalizedPartSpec = partitionSpec.toSeq.map { case (key, value) =>
val normalizedKey = partColNames.find(resolver(_, key)).getOrElse {
throw new AnalysisException(s"$key is not a valid partition column in table $tblName.")
}
val normalizedKey = normalizePartitionColumn(key, partColNames, tblName, resolver)
normalizedKey -> value
}

Expand All @@ -378,6 +376,16 @@ object PartitioningUtils {
normalizedPartSpec.toMap
}

def normalizePartitionColumn(
partition: String,
partColNames: Seq[String],
tblName: String,
resolver: Resolver): String = {
partColNames.find(resolver(_, partition)).getOrElse {
throw new AnalysisException(s"$partition is not a valid partition column in table $tblName.")
}
}

/**
* Resolves possible type conflicts between partitions by up-casting "lower" types using
* [[findWiderTypeForPartitionColumn]].
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.catalog._
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans
import org.apache.spark.sql.catalyst.dsl.plans.DslLogicalPlan
import org.apache.spark.sql.catalyst.expressions.JsonTuple
import org.apache.spark.sql.catalyst.expressions.{Expression, JsonTuple, PartitioningAttribute}
import org.apache.spark.sql.catalyst.parser.ParseException
import org.apache.spark.sql.catalyst.plans.logical.{Generate, InsertIntoDir, LogicalPlan, Project, ScriptTransformation}
import org.apache.spark.sql.execution.SparkSqlParser
Expand Down Expand Up @@ -687,7 +687,8 @@ class DDLParserSuite extends AnalysisTest with SharedSQLContext {
assertUnsupported(sql2_view)

val tableIdent = TableIdentifier("table_name", None)
val expected1_table = AlterTableDropPartitionCommand(

val expected1_table = AlterTableDropPartitionCommand.fromSpecs(
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you add tests case to check if the parser can accept the comparators added by this pr?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sure, will do, thanks.

tableIdent,
Seq(
Map("dt" -> "2008-08-08", "country" -> "us"),
Expand All @@ -698,9 +699,36 @@ class DDLParserSuite extends AnalysisTest with SharedSQLContext {
val expected2_table = expected1_table.copy(ifExists = false)
val expected1_purge = expected1_table.copy(purge = true)

comparePlans(parsed1_table, expected1_table)
comparePlans(parsed2_table, expected2_table)
comparePlans(parsed1_purge, expected1_purge)
comparePlans(parsed1_table.canonicalized, expected1_table.canonicalized)
comparePlans(parsed2_table.canonicalized, expected2_table.canonicalized)
comparePlans(parsed1_purge.canonicalized, expected1_purge.canonicalized)

// SPARK-23866: Support any comparison operator in ALTER TABLE ... DROP PARTITION
Seq((">", (a: Expression, b: Expression) => a > b),
(">=", (a: Expression, b: Expression) => a >= b),
("<", (a: Expression, b: Expression) => a < b),
("<=", (a: Expression, b: Expression) => a <= b),
("<>", (a: Expression, b: Expression) => a =!= b),
("!=", (a: Expression, b: Expression) => a =!= b)).foreach { case (op, predicateGen) =>
val genPlan = parser.parsePlan(sql1_table.replace("=", op))
val dtAttr = PartitioningAttribute("dt")
val countryAttr = PartitioningAttribute("country")
val expectedPlan = AlterTableDropPartitionCommand(
tableIdent,
Seq(
Seq(predicateGen(dtAttr, "2008-08-08"), predicateGen(countryAttr, "us")),
Seq(predicateGen(dtAttr, "2009-09-09"), predicateGen(countryAttr, "uk"))),
ifExists = true,
purge = false,
retainData = false)
comparePlans(genPlan.canonicalized, expectedPlan.canonicalized)
}

// SPARK-23866: <=> is not supported
intercept("ALTER TABLE table_name DROP PARTITION (dt <=> 'a')", "operator is not supported in")

// SPARK-23866: Invalid partition specification
intercept("ALTER TABLE table_name DROP PARTITION (dt)", "Invalid partition spec:")
}

test("alter table: archive partition (not supported)") {
Expand Down
Loading