Skip to content
Prev Previous commit
Next Next commit
remove ifExists logical
  • Loading branch information
panbingkun committed Jul 3, 2024
commit 9a8dcf38d7b7cd47c75f6a3bba5fd035217e2040
4 changes: 1 addition & 3 deletions docs/sql-ref-syntax-ddl-alter-table.md
Original file line number Diff line number Diff line change
Expand Up @@ -252,9 +252,7 @@ ALTER TABLE table_identifier SET TBLPROPERTIES ( key1 = val1, key2 = val2, ... )

`ALTER TABLE UNSET` command is used to drop the table property.

**Note:** If the specified property key does not exist, when you use the v1 command and do not specify `IF EXISTS`,
it will throw the error-condition `UNSET_NONEXISTENT_PROPERTIES` and finally `failed`,
however regardless of whether `IF EXISTS` is set or not, the v2 command will ignore it and finally `succeed`.
**Note:** If the specified property key does not exist, whether specify `IF EXISTS` or not, the command will ignore it and finally succeed.

##### Syntax

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -335,14 +335,6 @@ case class AlterTableUnsetPropertiesCommand(
override def run(sparkSession: SparkSession): Seq[Row] = {
val catalog = sparkSession.sessionState.catalog
val table = catalog.getTableRawMetadata(tableName)
if (!ifExists) {
val nonexistentKeys = propKeys.filter(key => !table.properties.contains(key)
&& key != TableCatalog.PROP_COMMENT)
if (nonexistentKeys.nonEmpty) {
throw QueryCompilationErrors.unsetNonExistentPropertiesError(
nonexistentKeys, table.identifier)
}
}
// If comment is in the table property, we reset it to None
val tableComment = if (propKeys.contains(TableCatalog.PROP_COMMENT)) None else table.comment
val newProperties = table.properties.filter { case (k, _) => !propKeys.contains(k) }
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -82,6 +82,24 @@ trait AlterTableUnsetTblPropertiesSuiteBase extends QueryTest with DDLCommandTes
}
}

test("alter table unset non-existent properties") {
withNamespaceAndTable("ns", "tbl") { t =>
sql(s"CREATE TABLE $t (col1 int, col2 string, a int, b int) $defaultUsing")
val tableIdent = TableIdentifier("tbl", Some("ns"), Some(catalog))

sql(s"ALTER TABLE $t SET TBLPROPERTIES ('k1' = 'v1', 'k2' = 'v2', 'k3' = 'v3')")
checkTblProps(tableIdent, Map("k1" -> "v1", "k2" -> "v2", "k3" -> "v3"))

// property to unset does not exist
sql(s"ALTER TABLE $t UNSET TBLPROPERTIES ('k3', 'k4')")
checkTblProps(tableIdent, Map("k1" -> "v1", "k2" -> "v2"))

// property to unset does not exist, but "IF EXISTS" is specified
sql(s"ALTER TABLE $t UNSET TBLPROPERTIES IF EXISTS ('k2', 'k3')")
checkTblProps(tableIdent, Map("k1" -> "v1"))
}
}

test("alter table unset reserved properties") {
import TableCatalog._
val keyParameters = Map[String, String](
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,9 +17,7 @@

package org.apache.spark.sql.execution.command.v1

import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.errors.DataTypeErrors.toSQLId
import org.apache.spark.sql.execution.command
import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION

Expand Down Expand Up @@ -61,33 +59,6 @@ trait AlterTableUnsetTblPropertiesSuiteBase extends command.AlterTableUnsetTblPr
override def getTblPropertyValue(tableIdent: TableIdentifier, key: String): String = {
getTableProperties(tableIdent).getOrElse(key, null)
}

/**
* When using the v1 command to unset `non-existent` properties, the command will
* throw the error-condition `"UNSET_NONEXISTENT_PROPERTIES` and finally `failed`
*/
test("alter table unset non-existent properties") {
withNamespaceAndTable("ns", "tbl") { t =>
sql(s"CREATE TABLE $t (col1 int, col2 string, a int, b int) $defaultUsing")
val tableIdent = TableIdentifier("tbl", Some("ns"), Some(catalog))

sql(s"ALTER TABLE $t SET TBLPROPERTIES ('k1' = 'v1', 'k2' = 'v2', 'k3' = 'v3')")
checkTblProps(tableIdent, Map("k1" -> "v1", "k2" -> "v2", "k3" -> "v3"))

// property to unset does not exist
checkError(
exception = intercept[AnalysisException] {
sql(s"ALTER TABLE $t UNSET TBLPROPERTIES ('k3', 'k4')")
},
errorClass = "UNSET_NONEXISTENT_PROPERTIES",
parameters = Map("properties" -> "`k4`", "table" -> toSQLId(tableIdent.nameParts))
)

// property to unset does not exist, but "IF EXISTS" is specified
sql(s"ALTER TABLE $t UNSET TBLPROPERTIES IF EXISTS ('k3', 'k4')")
checkTblProps(tableIdent, Map("k1" -> "v1", "k2" -> "v2"))
}
}
}

class AlterTableUnsetTblPropertiesSuite
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,26 +51,4 @@ class AlterTableUnsetTblPropertiesSuite
override def getTblPropertyValue(tableIdent: TableIdentifier, key: String): String = {
getTableMetadata(tableIdent).properties.asScala.toMap.getOrElse(key, null)
}

/**
* When using the v2 command to unset `non-existent` properties,
* the command will ignore `non-existent` properties and finally succeed
*/
test("alter table unset non-existent properties") {
withNamespaceAndTable("ns", "tbl") { t =>
sql(s"CREATE TABLE $t (col1 int, col2 string, a int, b int) $defaultUsing")
val tableIdent = TableIdentifier("tbl", Some("ns"), Some(catalog))

sql(s"ALTER TABLE $t SET TBLPROPERTIES ('k1' = 'v1', 'k2' = 'v2', 'k3' = 'v3')")
checkTblProps(tableIdent, Map("k1" -> "v1", "k2" -> "v2", "k3" -> "v3"))

// property to unset does not exist
sql(s"ALTER TABLE $t UNSET TBLPROPERTIES ('k3', 'k4')")
checkTblProps(tableIdent, Map("k1" -> "v1", "k2" -> "v2"))

// property to unset does not exist, but "IF EXISTS" is specified
sql(s"ALTER TABLE $t UNSET TBLPROPERTIES IF EXISTS ('k2', 'k3')")
checkTblProps(tableIdent, Map("k1" -> "v1"))
}
}
}