Skip to content

Commit 9fac2e6

Browse files
xupefeiallisonport-db
authored andcommitted
Implement PURGE to remove DVs from Delta tables
## Description This PR introduces a `REORG TABLE ... APPLY (PURGE)` SQL command that can materialize soft-delete operations by DVs. The command works by rewriting and bin-packing (if applicable) only files that have DVs attached, which is different from the `OPTIMIZE` command where all files (with and without) DV will be bin-packed. To achieve this, we hack the `OPTIMIZE` logic so files of any size with DVs will be rewritten. Follow-up: - Set the correct commit info. Now the resulting version is marked as `optimize` rather than `purge`. - Clean up DVs from the filesystem. New tests. Closes #1732 Signed-off-by: Venki Korukanti <[email protected]> GitOrigin-RevId: 98ef156d62698986bfb54681e386971e2fec08b8
1 parent dcad4fd commit 9fac2e6

File tree

11 files changed

+313
-31
lines changed

11 files changed

+313
-31
lines changed

core/src/main/antlr4/io/delta/sql/parser/DeltaSqlBase.g4

Lines changed: 8 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -87,8 +87,11 @@ statement
8787
| ALTER TABLE table=qualifiedName
8888
DROP CONSTRAINT (IF EXISTS)? name=identifier #dropTableConstraint
8989
| OPTIMIZE (path=STRING | table=qualifiedName)
90-
(WHERE partitionPredicate = predicateToken)?
90+
(WHERE partitionPredicate=predicateToken)?
9191
(zorderSpec)? #optimizeTable
92+
| REORG TABLE table=qualifiedName
93+
(WHERE partitionPredicate=predicateToken)?
94+
APPLY LEFT_PAREN PURGE RIGHT_PAREN #reorgTable
9295
| SHOW COLUMNS (IN | FROM) tableName=qualifiedName
9396
((IN | FROM) schemaName=identifier)? #showColumns
9497
| cloneTableHeader SHALLOW CLONE source=qualifiedName clause=temporalClause?
@@ -210,6 +213,7 @@ nonReserved
210213
| CONVERT | TO | DELTA | PARTITIONED | BY
211214
| DESC | DESCRIBE | LIMIT | DETAIL
212215
| GENERATE | FOR | TABLE | CHECK | EXISTS | OPTIMIZE
216+
| REORG | APPLY | PURGE
213217
| RESTORE | AS | OF
214218
| ZORDER | LEFT_PAREN | RIGHT_PAREN
215219
| SHOW | COLUMNS | IN | FROM | NO | STATISTICS
@@ -219,6 +223,7 @@ nonReserved
219223
// Define how the keywords above should appear in a user's SQL statement.
220224
ADD: 'ADD';
221225
ALTER: 'ALTER';
226+
APPLY: 'APPLY';
222227
AS: 'AS';
223228
BY: 'BY';
224229
CHECK: 'CHECK';
@@ -255,7 +260,9 @@ NULL: 'NULL';
255260
OF: 'OF';
256261
OR: 'OR';
257262
OPTIMIZE: 'OPTIMIZE';
263+
REORG: 'REORG';
258264
PARTITIONED: 'PARTITIONED';
265+
PURGE: 'PURGE';
259266
REPLACE: 'REPLACE';
260267
RESTORE: 'RESTORE';
261268
RETAIN: 'RETAIN';

core/src/main/scala/io/delta/sql/parser/DeltaSqlParser.scala

Lines changed: 23 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -346,7 +346,29 @@ class DeltaSqlAstBuilder extends DeltaSqlBaseBaseVisitor[AnyRef] {
346346
OptimizeTableCommand(
347347
Option(ctx.path).map(string),
348348
Option(ctx.table).map(visitTableIdentifier),
349-
Option(ctx.partitionPredicate).map(extractRawText(_)).toSeq, Map.empty)(interleaveBy)
349+
Option(ctx.partitionPredicate).map(extractRawText(_)).toSeq,
350+
Map.empty)(interleaveBy)
351+
}
352+
353+
/**
354+
* Creates a [[DeltaReorgTable]] logical plan.
355+
* Examples:
356+
* {{{
357+
* -- Physically delete dropped rows and columns of target table
358+
* REORG TABLE (delta.`/path/to/table` | delta_table_name)
359+
* [WHERE partition_predicate] APPLY (PURGE)
360+
* }}}
361+
*/
362+
override def visitReorgTable(ctx: ReorgTableContext): AnyRef = withOrigin(ctx) {
363+
if (ctx.table == null) {
364+
throw new ParseException("REORG command requires a file path or table name.", ctx)
365+
}
366+
367+
val targetIdentifier = visitTableIdentifier(ctx.table)
368+
val tableNameParts = targetIdentifier.database.toSeq :+ targetIdentifier.table
369+
val targetTable = createUnresolvedTable(tableNameParts, "REORG")
370+
371+
DeltaReorgTable(targetTable)(Option(ctx.partitionPredicate).map(extractRawText(_)).toSeq)
350372
}
351373

