@@ -149,6 +149,7 @@ object VacuumCommand extends VacuumCommandImpl with Serializable {
149149 clock : Clock = new SystemClock ): DataFrame = {
150150 recordDeltaOperation(deltaLog, " delta.gc" ) {
151151
152+ val vacuumStartTime = System .currentTimeMillis()
152153 val path = deltaLog.dataPath
153154 val deltaHadoopConf = deltaLog.newDeltaHadoopConf()
154155 val fs = path.getFileSystem(deltaHadoopConf)
@@ -210,6 +211,7 @@ object VacuumCommand extends VacuumCommandImpl with Serializable {
210211 org.apache.spark.sql.Encoders .product[FileNameAndSize ]
211212
212213 val dirCounts = allFilesAndDirs.where(col(" isDir" )).count() + 1 // +1 for the base path
214+ val filesAndDirsPresentBeforeDelete = allFilesAndDirs.count()
213215
214216 // The logic below is as follows:
215217 // 1. We take all the files and directories listed in our reservoir
@@ -264,6 +266,7 @@ object VacuumCommand extends VacuumCommandImpl with Serializable {
264266 val timeTakenToIdentifyEligibleFiles =
265267 System .currentTimeMillis() - startTimeToIdentifyEligibleFiles
266268
269+
267270 val numFiles = diffFiles.count()
268271 if (dryRun) {
269272 val stats = DeltaVacuumStats (
@@ -272,14 +275,19 @@ object VacuumCommand extends VacuumCommandImpl with Serializable {
272275 defaultRetentionMillis = snapshotTombstoneRetentionMillis,
273276 minRetainedTimestamp = deleteBeforeTimestamp,
274277 dirsPresentBeforeDelete = dirCounts,
278+ filesAndDirsPresentBeforeDelete = filesAndDirsPresentBeforeDelete,
275279 objectsDeleted = numFiles,
276280 sizeOfDataToDelete = sizeOfDataToDelete,
277281 timeTakenToIdentifyEligibleFiles = timeTakenToIdentifyEligibleFiles,
278- timeTakenForDelete = 0L )
282+ timeTakenForDelete = 0L ,
283+ vacuumStartTime = vacuumStartTime,
284+ vacuumEndTime = System .currentTimeMillis,
285+ numPartitionColumns = partitionColumns.size
286+ )
279287
280288 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 " )
283291
284292 return diffFiles.map(f => stringToPath(f).toString).toDF(" path" )
285293 }
@@ -308,12 +316,18 @@ object VacuumCommand extends VacuumCommandImpl with Serializable {
308316 defaultRetentionMillis = snapshotTombstoneRetentionMillis,
309317 minRetainedTimestamp = deleteBeforeTimestamp,
310318 dirsPresentBeforeDelete = dirCounts,
319+ filesAndDirsPresentBeforeDelete = filesAndDirsPresentBeforeDelete,
311320 objectsDeleted = filesDeleted,
312321 sizeOfDataToDelete = sizeOfDataToDelete,
313322 timeTakenToIdentifyEligibleFiles = timeTakenToIdentifyEligibleFiles,
314- timeTakenForDelete = timeTakenForDelete)
323+ timeTakenForDelete = timeTakenForDelete,
324+ vacuumStartTime = vacuumStartTime,
325+ vacuumEndTime = System .currentTimeMillis,
326+ numPartitionColumns = partitionColumns.size)
315327 recordDeltaEvent(deltaLog, " delta.gc.stats" , data = stats)
316328 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" )
317331
318332
319333 spark.createDataset(Seq (basePath)).toDF(" path" )
@@ -576,7 +590,12 @@ case class DeltaVacuumStats(
576590 defaultRetentionMillis : Long ,
577591 minRetainedTimestamp : Long ,
578592 dirsPresentBeforeDelete : Long ,
593+ filesAndDirsPresentBeforeDelete : Long ,
579594 objectsDeleted : Long ,
580595 sizeOfDataToDelete : Long ,
581596 timeTakenToIdentifyEligibleFiles : Long ,
582- timeTakenForDelete : Long )
597+ timeTakenForDelete : Long ,
598+ vacuumStartTime : Long ,
599+ vacuumEndTime : Long ,
600+ numPartitionColumns : Long
601+ )
0 commit comments