Skip to content
6 changes: 0 additions & 6 deletions common/utils/src/main/resources/error/error-conditions.json
Original file line number Diff line number Diff line change
Expand Up @@ -4252,12 +4252,6 @@
],
"sqlState" : "42883"
},
"UNSET_NONEXISTENT_PROPERTIES" : {
"message" : [
"Attempted to unset non-existent properties [<properties>] in table <table>."
],
"sqlState" : "42K0J"
},
"UNSUPPORTED_ADD_FILE" : {
"message" : [
"Don't support add file."
Expand Down
18 changes: 13 additions & 5 deletions docs/sql-ref-syntax-ddl-alter-table.md
Original file line number Diff line number Diff line change
Expand Up @@ -236,20 +236,28 @@ ALTER TABLE table_identifier DROP [ IF EXISTS ] partition_spec [PURGE]

### SET AND UNSET

#### SET TABLE PROPERTIES
#### SET PROPERTIES

`ALTER TABLE SET` command is used for setting the table properties. If a particular property was already set,
this overrides the old value with the new one.

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

##### Syntax

```sql
-- Set Table Properties
-- Set Properties
ALTER TABLE table_identifier SET TBLPROPERTIES ( key1 = val1, key2 = val2, ... )
```

#### UNSET PROPERTIES

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

-- Unset Table Properties
**Note:** If the specified property key does not exist, whether specify `IF EXISTS` or not, the command will ignore it and finally succeed.

##### Syntax

```sql
-- Unset Properties
ALTER TABLE table_identifier UNSET TBLPROPERTIES [ IF EXISTS ] ( key1, key2, ... )
Copy link
Contributor

Choose a reason for hiding this comment

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

let's remove [ IF EXISTS ] as a way to deprecate. It's no longer effective.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done.

```

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2658,16 +2658,6 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
"config" -> SQLConf.ALLOW_NON_EMPTY_LOCATION_IN_CTAS.key))
}

def unsetNonExistentPropertiesError(
properties: Seq[String], table: TableIdentifier): Throwable = {
new AnalysisException(
errorClass = "UNSET_NONEXISTENT_PROPERTIES",
messageParameters = Map(
"properties" -> properties.map(toSQLId).mkString(", "),
"table" -> toSQLId(table.nameParts))
)
}

def alterTableChangeColumnNotSupportedForColumnTypeError(
tableName: String,
originColumn: StructField,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1075,25 +1075,6 @@ class DDLParserSuite extends AnalysisTest {
ifExists = true))
}

// ALTER TABLE table_name UNSET TBLPROPERTIES [IF EXISTS] ('comment', 'key');
Copy link
Contributor Author

Choose a reason for hiding this comment

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

The tests have been migrated to AlterTableUnsetTblPropertiesParserSuite

test("alter table: alter table properties") {
val sql2_table = "ALTER TABLE table_name UNSET TBLPROPERTIES ('comment', 'test')"
val sql3_table = "ALTER TABLE table_name UNSET TBLPROPERTIES IF EXISTS ('comment', 'test')"

comparePlans(
parsePlan(sql2_table),
UnsetTableProperties(
UnresolvedTable(Seq("table_name"), "ALTER TABLE ... UNSET TBLPROPERTIES", true),
Seq("comment", "test"),
ifExists = false))
comparePlans(
parsePlan(sql3_table),
UnsetTableProperties(
UnresolvedTable(Seq("table_name"), "ALTER TABLE ... UNSET TBLPROPERTIES", true),
Seq("comment", "test"),
ifExists = true))
}