352374
override def visitDescribeDeltaDetail(

core/src/main/scala/org/apache/spark/sql/delta/DeltaAnalysis.scala

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -485,6 +485,14 @@ class DeltaAnalysis(session: SparkSession)
485485

486486
DeltaMergeInto.resolveReferencesAndSchema(deltaMerge, conf)(tryResolveReferences(session))
487487

488+
case reorg@DeltaReorgTable(_@ResolvedTable(_, _, t, _)) =>
489+
t match {
490+
case table: DeltaTableV2 =>
491+
DeltaReorgTableCommand(table)(reorg.predicates)
492+
case _ =>
493+
throw DeltaErrors.notADeltaTable(t.name())
494+
}
495+
488496
case deltaMerge: DeltaMergeInto =>
489497
val d = if (deltaMerge.childrenResolved && !deltaMerge.resolved) {
490498
DeltaMergeInto.resolveReferencesAndSchema(deltaMerge, conf)(tryResolveReferences(session))
Lines changed: 55 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,55 @@
1+
/*
2+
* Copyright (2021) The Delta Lake Project Authors.
3+
*
4+
* Licensed under the Apache License, Version 2.0 (the "License");
5+
* you may not use this file except in compliance with the License.
6+
* You may obtain a copy of the License at
7+
*
8+
* http://www.apache.org/licenses/LICENSE-2.0
9+
*
10+
* Unless required by applicable law or agreed to in writing, software
11+
* distributed under the License is distributed on an "AS IS" BASIS,
12+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13+
* See the License for the specific language governing permissions and
14+
* limitations under the License.
15+
*/
16+
17+
package org.apache.spark.sql.delta.commands
18+
19+
import org.apache.spark.sql.delta.catalog.DeltaTableV2
20+
21+
import org.apache.spark.sql.{Row, SparkSession}
22+
import org.apache.spark.sql.catalyst.plans.logical.{IgnoreCachedData, LeafCommand, LogicalPlan, UnaryCommand}
23+
24+
case class DeltaReorgTable(target: LogicalPlan)(val predicates: Seq[String]) extends UnaryCommand {
25+
26+
def child: LogicalPlan = target
27+
28+
protected def withNewChildInternal(newChild: LogicalPlan): LogicalPlan =
29+
copy(target = newChild)(predicates)
30+
31+
override val otherCopyArgs: Seq[AnyRef] = predicates :: Nil
32+
}
33+
34+
/**
35+
* The PURGE command.
36+
*/
37+
case class DeltaReorgTableCommand(target: DeltaTableV2)(val predicates: Seq[String])
38+
extends OptimizeTableCommandBase with LeafCommand with IgnoreCachedData {
39+
40+
override val otherCopyArgs: Seq[AnyRef] = predicates :: Nil
41+
42+
override def run(sparkSession: SparkSession): Seq[Row] = {
43+
val command = OptimizeTableCommand(
44+
Option(target.path.toString),
45+
target.catalogTable.map(_.identifier),
46+
predicates,
47+
options = Map.empty,
48+
optimizeContext = DeltaOptimizeContext(
49+
isPurge = true,
50+
minFileSize = Some(0L),
51+
maxDeletedRowsRatio = Some(0d))
52+
)(zOrderBy = Nil)
53+
command.run(sparkSession)
54+
}
55+
}

core/src/main/scala/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala

Lines changed: 39 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -109,7 +109,9 @@ case class OptimizeTableCommand(
109109
path: Option[String],
110110
tableId: Option[TableIdentifier],
111111
userPartitionPredicates: Seq[String],
112-
options: Map[String, String])(val zOrderBy: Seq[UnresolvedAttribute])
112+
options: Map[String, String],
113+
optimizeContext: DeltaOptimizeContext = DeltaOptimizeContext()
114+
)(val zOrderBy: Seq[UnresolvedAttribute])
113115
extends OptimizeTableCommandBase with LeafRunnableCommand {
114116

115117
override val otherCopyArgs: Seq[AnyRef] = zOrderBy :: Nil
@@ -138,7 +140,34 @@ case class OptimizeTableCommand(
138140
validateZorderByColumns(sparkSession, txn, zOrderBy)
139141
val zOrderByColumns = zOrderBy.map(_.name).toSeq
140142

141-
new OptimizeExecutor(sparkSession, txn, partitionPredicates, zOrderByColumns).optimize()
143+
new OptimizeExecutor(sparkSession, txn, partitionPredicates, zOrderByColumns, optimizeContext)
144+
.optimize()
145+
}
146+
}
147+
148+
/**
149+
* Stored all runtime context information that can control the execution of optimize.
150+
*
151+
* @param isPurge Whether the rewriting task is only for purging soft-deleted data instead of
152+
* for compaction. If [[isPurge]] is true, only files with DVs will be selected
153+
* for compaction.
154+
* @param minFileSize Files which are smaller than this threshold will be selected for compaction.
155+
* If not specified, [[DeltaSQLConf.DELTA_OPTIMIZE_MIN_FILE_SIZE]] will be used.
156+
* This parameter must be set to `0` when [[isPurge]] is true.
157+
* @param maxDeletedRowsRatio Files with a ratio of soft-deleted rows to the total rows larger than
158+
* this threshold will be rewritten by the OPTIMIZE command. If not
159+
* specified, [[DeltaSQLConf.DELTA_OPTIMIZE_MAX_DELETED_ROWS_RATIO]]
160+
* will be used. This parameter must be set to `0` when [[isPurge]] is
161+
* true.
162+
*/
163+
case class DeltaOptimizeContext(
164+
isPurge: Boolean = false,
165+
minFileSize: Option[Long] = None,
166+
maxDeletedRowsRatio: Option[Double] = None) {
167+
if (isPurge) {
168+
require(
169+
minFileSize.contains(0L) && maxDeletedRowsRatio.contains(0d),
170+
"minFileSize and maxDeletedRowsRatio must be 0 when running PURGE.")
142171
}
143172
}
144173

@@ -154,7 +183,8 @@ class OptimizeExecutor(
154183
sparkSession: SparkSession,
155184
txn: OptimisticTransaction,
156185
partitionPredicate: Seq[Expression],
157-
zOrderByColumns: Seq[String])
186+
zOrderByColumns: Seq[String],
187+
optimizeContext: DeltaOptimizeContext)
158188
extends DeltaCommand with SQLMetricsReporting with Serializable {
159189

160190
/** Timestamp to use in [[FileAction]] */
@@ -164,18 +194,16 @@ class OptimizeExecutor(
164194

165195
def optimize(): Seq[Row] = {
166196
recordDeltaOperation(txn.deltaLog, "delta.optimize") {
167-
val minFileSize = sparkSession.sessionState.conf.getConf(
168-
DeltaSQLConf.DELTA_OPTIMIZE_MIN_FILE_SIZE)
169-
val maxFileSize = sparkSession.sessionState.conf.getConf(
170-
DeltaSQLConf.DELTA_OPTIMIZE_MAX_FILE_SIZE)
171-
require(minFileSize > 0, "minFileSize must be > 0")
172-
require(maxFileSize > 0, "maxFileSize must be > 0")
197+
val minFileSize = optimizeContext.minFileSize.getOrElse(
198+
sparkSession.sessionState.conf.getConf(DeltaSQLConf.DELTA_OPTIMIZE_MIN_FILE_SIZE))
199+
val maxFileSize =
200+
sparkSession.sessionState.conf.getConf(DeltaSQLConf.DELTA_OPTIMIZE_MAX_FILE_SIZE)
201+
val maxDeletedRowsRatio = optimizeContext.maxDeletedRowsRatio.getOrElse(
202+
sparkSession.sessionState.conf.getConf(DeltaSQLConf.DELTA_OPTIMIZE_MAX_DELETED_ROWS_RATIO))
173203

174204
val candidateFiles = txn.filterFiles(partitionPredicate, keepNumRecords = true)
175205
val partitionSchema = txn.metadata.partitionSchema
176206

177-
val maxDeletedRowsRatio = sparkSession.sessionState.conf.getConf(
178-
DeltaSQLConf.DELTA_OPTIMIZE_MAX_DELETED_ROWS_RATIO)
179207
val filesToProcess = pruneCandidateFileList(minFileSize, maxDeletedRowsRatio, candidateFiles)
180208
val partitionsToCompact = filesToProcess.groupBy(_.partitionValues).toSeq
181209

core/src/test/scala/io/delta/sql/parser/DeltaSqlParserSuite.scala

Lines changed: 56 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -19,11 +19,10 @@ package io.delta.sql.parser
1919
import io.delta.tables.execution.VacuumTableCommand
2020

2121
import org.apache.spark.sql.delta.CloneTableSQLTestUtils
22-
import org.apache.spark.sql.delta.commands.OptimizeTableCommand
23-
22+
import org.apache.spark.sql.delta.commands.{OptimizeTableCommand, DeltaReorgTable}
2423
import org.apache.spark.SparkFunSuite
2524
import org.apache.spark.sql.catalyst.{TableIdentifier, TimeTravel}
26-
import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedRelation}
25+
import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedRelation, UnresolvedTable}
2726
import org.apache.spark.sql.catalyst.expressions.Literal
2827
import org.apache.spark.sql.catalyst.parser.ParseException
2928
import org.apache.spark.sql.catalyst.plans.SQLHelper
@@ -120,6 +119,60 @@ class DeltaSqlParserSuite extends SparkFunSuite with SQLHelper {
120119
Seq(unresolvedAttr("optimize"), unresolvedAttr("zorder"))))
121120
}
122121

