@@ -149,6 +149,7 @@ object VacuumCommand extends VacuumCommandImpl with Serializable {
149
149
clock : Clock = new SystemClock ): DataFrame = {
150
150
recordDeltaOperation(deltaLog, " delta.gc" ) {
151
151
152
+ val vacuumStartTime = System .currentTimeMillis()
152
153
val path = deltaLog.dataPath
153
154
val deltaHadoopConf = deltaLog.newDeltaHadoopConf()
154
155
val fs = path.getFileSystem(deltaHadoopConf)
@@ -210,6 +211,7 @@ object VacuumCommand extends VacuumCommandImpl with Serializable {
210
211
org.apache.spark.sql.Encoders .product[FileNameAndSize ]
211
212
212
213
val dirCounts = allFilesAndDirs.where(col(" isDir" )).count() + 1 // +1 for the base path
214
+ val filesAndDirsPresentBeforeDelete = allFilesAndDirs.count()
213
215
214
216
// The logic below is as follows:
215
217
// 1. We take all the files and directories listed in our reservoir
@@ -264,6 +266,7 @@ object VacuumCommand extends VacuumCommandImpl with Serializable {
264
266
val timeTakenToIdentifyEligibleFiles =
265
267
System .currentTimeMillis() - startTimeToIdentifyEligibleFiles
266
268
269
+
267
270
val numFiles = diffFiles.count()
268
271
if (dryRun) {
269
272
val stats = DeltaVacuumStats (
@@ -272,14 +275,19 @@ object VacuumCommand extends VacuumCommandImpl with Serializable {
272
275
defaultRetentionMillis = snapshotTombstoneRetentionMillis,
273
276
minRetainedTimestamp = deleteBeforeTimestamp,
274
277
dirsPresentBeforeDelete = dirCounts,
278
+ filesAndDirsPresentBeforeDelete = filesAndDirsPresentBeforeDelete,
275
279
objectsDeleted = numFiles,
276
280
sizeOfDataToDelete = sizeOfDataToDelete,
277
281
timeTakenToIdentifyEligibleFiles = timeTakenToIdentifyEligibleFiles,
278
- timeTakenForDelete = 0L )
282
+ timeTakenForDelete = 0L ,
283
+ vacuumStartTime = vacuumStartTime,
284
+ vacuumEndTime = System .currentTimeMillis,
285
+ numPartitionColumns = partitionColumns.size
286
+ )
279
287
280
288
recordDeltaEvent(deltaLog, " delta.gc.stats" , data = stats)
281
- logConsole (s " Found $numFiles files ( $sizeOfDataToDelete bytes) and directories in " +
282
- s " a total of $dirCounts directories that are safe to delete. " )
289
+ logInfo (s " Found $numFiles files ( $sizeOfDataToDelete bytes) and directories in " +
290
+ s " a total of $dirCounts directories that are safe to delete. Vacuum stats: $stats " )
283
291
284
292
return diffFiles.map(f => stringToPath(f).toString).toDF(" path" )
285
293
}
@@ -308,12 +316,18 @@ object VacuumCommand extends VacuumCommandImpl with Serializable {
308
316
defaultRetentionMillis = snapshotTombstoneRetentionMillis,
309
317
minRetainedTimestamp = deleteBeforeTimestamp,
310
318
dirsPresentBeforeDelete = dirCounts,
319
+ filesAndDirsPresentBeforeDelete = filesAndDirsPresentBeforeDelete,
311
320
objectsDeleted = filesDeleted,
312
321
sizeOfDataToDelete = sizeOfDataToDelete,
313
322
timeTakenToIdentifyEligibleFiles = timeTakenToIdentifyEligibleFiles,
314
- timeTakenForDelete = timeTakenForDelete)
323
+ timeTakenForDelete = timeTakenForDelete,
324
+ vacuumStartTime = vacuumStartTime,
325
+ vacuumEndTime = System .currentTimeMillis,
326
+ numPartitionColumns = partitionColumns.size)
315
327
recordDeltaEvent(deltaLog, " delta.gc.stats" , data = stats)
316
328
logVacuumEnd(deltaLog, spark, path, Some (filesDeleted), Some (dirCounts))
329
+ logInfo(s " Deleted $filesDeleted files ( $sizeOfDataToDelete bytes) and directories in " +
330
+ s " a total of $dirCounts directories. Vacuum stats: $stats" )
317
331
318
332
319
333
spark.createDataset(Seq (basePath)).toDF(" path" )
@@ -576,7 +590,12 @@ case class DeltaVacuumStats(
576
590
defaultRetentionMillis : Long ,
577
591
minRetainedTimestamp : Long ,
578
592
dirsPresentBeforeDelete : Long ,
593
+ filesAndDirsPresentBeforeDelete : Long ,
579
594
objectsDeleted : Long ,
580
595
sizeOfDataToDelete : Long ,
581
596
timeTakenToIdentifyEligibleFiles : Long ,
582
- timeTakenForDelete : Long )
597
+ timeTakenForDelete : Long ,
598
+ vacuumStartTime : Long ,
599
+ vacuumEndTime : Long ,
600
+ numPartitionColumns : Long
601
+ )
0 commit comments