test("alter table: add column") {
comparePlans(
parsePlan("ALTER TABLE table_name ADD COLUMN x int"),
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 @@ -19,6 +19,10 @@ package org.apache.spark.sql.execution.command

import org.apache.spark.sql.{AnalysisException, QueryTest}
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.parser.ParseException
import org.apache.spark.sql.connector.catalog.{CatalogV2Util, TableCatalog}
import org.apache.spark.sql.errors.DataTypeErrors.toSQLId
import org.apache.spark.sql.internal.SQLConf

/**
* This base suite contains unified tests for the `ALTER TABLE .. SET TBLPROPERTIES`
Expand All @@ -39,7 +43,26 @@ trait AlterTableSetTblPropertiesSuiteBase extends QueryTest with DDLCommandTestU

def checkTblProps(tableIdent: TableIdentifier, expectedTblProps: Map[String, String]): Unit

test("alter table set tblproperties") {
def getTblPropertyValue(tableIdent: TableIdentifier, key: String): String

test("table to alter does not exist") {
withNamespaceAndTable("ns", "does_not_exist") { t =>
val sqlText = s"ALTER TABLE $t SET TBLPROPERTIES ('k1' = 'v1')"
checkError(
exception = intercept[AnalysisException] {
sql(sqlText)
},
errorClass = "TABLE_OR_VIEW_NOT_FOUND",
parameters = Map("relationName" -> toSQLId(t)),
context = ExpectedContext(
fragment = t,
start = 12,
stop = 11 + t.length)
)
}
}

test("alter table set 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))
Expand All @@ -54,16 +77,53 @@ trait AlterTableSetTblPropertiesSuiteBase extends QueryTest with DDLCommandTestU

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

// table to alter does not exist
checkError(
exception = intercept[AnalysisException] {
sql("ALTER TABLE does_not_exist SET TBLPROPERTIES ('winner' = 'loser')")
},
errorClass = "TABLE_OR_VIEW_NOT_FOUND",
parameters = Map("relationName" -> "`does_not_exist`"),
context = ExpectedContext(fragment = "does_not_exist", start = 12, stop = 25)
)
test("alter table set reserved properties") {
import TableCatalog._
val keyParameters = Map[String, String](
PROP_PROVIDER -> "please use the USING clause to specify it",
PROP_LOCATION -> "please use the LOCATION clause to specify it",
PROP_OWNER -> "it will be set to the current user",
PROP_EXTERNAL -> "please use CREATE EXTERNAL TABLE"
)
withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "false")) {
CatalogV2Util.TABLE_RESERVED_PROPERTIES.filterNot(_ == PROP_COMMENT).foreach { key =>
withNamespaceAndTable("ns", "tbl") { t =>
val sqlText = s"ALTER TABLE $t SET TBLPROPERTIES ('$key'='bar')"
checkError(
exception = intercept[ParseException] {
sql(sqlText)
},
errorClass = "UNSUPPORTED_FEATURE.SET_TABLE_PROPERTY",
parameters = Map(
"property" -> key,
"msg" -> keyParameters.getOrElse(
key, "please remove it from the TBLPROPERTIES list.")),
context = ExpectedContext(
fragment = sqlText,
start = 0,
stop = 40 + t.length + key.length))
}
}
}
withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "true")) {
CatalogV2Util.TABLE_RESERVED_PROPERTIES.filterNot(_ == PROP_COMMENT).foreach { key =>
Seq("OPTIONS", "TBLPROPERTIES").foreach { clause =>
withNamespaceAndTable("ns", "tbl") { t =>
sql(s"CREATE TABLE $t (key int) USING parquet $clause ('$key'='bar')")
val tableIdent = TableIdentifier("tbl", Some("ns"), Some(catalog))

val originValue = getTblPropertyValue(tableIdent, key)
assert(originValue != "bar", "reserved properties should not have side effects")

sql(s"ALTER TABLE $t SET TBLPROPERTIES ('$key'='newValue')")
assert(getTblPropertyValue(tableIdent, key) == originValue,
"reserved properties should not have side effects")
}
}
}
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,65 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

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

import org.apache.spark.SparkThrowable
import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedTable}
import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan
import org.apache.spark.sql.catalyst.parser.ParseException
import org.apache.spark.sql.catalyst.plans.logical.UnsetTableProperties
import org.apache.spark.sql.test.SharedSparkSession

class AlterTableUnsetTblPropertiesParserSuite extends AnalysisTest with SharedSparkSession {

private def parseException(sqlText: String): SparkThrowable = {
intercept[ParseException](sql(sqlText).collect())
}

// ALTER TABLE table_name UNSET TBLPROPERTIES [IF EXISTS] ('comment', 'key');
test("alter table: alter table properties") {
val sql1 = "ALTER TABLE table_name UNSET TBLPROPERTIES ('comment', 'test')"
val sql2 = "ALTER TABLE table_name UNSET TBLPROPERTIES IF EXISTS ('comment', 'test')"

comparePlans(
parsePlan(sql1),
UnsetTableProperties(
UnresolvedTable(Seq("table_name"), "ALTER TABLE ... UNSET TBLPROPERTIES",
suggestAlternative = true),
Seq("comment", "test"),
ifExists = false))
comparePlans(
parsePlan(sql2),
UnsetTableProperties(
UnresolvedTable(Seq("table_name"), "ALTER TABLE ... UNSET TBLPROPERTIES",
suggestAlternative = true),
Seq("comment", "test"),
ifExists = true))
}

test("alter table unset properties - property values must NOT be set") {
val sql = "ALTER TABLE my_tab UNSET TBLPROPERTIES('key_without_value', 'key_with_value'='x')"
checkError(
exception = parseException(sql),
errorClass = "_LEGACY_ERROR_TEMP_0035",
parameters = Map("message" -> "Values should not be specified for key(s): [key_with_value]"),
context = ExpectedContext(
fragment = sql,
start = 0,
stop = 80))
}
}
Loading