122+
private def targetPlanForTable(tableParts: String*): UnresolvedTable =
123+
UnresolvedTable(tableParts.toSeq, "REORG", relationTypeMismatchHint = None)
124+
125+
test("REORG command is parsed as expected") {
126+
val parser = new DeltaSqlParser(null)
127+
128+
assert(parser.parsePlan("REORG TABLE tbl APPLY (PURGE)") ===
129+
DeltaReorgTable(targetPlanForTable("tbl"))(Seq.empty))
130+
131+
assert(parser.parsePlan("REORG TABLE tbl_${system:spark.testing} APPLY (PURGE)") ===
132+
DeltaReorgTable(targetPlanForTable("tbl_true"))(Seq.empty))
133+
134+
withSQLConf("tbl_var" -> "tbl") {
135+
assert(parser.parsePlan("REORG TABLE ${tbl_var} APPLY (PURGE)") ===
136+
DeltaReorgTable(targetPlanForTable("tbl"))(Seq.empty))
137+
138+
assert(parser.parsePlan("REORG TABLE ${spark:tbl_var} APPLY (PURGE)") ===
139+
DeltaReorgTable(targetPlanForTable("tbl"))(Seq.empty))
140+
141+
assert(parser.parsePlan("REORG TABLE ${sparkconf:tbl_var} APPLY (PURGE)") ===
142+
DeltaReorgTable(targetPlanForTable("tbl"))(Seq.empty))
143+
144+
assert(parser.parsePlan("REORG TABLE ${hiveconf:tbl_var} APPLY (PURGE)") ===
145+
DeltaReorgTable(targetPlanForTable("tbl"))(Seq.empty))
146+
147+
assert(parser.parsePlan("REORG TABLE ${hivevar:tbl_var} APPLY (PURGE)") ===
148+
DeltaReorgTable(targetPlanForTable("tbl"))(Seq.empty))
149+
}
150+
151+
assert(parser.parsePlan("REORG TABLE delta.`/path/to/tbl` APPLY (PURGE)") ===
152+
DeltaReorgTable(targetPlanForTable("delta", "/path/to/tbl"))(Seq.empty))
153+
154+
assert(parser.parsePlan("REORG TABLE tbl WHERE part = 1 APPLY (PURGE)") ===
155+
DeltaReorgTable(targetPlanForTable("tbl"))(Seq("part = 1")))
156+
}
157+
158+
test("REORG command new tokens are non-reserved keywords") {
159+
// new keywords: REORG, APPLY, PURGE
160+
val parser = new DeltaSqlParser(null)
161+
162+
// Use the new keywords in table name
163+
assert(parser.parsePlan("REORG TABLE reorg APPLY (PURGE)") ===
164+
DeltaReorgTable(targetPlanForTable("reorg"))(Seq.empty))
165+
assert(parser.parsePlan("REORG TABLE apply APPLY (PURGE)") ===
166+
DeltaReorgTable(targetPlanForTable("apply"))(Seq.empty))
167+
assert(parser.parsePlan("REORG TABLE purge APPLY (PURGE)") ===
168+
DeltaReorgTable(targetPlanForTable("purge"))(Seq.empty))
169+
170+
// Use the new keywords in column name
171+
assert(parser.parsePlan(
172+
"REORG TABLE tbl WHERE reorg = 1 AND apply = 2 AND purge = 3 APPLY (PURGE)") ===
173+
DeltaReorgTable(targetPlanForTable("tbl"))(Seq("reorg = 1 AND apply =2 AND purge = 3")))
174+
}
175+
123176
// scalastyle:off argcount
124177
private def checkCloneStmt(
125178
parser: DeltaSqlParser,

core/src/test/scala/org/apache/spark/sql/delta/DeletionVectorsTestUtils.scala

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -172,6 +172,14 @@ trait DeletionVectorsTestUtils extends QueryTest with SharedSparkSession {
172172
txn.commit(actions, Truncate())
173173
}
174174

175+
protected def getFileActionsInLastVersion(log: DeltaLog): (Seq[AddFile], Seq[RemoveFile]) = {
176+
val version = log.update().version
177+
val allFiles = log.getChanges(version).toSeq.head._2
178+
val add = allFiles.collect { case a: AddFile => a }
179+
val remove = allFiles.collect { case r: RemoveFile => r }
180+
(add, remove)
181+
}
182+
175183
protected def serializeRoaringBitmapArrayWithDefaultFormat(
176184
dv: RoaringBitmapArray): Array[Byte] = {
177185
val serializationFormat = RoaringBitmapArrayFormat.Portable

core/src/test/scala/org/apache/spark/sql/delta/DeltaGenerateSymlinkManifestSuite.scala

Lines changed: 2 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -600,10 +600,8 @@ trait DeltaGenerateSymlinkManifestSuiteBase extends QueryTest
600600
subClass = ExistingDeletionVectorsWithIncrementalManifestGeneration) {
601601
setEnabledIncrementalManifest(tablePath, enabled = true)
602602
}
603-
// Run optimize to delete the DVs and rewrite the data files
604-
withSQLConf(DeltaSQLConf.DELTA_OPTIMIZE_MAX_DELETED_ROWS_RATIO.key -> "0.00001") {
605-
spark.sql(s"OPTIMIZE delta.`$tablePath`")
606-
}
603+
// Purge
604+
spark.sql(s"REORG TABLE delta.`$tablePath` APPLY (PURGE)")
607605
assert(getFilesWithDeletionVectors(deltaLog).isEmpty)
608606
// Now it should work.
609607
setEnabledIncrementalManifest(tablePath, enabled = true)

core/src/test/scala/org/apache/spark/sql/delta/DeltaVacuumSuite.scala

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -833,12 +833,10 @@ class DeltaVacuumSuite
833833
// Helper method to remove the DVs in Delta table and rewrite the data files
834834
def purgeDVs(tableName: String): Unit = {
835835
withSQLConf(
836-
DeltaSQLConf.DELTA_OPTIMIZE_MAX_DELETED_ROWS_RATIO.key -> "0.0001",
837-
DeltaSQLConf.DELTA_OPTIMIZE_MIN_FILE_SIZE.key -> "2",
838836
// Set the max file size to low so that we always rewrite the single file without DVs
839837
// and not combining with other data files.
840838
DeltaSQLConf.DELTA_OPTIMIZE_MAX_FILE_SIZE.key -> "2") {
841-
spark.sql(s"OPTIMIZE $tableName")
839+
spark.sql(s"REORG TABLE $tableName APPLY (PURGE)")
842840
}
843841
}
844842

core/src/test/scala/org/apache/spark/sql/delta/deletionvectors/DeletionVectorsSuite.scala

Lines changed: 0 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -562,14 +562,6 @@ class DeletionVectorsSuite extends QueryTest
562562
Seq((count, sum)).toDF())
563563
}
564564

565-
private def getFileActionsInLastVersion(log: DeltaLog): (Seq[AddFile], Seq[RemoveFile]) = {
566-
val version = log.update().version
567-
val allFiles = log.getChanges(version).toSeq.head._2
568-
val add = allFiles.collect { case a: AddFile => a }
569-
val remove = allFiles.collect { case r: RemoveFile => r }
570-
(add, remove)
571-
}
572-
573565
private def assertPlanContains(queryDf: DataFrame, expected: String): Unit = {
574566
val optimizedPlan = queryDf.queryExecution.analyzed.toString()
575567
assert(optimizedPlan.contains(expected), s"Plan is missing `$expected`: $optimizedPlan")

0 commit comments

Comments
 (0)