From 58d957a66af9968843e93934955cb4f3f9525909 Mon Sep 17 00:00:00 2001 From: Udbhav30 Date: Sat, 8 Aug 2020 11:37:44 +0530 Subject: [PATCH 1/7] [SPARK-32481] Support truncate table to move data to trash --- .../scala/org/apache/spark/util/Utils.scala | 20 ++++++++++++++++++- .../apache/spark/sql/internal/SQLConf.scala | 9 +++++++++ .../spark/sql/execution/command/tables.scala | 4 +++- 3 files changed, 31 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 35d60bb514405..9040bd9012ce2 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -50,7 +50,7 @@ import com.google.common.net.InetAddresses import org.apache.commons.codec.binary.Hex import org.apache.commons.lang3.SystemUtils import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} +import org.apache.hadoop.fs.{FileSystem, FileUtil, Path, Trash} import org.apache.hadoop.io.compress.{CompressionCodecFactory, SplittableCompressionCodec} import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.conf.YarnConfiguration @@ -269,6 +269,24 @@ private[spark] object Utils extends Logging { file.setExecutable(true, true) } + /** + * Move data to trash on truncate table given + * spark.sql.truncate.trash.interval is positive + */ + def moveToTrashIfEnabled( + fs: FileSystem, + partitionPath: Path, + trashInterval: Int, + hadoopConf: Configuration): Unit = { + if (trashInterval < 0) { + fs.delete(partitionPath, true) + } else { + logDebug(s"will move data ${partitionPath.toString} to trash") + hadoopConf.setInt("fs.trash.interval", trashInterval) + Trash.moveToAppropriateTrash(fs, partitionPath, hadoopConf) + } + } + /** * Create a directory given the abstract pathname * @return true, if the directory is successfully created; otherwise, return false. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 3e82b8e12df02..b20cc5760d14a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2722,6 +2722,13 @@ object SQLConf { .booleanConf .createWithDefault(false) + val TRUNCATE_TRASH_INTERVAL = + buildConf("spark.sql.truncate.trash.interval") + .doc("This Configuration will decide whether move files to trash on truncate table" + + "If -1 files will be deleted without moving to trash") + .intConf + .createWithDefault(-1) + /** * Holds information about keys that have been deprecated. * @@ -3334,6 +3341,8 @@ class SQLConf extends Serializable with Logging { def legacyPathOptionBehavior: Boolean = getConf(SQLConf.LEGACY_PATH_OPTION_BEHAVIOR) + def truncateTrashInterval: Int = getConf(SQLConf.TRUNCATE_TRASH_INTERVAL) + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 7aebdddf1d59c..2b40291b26552 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -48,6 +48,7 @@ import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetDataSourceV2 import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} import org.apache.spark.sql.types._ import org.apache.spark.sql.util.SchemaUtils +import org.apache.spark.util.Utils /** * A command to create a table with the same definition of the given existing table. @@ -489,6 +490,7 @@ case class TruncateTableCommand( } val hadoopConf = spark.sessionState.newHadoopConf() val ignorePermissionAcl = SQLConf.get.truncateTableIgnorePermissionAcl + val trashInterval = SQLConf.get.truncateTrashInterval locations.foreach { location => if (location.isDefined) { val path = new Path(location.get) @@ -513,7 +515,7 @@ case class TruncateTableCommand( } } - fs.delete(path, true) + Utils.moveToTrashIfEnabled(fs, path, trashInterval, hadoopConf) // We should keep original permission/acl of the path. // For owner/group, only super-user can set it, for example on HDFS. Because From 81101d900ebb019a915b26cd52a2119cf85271e9 Mon Sep 17 00:00:00 2001 From: Udbhav30 Date: Fri, 21 Aug 2020 23:28:41 +0530 Subject: [PATCH 2/7] Handle review comments --- .../main/scala/org/apache/spark/util/Utils.scala | 11 +++++------ .../org/apache/spark/sql/internal/SQLConf.scala | 16 +++++++++------- .../spark/sql/execution/command/tables.scala | 4 ++-- 3 files changed, 16 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 9040bd9012ce2..0d32280642b7a 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -271,19 +271,18 @@ private[spark] object Utils extends Logging { /** * Move data to trash on truncate table given - * spark.sql.truncate.trash.interval is positive + * spark.sql.truncate.trash.enabled is true */ def moveToTrashIfEnabled( fs: FileSystem, partitionPath: Path, - trashInterval: Int, + isTrashEnabled: Boolean, hadoopConf: Configuration): Unit = { - if (trashInterval < 0) { - fs.delete(partitionPath, true) - } else { + if (isTrashEnabled && hadoopConf.getInt("fs.trash.interval", 0) > 0) { logDebug(s"will move data ${partitionPath.toString} to trash") - hadoopConf.setInt("fs.trash.interval", trashInterval) Trash.moveToAppropriateTrash(fs, partitionPath, hadoopConf) + } else { + fs.delete(partitionPath, true) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index b20cc5760d14a..be9a51e414141 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2722,12 +2722,14 @@ object SQLConf { .booleanConf .createWithDefault(false) - val TRUNCATE_TRASH_INTERVAL = - buildConf("spark.sql.truncate.trash.interval") - .doc("This Configuration will decide whether move files to trash on truncate table" + - "If -1 files will be deleted without moving to trash") - .intConf - .createWithDefault(-1) + val TRUNCATE_TRASH_ENABLED = + buildConf("spark.sql.truncate.trash.enabled") + .doc("This Configuration will decide whether move files to trash on truncate table given, " + + "'fs.trash.interval' is positive in Hadoop Configuration. " + + "Note that, in Hadoop conf if server side has this configured then the client side " + + "one will be ignored. ") + .booleanConf + .createWithDefault(false) /** * Holds information about keys that have been deprecated. @@ -3341,7 +3343,7 @@ class SQLConf extends Serializable with Logging { def legacyPathOptionBehavior: Boolean = getConf(SQLConf.LEGACY_PATH_OPTION_BEHAVIOR) - def truncateTrashInterval: Int = getConf(SQLConf.TRUNCATE_TRASH_INTERVAL) + def truncateTrashEnabled: Boolean = getConf(SQLConf.TRUNCATE_TRASH_ENABLED) /** ********************** SQLConf functionality methods ************ */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 2b40291b26552..7aebdd7e57293 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -490,7 +490,7 @@ case class TruncateTableCommand( } val hadoopConf = spark.sessionState.newHadoopConf() val ignorePermissionAcl = SQLConf.get.truncateTableIgnorePermissionAcl - val trashInterval = SQLConf.get.truncateTrashInterval + val isTrashEnabled = SQLConf.get.truncateTrashEnabled locations.foreach { location => if (location.isDefined) { val path = new Path(location.get) @@ -515,7 +515,7 @@ case class TruncateTableCommand( } } - Utils.moveToTrashIfEnabled(fs, path, trashInterval, hadoopConf) + Utils.moveToTrashIfEnabled(fs, path, isTrashEnabled, hadoopConf) // We should keep original permission/acl of the path. // For owner/group, only super-user can set it, for example on HDFS. Because From 502dd08e99eaed2bc71b49cdfb98bdaf3a9cdec1 Mon Sep 17 00:00:00 2001 From: Udbhav30 Date: Sat, 22 Aug 2020 02:15:56 +0530 Subject: [PATCH 3/7] add a warning msg --- core/src/main/scala/org/apache/spark/util/Utils.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 0d32280642b7a..79b5bcf2468e9 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -280,7 +280,10 @@ private[spark] object Utils extends Logging { hadoopConf: Configuration): Unit = { if (isTrashEnabled && hadoopConf.getInt("fs.trash.interval", 0) > 0) { logDebug(s"will move data ${partitionPath.toString} to trash") - Trash.moveToAppropriateTrash(fs, partitionPath, hadoopConf) + val isSuccess = Trash.moveToAppropriateTrash(fs, partitionPath, hadoopConf) + if (!isSuccess) { + logWarning(s"Failed to move data ${partitionPath.toString} to trash") + } } else { fs.delete(partitionPath, true) } From 97d2146c73c2cd5f973e9203980a8f8dcb781ec6 Mon Sep 17 00:00:00 2001 From: Udbhav30 Date: Sat, 22 Aug 2020 16:47:23 +0530 Subject: [PATCH 4/7] add tests and minor fixes --- .../scala/org/apache/spark/util/Utils.scala | 2 +- .../apache/spark/sql/internal/SQLConf.scala | 9 ++++---- .../sql/execution/command/DDLSuite.scala | 22 +++++++++++++++++++ 3 files changed, 28 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 79b5bcf2468e9..6f1813d5d8429 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -278,7 +278,7 @@ private[spark] object Utils extends Logging { partitionPath: Path, isTrashEnabled: Boolean, hadoopConf: Configuration): Unit = { - if (isTrashEnabled && hadoopConf.getInt("fs.trash.interval", 0) > 0) { + if (isTrashEnabled) { logDebug(s"will move data ${partitionPath.toString} to trash") val isSuccess = Trash.moveToAppropriateTrash(fs, partitionPath, hadoopConf) if (!isSuccess) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index be9a51e414141..160518f1283a9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2724,10 +2724,11 @@ object SQLConf { val TRUNCATE_TRASH_ENABLED = buildConf("spark.sql.truncate.trash.enabled") - .doc("This Configuration will decide whether move files to trash on truncate table given, " + - "'fs.trash.interval' is positive in Hadoop Configuration. " + - "Note that, in Hadoop conf if server side has this configured then the client side " + - "one will be ignored. ") + .doc("This configuration decides when truncating table, whether data files will be moved " + + "to trash directory or deleted permanently. The trash retention time is controlled by " + + "fs.trash.interval, and in default, the server side configuration value takes " + + "precedence over the client-side one. Note that if fs.trash.interval is non-positive, " + + "this will be a no-op and log a warning message.") .booleanConf .createWithDefault(false) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 17857a6ce173d..6854a66bbb5ee 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -3101,6 +3101,28 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { assert(spark.sessionState.catalog.isRegisteredFunction(rand)) } } + + test("Move data to trash on truncate table if enabled") { + withTable("tab1") { + withSQLConf(SQLConf.TRUNCATE_TRASH_ENABLED.key -> "true") { + sql("CREATE TABLE tab1 (col INT) USING parquet") + sql("INSERT INTO tab1 SELECT 1") + + val tablePath = new Path(spark.sessionState.catalog + .getTableMetadata(TableIdentifier("tab1")).storage.locationUri.get) + val hadoopConf = spark.sessionState.newHadoopConf() + val fs = tablePath.getFileSystem(hadoopConf) + // trash interval should be configured from hadoop side + hadoopConf.setInt("fs.trash.Interval", 5) + + val trashRoot = fs.getTrashRoot(tablePath) + assert(!fs.exists(trashRoot)) + sql("TRUNCATE TABLE tab1") + assert(fs.exists(trashRoot)) + fs.delete(trashRoot, true) + } + } + } } object FakeLocalFsFileSystem { From a52e9a596380ebcaeac8dbe2d9133867dc332ad7 Mon Sep 17 00:00:00 2001 From: Udbhav30 Date: Sun, 23 Aug 2020 01:09:34 +0530 Subject: [PATCH 5/7] add tests and minor fixes --- .../scala/org/apache/spark/util/Utils.scala | 2 +- .../sql/execution/command/DDLSuite.scala | 64 +++++++++++++++++-- 2 files changed, 58 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 6f1813d5d8429..f86d89cc0f0b0 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -271,7 +271,7 @@ private[spark] object Utils extends Logging { /** * Move data to trash on truncate table given - * spark.sql.truncate.trash.enabled is true + * 'spark.sql.truncate.trash.enabled' is true */ def moveToTrashIfEnabled( fs: FileSystem, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 6854a66bbb5ee..5a11adb1143b0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -3102,27 +3102,77 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } } - test("Move data to trash on truncate table if enabled") { + test("SPARK-32481 Move data to trash on truncate table if enabled") { + val trashIntervalKey = "fs.trash.interval" withTable("tab1") { withSQLConf(SQLConf.TRUNCATE_TRASH_ENABLED.key -> "true") { sql("CREATE TABLE tab1 (col INT) USING parquet") sql("INSERT INTO tab1 SELECT 1") - + // scalastyle:off hadoopConfiguration + val hadoopConf = spark.sparkContext.hadoopConfiguration + // scalastyle:on hadoopConfiguration + val originalValue = hadoopConf.get(trashIntervalKey, "0") val tablePath = new Path(spark.sessionState.catalog .getTableMetadata(TableIdentifier("tab1")).storage.locationUri.get) - val hadoopConf = spark.sessionState.newHadoopConf() - val fs = tablePath.getFileSystem(hadoopConf) - // trash interval should be configured from hadoop side - hadoopConf.setInt("fs.trash.Interval", 5) + val fs = tablePath.getFileSystem(hadoopConf) val trashRoot = fs.getTrashRoot(tablePath) assert(!fs.exists(trashRoot)) - sql("TRUNCATE TABLE tab1") + try { + hadoopConf.set(trashIntervalKey, "5") + sql("TRUNCATE TABLE tab1") + } finally { + hadoopConf.set(trashIntervalKey, originalValue) + } assert(fs.exists(trashRoot)) fs.delete(trashRoot, true) } } } + + test("SPARK-32481 delete data permanently on truncate table if trash interval is non-positive") { + val trashIntervalKey = "fs.trash.interval" + withTable("tab1") { + withSQLConf(SQLConf.TRUNCATE_TRASH_ENABLED.key -> "true") { + sql("CREATE TABLE tab1 (col INT) USING parquet") + sql("INSERT INTO tab1 SELECT 1") + // scalastyle:off hadoopConfiguration + val hadoopConf = spark.sparkContext.hadoopConfiguration + // scalastyle:on hadoopConfiguration + val originalValue = hadoopConf.get(trashIntervalKey, "0") + val tablePath = new Path(spark.sessionState.catalog + .getTableMetadata(TableIdentifier("tab1")).storage.locationUri.get) + + val fs = tablePath.getFileSystem(hadoopConf) + val trashRoot = fs.getTrashRoot(tablePath) + assert(!fs.exists(trashRoot)) + try { + hadoopConf.set(trashIntervalKey, "0") + sql("TRUNCATE TABLE tab1") + } finally { + hadoopConf.set(trashIntervalKey, originalValue) + } + assert(!fs.exists(trashRoot)) + } + } + } + + test("SPARK-32481 Donot move data to trash on truncate table if disabled") { + withTable("tab1") { + withSQLConf(SQLConf.TRUNCATE_TRASH_ENABLED.key -> "false") { + sql("CREATE TABLE tab1 (col INT) USING parquet") + sql("INSERT INTO tab1 SELECT 1") + val hadoopConf = spark.sessionState.newHadoopConf() + val tablePath = new Path(spark.sessionState.catalog + .getTableMetadata(TableIdentifier("tab1")).storage.locationUri.get) + + val fs = tablePath.getFileSystem(hadoopConf) + val trashRoot = fs.getTrashRoot(tablePath) + sql("TRUNCATE TABLE tab1") + assert(!fs.exists(trashRoot)) + } + } + } } object FakeLocalFsFileSystem { From 8848b3fc60279a30633341d7fb6912ad1f5c71e7 Mon Sep 17 00:00:00 2001 From: Udbhav30 Date: Sun, 23 Aug 2020 01:27:58 +0530 Subject: [PATCH 6/7] typo fix --- .../org/apache/spark/sql/execution/command/DDLSuite.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 5a11adb1143b0..edc7d98841fd4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -3108,9 +3108,9 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { withSQLConf(SQLConf.TRUNCATE_TRASH_ENABLED.key -> "true") { sql("CREATE TABLE tab1 (col INT) USING parquet") sql("INSERT INTO tab1 SELECT 1") - // scalastyle:off hadoopConfiguration + // scalastyle:off hadoopconfiguration val hadoopConf = spark.sparkContext.hadoopConfiguration - // scalastyle:on hadoopConfiguration + // scalastyle:on hadoopconfiguration val originalValue = hadoopConf.get(trashIntervalKey, "0") val tablePath = new Path(spark.sessionState.catalog .getTableMetadata(TableIdentifier("tab1")).storage.locationUri.get) @@ -3136,9 +3136,9 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { withSQLConf(SQLConf.TRUNCATE_TRASH_ENABLED.key -> "true") { sql("CREATE TABLE tab1 (col INT) USING parquet") sql("INSERT INTO tab1 SELECT 1") - // scalastyle:off hadoopConfiguration + // scalastyle:off hadoopconfiguration val hadoopConf = spark.sparkContext.hadoopConfiguration - // scalastyle:on hadoopConfiguration + // scalastyle:on hadoopconfiguration val originalValue = hadoopConf.get(trashIntervalKey, "0") val tablePath = new Path(spark.sessionState.catalog .getTableMetadata(TableIdentifier("tab1")).storage.locationUri.get) From a2df53b48db372ed8f9a303cd8c0c499e33f3adf Mon Sep 17 00:00:00 2001 From: Udbhav30 Date: Tue, 25 Aug 2020 03:31:53 +0530 Subject: [PATCH 7/7] Review comments fix --- core/src/main/scala/org/apache/spark/util/Utils.scala | 7 ++++--- .../main/scala/org/apache/spark/sql/internal/SQLConf.scala | 1 + .../org/apache/spark/sql/execution/command/DDLSuite.scala | 2 +- 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index f86d89cc0f0b0..a336c1260d344 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -270,20 +270,21 @@ private[spark] object Utils extends Logging { } /** - * Move data to trash on truncate table given - * 'spark.sql.truncate.trash.enabled' is true + * Move data to trash if 'spark.sql.truncate.trash.enabled' is true */ def moveToTrashIfEnabled( fs: FileSystem, partitionPath: Path, isTrashEnabled: Boolean, - hadoopConf: Configuration): Unit = { + hadoopConf: Configuration): Boolean = { if (isTrashEnabled) { logDebug(s"will move data ${partitionPath.toString} to trash") val isSuccess = Trash.moveToAppropriateTrash(fs, partitionPath, hadoopConf) if (!isSuccess) { logWarning(s"Failed to move data ${partitionPath.toString} to trash") + return fs.delete(partitionPath, true) } + isSuccess } else { fs.delete(partitionPath, true) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 160518f1283a9..c9db7b1e8960a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2729,6 +2729,7 @@ object SQLConf { "fs.trash.interval, and in default, the server side configuration value takes " + "precedence over the client-side one. Note that if fs.trash.interval is non-positive, " + "this will be a no-op and log a warning message.") + .version("3.1.0") .booleanConf .createWithDefault(false) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index edc7d98841fd4..d6d58a833dd7d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -3157,7 +3157,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } } - test("SPARK-32481 Donot move data to trash on truncate table if disabled") { + test("SPARK-32481 Do not move data to trash on truncate table if disabled") { withTable("tab1") { withSQLConf(SQLConf.TRUNCATE_TRASH_ENABLED.key -> "false") { sql("CREATE TABLE tab1 (col INT) USING parquet")