-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-32481][CORE][SQL] Support truncate table to move data to trash #29387
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
58d957a
81101d9
502dd08
97d2146
a52e9a5
8848b3f
a2df53b
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -2722,6 +2722,17 @@ object SQLConf { | |
| .booleanConf | ||
| .createWithDefault(false) | ||
|
|
||
| val TRUNCATE_TRASH_ENABLED = | ||
| buildConf("spark.sql.truncate.trash.enabled") | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. quick question, do we want to have each configuration for each operation? Looks like #29319 targets similar stuff. Maybe it'd make more sense to have a global configuration.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I will rework on #29319 and make it a global configuration.
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yep. It's too early to make it a global configuration. |
||
| .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 " + | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Does Spark SQL have the trash directory ?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hi, It is hdfs trash directory |
||
| "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.") | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Just a question. Is the following still true?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yes, this is still true.
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes. This is because when |
||
| .version("3.1.0") | ||
| .booleanConf | ||
Udbhav30 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| .createWithDefault(false) | ||
|
|
||
| /** | ||
| * Holds information about keys that have been deprecated. | ||
| * | ||
|
|
@@ -3334,6 +3345,8 @@ class SQLConf extends Serializable with Logging { | |
|
|
||
| def legacyPathOptionBehavior: Boolean = getConf(SQLConf.LEGACY_PATH_OPTION_BEHAVIOR) | ||
|
|
||
| def truncateTrashEnabled: Boolean = getConf(SQLConf.TRUNCATE_TRASH_ENABLED) | ||
|
|
||
| /** ********************** SQLConf functionality methods ************ */ | ||
|
|
||
| /** Set Spark SQL configuration properties. */ | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -3101,6 +3101,78 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { | |
| assert(spark.sessionState.catalog.isRegisteredFunction(rand)) | ||
| } | ||
| } | ||
|
|
||
| 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") { | ||
Udbhav30 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| sql("CREATE TABLE tab1 (col INT) USING parquet") | ||
| sql("INSERT INTO tab1 SELECT 1") | ||
| // scalastyle:off hadoopconfiguration | ||
| val hadoopConf = spark.sparkContext.hadoopConfiguration | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
| // 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) | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. All the FS support this operation?
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes a default impl is defined in Even though it is supported, it seems the trash mechanism is less useful in cloud object stores like S3, where renaming doesn't exist and therefore moving to trash is much more expensive. However user can disable that by the configs given here and in Hadoop itself. |
||
| assert(!fs.exists(trashRoot)) | ||
| 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 | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Can we use
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I tried to use 'spark.sessionState.newHadoopConf()' but the hadoop conf was not reflected, so ''moveToAppropriateTrash' returns false as 'fs.trash.interval' is 0
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The other places are the same. Let's try to following the Scalastyle warning if possible.
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Oh, should we change this underlying one? |
||
| // 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)) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @Udbhav30 This line of code is not Mac os friendly, the
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks @LuciferYang for pointing it out, i will raise follow-up PR and assert the particular folder that is |
||
| try { | ||
| hadoopConf.set(trashIntervalKey, "0") | ||
| sql("TRUNCATE TABLE tab1") | ||
| } finally { | ||
| hadoopConf.set(trashIntervalKey, originalValue) | ||
| } | ||
| assert(!fs.exists(trashRoot)) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| 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") | ||
| sql("INSERT INTO tab1 SELECT 1") | ||
| val hadoopConf = spark.sessionState.newHadoopConf() | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. In the other tests, this PR is using
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hii, In this test we did not update the
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @dongjoon-hyun See here. If |
||
| 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 { | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.