@@ -31,11 +31,14 @@ import org.apache.spark.sql.delta.sources.DeltaSQLConf
31
31
import org .apache .spark .sql .delta .util .DeltaFileOperations
32
32
import org .apache .spark .sql .delta .util .DeltaFileOperations .tryDeleteNonRecursive
33
33
import com .fasterxml .jackson .databind .annotation .JsonDeserialize
34
+ import org .apache .hadoop .conf .Configuration
34
35
import org .apache .hadoop .fs .{FileSystem , Path }
35
36
36
37
import org .apache .spark .broadcast .Broadcast
37
- import org .apache .spark .sql .{DataFrame , Dataset , SparkSession }
38
- import org .apache .spark .sql .internal .SQLConf
38
+ import org .apache .spark .sql .{Column , DataFrame , Dataset , SparkSession }
39
+ import org .apache .spark .sql .execution .metric .SQLMetric
40
+ import org .apache .spark .sql .execution .metric .SQLMetrics .createMetric
41
+ import org .apache .spark .sql .functions .{col , count , sum }
39
42
import org .apache .spark .util .{Clock , SerializableConfiguration , SystemClock }
40
43
41
44
/**
@@ -49,6 +52,8 @@ import org.apache.spark.util.{Clock, SerializableConfiguration, SystemClock}
49
52
*/
50
53
object VacuumCommand extends VacuumCommandImpl with Serializable {
51
54
55
+ case class FileNameAndSize (path : String , length : Long )
56
+
52
57
/**
53
58
* Additional check on retention duration to prevent people from shooting themselves in the foot.
54
59
*/
@@ -204,21 +209,32 @@ object VacuumCommand extends VacuumCommandImpl with Serializable {
204
209
val fs = reservoirBase.getFileSystem(hadoopConf.value.value)
205
210
fileStatusIterator.flatMap { fileStatus =>
206
211
if (fileStatus.isDir) {
207
- Iterator .single(relativize(fileStatus.getPath, fs, reservoirBase, isDir = true ))
212
+ Iterator .single(FileNameAndSize (
213
+ relativize(fileStatus.getPath, fs, reservoirBase, isDir = true ), 0L ))
208
214
} else {
209
215
val dirs = getAllSubdirs(basePath, fileStatus.path, fs)
210
216
val dirsWithSlash = dirs.map { p =>
211
- relativize(new Path (p), fs, reservoirBase, isDir = true )
217
+ val relativizedPath = relativize(new Path (p), fs, reservoirBase, isDir = true )
218
+ FileNameAndSize (relativizedPath, 0L )
212
219
}
213
220
dirsWithSlash ++ Iterator (
214
- relativize(new Path (fileStatus.path), fs, reservoirBase, isDir = false ))
221
+ FileNameAndSize (relativize(
222
+ fileStatus.getPath, fs, reservoirBase, isDir = false ), fileStatus.length))
215
223
}
216
224
}
217
- }.groupBy($" value" as ' path )
218
- .count()
225
+ }.groupBy(col(" path" )).agg(count(new Column (" *" )).as(" count" ), sum(" length" ).as(" length" ))
219
226
.join(validFiles, Seq (" path" ), " leftanti" )
220
227
.where(' count === 1 )
221
- .select(' path )
228
+
229
+ val sizeOfDataToDeleteRow = diff.agg(sum(" length" ).cast(" long" )).first
230
+ val sizeOfDataToDelete = if (sizeOfDataToDeleteRow.isNullAt(0 )) {
231
+ 0L
232
+ } else {
233
+ sizeOfDataToDeleteRow.getLong(0 )
234
+ }
235
+
236
+ val diffFiles = diff
237
+ .select(col(" path" ))
222
238
.as[String ]
223
239
.map { relativePath =>
224
240
assert(! stringToPath(relativePath).isAbsolute,
@@ -227,31 +243,33 @@ object VacuumCommand extends VacuumCommandImpl with Serializable {
227
243
}
228
244
229
245
if (dryRun) {
230
- val numFiles = diff .count()
246
+ val numFiles = diffFiles .count()
231
247
val stats = DeltaVacuumStats (
232
248
isDryRun = true ,
233
249
specifiedRetentionMillis = retentionMillis,
234
250
defaultRetentionMillis = deltaLog.tombstoneRetentionMillis,
235
251
minRetainedTimestamp = deleteBeforeTimestamp,
236
252
dirsPresentBeforeDelete = dirCounts,
237
- objectsDeleted = numFiles)
253
+ objectsDeleted = numFiles,
254
+ sizeOfDataToDelete = sizeOfDataToDelete)
238
255
239
256
recordDeltaEvent(deltaLog, " delta.gc.stats" , data = stats)
240
- logConsole(s " Found $numFiles files and directories in a total of " +
241
- s " $dirCounts directories that are safe to delete. " )
257
+ logConsole(s " Found $numFiles files ( $sizeOfDataToDelete bytes) and directories in " +
258
+ s " a total of $dirCounts directories that are safe to delete. " )
242
259
243
- return diff .map(f => stringToPath(f).toString).toDF(" path" )
260
+ return diffFiles .map(f => stringToPath(f).toString).toDF(" path" )
244
261
}
245
262
logVacuumStart(
246
263
spark,
247
264
deltaLog,
248
265
path,
249
- diff,
266
+ diffFiles,
267
+ sizeOfDataToDelete,
250
268
retentionMillis,
251
269
deltaLog.tombstoneRetentionMillis)
252
270
253
271
val filesDeleted = try {
254
- delete(diff , spark, basePath, hadoopConf, parallelDeleteEnabled,
272
+ delete(diffFiles , spark, basePath, hadoopConf, parallelDeleteEnabled,
255
273
parallelDeletePartitions)
256
274
} catch { case t : Throwable =>
257
275
logVacuumEnd(deltaLog, spark, path)
@@ -263,7 +281,8 @@ object VacuumCommand extends VacuumCommandImpl with Serializable {
263
281
defaultRetentionMillis = deltaLog.tombstoneRetentionMillis,
264
282
minRetainedTimestamp = deleteBeforeTimestamp,
265
283
dirsPresentBeforeDelete = dirCounts,
266
- objectsDeleted = filesDeleted)
284
+ objectsDeleted = filesDeleted,
285
+ sizeOfDataToDelete = sizeOfDataToDelete)
267
286
recordDeltaEvent(deltaLog, " delta.gc.stats" , data = stats)
268
287
logVacuumEnd(deltaLog, spark, path, Some (filesDeleted), Some (dirCounts))
269
288
@@ -277,22 +296,122 @@ object VacuumCommand extends VacuumCommandImpl with Serializable {
277
296
278
297
trait VacuumCommandImpl extends DeltaCommand {
279
298
299
+ private val supportedFsForLogging = Seq (
300
+ " wasbs" , " wasbss" , " abfs" , " abfss" , " adl" , " gs" , " file" , " hdfs"
301
+ )
302
+
303
+ /**
304
+ * Returns whether we should record vacuum metrics in the delta log.
305
+ */
306
+ private def shouldLogVacuum (
307
+ spark : SparkSession ,
308
+ deltaLog : DeltaLog ,
309
+ hadoopConf : Configuration ,
310
+ path : Path ): Boolean = {
311
+ val logVacuumConf = spark.sessionState.conf.getConf(DeltaSQLConf .DELTA_VACUUM_LOGGING_ENABLED )
312
+
313
+ if (logVacuumConf.nonEmpty) {
314
+ return logVacuumConf.get
315
+ }
316
+
317
+ val logStore = deltaLog.store
318
+
319
+ try {
320
+ val rawResolvedUri : URI = logStore.resolvePathOnPhysicalStorage(path, hadoopConf).toUri
321
+ val scheme = rawResolvedUri.getScheme
322
+ if (supportedFsForLogging.contains(scheme)) {
323
+ true
324
+ } else {
325
+ false
326
+ }
327
+ } catch {
328
+ case _ : UnsupportedOperationException =>
329
+ logWarning(" Vacuum event logging" +
330
+ " not enabled on this file system because we cannot detect your cloud storage type." )
331
+ false
332
+ }
333
+ }
334
+
335
+ /**
336
+ * Record Vacuum specific metrics in the commit log at the START of vacuum.
337
+ *
338
+ * @param spark - spark session
339
+ * @param deltaLog - DeltaLog of the table
340
+ * @param path - the (data) path to the root of the table
341
+ * @param diff - the list of paths (files, directories) that are safe to delete
342
+ * @param sizeOfDataToDelete - the amount of data (bytes) to be deleted
343
+ * @param specifiedRetentionMillis - the optional override retention period (millis) to keep
344
+ * logically removed files before deleting them
345
+ * @param defaultRetentionMillis - the default retention period (millis)
346
+ */
280
347
protected def logVacuumStart (
281
348
spark : SparkSession ,
282
349
deltaLog : DeltaLog ,
283
350
path : Path ,
284
351
diff : Dataset [String ],
352
+ sizeOfDataToDelete : Long ,
285
353
specifiedRetentionMillis : Option [Long ],
286
354
defaultRetentionMillis : Long ): Unit = {
287
- logInfo(s " Deleting untracked files and empty directories in $path" )
355
+ logInfo(s " Deleting untracked files and empty directories in $path. The amount of data to be " +
356
+ s " deleted is $sizeOfDataToDelete (in bytes) " )
357
+
358
+ // We perform an empty commit in order to record information about the Vacuum
359
+ if (shouldLogVacuum(spark, deltaLog, deltaLog.newDeltaHadoopConf(), path)) {
360
+ val checkEnabled =
361
+ spark.sessionState.conf.getConf(DeltaSQLConf .DELTA_VACUUM_RETENTION_CHECK_ENABLED )
362
+ val txn = deltaLog.startTransaction()
363
+ val metrics = Map [String , SQLMetric ](
364
+ " numFilesToDelete" -> createMetric(spark.sparkContext, " number of files to deleted" ),
365
+ " sizeOfDataToDelete" -> createMetric(spark.sparkContext,
366
+ " The total amount of data to be deleted in bytes" )
367
+ )
368
+ metrics(" numFilesToDelete" ).set(diff.count())
369
+ metrics(" sizeOfDataToDelete" ).set(sizeOfDataToDelete)
370
+ txn.registerSQLMetrics(spark, metrics)
371
+ txn.commit(actions = Seq (), DeltaOperations .VacuumStart (
372
+ checkEnabled,
373
+ specifiedRetentionMillis,
374
+ defaultRetentionMillis
375
+ ))
376
+ }
288
377
}
289
378
379
+ /**
380
+ * Record Vacuum specific metrics in the commit log at the END of vacuum.
381
+ *
382
+ * @param deltaLog - DeltaLog of the table
383
+ * @param spark - spark session
384
+ * @param path - the (data) path to the root of the table
385
+ * @param filesDeleted - if the vacuum completed this will contain the number of files deleted.
386
+ * if the vacuum failed, this will be None.
387
+ * @param dirCounts - if the vacuum completed this will contain the number of directories
388
+ * vacuumed. if the vacuum failed, this will be None.
389
+ */
290
390
protected def logVacuumEnd (
291
391
deltaLog : DeltaLog ,
292
392
spark : SparkSession ,
293
393
path : Path ,
294
394
filesDeleted : Option [Long ] = None ,
295
395
dirCounts : Option [Long ] = None ): Unit = {
396
+ if (shouldLogVacuum(spark, deltaLog, deltaLog.newDeltaHadoopConf(), path)) {
397
+ val txn = deltaLog.startTransaction()
398
+ val status = if (filesDeleted.isEmpty && dirCounts.isEmpty) { " FAILED" } else { " COMPLETED" }
399
+ if (filesDeleted.nonEmpty && dirCounts.nonEmpty) {
400
+ val metrics = Map [String , SQLMetric ](
401
+ " numDeletedFiles" -> createMetric(spark.sparkContext, " number of files deleted." ),
402
+ " numVacuumedDirectories" ->
403
+ createMetric(spark.sparkContext, " num of directories vacuumed." ),
404
+ " status" -> createMetric(spark.sparkContext, " status of vacuum" )
405
+ )
406
+ metrics(" numDeletedFiles" ).set(filesDeleted.get)
407
+ metrics(" numVacuumedDirectories" ).set(dirCounts.get)
408
+ txn.registerSQLMetrics(spark, metrics)
409
+ }
410
+ txn.commit(actions = Seq (), DeltaOperations .VacuumEnd (
411
+ status
412
+ ))
413
+ }
414
+
296
415
if (filesDeleted.nonEmpty) {
297
416
logConsole(s " Deleted ${filesDeleted.get} files and directories in a total " +
298
417
s " of ${dirCounts.get} directories. " )
@@ -362,4 +481,5 @@ case class DeltaVacuumStats(
362
481
defaultRetentionMillis : Long ,
363
482
minRetainedTimestamp : Long ,
364
483
dirsPresentBeforeDelete : Long ,
365
- objectsDeleted : Long )
484
+ objectsDeleted : Long ,
485
+ sizeOfDataToDelete : Long )
0 commit comments