Skip to content
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

[SPARK-32481][CORE][SQL] Support truncate table to move data to trash #29387

Closed
wants to merge 7 commits into from

Conversation

Udbhav30
Copy link
Contributor

@Udbhav30 Udbhav30 commented Aug 8, 2020

What changes were proposed in this pull request?

Instead of deleting the data, we can move the data to trash.
Based on the configuration provided by the user it will be deleted permanently from the trash.

Why are the changes needed?

Instead of directly deleting the data, we can provide flexibility to move data to the trash and then delete it permanently.

Does this PR introduce any user-facing change?

Yes, After truncate table the data is not permanently deleted now.
It is first moved to the trash and then after the given time deleted permanently;

How was this patch tested?

new UTs added

@Udbhav30
Copy link
Contributor Author

Udbhav30 commented Aug 8, 2020

cc @dongjoon-hyun please review

@Udbhav30 Udbhav30 force-pushed the tuncateTrash branch 2 times, most recently from ce6f124 to bd4ccf5 Compare August 9, 2020 07:22
Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

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

Please clean up the leftover of #29319 .

@Udbhav30
Copy link
Contributor Author

Please clean up the leftover of #29319 .

Hi @dongjoon-hyun, I have cleaned the MR kindly review.

@Udbhav30
Copy link
Contributor Author

Gentle ping @dongjoon-hyun

@dongjoon-hyun
Copy link
Member

Could you review this, @sunchao ?

@dongjoon-hyun
Copy link
Member

ok to test

Copy link
Member

@sunchao sunchao left a comment

Choose a reason for hiding this comment

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

This looks useful. One thing I'd recommend is to make this a boolean flag instead of an interval, and then rely on Hadoop side config fs.trash.interval to control the trash retention.

core/src/main/scala/org/apache/spark/util/Utils.scala Outdated Show resolved Hide resolved
@Udbhav30
Copy link
Contributor Author

This looks useful. One thing I'd recommend is to make this a boolean flag instead of an interval, and then rely on Hadoop side config fs.trash.interval to control the trash retention.

Sure I'll update it.

@SparkQA
Copy link

SparkQA commented Aug 20, 2020

Test build #127704 has finished for PR 29387 at commit 266d0eb.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@Udbhav30
Copy link
Contributor Author

@sunchao @dongjoon-hyun, I have updated the MR can you please review.

@SparkQA
Copy link

SparkQA commented Aug 21, 2020

Test build #127757 has finished for PR 29387 at commit 8d94930.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Aug 21, 2020

Test build #127763 has finished for PR 29387 at commit a9806aa.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Aug 22, 2020

Test build #127774 has finished for PR 29387 at commit 7447f18.

  • This patch fails Spark unit tests.
  • This patch does not merge cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Aug 22, 2020

Test build #127777 has finished for PR 29387 at commit bfd3f79.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Member

@sunchao sunchao left a comment

Choose a reason for hiding this comment

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

Thanks @Udbhav30 ! this LGTM now. Can you check why test is failing though? I'll let others (cc @dongjoon-hyun ) to chime in and help finishing this.

We may need to consider another flag in future to control whether when spark.sql.truncate.trash.enabled but Hadoop side trash is disabled, this should switch to permanently delete data or skip the deletion all together (current behavior). Sometimes later may not be what users want.

core/src/main/scala/org/apache/spark/util/Utils.scala Outdated Show resolved Hide resolved
@dongjoon-hyun
Copy link
Member

Thank you for helping this PR, @sunchao .

sql("CREATE TABLE tab1 (col INT) USING parquet")
sql("INSERT INTO tab1 SELECT 1")
// scalastyle:off hadoopconfiguration
val hadoopConf = spark.sparkContext.hadoopConfiguration
Copy link
Member

Choose a reason for hiding this comment

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

spark.sessionState.newHadoopConf()?

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()
Copy link
Member

Choose a reason for hiding this comment

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

