You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
Introduce an new boolean flag, optimisticExecution, this would perform an initial transaction log check for file generating operations and the most recent Vacuum operation run before running the rest of the Vacuum operations to evaluate if Vacuum has already been run within the retention period. This would make the Vacuum operation fast when there haven't been any new file generation operations outside the retention period.
Which Delta project/connector is this regarding?
Spark
Standalone
Flink
Kernel
Other (fill in here)
Overview
Run Vacuum multiple times in a row without any changes to a delta table and it will run in the same amount of time (too long), always performing the same operations: listing files, comparing the file list against the transaction log, and then performing any deletions.
Vacuum should introduce a new default enabled feature flag that first uses the transaction log to evaluate if there were any file generation operations run within N days of when the last Vacuum operation took place, if not it would not proceed with the rest of the Vacuum operation.
I implemented a wrapper around the Vacuum operation which does exactly this so that we can run Vacuum much more frequently and have it self evaluate if there's hypothetically anything to vacuum. This makes repeat runs of the Vacuum operation where files have only been added within the retention period take sub 1 second rather than the normal duration of a Vacuum operation.
This could be called optimisticExecution mode since technically there could still be files that fall outside the retention window that would not be caught but the key thing is being optimistic about how often a true vacuum operation should be run and base this off of the retention hours... so if we are retaining 7 days of history, this feature would prevent a full vacuum operation from execution more than once w/in 7 days.
Motivation
Decrease the cost of frequently running Vacuum.
Further details
draft of modifications to gc() method:
defgc(
spark: SparkSession,
deltaLog: DeltaLog,
dryRun: Boolean=true,
optimisticExecution: Boolean=true,
retentionHours: Option[Double] =None,
inventory: Option[DataFrame] =None,
clock: Clock=newSystemClock):DataFrame= {
recordDeltaOperation(deltaLog, "delta.gc") {
valvacuumStartTime=System.currentTimeMillis()
valpath= deltaLog.dataPath
valdeltaHadoopConf= deltaLog.newDeltaHadoopConf()
valfs= path.getFileSystem(deltaHadoopConf)
importorg.apache.spark.sql.delta.implicits._valsnapshot= deltaLog.update()
deltaLog.protocolWrite(snapshot.protocol)
valsnapshotTombstoneRetentionMillis=DeltaLog.tombstoneRetentionMillis(snapshot.metadata)
valretentionMillis= retentionHours.map(h =>TimeUnit.HOURS.toMillis(math.round(h)))
valdeleteBeforeTimestamp= retentionMillis match {
caseSome(millis) => clock.getTimeMillis() - millis
case _ => snapshot.minFileRetentionTimestamp
}
valbasePath= fs.makeQualified(path).toString
if (optimisticExecution) {
valvacuumNeeded= {
valhistory= snapshot.history.getHistory(None)
valvacuumDateThreshold=newDate(deleteBeforeTimestamp)
valfileGeneratingOperations=Seq(
"WRITE", "STREAMING UPDATE", "MERGE", "OVERWRITE", "UPDATE", "CREATE OR REPLACE TABLE"
)
vallatestVacuum= history.filter(_.operation =="VACUUM END")
.sortBy(_.timestamp)
.reverse
.headOption
vallatestVacuumTimestamp=if (latestVacuum.nonEmpty) {
latestVacuum.get.timestamp
} else {
valoldestFileGenOperation= history.filter(op => fileGeneratingOperations.contains(op.operation))
.sortBy(_.timestamp)
.headOption
if (oldestFileGenOperation.nonEmpty) {
oldestFileGenOperation.get.timestamp
} else {
newDate(System.currentTimeMillis())
}
}
if (latestVacuumTimestamp.before(vacuumDateThreshold)) {
valchangesAfterVacuum= history.filter(op =>
op.timestamp.after(latestVacuumTimestamp) && fileGeneratingOperations.contains(op.operation)
)
changesAfterVacuum.nonEmpty
} else {
false
}
}
if (!vacuumNeeded) {
logInfo(s"Garbage collection has already been run withing the retention period, skipping garbage collection."return spark.createDataset(Seq(basePath)).toDF("path")
}
}
logInfo(s"Starting garbage collection (dryRun = $dryRun) of untracked files older than "+s"${newDate(deleteBeforeTimestamp).toGMTString} in $path")
valhadoopConf= spark.sparkContext.broadcast(
newSerializableConfiguration(deltaHadoopConf))
valparallelDeleteEnabled=
spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_VACUUM_PARALLEL_DELETE_ENABLED)
valparallelDeletePartitions=
spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_VACUUM_PARALLEL_DELETE_PARALLELISM)
.getOrElse(spark.sessionState.conf.numShufflePartitions)
valstartTimeToIdentifyEligibleFiles=System.currentTimeMillis()
valvalidFiles=
getValidFilesFromSnapshot(
spark,
basePath,
snapshot,
retentionMillis,
hadoopConf,
clock,
checkAbsolutePathOnly =false)
valpartitionColumns= snapshot.metadata.partitionSchema.fieldNames
valparallelism= spark.sessionState.conf.parallelPartitionDiscoveryParallelism
valallFilesAndDirsWithDuplicates= inventory match {
caseSome(inventoryDF) => getFilesFromInventory(basePath, partitionColumns, inventoryDF)
caseNone=>DeltaFileOperations.recursiveListDirs(
spark,
Seq(basePath),
hadoopConf,
hiddenDirNameFilter =DeltaTableUtils.isHiddenDirectory(partitionColumns, _),
hiddenFileNameFilter =DeltaTableUtils.isHiddenDirectory(partitionColumns, _),
fileListingParallelism =Option(parallelism)
)
}
valallFilesAndDirs= allFilesAndDirsWithDuplicates.groupByKey(_.path)
.mapGroups { (k, v) =>valduplicates= v.toSeq
// of all the duplicates we can return the newest file.
duplicates.maxBy(_.modificationTime)
}
recordFrameProfile("Delta", "VacuumCommand.gc") {
try {
allFilesAndDirs.cache()
implicitvalfileNameAndSizeEncoder=
org.apache.spark.sql.Encoders.product[FileNameAndSize]
valdirCounts= allFilesAndDirs.where(col("isDir")).count() +1// +1 for the base pathvalfilesAndDirsPresentBeforeDelete= allFilesAndDirs.count()
// The logic below is as follows:// 1. We take all the files and directories listed in our reservoir// 2. We filter all files older than our tombstone retention period and directories// 3. We get the subdirectories of all files so that we can find non-empty directories// 4. We groupBy each path, and count to get how many files are in each sub-directory// 5. We subtract all the valid files and tombstones in our state// 6. We filter all paths with a count of 1, which will correspond to files not in the// state, and empty directories. We can safely delete all of thesevaldiff= allFilesAndDirs
.where(col("modificationTime") < deleteBeforeTimestamp || col("isDir"))
.mapPartitions { fileStatusIterator =>valreservoirBase=newPath(basePath)
valfs= reservoirBase.getFileSystem(hadoopConf.value.value)
fileStatusIterator.flatMap { fileStatus =>if (fileStatus.isDir) {
Iterator.single(FileNameAndSize(
relativize(fileStatus.getHadoopPath, fs, reservoirBase, isDir =true), 0L))
} else {
valdirs= getAllSubdirs(basePath, fileStatus.path, fs)
valdirsWithSlash= dirs.map { p =>valrelativizedPath= relativize(newPath(p), fs, reservoirBase, isDir =true)
FileNameAndSize(relativizedPath, 0L)
}
dirsWithSlash ++Iterator(
FileNameAndSize(relativize(
fileStatus.getHadoopPath, fs, reservoirBase, isDir =false),
fileStatus.length))
}
}
}.groupBy(col("path")).agg(count(newColumn("*")).as("count"),
sum("length").as("length"))
.join(validFiles, Seq("path"), "leftanti")
.where(col("count") ===1)
valsizeOfDataToDeleteRow= diff.agg(sum("length").cast("long")).first()
valsizeOfDataToDelete=if (sizeOfDataToDeleteRow.isNullAt(0)) {
0L
} else {
sizeOfDataToDeleteRow.getLong(0)
}
valdiffFiles= diff
.select(col("path"))
.as[String]
.map { relativePath =>
assert(!stringToPath(relativePath).isAbsolute,
"Shouldn't have any absolute paths for deletion here.")
pathToString(DeltaFileOperations.absolutePath(basePath, relativePath))
}
valtimeTakenToIdentifyEligibleFiles=System.currentTimeMillis() - startTimeToIdentifyEligibleFiles
valnumFiles= diffFiles.count()
if (dryRun) {
valstats=DeltaVacuumStats(
isDryRun =true,
specifiedRetentionMillis = retentionMillis,
defaultRetentionMillis = snapshotTombstoneRetentionMillis,
minRetainedTimestamp = deleteBeforeTimestamp,
dirsPresentBeforeDelete = dirCounts,
filesAndDirsPresentBeforeDelete = filesAndDirsPresentBeforeDelete,
objectsDeleted = numFiles,
sizeOfDataToDelete = sizeOfDataToDelete,
timeTakenToIdentifyEligibleFiles = timeTakenToIdentifyEligibleFiles,
timeTakenForDelete =0L,
vacuumStartTime = vacuumStartTime,
vacuumEndTime =System.currentTimeMillis,
numPartitionColumns = partitionColumns.size
)
recordDeltaEvent(deltaLog, "delta.gc.stats", data = stats)
logInfo(s"Found $numFiles files ($sizeOfDataToDelete bytes) and directories in "+s"a total of $dirCounts directories that are safe to delete. Vacuum stats: $stats")
return diffFiles.map(f => stringToPath(f).toString).toDF("path")
}
logVacuumStart(
spark,
deltaLog,
path,
diffFiles,
sizeOfDataToDelete,
retentionMillis,
snapshotTombstoneRetentionMillis)
valdeleteStartTime=System.currentTimeMillis()
valfilesDeleted=try {
delete(diffFiles, spark, basePath,
hadoopConf, parallelDeleteEnabled, parallelDeletePartitions)
} catch {
caset: Throwable=>
logVacuumEnd(deltaLog, spark, path)
throw t
}
valtimeTakenForDelete=System.currentTimeMillis() - deleteStartTime
valstats=DeltaVacuumStats(
isDryRun =false,
specifiedRetentionMillis = retentionMillis,
defaultRetentionMillis = snapshotTombstoneRetentionMillis,
minRetainedTimestamp = deleteBeforeTimestamp,
dirsPresentBeforeDelete = dirCounts,
filesAndDirsPresentBeforeDelete = filesAndDirsPresentBeforeDelete,
objectsDeleted = filesDeleted,
sizeOfDataToDelete = sizeOfDataToDelete,
timeTakenToIdentifyEligibleFiles = timeTakenToIdentifyEligibleFiles,
timeTakenForDelete = timeTakenForDelete,
vacuumStartTime = vacuumStartTime,
vacuumEndTime =System.currentTimeMillis,
numPartitionColumns = partitionColumns.size)
recordDeltaEvent(deltaLog, "delta.gc.stats", data = stats)
logVacuumEnd(deltaLog, spark, path, Some(filesDeleted), Some(dirCounts))
logInfo(s"Deleted $filesDeleted files ($sizeOfDataToDelete bytes) and directories in "+s"a total of $dirCounts directories. Vacuum stats: $stats")
spark.createDataset(Seq(basePath)).toDF("path")
} finally {
allFilesAndDirs.unpersist()
}
}
}
}
Willingness to contribute
The Delta Lake Community encourages new feature contributions. Would you or another member of your organization be willing to contribute an implementation of this feature?
Yes. I can contribute this feature independently.
Yes. I would be willing to contribute this feature with guidance from the Delta Lake community.
No. I cannot contribute this feature at this time.
LMK if this feature makes sense and I'll submit a PR.
The text was updated successfully, but these errors were encountered:
mwc360
changed the title
[Feature Request] Make VACUUM more efficient w/ transaction log check
[Feature Request] Make VACUUM more efficient via optimistic execution mode
May 16, 2024
Feature request
Introduce an new boolean flag, optimisticExecution, this would perform an initial transaction log check for file generating operations and the most recent Vacuum operation run before running the rest of the Vacuum operations to evaluate if Vacuum has already been run within the retention period. This would make the Vacuum operation fast when there haven't been any new file generation operations outside the retention period.
Which Delta project/connector is this regarding?
Overview
Run Vacuum multiple times in a row without any changes to a delta table and it will run in the same amount of time (too long), always performing the same operations: listing files, comparing the file list against the transaction log, and then performing any deletions.
Vacuum should introduce a new default enabled feature flag that first uses the transaction log to evaluate if there were any file generation operations run within N days of when the last Vacuum operation took place, if not it would not proceed with the rest of the Vacuum operation.
I implemented a wrapper around the Vacuum operation which does exactly this so that we can run Vacuum much more frequently and have it self evaluate if there's hypothetically anything to vacuum. This makes repeat runs of the Vacuum operation where files have only been added within the retention period take sub 1 second rather than the normal duration of a Vacuum operation.
This could be called optimisticExecution mode since technically there could still be files that fall outside the retention window that would not be caught but the key thing is being optimistic about how often a true vacuum operation should be run and base this off of the retention hours... so if we are retaining 7 days of history, this feature would prevent a full vacuum operation from execution more than once w/in 7 days.
Motivation
Decrease the cost of frequently running Vacuum.
Further details
draft of modifications to gc() method:
Willingness to contribute
The Delta Lake Community encourages new feature contributions. Would you or another member of your organization be willing to contribute an implementation of this feature?
LMK if this feature makes sense and I'll submit a PR.
The text was updated successfully, but these errors were encountered: