Skip to content

Commit 7d41fb7

Browse files
Arun Ravi M Vtdas
Arun Ravi M V
authored andcommitted
Use inventory reservoir as source for all files and dirs
- Currently, users have large tables with daily/hourly partitions for many years, among all these partitions only recent ones are subjected to change due to job reruns, corrections, and late arriving events. - When Vacuum is run on these tables, the listing of files is performed on all the partitions and it runs for several hours/days. This duration grows as tables grow and vacuum becomes a major overhead for customers especially when they have hundreds or thousands of such delta tables. File system scan takes the most amount of time in Vacuum operation for large tables, mostly due to the parallelism achievable and API throttling on the object stores. - This change provides a way for users to pass a reservoir of files generated externally (eg: from inventory reports of cloud stores) as a delta table or as a spark SQL query (having a predefined schema). The vacuum operation when provided with such a reservoir data frame will skip the listing operation and use it as a source of all files in the storage. "Resolves #1691". - Unit Testing (` build/sbt 'testOnly org.apache.spark.sql.delta.DeltaVacuumSuite'`) yes, the MR accepts an optional method to pass inventory. `VACUUM table_name [USING INVENTORY <reservoir-delta-table>] [RETAIN num HOURS] [DRY RUN]` `VACUUM table_name [USING INVENTORY <reservoir-query>] [RETAIN num HOURS] [DRY RUN]` eg: `VACUUM test_db.table using inventory select * from reservoir_table RETAIN 168 HOURS dry run` Closes #2257 Co-authored-by: Arun Ravi M V <arunravi.mv@grabtaxi.com> Signed-off-by: Bart Samwel <bart.samwel@databricks.com> GitOrigin-RevId: 2bc824e524c677dd5f3a7ed787762df60c3b6d86
1 parent 4c8a442 commit 7d41fb7

File tree

8 files changed

+310
-32
lines changed

8 files changed

+310
-32
lines changed

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

+12-1
Original file line numberDiff line numberDiff line change
@@ -73,6 +73,7 @@ singleStatement
7373
// If you add keywords here that should not be reserved, add them to 'nonReserved' list.
7474
statement
7575
: VACUUM (path=STRING | table=qualifiedName)
76+
(USING INVENTORY (inventoryTable=qualifiedName | LEFT_PAREN inventoryQuery=subQuery RIGHT_PAREN))?
7677
(RETAIN number HOURS)? (DRY RUN)? #vacuumTable
7778
| (DESC | DESCRIBE) DETAIL (path=STRING | table=qualifiedName) #describeDeltaDetail
7879
| GENERATE modeName=identifier FOR TABLE table=qualifiedName #generate
@@ -214,6 +215,14 @@ predicateToken
214215
: .+?
215216
;
216217

218+
// We don't have an expression rule in our grammar here, so we just grab the tokens and defer
219+
// parsing them to later. Although this is the same as `exprToken`, `predicateToken`, we have to re-define it to
220+
// workaround an ANTLR issue (https://github.com/delta-io/delta/issues/1205). Should we remove this after
221+
// https://github.com/delta-io/delta/pull/1800
222+
subQuery
223+
: .+?
224+
;
225+
217226
// We don't have an expression rule in our grammar here, so we just grab the tokens and defer
218227
// parsing them to later.
219228
exprToken
@@ -223,7 +232,7 @@ exprToken
223232
// Add keywords here so that people's queries don't break if they have a column name as one of
224233
// these tokens
225234
nonReserved
226-
: VACUUM | RETAIN | HOURS | DRY | RUN
235+
: VACUUM | USING | INVENTORY | RETAIN | HOURS | DRY | RUN
227236
| CONVERT | TO | DELTA | PARTITIONED | BY
228237
| DESC | DESCRIBE | LIMIT | DETAIL
229238
| GENERATE | FOR | TABLE | CHECK | EXISTS | OPTIMIZE
@@ -266,6 +275,7 @@ HISTORY: 'HISTORY';
266275
HOURS: 'HOURS';
267276
ICEBERG_COMPAT_VERSION: 'ICEBERG_COMPAT_VERSION';
268277
IF: 'IF';
278+
INVENTORY: 'INVENTORY';
269279
LEFT_PAREN: '(';
270280
LIMIT: 'LIMIT';
271281
LOCATION: 'LOCATION';
@@ -296,6 +306,7 @@ TO: 'TO';
296306
TRUE: 'TRUE';
297307
UNIFORM: 'UNIFORM';
298308
UPGRADE: 'UPGRADE';
309+
USING: 'USING';
299310
VACUUM: 'VACUUM';
300311
VERSION: 'VERSION';
301312
WHERE: 'WHERE';

spark/src/main/resources/error/delta-error-classes.json

+6
Original file line numberDiff line numberDiff line change
@@ -1176,6 +1176,12 @@
11761176
],
11771177
"sqlState" : "22006"
11781178
},
1179+
"DELTA_INVALID_INVENTORY_SCHEMA" : {
1180+
"message" : [
1181+
"The schema for the specified INVENTORY does not contain all of the required fields. Required fields are: <expectedSchema>"
1182+
],
1183+
"sqlState" : "42000"
1184+
},
11791185
"DELTA_INVALID_ISOLATION_LEVEL" : {
11801186
"message" : [
11811187
"invalid isolation level '<isolationLevel>'"

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

+6-4
Original file line numberDiff line numberDiff line change
@@ -319,10 +319,12 @@ class DeltaSqlAstBuilder extends DeltaSqlBaseBaseVisitor[AnyRef] {
319319
*/
320320
override def visitVacuumTable(ctx: VacuumTableContext): AnyRef = withOrigin(ctx) {
321321
VacuumTableCommand(
322-
Option(ctx.path).map(string),
323-
Option(ctx.table).map(visitTableIdentifier),
324-
Option(ctx.number).map(_.getText.toDouble),
325-
ctx.RUN != null)
322+
path = Option(ctx.path).map(string),
323+
table = Option(ctx.table).map(visitTableIdentifier),
324+
inventoryTable = Option(ctx.inventoryTable).map(visitTableIdentifier),
325+
inventoryQuery = Option(ctx.inventoryQuery).map(extractRawText),
326+
horizonHours = Option(ctx.number).map(_.getText.toDouble),
327+
dryRun = ctx.RUN != null)
326328
}
327329

328330
/** Provides a list of unresolved attributes for multi dimensional clustering. */

spark/src/main/scala/io/delta/tables/execution/VacuumTableCommand.scala

+17-3
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,8 @@ import org.apache.spark.sql.{Row, SparkSession}
2020
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
2121
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, UnaryNode}
2222
import org.apache.spark.sql.catalyst.TableIdentifier
23+
import org.apache.spark.sql.catalyst.analysis.UnresolvedTable
24+
import org.apache.spark.sql.delta.catalog.DeltaTableV2
2325
import org.apache.spark.sql.delta.{DeltaErrors, DeltaLog, DeltaTableIdentifier, DeltaTableUtils, UnresolvedDeltaPathOrIdentifier}
2426
import org.apache.spark.sql.delta.commands.DeltaCommand
2527
import org.apache.spark.sql.delta.commands.VacuumCommand
@@ -30,12 +32,16 @@ import org.apache.spark.sql.types.StringType
3032
/**
3133
* The `vacuum` command implementation for Spark SQL. Example SQL:
3234
* {{{
33-
* VACUUM ('/path/to/dir' | delta.`/path/to/dir`) [RETAIN number HOURS] [DRY RUN];
35+
* VACUUM ('/path/to/dir' | delta.`/path/to/dir`)
36+
* [USING INVENTORY (delta.`/path/to/dir`| ( sub_query ))]
37+
* [RETAIN number HOURS] [DRY RUN];
3438
* }}}
3539
*/
3640
case class VacuumTableCommand(
3741
override val child: LogicalPlan,
3842
horizonHours: Option[Double],
43+
inventoryTable: Option[LogicalPlan],
44+
inventoryQuery: Option[String],
3945
dryRun: Boolean) extends RunnableCommand with UnaryNode with DeltaCommand {
4046

4147
override val output: Seq[Attribute] =
@@ -53,17 +59,25 @@ case class VacuumTableCommand(
5359
"VACUUM",
5460
DeltaTableIdentifier(path = Some(deltaTable.path.toString)))
5561
}
56-
VacuumCommand.gc(sparkSession, deltaTable.deltaLog, dryRun, horizonHours).collect()
62+
val inventory = inventoryTable.map(sparkSession.sessionState.analyzer.execute)
63+
.map(p => Some(getDeltaTable(p, "VACUUM").toDf(sparkSession)))
64+
.getOrElse(inventoryQuery.map(sparkSession.sql))
65+
VacuumCommand.gc(sparkSession, deltaTable.deltaLog, dryRun, horizonHours,
66+
inventory).collect()
5767
}
5868
}
5969

6070
object VacuumTableCommand {
6171
def apply(
6272
path: Option[String],
6373
table: Option[TableIdentifier],
74+
inventoryTable: Option[TableIdentifier],
75+
inventoryQuery: Option[String],
6476
horizonHours: Option[Double],
6577
dryRun: Boolean): VacuumTableCommand = {
6678
val child = UnresolvedDeltaPathOrIdentifier(path, table, "VACUUM")
67-
VacuumTableCommand(child, horizonHours, dryRun)
79+
val unresolvedInventoryTable = inventoryTable.map(rt =>
80+
UnresolvedTable(rt.nameParts, "VACUUM", relationTypeMismatchHint = None))
81+
VacuumTableCommand(child, horizonHours, unresolvedInventoryTable, inventoryQuery, dryRun)
6882
}
6983
}

spark/src/main/scala/org/apache/spark/sql/delta/DeltaErrors.scala

+6-1
Original file line numberDiff line numberDiff line change
@@ -566,7 +566,12 @@ trait DeltaErrorsBase
566566
errorClass = "DELTA_INVALID_CHARACTERS_IN_COLUMN_NAME",
567567
messageParameters = Array(name))
568568
}
569-
569+
def invalidInventorySchema(expectedSchema: String): Throwable = {
570+
new DeltaAnalysisException(
571+
errorClass = "DELTA_INVALID_INVENTORY_SCHEMA",
572+
messageParameters = Array(expectedSchema)
573+
)
574+
}
570575
def invalidIsolationLevelException(s: String): Throwable = {
571576
new DeltaIllegalArgumentException(
572577
errorClass = "DELTA_INVALID_ISOLATION_LEVEL",

spark/src/main/scala/org/apache/spark/sql/delta/commands/VacuumCommand.scala

+67-15
Original file line numberDiff line numberDiff line change
@@ -20,9 +20,7 @@ package org.apache.spark.sql.delta.commands
2020
import java.net.URI
2121
import java.util.Date
2222
import java.util.concurrent.TimeUnit
23-
2423
import scala.collection.JavaConverters._
25-
2624
import org.apache.spark.sql.delta._
2725
import org.apache.spark.sql.delta.actions.{AddFile, FileAction, RemoveFile}
2826
import org.apache.spark.sql.delta.sources.DeltaSQLConf
@@ -31,12 +29,12 @@ import org.apache.spark.sql.delta.util.DeltaFileOperations.tryDeleteNonRecursive
3129
import com.fasterxml.jackson.databind.annotation.JsonDeserialize
3230
import org.apache.hadoop.conf.Configuration
3331
import org.apache.hadoop.fs.{FileSystem, Path}
34-
3532
import org.apache.spark.broadcast.Broadcast
36-
import org.apache.spark.sql.{Column, DataFrame, Dataset, SparkSession}
33+
import org.apache.spark.sql.{Column, DataFrame, Dataset, Encoder, SparkSession}
3734
import org.apache.spark.sql.execution.metric.SQLMetric
3835
import org.apache.spark.sql.execution.metric.SQLMetrics.createMetric
39-
import org.apache.spark.sql.functions.{col, count, sum}
36+
import org.apache.spark.sql.functions.{col, count, lit, replace, startswith, substr, sum}
37+
import org.apache.spark.sql.types.{BooleanType, LongType, StringType, StructField, StructType}
4038
import org.apache.spark.util.{Clock, SerializableConfiguration, SystemClock}
4139

4240
/**
@@ -51,6 +49,21 @@ import org.apache.spark.util.{Clock, SerializableConfiguration, SystemClock}
5149
object VacuumCommand extends VacuumCommandImpl with Serializable {
5250

5351
case class FileNameAndSize(path: String, length: Long)
52+
53+
/**
54+
* path : fully qualified uri
55+
* length: size in bytes
56+
* isDir: boolean indicating if it is a directory
57+
* modificationTime: file update time in milliseconds
58+
*/
59+
val INVENTORY_SCHEMA = StructType(
60+
Seq(
61+
StructField("path", StringType),
62+
StructField("length", LongType),
63+
StructField("isDir", BooleanType),
64+
StructField("modificationTime", LongType)
65+
))
66+
5467
/**
5568
* Additional check on retention duration to prevent people from shooting themselves in the foot.
5669
*/
@@ -125,19 +138,55 @@ object VacuumCommand extends VacuumCommandImpl with Serializable {
125138
}.toDF("path")
126139
}
127140

141+
def getFilesFromInventory(basePath: String,
142+
partitionColumns: Seq[String],
143+
inventory: DataFrame): Dataset[SerializableFileStatus] = {
144+
implicit val fileNameAndSizeEncoder: Encoder[SerializableFileStatus] =
145+
org.apache.spark.sql.Encoders.product[SerializableFileStatus]
146+
147+
// filter out required fields from provided inventory DF
148+
val inventorySchema = StructType(
149+
inventory.schema.fields.filter(f => INVENTORY_SCHEMA.fields.map(_.name).contains(f.name))
150+
)
151+
if (inventorySchema != INVENTORY_SCHEMA) {
152+
throw DeltaErrors.invalidInventorySchema(INVENTORY_SCHEMA.treeString)
153+
}
154+
155+
inventory
156+
.filter(startswith(col("path"), lit(s"$basePath/")))
157+
.select(
158+
substr(col("path"), lit(basePath.length + 2)).as("path"),
159+
col("length"), col("isDir"), col("modificationTime")
160+
)
161+
.flatMap {
162+
row =>
163+
val path = row.getString(0)
164+
if(!DeltaTableUtils.isHiddenDirectory(partitionColumns, path)) {
165+
Seq(SerializableFileStatus(path,
166+
row.getLong(1), row.getBoolean(2), row.getLong(3)))
167+
} else {
168+
None
169+
}
170+
}
171+
}
172+
128173
/**
129-
* Clears all untracked files and folders within this table. First lists all the files and
130-
* directories in the table, and gets the relative paths with respect to the base of the
131-
* table. Then it gets the list of all tracked files for this table, which may or may not
132-
* be within the table base path, and gets the relative paths of all the tracked files with
133-
* respect to the base of the table. Files outside of the table path will be ignored.
134-
* Then we take a diff of the files and delete directories that were already empty, and all files
135-
* that are within the table that are no longer tracked.
174+
* Clears all untracked files and folders within this table. If the inventory is not provided
175+
* then the command first lists all the files and directories in the table, if inventory is
176+
* provided then it will be used for identifying files and directories within the table and
177+
* gets the relative paths with respect to the base of the table. Then the command gets the
178+
* list of all tracked files for this table, which may or may not be within the table base path,
179+
* and gets the relative paths of all the tracked files with respect to the base of the table.
180+
* Files outside of the table path will be ignored. Then we take a diff of the files and delete
181+
* directories that were already empty, and all files that are within the table that are no longer
182+
* tracked.
136183
*
137184
* @param dryRun If set to true, no files will be deleted. Instead, we will list all files and
138185
* directories that will be cleared.
139186
* @param retentionHours An optional parameter to override the default Delta tombstone retention
140187
* period
188+
* @param inventory An optional dataframe of files and directories within the table generated
189+
* from sources like blob store inventory report
141190
* @return A Dataset containing the paths of the files/folders to delete in dryRun mode. Otherwise
142191
* returns the base path of the table.
143192
*/
@@ -146,6 +195,7 @@ object VacuumCommand extends VacuumCommandImpl with Serializable {
146195
deltaLog: DeltaLog,
147196
dryRun: Boolean = true,
148197
retentionHours: Option[Double] = None,
198+
inventory: Option[DataFrame] = None,
149199
clock: Clock = new SystemClock): DataFrame = {
150200
recordDeltaOperation(deltaLog, "delta.gc") {
151201

@@ -189,16 +239,18 @@ object VacuumCommand extends VacuumCommandImpl with Serializable {
189239

190240
val partitionColumns = snapshot.metadata.partitionSchema.fieldNames
191241
val parallelism = spark.sessionState.conf.parallelPartitionDiscoveryParallelism
192-
193-
val allFilesAndDirs = DeltaFileOperations.recursiveListDirs(
242+
val allFilesAndDirsWithDuplicates = inventory match {
243+
case Some(inventoryDF) => getFilesFromInventory(basePath, partitionColumns, inventoryDF)
244+
case None => DeltaFileOperations.recursiveListDirs(
194245
spark,
195246
Seq(basePath),
196247
hadoopConf,
197248
hiddenDirNameFilter = DeltaTableUtils.isHiddenDirectory(partitionColumns, _),
198249
hiddenFileNameFilter = DeltaTableUtils.isHiddenDirectory(partitionColumns, _),
199250
fileListingParallelism = Option(parallelism)
200251
)
201-
.groupByKey(_.path)
252+
}
253+
val allFilesAndDirs = allFilesAndDirsWithDuplicates.groupByKey(_.path)
202254
.mapGroups { (k, v) =>
203255
val duplicates = v.toSeq
204256
// of all the duplicates we can return the newest file.

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

+11-8
Original file line numberDiff line numberDiff line change
@@ -40,27 +40,30 @@ class DeltaSqlParserSuite extends SparkFunSuite with SQLHelper {
4040
// Setting `delegate` to `null` is fine. The following tests don't need to touch `delegate`.
4141
val parser = new DeltaSqlParser(null)
4242
assert(parser.parsePlan("vacuum 123_") ===
43-
VacuumTableCommand(UnresolvedTable(Seq("123_"), "VACUUM", None), None, false))
43+
VacuumTableCommand(UnresolvedTable(Seq("123_"), "VACUUM", None), None, None, None, false))
4444
assert(parser.parsePlan("vacuum 1a.123_") ===
45-
VacuumTableCommand(UnresolvedTable(Seq("1a", "123_"), "VACUUM", None), None, false))
45+
VacuumTableCommand(UnresolvedTable(Seq("1a", "123_"), "VACUUM", None),
46+
None, None, None, false))
4647
assert(parser.parsePlan("vacuum a.123A") ===
47-
VacuumTableCommand(UnresolvedTable(Seq("a", "123A"), "VACUUM", None), None, false))
48+
VacuumTableCommand(UnresolvedTable(Seq("a", "123A"), "VACUUM", None),
49+
None, None, None, false))
4850
assert(parser.parsePlan("vacuum a.123E3_column") ===
49-
VacuumTableCommand(UnresolvedTable(Seq("a", "123E3_column"), "VACUUM", None), None, false))
51+
VacuumTableCommand(UnresolvedTable(Seq("a", "123E3_column"), "VACUUM", None),
52+
None, None, None, false))
5053
assert(parser.parsePlan("vacuum a.123D_column") ===
5154
VacuumTableCommand(UnresolvedTable(Seq("a", "123D_column"), "VACUUM", None),
52-
None, false))
55+
None, None, None, false))
5356
assert(parser.parsePlan("vacuum a.123BD_column") ===
5457
VacuumTableCommand(UnresolvedTable(Seq("a", "123BD_column"), "VACUUM", None),
55-
None, false))
58+
None, None, None, false))
5659

5760
assert(parser.parsePlan("vacuum delta.`/tmp/table`") ===
5861
VacuumTableCommand(UnresolvedTable(Seq("delta", "/tmp/table"), "VACUUM", None),
59-
None, false))
62+
None, None, None, false))
6063

6164
assert(parser.parsePlan("vacuum \"/tmp/table\"") ===
6265
VacuumTableCommand(
63-
UnresolvedPathBasedDeltaTable("/tmp/table", Map.empty, "VACUUM"), None, false))
66+
UnresolvedPathBasedDeltaTable("/tmp/table", Map.empty, "VACUUM"), None, None, None, false))
6467
}
6568

6669
test("Restore command is parsed as expected") {

0 commit comments

Comments
 (0)