In the other tests, this PR is using spark.sparkContext.hadoopConfiguration.
If that is required, this test case looks misleading, withSQLConf(SQLConf.TRUNCATE_TRASH_ENABLED.key -> "false") { is required here? I'm wondering if this test case passed with withSQLConf(SQLConf.TRUNCATE_TRASH_ENABLED.key -> "true") {, too.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Hii, In this test we did not update the hadoopConf, so using spark.sessionState.newHadoopConf() doesn't make any difference

Copy link
Contributor Author

@Udbhav30 Udbhav30 Aug 24, 2020

Choose a reason for hiding this comment

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

@dongjoon-hyun See here. If fs.trash.interval is non positive then moveToAppropriateTrash function returns false. So to test this I have to add positive value to fs.trash.interval, but spark.sessionState.newHadoopConf() does not update the hadoopConf and so other testcase fails. And here this testcase is no-op so updating the hadoopConf is not required so I used spark.sessionState.newHadoopConf()

@dongjoon-hyun
Copy link
Member

Thank you for pinging me again, @Udbhav30 . I finished another round. I'll review after the PR is updated again.

@SparkQA
Copy link

SparkQA commented Aug 25, 2020

Test build #127859 has finished for PR 29387 at commit a2df53b.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@Udbhav30
Copy link
Contributor Author

Thank you for pinging me again, @Udbhav30 . I finished another round. I'll review after the PR is updated again.

Hii thanks for the review @dongjoon-hyun , i have updated the pr as per your suggestions

@dongjoon-hyun
Copy link
Member

Thanks!

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

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

+1, LGTM. Thank you, @Udbhav30 , @sunchao , @gatorsmile .
Merged to master for Apache Spark 3.1.0.

@@ -2722,6 +2722,17 @@ object SQLConf {
.booleanConf
.createWithDefault(false)

val TRUNCATE_TRASH_ENABLED =
buildConf("spark.sql.truncate.trash.enabled")
Copy link
Member

Choose a reason for hiding this comment

The 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.

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 will rework on #29319 and make it a global configuration.

Copy link
Member

Choose a reason for hiding this comment

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

Yep. It's too early to make it a global configuration.


val fs = tablePath.getFileSystem(hadoopConf)
val trashRoot = fs.getTrashRoot(tablePath)
assert(!fs.exists(trashRoot))
Copy link
Contributor

Choose a reason for hiding this comment

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

@Udbhav30 This line of code is not Mac os friendly, the trashRoot is /Users/xxx/.Trash/, it is the path to the trash can of Mac os. So normally, it exists...

Copy link
Contributor Author

Choose a reason for hiding this comment

The 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 trashRoot/pathToTable/tab1 in this case instead of trashRoot

@dongjoon-hyun
Copy link
Member

Ur, sorry, @Udbhav30 . It seems that Hadoop 2.7 doesn't have getTrashRoot inside FileSystem.

[error] /home/jenkins/workspace/spark-master-test-sbt-hadoop-2.7-hive-1.2/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala:3119: value getTrashRoot is not a member of org.apache.hadoop.fs.FileSystem
[error]         val trashRoot = fs.getTrashRoot(tablePath)
[error]                            ^
[error] /home/jenkins/workspace/spark-master-test-sbt-hadoop-2.7-hive-1.2/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala:3147: value getTrashRoot is not a member of org.apache.hadoop.fs.FileSystem
[error]         val trashRoot = fs.getTrashRoot(tablePath)
[error]                            ^
[error] /home/jenkins/workspace/spark-master-test-sbt-hadoop-2.7-hive-1.2/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala:3170: value getTrashRoot is not a member of org.apache.hadoop.fs.FileSystem
[error]         val trashRoot = fs.getTrashRoot(tablePath)
[error]                            ^
[warn] /home/jenkins/workspace/spark-master-test-sbt-hadoop-2.7-hive-1.2/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala:706: a pure expression does nothing in statement position; multiline expressions might require enclosing parentheses
[warn]       q1
[warn]       ^
[warn] one warning found
[error] three errors found
[error] (sql/test:compileIncremental) Compilation failed
[error] Total time: 262 s, completed Aug 26, 2020 10:50:44 AM

I'll revert this first to recover Jenkins.

@sunchao
Copy link
Member

sunchao commented Aug 26, 2020

Hmm this is a bummer, sorry missed that. It used to be just new Path(fs.getHomeDirectory(), ".Trash") but since 2.8 it added getTrashRoot to hide the details (even though implementation is still the same).

Perhaps we can use Trash.getCurrentTrashDir instead?

Udbhav30 added a commit to Udbhav30/spark that referenced this pull request Aug 26, 2020
Instead of deleting the data, we can move the data to trash.
Based on the configuration provided by the user it will be deleted permanently from the trash.

Instead of directly deleting the data, we can provide flexibility to move data to the trash and then delete it permanently.

Yes, After truncate table the data is not permanently deleted now.
It is first moved to the trash and then after the given time deleted permanently;

new UTs added

Closes apache#29387 from Udbhav30/tuncateTrash.

Authored-by: Udbhav30 <[email protected]>
Signed-off-by: Dongjoon Hyun <[email protected]>
@Udbhav30
Copy link
Contributor Author

Hmm this is a bummer, sorry missed that. It used to be just new Path(fs.getHomeDirectory(), ".Trash") but since 2.8 it added getTrashRoot to hide the details (even though implementation is still the same).

Perhaps we can use Trash.getCurrentTrashDir instead?

Hi @sunchao, I have raised a new PR in #29552, kindly review that

@Tagar
Copy link

Tagar commented Aug 26, 2020

@Udbhav30 generally, one user can have multiple trash directories.
The default one fs.getHomeDirectory() + ".Trash" as you mentioned, and there could be multiple non-default ones - one per encryption zone.
So each encryption zone's trash directory is encrypted with the same key and files can be moved to trash without reencryption.
For GDPR/ CCPA use cases we had some tables with PII created in an HDFS encryption zone and those couldn't use default trash location.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

8 participants